Skip to content

Commit 50ba868

Browse files
klorandkanakb
authored andcommitted
add desired replication factor option (#4)
1 parent 3103ad4 commit 50ba868

File tree

4 files changed

+19
-12
lines changed

4 files changed

+19
-12
lines changed

pom.xml

Lines changed: 2 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -5,7 +5,7 @@
55
<groupId>siftscience</groupId>
66
<artifactId>kafka-assigner</artifactId>
77
<packaging>jar</packaging>
8-
<version>1.0</version>
8+
<version>1.1</version>
99

1010
<name>kafka-assigner</name>
1111
<description>Tools for reassigning Kafka partitions with minimal movement</description>
@@ -130,4 +130,4 @@
130130
</plugins>
131131
</build>
132132

133-
</project>
133+
</project>

src/main/java/siftscience/kafka/tools/KafkaAssignmentGenerator.java

Lines changed: 7 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -75,6 +75,10 @@ public class KafkaAssignmentGenerator {
7575
usage = "comma-separated list of topics")
7676
private String topics = null;
7777

78+
@Option(name = "--desired_replication_factor",
79+
usage = "used for changing replication factor for topics, if not present it will use the existing number")
80+
private int desiredReplicationFactor = -1;
81+
7882
@Option(name = "--disable_rack_awareness",
7983
usage = "set to true to ignore rack configurations")
8084
private boolean disableRackAwareness = false;
@@ -126,7 +130,7 @@ private static void printCurrentBrokers(ZkUtils zkUtils) throws JSONException {
126130

127131
private static void printLeastDisruptiveReassignment(
128132
ZkUtils zkUtils, List<String> specifiedTopics, Set<Integer> specifiedBrokers,
129-
Set<Integer> excludedBrokers, Map<Integer, String> rackAssignment)
133+
Set<Integer> excludedBrokers, Map<Integer, String> rackAssignment, int desiredReplicationFactor)
130134
throws JSONException {
131135
// We need three inputs for rebalacing: the brokers, the topics, and the current assignment
132136
// of topics to brokers.
@@ -169,7 +173,7 @@ public Integer apply(Broker broker) {
169173
for (String topic : JavaConversions.seqAsJavaList(topics)) {
170174
Map<Integer, List<Integer>> partitionAssignment = initialAssignments.get(topic);
171175
Map<Integer, List<Integer>> finalAssignment = assigner.generateAssignment(
172-
topic, partitionAssignment, brokers, rackAssignment);
176+
topic, partitionAssignment, brokers, rackAssignment, desiredReplicationFactor);
173177
for (Map.Entry<Integer, List<Integer>> e : finalAssignment.entrySet()) {
174178
JSONObject partitionJson = new JSONObject();
175179
partitionJson.put("topic", topic);
@@ -284,7 +288,7 @@ private void runTool(String[] args) throws JSONException {
284288
break;
285289
case PRINT_REASSIGNMENT:
286290
printLeastDisruptiveReassignment(zkUtils, topics, brokerIdSet,
287-
excludedBrokerIdSet, rackAssignment);
291+
excludedBrokerIdSet, rackAssignment, desiredReplicationFactor);
288292
break;
289293
default:
290294
throw new UnsupportedOperationException("Invalid mode: " + mode);

src/main/java/siftscience/kafka/tools/KafkaTopicAssigner.java

Lines changed: 6 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -35,23 +35,26 @@ public KafkaTopicAssigner() {
3535
* in each list is the "leader" replica for a partition.
3636
* @param brokers a list of broker IDs as strings
3737
* @param rackAssignment a map from broker ID to rack ID if a rack is defined for that broker
38+
* @param desiredReplicationFactor used to change replication factor, use -1 to keep the same as
39+
* the original topic
3840
* @return the new assignment: a map from partition ID to ordered list of broker IDs
3941
*/
4042
public Map<Integer, List<Integer>> generateAssignment(
4143
String topic, Map<Integer, List<Integer>> currentAssignment, Set<Integer> brokers,
42-
Map<Integer, String> rackAssignment) {
44+
Map<Integer, String> rackAssignment, int desiredReplicationFactor) {
4345
// We need to do 2 things:
4446
// - Get the set of partitions as integers
4547
// - Figure out the replication factor (which should be the same for each partition)
46-
int replicationFactor = -1;
48+
// if desiredReplicationFactor is negative
49+
int replicationFactor = desiredReplicationFactor;
4750
Set<Integer> partitions = Sets.newTreeSet();
4851
for (Map.Entry<Integer, List<Integer>> entry : currentAssignment.entrySet()) {
4952
int partition = entry.getKey();
5053
List<Integer> replicas = entry.getValue();
5154
partitions.add(partition);
5255
if (replicationFactor < 0) {
5356
replicationFactor = replicas.size();
54-
} else {
57+
} else if (desiredReplicationFactor < 0) {
5558
Preconditions.checkState(replicationFactor == replicas.size(),
5659
"Topic " + topic + " has partition " + partition +
5760
" with unexpected replication factor " + replicas.size());

src/test/java/siftscience/kafka/tools/KafkaTopicAssignerTest.java

Lines changed: 4 additions & 4 deletions
Original file line numberDiff line numberDiff line change
@@ -35,7 +35,7 @@ public void testRackAwareExpansion() {
3535
);
3636
KafkaTopicAssigner assigner = new KafkaTopicAssigner();
3737
Map<Integer, List<Integer>> newAssignment = assigner.generateAssignment(
38-
topic, currentAssignment, newBrokers, rackAssignments);
38+
topic, currentAssignment, newBrokers, rackAssignments, -1);
3939

4040
Map<Integer, Integer> brokerReplicaCounts = verifyPartitionsAndBuildReplicaCounts(
4141
currentAssignment, newAssignment, 1);
@@ -68,7 +68,7 @@ public void testClusterExpansion() {
6868
Set<Integer> newBrokers = ImmutableSet.of(10, 11, 12, 13);
6969
KafkaTopicAssigner assigner = new KafkaTopicAssigner();
7070
Map<Integer, List<Integer>> newAssignment = assigner.generateAssignment(
71-
topic, currentAssignment, newBrokers, Collections.<Integer, String>emptyMap());
71+
topic, currentAssignment, newBrokers, Collections.<Integer, String>emptyMap(), -1);
7272

7373
Map<Integer, Integer> brokerReplicaCounts = verifyPartitionsAndBuildReplicaCounts(
7474
currentAssignment, newAssignment, 1);
@@ -93,7 +93,7 @@ public void testDecommission() {
9393
Set<Integer> newBrokers = ImmutableSet.of(10, 11, 13);
9494
KafkaTopicAssigner assigner = new KafkaTopicAssigner();
9595
Map<Integer, List<Integer>> newAssignment = assigner.generateAssignment(
96-
topic, currentAssignment, newBrokers, Collections.<Integer, String>emptyMap());
96+
topic, currentAssignment, newBrokers, Collections.<Integer, String>emptyMap(), -1);
9797

9898
Map<Integer, Integer> brokerReplicaCounts = verifyPartitionsAndBuildReplicaCounts(
9999
currentAssignment, newAssignment, 1);
@@ -133,7 +133,7 @@ public void testReplacement() {
133133
Set<Integer> newBrokers = ImmutableSet.of(10, 11, 13);
134134
KafkaTopicAssigner assigner = new KafkaTopicAssigner();
135135
Map<Integer, List<Integer>> newAssignment = assigner.generateAssignment(
136-
topic, currentAssignment, newBrokers, Collections.<Integer, String>emptyMap());
136+
topic, currentAssignment, newBrokers, Collections.<Integer, String>emptyMap(),-1);
137137

138138
// run basic sanity checks
139139
Map<Integer, Integer> brokerReplicaCounts = verifyPartitionsAndBuildReplicaCounts(

0 commit comments

Comments
 (0)