You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by ij...@apache.org on 2016/06/05 07:57:57 UTC

kafka git commit: KAFKA-3787; Preserve the message timestamp in mirror maker

Repository: kafka
Updated Branches:
  refs/heads/trunk 27cb6686f -> f4a263b5a


KAFKA-3787; Preserve the message timestamp in mirror maker

The timestamp of messages consumed by mirror maker is not preserved after sending to target cluster. The correct behavior is to keep create timestamp the same in both source and target clusters.

Author: Tao Xiao <xi...@gmail.com>

Reviewers: Jun Rao <ju...@gmail.com>, Ismael Juma <is...@juma.me.uk>

Closes #1466 from xiaotao183/KAFKA-3787


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

Branch: refs/heads/trunk
Commit: f4a263b5a89e028ad3b658dca2259b6da0197214
Parents: 27cb668
Author: Tao Xiao <xi...@gmail.com>
Authored: Sun Jun 5 08:42:42 2016 +0100
Committer: Ismael Juma <is...@juma.me.uk>
Committed: Sun Jun 5 08:42:42 2016 +0100

----------------------------------------------------------------------
 .../main/scala/kafka/tools/MirrorMaker.scala    |  4 +-
 .../unit/kafka/tools/MirrorMakerTest.scala      | 42 ++++++++++++++++++++
 2 files changed, 44 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/f4a263b5/core/src/main/scala/kafka/tools/MirrorMaker.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala
index 9d5f7e6..645882d 100755
--- a/core/src/main/scala/kafka/tools/MirrorMaker.scala
+++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala
@@ -673,9 +673,9 @@ object MirrorMaker extends Logging with KafkaMetricsGroup {
     def handle(record: BaseConsumerRecord): util.List[ProducerRecord[Array[Byte], Array[Byte]]]
   }
 
-  private object defaultMirrorMakerMessageHandler extends MirrorMakerMessageHandler {
+  private[tools] object defaultMirrorMakerMessageHandler extends MirrorMakerMessageHandler {
     override def handle(record: BaseConsumerRecord): util.List[ProducerRecord[Array[Byte], Array[Byte]]] = {
-      Collections.singletonList(new ProducerRecord[Array[Byte], Array[Byte]](record.topic, record.key, record.value))
+      Collections.singletonList(new ProducerRecord[Array[Byte], Array[Byte]](record.topic, null, record.timestamp, record.key, record.value))
     }
   }
 

http://git-wip-us.apache.org/repos/asf/kafka/blob/f4a263b5/core/src/test/scala/unit/kafka/tools/MirrorMakerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/tools/MirrorMakerTest.scala b/core/src/test/scala/unit/kafka/tools/MirrorMakerTest.scala
new file mode 100644
index 0000000..39a0ac9
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/tools/MirrorMakerTest.scala
@@ -0,0 +1,42 @@
+/*
+ * 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
+ *
+ *      http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * 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 kafka.tools
+
+import kafka.consumer.BaseConsumerRecord
+import org.apache.kafka.common.record.TimestampType
+import org.junit.Assert._
+import org.junit.Test
+
+class MirrorMakerTest {
+
+  @Test
+  def testDefaultMirrorMakerMessageHandler() {
+    val now = 12345L
+    val consumerRecord = BaseConsumerRecord("topic", 0, 1L, now, TimestampType.CREATE_TIME, "key".getBytes, "value".getBytes)
+
+    val result = MirrorMaker.defaultMirrorMakerMessageHandler.handle(consumerRecord)
+    assertEquals(1, result.size)
+
+    val producerRecord = result.get(0)
+    assertEquals(now, producerRecord.timestamp)
+    assertEquals("topic", producerRecord.topic)
+    assertNull(producerRecord.partition)
+    assertEquals("key", new String(producerRecord.key))
+    assertEquals("value", new String(producerRecord.value))
+  }
+}