You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by ew...@apache.org on 2016/02/09 01:35:10 UTC

kafka git commit: HOTFIX: RecordCollector should send a record to the specified partition

Repository: kafka
Updated Branches:
  refs/heads/trunk d2fc6f36c -> f7ad3d1b1


HOTFIX: RecordCollector should send a record to the specified partition

guozhangwang

Author: Yasuhiro Matsuda <ya...@confluent.io>

Reviewers: Ewen Cheslack-Postava <ew...@confluent.io>

Closes #887 from ymatsuda/hotfix4


Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/f7ad3d1b
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/f7ad3d1b
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/f7ad3d1b

Branch: refs/heads/trunk
Commit: f7ad3d1b1fe24f3d33caf5f168d1292d4ae0b979
Parents: d2fc6f3
Author: Yasuhiro Matsuda <ya...@confluent.io>
Authored: Mon Feb 8 16:35:05 2016 -0800
Committer: Ewen Cheslack-Postava <me...@ewencp.org>
Committed: Mon Feb 8 16:35:05 2016 -0800

----------------------------------------------------------------------
 .../processor/internals/RecordCollector.java    |   4 +-
 .../internals/RecordCollectorTest.java          | 121 +++++++++++++++++++
 2 files changed, 123 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/f7ad3d1b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java
----------------------------------------------------------------------
diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java
index 2943ead..cd8af9b 100644
--- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java
+++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java
@@ -75,8 +75,8 @@ public class RecordCollector {
                             StreamPartitioner<K, V> partitioner) {
         byte[] keyBytes = keySerializer.serialize(record.topic(), record.key());
         byte[] valBytes = valueSerializer.serialize(record.topic(), record.value());
-        Integer partition = null;
-        if (partitioner != null) {
+        Integer partition = record.partition();
+        if (partition == null && partitioner != null) {
             List<PartitionInfo> partitions = this.producer.partitionsFor(record.topic());
             if (partitions != null)
                 partition = partitioner.partition(record.key(), record.value(), partitions.size());

http://git-wip-us.apache.org/repos/asf/kafka/blob/f7ad3d1b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java
----------------------------------------------------------------------
diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java
new file mode 100644
index 0000000..32f2598
--- /dev/null
+++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/RecordCollectorTest.java
@@ -0,0 +1,121 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ * <p>
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * <p>
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.kafka.streams.processor.internals;
+
+import org.apache.kafka.clients.producer.MockProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.internals.DefaultPartitioner;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.serialization.ByteArraySerializer;
+import org.apache.kafka.common.serialization.StringSerializer;
+import org.apache.kafka.streams.processor.StreamPartitioner;
+import org.junit.Test;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+
+import static org.junit.Assert.assertEquals;
+
+/**
+ * Created by yasuhiro on 2/8/16.
+ */
+public class RecordCollectorTest {
+
+    private List<PartitionInfo> infos = Arrays.asList(
+            new PartitionInfo("topic1", 0, Node.noNode(), new Node[0], new Node[0]),
+            new PartitionInfo("topic1", 1, Node.noNode(), new Node[0], new Node[0]),
+            new PartitionInfo("topic1", 2, Node.noNode(), new Node[0], new Node[0])
+    );
+
+    private Cluster cluster = new Cluster(Collections.singletonList(Node.noNode()), infos, Collections.<String>emptySet());
+
+
+    private final ByteArraySerializer byteArraySerializer = new ByteArraySerializer();
+    private final StringSerializer stringSerializer = new StringSerializer();
+
+    private final StreamPartitioner<String, String> streamPartitioner = new StreamPartitioner<String, String>() {
+        @Override
+        public Integer partition(String key, String value, int numPartitions) {
+            return Integer.parseInt(key) % numPartitions;
+        }
+    };
+
+    @Test
+    public void testSpecificPartition() {
+
+        RecordCollector collector = new RecordCollector(
+                new MockProducer<>(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer)
+        );
+
+        collector.send(new ProducerRecord<>("topic1", 0, "999", "0"), stringSerializer, stringSerializer);
+        collector.send(new ProducerRecord<>("topic1", 0, "999", "0"), stringSerializer, stringSerializer);
+        collector.send(new ProducerRecord<>("topic1", 0, "999", "0"), stringSerializer, stringSerializer);
+
+        collector.send(new ProducerRecord<>("topic1", 1, "999", "0"), stringSerializer, stringSerializer);
+        collector.send(new ProducerRecord<>("topic1", 1, "999", "0"), stringSerializer, stringSerializer);
+
+        collector.send(new ProducerRecord<>("topic1", 2, "999", "0"), stringSerializer, stringSerializer);
+
+        Map<TopicPartition, Long> offsets = collector.offsets();
+
+        assertEquals((Long) 2L, offsets.get(new TopicPartition("topic1", 0)));
+        assertEquals((Long) 1L, offsets.get(new TopicPartition("topic1", 1)));
+        assertEquals((Long) 0L, offsets.get(new TopicPartition("topic1", 2)));
+
+        // ignore StreamPartitioner
+        collector.send(new ProducerRecord<>("topic1", 0, "999", "0"), stringSerializer, stringSerializer, streamPartitioner);
+        collector.send(new ProducerRecord<>("topic1", 1, "999", "0"), stringSerializer, stringSerializer, streamPartitioner);
+        collector.send(new ProducerRecord<>("topic1", 2, "999", "0"), stringSerializer, stringSerializer, streamPartitioner);
+
+        assertEquals((Long) 3L, offsets.get(new TopicPartition("topic1", 0)));
+        assertEquals((Long) 2L, offsets.get(new TopicPartition("topic1", 1)));
+        assertEquals((Long) 1L, offsets.get(new TopicPartition("topic1", 2)));
+    }
+
+    @Test
+    public void testStreamPartitioner() {
+
+        RecordCollector collector = new RecordCollector(
+                new MockProducer<>(cluster, true, new DefaultPartitioner(), byteArraySerializer, byteArraySerializer)
+        );
+
+        collector.send(new ProducerRecord<>("topic1", "3", "0"), stringSerializer, stringSerializer, streamPartitioner);
+        collector.send(new ProducerRecord<>("topic1", "9", "0"), stringSerializer, stringSerializer, streamPartitioner);
+        collector.send(new ProducerRecord<>("topic1", "27", "0"), stringSerializer, stringSerializer, streamPartitioner);
+        collector.send(new ProducerRecord<>("topic1", "81", "0"), stringSerializer, stringSerializer, streamPartitioner);
+        collector.send(new ProducerRecord<>("topic1", "243", "0"), stringSerializer, stringSerializer, streamPartitioner);
+
+        collector.send(new ProducerRecord<>("topic1", "28", "0"), stringSerializer, stringSerializer, streamPartitioner);
+        collector.send(new ProducerRecord<>("topic1", "82", "0"), stringSerializer, stringSerializer, streamPartitioner);
+        collector.send(new ProducerRecord<>("topic1", "244", "0"), stringSerializer, stringSerializer, streamPartitioner);
+
+        collector.send(new ProducerRecord<>("topic1", "245", "0"), stringSerializer, stringSerializer, streamPartitioner);
+
+        Map<TopicPartition, Long> offsets = collector.offsets();
+
+        assertEquals((Long) 4L, offsets.get(new TopicPartition("topic1", 0)));
+        assertEquals((Long) 2L, offsets.get(new TopicPartition("topic1", 1)));
+        assertEquals((Long) 0L, offsets.get(new TopicPartition("topic1", 2)));
+    }
+
+}