Versions Compared

Key

  • This line was added.
  • This line was removed.
  • Formatting was changed.
Comment: syntax highlight

...

Anchor
alterPartitionCount
alterPartitionCount
To support kafka-topics.sh --alter --partitions ... the following methods will be added to AdminClient to support changing topics' partition counts

Code Block
languagejava
linenumberstrue
/**
 * Change the partition count of the topics given as the keys of {@code counts}
 * to the corresponding values. Currently it is only possible to increase 
 * the partition count. 
 */
public AlterPartitionCountsResult alterPartitionCounts(Map<String, Integer> counts,
                    AlterPartitionCountsOptions options)
public AlterPartitionCountsResult alterPartitionCounts(Map<String, Integer> counts) 

Where:

Code Block
languagejava
linenumberstrue
public class AlterPartitionCountsOptions {
    public AlterPartitionCountsOptions() { ... }
    public Integer timeoutMs() { ... }
    public AlterPartitionCountsOptions timeoutMs(Integer timeoutMs) { ... }
}
 
public class AlterPartitionCountsResult {
    // package access constructor
    Map<String, KafkaFuture<Void>> values() { ... }
    KafkaFuture<Void> all() { ... }
}

...

Anchor
alterReplicationCount
alterReplicationCount
To support kafka-topics.sh --alter --replication-factor ... the following methods will be added to AdminClient to support changing topics' replication factors.

Code Block
languagejava
linenumberstrue
/**
 * Change the replication factor of the topics given as the keys of 
 * replicationFactors to the corresponding values.
 */
AlterReplicationFactorsResult alterReplicationFactors(Map<String, Short> replicationFactors)
AlterReplicationFactorsResult alterReplicationFactors(Map<String, Short> replicationFactors, 
                        AlterReplicationFactorsOptions options)

Where:

Code Block
languagejava
linenumberstrue
public class AlterReplicationFactorsOptions {
    public AlterReplicationFactorsOptions() { ... }
    // TODO validateOnly?
    public Integer timeoutMs() { ... }
    public AlterReplicationFactorsOptions timeoutMs(Integer timeoutMs) { ... }
}
public class AlterReplicationFactorsResult {
    Map<String, KafkaFuture<Void>> values() { ... }
    KafkaFuture<Void> all() { ... }
}

...

Anchor
reassignPartitions
reassignPartitions
To support kafka-topics.sh --alter --replica-assignment ... and kafka-reassign-partitions.sh the following methods will be added to AdminClient to support changing the brokers hosting the partitions of a topic

Code Block
languagejava
linenumberstrue
/**
 * Reassign the partitions given as the key of the given <code>assignments</code> to the corresponding 
 * list of brokers.
 */
ReassignPartitionsResult reassignPartitions(Map<TopicPartition, List<Integer>> assignments)
ReassignPartitionsResult reassignPartitions(Map<TopicPartition, List<Integer>> assignments, 
                        ReassignPartitionsOptions options)

Where:

Code Block
languagejava
linenumberstrue
class ReassignPartitionsOptions {
    boolean validateOnly()
    ReassignPartitionsOptions validateOnly(boolean validateOnly)
    long timeoutMs()
    ReassignPartitionsOptions timeoutMs(long timeoutMs)
}
class ReassignPartitionsResult {
    Map<String, KafkaFuture<Void>> values();
}

...