You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by jg...@apache.org on 2016/12/13 18:41:28 UTC
[1/9] kafka git commit: KAFKA-4390;
Replace MessageSet usage with client-side alternatives
Repository: kafka
Updated Branches:
refs/heads/trunk 6626b058c -> 67f1e5b91
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala
index 3d1b485..4467394 100644
--- a/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/FetchRequestTest.scala
@@ -24,12 +24,12 @@ import kafka.utils.TestUtils
import kafka.utils.TestUtils._
import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord}
import org.apache.kafka.common.TopicPartition
-import org.apache.kafka.common.protocol.{ApiKeys, Errors, ProtoUtils}
-import org.apache.kafka.common.record.{LogEntry, MemoryRecords}
+import org.apache.kafka.common.protocol.{ApiKeys, Errors}
+import org.apache.kafka.common.record.LogEntry
import org.apache.kafka.common.requests.{FetchRequest, FetchResponse}
import org.apache.kafka.common.serialization.StringSerializer
-import org.junit.Test
import org.junit.Assert._
+import org.junit.Test
import scala.collection.JavaConverters._
import scala.util.Random
@@ -120,13 +120,13 @@ class FetchRequestTest extends BaseRequestTest {
val fetchResponse3 = sendFetchRequest(leaderId, fetchRequest3)
assertEquals(shuffledTopicPartitions3, fetchResponse3.responseData.keySet.asScala.toSeq)
val responseSize3 = fetchResponse3.responseData.asScala.values.map { partitionData =>
- logEntries(partitionData).map(_.size).sum
+ logEntries(partitionData).map(_.sizeInBytes).sum
}.sum
assertTrue(responseSize3 <= maxResponseBytes)
val partitionData3 = fetchResponse3.responseData.get(partitionWithLargeMessage1)
assertEquals(Errors.NONE.code, partitionData3.errorCode)
assertTrue(partitionData3.highWatermark > 0)
- val size3 = logEntries(partitionData3).map(_.size).sum
+ val size3 = logEntries(partitionData3).map(_.sizeInBytes).sum
assertTrue(s"Expected $size3 to be smaller than $maxResponseBytes", size3 <= maxResponseBytes)
assertTrue(s"Expected $size3 to be larger than $maxPartitionBytes", size3 > maxPartitionBytes)
assertTrue(maxPartitionBytes < partitionData3.records.sizeInBytes)
@@ -138,13 +138,13 @@ class FetchRequestTest extends BaseRequestTest {
val fetchResponse4 = sendFetchRequest(leaderId, fetchRequest4)
assertEquals(shuffledTopicPartitions4, fetchResponse4.responseData.keySet.asScala.toSeq)
val nonEmptyPartitions4 = fetchResponse4.responseData.asScala.toSeq.collect {
- case (tp, partitionData) if logEntries(partitionData).map(_.size).sum > 0 => tp
+ case (tp, partitionData) if logEntries(partitionData).map(_.sizeInBytes).sum > 0 => tp
}
assertEquals(Seq(partitionWithLargeMessage2), nonEmptyPartitions4)
val partitionData4 = fetchResponse4.responseData.get(partitionWithLargeMessage2)
assertEquals(Errors.NONE.code, partitionData4.errorCode)
assertTrue(partitionData4.highWatermark > 0)
- val size4 = logEntries(partitionData4).map(_.size).sum
+ val size4 = logEntries(partitionData4).map(_.sizeInBytes).sum
assertTrue(s"Expected $size4 to be larger than $maxResponseBytes", size4 > maxResponseBytes)
assertTrue(maxResponseBytes < partitionData4.records.sizeInBytes)
}
@@ -161,12 +161,11 @@ class FetchRequestTest extends BaseRequestTest {
assertEquals(Errors.NONE.code, partitionData.errorCode)
assertTrue(partitionData.highWatermark > 0)
assertEquals(maxPartitionBytes, partitionData.records.sizeInBytes)
- assertEquals(0, logEntries(partitionData).map(_.size).sum)
+ assertEquals(0, logEntries(partitionData).map(_.sizeInBytes).sum)
}
private def logEntries(partitionData: FetchResponse.PartitionData): Seq[LogEntry] = {
- val memoryRecords = partitionData.records
- memoryRecords.iterator.asScala.toIndexedSeq
+ partitionData.records.deepIterator.asScala.toIndexedSeq
}
private def checkFetchResponse(expectedPartitions: Seq[TopicPartition], fetchResponse: FetchResponse,
@@ -181,25 +180,25 @@ class FetchRequestTest extends BaseRequestTest {
assertEquals(Errors.NONE.code, partitionData.errorCode)
assertTrue(partitionData.highWatermark > 0)
- val memoryRecords = partitionData.records
- responseBufferSize += memoryRecords.sizeInBytes
+ val records = partitionData.records
+ responseBufferSize += records.sizeInBytes
- val messages = memoryRecords.iterator.asScala.toIndexedSeq
- assertTrue(messages.size < numMessagesPerPartition)
- val messagesSize = messages.map(_.size).sum
- responseSize += messagesSize
- if (messagesSize == 0 && !emptyResponseSeen) {
- assertEquals(0, memoryRecords.sizeInBytes)
+ val entries = records.shallowIterator.asScala.toIndexedSeq
+ assertTrue(entries.size < numMessagesPerPartition)
+ val entriesSize = entries.map(_.sizeInBytes).sum
+ responseSize += entriesSize
+ if (entriesSize == 0 && !emptyResponseSeen) {
+ assertEquals(0, records.sizeInBytes)
emptyResponseSeen = true
}
- else if (messagesSize != 0 && !emptyResponseSeen) {
- assertTrue(messagesSize <= maxPartitionBytes)
- assertEquals(maxPartitionBytes, memoryRecords.sizeInBytes)
+ else if (entriesSize != 0 && !emptyResponseSeen) {
+ assertTrue(entriesSize <= maxPartitionBytes)
+ assertEquals(maxPartitionBytes, records.sizeInBytes)
}
- else if (messagesSize != 0 && emptyResponseSeen)
- fail(s"Expected partition with size 0, but found $tp with size $messagesSize")
- else if (memoryRecords.sizeInBytes != 0 && emptyResponseSeen)
- fail(s"Expected partition buffer with size 0, but found $tp with size ${memoryRecords.sizeInBytes}")
+ else if (entriesSize != 0 && emptyResponseSeen)
+ fail(s"Expected partition with size 0, but found $tp with size $entriesSize")
+ else if (records.sizeInBytes != 0 && emptyResponseSeen)
+ fail(s"Expected partition buffer with size 0, but found $tp with size ${records.sizeInBytes}")
}
@@ -208,7 +207,7 @@ class FetchRequestTest extends BaseRequestTest {
}
private def createTopics(numTopics: Int, numPartitions: Int): Map[TopicPartition, Int] = {
- val topics = (0 until numPartitions).map(t => s"topic${t}")
+ val topics = (0 until numPartitions).map(t => s"topic$t")
val topicConfig = new Properties
topicConfig.setProperty(LogConfig.MinInSyncReplicasProp, 2.toString)
topics.flatMap { topic =>
@@ -223,7 +222,7 @@ class FetchRequestTest extends BaseRequestTest {
tp <- topicPartitions.toSeq
messageIndex <- 0 until numMessagesPerPartition
} yield {
- val suffix = s"${tp}-${messageIndex}"
+ val suffix = s"$tp-$messageIndex"
new ProducerRecord(tp.topic, tp.partition, s"key $suffix", s"value $suffix")
}
records.map(producer.send).foreach(_.get)
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala
index 2d51be9..aad37d1 100644
--- a/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ISRExpirationTest.scala
@@ -17,21 +17,20 @@
package kafka.server
import java.util.Properties
+import java.util.concurrent.atomic.AtomicBoolean
-import org.apache.kafka.common.metrics.Metrics
-import org.junit.{After, Before, Test}
-
-import collection.mutable.HashMap
-import collection.mutable.Map
import kafka.cluster.{Partition, Replica}
-import org.easymock.EasyMock
import kafka.log.Log
-import org.junit.Assert._
import kafka.utils._
-import java.util.concurrent.atomic.AtomicBoolean
-
-import kafka.message.MessageSet
+import org.apache.kafka.common.metrics.Metrics
+import org.apache.kafka.common.record.MemoryRecords
import org.apache.kafka.common.utils.Time
+import org.easymock.EasyMock
+import org.junit.Assert._
+import org.junit.{After, Before, Test}
+
+import scala.collection.mutable.{HashMap, Map}
+
class IsrExpirationTest {
@@ -76,7 +75,7 @@ class IsrExpirationTest {
// let the follower catch up to the Leader logEndOffset (15)
(partition0.assignedReplicas() - leaderReplica).foreach(
- r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(15L), MessageSet.Empty),
+ r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(15L), MemoryRecords.EMPTY),
-1L,
-1,
true)))
@@ -127,7 +126,7 @@ class IsrExpirationTest {
// Make the remote replica not read to the end of log. It should be not be out of sync for at least 100 ms
for(replica <- partition0.assignedReplicas() - leaderReplica)
- replica.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(10L), MessageSet.Empty), -1L, -1, false))
+ replica.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(10L), MemoryRecords.EMPTY), -1L, -1, false))
// Simulate 2 fetch requests spanning more than 100 ms which do not read to the end of the log.
// The replicas will no longer be in ISR. We do 2 fetches because we want to simulate the case where the replica is lagging but is not stuck
@@ -137,7 +136,7 @@ class IsrExpirationTest {
time.sleep(75)
(partition0.assignedReplicas() - leaderReplica).foreach(
- r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(11L), MessageSet.Empty), -1L, -1, false)))
+ r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(11L), MemoryRecords.EMPTY), -1L, -1, false)))
partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs)
assertEquals("No replica should be out of sync", Set.empty[Int], partition0OSR.map(_.brokerId))
@@ -149,7 +148,7 @@ class IsrExpirationTest {
// Now actually make a fetch to the end of the log. The replicas should be back in ISR
(partition0.assignedReplicas() - leaderReplica).foreach(
- r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(15L), MessageSet.Empty), -1L, -1, true)))
+ r => r.updateLogReadResult(new LogReadResult(FetchDataInfo(new LogOffsetMetadata(15L), MemoryRecords.EMPTY), -1L, -1, true)))
partition0OSR = partition0.getOutOfSyncReplicas(leaderReplica, configs.head.replicaLagTimeMaxMs)
assertEquals("No replica should be out of sync", Set.empty[Int], partition0OSR.map(_.brokerId))
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
index 70445d7..b577e7d 100755
--- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
+++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala
@@ -26,12 +26,12 @@ import kafka.api.{FetchRequestBuilder, OffsetRequest, PartitionOffsetRequestInfo
import kafka.common.TopicAndPartition
import kafka.consumer.SimpleConsumer
import kafka.log.{Log, LogSegment}
-import kafka.message.{ByteBufferMessageSet, Message, NoCompressionCodec}
import kafka.utils.TestUtils._
import kafka.utils._
import kafka.zk.ZooKeeperTestHarness
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.record.{MemoryRecords, Record}
import org.apache.kafka.common.utils.{Time, Utils}
import org.easymock.{EasyMock, IAnswer}
import org.junit.Assert._
@@ -89,9 +89,9 @@ class LogOffsetTest extends ZooKeeperTestHarness {
"Log for partition [topic,0] should be created")
val log = logManager.getLog(TopicAndPartition(topic, part)).get
- val message = new Message(Integer.toString(42).getBytes())
+ val record = Record.create(Integer.toString(42).getBytes())
for (_ <- 0 until 20)
- log.append(new ByteBufferMessageSet(NoCompressionCodec, message))
+ log.append(MemoryRecords.withRecords(record))
log.flush()
val offsets = server.apis.fetchOffsets(logManager, new TopicPartition(topic, part), OffsetRequest.LatestTime, 15)
@@ -150,9 +150,9 @@ class LogOffsetTest extends ZooKeeperTestHarness {
val logManager = server.getLogManager
val log = logManager.createLog(TopicAndPartition(topic, part), logManager.defaultConfig)
- val message = new Message(Integer.toString(42).getBytes())
+ val record = Record.create(Integer.toString(42).getBytes())
for (_ <- 0 until 20)
- log.append(new ByteBufferMessageSet(NoCompressionCodec, message))
+ log.append(MemoryRecords.withRecords(record))
log.flush()
val now = time.milliseconds + 30000 // pretend it is the future to avoid race conditions with the fs
@@ -179,9 +179,9 @@ class LogOffsetTest extends ZooKeeperTestHarness {
val logManager = server.getLogManager
val log = logManager.createLog(TopicAndPartition(topic, part), logManager.defaultConfig)
- val message = new Message(Integer.toString(42).getBytes())
+ val record = Record.create(Integer.toString(42).getBytes())
for (_ <- 0 until 20)
- log.append(new ByteBufferMessageSet(NoCompressionCodec, message))
+ log.append(MemoryRecords.withRecords(record))
log.flush()
val offsets = server.apis.fetchOffsets(logManager, new TopicPartition(topic, part), OffsetRequest.EarliestTime, 10)
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala
index bd74dee..51be54c 100644
--- a/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ProduceRequestTest.scala
@@ -54,11 +54,11 @@ class ProduceRequestTest extends BaseRequestTest {
}
sendAndCheck(JTestUtils.partitionRecordsBuffer(0, CompressionType.NONE,
- new Record(System.currentTimeMillis(), "key".getBytes, "value".getBytes)), 0)
+ Record.create(System.currentTimeMillis(), "key".getBytes, "value".getBytes)), 0)
sendAndCheck(JTestUtils.partitionRecordsBuffer(0, CompressionType.GZIP,
- new Record(System.currentTimeMillis(), "key1".getBytes, "value1".getBytes),
- new Record(System.currentTimeMillis(), "key2".getBytes, "value2".getBytes)), 1)
+ Record.create(System.currentTimeMillis(), "key1".getBytes, "value1".getBytes),
+ Record.create(System.currentTimeMillis(), "key2".getBytes, "value2".getBytes)), 1)
}
/* returns a pair of partition id and leader id */
@@ -74,7 +74,7 @@ class ProduceRequestTest extends BaseRequestTest {
val (partition, leader) = createTopicAndFindPartitionWithLeader("topic")
val timestamp = 1000000
val recordBuffer = JTestUtils.partitionRecordsBuffer(0, CompressionType.LZ4,
- new Record(timestamp, "key".getBytes, "value".getBytes))
+ Record.create(timestamp, "key".getBytes, "value".getBytes))
// Change the lz4 checksum value so that it doesn't match the contents
recordBuffer.array.update(40, 0)
val topicPartition = new TopicPartition("topic", partition)
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
index 378d382..a643f63 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerQuotasTest.scala
@@ -16,28 +16,29 @@
*/
package kafka.server
-
import java.util.Properties
import java.util.concurrent.atomic.AtomicBoolean
import kafka.cluster.Replica
import kafka.common.TopicAndPartition
import kafka.log.Log
-import kafka.message.{ByteBufferMessageSet, Message}
import kafka.utils._
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.metrics.Metrics
+import org.apache.kafka.common.record.{MemoryRecords, Record}
import org.apache.kafka.common.requests.FetchRequest.PartitionData
import org.easymock.EasyMock
import EasyMock._
import org.junit.Assert._
import org.junit.{After, Test}
+import scala.collection.JavaConverters._
+
class ReplicaManagerQuotasTest {
val configs = TestUtils.createBrokerConfigs(2, TestUtils.MockZkConnect).map(KafkaConfig.fromProps(_, new Properties()))
val time = new MockTime
val metrics = new Metrics
- val message = new Message("some-data-in-a-message".getBytes())
+ val record = Record.create("some-data-in-a-message".getBytes())
val topicAndPartition1 = TopicAndPartition("test-topic", 1)
val topicAndPartition2 = TopicAndPartition("test-topic", 2)
val fetchInfo = Seq(new TopicPartition(topicAndPartition1.topic, topicAndPartition1.partition) -> new PartitionData(0, 100),
@@ -63,10 +64,10 @@ class ReplicaManagerQuotasTest {
readPartitionInfo = fetchInfo,
quota = quota)
assertEquals("Given two partitions, with only one throttled, we should get the first", 1,
- fetch.find(_._1 == topicAndPartition1).get._2.info.messageSet.size)
+ fetch.find(_._1 == topicAndPartition1).get._2.info.records.shallowIterator.asScala.size)
assertEquals("But we shouldn't get the second", 0,
- fetch.find(_._1 == topicAndPartition2).get._2.info.messageSet.size)
+ fetch.find(_._1 == topicAndPartition2).get._2.info.records.shallowIterator.asScala.size)
}
@Test
@@ -88,9 +89,9 @@ class ReplicaManagerQuotasTest {
readPartitionInfo = fetchInfo,
quota = quota)
assertEquals("Given two partitions, with both throttled, we should get no messages", 0,
- fetch.find(_._1 == topicAndPartition1).get._2.info.messageSet.size)
+ fetch.find(_._1 == topicAndPartition1).get._2.info.records.shallowIterator.asScala.size)
assertEquals("Given two partitions, with both throttled, we should get no messages", 0,
- fetch.find(_._1 == topicAndPartition2).get._2.info.messageSet.size)
+ fetch.find(_._1 == topicAndPartition2).get._2.info.records.shallowIterator.asScala.size)
}
@Test
@@ -112,9 +113,9 @@ class ReplicaManagerQuotasTest {
readPartitionInfo = fetchInfo,
quota = quota)
assertEquals("Given two partitions, with both non-throttled, we should get both messages", 1,
- fetch.find(_._1 == topicAndPartition1).get._2.info.messageSet.size)
+ fetch.find(_._1 == topicAndPartition1).get._2.info.records.shallowIterator.asScala.size)
assertEquals("Given two partitions, with both non-throttled, we should get both messages", 1,
- fetch.find(_._1 == topicAndPartition2).get._2.info.messageSet.size)
+ fetch.find(_._1 == topicAndPartition2).get._2.info.records.shallowIterator.asScala.size)
}
@Test
@@ -136,13 +137,13 @@ class ReplicaManagerQuotasTest {
readPartitionInfo = fetchInfo,
quota = quota)
assertEquals("Given two partitions, with only one throttled, we should get the first", 1,
- fetch.find(_._1 == topicAndPartition1).get._2.info.messageSet.size)
+ fetch.find(_._1 == topicAndPartition1).get._2.info.records.shallowIterator.asScala.size)
assertEquals("But we should get the second too since it's throttled but in sync", 1,
- fetch.find(_._1 == topicAndPartition2).get._2.info.messageSet.size)
+ fetch.find(_._1 == topicAndPartition2).get._2.info.records.shallowIterator.asScala.size)
}
- def setUpMocks(fetchInfo: Seq[(TopicPartition, PartitionData)], message: Message = this.message, bothReplicasInSync: Boolean = false) {
+ def setUpMocks(fetchInfo: Seq[(TopicPartition, PartitionData)], record: Record = this.record, bothReplicasInSync: Boolean = false) {
val zkUtils = createNiceMock(classOf[ZkUtils])
val scheduler = createNiceMock(classOf[KafkaScheduler])
@@ -153,16 +154,16 @@ class ReplicaManagerQuotasTest {
//if we ask for len 1 return a message
expect(log.read(anyObject(), geq(1), anyObject(), anyObject())).andReturn(
- new FetchDataInfo(
+ FetchDataInfo(
new LogOffsetMetadata(0L, 0L, 0),
- new ByteBufferMessageSet(message)
+ MemoryRecords.withRecords(record)
)).anyTimes()
//if we ask for len = 0, return 0 messages
expect(log.read(anyObject(), EasyMock.eq(0), anyObject(), anyObject())).andReturn(
- new FetchDataInfo(
+ FetchDataInfo(
new LogOffsetMetadata(0L, 0L, 0),
- new ByteBufferMessageSet()
+ MemoryRecords.EMPTY
)).anyTimes()
replay(log)
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
index c6d66ba..421de32 100644
--- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala
@@ -17,18 +17,16 @@
package kafka.server
-
import java.io.File
import java.util.concurrent.atomic.AtomicBoolean
-import kafka.api.FetchResponsePartitionData
import kafka.cluster.Broker
import kafka.common.TopicAndPartition
-import kafka.message.{ByteBufferMessageSet, Message, MessageSet}
import kafka.utils.{MockScheduler, MockTime, TestUtils, ZkUtils}
import org.I0Itec.zkclient.ZkClient
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.record.{MemoryRecords, Record, Records}
import org.apache.kafka.common.requests.{LeaderAndIsrRequest, PartitionState}
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.common.requests.FetchRequest.PartitionData
@@ -105,11 +103,11 @@ class ReplicaManagerTest {
def callback(responseStatus: Map[TopicPartition, PartitionResponse]) = {
assert(responseStatus.values.head.errorCode == Errors.INVALID_REQUIRED_ACKS.code)
}
- rm.appendMessages(
+ rm.appendRecords(
timeout = 0,
requiredAcks = 3,
internalTopicsAllowed = false,
- messagesPerPartition = Map(new TopicPartition("test1", 0) -> new ByteBufferMessageSet(new Message("first message".getBytes))),
+ entriesPerPartition = Map(new TopicPartition("test1", 0) -> MemoryRecords.withRecords(Record.create("first message".getBytes()))),
responseCallback = callback)
} finally {
rm.shutdown(checkpointHW = false)
@@ -135,7 +133,7 @@ class ReplicaManagerTest {
}
var fetchCallbackFired = false
- def fetchCallback(responseStatus: Seq[(TopicAndPartition, FetchResponsePartitionData)]) = {
+ def fetchCallback(responseStatus: Seq[(TopicAndPartition, FetchPartitionData)]) = {
assertEquals("Should give NotLeaderForPartitionException", Errors.NOT_LEADER_FOR_PARTITION.code, responseStatus.map(_._2).head.error)
fetchCallbackFired = true
}
@@ -158,11 +156,11 @@ class ReplicaManagerTest {
rm.getLeaderReplicaIfLocal(topic, 0)
// Append a message.
- rm.appendMessages(
+ rm.appendRecords(
timeout = 1000,
requiredAcks = -1,
internalTopicsAllowed = false,
- messagesPerPartition = Map(new TopicPartition(topic, 0) -> new ByteBufferMessageSet(new Message("first message".getBytes))),
+ entriesPerPartition = Map(new TopicPartition(topic, 0) -> MemoryRecords.withRecords(Record.create("first message".getBytes()))),
responseCallback = produceCallback)
// Fetch some messages
@@ -220,19 +218,19 @@ class ReplicaManagerTest {
// Append a couple of messages.
for(i <- 1 to 2)
- rm.appendMessages(
+ rm.appendRecords(
timeout = 1000,
requiredAcks = -1,
internalTopicsAllowed = false,
- messagesPerPartition = Map(new TopicPartition(topic, 0) -> new ByteBufferMessageSet(new Message("message %d".format(i).getBytes))),
+ entriesPerPartition = Map(new TopicPartition(topic, 0) -> MemoryRecords.withRecords(Record.create("message %d".format(i).getBytes))),
responseCallback = produceCallback)
var fetchCallbackFired = false
var fetchError = 0
- var fetchedMessages: MessageSet = null
- def fetchCallback(responseStatus: Seq[(TopicAndPartition, FetchResponsePartitionData)]) = {
+ var fetchedRecords: Records = null
+ def fetchCallback(responseStatus: Seq[(TopicAndPartition, FetchPartitionData)]) = {
fetchError = responseStatus.map(_._2).head.error
- fetchedMessages = responseStatus.map(_._2).head.messages
+ fetchedRecords = responseStatus.map(_._2).head.records
fetchCallbackFired = true
}
@@ -249,7 +247,7 @@ class ReplicaManagerTest {
assertTrue(fetchCallbackFired)
assertEquals("Should not give an exception", Errors.NONE.code, fetchError)
- assertTrue("Should return some data", fetchedMessages.iterator.hasNext)
+ assertTrue("Should return some data", fetchedRecords.shallowIterator.hasNext)
fetchCallbackFired = false
// Fetch a message above the high watermark as a consumer
@@ -264,7 +262,7 @@ class ReplicaManagerTest {
assertTrue(fetchCallbackFired)
assertEquals("Should not give an exception", Errors.NONE.code, fetchError)
- assertEquals("Should return empty response", MessageSet.Empty, fetchedMessages)
+ assertEquals("Should return empty response", MemoryRecords.EMPTY, fetchedRecords)
} finally {
rm.shutdown(checkpointHW = false)
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
index b1ebeee..2f73a94 100644
--- a/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
+++ b/core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala
@@ -21,7 +21,6 @@ import kafka.utils._
import kafka.cluster.Replica
import kafka.common.TopicAndPartition
import kafka.log.Log
-import kafka.message.{ByteBufferMessageSet, Message, MessageSet}
import kafka.server.QuotaFactory.UnboundedQuota
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.requests.FetchRequest.PartitionData
@@ -30,8 +29,10 @@ import java.util.Properties
import java.util.concurrent.atomic.AtomicBoolean
import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.record.{MemoryRecords, Record}
import org.easymock.EasyMock
import org.junit.Assert._
+import scala.collection.JavaConverters._
class SimpleFetchTest {
@@ -53,8 +54,8 @@ class SimpleFetchTest {
val partitionHW = 5
val fetchSize = 100
- val messagesToHW = new Message("messageToHW".getBytes())
- val messagesToLEO = new Message("messageToLEO".getBytes())
+ val messagesToHW = Record.create("messageToHW".getBytes())
+ val messagesToLEO = Record.create("messageToLEO".getBytes())
val topic = "test-topic"
val partitionId = 0
@@ -79,14 +80,14 @@ class SimpleFetchTest {
EasyMock.expect(log.logEndOffset).andReturn(leaderLEO).anyTimes()
EasyMock.expect(log.logEndOffsetMetadata).andReturn(new LogOffsetMetadata(leaderLEO)).anyTimes()
EasyMock.expect(log.read(0, fetchSize, Some(partitionHW), true)).andReturn(
- new FetchDataInfo(
+ FetchDataInfo(
new LogOffsetMetadata(0L, 0L, 0),
- new ByteBufferMessageSet(messagesToHW)
+ MemoryRecords.withRecords(messagesToHW)
)).anyTimes()
EasyMock.expect(log.read(0, fetchSize, None, true)).andReturn(
- new FetchDataInfo(
+ FetchDataInfo(
new LogOffsetMetadata(0L, 0L, 0),
- new ByteBufferMessageSet(messagesToLEO)
+ MemoryRecords.withRecords(messagesToLEO)
)).anyTimes()
EasyMock.replay(log)
@@ -110,7 +111,7 @@ class SimpleFetchTest {
// create the follower replica with defined log end offset
val followerReplica= new Replica(configs(1).brokerId, partition, time)
val leo = new LogOffsetMetadata(followerLEO, 0L, followerLEO.toInt)
- followerReplica.updateLogReadResult(new LogReadResult(FetchDataInfo(leo, MessageSet.Empty), -1L, -1, true))
+ followerReplica.updateLogReadResult(new LogReadResult(FetchDataInfo(leo, MemoryRecords.EMPTY), -1L, -1, true))
// add both of them to ISR
val allReplicas = List(leaderReplica, followerReplica)
@@ -153,7 +154,7 @@ class SimpleFetchTest {
fetchMaxBytes = Int.MaxValue,
hardMaxBytesLimit = false,
readPartitionInfo = fetchInfo,
- quota = UnboundedQuota).find(_._1 == topicAndPartition).get._2.info.messageSet.head.message)
+ quota = UnboundedQuota).find(_._1 == topicAndPartition).get._2.info.records.shallowIterator.next().record)
assertEquals("Reading any data can return messages up to the end of the log", messagesToLEO,
replicaManager.readFromLocalLog(
replicaId = Request.OrdinaryConsumerId,
@@ -162,7 +163,7 @@ class SimpleFetchTest {
fetchMaxBytes = Int.MaxValue,
hardMaxBytesLimit = false,
readPartitionInfo = fetchInfo,
- quota = UnboundedQuota).find(_._1 == topicAndPartition).get._2.info.messageSet.head.message)
+ quota = UnboundedQuota).find(_._1 == topicAndPartition).get._2.info.records.shallowIterator().next().record)
assertEquals("Counts should increment after fetch", initialTopicCount+2, BrokerTopicStats.getBrokerTopicStats(topic).totalFetchRequestRate.count())
assertEquals("Counts should increment after fetch", initialAllTopicsCount+2, BrokerTopicStats.getBrokerAllTopicsStats().totalFetchRequestRate.count())
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/utils/TestUtils.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
index 33ab58c..ede145a 100755
--- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala
+++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala
@@ -21,7 +21,7 @@ import java.io._
import java.nio._
import java.nio.channels._
import java.util.concurrent.{Callable, Executors, TimeUnit}
-import java.util.{Properties, Random}
+import java.util.Properties
import java.security.cert.X509Certificate
import javax.net.ssl.X509TrustManager
import charset.Charset
@@ -48,6 +48,7 @@ import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, Produce
import org.apache.kafka.clients.consumer.{KafkaConsumer, RangeAssignor}
import org.apache.kafka.clients.CommonClientConfigs
import org.apache.kafka.common.network.Mode
+import org.apache.kafka.common.record._
import org.apache.kafka.common.serialization.{ByteArraySerializer, Serializer}
import org.apache.kafka.common.utils.Time
import org.apache.kafka.test.{TestUtils => JTestUtils}
@@ -269,16 +270,16 @@ object TestUtils extends Logging {
}
/**
- * Wrap the message in a message set
- *
- * @param payload The bytes of the message
+ * Wrap a single record log buffer.
*/
- def singleMessageSet(payload: Array[Byte],
- codec: CompressionCodec = NoCompressionCodec,
+ def singletonRecords(value: Array[Byte],
key: Array[Byte] = null,
- timestamp: Long = Message.NoTimestamp,
- magicValue: Byte = Message.CurrentMagicValue) =
- new ByteBufferMessageSet(compressionCodec = codec, messages = new Message(payload, key, timestamp, magicValue))
+ codec: CompressionType = CompressionType.NONE,
+ timestamp: Long = Record.NO_TIMESTAMP,
+ magicValue: Byte = Record.CURRENT_MAGIC_VALUE) = {
+ val record = Record.create(magicValue, timestamp, key, value)
+ MemoryRecords.withRecords(codec, record)
+ }
/**
* Generate an array of random bytes
[4/9] kafka git commit: KAFKA-4390;
Replace MessageSet usage with client-side alternatives
Posted by jg...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/server/ReplicaManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala
index af64ffe..9e4c149 100644
--- a/core/src/main/scala/kafka/server/ReplicaManager.scala
+++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
@@ -26,7 +26,7 @@ import kafka.cluster.{Partition, Replica}
import kafka.common._
import kafka.controller.KafkaController
import kafka.log.{LogAppendInfo, LogManager}
-import kafka.message.{ByteBufferMessageSet, InvalidMessageException, Message, MessageSet}
+import kafka.message.InvalidMessageException
import kafka.metrics.KafkaMetricsGroup
import kafka.server.QuotaFactory.UnboundedQuota
import kafka.utils._
@@ -34,6 +34,7 @@ import org.apache.kafka.common.errors.{ControllerMovedException, CorruptRecordEx
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.record._
import org.apache.kafka.common.requests.{LeaderAndIsrRequest, PartitionState, StopReplicaRequest, UpdateMetadataRequest}
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.common.utils.Time
@@ -78,9 +79,11 @@ case class LogReadResult(info: FetchDataInfo,
}
}
+case class FetchPartitionData(error: Short = Errors.NONE.code, hw: Long = -1L, records: Records)
+
object LogReadResult {
val UnknownLogReadResult = LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata,
- MessageSet.Empty),
+ MemoryRecords.EMPTY),
-1L,
-1,
false)
@@ -276,11 +279,7 @@ class ReplicaManager(val config: KafkaConfig,
}
def getPartition(topic: String, partitionId: Int): Option[Partition] = {
- val partition = allPartitions.get((topic, partitionId))
- if (partition == null)
- None
- else
- Some(partition)
+ Option(allPartitions.get((topic, partitionId)))
}
def getReplicaOrException(topic: String, partition: Int): Replica = {
@@ -318,15 +317,15 @@ class ReplicaManager(val config: KafkaConfig,
* Append messages to leader replicas of the partition, and wait for them to be replicated to other replicas;
* the callback function will be triggered either when timeout or the required acks are satisfied
*/
- def appendMessages(timeout: Long,
- requiredAcks: Short,
- internalTopicsAllowed: Boolean,
- messagesPerPartition: Map[TopicPartition, MessageSet],
- responseCallback: Map[TopicPartition, PartitionResponse] => Unit) {
+ def appendRecords(timeout: Long,
+ requiredAcks: Short,
+ internalTopicsAllowed: Boolean,
+ entriesPerPartition: Map[TopicPartition, MemoryRecords],
+ responseCallback: Map[TopicPartition, PartitionResponse] => Unit) {
if (isValidRequiredAcks(requiredAcks)) {
val sTime = time.milliseconds
- val localProduceResults = appendToLocalLog(internalTopicsAllowed, messagesPerPartition, requiredAcks)
+ val localProduceResults = appendToLocalLog(internalTopicsAllowed, entriesPerPartition, requiredAcks)
debug("Produce to local log in %d ms".format(time.milliseconds - sTime))
val produceStatus = localProduceResults.map { case (topicPartition, result) =>
@@ -336,13 +335,13 @@ class ReplicaManager(val config: KafkaConfig,
new PartitionResponse(result.errorCode, result.info.firstOffset, result.info.logAppendTime)) // response status
}
- if (delayedRequestRequired(requiredAcks, messagesPerPartition, localProduceResults)) {
+ if (delayedRequestRequired(requiredAcks, entriesPerPartition, localProduceResults)) {
// create delayed produce operation
val produceMetadata = ProduceMetadata(requiredAcks, produceStatus)
val delayedProduce = new DelayedProduce(timeout, produceMetadata, this, responseCallback)
// create a list of (topic, partition) pairs to use as keys for this delayed produce operation
- val producerRequestKeys = messagesPerPartition.keys.map(new TopicPartitionOperationKey(_)).toSeq
+ val producerRequestKeys = entriesPerPartition.keys.map(new TopicPartitionOperationKey(_)).toSeq
// try to complete the request immediately, otherwise put it into the purgatory
// this is because while the delayed produce operation is being created, new
@@ -357,9 +356,9 @@ class ReplicaManager(val config: KafkaConfig,
} else {
// If required.acks is outside accepted range, something is wrong with the client
// Just return an error and don't handle the request at all
- val responseStatus = messagesPerPartition.map { case (topicAndPartition, _) =>
+ val responseStatus = entriesPerPartition.map { case (topicAndPartition, _) =>
topicAndPartition -> new PartitionResponse(Errors.INVALID_REQUIRED_ACKS.code,
- LogAppendInfo.UnknownLogAppendInfo.firstOffset, Message.NoTimestamp)
+ LogAppendInfo.UnknownLogAppendInfo.firstOffset, Record.NO_TIMESTAMP)
}
responseCallback(responseStatus)
}
@@ -370,11 +369,12 @@ class ReplicaManager(val config: KafkaConfig,
// 1. required acks = -1
// 2. there is data to append
// 3. at least one partition append was successful (fewer errors than partitions)
- private def delayedRequestRequired(requiredAcks: Short, messagesPerPartition: Map[TopicPartition, MessageSet],
- localProduceResults: Map[TopicPartition, LogAppendResult]): Boolean = {
+ private def delayedRequestRequired(requiredAcks: Short,
+ entriesPerPartition: Map[TopicPartition, MemoryRecords],
+ localProduceResults: Map[TopicPartition, LogAppendResult]): Boolean = {
requiredAcks == -1 &&
- messagesPerPartition.nonEmpty &&
- localProduceResults.values.count(_.error.isDefined) < messagesPerPartition.size
+ entriesPerPartition.nonEmpty &&
+ localProduceResults.values.count(_.error.isDefined) < entriesPerPartition.size
}
private def isValidRequiredAcks(requiredAcks: Short): Boolean = {
@@ -385,10 +385,10 @@ class ReplicaManager(val config: KafkaConfig,
* Append the messages to the local replica logs
*/
private def appendToLocalLog(internalTopicsAllowed: Boolean,
- messagesPerPartition: Map[TopicPartition, MessageSet],
+ entriesPerPartition: Map[TopicPartition, MemoryRecords],
requiredAcks: Short): Map[TopicPartition, LogAppendResult] = {
- trace("Append [%s] to local log ".format(messagesPerPartition))
- messagesPerPartition.map { case (topicPartition, messages) =>
+ trace("Append [%s] to local log ".format(entriesPerPartition))
+ entriesPerPartition.map { case (topicPartition, records) =>
BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).totalProduceRequestRate.mark()
BrokerTopicStats.getBrokerAllTopicsStats().totalProduceRequestRate.mark()
@@ -402,7 +402,7 @@ class ReplicaManager(val config: KafkaConfig,
val partitionOpt = getPartition(topicPartition.topic, topicPartition.partition)
val info = partitionOpt match {
case Some(partition) =>
- partition.appendMessagesToLeader(messages.asInstanceOf[ByteBufferMessageSet], requiredAcks)
+ partition.appendRecordsToLeader(records, requiredAcks)
case None => throw new UnknownTopicOrPartitionException("Partition %s doesn't exist on %d"
.format(topicPartition, localBrokerId))
}
@@ -414,13 +414,13 @@ class ReplicaManager(val config: KafkaConfig,
info.lastOffset - info.firstOffset + 1
// update stats for successfully appended bytes and messages as bytesInRate and messageInRate
- BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).bytesInRate.mark(messages.sizeInBytes)
- BrokerTopicStats.getBrokerAllTopicsStats.bytesInRate.mark(messages.sizeInBytes)
+ BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).bytesInRate.mark(records.sizeInBytes)
+ BrokerTopicStats.getBrokerAllTopicsStats.bytesInRate.mark(records.sizeInBytes)
BrokerTopicStats.getBrokerTopicStats(topicPartition.topic).messagesInRate.mark(numAppendedMessages)
BrokerTopicStats.getBrokerAllTopicsStats.messagesInRate.mark(numAppendedMessages)
trace("%d bytes written to log %s-%d beginning at offset %d and ending at offset %d"
- .format(messages.sizeInBytes, topicPartition.topic, topicPartition.partition, info.firstOffset, info.lastOffset))
+ .format(records.sizeInBytes, topicPartition.topic, topicPartition.partition, info.firstOffset, info.lastOffset))
(topicPartition, LogAppendResult(info))
} catch {
// NOTE: Failed produce requests metric is not incremented for known exceptions
@@ -434,6 +434,7 @@ class ReplicaManager(val config: KafkaConfig,
_: RecordTooLargeException |
_: RecordBatchTooLargeException |
_: CorruptRecordException |
+ _: InvalidRecordException |
_: InvalidMessageException |
_: InvalidTimestampException) =>
(topicPartition, LogAppendResult(LogAppendInfo.UnknownLogAppendInfo, Some(e)))
@@ -458,7 +459,7 @@ class ReplicaManager(val config: KafkaConfig,
hardMaxBytesLimit: Boolean,
fetchInfos: Seq[(TopicPartition, PartitionData)],
quota: ReplicaQuota = UnboundedQuota,
- responseCallback: Seq[(TopicAndPartition, FetchResponsePartitionData)] => Unit) {
+ responseCallback: Seq[(TopicAndPartition, FetchPartitionData)] => Unit) {
val isFromFollower = replicaId >= 0
val fetchOnlyFromLeader: Boolean = replicaId != Request.DebuggingConsumerId
val fetchOnlyCommitted: Boolean = ! Request.isValidBrokerId(replicaId)
@@ -480,7 +481,7 @@ class ReplicaManager(val config: KafkaConfig,
// check if this fetch request can be satisfied right away
val logReadResultValues = logReadResults.map { case (_, v) => v }
- val bytesReadable = logReadResultValues.map(_.info.messageSet.sizeInBytes).sum
+ val bytesReadable = logReadResultValues.map(_.info.records.sizeInBytes).sum
val errorReadingData = logReadResultValues.foldLeft(false) ((errorIncurred, readResult) =>
errorIncurred || (readResult.errorCode != Errors.NONE.code))
@@ -490,7 +491,7 @@ class ReplicaManager(val config: KafkaConfig,
// 4) some error happens while reading data
if (timeout <= 0 || fetchInfos.isEmpty || bytesReadable >= fetchMinBytes || errorReadingData) {
val fetchPartitionData = logReadResults.map { case (tp, result) =>
- tp -> FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)
+ tp -> FetchPartitionData(result.errorCode, result.hw, result.info.records)
}
responseCallback(fetchPartitionData)
} else {
@@ -568,16 +569,16 @@ class ReplicaManager(val config: KafkaConfig,
// If the partition is being throttled, simply return an empty set.
if (shouldLeaderThrottle(quota, TopicAndPartition(tp.topic, tp.partition), replicaId))
- FetchDataInfo(fetch.fetchOffsetMetadata, MessageSet.Empty)
+ FetchDataInfo(fetch.fetchOffsetMetadata, MemoryRecords.EMPTY)
// For FetchRequest version 3, we replace incomplete message sets with an empty one as consumers can make
// progress in such cases and don't need to report a `RecordTooLargeException`
- else if (!hardMaxBytesLimit && fetch.firstMessageSetIncomplete)
- FetchDataInfo(fetch.fetchOffsetMetadata, MessageSet.Empty)
+ else if (!hardMaxBytesLimit && fetch.firstEntryIncomplete)
+ FetchDataInfo(fetch.fetchOffsetMetadata, MemoryRecords.EMPTY)
else fetch
case None =>
error(s"Leader for partition $tp does not have a local log")
- FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty)
+ FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY)
}
val readToEndOfLog = initialLogEndOffset.messageOffset - logReadInfo.fetchOffsetMetadata.messageOffset <= 0
@@ -590,12 +591,14 @@ class ReplicaManager(val config: KafkaConfig,
_: NotLeaderForPartitionException |
_: ReplicaNotAvailableException |
_: OffsetOutOfRangeException) =>
- LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, partitionFetchSize, false, Some(e))
+ LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY), -1L,
+ partitionFetchSize, false, Some(e))
case e: Throwable =>
BrokerTopicStats.getBrokerTopicStats(topic).failedFetchRequestRate.mark()
BrokerTopicStats.getBrokerAllTopicsStats().failedFetchRequestRate.mark()
- error(s"Error processing fetch operation on partition ${tp}, offset $offset", e)
- LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MessageSet.Empty), -1L, partitionFetchSize, false, Some(e))
+ error(s"Error processing fetch operation on partition $tp, offset $offset", e)
+ LogReadResult(FetchDataInfo(LogOffsetMetadata.UnknownOffsetMetadata, MemoryRecords.EMPTY), -1L,
+ partitionFetchSize, false, Some(e))
}
}
@@ -604,7 +607,7 @@ class ReplicaManager(val config: KafkaConfig,
var minOneMessage = !hardMaxBytesLimit
readPartitionInfo.foreach { case (tp, fetchInfo) =>
val readResult = read(tp, fetchInfo, limitBytes, minOneMessage)
- val messageSetSize = readResult.info.messageSet.sizeInBytes
+ val messageSetSize = readResult.info.records.sizeInBytes
// Once we read from a non-empty partition, we stop ignoring request and partition level size limits
if (messageSetSize > 0)
minOneMessage = false
@@ -625,9 +628,9 @@ class ReplicaManager(val config: KafkaConfig,
quota.isThrottled(topicPartition) && quota.isQuotaExceeded && !isReplicaInSync
}
- def getMessageFormatVersion(topicAndPartition: TopicAndPartition): Option[Byte] =
+ def getMagicAndTimestampType(topicAndPartition: TopicAndPartition): Option[(Byte, TimestampType)] =
getReplica(topicAndPartition.topic, topicAndPartition.partition).flatMap { replica =>
- replica.log.map(_.config.messageFormatVersion.messageFormatVersion)
+ replica.log.map(log => (log.config.messageFormatVersion.messageFormatVersion, log.config.messageTimestampType))
}
def maybeUpdateMetadataCache(correlationId: Int, updateMetadataRequest: UpdateMetadataRequest, metadataCache: MetadataCache) {
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/tools/DumpLogSegments.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala
index 221ef6c..ceff78c 100755
--- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala
+++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala
@@ -23,15 +23,16 @@ import java.nio.ByteBuffer
import joptsimple.OptionParser
import kafka.coordinator.{GroupMetadataKey, GroupMetadataManager, OffsetKey}
import kafka.log._
-import kafka.message._
import kafka.serializer.Decoder
import kafka.utils._
import org.apache.kafka.clients.consumer.internals.ConsumerProtocol
import org.apache.kafka.common.KafkaException
+import org.apache.kafka.common.record.{CompressionType, FileRecords, LogEntry, Record}
import org.apache.kafka.common.utils.Utils
import scala.collection.mutable
import scala.collection.mutable.ArrayBuffer
+import scala.collection.JavaConverters._
object DumpLogSegments {
@@ -69,7 +70,7 @@ object DumpLogSegments {
CommandLineUtils.checkRequiredArgs(parser, options, filesOpt)
- val printDataLog = (options.has(printOpt) || options.has(offsetsOpt) || options.has(valueDecoderOpt) || options.has(keyDecoderOpt))
+ val printDataLog = options.has(printOpt) || options.has(offsetsOpt) || options.has(valueDecoderOpt) || options.has(keyDecoderOpt)
val verifyOnly = options.has(verifyOpt)
val indexSanityOnly = options.has(indexSanityOpt)
@@ -132,7 +133,7 @@ object DumpLogSegments {
maxMessageSize: Int) {
val startOffset = file.getName().split("\\.")(0).toLong
val logFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.LogFileSuffix)
- val messageSet = new FileMessageSet(logFile, false)
+ val fileRecords = FileRecords.open(logFile, false)
val index = new OffsetIndex(file, baseOffset = startOffset)
//Check that index passes sanityCheck, this is the check that determines if indexes will be rebuilt on startup or not.
@@ -144,11 +145,11 @@ object DumpLogSegments {
for(i <- 0 until index.entries) {
val entry = index.entry(i)
- val partialFileMessageSet: FileMessageSet = messageSet.read(entry.position, maxMessageSize)
- val messageAndOffset = getIterator(partialFileMessageSet.head, isDeepIteration = true).next()
- if(messageAndOffset.offset != entry.offset + index.baseOffset) {
+ val slice = fileRecords.read(entry.position, maxMessageSize)
+ val logEntry = getIterator(slice.shallowIterator.next, isDeepIteration = true).next()
+ if (logEntry.offset != entry.offset + index.baseOffset) {
var misMatchesSeq = misMatchesForIndexFilesMap.getOrElse(file.getAbsolutePath, List[(Long, Long)]())
- misMatchesSeq ::=(entry.offset + index.baseOffset, messageAndOffset.offset)
+ misMatchesSeq ::=(entry.offset + index.baseOffset, logEntry.offset)
misMatchesForIndexFilesMap.put(file.getAbsolutePath, misMatchesSeq)
}
// since it is a sparse file, in the event of a crash there may be many zero entries, stop if we see one
@@ -164,9 +165,9 @@ object DumpLogSegments {
verifyOnly: Boolean,
timeIndexDumpErrors: TimeIndexDumpErrors,
maxMessageSize: Int) {
- val startOffset = file.getName().split("\\.")(0).toLong
+ val startOffset = file.getName.split("\\.")(0).toLong
val logFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.LogFileSuffix)
- val messageSet = new FileMessageSet(logFile, false)
+ val fileRecords = FileRecords.open(logFile, false)
val indexFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.IndexFileSuffix)
val index = new OffsetIndex(indexFile, baseOffset = startOffset)
val timeIndex = new TimeIndex(file, baseOffset = startOffset)
@@ -178,26 +179,26 @@ object DumpLogSegments {
return
}
- var prevTimestamp = Message.NoTimestamp
+ var prevTimestamp = Record.NO_TIMESTAMP
for(i <- 0 until timeIndex.entries) {
val entry = timeIndex.entry(i)
val position = index.lookup(entry.offset + timeIndex.baseOffset).position
- val partialFileMessageSet: FileMessageSet = messageSet.read(position, Int.MaxValue)
- val shallowIter = partialFileMessageSet.iterator
- var maxTimestamp = Message.NoTimestamp
+ val partialFileRecords = fileRecords.read(position, Int.MaxValue)
+ val shallowEntries = partialFileRecords.shallowIterator.asScala
+ var maxTimestamp = Record.NO_TIMESTAMP
// We first find the message by offset then check if the timestamp is correct.
- val wrapperMessageOpt = shallowIter.find(_.offset >= entry.offset + timeIndex.baseOffset)
- wrapperMessageOpt match {
+ val maybeLogEntry = shallowEntries.find(_.offset >= entry.offset + timeIndex.baseOffset)
+ maybeLogEntry match {
case None =>
timeIndexDumpErrors.recordShallowOffsetNotFound(file, entry.offset + timeIndex.baseOffset,
-1.toLong)
- case Some(wrapperMessage) if wrapperMessage.offset != entry.offset + timeIndex.baseOffset =>
+ case Some(logEntry) if logEntry.offset != entry.offset + timeIndex.baseOffset =>
timeIndexDumpErrors.recordShallowOffsetNotFound(file, entry.offset + timeIndex.baseOffset,
- wrapperMessage.offset)
- case Some(wrapperMessage) =>
- val deepIter = getIterator(wrapperMessage, isDeepIteration = true)
- for (messageAndOffset <- deepIter)
- maxTimestamp = math.max(maxTimestamp, messageAndOffset.message.timestamp)
+ logEntry.offset)
+ case Some(shallowLogEntry) =>
+ val deepIter = getIterator(shallowLogEntry, isDeepIteration = true)
+ for (deepLogEntry <- deepIter)
+ maxTimestamp = math.max(maxTimestamp, deepLogEntry.record.timestamp)
if (maxTimestamp != entry.timestamp)
timeIndexDumpErrors.recordMismatchTimeIndex(file, entry.timestamp, maxTimestamp)
@@ -216,20 +217,20 @@ object DumpLogSegments {
}
private trait MessageParser[K, V] {
- def parse(message: Message): (Option[K], Option[V])
+ def parse(record: Record): (Option[K], Option[V])
}
private class DecoderMessageParser[K, V](keyDecoder: Decoder[K], valueDecoder: Decoder[V]) extends MessageParser[K, V] {
- override def parse(message: Message): (Option[K], Option[V]) = {
- if (message.isNull) {
+ override def parse(record: Record): (Option[K], Option[V]) = {
+ if (record.hasNullValue) {
(None, None)
} else {
- val key = if (message.hasKey)
- Some(keyDecoder.fromBytes(Utils.readBytes(message.key)))
+ val key = if (record.hasKey)
+ Some(keyDecoder.fromBytes(Utils.readBytes(record.key)))
else
None
- val payload = Some(valueDecoder.fromBytes(Utils.readBytes(message.payload)))
+ val payload = Some(valueDecoder.fromBytes(Utils.readBytes(record.value)))
(key, payload)
}
@@ -249,7 +250,7 @@ object DumpLogSegments {
val topicPartition = offsetKey.key.topicPartition
val offset = GroupMetadataManager.readOffsetMessageValue(payload)
- val keyString = s"offset::${group}:${topicPartition.topic}:${topicPartition.partition}"
+ val keyString = s"offset::$group:${topicPartition.topic}:${topicPartition.partition}"
val valueString = if (offset.metadata.isEmpty)
String.valueOf(offset.offset)
else
@@ -271,27 +272,27 @@ object DumpLogSegments {
if (userData.isEmpty)
s"${member.memberId}=${partitionAssignment.partitions()}"
else
- s"${member.memberId}=${partitionAssignment.partitions()}:${userData}"
+ s"${member.memberId}=${partitionAssignment.partitions()}:$userData"
} else {
s"${member.memberId}=${hex(member.assignment)}"
}
}.mkString("{", ",", "}")
- val keyString = s"metadata::${groupId}"
- val valueString = s"${protocolType}:${group.protocol}:${group.generationId}:${assignment}"
+ val keyString = s"metadata::$groupId"
+ val valueString = s"$protocolType:${group.protocol}:${group.generationId}:$assignment"
(Some(keyString), Some(valueString))
}
- override def parse(message: Message): (Option[String], Option[String]) = {
- if (message.isNull)
+ override def parse(record: Record): (Option[String], Option[String]) = {
+ if (record.hasNullValue)
(None, None)
- else if (!message.hasKey) {
+ else if (!record.hasKey) {
throw new KafkaException("Failed to decode message using offset topic decoder (message had a missing key)")
} else {
- GroupMetadataManager.readMessageKey(message.key) match {
- case offsetKey: OffsetKey => parseOffsets(offsetKey, message.payload)
- case groupMetadataKey: GroupMetadataKey => parseGroupMetadata(groupMetadataKey, message.payload)
+ GroupMetadataManager.readMessageKey(record.key) match {
+ case offsetKey: OffsetKey => parseOffsets(offsetKey, record.value)
+ case groupMetadataKey: GroupMetadataKey => parseGroupMetadata(groupMetadataKey, record.value)
case _ => throw new KafkaException("Failed to decode message using offset topic decoder (message had an invalid key)")
}
}
@@ -307,70 +308,51 @@ object DumpLogSegments {
parser: MessageParser[_, _]) {
val startOffset = file.getName().split("\\.")(0).toLong
println("Starting offset: " + startOffset)
- val messageSet = new FileMessageSet(file, false)
+ val messageSet = FileRecords.open(file, false)
var validBytes = 0L
var lastOffset = -1l
- val shallowIterator = messageSet.iterator(maxMessageSize)
- for(shallowMessageAndOffset <- shallowIterator) { // this only does shallow iteration
- val itr = getIterator(shallowMessageAndOffset, isDeepIteration)
- for (messageAndOffset <- itr) {
- val msg = messageAndOffset.message
+ val shallowIterator = messageSet.shallowIterator(maxMessageSize).asScala
+ for (shallowLogEntry <- shallowIterator) { // this only does shallow iteration
+ val itr = getIterator(shallowLogEntry, isDeepIteration)
+ for (deepLogEntry <- itr) {
+ val record = deepLogEntry.record()
if(lastOffset == -1)
- lastOffset = messageAndOffset.offset
+ lastOffset = deepLogEntry.offset
// If we are iterating uncompressed messages, offsets must be consecutive
- else if (msg.compressionCodec == NoCompressionCodec && messageAndOffset.offset != lastOffset +1) {
+ else if (record.compressionType == CompressionType.NONE && deepLogEntry.offset != lastOffset +1) {
var nonConsecutivePairsSeq = nonConsecutivePairsForLogFilesMap.getOrElse(file.getAbsolutePath, List[(Long, Long)]())
- nonConsecutivePairsSeq ::=(lastOffset, messageAndOffset.offset)
+ nonConsecutivePairsSeq ::=(lastOffset, deepLogEntry.offset)
nonConsecutivePairsForLogFilesMap.put(file.getAbsolutePath, nonConsecutivePairsSeq)
}
- lastOffset = messageAndOffset.offset
-
- print("offset: " + messageAndOffset.offset + " position: " + validBytes +
- " " + msg.timestampType + ": " + msg.timestamp + " isvalid: " + msg.isValid +
- " payloadsize: " + msg.payloadSize + " magic: " + msg.magic +
- " compresscodec: " + msg.compressionCodec + " crc: " + msg.checksum)
- if(msg.hasKey)
- print(" keysize: " + msg.keySize)
- if(printContents) {
- val (key, payload) = parser.parse(msg)
- key.map(key => print(s" key: ${key}"))
- payload.map(payload => print(s" payload: ${payload}"))
+ lastOffset = deepLogEntry.offset
+
+ print("offset: " + deepLogEntry.offset + " position: " + validBytes +
+ " " + record.timestampType + ": " + record.timestamp + " isvalid: " + record.isValid +
+ " payloadsize: " + record.valueSize + " magic: " + record.magic +
+ " compresscodec: " + record.compressionType + " crc: " + record.checksum)
+ if (record.hasKey)
+ print(" keysize: " + record.keySize)
+ if (printContents) {
+ val (key, payload) = parser.parse(record)
+ key.foreach(key => print(s" key: $key"))
+ payload.foreach(payload => print(s" payload: $payload"))
}
println()
}
- validBytes += MessageSet.entrySize(shallowMessageAndOffset.message)
+
+ validBytes += shallowLogEntry.sizeInBytes
}
val trailingBytes = messageSet.sizeInBytes - validBytes
if(trailingBytes > 0)
println("Found %d invalid bytes at the end of %s".format(trailingBytes, file.getName))
}
- private def getIterator(messageAndOffset: MessageAndOffset, isDeepIteration: Boolean) = {
- if (isDeepIteration) {
- val message = messageAndOffset.message
- message.compressionCodec match {
- case NoCompressionCodec =>
- getSingleMessageIterator(messageAndOffset)
- case _ =>
- ByteBufferMessageSet.deepIterator(messageAndOffset)
- }
- } else
- getSingleMessageIterator(messageAndOffset)
- }
-
- private def getSingleMessageIterator(messageAndOffset: MessageAndOffset) = {
- new IteratorTemplate[MessageAndOffset] {
- var messageIterated = false
-
- override def makeNext(): MessageAndOffset = {
- if (!messageIterated) {
- messageIterated = true
- messageAndOffset
- } else
- allDone()
- }
- }
+ private def getIterator(logEntry: LogEntry, isDeepIteration: Boolean): Iterator[LogEntry] = {
+ if (isDeepIteration)
+ logEntry.iterator.asScala
+ else
+ Iterator(logEntry)
}
class TimeIndexDumpErrors {
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
index 479b43c..c483021 100644
--- a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
+++ b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala
@@ -17,25 +17,26 @@
package kafka.tools
-import joptsimple.OptionParser
-import kafka.cluster.BrokerEndPoint
-import kafka.message.{ByteBufferMessageSet, MessageAndOffset, MessageSet}
+import java.text.SimpleDateFormat
+import java.util.Date
import java.util.concurrent.CountDownLatch
import java.util.concurrent.atomic.AtomicReference
-
-import kafka.client.ClientUtils
import java.util.regex.{Pattern, PatternSyntaxException}
+import joptsimple.OptionParser
import kafka.api._
-import java.text.SimpleDateFormat
-import java.util.Date
-
+import kafka.client.ClientUtils
+import kafka.cluster.BrokerEndPoint
import kafka.common.TopicAndPartition
-import kafka.utils._
import kafka.consumer.{ConsumerConfig, SimpleConsumer, Whitelist}
+import kafka.message.{ByteBufferMessageSet, MessageSet}
+import kafka.utils._
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.utils.Time
+import scala.collection.JavaConverters._
+
+
/**
* For verifying the consistency among replicas.
*
@@ -149,15 +150,15 @@ object ReplicaVerificationTool extends Logging {
debug("Selected topic partitions: " + topicPartitionReplicaList)
val topicAndPartitionsPerBroker: Map[Int, Seq[TopicAndPartition]] = topicPartitionReplicaList.groupBy(_.replicaId)
.map { case (brokerId, partitions) =>
- brokerId -> partitions.map { case partition => new TopicAndPartition(partition.topic, partition.partitionId) } }
+ brokerId -> partitions.map { partition => TopicAndPartition(partition.topic, partition.partitionId) } }
debug("Topic partitions per broker: " + topicAndPartitionsPerBroker)
val expectedReplicasPerTopicAndPartition: Map[TopicAndPartition, Int] =
- topicPartitionReplicaList.groupBy(replica => new TopicAndPartition(replica.topic, replica.partitionId))
+ topicPartitionReplicaList.groupBy(replica => TopicAndPartition(replica.topic, replica.partitionId))
.map { case (topicAndPartition, replicaSet) => topicAndPartition -> replicaSet.size }
debug("Expected replicas per topic partition: " + expectedReplicasPerTopicAndPartition)
val leadersPerBroker: Map[Int, Seq[TopicAndPartition]] = filteredTopicMetadata.flatMap { topicMetadataResponse =>
topicMetadataResponse.partitionsMetadata.map { partitionMetadata =>
- (new TopicAndPartition(topicMetadataResponse.topic, partitionMetadata.partitionId), partitionMetadata.leader.get.id)
+ (TopicAndPartition(topicMetadataResponse.topic, partitionMetadata.partitionId), partitionMetadata.leader.get.id)
}
}.groupBy(_._2).mapValues(topicAndPartitionAndLeaderIds => topicAndPartitionAndLeaderIds.map { case (topicAndPartition, _) =>
topicAndPartition
@@ -200,8 +201,6 @@ object ReplicaVerificationTool extends Logging {
private case class TopicPartitionReplica(topic: String, partitionId: Int, replicaId: Int)
-private case class ReplicaAndMessageIterator(replicaId: Int, iterator: Iterator[MessageAndOffset])
-
private case class MessageInfo(replicaId: Int, offset: Long, nextOffset: Long, checksum: Long)
private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPartition, Int],
@@ -276,41 +275,42 @@ private class ReplicaBuffer(expectedReplicasPerTopicAndPartition: Map[TopicAndPa
assert(fetchResponsePerReplica.size == expectedReplicasPerTopicAndPartition(topicAndPartition),
"fetched " + fetchResponsePerReplica.size + " replicas for " + topicAndPartition + ", but expected "
+ expectedReplicasPerTopicAndPartition(topicAndPartition) + " replicas")
- val messageIteratorMap = fetchResponsePerReplica.map {
+ val logEntryIteratorMap = fetchResponsePerReplica.map {
case(replicaId, fetchResponse) =>
- replicaId -> fetchResponse.messages.asInstanceOf[ByteBufferMessageSet].shallowIterator}
+ replicaId -> fetchResponse.messages.asInstanceOf[ByteBufferMessageSet].asRecords.shallowIterator.asScala
+ }
val maxHw = fetchResponsePerReplica.values.map(_.hw).max
// Iterate one message at a time from every replica, until high watermark is reached.
var isMessageInAllReplicas = true
while (isMessageInAllReplicas) {
var messageInfoFromFirstReplicaOpt: Option[MessageInfo] = None
- for ( (replicaId, messageIterator) <- messageIteratorMap) {
+ for ( (replicaId, logEntries) <- logEntryIteratorMap) {
try {
- if (messageIterator.hasNext) {
- val messageAndOffset = messageIterator.next()
+ if (logEntries.hasNext) {
+ val logEntry = logEntries.next()
// only verify up to the high watermark
- if (messageAndOffset.offset >= fetchResponsePerReplica.get(replicaId).hw)
+ if (logEntry.offset >= fetchResponsePerReplica.get(replicaId).hw)
isMessageInAllReplicas = false
else {
messageInfoFromFirstReplicaOpt match {
case None =>
messageInfoFromFirstReplicaOpt = Some(
- MessageInfo(replicaId, messageAndOffset.offset,messageAndOffset.nextOffset, messageAndOffset.message.checksum))
+ MessageInfo(replicaId, logEntry.offset,logEntry.nextOffset, logEntry.record.checksum))
case Some(messageInfoFromFirstReplica) =>
- if (messageInfoFromFirstReplica.offset != messageAndOffset.offset) {
+ if (messageInfoFromFirstReplica.offset != logEntry.offset) {
println(ReplicaVerificationTool.getCurrentTimeString + ": partition " + topicAndPartition
+ ": replica " + messageInfoFromFirstReplica.replicaId + "'s offset "
+ messageInfoFromFirstReplica.offset + " doesn't match replica "
- + replicaId + "'s offset " + messageAndOffset.offset)
+ + replicaId + "'s offset " + logEntry.offset)
System.exit(1)
}
- if (messageInfoFromFirstReplica.checksum != messageAndOffset.message.checksum)
+ if (messageInfoFromFirstReplica.checksum != logEntry.record.checksum)
println(ReplicaVerificationTool.getCurrentTimeString + ": partition "
- + topicAndPartition + " has unmatched checksum at offset " + messageAndOffset.offset + "; replica "
+ + topicAndPartition + " has unmatched checksum at offset " + logEntry.offset + "; replica "
+ messageInfoFromFirstReplica.replicaId + "'s checksum " + messageInfoFromFirstReplica.checksum
- + "; replica " + replicaId + "'s checksum " + messageAndOffset.message.checksum)
+ + "; replica " + replicaId + "'s checksum " + logEntry.record.checksum)
}
}
} else
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala
index f36e146..51e987a 100644
--- a/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala
+++ b/core/src/test/scala/integration/kafka/api/GroupCoordinatorIntegrationTest.scala
@@ -50,11 +50,13 @@ class GroupCoordinatorIntegrationTest extends KafkaServerTestHarness {
def getGroupMetadataLogOpt: Option[Log] =
logManager.getLog(TopicAndPartition(Topic.GroupMetadataTopicName, 0))
- TestUtils.waitUntilTrue(() => getGroupMetadataLogOpt.exists(_.logSegments.exists(_.log.nonEmpty)),
+ TestUtils.waitUntilTrue(() => getGroupMetadataLogOpt.exists(_.logSegments.exists(_.log.shallowIterator.asScala.nonEmpty)),
"Commit message not appended in time")
val logSegments = getGroupMetadataLogOpt.get.logSegments
- val incorrectCompressionCodecs = logSegments.flatMap(_.log.map(_.message.compressionCodec)).filter(_ != offsetsTopicCompressionCodec)
+ val incorrectCompressionCodecs = logSegments
+ .flatMap(_.log.shallowIterator.asScala.map(_.record.compressionType.id))
+ .filter(_ != offsetsTopicCompressionCodec.codec)
assertEquals("Incorrect compression codecs should be empty", Seq.empty, incorrectCompressionCodecs)
consumer.close()
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/kafka/tools/TestLogCleaning.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/kafka/tools/TestLogCleaning.scala b/core/src/test/scala/kafka/tools/TestLogCleaning.scala
index 51f02d1..ecf7408 100755
--- a/core/src/test/scala/kafka/tools/TestLogCleaning.scala
+++ b/core/src/test/scala/kafka/tools/TestLogCleaning.scala
@@ -21,12 +21,15 @@ import joptsimple.OptionParser
import java.util.Properties
import java.util.Random
import java.io._
+
import kafka.consumer._
import kafka.serializer._
import kafka.utils._
-import kafka.log.FileMessageSet
import kafka.log.Log
-import org.apache.kafka.clients.producer.{ProducerRecord, KafkaProducer, ProducerConfig}
+import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord}
+import org.apache.kafka.common.record.FileRecords
+
+import scala.collection.JavaConverters._
/**
* This is a torture test that runs against an existing broker. Here is how it works:
@@ -135,15 +138,15 @@ object TestLogCleaning {
def dumpLog(dir: File) {
require(dir.exists, "Non-existent directory: " + dir.getAbsolutePath)
- for(file <- dir.list.sorted; if file.endsWith(Log.LogFileSuffix)) {
- val ms = new FileMessageSet(new File(dir, file))
- for(entry <- ms) {
- val key = TestUtils.readString(entry.message.key)
+ for (file <- dir.list.sorted; if file.endsWith(Log.LogFileSuffix)) {
+ val fileRecords = FileRecords.open(new File(dir, file))
+ for (entry <- fileRecords.shallowIterator.asScala) {
+ val key = TestUtils.readString(entry.record.key)
val content =
- if(entry.message.isNull)
+ if(entry.record.hasNullValue)
null
else
- TestUtils.readString(entry.message.payload)
+ TestUtils.readString(entry.record.value)
println("offset = %s, key = %s, content = %s".format(entry.offset, key, content))
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/other/kafka/StressTestLog.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/other/kafka/StressTestLog.scala b/core/src/test/scala/other/kafka/StressTestLog.scala
index f5cee0c..3381fb7 100755
--- a/core/src/test/scala/other/kafka/StressTestLog.scala
+++ b/core/src/test/scala/other/kafka/StressTestLog.scala
@@ -20,10 +20,10 @@ package kafka
import java.util.Properties
import java.util.concurrent.atomic._
-import kafka.message._
import kafka.log._
import kafka.utils._
import org.apache.kafka.clients.consumer.OffsetOutOfRangeException
+import org.apache.kafka.common.record.FileRecords
import org.apache.kafka.common.utils.Utils
/**
@@ -36,13 +36,13 @@ object StressTestLog {
def main(args: Array[String]) {
val dir = TestUtils.randomPartitionLogDir(TestUtils.tempDir())
val time = new MockTime
- val logProprties = new Properties()
- logProprties.put(LogConfig.SegmentBytesProp, 64*1024*1024: java.lang.Integer)
- logProprties.put(LogConfig.MaxMessageBytesProp, Int.MaxValue: java.lang.Integer)
- logProprties.put(LogConfig.SegmentIndexBytesProp, 1024*1024: java.lang.Integer)
+ val logProperties = new Properties()
+ logProperties.put(LogConfig.SegmentBytesProp, 64*1024*1024: java.lang.Integer)
+ logProperties.put(LogConfig.MaxMessageBytesProp, Int.MaxValue: java.lang.Integer)
+ logProperties.put(LogConfig.SegmentIndexBytesProp, 1024*1024: java.lang.Integer)
val log = new Log(dir = dir,
- config = LogConfig(logProprties),
+ config = LogConfig(logProperties),
recoveryPoint = 0L,
scheduler = time.scheduler,
time = time)
@@ -84,7 +84,7 @@ object StressTestLog {
class WriterThread(val log: Log) extends WorkerThread {
@volatile var offset = 0
override def work() {
- val logAppendInfo = log.append(TestUtils.singleMessageSet(offset.toString.getBytes))
+ val logAppendInfo = log.append(TestUtils.singletonRecords(offset.toString.getBytes))
require(logAppendInfo.firstOffset == offset && logAppendInfo.lastOffset == offset)
offset += 1
if(offset % 1000 == 0)
@@ -96,11 +96,11 @@ object StressTestLog {
@volatile var offset = 0
override def work() {
try {
- log.read(offset, 1024, Some(offset+1)).messageSet match {
- case read: FileMessageSet if read.sizeInBytes > 0 => {
- val first = read.head
+ log.read(offset, 1024, Some(offset+1)).records match {
+ case read: FileRecords if read.sizeInBytes > 0 => {
+ val first = read.shallowIterator.next()
require(first.offset == offset, "We should either read nothing or the message we asked for.")
- require(MessageSet.entrySize(first.message) == read.sizeInBytes, "Expected %d but got %d.".format(MessageSet.entrySize(first.message), read.sizeInBytes))
+ require(first.sizeInBytes == read.sizeInBytes, "Expected %d but got %d.".format(first.sizeInBytes, read.sizeInBytes))
offset += 1
}
case _ =>
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala
index 6fef2b3..f0883ad 100755
--- a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala
+++ b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala
@@ -22,13 +22,14 @@ import java.nio._
import java.nio.channels._
import java.util.{Properties, Random}
+import joptsimple._
import kafka.log._
-import kafka.utils._
import kafka.message._
+import kafka.utils._
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record}
+import org.apache.kafka.common.utils.{Time, Utils}
import scala.math._
-import joptsimple._
-import org.apache.kafka.common.utils.{Time, Utils}
/**
* This test does linear writes using either a kafka log or a file and measures throughput and latency.
@@ -64,7 +65,7 @@ object TestLinearWriteSpeed {
.withRequiredArg
.describedAs("ms")
.ofType(classOf[java.lang.Long])
- .defaultsTo(1000)
+ .defaultsTo(1000L)
val maxThroughputOpt = parser.accepts("max-throughput-mb", "The maximum throughput.")
.withRequiredArg
.describedAs("mb")
@@ -81,7 +82,7 @@ object TestLinearWriteSpeed {
.ofType(classOf[java.lang.String])
.defaultsTo(NoCompressionCodec.name)
val mmapOpt = parser.accepts("mmap", "Do writes to memory-mapped files.")
- val channelOpt = parser.accepts("channel", "Do writes to file channesl.")
+ val channelOpt = parser.accepts("channel", "Do writes to file channels.")
val logOpt = parser.accepts("log", "Do writes to kafka logs.")
val options = parser.parse(args : _*)
@@ -101,9 +102,9 @@ object TestLinearWriteSpeed {
val rand = new Random
rand.nextBytes(buffer.array)
val numMessages = bufferSize / (messageSize + MessageSet.LogOverhead)
- val messageSet = new ByteBufferMessageSet(compressionCodec = compressionCodec,
- messages = (0 until numMessages).map(_ => new Message(new Array[Byte](messageSize))): _*)
-
+ val messageSet = MemoryRecords.withRecords(CompressionType.forId(compressionCodec.codec),
+ (0 until numMessages).map(_ => Record.create(new Array[Byte](messageSize))): _*)
+
val writables = new Array[Writable](numFiles)
val scheduler = new KafkaScheduler(1)
scheduler.startup()
@@ -199,7 +200,7 @@ object TestLinearWriteSpeed {
}
}
- class LogWritable(val dir: File, config: LogConfig, scheduler: Scheduler, val messages: ByteBufferMessageSet) extends Writable {
+ class LogWritable(val dir: File, config: LogConfig, scheduler: Scheduler, val messages: MemoryRecords) extends Writable {
Utils.delete(dir)
val log = new Log(dir, config, 0L, scheduler, Time.SYSTEM)
def write(): Int = {
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
index d1fcbc0..b98822d 100644
--- a/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
+++ b/core/src/test/scala/unit/kafka/admin/DeleteTopicTest.scala
@@ -301,7 +301,7 @@ class DeleteTopicTest extends ZooKeeperTestHarness {
var counter = 0
for (_ <- 0 until numDups; key <- 0 until numKeys) yield {
val count = counter
- log.append(TestUtils.singleMessageSet(payload = counter.toString.getBytes, key = key.toString.getBytes), assignOffsets = true)
+ log.append(TestUtils.singletonRecords(value = counter.toString.getBytes, key = key.toString.getBytes), assignOffsets = true)
counter += 1
(key, count)
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
index a981e68..1c5a526 100644
--- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
@@ -18,20 +18,20 @@
package kafka.coordinator
import kafka.utils.timer.MockTimer
-import org.apache.kafka.common.record.Record
+import org.apache.kafka.common.record.{MemoryRecords, Record, TimestampType}
import org.junit.Assert._
import kafka.common.{OffsetAndMetadata, Topic}
-import kafka.message.{Message, MessageSet}
-import kafka.server.{DelayedOperationPurgatory, ReplicaManager, KafkaConfig}
+import kafka.server.{DelayedOperationPurgatory, KafkaConfig, ReplicaManager}
import kafka.utils._
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.Errors
-import org.apache.kafka.common.requests.{OffsetCommitRequest, JoinGroupRequest}
+import org.apache.kafka.common.requests.{JoinGroupRequest, OffsetCommitRequest}
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
-import org.easymock.{Capture, IAnswer, EasyMock}
+import org.easymock.{Capture, EasyMock, IAnswer}
import org.junit.{After, Before, Test}
import org.scalatest.junit.JUnitSuite
import java.util.concurrent.TimeUnit
+
import scala.collection._
import scala.concurrent.duration.Duration
import scala.concurrent.{Await, Future, Promise}
@@ -305,7 +305,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
EasyMock.reset(replicaManager)
EasyMock.expect(replicaManager.getPartition(Topic.GroupMetadataTopicName, groupPartitionId)).andReturn(None)
- EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes()
+ EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
+ .andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes()
EasyMock.replay(replicaManager)
timer.advanceClock(DefaultSessionTimeout + 100)
@@ -988,17 +989,18 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
- EasyMock.expect(replicaManager.appendMessages(EasyMock.anyLong(),
+ EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
EasyMock.anyShort(),
EasyMock.anyBoolean(),
- EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MessageSet]],
+ EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
override def answer = capturedArgument.getValue.apply(
Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) ->
new PartitionResponse(Errors.NONE.code, 0L, Record.NO_TIMESTAMP)
)
)})
- EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes()
+ EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
+ .andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes()
EasyMock.replay(replicaManager)
groupCoordinator.handleSyncGroup(groupId, generation, leaderId, assignment, responseCallback)
@@ -1069,17 +1071,18 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
- EasyMock.expect(replicaManager.appendMessages(EasyMock.anyLong(),
+ EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
EasyMock.anyShort(),
EasyMock.anyBoolean(),
- EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MessageSet]],
+ EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
override def answer = capturedArgument.getValue.apply(
Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) ->
new PartitionResponse(Errors.NONE.code, 0L, Record.NO_TIMESTAMP)
)
)})
- EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes()
+ EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
+ .andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes()
EasyMock.replay(replicaManager)
groupCoordinator.handleCommitOffsets(groupId, consumerId, generationId, offsets, responseCallback)
@@ -1090,7 +1093,8 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
val (responseFuture, responseCallback) = setupHeartbeatCallback
EasyMock.expect(replicaManager.getPartition(Topic.GroupMetadataTopicName, groupPartitionId)).andReturn(None)
- EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes()
+ EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
+ .andReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME)).anyTimes()
EasyMock.replay(replicaManager)
groupCoordinator.handleLeaveGroup(groupId, consumerId, responseCallback)
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala
index 6c03476..62b7f42 100644
--- a/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/coordinator/GroupMetadataManagerTest.scala
@@ -21,12 +21,11 @@ import kafka.api.ApiVersion
import kafka.cluster.Partition
import kafka.common.{OffsetAndMetadata, Topic}
import kafka.log.LogAppendInfo
-import kafka.message.{ByteBufferMessageSet, Message, MessageSet}
import kafka.server.{KafkaConfig, ReplicaManager}
import kafka.utils.{KafkaScheduler, MockTime, TestUtils, ZkUtils}
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.Errors
-import org.apache.kafka.common.record.Record
+import org.apache.kafka.common.record.{MemoryRecords, Record, TimestampType}
import org.apache.kafka.common.requests.OffsetFetchResponse
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.easymock.{Capture, EasyMock, IAnswer}
@@ -34,6 +33,7 @@ import org.junit.{After, Before, Test}
import org.junit.Assert._
import scala.collection._
+import JavaConverters._
class GroupMetadataManagerTest {
@@ -50,7 +50,6 @@ class GroupMetadataManagerTest {
val rebalanceTimeout = 60000
val sessionTimeout = 10000
-
@Before
def setUp() {
val config = KafkaConfig.fromProps(TestUtils.createBrokerConfig(nodeId = 0, zkConnect = ""))
@@ -176,7 +175,7 @@ class GroupMetadataManagerTest {
@Test
def testStoreNonEmptyGroupWhenCoordinatorHasMoved() {
- EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(None)
+ EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())).andReturn(None)
val memberId = "memberId"
val clientId = "clientId"
val clientHost = "localhost"
@@ -245,7 +244,7 @@ class GroupMetadataManagerTest {
@Test
def testCommitOffsetWhenCoordinatorHasMoved() {
- EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(None)
+ EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject())).andReturn(None)
val memberId = ""
val generationId = -1
val topicPartition = new TopicPartition("foo", 0)
@@ -363,7 +362,7 @@ class GroupMetadataManagerTest {
time.sleep(2)
EasyMock.reset(partition)
- EasyMock.expect(partition.appendMessagesToLeader(EasyMock.anyObject(classOf[ByteBufferMessageSet]), EasyMock.anyInt()))
+ EasyMock.expect(partition.appendRecordsToLeader(EasyMock.anyObject(classOf[MemoryRecords]), EasyMock.anyInt()))
.andReturn(LogAppendInfo.UnknownLogAppendInfo)
EasyMock.replay(partition)
@@ -391,24 +390,74 @@ class GroupMetadataManagerTest {
// expect the group metadata tombstone
EasyMock.reset(partition)
- val messageSetCapture: Capture[ByteBufferMessageSet] = EasyMock.newCapture()
+ val recordsCapture: Capture[MemoryRecords] = EasyMock.newCapture()
+
+ EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
+ .andStubReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME))
+ EasyMock.expect(replicaManager.getPartition(Topic.GroupMetadataTopicName, groupPartitionId)).andStubReturn(Some(partition))
+ EasyMock.expect(partition.appendRecordsToLeader(EasyMock.capture(recordsCapture), EasyMock.anyInt()))
+ .andReturn(LogAppendInfo.UnknownLogAppendInfo)
+ EasyMock.replay(replicaManager, partition)
+
+ groupMetadataManager.cleanupGroupMetadata()
+
+ assertTrue(recordsCapture.hasCaptured)
+
+ val records = recordsCapture.getValue.records.asScala.toList
+ assertEquals(1, records.size)
+
+ val metadataTombstone = records.head
+ assertTrue(metadataTombstone.hasKey)
+ assertTrue(metadataTombstone.hasNullValue)
+ assertEquals(Record.MAGIC_VALUE_V1, metadataTombstone.magic)
+ assertEquals(TimestampType.CREATE_TIME, metadataTombstone.timestampType)
+ assertTrue(metadataTombstone.timestamp > 0)
+
+ val groupKey = GroupMetadataManager.readMessageKey(metadataTombstone.key).asInstanceOf[GroupMetadataKey]
+ assertEquals(groupId, groupKey.key)
+
+ // the full group should be gone since all offsets were removed
+ assertEquals(None, groupMetadataManager.getGroup(groupId))
+ val cachedOffsets = groupMetadataManager.getOffsets(groupId, Seq(topicPartition1, topicPartition2))
+ assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), cachedOffsets.get(topicPartition1).map(_.offset))
+ assertEquals(Some(OffsetFetchResponse.INVALID_OFFSET), cachedOffsets.get(topicPartition2).map(_.offset))
+ }
+
+ @Test
+ def testGroupMetadataRemovalWithLogAppendTime() {
+ val topicPartition1 = new TopicPartition("foo", 0)
+ val topicPartition2 = new TopicPartition("foo", 1)
+
+ groupMetadataManager.addPartitionOwnership(groupPartitionId)
+
+ val group = new GroupMetadata(groupId)
+ groupMetadataManager.addGroup(group)
+ group.generationId = 5
+
+ // expect the group metadata tombstone
+ EasyMock.reset(partition)
+ val recordsCapture: Capture[MemoryRecords] = EasyMock.newCapture()
- EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andStubReturn(Some(Message.MagicValue_V1))
+ EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
+ .andStubReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.LOG_APPEND_TIME))
EasyMock.expect(replicaManager.getPartition(Topic.GroupMetadataTopicName, groupPartitionId)).andStubReturn(Some(partition))
- EasyMock.expect(partition.appendMessagesToLeader(EasyMock.capture(messageSetCapture), EasyMock.anyInt()))
+ EasyMock.expect(partition.appendRecordsToLeader(EasyMock.capture(recordsCapture), EasyMock.anyInt()))
.andReturn(LogAppendInfo.UnknownLogAppendInfo)
EasyMock.replay(replicaManager, partition)
groupMetadataManager.cleanupGroupMetadata()
- assertTrue(messageSetCapture.hasCaptured)
+ assertTrue(recordsCapture.hasCaptured)
- val messageSet = messageSetCapture.getValue
- assertEquals(1, messageSet.size)
+ val records = recordsCapture.getValue.records.asScala.toList
+ assertEquals(1, records.size)
- val metadataTombstone = messageSet.head.message
+ val metadataTombstone = records.head
assertTrue(metadataTombstone.hasKey)
- assertTrue(metadataTombstone.isNull)
+ assertTrue(metadataTombstone.hasNullValue)
+ assertEquals(Record.MAGIC_VALUE_V1, metadataTombstone.magic)
+ assertEquals(TimestampType.LOG_APPEND_TIME, metadataTombstone.timestampType)
+ assertTrue(metadataTombstone.timestamp > 0)
val groupKey = GroupMetadataManager.readMessageKey(metadataTombstone.key).asInstanceOf[GroupMetadataKey]
assertEquals(groupId, groupKey.key)
@@ -463,22 +512,22 @@ class GroupMetadataManagerTest {
// expect the offset tombstone
EasyMock.reset(partition)
- val messageSetCapture: Capture[ByteBufferMessageSet] = EasyMock.newCapture()
+ val recordsCapture: Capture[MemoryRecords] = EasyMock.newCapture()
- EasyMock.expect(partition.appendMessagesToLeader(EasyMock.capture(messageSetCapture), EasyMock.anyInt()))
+ EasyMock.expect(partition.appendRecordsToLeader(EasyMock.capture(recordsCapture), EasyMock.anyInt()))
.andReturn(LogAppendInfo.UnknownLogAppendInfo)
EasyMock.replay(partition)
groupMetadataManager.cleanupGroupMetadata()
- assertTrue(messageSetCapture.hasCaptured)
+ assertTrue(recordsCapture.hasCaptured)
// verify the tombstones are correct and only for the expired offsets
- val messageSet = messageSetCapture.getValue
- assertEquals(2, messageSet.size)
- messageSet.map(_.message).foreach { message =>
+ val records = recordsCapture.getValue.records.asScala.toList
+ assertEquals(2, records.size)
+ records.foreach { message =>
assertTrue(message.hasKey)
- assertTrue(message.isNull)
+ assertTrue(message.hasNullValue)
val offsetKey = GroupMetadataManager.readMessageKey(message.key).asInstanceOf[OffsetKey]
assertEquals(groupId, offsetKey.key.group)
assertEquals("foo", offsetKey.key.topicPartition.topic)
@@ -539,7 +588,7 @@ class GroupMetadataManagerTest {
// expect the offset tombstone
EasyMock.reset(partition)
- EasyMock.expect(partition.appendMessagesToLeader(EasyMock.anyObject(classOf[ByteBufferMessageSet]), EasyMock.anyInt()))
+ EasyMock.expect(partition.appendRecordsToLeader(EasyMock.anyObject(classOf[MemoryRecords]), EasyMock.anyInt()))
.andReturn(LogAppendInfo.UnknownLogAppendInfo)
EasyMock.replay(partition)
@@ -557,17 +606,18 @@ class GroupMetadataManagerTest {
private def expectAppendMessage(error: Errors) {
val capturedArgument: Capture[Map[TopicPartition, PartitionResponse] => Unit] = EasyMock.newCapture()
- EasyMock.expect(replicaManager.appendMessages(EasyMock.anyLong(),
+ EasyMock.expect(replicaManager.appendRecords(EasyMock.anyLong(),
EasyMock.anyShort(),
EasyMock.anyBoolean(),
- EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MessageSet]],
+ EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MemoryRecords]],
EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
override def answer = capturedArgument.getValue.apply(
Map(new TopicPartition(Topic.GroupMetadataTopicName, groupPartitionId) ->
new PartitionResponse(error.code, 0L, Record.NO_TIMESTAMP)
)
)})
- EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andStubReturn(Some(Message.MagicValue_V1))
+ EasyMock.expect(replicaManager.getMagicAndTimestampType(EasyMock.anyObject()))
+ .andStubReturn(Some(Record.MAGIC_VALUE_V1, TimestampType.CREATE_TIME))
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala
index 791bdb0..296dc15 100755
--- a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala
+++ b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala
@@ -25,9 +25,10 @@ import org.junit.Assert._
import org.junit.runner.RunWith
import org.junit.runners.Parameterized
import org.junit.runners.Parameterized.Parameters
-import org.apache.kafka.common.record.CompressionType
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record}
import org.apache.kafka.common.utils.Utils
import java.util.{Collection, Properties}
+
import scala.collection.JavaConverters._
@RunWith(value = classOf[Parameterized])
@@ -50,22 +51,22 @@ class BrokerCompressionTest(messageCompression: String, brokerCompression: Strin
def testBrokerSideCompression() {
val messageCompressionCode = CompressionCodec.getCompressionCodec(messageCompression)
val logProps = new Properties()
- logProps.put(LogConfig.CompressionTypeProp,brokerCompression)
+ logProps.put(LogConfig.CompressionTypeProp, brokerCompression)
/*configure broker-side compression */
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
/* append two messages */
- log.append(new ByteBufferMessageSet(messageCompressionCode, new Message("hello".getBytes), new Message("there".getBytes)))
+ log.append(MemoryRecords.withRecords(CompressionType.forId(messageCompressionCode.codec),
+ Record.create("hello".getBytes), Record.create("there".getBytes)))
- def readMessage(offset: Int) = log.read(offset, 4096).messageSet.head.message
+ def readMessage(offset: Int) = log.read(offset, 4096).records.shallowIterator.next().record
if (!brokerCompression.equals("producer")) {
val brokerCompressionCode = BrokerCompressionCodec.getCompressionCodec(brokerCompression)
- assertEquals("Compression at offset 0 should produce " + brokerCompressionCode.name, brokerCompressionCode, readMessage(0).compressionCodec)
+ assertEquals("Compression at offset 0 should produce " + brokerCompressionCode.name, brokerCompressionCode.codec, readMessage(0).compressionType.id)
}
else
- assertEquals("Compression at offset 0 should produce " + messageCompressionCode.name, messageCompressionCode, readMessage(0).compressionCodec)
-
+ assertEquals("Compression at offset 0 should produce " + messageCompressionCode.name, messageCompressionCode.codec, readMessage(0).compressionType.id)
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala
deleted file mode 100644
index a7f0446..0000000
--- a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala
+++ /dev/null
@@ -1,354 +0,0 @@
-/**
- * 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.log
-
-import java.io._
-import java.nio._
-import java.nio.channels._
-
-import kafka.common.LongRef
-import org.junit.Assert._
-import kafka.utils.TestUtils._
-import kafka.message._
-import kafka.common.KafkaException
-import org.easymock.EasyMock
-import org.junit.Test
-
-class FileMessageSetTest extends BaseMessageSetTestCases {
-
- val messageSet = createMessageSet(messages)
-
- def createMessageSet(messages: Seq[Message]): FileMessageSet = {
- val set = new FileMessageSet(tempFile())
- set.append(new ByteBufferMessageSet(NoCompressionCodec, messages: _*))
- set.flush()
- set
- }
-
- /**
- * Test that the cached size variable matches the actual file size as we append messages
- */
- @Test
- def testFileSize() {
- assertEquals(messageSet.channel.size, messageSet.sizeInBytes)
- for (_ <- 0 until 20) {
- messageSet.append(singleMessageSet("abcd".getBytes))
- assertEquals(messageSet.channel.size, messageSet.sizeInBytes)
- }
- }
-
- /**
- * Test that adding invalid bytes to the end of the log doesn't break iteration
- */
- @Test
- def testIterationOverPartialAndTruncation() {
- testPartialWrite(0, messageSet)
- testPartialWrite(2, messageSet)
- testPartialWrite(4, messageSet)
- testPartialWrite(5, messageSet)
- testPartialWrite(6, messageSet)
- }
-
- def testPartialWrite(size: Int, messageSet: FileMessageSet) {
- val buffer = ByteBuffer.allocate(size)
- for (_ <- 0 until size)
- buffer.put(0: Byte)
- buffer.rewind()
- messageSet.channel.write(buffer)
- // appending those bytes should not change the contents
- checkEquals(messages.iterator, messageSet.map(m => m.message).iterator)
- }
-
- /**
- * Iterating over the file does file reads but shouldn't change the position of the underlying FileChannel.
- */
- @Test
- def testIterationDoesntChangePosition() {
- val position = messageSet.channel.position
- checkEquals(messages.iterator, messageSet.map(m => m.message).iterator)
- assertEquals(position, messageSet.channel.position)
- }
-
- /**
- * Test a simple append and read.
- */
- @Test
- def testRead() {
- var read = messageSet.read(0, messageSet.sizeInBytes)
- checkEquals(messageSet.iterator, read.iterator)
- val items = read.iterator.toList
- val sec = items.tail.head
- read = messageSet.read(position = MessageSet.entrySize(sec.message), size = messageSet.sizeInBytes)
- assertEquals("Try a read starting from the second message", items.tail, read.toList)
- read = messageSet.read(MessageSet.entrySize(sec.message), MessageSet.entrySize(sec.message))
- assertEquals("Try a read of a single message starting from the second message", List(items.tail.head), read.toList)
- }
-
- /**
- * Test the MessageSet.searchFor API.
- */
- @Test
- def testSearch() {
- // append a new message with a high offset
- val lastMessage = new Message("test".getBytes)
- messageSet.append(new ByteBufferMessageSet(NoCompressionCodec, new LongRef(50), lastMessage))
- val messages = messageSet.toSeq
- var position = 0
- val message1Size = MessageSet.entrySize(messages.head.message)
- assertEquals("Should be able to find the first message by its offset",
- (OffsetPosition(0L, position), message1Size),
- messageSet.searchForOffsetWithSize(0, 0))
- position += message1Size
- val message2Size = MessageSet.entrySize(messages(1).message)
- assertEquals("Should be able to find second message when starting from 0",
- (OffsetPosition(1L, position), message2Size),
- messageSet.searchForOffsetWithSize(1, 0))
- assertEquals("Should be able to find second message starting from its offset",
- (OffsetPosition(1L, position), message2Size),
- messageSet.searchForOffsetWithSize(1, position))
- position += message2Size + MessageSet.entrySize(messages(2).message)
- val message4Size = MessageSet.entrySize(messages(3).message)
- assertEquals("Should be able to find fourth message from a non-existant offset",
- (OffsetPosition(50L, position), message4Size),
- messageSet.searchForOffsetWithSize(3, position))
- assertEquals("Should be able to find fourth message by correct offset",
- (OffsetPosition(50L, position), message4Size),
- messageSet.searchForOffsetWithSize(50, position))
- }
-
- /**
- * Test that the message set iterator obeys start and end slicing
- */
- @Test
- def testIteratorWithLimits() {
- val message = messageSet.toList(1)
- val start = messageSet.searchForOffsetWithSize(1, 0)._1.position
- val size = message.message.size + 12
- val slice = messageSet.read(start, size)
- assertEquals(List(message), slice.toList)
- val slice2 = messageSet.read(start, size - 1)
- assertEquals(List(), slice2.toList)
- }
-
- /**
- * Test the truncateTo method lops off messages and appropriately updates the size
- */
- @Test
- def testTruncate() {
- val message = messageSet.toList.head
- val end = messageSet.searchForOffsetWithSize(1, 0)._1.position
- messageSet.truncateTo(end)
- assertEquals(List(message), messageSet.toList)
- assertEquals(MessageSet.entrySize(message.message), messageSet.sizeInBytes)
- }
-
- /**
- * Test that truncateTo only calls truncate on the FileChannel if the size of the
- * FileChannel is bigger than the target size. This is important because some JVMs
- * change the mtime of the file, even if truncate should do nothing.
- */
- @Test
- def testTruncateNotCalledIfSizeIsSameAsTargetSize() {
- val channelMock = EasyMock.createMock(classOf[FileChannel])
-
- EasyMock.expect(channelMock.size).andReturn(42L).atLeastOnce()
- EasyMock.expect(channelMock.position(42L)).andReturn(null)
- EasyMock.replay(channelMock)
-
- val msgSet = new FileMessageSet(tempFile(), channelMock)
- msgSet.truncateTo(42)
-
- EasyMock.verify(channelMock)
- }
-
- /**
- * Expect a KafkaException if targetSize is bigger than the size of
- * the FileMessageSet.
- */
- @Test
- def testTruncateNotCalledIfSizeIsBiggerThanTargetSize() {
- val channelMock = EasyMock.createMock(classOf[FileChannel])
-
- EasyMock.expect(channelMock.size).andReturn(42L).atLeastOnce()
- EasyMock.expect(channelMock.position(42L)).andReturn(null)
- EasyMock.replay(channelMock)
-
- val msgSet = new FileMessageSet(tempFile(), channelMock)
-
- try {
- msgSet.truncateTo(43)
- fail("Should throw KafkaException")
- } catch {
- case _: KafkaException => // expected
- }
-
- EasyMock.verify(channelMock)
- }
-
- /**
- * see #testTruncateNotCalledIfSizeIsSameAsTargetSize
- */
- @Test
- def testTruncateIfSizeIsDifferentToTargetSize() {
- val channelMock = EasyMock.createMock(classOf[FileChannel])
-
- EasyMock.expect(channelMock.size).andReturn(42L).atLeastOnce()
- EasyMock.expect(channelMock.position(42L)).andReturn(null).once()
- EasyMock.expect(channelMock.truncate(23L)).andReturn(null).once()
- EasyMock.expect(channelMock.position(23L)).andReturn(null).once()
- EasyMock.replay(channelMock)
-
- val msgSet = new FileMessageSet(tempFile(), channelMock)
- msgSet.truncateTo(23)
-
- EasyMock.verify(channelMock)
- }
-
-
- /**
- * Test the new FileMessageSet with pre allocate as true
- */
- @Test
- def testPreallocateTrue() {
- val temp = tempFile()
- val set = new FileMessageSet(temp, false, 512 *1024 *1024, true)
- val position = set.channel.position
- val size = set.sizeInBytes()
- assertEquals(0, position)
- assertEquals(0, size)
- assertEquals(512 *1024 *1024, temp.length)
- }
-
- /**
- * Test the new FileMessageSet with pre allocate as false
- */
- @Test
- def testPreallocateFalse() {
- val temp = tempFile()
- val set = new FileMessageSet(temp, false, 512 *1024 *1024, false)
- val position = set.channel.position
- val size = set.sizeInBytes()
- assertEquals(0, position)
- assertEquals(0, size)
- assertEquals(0, temp.length)
- }
-
- /**
- * Test the new FileMessageSet with pre allocate as true and file has been clearly shut down, the file will be truncate to end of valid data.
- */
- @Test
- def testPreallocateClearShutdown() {
- val temp = tempFile()
- val set = new FileMessageSet(temp, false, 512 *1024 *1024, true)
- set.append(new ByteBufferMessageSet(NoCompressionCodec, messages: _*))
- val oldposition = set.channel.position
- val oldsize = set.sizeInBytes()
- assertEquals(messageSet.sizeInBytes, oldposition)
- assertEquals(messageSet.sizeInBytes, oldsize)
- set.close()
-
- val tempReopen = new File(temp.getAbsolutePath())
- val setReopen = new FileMessageSet(tempReopen, true, 512 *1024 *1024, true)
- val position = setReopen.channel.position
- val size = setReopen.sizeInBytes()
-
- assertEquals(oldposition, position)
- assertEquals(oldposition, size)
- assertEquals(oldposition, tempReopen.length)
- }
-
- @Test
- def testFormatConversionWithPartialMessage() {
- val message = messageSet.toList(1)
- val start = messageSet.searchForOffsetWithSize(1, 0)._1.position
- val size = message.message.size + 12
- val slice = messageSet.read(start, size - 1)
- val messageV0 = slice.toMessageFormat(Message.MagicValue_V0)
- assertEquals("No message should be there", 0, messageV0.size)
- assertEquals(s"There should be ${size - 1} bytes", size - 1, messageV0.sizeInBytes)
- }
-
- @Test
- def testMessageFormatConversion() {
-
- // Prepare messages.
- val offsets = Seq(0L, 2L)
- val messagesV0 = Seq(new Message("hello".getBytes, "k1".getBytes, Message.NoTimestamp, Message.MagicValue_V0),
- new Message("goodbye".getBytes, "k2".getBytes, Message.NoTimestamp, Message.MagicValue_V0))
- val messageSetV0 = new ByteBufferMessageSet(
- compressionCodec = NoCompressionCodec,
- offsetSeq = offsets,
- messages = messagesV0:_*)
- val compressedMessageSetV0 = new ByteBufferMessageSet(
- compressionCodec = DefaultCompressionCodec,
- offsetSeq = offsets,
- messages = messagesV0:_*)
-
- val messagesV1 = Seq(new Message("hello".getBytes, "k1".getBytes, 1L, Message.MagicValue_V1),
- new Message("goodbye".getBytes, "k2".getBytes, 2L, Message.MagicValue_V1))
- val messageSetV1 = new ByteBufferMessageSet(
- compressionCodec = NoCompressionCodec,
- offsetSeq = offsets,
- messages = messagesV1:_*)
- val compressedMessageSetV1 = new ByteBufferMessageSet(
- compressionCodec = DefaultCompressionCodec,
- offsetSeq = offsets,
- messages = messagesV1:_*)
-
- // Down conversion
- // down conversion for non-compressed messages
- var fileMessageSet = new FileMessageSet(tempFile())
- fileMessageSet.append(messageSetV1)
- fileMessageSet.flush()
- var convertedMessageSet = fileMessageSet.toMessageFormat(Message.MagicValue_V0)
- verifyConvertedMessageSet(convertedMessageSet, Message.MagicValue_V0)
-
- // down conversion for compressed messages
- fileMessageSet = new FileMessageSet(tempFile())
- fileMessageSet.append(compressedMessageSetV1)
- fileMessageSet.flush()
- convertedMessageSet = fileMessageSet.toMessageFormat(Message.MagicValue_V0)
- verifyConvertedMessageSet(convertedMessageSet, Message.MagicValue_V0)
-
- // Up conversion. In reality we only do down conversion, but up conversion should work as well.
- // up conversion for non-compressed messages
- fileMessageSet = new FileMessageSet(tempFile())
- fileMessageSet.append(messageSetV0)
- fileMessageSet.flush()
- convertedMessageSet = fileMessageSet.toMessageFormat(Message.MagicValue_V1)
- verifyConvertedMessageSet(convertedMessageSet, Message.MagicValue_V1)
-
- // up conversion for compressed messages
- fileMessageSet = new FileMessageSet(tempFile())
- fileMessageSet.append(compressedMessageSetV0)
- fileMessageSet.flush()
- convertedMessageSet = fileMessageSet.toMessageFormat(Message.MagicValue_V1)
- verifyConvertedMessageSet(convertedMessageSet, Message.MagicValue_V1)
-
- def verifyConvertedMessageSet(convertedMessageSet: MessageSet, magicByte: Byte) {
- var i = 0
- for (messageAndOffset <- convertedMessageSet) {
- assertEquals("magic byte should be 1", magicByte, messageAndOffset.message.magic)
- assertEquals("offset should not change", offsets(i), messageAndOffset.offset)
- assertEquals("key should not change", messagesV0(i).key, messageAndOffset.message.key)
- assertEquals("payload should not change", messagesV0(i).payload, messageAndOffset.message.payload)
- i += 1
- }
- }
- }
-}
[2/9] kafka git commit: KAFKA-4390;
Replace MessageSet usage with client-side alternatives
Posted by jg...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/log/LogTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala
index d18719a..fcf9c89 100755
--- a/core/src/test/scala/unit/kafka/log/LogTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogTest.scala
@@ -22,15 +22,16 @@ import java.util.Properties
import org.apache.kafka.common.errors.{CorruptRecordException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException}
import kafka.api.ApiVersion
-import kafka.common.LongRef
import org.junit.Assert._
import org.scalatest.junit.JUnitSuite
import org.junit.{After, Before, Test}
-import kafka.message._
import kafka.utils._
import kafka.server.KafkaConfig
+import org.apache.kafka.common.record._
import org.apache.kafka.common.utils.Utils
+import scala.collection.JavaConverters._
+
class LogTest extends JUnitSuite {
val tmpDir = TestUtils.tempDir()
@@ -63,7 +64,7 @@ class LogTest extends JUnitSuite {
*/
@Test
def testTimeBasedLogRoll() {
- val set = TestUtils.singleMessageSet("test".getBytes)
+ val set = TestUtils.singletonRecords("test".getBytes)
val logProps = new Properties()
logProps.put(LogConfig.SegmentMsProp, (1 * 60 * 60L): java.lang.Long)
@@ -91,7 +92,7 @@ class LogTest extends JUnitSuite {
// Append a message with timestamp to a segment whose first messgae do not have a timestamp.
val setWithTimestamp =
- TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds + log.config.segmentMs + 1)
+ TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds + log.config.segmentMs + 1)
log.append(setWithTimestamp)
assertEquals("Segment should not have been rolled out because the log rolling should be based on wall clock.", 4, log.numberOfSegments)
@@ -105,14 +106,14 @@ class LogTest extends JUnitSuite {
log.append(setWithTimestamp)
assertEquals("Log should not roll because the roll should depend on timestamp of the first message.", 5, log.numberOfSegments)
- val setWithExpiredTimestamp = TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds)
+ val setWithExpiredTimestamp = TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds)
log.append(setWithExpiredTimestamp)
assertEquals("Log should roll because the timestamp in the message should make the log segment expire.", 6, log.numberOfSegments)
val numSegments = log.numberOfSegments
time.sleep(log.config.segmentMs + 1)
- log.append(new ByteBufferMessageSet())
- assertEquals("Appending an empty message set should not roll log even if succient time has passed.", numSegments, log.numberOfSegments)
+ log.append(MemoryRecords.withLogEntries())
+ assertEquals("Appending an empty message set should not roll log even if sufficient time has passed.", numSegments, log.numberOfSegments)
}
/**
@@ -121,7 +122,7 @@ class LogTest extends JUnitSuite {
*/
@Test
def testTimeBasedLogRollJitter() {
- val set = TestUtils.singleMessageSet("test".getBytes)
+ val set = TestUtils.singletonRecords("test".getBytes)
val maxJitter = 20 * 60L
val logProps = new Properties()
@@ -149,7 +150,7 @@ class LogTest extends JUnitSuite {
*/
@Test
def testSizeBasedLogRoll() {
- val set = TestUtils.singleMessageSet("test".getBytes)
+ val set = TestUtils.singletonRecords("test".getBytes)
val setSize = set.sizeInBytes
val msgPerSeg = 10
val segmentSize = msgPerSeg * (setSize - 1) // each segment will be 10 messages
@@ -176,7 +177,7 @@ class LogTest extends JUnitSuite {
def testLoadEmptyLog() {
createEmptyLogs(logDir, 0)
val log = new Log(logDir, logConfig, recoveryPoint = 0L, time.scheduler, time = time)
- log.append(TestUtils.singleMessageSet("test".getBytes))
+ log.append(TestUtils.singletonRecords("test".getBytes))
}
/**
@@ -189,16 +190,17 @@ class LogTest extends JUnitSuite {
// We use need to use magic value 1 here because the test is message size sensitive.
logProps.put(LogConfig.MessageFormatVersionProp, ApiVersion.latestVersion.toString)
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
- val messages = (0 until 100 by 2).map(id => new Message(id.toString.getBytes)).toArray
+ val records = (0 until 100 by 2).map(id => Record.create(id.toString.getBytes)).toArray
+
+ for(i <- records.indices)
+ log.append(MemoryRecords.withRecords(records(i)))
- for(i <- 0 until messages.length)
- log.append(new ByteBufferMessageSet(NoCompressionCodec, messages = messages(i)))
- for(i <- 0 until messages.length) {
- val read = log.read(i, 100, Some(i+1)).messageSet.head
+ for(i <- records.indices) {
+ val read = log.read(i, 100, Some(i+1)).records.shallowIterator.next()
assertEquals("Offset read should match order appended.", i, read.offset)
- assertEquals("Message should match appended.", messages(i), read.message)
+ assertEquals("Message should match appended.", records(i), read.record)
}
- assertEquals("Reading beyond the last message returns nothing.", 0, log.read(messages.length, 100, None).messageSet.size)
+ assertEquals("Reading beyond the last message returns nothing.", 0, log.read(records.length, 100, None).records.shallowIterator.asScala.size)
}
/**
@@ -211,16 +213,16 @@ class LogTest extends JUnitSuite {
logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer)
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray
- val messages = messageIds.map(id => new Message(id.toString.getBytes))
+ val records = messageIds.map(id => Record.create(id.toString.getBytes))
// now test the case that we give the offsets and use non-sequential offsets
- for(i <- 0 until messages.length)
- log.append(new ByteBufferMessageSet(NoCompressionCodec, new LongRef(messageIds(i)), messages = messages(i)), assignOffsets = false)
+ for(i <- records.indices)
+ log.append(MemoryRecords.withLogEntries(LogEntry.create(messageIds(i), records(i))), assignOffsets = false)
for(i <- 50 until messageIds.max) {
val idx = messageIds.indexWhere(_ >= i)
- val read = log.read(i, 100, None).messageSet.head
+ val read = log.read(i, 100, None).records.shallowIterator.next()
assertEquals("Offset read should match message id.", messageIds(idx), read.offset)
- assertEquals("Message should match appended.", messages(idx), read.message)
+ assertEquals("Message should match appended.", records(idx), read.record)
}
}
@@ -238,12 +240,12 @@ class LogTest extends JUnitSuite {
// keep appending until we have two segments with only a single message in the second segment
while(log.numberOfSegments == 1)
- log.append(new ByteBufferMessageSet(NoCompressionCodec, messages = new Message("42".getBytes)))
+ log.append(MemoryRecords.withRecords(Record.create("42".getBytes)))
// now manually truncate off all but one message from the first segment to create a gap in the messages
log.logSegments.head.truncateTo(1)
- assertEquals("A read should now return the last message in the log", log.logEndOffset - 1, log.read(1, 200, None).messageSet.head.offset)
+ assertEquals("A read should now return the last message in the log", log.logEndOffset - 1, log.read(1, 200, None).records.shallowIterator.next().offset)
}
@Test
@@ -252,12 +254,11 @@ class LogTest extends JUnitSuite {
logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer)
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray
- val messages = messageIds.map(id => new Message(id.toString.getBytes))
+ val records = messageIds.map(id => Record.create(id.toString.getBytes))
// now test the case that we give the offsets and use non-sequential offsets
- for (i <- 0 until messages.length)
- log.append(new ByteBufferMessageSet(NoCompressionCodec, new LongRef(messageIds(i)), messages = messages(i)),
- assignOffsets = false)
+ for (i <- records.indices)
+ log.append(MemoryRecords.withLogEntries(LogEntry.create(messageIds(i), records(i))), assignOffsets = false)
for (i <- 50 until messageIds.max) {
val idx = messageIds.indexWhere(_ >= i)
@@ -265,13 +266,13 @@ class LogTest extends JUnitSuite {
log.read(i, 1, minOneMessage = true),
log.read(i, 100, minOneMessage = true),
log.read(i, 100, Some(10000), minOneMessage = true)
- ).map(_.messageSet.head)
+ ).map(_.records.shallowIterator.next())
reads.foreach { read =>
assertEquals("Offset read should match message id.", messageIds(idx), read.offset)
- assertEquals("Message should match appended.", messages(idx), read.message)
+ assertEquals("Message should match appended.", records(idx), read.record)
}
- assertEquals(Seq.empty, log.read(i, 1, Some(1), minOneMessage = true).messageSet.toIndexedSeq)
+ assertEquals(Seq.empty, log.read(i, 1, Some(1), minOneMessage = true).records.shallowIterator.asScala.toIndexedSeq)
}
}
@@ -282,15 +283,14 @@ class LogTest extends JUnitSuite {
logProps.put(LogConfig.SegmentBytesProp, 71: java.lang.Integer)
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
val messageIds = ((0 until 50) ++ (50 until 200 by 7)).toArray
- val messages = messageIds.map(id => new Message(id.toString.getBytes))
+ val records = messageIds.map(id => Record.create(id.toString.getBytes))
// now test the case that we give the offsets and use non-sequential offsets
- for (i <- 0 until messages.length)
- log.append(new ByteBufferMessageSet(NoCompressionCodec, new LongRef(messageIds(i)), messages = messages(i)),
- assignOffsets = false)
+ for (i <- records.indices)
+ log.append(MemoryRecords.withLogEntries(LogEntry.create(messageIds(i), records(i))), assignOffsets = false)
for (i <- 50 until messageIds.max) {
- assertEquals(MessageSet.Empty, log.read(i, 0).messageSet)
+ assertEquals(MemoryRecords.EMPTY, log.read(i, 0).records)
// we return an incomplete message instead of an empty one for the case below
// we use this mechanism to tell consumers of the fetch request version 2 and below that the message size is
@@ -298,9 +298,9 @@ class LogTest extends JUnitSuite {
// in fetch request version 3, we no longer need this as we return oversized messages from the first non-empty
// partition
val fetchInfo = log.read(i, 1)
- assertTrue(fetchInfo.firstMessageSetIncomplete)
- assertTrue(fetchInfo.messageSet.isInstanceOf[FileMessageSet])
- assertEquals(1, fetchInfo.messageSet.sizeInBytes)
+ assertTrue(fetchInfo.firstEntryIncomplete)
+ assertTrue(fetchInfo.records.isInstanceOf[FileRecords])
+ assertEquals(1, fetchInfo.records.sizeInBytes)
}
}
@@ -318,9 +318,9 @@ class LogTest extends JUnitSuite {
// set up replica log starting with offset 1024 and with one message (at offset 1024)
logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer)
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
- log.append(new ByteBufferMessageSet(NoCompressionCodec, messages = new Message("42".getBytes)))
+ log.append(MemoryRecords.withRecords(Record.create("42".getBytes)))
- assertEquals("Reading at the log end offset should produce 0 byte read.", 0, log.read(1025, 1000).messageSet.sizeInBytes)
+ assertEquals("Reading at the log end offset should produce 0 byte read.", 0, log.read(1025, 1000).records.sizeInBytes)
try {
log.read(0, 1000)
@@ -336,7 +336,7 @@ class LogTest extends JUnitSuite {
case _: OffsetOutOfRangeException => // This is good.
}
- assertEquals("Reading from below the specified maxOffset should produce 0 byte read.", 0, log.read(1025, 1000, Some(1024)).messageSet.sizeInBytes)
+ assertEquals("Reading from below the specified maxOffset should produce 0 byte read.", 0, log.read(1025, 1000, Some(1024)).records.sizeInBytes)
}
/**
@@ -350,21 +350,22 @@ class LogTest extends JUnitSuite {
logProps.put(LogConfig.SegmentBytesProp, 100: java.lang.Integer)
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
val numMessages = 100
- val messageSets = (0 until numMessages).map(i => TestUtils.singleMessageSet(i.toString.getBytes))
+ val messageSets = (0 until numMessages).map(i => TestUtils.singletonRecords(i.toString.getBytes))
messageSets.foreach(log.append(_))
log.flush
/* do successive reads to ensure all our messages are there */
var offset = 0L
for(i <- 0 until numMessages) {
- val messages = log.read(offset, 1024*1024).messageSet
- assertEquals("Offsets not equal", offset, messages.head.offset)
- assertEquals("Messages not equal at offset " + offset, messageSets(i).head.message,
- messages.head.message.toFormatVersion(messageSets(i).head.message.magic))
- offset = messages.head.offset + 1
+ val messages = log.read(offset, 1024*1024).records.shallowIterator
+ val head = messages.next()
+ assertEquals("Offsets not equal", offset, head.offset)
+ assertEquals("Messages not equal at offset " + offset, messageSets(i).shallowIterator.next().record,
+ head.record.convert(messageSets(i).shallowIterator.next().record.magic))
+ offset = head.offset + 1
}
- val lastRead = log.read(startOffset = numMessages, maxLength = 1024*1024, maxOffset = Some(numMessages + 1)).messageSet
- assertEquals("Should be no more messages", 0, lastRead.size)
+ val lastRead = log.read(startOffset = numMessages, maxLength = 1024*1024, maxOffset = Some(numMessages + 1)).records
+ assertEquals("Should be no more messages", 0, lastRead.shallowIterator.asScala.size)
// check that rolling the log forced a flushed the log--the flush is asyn so retry in case of failure
TestUtils.retry(1000L){
@@ -383,10 +384,10 @@ class LogTest extends JUnitSuite {
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
/* append 2 compressed message sets, each with two messages giving offsets 0, 1, 2, 3 */
- log.append(new ByteBufferMessageSet(DefaultCompressionCodec, new Message("hello".getBytes), new Message("there".getBytes)))
- log.append(new ByteBufferMessageSet(DefaultCompressionCodec, new Message("alpha".getBytes), new Message("beta".getBytes)))
+ log.append(MemoryRecords.withRecords(CompressionType.GZIP, Record.create("hello".getBytes), Record.create("there".getBytes)))
+ log.append(MemoryRecords.withRecords(CompressionType.GZIP, Record.create("alpha".getBytes), Record.create("beta".getBytes)))
- def read(offset: Int) = ByteBufferMessageSet.deepIterator(log.read(offset, 4096).messageSet.head)
+ def read(offset: Int) = log.read(offset, 4096).records.deepIterator
/* we should always get the first message in the compressed set when reading any offset in the set */
assertEquals("Read at offset 0 should produce 0", 0, read(0).next().offset)
@@ -408,7 +409,7 @@ class LogTest extends JUnitSuite {
logProps.put(LogConfig.RetentionMsProp, 0: java.lang.Integer)
val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time)
for(i <- 0 until messagesToAppend)
- log.append(TestUtils.singleMessageSet(payload = i.toString.getBytes, timestamp = time.milliseconds - 10))
+ log.append(TestUtils.singletonRecords(value = i.toString.getBytes, timestamp = time.milliseconds - 10))
val currOffset = log.logEndOffset
assertEquals(currOffset, messagesToAppend)
@@ -422,7 +423,7 @@ class LogTest extends JUnitSuite {
assertEquals("Still no change in the logEndOffset", currOffset, log.logEndOffset)
assertEquals("Should still be able to append and should get the logEndOffset assigned to the new append",
currOffset,
- log.append(TestUtils.singleMessageSet("hello".getBytes)).firstOffset)
+ log.append(TestUtils.singletonRecords("hello".getBytes)).firstOffset)
// cleanup the log
log.delete()
@@ -435,7 +436,7 @@ class LogTest extends JUnitSuite {
*/
@Test
def testMessageSetSizeCheck() {
- val messageSet = new ByteBufferMessageSet(NoCompressionCodec, new Message ("You".getBytes), new Message("bethe".getBytes))
+ val messageSet = MemoryRecords.withRecords(Record.create("You".getBytes), Record.create("bethe".getBytes))
// append messages to log
val configSegmentSize = messageSet.sizeInBytes - 1
val logProps = new Properties()
@@ -454,17 +455,17 @@ class LogTest extends JUnitSuite {
@Test
def testCompactedTopicConstraints() {
- val keyedMessage = new Message(bytes = "this message has a key".getBytes, key = "and here it is".getBytes, Message.NoTimestamp, Message.CurrentMagicValue)
- val anotherKeyedMessage = new Message(bytes = "this message also has a key".getBytes, key ="another key".getBytes, Message.NoTimestamp, Message.CurrentMagicValue)
- val unkeyedMessage = new Message(bytes = "this message does not have a key".getBytes)
+ val keyedMessage = Record.create(Record.CURRENT_MAGIC_VALUE, Record.NO_TIMESTAMP, "and here it is".getBytes, "this message has a key".getBytes)
+ val anotherKeyedMessage = Record.create(Record.CURRENT_MAGIC_VALUE, Record.NO_TIMESTAMP, "another key".getBytes, "this message also has a key".getBytes)
+ val unkeyedMessage = Record.create("this message does not have a key".getBytes)
- val messageSetWithUnkeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, unkeyedMessage, keyedMessage)
- val messageSetWithOneUnkeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, unkeyedMessage)
- val messageSetWithCompressedKeyedMessage = new ByteBufferMessageSet(GZIPCompressionCodec, keyedMessage)
- val messageSetWithCompressedUnkeyedMessage = new ByteBufferMessageSet(GZIPCompressionCodec, keyedMessage, unkeyedMessage)
+ val messageSetWithUnkeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, unkeyedMessage, keyedMessage)
+ val messageSetWithOneUnkeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, unkeyedMessage)
+ val messageSetWithCompressedKeyedMessage = MemoryRecords.withRecords(CompressionType.GZIP, keyedMessage)
+ val messageSetWithCompressedUnkeyedMessage = MemoryRecords.withRecords(CompressionType.GZIP, keyedMessage, unkeyedMessage)
- val messageSetWithKeyedMessage = new ByteBufferMessageSet(NoCompressionCodec, keyedMessage)
- val messageSetWithKeyedMessages = new ByteBufferMessageSet(NoCompressionCodec, keyedMessage, anotherKeyedMessage)
+ val messageSetWithKeyedMessage = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage)
+ val messageSetWithKeyedMessages = MemoryRecords.withRecords(CompressionType.NONE, keyedMessage, anotherKeyedMessage)
val logProps = new Properties()
logProps.put(LogConfig.CleanupPolicyProp, LogConfig.Compact)
@@ -502,8 +503,8 @@ class LogTest extends JUnitSuite {
*/
@Test
def testMessageSizeCheck() {
- val first = new ByteBufferMessageSet(NoCompressionCodec, new Message ("You".getBytes), new Message("bethe".getBytes))
- val second = new ByteBufferMessageSet(NoCompressionCodec, new Message("change (I need more bytes)".getBytes))
+ val first = MemoryRecords.withRecords(CompressionType.NONE, Record.create("You".getBytes), Record.create("bethe".getBytes))
+ val second = MemoryRecords.withRecords(CompressionType.NONE, Record.create("change (I need more bytes)".getBytes))
// append messages to log
val maxMessageSize = second.sizeInBytes - 1
@@ -537,7 +538,7 @@ class LogTest extends JUnitSuite {
val config = LogConfig(logProps)
var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
for(i <- 0 until numMessages)
- log.append(TestUtils.singleMessageSet(payload = TestUtils.randomBytes(messageSize),
+ log.append(TestUtils.singletonRecords(value = TestUtils.randomBytes(messageSize),
timestamp = time.milliseconds + i * 10))
assertEquals("After appending %d messages to an empty log, the log end offset should be %d".format(numMessages, numMessages), numMessages, log.logEndOffset)
val lastIndexOffset = log.activeSegment.index.lastOffset
@@ -585,7 +586,7 @@ class LogTest extends JUnitSuite {
val log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
val messages = (0 until numMessages).map { i =>
- new ByteBufferMessageSet(NoCompressionCodec, new LongRef(100 + i), new Message(i.toString.getBytes(), time.milliseconds + i, Message.MagicValue_V1))
+ MemoryRecords.withLogEntries(LogEntry.create(100 + i, Record.create(Record.MAGIC_VALUE_V1, time.milliseconds + i, i.toString.getBytes())))
}
messages.foreach(log.append(_, assignOffsets = false))
val timeIndexEntries = log.logSegments.foldLeft(0) { (entries, segment) => entries + segment.timeIndex.entries }
@@ -608,7 +609,7 @@ class LogTest extends JUnitSuite {
val config = LogConfig(logProps)
var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
for(i <- 0 until numMessages)
- log.append(TestUtils.singleMessageSet(payload = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10))
+ log.append(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10))
val indexFiles = log.logSegments.map(_.index.file)
val timeIndexFiles = log.logSegments.map(_.timeIndex.file)
log.close()
@@ -623,7 +624,7 @@ class LogTest extends JUnitSuite {
assertTrue("The index should have been rebuilt", log.logSegments.head.index.entries > 0)
assertTrue("The time index should have been rebuilt", log.logSegments.head.timeIndex.entries > 0)
for(i <- 0 until numMessages) {
- assertEquals(i, log.read(i, 100, None).messageSet.head.offset)
+ assertEquals(i, log.read(i, 100, None).records.shallowIterator.next().offset)
if (i == 0)
assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetsByTimestamp(time.milliseconds + i * 10).get.offset)
else
@@ -647,7 +648,7 @@ class LogTest extends JUnitSuite {
val config = LogConfig(logProps)
var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
for(i <- 0 until numMessages)
- log.append(TestUtils.singleMessageSet(payload = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10))
+ log.append(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10))
val timeIndexFiles = log.logSegments.map(_.timeIndex.file)
log.close()
@@ -676,7 +677,7 @@ class LogTest extends JUnitSuite {
val config = LogConfig(logProps)
var log = new Log(logDir, config, recoveryPoint = 0L, time.scheduler, time)
for(i <- 0 until numMessages)
- log.append(TestUtils.singleMessageSet(payload = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10))
+ log.append(TestUtils.singletonRecords(value = TestUtils.randomBytes(10), timestamp = time.milliseconds + i * 10))
val indexFiles = log.logSegments.map(_.index.file)
val timeIndexFiles = log.logSegments.map(_.timeIndex.file)
log.close()
@@ -699,7 +700,7 @@ class LogTest extends JUnitSuite {
log = new Log(logDir, config, recoveryPoint = 200L, time.scheduler, time)
assertEquals("Should have %d messages when log is reopened".format(numMessages), numMessages, log.logEndOffset)
for(i <- 0 until numMessages) {
- assertEquals(i, log.read(i, 100, None).messageSet.head.offset)
+ assertEquals(i, log.read(i, 100, None).records.shallowIterator.next().offset)
if (i == 0)
assertEquals(log.logSegments.head.baseOffset, log.fetchOffsetsByTimestamp(time.milliseconds + i * 10).get.offset)
else
@@ -713,7 +714,7 @@ class LogTest extends JUnitSuite {
*/
@Test
def testTruncateTo() {
- val set = TestUtils.singleMessageSet("test".getBytes)
+ val set = TestUtils.singletonRecords("test".getBytes)
val setSize = set.sizeInBytes
val msgPerSeg = 10
val segmentSize = msgPerSeg * setSize // each segment will be 10 messages
@@ -770,7 +771,7 @@ class LogTest extends JUnitSuite {
*/
@Test
def testIndexResizingAtTruncation() {
- val setSize = TestUtils.singleMessageSet(payload = "test".getBytes).sizeInBytes
+ val setSize = TestUtils.singletonRecords(value = "test".getBytes).sizeInBytes
val msgPerSeg = 10
val segmentSize = msgPerSeg * setSize // each segment will be 10 messages
val logProps = new Properties()
@@ -781,12 +782,12 @@ class LogTest extends JUnitSuite {
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
for (i<- 1 to msgPerSeg)
- log.append(TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds + i))
+ log.append(TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds + i))
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
time.sleep(msgPerSeg)
for (i<- 1 to msgPerSeg)
- log.append(TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds + i))
+ log.append(TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds + i))
assertEquals("There should be exactly 2 segment.", 2, log.numberOfSegments)
val expectedEntries = msgPerSeg - 1
@@ -800,7 +801,7 @@ class LogTest extends JUnitSuite {
time.sleep(msgPerSeg)
for (i<- 1 to msgPerSeg)
- log.append(TestUtils.singleMessageSet(payload = "test".getBytes, timestamp = time.milliseconds + i))
+ log.append(TestUtils.singletonRecords(value = "test".getBytes, timestamp = time.milliseconds + i))
assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments)
}
@@ -814,7 +815,7 @@ class LogTest extends JUnitSuite {
val bogusIndex2 = Log.indexFilename(logDir, 5)
val bogusTimeIndex2 = Log.timeIndexFilename(logDir, 5)
- val set = TestUtils.singleMessageSet("test".getBytes)
+ val set = TestUtils.singletonRecords("test".getBytes)
val logProps = new Properties()
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
@@ -842,7 +843,7 @@ class LogTest extends JUnitSuite {
*/
@Test
def testReopenThenTruncate() {
- val set = TestUtils.singleMessageSet("test".getBytes)
+ val set = TestUtils.singletonRecords("test".getBytes)
val logProps = new Properties()
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
@@ -875,7 +876,7 @@ class LogTest extends JUnitSuite {
*/
@Test
def testAsyncDelete() {
- val set = TestUtils.singleMessageSet("test".getBytes)
+ val set = TestUtils.singletonRecords("test".getBytes)
val asyncDeleteMs = 1000
val logProps = new Properties()
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
@@ -921,7 +922,7 @@ class LogTest extends JUnitSuite {
*/
@Test
def testOpenDeletesObsoleteFiles() {
- val set = TestUtils.singleMessageSet("test".getBytes)
+ val set = TestUtils.singletonRecords("test".getBytes)
val logProps = new Properties()
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
@@ -957,10 +958,10 @@ class LogTest extends JUnitSuite {
recoveryPoint = 0L,
time.scheduler,
time)
- log.append(new ByteBufferMessageSet(new Message(bytes = null)))
- val messageSet = log.read(0, 4096, None).messageSet
- assertEquals(0, messageSet.head.offset)
- assertTrue("Message payload should be null.", messageSet.head.message.isNull)
+ log.append(MemoryRecords.withRecords(Record.create(null)))
+ val head = log.read(0, 4096, None).records.shallowIterator().next()
+ assertEquals(0, head.offset)
+ assertTrue("Message payload should be null.", head.record.hasNullValue)
}
@Test(expected = classOf[IllegalArgumentException])
@@ -970,9 +971,9 @@ class LogTest extends JUnitSuite {
recoveryPoint = 0L,
time.scheduler,
time)
- val messages = (0 until 2).map(id => new Message(id.toString.getBytes)).toArray
- messages.foreach(message => log.append(new ByteBufferMessageSet(message)))
- val invalidMessage = new ByteBufferMessageSet(new Message(1.toString.getBytes))
+ val messages = (0 until 2).map(id => Record.create(id.toString.getBytes)).toArray
+ messages.foreach(record => log.append(MemoryRecords.withRecords(record)))
+ val invalidMessage = MemoryRecords.withRecords(Record.create(1.toString.getBytes))
log.append(invalidMessage, assignOffsets = false)
}
@@ -984,7 +985,7 @@ class LogTest extends JUnitSuite {
logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
logProps.put(LogConfig.MaxMessageBytesProp, 64*1024: java.lang.Integer)
val config = LogConfig(logProps)
- val set = TestUtils.singleMessageSet("test".getBytes)
+ val set = TestUtils.singletonRecords("test".getBytes)
val recoveryPoint = 50L
for (_ <- 0 until 50) {
// create a log and write some messages to it
@@ -997,7 +998,7 @@ class LogTest extends JUnitSuite {
val numMessages = 50 + TestUtils.random.nextInt(50)
for (_ <- 0 until numMessages)
log.append(set)
- val messages = log.logSegments.flatMap(_.log.iterator.toList)
+ val messages = log.logSegments.flatMap(_.log.deepIterator.asScala.toList)
log.close()
// corrupt index and log by appending random bytes
@@ -1007,7 +1008,8 @@ class LogTest extends JUnitSuite {
// attempt recovery
log = new Log(logDir, config, recoveryPoint, time.scheduler, time)
assertEquals(numMessages, log.logEndOffset)
- assertEquals("Messages in the log after recovery should be the same.", messages, log.logSegments.flatMap(_.log.iterator.toList))
+ assertEquals("Messages in the log after recovery should be the same.", messages,
+ log.logSegments.flatMap(_.log.deepIterator.asScala.toList))
Utils.delete(logDir)
}
}
@@ -1020,7 +1022,7 @@ class LogTest extends JUnitSuite {
logProps.put(LogConfig.MaxMessageBytesProp, 64*1024: java.lang.Integer)
logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
val config = LogConfig(logProps)
- val set = TestUtils.singleMessageSet("test".getBytes)
+ val set = TestUtils.singletonRecords("test".getBytes)
val parentLogDir = logDir.getParentFile
assertTrue("Data directory %s must exist", parentLogDir.isDirectory)
val cleanShutdownFile = new File(parentLogDir, Log.CleanShutdownFile)
@@ -1121,7 +1123,7 @@ class LogTest extends JUnitSuite {
@Test
def testDeleteOldSegmentsMethod() {
- val set = TestUtils.singleMessageSet("test".getBytes)
+ val set = TestUtils.singletonRecords("test".getBytes)
val logProps = new Properties()
logProps.put(LogConfig.SegmentBytesProp, set.sizeInBytes * 5: java.lang.Integer)
logProps.put(LogConfig.SegmentIndexBytesProp, 1000: java.lang.Integer)
@@ -1154,7 +1156,7 @@ class LogTest extends JUnitSuite {
@Test
def shouldDeleteSizeBasedSegments() {
- val set = TestUtils.singleMessageSet("test".getBytes)
+ val set = TestUtils.singletonRecords("test".getBytes)
val log = createLog(set.sizeInBytes, retentionBytes = set.sizeInBytes * 10)
// append some messages to create some segments
@@ -1167,7 +1169,7 @@ class LogTest extends JUnitSuite {
@Test
def shouldNotDeleteSizeBasedSegmentsWhenUnderRetentionSize() {
- val set = TestUtils.singleMessageSet("test".getBytes)
+ val set = TestUtils.singletonRecords("test".getBytes)
val log = createLog(set.sizeInBytes, retentionBytes = set.sizeInBytes * 15)
// append some messages to create some segments
@@ -1180,7 +1182,7 @@ class LogTest extends JUnitSuite {
@Test
def shouldDeleteTimeBasedSegmentsReadyToBeDeleted() {
- val set = TestUtils.singleMessageSet("test".getBytes, timestamp = 10)
+ val set = TestUtils.singletonRecords("test".getBytes, timestamp = 10)
val log = createLog(set.sizeInBytes, retentionMs = 10000)
// append some messages to create some segments
@@ -1193,7 +1195,7 @@ class LogTest extends JUnitSuite {
@Test
def shouldNotDeleteTimeBasedSegmentsWhenNoneReadyToBeDeleted() {
- val set = TestUtils.singleMessageSet("test".getBytes, timestamp = time.milliseconds)
+ val set = TestUtils.singletonRecords("test".getBytes, timestamp = time.milliseconds)
val log = createLog(set.sizeInBytes, retentionMs = 10000000)
// append some messages to create some segments
@@ -1206,7 +1208,7 @@ class LogTest extends JUnitSuite {
@Test
def shouldNotDeleteSegmentsWhenPolicyDoesNotIncludeDelete() {
- val set = TestUtils.singleMessageSet("test".getBytes, key = "test".getBytes(), timestamp = 10L)
+ val set = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes(), timestamp = 10L)
val log = createLog(set.sizeInBytes,
retentionMs = 10000,
cleanupPolicy = "compact")
@@ -1225,7 +1227,7 @@ class LogTest extends JUnitSuite {
@Test
def shouldDeleteSegmentsReadyToBeDeletedWhenCleanupPolicyIsCompactAndDelete() {
- val set = TestUtils.singleMessageSet("test".getBytes, key = "test".getBytes,timestamp = 10L)
+ val set = TestUtils.singletonRecords("test".getBytes, key = "test".getBytes,timestamp = 10L)
val log = createLog(set.sizeInBytes,
retentionMs = 10000,
cleanupPolicy = "compact,delete")
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala
new file mode 100644
index 0000000..72c5b16
--- /dev/null
+++ b/core/src/test/scala/unit/kafka/log/LogValidatorTest.scala
@@ -0,0 +1,395 @@
+/**
+ * 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.log
+
+import java.nio.ByteBuffer
+
+import kafka.common.LongRef
+import kafka.message._
+import org.apache.kafka.common.errors.InvalidTimestampException
+import org.apache.kafka.common.record._
+import org.junit.Assert._
+import org.junit.Test
+import org.scalatest.junit.JUnitSuite
+
+import scala.collection.JavaConverters._
+
+class LogValidatorTest extends JUnitSuite {
+
+ @Test
+ def testLogAppendTimeNonCompressed() {
+ val now = System.currentTimeMillis()
+ // The timestamps should be overwritten
+ val records = createRecords(magicValue = Record.MAGIC_VALUE_V1, timestamp = 0L, codec = CompressionType.NONE)
+ val validatedResults = LogValidator.validateMessagesAndAssignOffsets(records,
+ offsetCounter = new LongRef(0),
+ now = now,
+ sourceCodec = NoCompressionCodec,
+ targetCodec = NoCompressionCodec,
+ messageFormatVersion = Record.MAGIC_VALUE_V1,
+ messageTimestampType = TimestampType.LOG_APPEND_TIME,
+ messageTimestampDiffMaxMs = 1000L)
+ val validatedRecords = validatedResults.validatedRecords
+ assertEquals("number of messages should not change", records.deepIterator.asScala.size, validatedRecords.deepIterator.asScala.size)
+ validatedRecords.deepIterator.asScala.foreach(logEntry => validateLogAppendTime(now, logEntry.record))
+ assertEquals(s"Max timestamp should be $now", now, validatedResults.maxTimestamp)
+ assertEquals(s"The offset of max timestamp should be 0", 0, validatedResults.shallowOffsetOfMaxTimestamp)
+ assertFalse("Message size should not have been changed", validatedResults.messageSizeMaybeChanged)
+ }
+
+ @Test
+ def testLogAppendTimeWithRecompression() {
+ val now = System.currentTimeMillis()
+ // The timestamps should be overwritten
+ val records = createRecords(magicValue = Record.MAGIC_VALUE_V0, codec = CompressionType.GZIP)
+ val validatedResults = LogValidator.validateMessagesAndAssignOffsets(
+ records,
+ offsetCounter = new LongRef(0),
+ now = now,
+ sourceCodec = DefaultCompressionCodec,
+ targetCodec = DefaultCompressionCodec,
+ messageFormatVersion = Record.MAGIC_VALUE_V1,
+ messageTimestampType = TimestampType.LOG_APPEND_TIME,
+ messageTimestampDiffMaxMs = 1000L)
+ val validatedRecords = validatedResults.validatedRecords
+
+ assertEquals("number of messages should not change", records.deepIterator.asScala.size, validatedRecords.deepIterator.asScala.size)
+ validatedRecords.deepIterator.asScala.foreach(logEntry => validateLogAppendTime(now, logEntry.record))
+ assertTrue("MessageSet should still valid", validatedRecords.shallowIterator.next().record.isValid)
+ assertEquals(s"Max timestamp should be $now", now, validatedResults.maxTimestamp)
+ assertEquals(s"The offset of max timestamp should be ${records.deepIterator.asScala.size - 1}",
+ records.deepIterator.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestamp)
+ assertTrue("Message size may have been changed", validatedResults.messageSizeMaybeChanged)
+ }
+
+ @Test
+ def testLogAppendTimeWithoutRecompression() {
+ val now = System.currentTimeMillis()
+ // The timestamps should be overwritten
+ val records = createRecords(magicValue = Record.MAGIC_VALUE_V1,
+ timestamp = 0L, codec = CompressionType.GZIP)
+ val validatedResults = LogValidator.validateMessagesAndAssignOffsets(
+ records,
+ offsetCounter = new LongRef(0),
+ now = now,
+ sourceCodec = DefaultCompressionCodec,
+ targetCodec = DefaultCompressionCodec,
+ messageFormatVersion = Record.MAGIC_VALUE_V1,
+ messageTimestampType = TimestampType.LOG_APPEND_TIME,
+ messageTimestampDiffMaxMs = 1000L)
+ val validatedRecords = validatedResults.validatedRecords
+
+ assertEquals("number of messages should not change", records.deepIterator.asScala.size,
+ validatedRecords.deepIterator.asScala.size)
+ validatedRecords.deepIterator.asScala.foreach(logEntry => validateLogAppendTime(now, logEntry.record))
+ assertTrue("MessageSet should still valid", validatedRecords.shallowIterator.next().record.isValid)
+ assertEquals(s"Max timestamp should be $now", now, validatedResults.maxTimestamp)
+ assertEquals(s"The offset of max timestamp should be ${records.deepIterator.asScala.size - 1}",
+ records.deepIterator.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestamp)
+ assertFalse("Message size should not have been changed", validatedResults.messageSizeMaybeChanged)
+ }
+
+ @Test
+ def testCreateTimeNonCompressed() {
+ val now = System.currentTimeMillis()
+ val timestampSeq = Seq(now - 1, now + 1, now)
+ val records =
+ MemoryRecords.withRecords(CompressionType.NONE,
+ Record.create(Record.MAGIC_VALUE_V1, timestampSeq(0), "hello".getBytes),
+ Record.create(Record.MAGIC_VALUE_V1, timestampSeq(1), "there".getBytes),
+ Record.create(Record.MAGIC_VALUE_V1, timestampSeq(2), "beautiful".getBytes))
+
+ val validatingResults = LogValidator.validateMessagesAndAssignOffsets(records,
+ offsetCounter = new LongRef(0),
+ now = System.currentTimeMillis(),
+ sourceCodec = NoCompressionCodec,
+ targetCodec = NoCompressionCodec,
+ messageFormatVersion = Record.MAGIC_VALUE_V1,
+ messageTimestampType = TimestampType.CREATE_TIME,
+ messageTimestampDiffMaxMs = 1000L)
+ val validatedRecords = validatingResults.validatedRecords
+
+ var i = 0
+ for (logEntry <- validatedRecords.deepIterator.asScala) {
+ logEntry.record.ensureValid()
+ assertEquals(logEntry.record.timestamp, timestampSeq(i))
+ assertEquals(logEntry.record.timestampType, TimestampType.CREATE_TIME)
+ i += 1
+ }
+ assertEquals(s"Max timestamp should be ${now + 1}", now + 1, validatingResults.maxTimestamp)
+ assertEquals(s"Offset of max timestamp should be 1", 1, validatingResults.shallowOffsetOfMaxTimestamp)
+ assertFalse("Message size should not have been changed", validatingResults.messageSizeMaybeChanged)
+ }
+
+ @Test
+ def testCreateTimeCompressed() {
+ val now = System.currentTimeMillis()
+ val timestampSeq = Seq(now - 1, now + 1, now)
+ val records =
+ MemoryRecords.withRecords(CompressionType.GZIP,
+ Record.create(Record.MAGIC_VALUE_V1, timestampSeq(0), "hello".getBytes),
+ Record.create(Record.MAGIC_VALUE_V1, timestampSeq(1), "there".getBytes),
+ Record.create(Record.MAGIC_VALUE_V1, timestampSeq(2), "beautiful".getBytes))
+
+ val validatedResults =
+ LogValidator.validateMessagesAndAssignOffsets(records,
+ offsetCounter = new LongRef(0),
+ now = System.currentTimeMillis(),
+ sourceCodec = DefaultCompressionCodec,
+ targetCodec = DefaultCompressionCodec,
+ messageFormatVersion = Record.MAGIC_VALUE_V1,
+ messageTimestampType = TimestampType.CREATE_TIME,
+ messageTimestampDiffMaxMs = 1000L)
+ val validatedRecords = validatedResults.validatedRecords
+
+ var i = 0
+ for (logEntry <- validatedRecords.deepIterator.asScala) {
+ logEntry.record.ensureValid()
+ assertEquals(logEntry.record.timestamp, timestampSeq(i))
+ assertEquals(logEntry.record.timestampType, TimestampType.CREATE_TIME)
+ i += 1
+ }
+ assertEquals(s"Max timestamp should be ${now + 1}", now + 1, validatedResults.maxTimestamp)
+ assertEquals(s"Offset of max timestamp should be ${validatedRecords.deepIterator.asScala.size - 1}",
+ validatedRecords.deepIterator.asScala.size - 1, validatedResults.shallowOffsetOfMaxTimestamp)
+ assertFalse("Message size should not have been changed", validatedResults.messageSizeMaybeChanged)
+ }
+
+ @Test(expected = classOf[InvalidTimestampException])
+ def testInvalidCreateTimeNonCompressed() {
+ val now = System.currentTimeMillis()
+ val records = createRecords(magicValue = Record.MAGIC_VALUE_V1, timestamp = now - 1001L,
+ codec = CompressionType.NONE)
+ LogValidator.validateMessagesAndAssignOffsets(
+ records,
+ offsetCounter = new LongRef(0),
+ now = System.currentTimeMillis(),
+ sourceCodec = NoCompressionCodec,
+ targetCodec = NoCompressionCodec,
+ messageFormatVersion = Record.MAGIC_VALUE_V1,
+ messageTimestampType = TimestampType.CREATE_TIME,
+ messageTimestampDiffMaxMs = 1000L)
+ }
+
+ @Test(expected = classOf[InvalidTimestampException])
+ def testInvalidCreateTimeCompressed() {
+ val now = System.currentTimeMillis()
+ val records = createRecords(magicValue = Record.MAGIC_VALUE_V1, timestamp = now - 1001L,
+ codec = CompressionType.GZIP)
+ LogValidator.validateMessagesAndAssignOffsets(
+ records,
+ offsetCounter = new LongRef(0),
+ now = System.currentTimeMillis(),
+ sourceCodec = DefaultCompressionCodec,
+ targetCodec = DefaultCompressionCodec,
+ messageFormatVersion = Record.MAGIC_VALUE_V1,
+ messageTimestampType = TimestampType.CREATE_TIME,
+ messageTimestampDiffMaxMs = 1000L)
+ }
+ @Test
+ def testAbsoluteOffsetAssignmentNonCompressed() {
+ val records = createRecords(magicValue = Record.MAGIC_VALUE_V0, codec = CompressionType.NONE)
+ val offset = 1234567
+ checkOffsets(records, 0)
+ checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records,
+ offsetCounter = new LongRef(offset),
+ now = System.currentTimeMillis(),
+ sourceCodec = NoCompressionCodec,
+ targetCodec = NoCompressionCodec,
+ messageFormatVersion = Record.MAGIC_VALUE_V0,
+ messageTimestampType = TimestampType.CREATE_TIME,
+ messageTimestampDiffMaxMs = 1000L).validatedRecords, offset)
+ }
+
+ @Test
+ def testAbsoluteOffsetAssignmentCompressed() {
+ val records = createRecords(magicValue = Record.MAGIC_VALUE_V0, codec = CompressionType.GZIP)
+ val offset = 1234567
+ checkOffsets(records, 0)
+ checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records,
+ offsetCounter = new LongRef(offset),
+ now = System.currentTimeMillis(),
+ sourceCodec = DefaultCompressionCodec,
+ targetCodec = DefaultCompressionCodec,
+ messageFormatVersion = Record.MAGIC_VALUE_V0,
+ messageTimestampType = TimestampType.CREATE_TIME,
+ messageTimestampDiffMaxMs = 1000L).validatedRecords, offset)
+ }
+
+ @Test
+ def testRelativeOffsetAssignmentNonCompressed() {
+ val now = System.currentTimeMillis()
+ val records = createRecords(magicValue = Record.MAGIC_VALUE_V1, timestamp = now, codec = CompressionType.NONE)
+ val offset = 1234567
+ checkOffsets(records, 0)
+ val messageWithOffset = LogValidator.validateMessagesAndAssignOffsets(records,
+ offsetCounter = new LongRef(offset),
+ now = System.currentTimeMillis(),
+ sourceCodec = NoCompressionCodec,
+ targetCodec = NoCompressionCodec,
+ messageTimestampType = TimestampType.CREATE_TIME,
+ messageTimestampDiffMaxMs = 5000L).validatedRecords
+ checkOffsets(messageWithOffset, offset)
+ }
+
+ @Test
+ def testRelativeOffsetAssignmentCompressed() {
+ val now = System.currentTimeMillis()
+ val records = createRecords(magicValue = Record.MAGIC_VALUE_V1, timestamp = now, codec = CompressionType.GZIP)
+ val offset = 1234567
+ checkOffsets(records, 0)
+ val compressedMessagesWithOffset = LogValidator.validateMessagesAndAssignOffsets(
+ records,
+ offsetCounter = new LongRef(offset),
+ now = System.currentTimeMillis(),
+ sourceCodec = DefaultCompressionCodec,
+ targetCodec = DefaultCompressionCodec,
+ messageTimestampType = TimestampType.CREATE_TIME,
+ messageTimestampDiffMaxMs = 5000L).validatedRecords
+ checkOffsets(compressedMessagesWithOffset, offset)
+ }
+
+ @Test
+ def testOffsetAssignmentAfterMessageFormatConversionV0NonCompressed() {
+ val records = createRecords(magicValue = Record.MAGIC_VALUE_V0, codec = CompressionType.NONE)
+ checkOffsets(records, 0)
+ val offset = 1234567
+ checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records,
+ offsetCounter = new LongRef(offset),
+ now = System.currentTimeMillis(),
+ sourceCodec = NoCompressionCodec,
+ targetCodec = NoCompressionCodec,
+ messageFormatVersion = Record.MAGIC_VALUE_V1,
+ messageTimestampType = TimestampType.LOG_APPEND_TIME,
+ messageTimestampDiffMaxMs = 1000L).validatedRecords, offset)
+ }
+
+ @Test
+ def testOffsetAssignmentAfterMessageFormatConversionV0Compressed() {
+ val records = createRecords(magicValue = Record.MAGIC_VALUE_V0, codec = CompressionType.GZIP)
+ val offset = 1234567
+ checkOffsets(records, 0)
+ checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records,
+ offsetCounter = new LongRef(offset),
+ now = System.currentTimeMillis(),
+ sourceCodec = DefaultCompressionCodec,
+ targetCodec = DefaultCompressionCodec,
+ messageFormatVersion = Record.MAGIC_VALUE_V1,
+ messageTimestampType = TimestampType.LOG_APPEND_TIME,
+ messageTimestampDiffMaxMs = 1000L).validatedRecords, offset)
+ }
+
+ @Test
+ def testOffsetAssignmentAfterMessageFormatConversionV1NonCompressed() {
+ val offset = 1234567
+ val now = System.currentTimeMillis()
+ val records = createRecords(Record.MAGIC_VALUE_V1, now, codec = CompressionType.NONE)
+ checkOffsets(records, 0)
+ checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records,
+ offsetCounter = new LongRef(offset),
+ now = System.currentTimeMillis(),
+ sourceCodec = NoCompressionCodec,
+ targetCodec = NoCompressionCodec,
+ messageFormatVersion = Record.MAGIC_VALUE_V0,
+ messageTimestampType = TimestampType.CREATE_TIME,
+ messageTimestampDiffMaxMs = 5000L).validatedRecords, offset)
+ }
+
+ @Test
+ def testOffsetAssignmentAfterMessageFormatConversionV1Compressed() {
+ val offset = 1234567
+ val now = System.currentTimeMillis()
+ val records = createRecords(Record.MAGIC_VALUE_V1, now, CompressionType.GZIP)
+ checkOffsets(records, 0)
+ checkOffsets(LogValidator.validateMessagesAndAssignOffsets(records,
+ offsetCounter = new LongRef(offset),
+ now = System.currentTimeMillis(),
+ sourceCodec = DefaultCompressionCodec,
+ targetCodec = DefaultCompressionCodec,
+ messageFormatVersion = Record.MAGIC_VALUE_V0,
+ messageTimestampType = TimestampType.CREATE_TIME,
+ messageTimestampDiffMaxMs = 5000L).validatedRecords, offset)
+ }
+
+ @Test(expected = classOf[InvalidRecordException])
+ def testInvalidInnerMagicVersion(): Unit = {
+ val offset = 1234567
+ val records = recordsWithInvalidInnerMagic(offset)
+ LogValidator.validateMessagesAndAssignOffsets(records,
+ offsetCounter = new LongRef(offset),
+ now = System.currentTimeMillis(),
+ sourceCodec = SnappyCompressionCodec,
+ targetCodec = SnappyCompressionCodec,
+ messageTimestampType = TimestampType.CREATE_TIME,
+ messageTimestampDiffMaxMs = 5000L)
+ }
+
+ private def createRecords(magicValue: Byte = Message.CurrentMagicValue,
+ timestamp: Long = Message.NoTimestamp,
+ codec: CompressionType = CompressionType.NONE): MemoryRecords = {
+ if (magicValue == Record.MAGIC_VALUE_V0) {
+ MemoryRecords.withRecords(
+ codec,
+ Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, "hello".getBytes),
+ Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, "there".getBytes),
+ Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, "beautiful".getBytes))
+ } else {
+ MemoryRecords.withRecords(
+ codec,
+ Record.create(Record.MAGIC_VALUE_V1, timestamp, "hello".getBytes),
+ Record.create(Record.MAGIC_VALUE_V1, timestamp, "there".getBytes),
+ Record.create(Record.MAGIC_VALUE_V1, timestamp, "beautiful".getBytes))
+ }
+ }
+
+ /* check that offsets are assigned consecutively from the given base offset */
+ private def checkOffsets(records: MemoryRecords, baseOffset: Long) {
+ assertTrue("Message set should not be empty", records.deepIterator.asScala.nonEmpty)
+ var offset = baseOffset
+ for (entry <- records.deepIterator.asScala) {
+ assertEquals("Unexpected offset in message set iterator", offset, entry.offset)
+ offset += 1
+ }
+ }
+
+ private def recordsWithInvalidInnerMagic(initialOffset: Long): MemoryRecords = {
+ val records = (0 until 20).map(id =>
+ Record.create(Record.MAGIC_VALUE_V0,
+ Record.NO_TIMESTAMP,
+ id.toString.getBytes,
+ id.toString.getBytes))
+
+ val buffer = ByteBuffer.allocate(math.min(math.max(records.map(_.sizeInBytes()).sum / 2, 1024), 1 << 16))
+ val builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V1, CompressionType.GZIP,
+ TimestampType.CREATE_TIME)
+
+ var offset = initialOffset
+ records.foreach { record =>
+ builder.appendUnchecked(offset, record)
+ offset += 1
+ }
+
+ builder.build()
+ }
+
+ def validateLogAppendTime(now: Long, record: Record) {
+ record.ensureValid()
+ assertEquals(s"Timestamp of message $record should be $now", now, record.timestamp)
+ assertEquals(TimestampType.LOG_APPEND_TIME, record.timestampType)
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala b/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala
index 476a577..bd3ed68 100644
--- a/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala
+++ b/core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala
@@ -18,13 +18,11 @@
package kafka.message
import java.nio.ByteBuffer
-import java.nio.channels.{FileChannel, GatheringByteChannel}
-import java.nio.file.StandardOpenOption
+import java.nio.channels.GatheringByteChannel
import org.junit.Assert._
import kafka.utils.TestUtils._
-import kafka.log.FileMessageSet
-import kafka.utils.TestUtils
+import org.apache.kafka.common.record.FileRecords
import org.scalatest.junit.JUnitSuite
import org.junit.Test
@@ -94,7 +92,7 @@ trait BaseMessageSetTestCases extends JUnitSuite {
@Test
def testWriteToChannelThatConsumesPartially() {
val bytesToConsumePerBuffer = 50
- val messages = (0 until 10).map(_ => new Message(TestUtils.randomString(100).getBytes))
+ val messages = (0 until 10).map(_ => new Message(randomString(100).getBytes))
val messageSet = createMessageSet(messages)
val messageSetSize = messageSet.sizeInBytes
@@ -119,15 +117,15 @@ trait BaseMessageSetTestCases extends JUnitSuite {
// do the write twice to ensure the message set is restored to its original state
for (_ <- 0 to 1) {
val file = tempFile()
- val channel = FileChannel.open(file.toPath, StandardOpenOption.READ, StandardOpenOption.WRITE)
+ val fileRecords = FileRecords.open(file, true)
try {
- val written = write(channel)
+ val written = write(fileRecords.channel)
+ fileRecords.resize() // resize since we wrote to the channel directly
+
assertEquals("Expect to write the number of bytes in the set.", set.sizeInBytes, written)
- val newSet = new FileMessageSet(file, channel)
- checkEquals(set.iterator, newSet.iterator)
- } finally channel.close()
+ checkEquals(set.asRecords.deepIterator, fileRecords.deepIterator())
+ } finally fileRecords.close()
}
}
}
-
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala
index e2cfb87..5e22433 100644
--- a/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala
+++ b/core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala
@@ -17,13 +17,9 @@
package kafka.message
-import java.io.DataOutputStream
import java.nio._
-import kafka.common.LongRef
import kafka.utils.TestUtils
-import org.apache.kafka.common.errors.InvalidTimestampException
-import org.apache.kafka.common.record.TimestampType
import org.junit.Assert._
import org.junit.Test
@@ -151,295 +147,6 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases {
assertEquals("second offset should be 2", 2L, iter.next().offset)
}
- @Test
- def testLogAppendTime() {
- val now = System.currentTimeMillis()
- // The timestamps should be overwritten
- val messages = getMessages(magicValue = Message.MagicValue_V1, timestamp = 0L, codec = NoCompressionCodec)
- val compressedMessagesWithRecompresion = getMessages(magicValue = Message.MagicValue_V0, codec = DefaultCompressionCodec)
- val compressedMessagesWithoutRecompression =
- getMessages(magicValue = Message.MagicValue_V1, timestamp = 0L, codec = DefaultCompressionCodec)
-
- val validatingResults = messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
- now = now,
- sourceCodec = NoCompressionCodec,
- targetCodec = NoCompressionCodec,
- messageFormatVersion = 1,
- messageTimestampType = TimestampType.LOG_APPEND_TIME,
- messageTimestampDiffMaxMs = 1000L)
- val validatedMessages = validatingResults.validatedMessages
-
- val validatingCompressedMessagesResults =
- compressedMessagesWithRecompresion.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
- now = now,
- sourceCodec = DefaultCompressionCodec,
- targetCodec = DefaultCompressionCodec,
- messageFormatVersion = 1,
- messageTimestampType = TimestampType.LOG_APPEND_TIME,
- messageTimestampDiffMaxMs = 1000L)
- val validatedCompressedMessages = validatingCompressedMessagesResults.validatedMessages
-
- val validatingCompressedMessagesWithoutRecompressionResults =
- compressedMessagesWithoutRecompression.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
- now = now,
- sourceCodec = DefaultCompressionCodec,
- targetCodec = DefaultCompressionCodec,
- messageFormatVersion = 1,
- messageTimestampType = TimestampType.LOG_APPEND_TIME,
- messageTimestampDiffMaxMs = 1000L)
-
- val validatedCompressedMessagesWithoutRecompression = validatingCompressedMessagesWithoutRecompressionResults.validatedMessages
-
- assertEquals("message set size should not change", messages.size, validatedMessages.size)
- validatedMessages.foreach(messageAndOffset => validateLogAppendTime(messageAndOffset.message))
- assertEquals(s"Max timestamp should be $now", now, validatingResults.maxTimestamp)
- assertEquals(s"The offset of max timestamp should be 0", 0, validatingResults.offsetOfMaxTimestamp)
- assertFalse("Message size should not have been changed", validatingResults.messageSizeMaybeChanged)
-
- assertEquals("message set size should not change", compressedMessagesWithRecompresion.size, validatedCompressedMessages.size)
- validatedCompressedMessages.foreach(messageAndOffset => validateLogAppendTime(messageAndOffset.message))
- assertTrue("MessageSet should still valid", validatedCompressedMessages.shallowIterator.next().message.isValid)
- assertEquals(s"Max timestamp should be $now", now, validatingCompressedMessagesResults.maxTimestamp)
- assertEquals(s"The offset of max timestamp should be ${compressedMessagesWithRecompresion.size - 1}",
- compressedMessagesWithRecompresion.size - 1, validatingCompressedMessagesResults.offsetOfMaxTimestamp)
- assertTrue("Message size may have been changed", validatingCompressedMessagesResults.messageSizeMaybeChanged)
-
- assertEquals("message set size should not change", compressedMessagesWithoutRecompression.size,
- validatedCompressedMessagesWithoutRecompression.size)
- validatedCompressedMessagesWithoutRecompression.foreach(messageAndOffset => validateLogAppendTime(messageAndOffset.message))
- assertTrue("MessageSet should still valid", validatedCompressedMessagesWithoutRecompression.shallowIterator.next().message.isValid)
- assertEquals(s"Max timestamp should be $now", now, validatingCompressedMessagesWithoutRecompressionResults.maxTimestamp)
- assertEquals(s"The offset of max timestamp should be ${compressedMessagesWithoutRecompression.size - 1}",
- compressedMessagesWithoutRecompression.size - 1, validatingCompressedMessagesWithoutRecompressionResults.offsetOfMaxTimestamp)
- assertFalse("Message size should not have been changed", validatingCompressedMessagesWithoutRecompressionResults.messageSizeMaybeChanged)
-
- def validateLogAppendTime(message: Message) {
- message.ensureValid()
- assertEquals(s"Timestamp of message $message should be $now", now, message.timestamp)
- assertEquals(TimestampType.LOG_APPEND_TIME, message.timestampType)
- }
- }
-
- @Test
- def testCreateTime() {
- val now = System.currentTimeMillis()
- val timestampSeq = Seq(now - 1, now + 1, now)
- val messages =
- new ByteBufferMessageSet(NoCompressionCodec,
- new Message("hello".getBytes, timestamp = timestampSeq(0), magicValue = Message.MagicValue_V1),
- new Message("there".getBytes, timestamp = timestampSeq(1), magicValue = Message.MagicValue_V1),
- new Message("beautiful".getBytes, timestamp = timestampSeq(2), magicValue = Message.MagicValue_V1))
- val compressedMessages =
- new ByteBufferMessageSet(DefaultCompressionCodec,
- new Message("hello".getBytes, timestamp = timestampSeq(0), magicValue = Message.MagicValue_V1),
- new Message("there".getBytes, timestamp = timestampSeq(1), magicValue = Message.MagicValue_V1),
- new Message("beautiful".getBytes, timestamp = timestampSeq(2), magicValue = Message.MagicValue_V1))
-
- val validatingResults = messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
- now = System.currentTimeMillis(),
- sourceCodec = NoCompressionCodec,
- targetCodec = NoCompressionCodec,
- messageFormatVersion = 1,
- messageTimestampType = TimestampType.CREATE_TIME,
- messageTimestampDiffMaxMs = 1000L)
- val validatedMessages = validatingResults.validatedMessages
-
- val validatingCompressedMessagesResults =
- compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
- now = System.currentTimeMillis(),
- sourceCodec = DefaultCompressionCodec,
- targetCodec = DefaultCompressionCodec,
- messageFormatVersion = 1,
- messageTimestampType = TimestampType.CREATE_TIME,
- messageTimestampDiffMaxMs = 1000L)
- val validatedCompressedMessages = validatingCompressedMessagesResults.validatedMessages
-
- var i = 0
- for (messageAndOffset <- validatedMessages) {
- messageAndOffset.message.ensureValid()
- assertEquals(messageAndOffset.message.timestamp, timestampSeq(i))
- assertEquals(messageAndOffset.message.timestampType, TimestampType.CREATE_TIME)
- i += 1
- }
- assertEquals(s"Max timestamp should be ${now + 1}", now + 1, validatingResults.maxTimestamp)
- assertEquals(s"Offset of max timestamp should be 1", 1, validatingResults.offsetOfMaxTimestamp)
- assertFalse("Message size should not have been changed", validatingResults.messageSizeMaybeChanged)
- i = 0
- for (messageAndOffset <- validatedCompressedMessages) {
- messageAndOffset.message.ensureValid()
- assertEquals(messageAndOffset.message.timestamp, timestampSeq(i))
- assertEquals(messageAndOffset.message.timestampType, TimestampType.CREATE_TIME)
- i += 1
- }
- assertEquals(s"Max timestamp should be ${now + 1}", now + 1, validatingResults.maxTimestamp)
- assertEquals(s"Offset of max timestamp should be ${validatedCompressedMessages.size - 1}",
- validatedCompressedMessages.size - 1, validatingCompressedMessagesResults.offsetOfMaxTimestamp)
- assertFalse("Message size should not have been changed", validatingCompressedMessagesResults.messageSizeMaybeChanged)
- }
-
- @Test
- def testInvalidCreateTime() {
- val now = System.currentTimeMillis()
- val messages = getMessages(magicValue = Message.MagicValue_V1, timestamp = now - 1001L, codec = NoCompressionCodec)
- val compressedMessages = getMessages(magicValue = Message.MagicValue_V1, timestamp = now - 1001L, codec = DefaultCompressionCodec)
-
- try {
- messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
- now = System.currentTimeMillis(),
- sourceCodec = NoCompressionCodec,
- targetCodec = NoCompressionCodec,
- messageFormatVersion = 1,
- messageTimestampType = TimestampType.CREATE_TIME,
- messageTimestampDiffMaxMs = 1000L)
- fail("Should throw InvalidMessageException.")
- } catch {
- case _: InvalidTimestampException =>
- }
-
- try {
- compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(0),
- now = System.currentTimeMillis(),
- sourceCodec = DefaultCompressionCodec,
- targetCodec = DefaultCompressionCodec,
- messageFormatVersion = 1,
- messageTimestampType = TimestampType.CREATE_TIME,
- messageTimestampDiffMaxMs = 1000L)
- fail("Should throw InvalidMessageException.")
- } catch {
- case _: InvalidTimestampException =>
- }
- }
-
- @Test
- def testAbsoluteOffsetAssignment() {
- val messages = getMessages(magicValue = Message.MagicValue_V0, codec = NoCompressionCodec)
- val compressedMessages = getMessages(magicValue = Message.MagicValue_V0, codec = DefaultCompressionCodec)
- // check uncompressed offsets
- checkOffsets(messages, 0)
- val offset = 1234567
- checkOffsets(messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
- now = System.currentTimeMillis(),
- sourceCodec = NoCompressionCodec,
- targetCodec = NoCompressionCodec,
- messageFormatVersion = 0,
- messageTimestampType = TimestampType.CREATE_TIME,
- messageTimestampDiffMaxMs = 1000L).validatedMessages, offset)
-
- // check compressed messages
- checkOffsets(compressedMessages, 0)
- checkOffsets(compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
- now = System.currentTimeMillis(),
- sourceCodec = DefaultCompressionCodec,
- targetCodec = DefaultCompressionCodec,
- messageFormatVersion = 0,
- messageTimestampType = TimestampType.CREATE_TIME,
- messageTimestampDiffMaxMs = 1000L).validatedMessages, offset)
-
- }
-
- @Test
- def testRelativeOffsetAssignment() {
- val now = System.currentTimeMillis()
- val messages = getMessages(magicValue = Message.MagicValue_V1, timestamp = now, codec = NoCompressionCodec)
- val compressedMessages = getMessages(magicValue = Message.MagicValue_V1, timestamp = now, codec = DefaultCompressionCodec)
-
- // check uncompressed offsets
- checkOffsets(messages, 0)
- val offset = 1234567
- val messageWithOffset = messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
- now = System.currentTimeMillis(),
- sourceCodec = NoCompressionCodec,
- targetCodec = NoCompressionCodec,
- messageTimestampType = TimestampType.CREATE_TIME,
- messageTimestampDiffMaxMs = 5000L).validatedMessages
- checkOffsets(messageWithOffset, offset)
-
- // check compressed messages
- checkOffsets(compressedMessages, 0)
- val compressedMessagesWithOffset = compressedMessages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
- now = System.currentTimeMillis(),
- sourceCodec = DefaultCompressionCodec,
- targetCodec = DefaultCompressionCodec,
- messageTimestampType = TimestampType.CREATE_TIME,
- messageTimestampDiffMaxMs = 5000L).validatedMessages
- checkOffsets(compressedMessagesWithOffset, offset)
- }
-
- @Test(expected = classOf[InvalidMessageException])
- def testInvalidInnerMagicVersion(): Unit = {
- val offset = 1234567
- val messages = messageSetWithInvalidInnerMagic(SnappyCompressionCodec, offset)
- messages.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
- now = System.currentTimeMillis(),
- sourceCodec = SnappyCompressionCodec,
- targetCodec = SnappyCompressionCodec,
- messageTimestampType = TimestampType.CREATE_TIME,
- messageTimestampDiffMaxMs = 5000L).validatedMessages
- }
-
-
- @Test
- def testOffsetAssignmentAfterMessageFormatConversion() {
- // Check up conversion
- val messagesV0 = getMessages(magicValue = Message.MagicValue_V0, codec = NoCompressionCodec)
- val compressedMessagesV0 = getMessages(magicValue = Message.MagicValue_V0, codec = DefaultCompressionCodec)
- // check uncompressed offsets
- checkOffsets(messagesV0, 0)
- val offset = 1234567
- checkOffsets(messagesV0.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
- now = System.currentTimeMillis(),
- sourceCodec = NoCompressionCodec,
- targetCodec = NoCompressionCodec,
- messageFormatVersion = 1,
- messageTimestampType = TimestampType.LOG_APPEND_TIME,
- messageTimestampDiffMaxMs = 1000L).validatedMessages, offset)
-
- // check compressed messages
- checkOffsets(compressedMessagesV0, 0)
- checkOffsets(compressedMessagesV0.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
- now = System.currentTimeMillis(),
- sourceCodec = DefaultCompressionCodec,
- targetCodec = DefaultCompressionCodec,
- messageFormatVersion = 1,
- messageTimestampType = TimestampType.LOG_APPEND_TIME,
- messageTimestampDiffMaxMs = 1000L).validatedMessages, offset)
-
- // Check down conversion
- val now = System.currentTimeMillis()
- val messagesV1 = getMessages(Message.MagicValue_V1, now, NoCompressionCodec)
- val compressedMessagesV1 = getMessages(Message.MagicValue_V1, now, DefaultCompressionCodec)
-
- // check uncompressed offsets
- checkOffsets(messagesV1, 0)
- checkOffsets(messagesV1.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
- now = System.currentTimeMillis(),
- sourceCodec = NoCompressionCodec,
- targetCodec = NoCompressionCodec,
- messageFormatVersion = 0,
- messageTimestampType = TimestampType.CREATE_TIME,
- messageTimestampDiffMaxMs = 5000L).validatedMessages, offset)
-
- // check compressed messages
- checkOffsets(compressedMessagesV1, 0)
- checkOffsets(compressedMessagesV1.validateMessagesAndAssignOffsets(offsetCounter = new LongRef(offset),
- now = System.currentTimeMillis(),
- sourceCodec = DefaultCompressionCodec,
- targetCodec = DefaultCompressionCodec,
- messageFormatVersion = 0,
- messageTimestampType = TimestampType.CREATE_TIME,
- messageTimestampDiffMaxMs = 5000L).validatedMessages, offset)
- }
-
- @Test
- def testWriteFullyTo() {
- checkWriteFullyToWithMessageSet(createMessageSet(Array[Message]()))
- checkWriteFullyToWithMessageSet(createMessageSet(messages))
- }
-
- def checkWriteFullyToWithMessageSet(messageSet: ByteBufferMessageSet) {
- checkWriteWithMessageSet(messageSet, messageSet.writeFullyTo)
- }
-
/* check that offsets are assigned based on byte offset from the given base offset */
def checkOffsets(messages: ByteBufferMessageSet, baseOffset: Long) {
assertTrue("Message set should not be empty", messages.nonEmpty)
@@ -457,59 +164,4 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases {
assertTrue(shallowOffsets.subsetOf(deepOffsets))
}
- private def getMessages(magicValue: Byte = Message.CurrentMagicValue,
- timestamp: Long = Message.NoTimestamp,
- codec: CompressionCodec = NoCompressionCodec): ByteBufferMessageSet = {
- if (magicValue == Message.MagicValue_V0) {
- new ByteBufferMessageSet(
- codec,
- new Message("hello".getBytes, Message.NoTimestamp, Message.MagicValue_V0),
- new Message("there".getBytes, Message.NoTimestamp, Message.MagicValue_V0),
- new Message("beautiful".getBytes, Message.NoTimestamp, Message.MagicValue_V0))
- } else {
- new ByteBufferMessageSet(
- codec,
- new Message("hello".getBytes, timestamp = timestamp, magicValue = Message.MagicValue_V1),
- new Message("there".getBytes, timestamp = timestamp, magicValue = Message.MagicValue_V1),
- new Message("beautiful".getBytes, timestamp = timestamp, magicValue = Message.MagicValue_V1))
- }
- }
-
- private def messageSetWithInvalidInnerMagic(codec: CompressionCodec,
- initialOffset: Long): ByteBufferMessageSet = {
- val messages = (0 until 20).map(id =>
- new Message(key = id.toString.getBytes,
- bytes = id.toString.getBytes,
- timestamp = Message.NoTimestamp,
- magicValue = Message.MagicValue_V0))
-
- val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16))
- var lastOffset = initialOffset
-
- messageWriter.write(
- codec = codec,
- timestamp = System.currentTimeMillis(),
- timestampType = TimestampType.CREATE_TIME,
- magicValue = Message.MagicValue_V1) { outputStream =>
-
- val output = new DataOutputStream(CompressionFactory(codec, Message.MagicValue_V1, outputStream))
- try {
- for (message <- messages) {
- val innerOffset = lastOffset - initialOffset
- output.writeLong(innerOffset)
- output.writeInt(message.size)
- output.write(message.buffer.array, message.buffer.arrayOffset, message.buffer.limit)
- lastOffset += 1
- }
- } finally {
- output.close()
- }
- }
- val buffer = ByteBuffer.allocate(messageWriter.size + MessageSet.LogOverhead)
- ByteBufferMessageSet.writeMessage(buffer, messageWriter, lastOffset - 1)
- buffer.rewind()
-
- new ByteBufferMessageSet(buffer)
- }
-
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala b/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala
index e8abfe1..5d2c8fb 100644
--- a/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala
+++ b/core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala
@@ -70,7 +70,7 @@ class MessageCompressionTest extends JUnitSuite {
testCompressSize(GZIPCompressionCodec, messages, 396)
if(isSnappyAvailable)
- testCompressSize(SnappyCompressionCodec, messages, 502)
+ testCompressSize(SnappyCompressionCodec, messages, 1063)
if(isLZ4Available)
testCompressSize(LZ4CompressionCodec, messages, 387)
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/message/MessageTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/message/MessageTest.scala b/core/src/test/scala/unit/kafka/message/MessageTest.scala
index 5c02125..46c25af 100755
--- a/core/src/test/scala/unit/kafka/message/MessageTest.scala
+++ b/core/src/test/scala/unit/kafka/message/MessageTest.scala
@@ -48,7 +48,7 @@ class MessageTest extends JUnitSuite {
val magicValues = Array(Message.MagicValue_V0, Message.MagicValue_V1)
for(k <- keys; v <- vals; codec <- codecs; t <- timestamps; mv <- magicValues) {
val timestamp = ensureValid(mv, t)
- messages += new MessageTestVal(k, v, codec, timestamp, mv, new Message(v, k, timestamp, codec, mv))
+ messages += MessageTestVal(k, v, codec, timestamp, mv, new Message(v, k, timestamp, codec, mv))
}
def ensureValid(magicValue: Byte, timestamp: Long): Long =
@@ -96,7 +96,7 @@ class MessageTest extends JUnitSuite {
@Test
def testEquality() {
- for(v <- messages) {
+ for (v <- messages) {
assertFalse("Should not equal null", v.message.equals(null))
assertFalse("Should not equal a random string", v.message.equals("asdf"))
assertTrue("Should equal itself", v.message.equals(v.message))
@@ -105,40 +105,6 @@ class MessageTest extends JUnitSuite {
}
}
- @Test
- def testMessageFormatConversion() {
-
- def convertAndVerify(v: MessageTestVal, fromMessageFormat: Byte, toMessageFormat: Byte) {
- assertEquals("Message should be the same when convert to the same version.",
- v.message.toFormatVersion(fromMessageFormat), v.message)
- val convertedMessage = v.message.toFormatVersion(toMessageFormat)
- assertEquals("Size difference is not expected value", convertedMessage.size - v.message.size,
- Message.headerSizeDiff(fromMessageFormat, toMessageFormat))
- assertTrue("Message should still be valid", convertedMessage.isValid)
- assertEquals("Timestamp should be NoTimestamp", convertedMessage.timestamp, Message.NoTimestamp)
- assertEquals(s"Magic value should be $toMessageFormat now", convertedMessage.magic, toMessageFormat)
- if (convertedMessage.hasKey)
- assertEquals("Message key should not change", convertedMessage.key, ByteBuffer.wrap(v.key))
- else
- assertNull(convertedMessage.key)
- if(v.payload == null) {
- assertTrue(convertedMessage.isNull)
- assertEquals("Payload should be null", null, convertedMessage.payload)
- } else {
- assertEquals("Message payload should not change", convertedMessage.payload, ByteBuffer.wrap(v.payload))
- }
- assertEquals("Compression codec should not change", convertedMessage.compressionCodec, v.codec)
- }
-
- for (v <- messages) {
- if (v.magicValue == Message.MagicValue_V0) {
- convertAndVerify(v, Message.MagicValue_V0, Message.MagicValue_V1)
- } else if (v.magicValue == Message.MagicValue_V1) {
- convertAndVerify(v, Message.MagicValue_V1, Message.MagicValue_V0)
- }
- }
- }
-
@Test(expected = classOf[IllegalArgumentException])
def testInvalidTimestampAndMagicValueCombination() {
new Message("hello".getBytes, 0L, Message.MagicValue_V0)
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
index 7d6ad91..5c9f035 100644
--- a/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
+++ b/core/src/test/scala/unit/kafka/server/AbstractFetcherThreadTest.scala
@@ -19,17 +19,17 @@ package kafka.server
import com.yammer.metrics.Metrics
import kafka.cluster.BrokerEndPoint
-import kafka.message.{ByteBufferMessageSet, Message, NoCompressionCodec}
import kafka.server.AbstractFetcherThread.{FetchRequest, PartitionData}
import kafka.utils.TestUtils
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.protocol.Errors
import org.apache.kafka.common.utils.Utils
+import org.apache.kafka.common.record.{MemoryRecords, Record}
import org.junit.Assert.{assertFalse, assertTrue}
import org.junit.{Before, Test}
import scala.collection.JavaConverters._
-import scala.collection.{mutable, Map}
+import scala.collection.{Map, mutable}
class AbstractFetcherThreadTest {
@@ -91,10 +91,10 @@ class AbstractFetcherThreadTest {
override def offset(topicAndPartition: TopicPartition): Long = offsets(topicAndPartition)
}
- class TestPartitionData(byteBufferMessageSet: ByteBufferMessageSet) extends PartitionData {
+ class TestPartitionData(records: MemoryRecords = MemoryRecords.EMPTY) extends PartitionData {
override def errorCode: Short = Errors.NONE.code
- override def toByteBufferMessageSet: ByteBufferMessageSet = byteBufferMessageSet
+ override def toRecords: MemoryRecords = records
override def highWatermark: Long = 0L
@@ -119,7 +119,7 @@ class AbstractFetcherThreadTest {
override def handlePartitionsWithErrors(partitions: Iterable[TopicPartition]): Unit = {}
override protected def fetch(fetchRequest: DummyFetchRequest): Seq[(TopicPartition, TestPartitionData)] =
- fetchRequest.offsets.mapValues(_ => new TestPartitionData(new ByteBufferMessageSet())).toSeq
+ fetchRequest.offsets.mapValues(_ => new TestPartitionData()).toSeq
override protected def buildFetchRequest(partitionMap: collection.Seq[(TopicPartition, PartitionFetchState)]): DummyFetchRequest =
new DummyFetchRequest(partitionMap.map { case (k, v) => (k, v.offset) }.toMap)
@@ -156,8 +156,8 @@ class AbstractFetcherThreadTest {
@volatile var fetchCount = 0
private val normalPartitionDataSet = List(
- new TestPartitionData(new ByteBufferMessageSet(NoCompressionCodec, Seq(0L), new Message("hello".getBytes))),
- new TestPartitionData(new ByteBufferMessageSet(NoCompressionCodec, Seq(1L), new Message("hello".getBytes)))
+ new TestPartitionData(MemoryRecords.withRecords(0L, Record.create("hello".getBytes()))),
+ new TestPartitionData(MemoryRecords.withRecords(1L, Record.create("hello".getBytes())))
)
override def processPartitionData(topicAndPartition: TopicPartition,
@@ -170,10 +170,10 @@ class AbstractFetcherThreadTest {
.format(topicAndPartition, fetchOffset, logEndOffset))
// Now check message's crc
- val messages = partitionData.toByteBufferMessageSet
- for (messageAndOffset <- messages.shallowIterator) {
- messageAndOffset.message.ensureValid()
- logEndOffset = messageAndOffset.nextOffset
+ val records = partitionData.toRecords
+ for (entry <- records.shallowIterator.asScala) {
+ entry.record.ensureValid()
+ logEndOffset = entry.nextOffset
}
}
@@ -181,12 +181,12 @@ class AbstractFetcherThreadTest {
fetchCount += 1
// Set the first fetch to get a corrupted message
if (fetchCount == 1) {
- val corruptedMessage = new Message("hello".getBytes)
- val badChecksum = (corruptedMessage.checksum + 1 % Int.MaxValue).toInt
+ val corruptedRecord = Record.create("hello".getBytes())
+ val badChecksum = (corruptedRecord.checksum + 1 % Int.MaxValue).toInt
// Garble checksum
- Utils.writeUnsignedInt(corruptedMessage.buffer, Message.CrcOffset, badChecksum)
- val byteBufferMessageSet = new ByteBufferMessageSet(NoCompressionCodec, corruptedMessage)
- fetchRequest.offsets.mapValues(_ => new TestPartitionData(byteBufferMessageSet)).toSeq
+ Utils.writeUnsignedInt(corruptedRecord.buffer, Record.CRC_OFFSET, badChecksum)
+ val records = MemoryRecords.withRecords(corruptedRecord)
+ fetchRequest.offsets.mapValues(_ => new TestPartitionData(records)).toSeq
} else
// Then, the following fetches get the normal data
fetchRequest.offsets.mapValues(v => normalPartitionDataSet(v.toInt)).toSeq
[3/9] kafka git commit: KAFKA-4390;
Replace MessageSet usage with client-side alternatives
Posted by jg...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
index 250c8b8..65c2d05 100755
--- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala
@@ -22,10 +22,9 @@ import java.util.Properties
import kafka.api.{KAFKA_0_10_0_IV1, KAFKA_0_9_0}
import kafka.common.TopicAndPartition
-import kafka.message._
import kafka.server.OffsetCheckpoint
import kafka.utils._
-import org.apache.kafka.common.record.CompressionType
+import org.apache.kafka.common.record.{CompressionType, MemoryRecords, Record}
import org.apache.kafka.common.utils.Utils
import org.junit.Assert._
import org.junit._
@@ -43,7 +42,7 @@ import scala.util.Random
@RunWith(value = classOf[Parameterized])
class LogCleanerIntegrationTest(compressionCodec: String) {
- val codec = CompressionCodec.getCompressionCodec(compressionCodec)
+ val codec = CompressionType.forName(compressionCodec)
val time = new MockTime()
val segmentSize = 256
val deleteDelay = 1000
@@ -56,7 +55,7 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
@Test
def cleanerTest() {
val largeMessageKey = 20
- val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, Message.MagicValue_V1)
+ val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, Record.MAGIC_VALUE_V1)
val maxMessageSize = largeMessageSet.sizeInBytes
cleaner = makeCleaner(parts = 3, maxMessageSize = maxMessageSize)
@@ -133,13 +132,13 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
}
// returns (value, ByteBufferMessageSet)
- private def createLargeSingleMessageSet(key: Int, messageFormatVersion: Byte): (String, ByteBufferMessageSet) = {
+ private def createLargeSingleMessageSet(key: Int, messageFormatVersion: Byte): (String, MemoryRecords) = {
def messageValue(length: Int): String = {
val random = new Random(0)
new String(random.alphanumeric.take(length).toArray)
}
val value = messageValue(128)
- val messageSet = TestUtils.singleMessageSet(payload = value.getBytes, codec = codec, key = key.toString.getBytes,
+ val messageSet = TestUtils.singletonRecords(value = value.getBytes, codec = codec, key = key.toString.getBytes,
magicValue = messageFormatVersion)
(value, messageSet)
}
@@ -147,9 +146,9 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
@Test
def testCleanerWithMessageFormatV0(): Unit = {
val largeMessageKey = 20
- val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, Message.MagicValue_V0)
+ val (largeMessageValue, largeMessageSet) = createLargeSingleMessageSet(largeMessageKey, Record.MAGIC_VALUE_V0)
val maxMessageSize = codec match {
- case NoCompressionCodec => largeMessageSet.sizeInBytes
+ case CompressionType.NONE => largeMessageSet.sizeInBytes
case _ =>
// the broker assigns absolute offsets for message format 0 which potentially causes the compressed size to
// increase because the broker offsets are larger than the ones assigned by the client
@@ -165,7 +164,7 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_9_0.version)
log.config = new LogConfig(props)
- val appends = writeDups(numKeys = 100, numDups = 3, log = log, codec = codec, magicValue = Message.MagicValue_V0)
+ val appends = writeDups(numKeys = 100, numDups = 3, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V0)
val startSize = log.size
cleaner.startup()
@@ -177,14 +176,14 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
checkLogAfterAppendingDups(log, startSize, appends)
val appends2: Seq[(Int, String, Long)] = {
- val dupsV0 = writeDups(numKeys = 40, numDups = 3, log = log, codec = codec, magicValue = Message.MagicValue_V0)
+ val dupsV0 = writeDups(numKeys = 40, numDups = 3, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V0)
val appendInfo = log.append(largeMessageSet, assignOffsets = true)
val largeMessageOffset = appendInfo.firstOffset
// also add some messages with version 1 to check that we handle mixed format versions correctly
props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_10_0_IV1.version)
log.config = new LogConfig(props)
- val dupsV1 = writeDups(startKey = 30, numKeys = 40, numDups = 3, log = log, codec = codec, magicValue = Message.MagicValue_V1)
+ val dupsV1 = writeDups(startKey = 30, numKeys = 40, numDups = 3, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V1)
appends ++ dupsV0 ++ Seq((largeMessageKey, largeMessageValue, largeMessageOffset)) ++ dupsV1
}
val firstDirty2 = log.activeSegment.baseOffset
@@ -205,15 +204,15 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
// with compression enabled, these messages will be written as a single message containing
// all of the individual messages
- var appendsV0 = writeDupsSingleMessageSet(numKeys = 2, numDups = 3, log = log, codec = codec, magicValue = Message.MagicValue_V0)
- appendsV0 ++= writeDupsSingleMessageSet(numKeys = 2, startKey = 3, numDups = 2, log = log, codec = codec, magicValue = Message.MagicValue_V0)
+ var appendsV0 = writeDupsSingleMessageSet(numKeys = 2, numDups = 3, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V0)
+ appendsV0 ++= writeDupsSingleMessageSet(numKeys = 2, startKey = 3, numDups = 2, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V0)
props.put(LogConfig.MessageFormatVersionProp, KAFKA_0_10_0_IV1.version)
log.config = new LogConfig(props)
- var appendsV1 = writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = Message.MagicValue_V1)
- appendsV1 ++= writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = Message.MagicValue_V1)
- appendsV1 ++= writeDupsSingleMessageSet(startKey = 6, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = Message.MagicValue_V1)
+ var appendsV1 = writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V1)
+ appendsV1 ++= writeDupsSingleMessageSet(startKey = 4, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V1)
+ appendsV1 ++= writeDupsSingleMessageSet(startKey = 6, numKeys = 2, numDups = 2, log = log, codec = codec, magicValue = Record.MAGIC_VALUE_V1)
val appends = appendsV0 ++ appendsV1
@@ -250,32 +249,27 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
}
private def readFromLog(log: Log): Iterable[(Int, String, Long)] = {
-
- def messageIterator(entry: MessageAndOffset): Iterator[MessageAndOffset] =
- // create single message iterator or deep iterator depending on compression codec
- if (entry.message.compressionCodec == NoCompressionCodec) Iterator(entry)
- else ByteBufferMessageSet.deepIterator(entry)
-
- for (segment <- log.logSegments; entry <- segment.log; messageAndOffset <- messageIterator(entry)) yield {
- val key = TestUtils.readString(messageAndOffset.message.key).toInt
- val value = TestUtils.readString(messageAndOffset.message.payload)
- (key, value, messageAndOffset.offset)
+ import JavaConverters._
+ for (segment <- log.logSegments; deepLogEntry <- segment.log.deepIterator.asScala) yield {
+ val key = TestUtils.readString(deepLogEntry.record.key).toInt
+ val value = TestUtils.readString(deepLogEntry.record.value)
+ (key, value, deepLogEntry.offset)
}
}
- private def writeDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionCodec,
- startKey: Int = 0, magicValue: Byte = Message.CurrentMagicValue): Seq[(Int, String, Long)] = {
+ private def writeDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionType,
+ startKey: Int = 0, magicValue: Byte = Record.CURRENT_MAGIC_VALUE): Seq[(Int, String, Long)] = {
for(_ <- 0 until numDups; key <- startKey until (startKey + numKeys)) yield {
- val payload = counter.toString
- val appendInfo = log.append(TestUtils.singleMessageSet(payload = payload.toString.getBytes, codec = codec,
+ val value = counter.toString
+ val appendInfo = log.append(TestUtils.singletonRecords(value = value.toString.getBytes, codec = codec,
key = key.toString.getBytes, magicValue = magicValue), assignOffsets = true)
counter += 1
- (key, payload, appendInfo.firstOffset)
+ (key, value, appendInfo.firstOffset)
}
}
- private def writeDupsSingleMessageSet(numKeys: Int, numDups: Int, log: Log, codec: CompressionCodec,
- startKey: Int = 0, magicValue: Byte = Message.CurrentMagicValue): Seq[(Int, String, Long)] = {
+ private def writeDupsSingleMessageSet(numKeys: Int, numDups: Int, log: Log, codec: CompressionType,
+ startKey: Int = 0, magicValue: Byte = Record.CURRENT_MAGIC_VALUE): Seq[(Int, String, Long)] = {
val kvs = for (_ <- 0 until numDups; key <- startKey until (startKey + numKeys)) yield {
val payload = counter.toString
counter += 1
@@ -283,11 +277,11 @@ class LogCleanerIntegrationTest(compressionCodec: String) {
}
val messages = kvs.map { case (key, payload) =>
- new Message(payload.toString.getBytes, key.toString.getBytes, Message.NoTimestamp, magicValue)
+ Record.create(magicValue, key.toString.getBytes, payload.toString.getBytes)
}
- val messageSet = new ByteBufferMessageSet(compressionCodec = codec, messages: _*)
- val appendInfo = log.append(messageSet, assignOffsets = true)
+ val records = MemoryRecords.withRecords(codec, messages: _*)
+ val appendInfo = log.append(records, assignOffsets = true)
val offsets = appendInfo.firstOffset to appendInfo.lastOffset
kvs.zip(offsets).map { case (kv, offset) => (kv._1, kv._2, offset) }
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala
index 5e029fc..abab3bf 100644
--- a/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerLagIntegrationTest.scala
@@ -21,7 +21,6 @@ import java.io.File
import java.util.Properties
import kafka.common.TopicAndPartition
-import kafka.message._
import kafka.utils._
import org.apache.kafka.common.record.CompressionType
import org.apache.kafka.common.utils.Utils
@@ -33,7 +32,6 @@ import org.junit.runners.Parameterized.Parameters
import scala.collection._
-
/**
* This is an integration test that tests the fully integrated log cleaner
*/
@@ -52,7 +50,7 @@ class LogCleanerLagIntegrationTest(compressionCodecName: String) extends Logging
val logDir = TestUtils.tempDir()
var counter = 0
val topics = Array(TopicAndPartition("log", 0), TopicAndPartition("log", 1), TopicAndPartition("log", 2))
- val compressionCodec = CompressionCodec.getCompressionCodec(compressionCodecName)
+ val compressionCodec = CompressionType.forName(compressionCodecName)
@Test
def cleanerTest(): Unit = {
@@ -96,7 +94,7 @@ class LogCleanerLagIntegrationTest(compressionCodecName: String) extends Logging
val compactedSize = log.logSegments(0L, activeSegAtT0.baseOffset).map(_.size).sum
debug(s"after cleaning the compacted size up to active segment at T0: $compactedSize")
- val lastCleaned = cleaner.cleanerManager.allCleanerCheckpoints.get(TopicAndPartition("log", 0)).get
+ val lastCleaned = cleaner.cleanerManager.allCleanerCheckpoints(TopicAndPartition("log", 0))
assertTrue(s"log cleaner should have processed up to offset $firstBlock1SegmentBaseOffset, but lastCleaned=$lastCleaned", lastCleaned >= firstBlock1SegmentBaseOffset)
assertTrue(s"log should have been compacted: size up to offset of active segment at T0=$sizeUpToActiveSegmentAtT0 compacted size=$compactedSize",
sizeUpToActiveSegmentAtT0 > compactedSize)
@@ -106,23 +104,19 @@ class LogCleanerLagIntegrationTest(compressionCodecName: String) extends Logging
}
private def readFromLog(log: Log): Iterable[(Int, Int)] = {
- for (segment <- log.logSegments; entry <- segment.log; messageAndOffset <- {
- // create single message iterator or deep iterator depending on compression codec
- if (entry.message.compressionCodec == NoCompressionCodec)
- Stream.cons(entry, Stream.empty).iterator
- else
- ByteBufferMessageSet.deepIterator(entry)
- }) yield {
- val key = TestUtils.readString(messageAndOffset.message.key).toInt
- val value = TestUtils.readString(messageAndOffset.message.payload).toInt
+ import JavaConverters._
+
+ for (segment <- log.logSegments; logEntry <- segment.log.deepIterator.asScala) yield {
+ val key = TestUtils.readString(logEntry.record.key).toInt
+ val value = TestUtils.readString(logEntry.record.value).toInt
key -> value
}
}
- private def writeDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionCodec, timestamp: Long): Seq[(Int, Int)] = {
+ private def writeDups(numKeys: Int, numDups: Int, log: Log, codec: CompressionType, timestamp: Long): Seq[(Int, Int)] = {
for (_ <- 0 until numDups; key <- 0 until numKeys) yield {
val count = counter
- log.append(TestUtils.singleMessageSet(payload = counter.toString.getBytes, codec = codec, key = key.toString.getBytes, timestamp = timestamp), assignOffsets = true)
+ log.append(TestUtils.singletonRecords(value = counter.toString.getBytes, codec = codec, key = key.toString.getBytes, timestamp = timestamp), assignOffsets = true)
counter += 1
(key, count)
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
index 0cd52d6..5dfa268 100644
--- a/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerManagerTest.scala
@@ -21,8 +21,8 @@ import java.io.File
import java.util.Properties
import kafka.common._
-import kafka.message._
import kafka.utils._
+import org.apache.kafka.common.record.{MemoryRecords, Record}
import org.apache.kafka.common.utils.Utils
import org.junit.Assert._
import org.junit.{After, Test}
@@ -54,8 +54,8 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
*/
@Test
def testLogsWithSegmentsToDeleteShouldNotConsiderCleanupPolicyDeleteLogs(): Unit = {
- val messageSet = TestUtils.singleMessageSet("test".getBytes)
- val log: Log = createLog(messageSet.sizeInBytes * 5, LogConfig.Delete)
+ val records = TestUtils.singletonRecords("test".getBytes)
+ val log: Log = createLog(records.sizeInBytes * 5, LogConfig.Delete)
val cleanerManager: LogCleanerManager = createCleanerManager(log)
val readyToDelete = cleanerManager.deletableLogs().size
@@ -67,8 +67,8 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
*/
@Test
def testLogsWithSegmentsToDeleteShouldConsiderCleanupPolicyCompactDeleteLogs(): Unit = {
- val messageSet = TestUtils.singleMessageSet("test".getBytes, key="test".getBytes)
- val log: Log = createLog(messageSet.sizeInBytes * 5, LogConfig.Compact + "," + LogConfig.Delete)
+ val records = TestUtils.singletonRecords("test".getBytes, key="test".getBytes)
+ val log: Log = createLog(records.sizeInBytes * 5, LogConfig.Compact + "," + LogConfig.Delete)
val cleanerManager: LogCleanerManager = createCleanerManager(log)
val readyToDelete = cleanerManager.deletableLogs().size
@@ -81,8 +81,8 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
*/
@Test
def testLogsWithSegmentsToDeleteShouldNotConsiderCleanupPolicyCompactLogs(): Unit = {
- val messageSet = TestUtils.singleMessageSet("test".getBytes, key="test".getBytes)
- val log: Log = createLog(messageSet.sizeInBytes * 5, LogConfig.Compact)
+ val records = TestUtils.singletonRecords("test".getBytes, key="test".getBytes)
+ val log: Log = createLog(records.sizeInBytes * 5, LogConfig.Compact)
val cleanerManager: LogCleanerManager = createCleanerManager(log)
val readyToDelete = cleanerManager.deletableLogs().size
@@ -100,7 +100,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
while(log.numberOfSegments < 8)
- log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = time.milliseconds))
+ log.append(logEntries(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = time.milliseconds))
val topicAndPartition = TopicAndPartition("log", 0)
val lastClean = Map(topicAndPartition-> 0L)
@@ -123,7 +123,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
val t0 = time.milliseconds
while(log.numberOfSegments < 4)
- log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = t0))
+ log.append(logEntries(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = t0))
val activeSegAtT0 = log.activeSegment
@@ -131,7 +131,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
val t1 = time.milliseconds
while (log.numberOfSegments < 8)
- log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = t1))
+ log.append(logEntries(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = t1))
val topicAndPartition = TopicAndPartition("log", 0)
val lastClean = Map(topicAndPartition-> 0L)
@@ -155,7 +155,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
val t0 = time.milliseconds
while (log.numberOfSegments < 8)
- log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = t0))
+ log.append(logEntries(log.logEndOffset.toInt, log.logEndOffset.toInt, timestamp = t0))
time.sleep(compactionLag + 1)
@@ -192,10 +192,7 @@ class LogCleanerManagerTest extends JUnitSuite with Logging {
private def makeLog(dir: File = logDir, config: LogConfig = logConfig) =
new Log(dir = dir, config = config, recoveryPoint = 0L, scheduler = time.scheduler, time = time)
- private def message(key: Int, value: Int, timestamp: Long) =
- new ByteBufferMessageSet(new Message(key = key.toString.getBytes,
- bytes = value.toString.getBytes,
- timestamp = timestamp,
- magicValue = Message.MagicValue_V1))
+ private def logEntries(key: Int, value: Int, timestamp: Long) =
+ MemoryRecords.withRecords(Record.create(timestamp, key.toString.getBytes, value.toString.getBytes))
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
index d80fba1..a99d4b9 100755
--- a/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogCleanerTest.scala
@@ -17,21 +17,21 @@
package kafka.log
-import java.io.{DataOutputStream, File}
+import java.io.File
import java.nio._
import java.nio.file.Paths
import java.util.Properties
import kafka.common._
-import kafka.message._
import kafka.utils._
-import org.apache.kafka.common.record.{MemoryRecords, TimestampType}
+import org.apache.kafka.common.record._
import org.apache.kafka.common.utils.Utils
import org.junit.Assert._
import org.junit.{After, Test}
import org.scalatest.junit.JUnitSuite
import scala.collection._
+import JavaConverters._
/**
* Unit tests for the log cleaning logic
@@ -66,7 +66,7 @@ class LogCleanerTest extends JUnitSuite {
// append messages to the log until we have four segments
while(log.numberOfSegments < 4)
- log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
+ log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
val keysFound = keysInLog(log)
assertEquals(0L until log.logEndOffset, keysFound)
@@ -100,7 +100,7 @@ class LogCleanerTest extends JUnitSuite {
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
while(log.numberOfSegments < 2)
- log.append(message(log.logEndOffset.toInt, Array.fill(largeMessageSize)(0: Byte)))
+ log.append(record(log.logEndOffset.toInt, Array.fill(largeMessageSize)(0: Byte)))
val keysFound = keysInLog(log)
assertEquals(0L until log.logEndOffset, keysFound)
@@ -123,23 +123,23 @@ class LogCleanerTest extends JUnitSuite {
logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer)
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
-
+
// append messages with the keys 0 through N
while(log.numberOfSegments < 2)
- log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
-
+ log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
+
// delete all even keys between 0 and N
val leo = log.logEndOffset
for(key <- 0 until leo.toInt by 2)
- log.append(deleteMessage(key))
-
+ log.append(tombstoneRecord(key))
+
// append some new unique keys to pad out to a new active segment
while(log.numberOfSegments < 4)
- log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
-
+ log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
+
cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0, log.activeSegment.baseOffset))
val keys = keysInLog(log).toSet
- assertTrue("None of the keys we deleted should still exist.",
+ assertTrue("None of the keys we deleted should still exist.",
(0 until leo.toInt by 2).forall(!keys.contains(_)))
}
@@ -151,11 +151,11 @@ class LogCleanerTest extends JUnitSuite {
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
- log.append(message(0,0)) // offset 0
- log.append(message(1,1)) // offset 1
- log.append(message(0,0)) // offset 2
- log.append(message(1,1)) // offset 3
- log.append(message(0,0)) // offset 4
+ log.append(record(0,0)) // offset 0
+ log.append(record(1,1)) // offset 1
+ log.append(record(0,0)) // offset 2
+ log.append(record(1,1)) // offset 3
+ log.append(record(0,0)) // offset 4
// roll the segment, so we can clean the messages already appended
log.roll()
@@ -180,11 +180,11 @@ class LogCleanerTest extends JUnitSuite {
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
- log.append(message(0,0)) // offset 0
- log.append(message(1,1)) // offset 1
- log.append(message(0,0)) // offset 2
- log.append(message(1,1)) // offset 3
- log.append(message(0,0)) // offset 4
+ log.append(record(0,0)) // offset 0
+ log.append(record(1,1)) // offset 1
+ log.append(record(0,0)) // offset 2
+ log.append(record(1,1)) // offset 3
+ log.append(record(0,0)) // offset 4
// roll the segment, so we can clean the messages already appended
log.roll()
@@ -218,18 +218,18 @@ class LogCleanerTest extends JUnitSuite {
// append messages with the keys 0 through N-1, values equal offset
while(log.numberOfSegments <= numCleanableSegments)
- log.append(message(log.logEndOffset.toInt % N, log.logEndOffset.toInt))
+ log.append(record(log.logEndOffset.toInt % N, log.logEndOffset.toInt))
// at this point one message past the cleanable segments has been added
// the entire segment containing the first uncleanable offset should not be cleaned.
val firstUncleanableOffset = log.logEndOffset + 1 // +1 so it is past the baseOffset
while(log.numberOfSegments < numTotalSegments - 1)
- log.append(message(log.logEndOffset.toInt % N, log.logEndOffset.toInt))
+ log.append(record(log.logEndOffset.toInt % N, log.logEndOffset.toInt))
// the last (active) segment has just one message
- def distinctValuesBySegment = log.logSegments.map(s => s.log.map(m => TestUtils.readString(m.message.payload)).toSet.size).toSeq
+ def distinctValuesBySegment = log.logSegments.map(s => s.log.shallowIterator.asScala.map(m => TestUtils.readString(m.record.value)).toSet.size).toSeq
val disctinctValuesBySegmentBeforeClean = distinctValuesBySegment
assertTrue("Test is not effective unless each segment contains duplicates. Increase segment size or decrease number of keys.",
@@ -253,7 +253,7 @@ class LogCleanerTest extends JUnitSuite {
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
// create 6 segments with only one message in each segment
- val messageSet = TestUtils.singleMessageSet(payload = Array.fill[Byte](50)(0), key = 1.toString.getBytes)
+ val messageSet = TestUtils.singletonRecords(value = Array.fill[Byte](50)(0), key = 1.toString.getBytes)
for (_ <- 0 until 6)
log.append(messageSet, assignOffsets = true)
@@ -271,7 +271,7 @@ class LogCleanerTest extends JUnitSuite {
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
// create 6 segments with only one message in each segment
- val messageSet = TestUtils.singleMessageSet(payload = Array.fill[Byte](50)(0), key = 1.toString.getBytes)
+ val messageSet = TestUtils.singletonRecords(value = Array.fill[Byte](50)(0), key = 1.toString.getBytes)
for (_ <- 0 until 6)
log.append(messageSet, assignOffsets = true)
@@ -305,14 +305,14 @@ class LogCleanerTest extends JUnitSuite {
// append unkeyed messages
while(log.numberOfSegments < 2)
- log.append(unkeyedMessage(log.logEndOffset.toInt))
+ log.append(unkeyedRecord(log.logEndOffset.toInt))
val numInvalidMessages = unkeyedMessageCountInLog(log)
val sizeWithUnkeyedMessages = log.size
// append keyed messages
while(log.numberOfSegments < 3)
- log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
+ log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
val expectedSizeAfterCleaning = log.size - sizeWithUnkeyedMessages
val (_, stats) = cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0, log.activeSegment.baseOffset))
@@ -321,17 +321,17 @@ class LogCleanerTest extends JUnitSuite {
assertEquals("Log should only contain keyed messages after cleaning.", expectedSizeAfterCleaning, log.size)
assertEquals("Cleaner should have seen %d invalid messages.", numInvalidMessages, stats.invalidMessagesRead)
}
-
+
/* extract all the keys from a log */
def keysInLog(log: Log): Iterable[Int] =
- log.logSegments.flatMap(s => s.log.filter(!_.message.isNull).filter(_.message.hasKey).map(m => TestUtils.readString(m.message.key).toInt))
+ log.logSegments.flatMap(s => s.log.shallowIterator.asScala.filter(!_.record.hasNullValue).filter(_.record.hasKey).map(m => TestUtils.readString(m.record.key).toInt))
/* extract all the offsets from a log */
def offsetsInLog(log: Log): Iterable[Long] =
- log.logSegments.flatMap(s => s.log.filter(!_.message.isNull).filter(_.message.hasKey).map(m => m.offset))
+ log.logSegments.flatMap(s => s.log.shallowIterator.asScala.filter(!_.record.hasNullValue).filter(_.record.hasKey).map(m => m.offset))
def unkeyedMessageCountInLog(log: Log) =
- log.logSegments.map(s => s.log.filter(!_.message.isNull).count(m => !m.message.hasKey)).sum
+ log.logSegments.map(s => s.log.shallowIterator.asScala.filter(!_.record.hasNullValue).count(m => !m.record.hasKey)).sum
def abortCheckDone(topicAndPartition: TopicAndPartition): Unit = {
throw new LogCleaningAbortedException()
@@ -350,7 +350,7 @@ class LogCleanerTest extends JUnitSuite {
// append messages to the log until we have four segments
while(log.numberOfSegments < 4)
- log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
+ log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
val keys = keysInLog(log)
val map = new FakeOffsetMap(Int.MaxValue)
@@ -371,20 +371,20 @@ class LogCleanerTest extends JUnitSuite {
logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
-
+
// append some messages to the log
var i = 0
while(log.numberOfSegments < 10) {
- log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes))
+ log.append(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes))
i += 1
}
-
+
// grouping by very large values should result in a single group with all the segments in it
var groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
assertEquals(1, groups.size)
assertEquals(log.numberOfSegments, groups.head.size)
checkSegmentOrder(groups)
-
+
// grouping by very small values should result in all groups having one entry
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = 1, maxIndexSize = Int.MaxValue)
assertEquals(log.numberOfSegments, groups.size)
@@ -396,20 +396,20 @@ class LogCleanerTest extends JUnitSuite {
checkSegmentOrder(groups)
val groupSize = 3
-
+
// check grouping by log size
val logSize = log.logSegments.take(groupSize).map(_.size).sum.toInt + 1
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = logSize, maxIndexSize = Int.MaxValue)
checkSegmentOrder(groups)
assertTrue("All but the last group should be the target size.", groups.dropRight(1).forall(_.size == groupSize))
-
+
// check grouping by index size
val indexSize = log.logSegments.take(groupSize).map(_.index.sizeInBytes).sum + 1
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = indexSize)
checkSegmentOrder(groups)
assertTrue("All but the last group should be the target size.", groups.dropRight(1).forall(_.size == groupSize))
}
-
+
/**
* Validate the logic for grouping log segments together for cleaning when only a small number of
* messages are retained, but the range of offsets is greater than Int.MaxValue. A group should not
@@ -425,47 +425,45 @@ class LogCleanerTest extends JUnitSuite {
logProps.put(LogConfig.IndexIntervalBytesProp, 1: java.lang.Integer)
val log = makeLog(config = LogConfig.fromProps(logConfig.originals, logProps))
-
+
// fill up first segment
while (log.numberOfSegments == 1)
- log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes))
-
+ log.append(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes))
+
// forward offset and append message to next segment at offset Int.MaxValue
- val messageSet = new ByteBufferMessageSet(NoCompressionCodec, new LongRef(Int.MaxValue - 1),
- new Message("hello".getBytes, "hello".getBytes, Message.NoTimestamp, Message.MagicValue_V1))
- log.append(messageSet, assignOffsets = false)
- log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes))
+ val records = MemoryRecords.withLogEntries(LogEntry.create(Int.MaxValue - 1, Record.create("hello".getBytes, "hello".getBytes)))
+ log.append(records, assignOffsets = false)
+ log.append(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes))
assertEquals(Int.MaxValue, log.activeSegment.index.lastOffset)
-
+
// grouping should result in a single group with maximum relative offset of Int.MaxValue
var groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
assertEquals(1, groups.size)
-
+
// append another message, making last offset of second segment > Int.MaxValue
- log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes))
-
+ log.append(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes))
+
// grouping should not group the two segments to ensure that maximum relative offset in each group <= Int.MaxValue
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
assertEquals(2, groups.size)
checkSegmentOrder(groups)
-
+
// append more messages, creating new segments, further grouping should still occur
while (log.numberOfSegments < 4)
- log.append(TestUtils.singleMessageSet(payload = "hello".getBytes, key = "hello".getBytes))
+ log.append(TestUtils.singletonRecords(value = "hello".getBytes, key = "hello".getBytes))
groups = cleaner.groupSegmentsBySize(log.logSegments, maxSize = Int.MaxValue, maxIndexSize = Int.MaxValue)
assertEquals(log.numberOfSegments - 1, groups.size)
for (group <- groups)
assertTrue("Relative offset greater than Int.MaxValue", group.last.index.lastOffset - group.head.index.baseOffset <= Int.MaxValue)
checkSegmentOrder(groups)
-
}
-
+
private def checkSegmentOrder(groups: Seq[Seq[LogSegment]]): Unit = {
val offsets = groups.flatMap(_.map(_.baseOffset))
assertEquals("Offsets should be in increasing order.", offsets.sorted, offsets)
}
-
+
/**
* Test building an offset map off the log
*/
@@ -496,8 +494,7 @@ class LogCleanerTest extends JUnitSuite {
checkRange(map, segments(1).baseOffset.toInt, segments(3).baseOffset.toInt)
checkRange(map, segments(3).baseOffset.toInt, log.logEndOffset.toInt)
}
-
-
+
/**
* Tests recovery if broker crashes at the following stages during the cleaning sequence
* <ol>
@@ -516,8 +513,8 @@ class LogCleanerTest extends JUnitSuite {
logProps.put(LogConfig.FileDeleteDelayMsProp, 10: java.lang.Integer)
val config = LogConfig.fromProps(logConfig.originals, logProps)
-
- def recoverAndCheck(config: LogConfig, expectedKeys : Iterable[Int]) : Log = {
+
+ def recoverAndCheck(config: LogConfig, expectedKeys : Iterable[Int]) : Log = {
// Recover log file and check that after recovery, keys are as expected
// and all temporary files have been deleted
val recoveredLog = makeLog(config = config)
@@ -530,25 +527,25 @@ class LogCleanerTest extends JUnitSuite {
assertEquals(expectedKeys, keysInLog(recoveredLog))
recoveredLog
}
-
+
// create a log and append some messages
var log = makeLog(config = config)
var messageCount = 0
while(log.numberOfSegments < 10) {
- log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
+ log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
messageCount += 1
}
val allKeys = keysInLog(log)
-
+
// pretend we have odd-numbered keys
val offsetMap = new FakeOffsetMap(Int.MaxValue)
for (k <- 1 until messageCount by 2)
offsetMap.put(key(k), Long.MaxValue)
-
+
// clean the log
cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
var cleanedKeys = keysInLog(log)
-
+
// 1) Simulate recovery just after .cleaned file is created, before rename to .swap
// On recovery, clean operation is aborted. All messages should be present in the log
log.logSegments.head.changeFileSuffixes("", Log.CleanedFileSuffix)
@@ -556,44 +553,44 @@ class LogCleanerTest extends JUnitSuite {
Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, "")))
}
log = recoverAndCheck(config, allKeys)
-
+
// clean again
cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
cleanedKeys = keysInLog(log)
-
+
// 2) Simulate recovery just after swap file is created, before old segment files are
- // renamed to .deleted. Clean operation is resumed during recovery.
+ // renamed to .deleted. Clean operation is resumed during recovery.
log.logSegments.head.changeFileSuffixes("", Log.SwapFileSuffix)
for (file <- dir.listFiles if file.getName.endsWith(Log.DeletedFileSuffix)) {
Utils.atomicMoveWithFallback(file.toPath, Paths.get(CoreUtils.replaceSuffix(file.getPath, Log.DeletedFileSuffix, "")))
- }
+ }
log = recoverAndCheck(config, cleanedKeys)
// add some more messages and clean the log again
while(log.numberOfSegments < 10) {
- log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
+ log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
messageCount += 1
}
for (k <- 1 until messageCount by 2)
- offsetMap.put(key(k), Long.MaxValue)
+ offsetMap.put(key(k), Long.MaxValue)
cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
cleanedKeys = keysInLog(log)
-
+
// 3) Simulate recovery after swap file is created and old segments files are renamed
// to .deleted. Clean operation is resumed during recovery.
log.logSegments.head.changeFileSuffixes("", Log.SwapFileSuffix)
log = recoverAndCheck(config, cleanedKeys)
-
+
// add some more messages and clean the log again
while(log.numberOfSegments < 10) {
- log.append(message(log.logEndOffset.toInt, log.logEndOffset.toInt))
+ log.append(record(log.logEndOffset.toInt, log.logEndOffset.toInt))
messageCount += 1
}
for (k <- 1 until messageCount by 2)
- offsetMap.put(key(k), Long.MaxValue)
+ offsetMap.put(key(k), Long.MaxValue)
cleaner.cleanSegments(log, log.logSegments.take(9).toSeq, offsetMap, 0L, new CleanerStats())
cleanedKeys = keysInLog(log)
-
+
// 4) Simulate recovery after swap is complete, but async deletion
// is not yet complete. Clean operation is resumed during recovery.
recoverAndCheck(config, cleanedKeys)
@@ -631,11 +628,11 @@ class LogCleanerTest extends JUnitSuite {
val log = makeLog()
val cleaner = makeCleaner(2)
- log.append(message(0,0))
- log.append(message(1,1))
- log.append(message(2,2))
- log.append(message(3,3))
- log.append(message(4,4))
+ log.append(record(0,0))
+ log.append(record(1,1))
+ log.append(record(2,2))
+ log.append(record(3,3))
+ log.append(record(4,4))
log.roll()
val stats = new CleanerStats()
@@ -653,7 +650,7 @@ class LogCleanerTest extends JUnitSuite {
*/
@Test
def testCleanCorruptMessageSet() {
- val codec = SnappyCompressionCodec
+ val codec = CompressionType.GZIP
val logProps = new Properties()
logProps.put(LogConfig.CompressionTypeProp, codec.name)
@@ -682,10 +679,10 @@ class LogCleanerTest extends JUnitSuite {
cleaner.clean(LogToClean(TopicAndPartition("test", 0), log, 0, log.activeSegment.baseOffset))
- for (segment <- log.logSegments; shallowMessage <- segment.log.iterator; deepMessage <- ByteBufferMessageSet.deepIterator(shallowMessage)) {
- assertEquals(shallowMessage.message.magic, deepMessage.message.magic)
- val value = TestUtils.readString(deepMessage.message.payload).toLong
- assertEquals(deepMessage.offset, value)
+ for (segment <- log.logSegments; shallowLogEntry <- segment.log.shallowIterator.asScala; deepLogEntry <- shallowLogEntry.asScala) {
+ assertEquals(shallowLogEntry.record.magic, deepLogEntry.record.magic)
+ val value = TestUtils.readString(deepLogEntry.record.value).toLong
+ assertEquals(deepLogEntry.offset, value)
}
}
@@ -704,7 +701,7 @@ class LogCleanerTest extends JUnitSuite {
val corruptedMessage = invalidCleanedMessage(offset, set)
val records = MemoryRecords.readableRecords(corruptedMessage.buffer)
- for (logEntry <- records.iterator.asScala) {
+ for (logEntry <- records.deepIterator.asScala) {
val offset = logEntry.offset
val value = TestUtils.readString(logEntry.record.value).toLong
assertEquals(offset, value)
@@ -718,94 +715,64 @@ class LogCleanerTest extends JUnitSuite {
private def invalidCleanedMessage(initialOffset: Long,
keysAndValues: Iterable[(Int, Int)],
- codec: CompressionCodec = SnappyCompressionCodec): ByteBufferMessageSet = {
+ codec: CompressionType = CompressionType.GZIP): MemoryRecords = {
// this function replicates the old versions of the cleaner which under some circumstances
// would write invalid compressed message sets with the outer magic set to 1 and the inner
// magic set to 0
-
- val messages = keysAndValues.map(kv =>
- new Message(key = kv._1.toString.getBytes,
- bytes = kv._2.toString.getBytes,
- timestamp = Message.NoTimestamp,
- magicValue = Message.MagicValue_V0))
-
- val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16))
- var lastOffset = initialOffset
-
- messageWriter.write(
- codec = codec,
- timestamp = Message.NoTimestamp,
- timestampType = TimestampType.CREATE_TIME,
- magicValue = Message.MagicValue_V1) { outputStream =>
-
- val output = new DataOutputStream(CompressionFactory(codec, Message.MagicValue_V1, outputStream))
- try {
- for (message <- messages) {
- val innerOffset = lastOffset - initialOffset
- output.writeLong(innerOffset)
- output.writeInt(message.size)
- output.write(message.buffer.array, message.buffer.arrayOffset, message.buffer.limit)
- lastOffset += 1
- }
- } finally {
- output.close()
- }
+ val records = keysAndValues.map(kv =>
+ Record.create(Record.MAGIC_VALUE_V0,
+ Record.NO_TIMESTAMP,
+ kv._1.toString.getBytes,
+ kv._2.toString.getBytes))
+
+ val buffer = ByteBuffer.allocate(math.min(math.max(records.map(_.sizeInBytes()).sum / 2, 1024), 1 << 16))
+ val builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V1, codec, TimestampType.CREATE_TIME)
+
+ var offset = initialOffset
+ records.foreach { record =>
+ builder.appendUnchecked(offset, record)
+ offset += 1
}
- val buffer = ByteBuffer.allocate(messageWriter.size + MessageSet.LogOverhead)
- ByteBufferMessageSet.writeMessage(buffer, messageWriter, lastOffset - 1)
- buffer.rewind()
- new ByteBufferMessageSet(buffer)
+ builder.build()
}
private def messageWithOffset(key: Int, value: Int, offset: Long) =
- new ByteBufferMessageSet(NoCompressionCodec, Seq(offset),
- new Message(key = key.toString.getBytes,
- bytes = value.toString.getBytes,
- timestamp = Message.NoTimestamp,
- magicValue = Message.MagicValue_V1))
-
-
+ MemoryRecords.withLogEntries(LogEntry.create(offset, Record.create(key.toString.getBytes, value.toString.getBytes)))
+
def makeLog(dir: File = dir, config: LogConfig = logConfig) =
new Log(dir = dir, config = config, recoveryPoint = 0L, scheduler = time.scheduler, time = time)
def noOpCheckDone(topicAndPartition: TopicAndPartition) { /* do nothing */ }
def makeCleaner(capacity: Int, checkDone: (TopicAndPartition) => Unit = noOpCheckDone, maxMessageSize: Int = 64*1024) =
- new Cleaner(id = 0,
- offsetMap = new FakeOffsetMap(capacity),
+ new Cleaner(id = 0,
+ offsetMap = new FakeOffsetMap(capacity),
ioBufferSize = maxMessageSize,
maxIoBufferSize = maxMessageSize,
dupBufferLoadFactor = 0.75,
- throttler = throttler,
+ throttler = throttler,
time = time,
checkDone = checkDone )
-
+
def writeToLog(log: Log, seq: Iterable[(Int, Int)]): Iterable[Long] = {
for((key, value) <- seq)
- yield log.append(message(key, value)).firstOffset
+ yield log.append(record(key, value)).firstOffset
}
-
+
def key(id: Int) = ByteBuffer.wrap(id.toString.getBytes)
-
- def message(key: Int, value: Int): ByteBufferMessageSet =
- message(key, value.toString.getBytes)
-
- def message(key: Int, value: Array[Byte]) =
- new ByteBufferMessageSet(new Message(key = key.toString.getBytes,
- bytes = value,
- timestamp = Message.NoTimestamp,
- magicValue = Message.MagicValue_V1))
-
- def unkeyedMessage(value: Int) =
- new ByteBufferMessageSet(new Message(bytes = value.toString.getBytes))
-
- def deleteMessage(key: Int) =
- new ByteBufferMessageSet(new Message(key = key.toString.getBytes,
- bytes = null,
- timestamp = Message.NoTimestamp,
- magicValue = Message.MagicValue_V1))
-
+
+ def record(key: Int, value: Int): MemoryRecords =
+ record(key, value.toString.getBytes)
+
+ def record(key: Int, value: Array[Byte]) =
+ MemoryRecords.withRecords(Record.create(key.toString.getBytes, value))
+
+ def unkeyedRecord(value: Int) =
+ MemoryRecords.withRecords(Record.create(value.toString.getBytes))
+
+ def tombstoneRecord(key: Int) = record(key, null)
+
}
class FakeOffsetMap(val slots: Int) extends OffsetMap {
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
index 5421da9..40e6228 100755
--- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala
@@ -67,7 +67,7 @@ class LogManagerTest {
val log = logManager.createLog(TopicAndPartition(name, 0), logConfig)
val logFile = new File(logDir, name + "-0")
assertTrue(logFile.exists)
- log.append(TestUtils.singleMessageSet("test".getBytes()))
+ log.append(TestUtils.singletonRecords("test".getBytes()))
}
/**
@@ -89,7 +89,7 @@ class LogManagerTest {
val log = logManager.createLog(TopicAndPartition(name, 0), logConfig)
var offset = 0L
for(_ <- 0 until 200) {
- val set = TestUtils.singleMessageSet("test".getBytes())
+ val set = TestUtils.singletonRecords("test".getBytes())
val info = log.append(set)
offset = info.lastOffset
}
@@ -101,7 +101,7 @@ class LogManagerTest {
assertEquals("Now there should only be only one segment in the index.", 1, log.numberOfSegments)
time.sleep(log.config.fileDeleteDelayMs + 1)
assertEquals("Files should have been deleted", log.numberOfSegments * 3, log.dir.list.length)
- assertEquals("Should get empty fetch off new log.", 0, log.read(offset+1, 1024).messageSet.sizeInBytes)
+ assertEquals("Should get empty fetch off new log.", 0, log.read(offset+1, 1024).records.sizeInBytes)
try {
log.read(0, 1024)
@@ -110,7 +110,7 @@ class LogManagerTest {
case _: OffsetOutOfRangeException => // This is good.
}
// log should still be appendable
- log.append(TestUtils.singleMessageSet("test".getBytes()))
+ log.append(TestUtils.singletonRecords("test".getBytes()))
}
/**
@@ -118,7 +118,7 @@ class LogManagerTest {
*/
@Test
def testCleanupSegmentsToMaintainSize() {
- val setSize = TestUtils.singleMessageSet("test".getBytes()).sizeInBytes
+ val setSize = TestUtils.singletonRecords("test".getBytes()).sizeInBytes
logManager.shutdown()
val logProps = new Properties()
logProps.put(LogConfig.SegmentBytesProp, 10 * setSize: java.lang.Integer)
@@ -135,7 +135,7 @@ class LogManagerTest {
// add a bunch of messages that should be larger than the retentionSize
val numMessages = 200
for (_ <- 0 until numMessages) {
- val set = TestUtils.singleMessageSet("test".getBytes())
+ val set = TestUtils.singletonRecords("test".getBytes())
val info = log.append(set)
offset = info.firstOffset
}
@@ -147,7 +147,7 @@ class LogManagerTest {
assertEquals("Now there should be exactly 6 segments", 6, log.numberOfSegments)
time.sleep(log.config.fileDeleteDelayMs + 1)
assertEquals("Files should have been deleted", log.numberOfSegments * 3, log.dir.list.length)
- assertEquals("Should get empty fetch off new log.", 0, log.read(offset + 1, 1024).messageSet.sizeInBytes)
+ assertEquals("Should get empty fetch off new log.", 0, log.read(offset + 1, 1024).records.sizeInBytes)
try {
log.read(0, 1024)
fail("Should get exception from fetching earlier.")
@@ -155,7 +155,7 @@ class LogManagerTest {
case _: OffsetOutOfRangeException => // This is good.
}
// log should still be appendable
- log.append(TestUtils.singleMessageSet("test".getBytes()))
+ log.append(TestUtils.singletonRecords("test".getBytes()))
}
/**
@@ -169,7 +169,7 @@ class LogManagerTest {
val log = logManager.createLog(TopicAndPartition(name, 0), LogConfig.fromProps(logConfig.originals, logProps))
var offset = 0L
for (_ <- 0 until 200) {
- val set = TestUtils.singleMessageSet("test".getBytes(), key="test".getBytes())
+ val set = TestUtils.singletonRecords("test".getBytes(), key="test".getBytes())
val info = log.append(set)
offset = info.lastOffset
}
@@ -198,7 +198,7 @@ class LogManagerTest {
val log = logManager.createLog(TopicAndPartition(name, 0), config)
val lastFlush = log.lastFlushTime
for (_ <- 0 until 200) {
- val set = TestUtils.singleMessageSet("test".getBytes())
+ val set = TestUtils.singletonRecords("test".getBytes())
log.append(set)
}
time.sleep(logManager.InitialTaskDelayMs)
@@ -280,7 +280,7 @@ class LogManagerTest {
val logs = topicAndPartitions.map(this.logManager.createLog(_, logConfig))
logs.foreach(log => {
for (_ <- 0 until 50)
- log.append(TestUtils.singleMessageSet("test".getBytes()))
+ log.append(TestUtils.singletonRecords("test".getBytes()))
log.flush()
})
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
----------------------------------------------------------------------
diff --git a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
index f02c5cb..d99981a 100644
--- a/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
+++ b/core/src/test/scala/unit/kafka/log/LogSegmentTest.scala
@@ -16,15 +16,13 @@
*/
package kafka.log
-import org.junit.Assert._
-import java.util.concurrent.atomic._
-
-import kafka.common.LongRef
-import org.junit.{After, Test}
import kafka.utils.TestUtils
-import kafka.message._
+import org.apache.kafka.common.record.{FileRecords, MemoryRecords, Record}
import org.apache.kafka.common.utils.Time
+import org.junit.Assert._
+import org.junit.{After, Test}
+import scala.collection.JavaConverters._
import scala.collection._
class LogSegmentTest {
@@ -34,7 +32,7 @@ class LogSegmentTest {
/* create a segment with the given base offset */
def createSegment(offset: Long, indexIntervalBytes: Int = 10): LogSegment = {
val msFile = TestUtils.tempFile()
- val ms = new FileMessageSet(msFile)
+ val ms = FileRecords.open(msFile)
val idxFile = TestUtils.tempFile()
val timeIdxFile = TestUtils.tempFile()
idxFile.delete()
@@ -47,12 +45,10 @@ class LogSegmentTest {
}
/* create a ByteBufferMessageSet for the given messages starting from the given offset */
- def messages(offset: Long, messages: String*): ByteBufferMessageSet = {
- new ByteBufferMessageSet(compressionCodec = NoCompressionCodec,
- offsetCounter = new LongRef(offset),
- messages = messages.map(s => new Message(s.getBytes, offset * 10, Message.MagicValue_V1)):_*)
+ def records(offset: Long, records: String*): MemoryRecords = {
+ MemoryRecords.withRecords(offset, records.map(s => Record.create(Record.MAGIC_VALUE_V1, offset * 10, s.getBytes)):_*)
}
-
+
@After
def teardown() {
for(seg <- segments) {
@@ -60,7 +56,7 @@ class LogSegmentTest {
seg.log.delete()
}
}
-
+
/**
* A read on an empty log segment should return null
*/
@@ -70,7 +66,7 @@ class LogSegmentTest {
val read = seg.read(startOffset = 40, maxSize = 300, maxOffset = None)
assertNull("Read beyond the last offset in the segment should be null", read)
}
-
+
/**
* Reading from before the first offset in the segment should return messages
* beginning with the first message in the segment
@@ -78,12 +74,12 @@ class LogSegmentTest {
@Test
def testReadBeforeFirstOffset() {
val seg = createSegment(40)
- val ms = messages(50, "hello", "there", "little", "bee")
- seg.append(50, Message.NoTimestamp, -1L, ms)
- val read = seg.read(startOffset = 41, maxSize = 300, maxOffset = None).messageSet
- assertEquals(ms.toList, read.toList)
+ val ms = records(50, "hello", "there", "little", "bee")
+ seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
+ val read = seg.read(startOffset = 41, maxSize = 300, maxOffset = None).records
+ assertEquals(ms.deepIterator.asScala.toList, read.deepIterator.asScala.toList)
}
-
+
/**
* If we set the startOffset and maxOffset for the read to be the same value
* we should get only the first message in the log
@@ -92,28 +88,28 @@ class LogSegmentTest {
def testMaxOffset() {
val baseOffset = 50
val seg = createSegment(baseOffset)
- val ms = messages(baseOffset, "hello", "there", "beautiful")
- seg.append(baseOffset, Message.NoTimestamp, -1L, ms)
- def validate(offset: Long) =
- assertEquals(ms.filter(_.offset == offset).toList,
- seg.read(startOffset = offset, maxSize = 1024, maxOffset = Some(offset+1)).messageSet.toList)
+ val ms = records(baseOffset, "hello", "there", "beautiful")
+ seg.append(baseOffset, Record.NO_TIMESTAMP, -1L, ms)
+ def validate(offset: Long) =
+ assertEquals(ms.deepIterator.asScala.filter(_.offset == offset).toList,
+ seg.read(startOffset = offset, maxSize = 1024, maxOffset = Some(offset+1)).records.deepIterator.asScala.toList)
validate(50)
validate(51)
validate(52)
}
-
+
/**
* If we read from an offset beyond the last offset in the segment we should get null
*/
@Test
def testReadAfterLast() {
val seg = createSegment(40)
- val ms = messages(50, "hello", "there")
- seg.append(50, Message.NoTimestamp, -1L, ms)
+ val ms = records(50, "hello", "there")
+ seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
val read = seg.read(startOffset = 52, maxSize = 200, maxOffset = None)
assertNull("Read beyond the last offset in the segment should give null", read)
}
-
+
/**
* If we read from an offset which doesn't exist we should get a message set beginning
* with the least offset greater than the given startOffset.
@@ -121,14 +117,14 @@ class LogSegmentTest {
@Test
def testReadFromGap() {
val seg = createSegment(40)
- val ms = messages(50, "hello", "there")
- seg.append(50, Message.NoTimestamp, -1L, ms)
- val ms2 = messages(60, "alpha", "beta")
- seg.append(60, Message.NoTimestamp, -1L, ms2)
+ val ms = records(50, "hello", "there")
+ seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
+ val ms2 = records(60, "alpha", "beta")
+ seg.append(60, Record.NO_TIMESTAMP, -1L, ms2)
val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None)
- assertEquals(ms2.toList, read.messageSet.toList)
+ assertEquals(ms2.deepIterator.asScala.toList, read.records.deepIterator.asScala.toList)
}
-
+
/**
* In a loop append two messages then truncate off the second of those messages and check that we can read
* the first but not the second message.
@@ -138,18 +134,18 @@ class LogSegmentTest {
val seg = createSegment(40)
var offset = 40
for (_ <- 0 until 30) {
- val ms1 = messages(offset, "hello")
- seg.append(offset, Message.NoTimestamp, -1L, ms1)
- val ms2 = messages(offset + 1, "hello")
- seg.append(offset + 1, Message.NoTimestamp, -1L, ms2)
+ val ms1 = records(offset, "hello")
+ seg.append(offset, Record.NO_TIMESTAMP, -1L, ms1)
+ val ms2 = records(offset + 1, "hello")
+ seg.append(offset + 1, Record.NO_TIMESTAMP, -1L, ms2)
// check that we can read back both messages
val read = seg.read(offset, None, 10000)
- assertEquals(List(ms1.head, ms2.head), read.messageSet.toList)
+ assertEquals(List(ms1.deepIterator.next(), ms2.deepIterator.next()), read.records.deepIterator.asScala.toList)
// now truncate off the last message
seg.truncateTo(offset + 1)
val read2 = seg.read(offset, None, 10000)
- assertEquals(1, read2.messageSet.size)
- assertEquals(ms1.head, read2.messageSet.head)
+ assertEquals(1, read2.records.deepIterator.asScala.size)
+ assertEquals(ms1.deepIterator.next(), read2.records.deepIterator.next())
offset += 1
}
}
@@ -157,10 +153,10 @@ class LogSegmentTest {
@Test
def testReloadLargestTimestampAfterTruncation() {
val numMessages = 30
- val seg = createSegment(40, 2 * messages(0, "hello").sizeInBytes - 1)
+ val seg = createSegment(40, 2 * records(0, "hello").sizeInBytes - 1)
var offset = 40
for (_ <- 0 until numMessages) {
- seg.append(offset, offset, offset, messages(offset, "hello"))
+ seg.append(offset, offset, offset, records(offset, "hello"))
offset += 1
}
val expectedNumEntries = numMessages / 2 - 1
@@ -179,10 +175,10 @@ class LogSegmentTest {
def testTruncateFull() {
// test the case where we fully truncate the log
val seg = createSegment(40)
- seg.append(40, Message.NoTimestamp, -1L, messages(40, "hello", "there"))
+ seg.append(40, Record.NO_TIMESTAMP, -1L, records(40, "hello", "there"))
seg.truncateTo(0)
assertNull("Segment should be empty.", seg.read(0, None, 1024))
- seg.append(40, Message.NoTimestamp, -1L, messages(40, "hello", "there"))
+ seg.append(40, Record.NO_TIMESTAMP, -1L, records(40, "hello", "there"))
}
/**
@@ -190,11 +186,11 @@ class LogSegmentTest {
*/
@Test
def testFindOffsetByTimestamp() {
- val messageSize = messages(0, s"msg00").sizeInBytes
+ val messageSize = records(0, s"msg00").sizeInBytes
val seg = createSegment(40, messageSize * 2 - 1)
// Produce some messages
for (i <- 40 until 50)
- seg.append(i, i * 10, i, messages(i, s"msg$i"))
+ seg.append(i, i * 10, i, records(i, s"msg$i"))
assertEquals(490, seg.largestTimestamp)
// Search for an indexed timestamp
@@ -218,10 +214,10 @@ class LogSegmentTest {
def testNextOffsetCalculation() {
val seg = createSegment(40)
assertEquals(40, seg.nextOffset)
- seg.append(50, Message.NoTimestamp, -1L, messages(50, "hello", "there", "you"))
+ seg.append(50, Record.NO_TIMESTAMP, -1L, records(50, "hello", "there", "you"))
assertEquals(53, seg.nextOffset())
}
-
+
/**
* Test that we can change the file suffixes for the log and index files
*/
@@ -236,7 +232,7 @@ class LogSegmentTest {
assertTrue(seg.log.file.exists)
assertTrue(seg.index.file.exists)
}
-
+
/**
* Create a segment with some data and an index. Then corrupt the index,
* and recover the segment, the entries should all be readable.
@@ -245,12 +241,12 @@ class LogSegmentTest {
def testRecoveryFixesCorruptIndex() {
val seg = createSegment(0)
for(i <- 0 until 100)
- seg.append(i, Message.NoTimestamp, -1L, messages(i, i.toString))
+ seg.append(i, Record.NO_TIMESTAMP, -1L, records(i, i.toString))
val indexFile = seg.index.file
TestUtils.writeNonsenseToFile(indexFile, 5, indexFile.length.toInt)
seg.recover(64*1024)
for(i <- 0 until 100)
- assertEquals(i, seg.read(i, Some(i + 1), 1024).messageSet.head.offset)
+ assertEquals(i, seg.read(i, Some(i + 1), 1024).records.deepIterator.next().offset)
}
/**
@@ -261,7 +257,7 @@ class LogSegmentTest {
def testRecoveryFixesCorruptTimeIndex() {
val seg = createSegment(0)
for(i <- 0 until 100)
- seg.append(i, i * 10, i, messages(i, i.toString))
+ seg.append(i, i * 10, i, records(i, i.toString))
val timeIndexFile = seg.timeIndex.file
TestUtils.writeNonsenseToFile(timeIndexFile, 5, timeIndexFile.length.toInt)
seg.recover(64*1024)
@@ -271,7 +267,7 @@ class LogSegmentTest {
assertEquals(i + 1, seg.findOffsetByTimestamp(i * 10 + 1).get.offset)
}
}
-
+
/**
* Randomly corrupt a log a number of times and attempt recovery.
*/
@@ -281,13 +277,15 @@ class LogSegmentTest {
for (_ <- 0 until 10) {
val seg = createSegment(0)
for(i <- 0 until messagesAppended)
- seg.append(i, Message.NoTimestamp, -1L, messages(i, i.toString))
+ seg.append(i, Record.NO_TIMESTAMP, -1L, records(i, i.toString))
val offsetToBeginCorruption = TestUtils.random.nextInt(messagesAppended)
// start corrupting somewhere in the middle of the chosen record all the way to the end
- val position = seg.log.searchForOffsetWithSize(offsetToBeginCorruption, 0)._1.position + TestUtils.random.nextInt(15)
- TestUtils.writeNonsenseToFile(seg.log.file, position, seg.log.file.length.toInt - position)
+
+ val recordPosition = seg.log.searchForOffsetWithSize(offsetToBeginCorruption, 0)
+ val position = recordPosition.position + TestUtils.random.nextInt(15)
+ TestUtils.writeNonsenseToFile(seg.log.file, position, (seg.log.file.length - position).toInt)
seg.recover(64*1024)
- assertEquals("Should have truncated off bad messages.", (0 until offsetToBeginCorruption).toList, seg.log.map(_.offset).toList)
+ assertEquals("Should have truncated off bad messages.", (0 until offsetToBeginCorruption).toList, seg.log.shallowIterator.asScala.map(_.offset).toList)
seg.delete()
}
}
@@ -304,12 +302,12 @@ class LogSegmentTest {
@Test
def testCreateWithInitFileSizeAppendMessage() {
val seg = createSegment(40, false, 512*1024*1024, true)
- val ms = messages(50, "hello", "there")
- seg.append(50, Message.NoTimestamp, -1L, ms)
- val ms2 = messages(60, "alpha", "beta")
- seg.append(60, Message.NoTimestamp, -1L, ms2)
+ val ms = records(50, "hello", "there")
+ seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
+ val ms2 = records(60, "alpha", "beta")
+ seg.append(60, Record.NO_TIMESTAMP, -1L, ms2)
val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None)
- assertEquals(ms2.toList, read.messageSet.toList)
+ assertEquals(ms2.deepIterator.asScala.toList, read.records.deepIterator.asScala.toList)
}
/* create a segment with pre allocate and clearly shut down*/
@@ -318,12 +316,12 @@ class LogSegmentTest {
val tempDir = TestUtils.tempDir()
val seg = new LogSegment(tempDir, 40, 10, 1000, 0, Time.SYSTEM, false, 512*1024*1024, true)
- val ms = messages(50, "hello", "there")
- seg.append(50, Message.NoTimestamp, -1L, ms)
- val ms2 = messages(60, "alpha", "beta")
- seg.append(60, Message.NoTimestamp, -1L, ms2)
+ val ms = records(50, "hello", "there")
+ seg.append(50, Record.NO_TIMESTAMP, -1L, ms)
+ val ms2 = records(60, "alpha", "beta")
+ seg.append(60, Record.NO_TIMESTAMP, -1L, ms2)
val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None)
- assertEquals(ms2.toList, read.messageSet.toList)
+ assertEquals(ms2.deepIterator.asScala.toList, read.records.deepIterator.asScala.toList)
val oldSize = seg.log.sizeInBytes()
val oldPosition = seg.log.channel.position
val oldFileSize = seg.log.file.length
@@ -336,7 +334,7 @@ class LogSegmentTest {
segments += segReopen
val readAgain = segReopen.read(startOffset = 55, maxSize = 200, maxOffset = None)
- assertEquals(ms2.toList, readAgain.messageSet.toList)
+ assertEquals(ms2.deepIterator.asScala.toList, readAgain.records.deepIterator.asScala.toList)
val size = segReopen.log.sizeInBytes()
val position = segReopen.log.channel.position
val fileSize = segReopen.log.file.length
[8/9] kafka git commit: KAFKA-4390;
Replace MessageSet usage with client-side alternatives
Posted by jg...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/LogInputStream.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/LogInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/LogInputStream.java
index 4a4d569..a9af651 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/LogInputStream.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/LogInputStream.java
@@ -20,10 +20,14 @@ import java.io.IOException;
/**
* An abstraction between an underlying input stream and record iterators, a LogInputStream
- * returns only the shallow log entries, depending on {@link org.apache.kafka.common.record.RecordsIterator.DeepRecordsIterator}
- * for the deep iteration.
+ * returns only the shallow log entries, depending on {@link RecordsIterator.DeepRecordsIterator}
+ * for the deep iteration. The generic typing allows for implementations which present only
+ * a view of the log entries, which enables more efficient iteration when the record data is
+ * not actually needed. See for example {@link org.apache.kafka.common.record.FileLogInputStream.FileChannelLogEntry}
+ * in which the record is not brought into memory until needed.
+ * @param <T> Type parameter of the log entry
*/
-interface LogInputStream {
+interface LogInputStream<T extends LogEntry> {
/**
* Get the next log entry from the underlying input stream.
@@ -31,5 +35,5 @@ interface LogInputStream {
* @return The next log entry or null if there is none
* @throws IOException for any IO errors
*/
- LogEntry nextEntry() throws IOException;
+ T nextEntry() throws IOException;
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
index 65ccf98..b945062 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
@@ -12,197 +12,185 @@
*/
package org.apache.kafka.common.record;
-import java.io.DataInputStream;
+import org.apache.kafka.common.record.ByteBufferLogInputStream.ByteBufferLogEntry;
+
import java.io.IOException;
import java.nio.ByteBuffer;
import java.nio.channels.GatheringByteChannel;
+import java.util.ArrayList;
+import java.util.Arrays;
import java.util.Iterator;
+import java.util.List;
/**
- * A {@link Records} implementation backed by a ByteBuffer.
+ * A {@link Records} implementation backed by a ByteBuffer. This is used only for reading or
+ * modifying in-place an existing buffer of log entries. To create a new buffer see {@link MemoryRecordsBuilder},
+ * or one of the {@link #builder(ByteBuffer, byte, CompressionType, TimestampType) builder} variants.
*/
-public class MemoryRecords implements Records {
+public class MemoryRecords extends AbstractRecords {
public final static MemoryRecords EMPTY = MemoryRecords.readableRecords(ByteBuffer.allocate(0));
- private final static int WRITE_LIMIT_FOR_READABLE_ONLY = -1;
-
- // the compressor used for appends-only
- private final Compressor compressor;
-
- // the write limit for writable buffer, which may be smaller than the buffer capacity
- private final int writeLimit;
-
- // the capacity of the initial buffer, which is only used for de-allocation of writable records
- private final int initialCapacity;
-
// the underlying buffer used for read; while the records are still writable it is null
private ByteBuffer buffer;
-
- // indicate if the memory records is writable or not (i.e. used for appends or read-only)
- private boolean writable;
+ private int validBytes = -1;
// Construct a writable memory records
- private MemoryRecords(ByteBuffer buffer, CompressionType type, boolean writable, int writeLimit) {
- this.writable = writable;
- this.writeLimit = writeLimit;
- this.initialCapacity = buffer.capacity();
- if (this.writable) {
- this.buffer = null;
- this.compressor = new Compressor(buffer, type);
- } else {
- this.buffer = buffer;
- this.compressor = null;
- }
- }
-
- public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type, int writeLimit) {
- return new MemoryRecords(buffer, type, true, writeLimit);
- }
-
- public static MemoryRecords emptyRecords(ByteBuffer buffer, CompressionType type) {
- // use the buffer capacity as the default write limit
- return emptyRecords(buffer, type, buffer.capacity());
+ private MemoryRecords(ByteBuffer buffer) {
+ this.buffer = buffer;
}
- public static MemoryRecords readableRecords(ByteBuffer buffer) {
- return new MemoryRecords(buffer, CompressionType.NONE, false, WRITE_LIMIT_FOR_READABLE_ONLY);
+ @Override
+ public int sizeInBytes() {
+ return buffer.limit();
}
- /**
- * Append the given record and offset to the buffer
- */
- public void append(long offset, Record record) {
- if (!writable)
- throw new IllegalStateException("Memory records is not writable");
-
- int size = record.size();
- compressor.putLong(offset);
- compressor.putInt(size);
- compressor.put(record.buffer());
- compressor.recordWritten(size + Records.LOG_OVERHEAD);
- record.buffer().rewind();
+ @Override
+ public long writeTo(GatheringByteChannel channel, long position, int length) throws IOException {
+ ByteBuffer dup = buffer.duplicate();
+ int pos = (int) position;
+ dup.position(pos);
+ dup.limit(pos + length);
+ return channel.write(dup);
}
/**
- * Append a new record and offset to the buffer
- * @return crc of the record
+ * Write all records to the given channel (including partial records).
+ * @param channel The channel to write to
+ * @return The number of bytes written
+ * @throws IOException For any IO errors writing to the channel
*/
- public long append(long offset, long timestamp, byte[] key, byte[] value) {
- if (!writable)
- throw new IllegalStateException("Memory records is not writable");
-
- int size = Record.recordSize(key, value);
- compressor.putLong(offset);
- compressor.putInt(size);
- long crc = compressor.putRecord(timestamp, key, value);
- compressor.recordWritten(size + Records.LOG_OVERHEAD);
- return crc;
+ public int writeFullyTo(GatheringByteChannel channel) throws IOException {
+ buffer.mark();
+ int written = 0;
+ while (written < sizeInBytes())
+ written += channel.write(buffer);
+ buffer.reset();
+ return written;
}
/**
- * Check if we have room for a new record containing the given key/value pair
- *
- * Note that the return value is based on the estimate of the bytes written to the compressor, which may not be
- * accurate if compression is really used. When this happens, the following append may cause dynamic buffer
- * re-allocation in the underlying byte buffer stream.
- *
- * There is an exceptional case when appending a single message whose size is larger than the batch size, the
- * capacity will be the message size which is larger than the write limit, i.e. the batch size. In this case
- * the checking should be based on the capacity of the initialized buffer rather than the write limit in order
- * to accept this single record.
+ * The total number of bytes in this message set not including any partial, trailing messages. This
+ * may be smaller than what is returned by {@link #sizeInBytes()}.
+ * @return The number of valid bytes
*/
- public boolean hasRoomFor(byte[] key, byte[] value) {
- if (!this.writable)
- return false;
+ public int validBytes() {
+ if (validBytes >= 0)
+ return validBytes;
- return this.compressor.numRecordsWritten() == 0 ?
- this.initialCapacity >= Records.LOG_OVERHEAD + Record.recordSize(key, value) :
- this.writeLimit >= this.compressor.estimatedBytesWritten() + Records.LOG_OVERHEAD + Record.recordSize(key, value);
- }
+ int bytes = 0;
+ Iterator<ByteBufferLogEntry> iterator = shallowIterator();
+ while (iterator.hasNext())
+ bytes += iterator.next().sizeInBytes();
- public boolean isFull() {
- return !this.writable || this.writeLimit <= this.compressor.estimatedBytesWritten();
+ this.validBytes = bytes;
+ return bytes;
}
/**
- * Close this batch for no more appends
+ * Filter the records into the provided ByteBuffer.
+ * @param filter The filter function
+ * @param buffer The byte buffer to write the filtered records to
+ * @return A FilterResult with a summary of the output (for metrics)
*/
- public void close() {
- if (writable) {
- // close the compressor to fill-in wrapper message metadata if necessary
- compressor.close();
-
- // flip the underlying buffer to be ready for reads
- buffer = compressor.buffer();
- buffer.flip();
-
- // reset the writable flag
- writable = false;
+ public FilterResult filterTo(LogEntryFilter filter, ByteBuffer buffer) {
+ long maxTimestamp = Record.NO_TIMESTAMP;
+ long shallowOffsetOfMaxTimestamp = -1L;
+ int messagesRead = 0;
+ int bytesRead = 0;
+ int messagesRetained = 0;
+ int bytesRetained = 0;
+
+ Iterator<ByteBufferLogEntry> shallowIterator = shallowIterator();
+ while (shallowIterator.hasNext()) {
+ ByteBufferLogEntry shallowEntry = shallowIterator.next();
+ bytesRead += shallowEntry.sizeInBytes();
+
+ // We use the absolute offset to decide whether to retain the message or not (this is handled by the
+ // deep iterator). Because of KAFKA-4298, we have to allow for the possibility that a previous version
+ // corrupted the log by writing a compressed message set with a wrapper magic value not matching the magic
+ // of the inner messages. This will be fixed as we recopy the messages to the destination buffer.
+
+ Record shallowRecord = shallowEntry.record();
+ byte shallowMagic = shallowRecord.magic();
+ boolean writeOriginalEntry = true;
+ List<LogEntry> retainedEntries = new ArrayList<>();
+
+ for (LogEntry deepEntry : shallowEntry) {
+ Record deepRecord = deepEntry.record();
+ messagesRead += 1;
+
+ if (filter.shouldRetain(deepEntry)) {
+ // Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite
+ // the corrupted entry with correct data.
+ if (shallowMagic != deepRecord.magic())
+ writeOriginalEntry = false;
+
+ retainedEntries.add(deepEntry);
+ } else {
+ writeOriginalEntry = false;
+ }
+ }
+
+ if (writeOriginalEntry) {
+ // There are no messages compacted out and no message format conversion, write the original message set back
+ shallowEntry.writeTo(buffer);
+ messagesRetained += retainedEntries.size();
+ bytesRetained += shallowEntry.sizeInBytes();
+
+ if (shallowRecord.timestamp() > maxTimestamp) {
+ maxTimestamp = shallowRecord.timestamp();
+ shallowOffsetOfMaxTimestamp = shallowEntry.offset();
+ }
+ } else if (!retainedEntries.isEmpty()) {
+ ByteBuffer slice = buffer.slice();
+ MemoryRecordsBuilder builder = builderWithEntries(slice, shallowRecord.timestampType(), shallowRecord.compressionType(),
+ shallowRecord.timestamp(), retainedEntries);
+ MemoryRecords records = builder.build();
+ buffer.position(buffer.position() + slice.position());
+ messagesRetained += retainedEntries.size();
+ bytesRetained += records.sizeInBytes();
+
+ MemoryRecordsBuilder.RecordsInfo info = builder.info();
+ if (info.maxTimestamp > maxTimestamp) {
+ maxTimestamp = info.maxTimestamp;
+ shallowOffsetOfMaxTimestamp = info.shallowOffsetOfMaxTimestamp;
+ }
+ }
}
+
+ return new FilterResult(messagesRead, bytesRead, messagesRetained, bytesRetained, maxTimestamp, shallowOffsetOfMaxTimestamp);
}
/**
- * The size of this record set
+ * Get the byte buffer that backs this instance for reading.
*/
- @Override
- public int sizeInBytes() {
- if (writable) {
- return compressor.buffer().position();
- } else {
- return buffer.limit();
- }
+ public ByteBuffer buffer() {
+ return buffer.duplicate();
}
@Override
- public long writeTo(GatheringByteChannel channel, long offset, int length) throws IOException {
- ByteBuffer dup = buffer.duplicate();
- int position = (int) offset;
- dup.position(position);
- dup.limit(position + length);
- return channel.write(dup);
+ public Iterator<ByteBufferLogEntry> shallowIterator() {
+ return RecordsIterator.shallowIterator(new ByteBufferLogInputStream(buffer.duplicate(), Integer.MAX_VALUE));
}
- /**
- * The compression rate of this record set
- */
- public double compressionRate() {
- if (compressor == null)
- return 1.0;
- else
- return compressor.compressionRate();
+ @Override
+ public Iterator<LogEntry> deepIterator() {
+ return deepIterator(false);
}
- /**
- * Return the capacity of the initial buffer, for writable records
- * it may be different from the current buffer's capacity
- */
- public int initialCapacity() {
- return this.initialCapacity;
+ public Iterator<LogEntry> deepIterator(boolean ensureMatchingMagic) {
+ return deepIterator(ensureMatchingMagic, Integer.MAX_VALUE);
}
- /**
- * Get the byte buffer that backs this records instance for reading
- */
- public ByteBuffer buffer() {
- if (writable)
- throw new IllegalStateException("The memory records must not be writable any more before getting its underlying buffer");
-
- return buffer.duplicate();
+ public Iterator<LogEntry> deepIterator(boolean ensureMatchingMagic, int maxMessageSize) {
+ return new RecordsIterator(new ByteBufferLogInputStream(buffer.duplicate(), maxMessageSize), false,
+ ensureMatchingMagic, maxMessageSize);
}
@Override
- public Iterator<LogEntry> iterator() {
- ByteBuffer input = this.buffer.duplicate();
- if (writable)
- // flip on a duplicate buffer for reading
- input.flip();
- return new RecordsIterator(new ByteBufferLogInputStream(input), false);
- }
-
- @Override
public String toString() {
- Iterator<LogEntry> iter = iterator();
+ Iterator<LogEntry> iter = deepIterator();
StringBuilder builder = new StringBuilder();
builder.append('[');
while (iter.hasNext()) {
@@ -214,16 +202,13 @@ public class MemoryRecords implements Records {
builder.append("record=");
builder.append(entry.record());
builder.append(")");
+ if (iter.hasNext())
+ builder.append(", ");
}
builder.append(']');
return builder.toString();
}
- /** Visible for testing */
- public boolean isWritable() {
- return writable;
- }
-
@Override
public boolean equals(Object o) {
if (this == o) return true;
@@ -232,7 +217,6 @@ public class MemoryRecords implements Records {
MemoryRecords that = (MemoryRecords) o;
return buffer.equals(that.buffer);
-
}
@Override
@@ -240,28 +224,153 @@ public class MemoryRecords implements Records {
return buffer.hashCode();
}
- private static class ByteBufferLogInputStream implements LogInputStream {
- private final DataInputStream stream;
- private final ByteBuffer buffer;
+ public interface LogEntryFilter {
+ boolean shouldRetain(LogEntry entry);
+ }
- private ByteBufferLogInputStream(ByteBuffer buffer) {
- this.stream = new DataInputStream(new ByteBufferInputStream(buffer));
- this.buffer = buffer;
+ public static class FilterResult {
+ public final int messagesRead;
+ public final int bytesRead;
+ public final int messagesRetained;
+ public final int bytesRetained;
+ public final long maxTimestamp;
+ public final long shallowOffsetOfMaxTimestamp;
+
+ public FilterResult(int messagesRead,
+ int bytesRead,
+ int messagesRetained,
+ int bytesRetained,
+ long maxTimestamp,
+ long shallowOffsetOfMaxTimestamp) {
+ this.messagesRead = messagesRead;
+ this.bytesRead = bytesRead;
+ this.messagesRetained = messagesRetained;
+ this.bytesRetained = bytesRetained;
+ this.maxTimestamp = maxTimestamp;
+ this.shallowOffsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp;
}
+ }
- public LogEntry nextEntry() throws IOException {
- long offset = stream.readLong();
- int size = stream.readInt();
- if (size < 0)
- throw new IllegalStateException("Record with size " + size);
-
- ByteBuffer slice = buffer.slice();
- int newPos = buffer.position() + size;
- if (newPos > buffer.limit())
- return null;
- buffer.position(newPos);
- slice.limit(size);
- return new LogEntry(offset, new Record(slice));
- }
+ public static MemoryRecordsBuilder builder(ByteBuffer buffer,
+ CompressionType compressionType,
+ TimestampType timestampType,
+ int writeLimit) {
+ return new MemoryRecordsBuilder(buffer, Record.CURRENT_MAGIC_VALUE, compressionType, timestampType, 0L, System.currentTimeMillis(), writeLimit);
+ }
+
+ public static MemoryRecordsBuilder builder(ByteBuffer buffer,
+ byte magic,
+ CompressionType compressionType,
+ TimestampType timestampType,
+ long baseOffset,
+ long logAppendTime) {
+ return new MemoryRecordsBuilder(buffer, magic, compressionType, timestampType, baseOffset, logAppendTime, buffer.capacity());
+ }
+
+ public static MemoryRecordsBuilder builder(ByteBuffer buffer,
+ CompressionType compressionType,
+ TimestampType timestampType) {
+ // use the buffer capacity as the default write limit
+ return builder(buffer, compressionType, timestampType, buffer.capacity());
+ }
+
+ public static MemoryRecordsBuilder builder(ByteBuffer buffer,
+ byte magic,
+ CompressionType compressionType,
+ TimestampType timestampType) {
+ return builder(buffer, magic, compressionType, timestampType, 0L);
+ }
+
+ public static MemoryRecordsBuilder builder(ByteBuffer buffer,
+ byte magic,
+ CompressionType compressionType,
+ TimestampType timestampType,
+ long baseOffset) {
+ return builder(buffer, magic, compressionType, timestampType, baseOffset, System.currentTimeMillis());
+ }
+
+ public static MemoryRecords readableRecords(ByteBuffer buffer) {
+ return new MemoryRecords(buffer);
+ }
+
+ public static MemoryRecords withLogEntries(CompressionType compressionType, List<LogEntry> entries) {
+ return withLogEntries(TimestampType.CREATE_TIME, compressionType, System.currentTimeMillis(), entries);
+ }
+
+ public static MemoryRecords withLogEntries(LogEntry ... entries) {
+ return withLogEntries(CompressionType.NONE, Arrays.asList(entries));
+ }
+
+ public static MemoryRecords withRecords(CompressionType compressionType, long initialOffset, List<Record> records) {
+ return withRecords(initialOffset, TimestampType.CREATE_TIME, compressionType, System.currentTimeMillis(), records);
}
+
+ public static MemoryRecords withRecords(Record ... records) {
+ return withRecords(CompressionType.NONE, 0L, Arrays.asList(records));
+ }
+
+ public static MemoryRecords withRecords(long initialOffset, Record ... records) {
+ return withRecords(CompressionType.NONE, initialOffset, Arrays.asList(records));
+ }
+
+ public static MemoryRecords withRecords(CompressionType compressionType, Record ... records) {
+ return withRecords(compressionType, 0L, Arrays.asList(records));
+ }
+
+ public static MemoryRecords withRecords(TimestampType timestampType, CompressionType compressionType, Record ... records) {
+ return withRecords(0L, timestampType, compressionType, System.currentTimeMillis(), Arrays.asList(records));
+ }
+
+ public static MemoryRecords withRecords(long initialOffset,
+ TimestampType timestampType,
+ CompressionType compressionType,
+ long logAppendTime,
+ List<Record> records) {
+ return withLogEntries(timestampType, compressionType, logAppendTime, buildLogEntries(initialOffset, records));
+ }
+
+ private static MemoryRecords withLogEntries(TimestampType timestampType,
+ CompressionType compressionType,
+ long logAppendTime,
+ List<LogEntry> entries) {
+ if (entries.isEmpty())
+ return MemoryRecords.EMPTY;
+ return builderWithEntries(timestampType, compressionType, logAppendTime, entries).build();
+ }
+
+ private static List<LogEntry> buildLogEntries(long initialOffset, List<Record> records) {
+ List<LogEntry> entries = new ArrayList<>();
+ for (Record record : records)
+ entries.add(LogEntry.create(initialOffset++, record));
+ return entries;
+ }
+
+ public static MemoryRecordsBuilder builderWithEntries(TimestampType timestampType,
+ CompressionType compressionType,
+ long logAppendTime,
+ List<LogEntry> entries) {
+ ByteBuffer buffer = ByteBuffer.allocate(estimatedSize(compressionType, entries));
+ return builderWithEntries(buffer, timestampType, compressionType, logAppendTime, entries);
+ }
+
+ private static MemoryRecordsBuilder builderWithEntries(ByteBuffer buffer,
+ TimestampType timestampType,
+ CompressionType compressionType,
+ long logAppendTime,
+ List<LogEntry> entries) {
+ if (entries.isEmpty())
+ throw new IllegalArgumentException();
+
+ LogEntry firstEntry = entries.iterator().next();
+ long firstOffset = firstEntry.offset();
+ byte magic = firstEntry.record().magic();
+
+ MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compressionType, timestampType,
+ firstOffset, logAppendTime);
+ for (LogEntry entry : entries)
+ builder.append(entry);
+
+ return builder;
+ }
+
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java
new file mode 100644
index 0000000..b90a9e6
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecordsBuilder.java
@@ -0,0 +1,461 @@
+/**
+ * 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 org.apache.kafka.common.record;
+
+import org.apache.kafka.common.KafkaException;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.io.OutputStream;
+import java.lang.reflect.Constructor;
+import java.nio.ByteBuffer;
+import java.util.zip.GZIPInputStream;
+import java.util.zip.GZIPOutputStream;
+
+/**
+ * This class is used to write new log data in memory, i.e. this is the write path for {@link MemoryRecords}.
+ * It transparently handles compression and exposes methods for appending new entries, possibly with message
+ * format conversion.
+ */
+public class MemoryRecordsBuilder {
+
+ static private final float COMPRESSION_RATE_DAMPING_FACTOR = 0.9f;
+ static private final float COMPRESSION_RATE_ESTIMATION_FACTOR = 1.05f;
+ static private final int COMPRESSION_DEFAULT_BUFFER_SIZE = 1024;
+
+ private static final float[] TYPE_TO_RATE;
+
+ static {
+ int maxTypeId = -1;
+ for (CompressionType type : CompressionType.values())
+ maxTypeId = Math.max(maxTypeId, type.id);
+ TYPE_TO_RATE = new float[maxTypeId + 1];
+ for (CompressionType type : CompressionType.values()) {
+ TYPE_TO_RATE[type.id] = type.rate;
+ }
+ }
+
+ // dynamically load the snappy and lz4 classes to avoid runtime dependency if we are not using compression
+ // caching constructors to avoid invoking of Class.forName method for each batch
+ private static MemoizingConstructorSupplier snappyOutputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
+ @Override
+ public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
+ return Class.forName("org.xerial.snappy.SnappyOutputStream")
+ .getConstructor(OutputStream.class, Integer.TYPE);
+ }
+ });
+
+ private static MemoizingConstructorSupplier lz4OutputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
+ @Override
+ public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
+ return Class.forName("org.apache.kafka.common.record.KafkaLZ4BlockOutputStream")
+ .getConstructor(OutputStream.class, Boolean.TYPE);
+ }
+ });
+
+ private static MemoizingConstructorSupplier snappyInputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
+ @Override
+ public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
+ return Class.forName("org.xerial.snappy.SnappyInputStream")
+ .getConstructor(InputStream.class);
+ }
+ });
+
+ private static MemoizingConstructorSupplier lz4InputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
+ @Override
+ public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
+ return Class.forName("org.apache.kafka.common.record.KafkaLZ4BlockInputStream")
+ .getConstructor(InputStream.class, Boolean.TYPE);
+ }
+ });
+
+ private final TimestampType timestampType;
+ private final CompressionType compressionType;
+ private final DataOutputStream appendStream;
+ private final ByteBufferOutputStream bufferStream;
+ private final byte magic;
+ private final int initPos;
+ private final long baseOffset;
+ private final long logAppendTime;
+ private final int writeLimit;
+ private final int initialCapacity;
+
+ private MemoryRecords builtRecords;
+ private long writtenUncompressed;
+ private long numRecords;
+ private float compressionRate;
+ private long maxTimestamp;
+ private long offsetOfMaxTimestamp;
+ private long lastOffset = -1;
+
+ public MemoryRecordsBuilder(ByteBuffer buffer,
+ byte magic,
+ CompressionType compressionType,
+ TimestampType timestampType,
+ long baseOffset,
+ long logAppendTime,
+ int writeLimit) {
+ this.magic = magic;
+ this.timestampType = timestampType;
+ this.compressionType = compressionType;
+ this.baseOffset = baseOffset;
+ this.logAppendTime = logAppendTime;
+ this.initPos = buffer.position();
+ this.numRecords = 0;
+ this.writtenUncompressed = 0;
+ this.compressionRate = 1;
+ this.maxTimestamp = Record.NO_TIMESTAMP;
+ this.writeLimit = writeLimit;
+ this.initialCapacity = buffer.capacity();
+
+ if (compressionType != CompressionType.NONE) {
+ // for compressed records, leave space for the header and the shallow message metadata
+ // and move the starting position to the value payload offset
+ buffer.position(initPos + Records.LOG_OVERHEAD + Record.recordOverhead(magic));
+ }
+
+ // create the stream
+ bufferStream = new ByteBufferOutputStream(buffer);
+ appendStream = wrapForOutput(bufferStream, compressionType, magic, COMPRESSION_DEFAULT_BUFFER_SIZE);
+ }
+
+ public ByteBuffer buffer() {
+ return bufferStream.buffer();
+ }
+
+ public int initialCapacity() {
+ return initialCapacity;
+ }
+
+ public double compressionRate() {
+ return compressionRate;
+ }
+
+ /**
+ * Close this builder and return the resulting buffer.
+ * @return The built log buffer
+ */
+ public MemoryRecords build() {
+ close();
+ return builtRecords;
+ }
+
+ /**
+ * Get the max timestamp and its offset. If the log append time is used, then the offset will
+ * be either the first offset in the set if no compression is used or the last offset otherwise.
+ * @return The max timestamp and its offset
+ */
+ public RecordsInfo info() {
+ if (timestampType == TimestampType.LOG_APPEND_TIME)
+ return new RecordsInfo(logAppendTime, lastOffset);
+ else
+ return new RecordsInfo(maxTimestamp, compressionType == CompressionType.NONE ? offsetOfMaxTimestamp : lastOffset);
+ }
+
+ public void close() {
+ if (builtRecords != null)
+ return;
+
+ try {
+ appendStream.close();
+ } catch (IOException e) {
+ throw new KafkaException(e);
+ }
+
+ if (compressionType != CompressionType.NONE)
+ writerCompressedWrapperHeader();
+
+ ByteBuffer buffer = buffer().duplicate();
+ buffer.flip();
+ buffer.position(initPos);
+ builtRecords = MemoryRecords.readableRecords(buffer.slice());
+ }
+
+ private void writerCompressedWrapperHeader() {
+ ByteBuffer buffer = bufferStream.buffer();
+ int pos = buffer.position();
+ buffer.position(initPos);
+
+ int wrapperSize = pos - initPos - Records.LOG_OVERHEAD;
+ int writtenCompressed = wrapperSize - Record.recordOverhead(magic);
+ LogEntry.writeHeader(buffer, lastOffset, wrapperSize);
+
+ long timestamp = timestampType == TimestampType.LOG_APPEND_TIME ? logAppendTime : maxTimestamp;
+ Record.writeCompressedRecordHeader(buffer, magic, wrapperSize, timestamp, compressionType, timestampType);
+
+ buffer.position(pos);
+
+ // update the compression ratio
+ this.compressionRate = (float) writtenCompressed / this.writtenUncompressed;
+ TYPE_TO_RATE[compressionType.id] = TYPE_TO_RATE[compressionType.id] * COMPRESSION_RATE_DAMPING_FACTOR +
+ compressionRate * (1 - COMPRESSION_RATE_DAMPING_FACTOR);
+ }
+
+ /**
+ * Append a new record and offset to the buffer
+ * @param offset The absolute offset of the record in the log buffer
+ * @param timestamp The record timestamp
+ * @param key The record key
+ * @param value The record value
+ * @return crc of the record
+ */
+ public long append(long offset, long timestamp, byte[] key, byte[] value) {
+ try {
+ if (lastOffset > 0 && offset <= lastOffset)
+ throw new IllegalArgumentException(String.format("Illegal offset %s following previous offset %s (Offsets must increase monotonically).", offset, lastOffset));
+
+ int size = Record.recordSize(magic, key, value);
+ LogEntry.writeHeader(appendStream, toInnerOffset(offset), size);
+
+ if (timestampType == TimestampType.LOG_APPEND_TIME)
+ timestamp = logAppendTime;
+ long crc = Record.write(appendStream, magic, timestamp, key, value, CompressionType.NONE, timestampType);
+ recordWritten(offset, timestamp, size + Records.LOG_OVERHEAD);
+ return crc;
+ } catch (IOException e) {
+ throw new KafkaException("I/O exception when writing to the append stream, closing", e);
+ }
+ }
+
+ /**
+ * Add the record, converting to the desired magic value if necessary.
+ * @param offset The offset of the record
+ * @param record The record to add
+ */
+ public void convertAndAppend(long offset, Record record) {
+ if (magic == record.magic()) {
+ append(offset, record);
+ return;
+ }
+
+ if (lastOffset > 0 && offset <= lastOffset)
+ throw new IllegalArgumentException(String.format("Illegal offset %s following previous offset %s (Offsets must increase monotonically).", offset, lastOffset));
+
+ try {
+ int size = record.convertedSize(magic);
+ LogEntry.writeHeader(appendStream, toInnerOffset(offset), size);
+ long timestamp = timestampType == TimestampType.LOG_APPEND_TIME ? logAppendTime : record.timestamp();
+ record.convertTo(appendStream, magic, timestamp, timestampType);
+ recordWritten(offset, timestamp, size + Records.LOG_OVERHEAD);
+ } catch (IOException e) {
+ throw new KafkaException("I/O exception when writing to the append stream, closing", e);
+ }
+ }
+
+ /**
+ * Add a record without doing offset/magic validation (this should only be used in testing).
+ * @param offset The offset of the record
+ * @param record The record to add
+ */
+ public void appendUnchecked(long offset, Record record) {
+ try {
+ int size = record.sizeInBytes();
+ LogEntry.writeHeader(appendStream, toInnerOffset(offset), size);
+
+ ByteBuffer buffer = record.buffer().duplicate();
+ appendStream.write(buffer.array(), buffer.arrayOffset(), buffer.limit());
+
+ recordWritten(offset, record.timestamp(), size + Records.LOG_OVERHEAD);
+ } catch (IOException e) {
+ throw new KafkaException("I/O exception when writing to the append stream, closing", e);
+ }
+ }
+
+ /**
+ * Append the given log entry. The entry's record must have a magic which matches the magic use to
+ * construct this builder and the offset must be greater than the last appended entry.
+ * @param entry The entry to append
+ */
+ public void append(LogEntry entry) {
+ append(entry.offset(), entry.record());
+ }
+
+ /**
+ * Add a record with a given offset. The record must have a magic which matches the magic use to
+ * construct this builder and the offset must be greater than the last appended entry.
+ * @param offset The offset of the record
+ * @param record The record to add
+ */
+ public void append(long offset, Record record) {
+ if (record.magic() != magic)
+ throw new IllegalArgumentException("Inner log entries must have matching magic values as the wrapper");
+ if (lastOffset > 0 && offset <= lastOffset)
+ throw new IllegalArgumentException(String.format("Illegal offset %s following previous offset %s (Offsets must increase monotonically).", offset, lastOffset));
+ appendUnchecked(offset, record);
+ }
+
+ private long toInnerOffset(long offset) {
+ // use relative offsets for compressed messages with magic v1
+ if (magic > 0 && compressionType != CompressionType.NONE)
+ return offset - baseOffset;
+ return offset;
+ }
+
+ private void recordWritten(long offset, long timestamp, int size) {
+ numRecords += 1;
+ writtenUncompressed += size;
+ lastOffset = offset;
+
+ if (timestamp > maxTimestamp) {
+ maxTimestamp = timestamp;
+ offsetOfMaxTimestamp = offset;
+ }
+ }
+
+ /**
+ * Get an estimate of the number of bytes written (based on the estimation factor hard-coded in {@link CompressionType}.
+ * @return The estimated number of bytes written
+ */
+ private int estimatedBytesWritten() {
+ if (compressionType == CompressionType.NONE) {
+ return buffer().position();
+ } else {
+ // estimate the written bytes to the underlying byte buffer based on uncompressed written bytes
+ return (int) (writtenUncompressed * TYPE_TO_RATE[compressionType.id] * COMPRESSION_RATE_ESTIMATION_FACTOR);
+ }
+ }
+
+ /**
+ * Check if we have room for a new record containing the given key/value pair
+ *
+ * Note that the return value is based on the estimate of the bytes written to the compressor, which may not be
+ * accurate if compression is really used. When this happens, the following append may cause dynamic buffer
+ * re-allocation in the underlying byte buffer stream.
+ *
+ * There is an exceptional case when appending a single message whose size is larger than the batch size, the
+ * capacity will be the message size which is larger than the write limit, i.e. the batch size. In this case
+ * the checking should be based on the capacity of the initialized buffer rather than the write limit in order
+ * to accept this single record.
+ */
+ public boolean hasRoomFor(byte[] key, byte[] value) {
+ return !isFull() && (numRecords == 0 ?
+ this.initialCapacity >= Records.LOG_OVERHEAD + Record.recordSize(magic, key, value) :
+ this.writeLimit >= estimatedBytesWritten() + Records.LOG_OVERHEAD + Record.recordSize(magic, key, value));
+ }
+
+ public boolean isClosed() {
+ return builtRecords != null;
+ }
+
+ public boolean isFull() {
+ return isClosed() || this.writeLimit <= estimatedBytesWritten();
+ }
+
+ public int sizeInBytes() {
+ return builtRecords != null ? builtRecords.sizeInBytes() : estimatedBytesWritten();
+ }
+
+ private static DataOutputStream wrapForOutput(ByteBufferOutputStream buffer, CompressionType type, byte messageVersion, int bufferSize) {
+ try {
+ switch (type) {
+ case NONE:
+ return buffer;
+ case GZIP:
+ return new DataOutputStream(new GZIPOutputStream(buffer, bufferSize));
+ case SNAPPY:
+ try {
+ OutputStream stream = (OutputStream) snappyOutputStreamSupplier.get().newInstance(buffer, bufferSize);
+ return new DataOutputStream(stream);
+ } catch (Exception e) {
+ throw new KafkaException(e);
+ }
+ case LZ4:
+ try {
+ OutputStream stream = (OutputStream) lz4OutputStreamSupplier.get().newInstance(buffer,
+ messageVersion == Record.MAGIC_VALUE_V0);
+ return new DataOutputStream(stream);
+ } catch (Exception e) {
+ throw new KafkaException(e);
+ }
+ default:
+ throw new IllegalArgumentException("Unknown compression type: " + type);
+ }
+ } catch (IOException e) {
+ throw new KafkaException(e);
+ }
+ }
+
+ public static DataInputStream wrapForInput(ByteBufferInputStream buffer, CompressionType type, byte messageVersion) {
+ try {
+ switch (type) {
+ case NONE:
+ return buffer;
+ case GZIP:
+ return new DataInputStream(new GZIPInputStream(buffer));
+ case SNAPPY:
+ try {
+ InputStream stream = (InputStream) snappyInputStreamSupplier.get().newInstance(buffer);
+ return new DataInputStream(stream);
+ } catch (Exception e) {
+ throw new KafkaException(e);
+ }
+ case LZ4:
+ try {
+ InputStream stream = (InputStream) lz4InputStreamSupplier.get().newInstance(buffer,
+ messageVersion == Record.MAGIC_VALUE_V0);
+ return new DataInputStream(stream);
+ } catch (Exception e) {
+ throw new KafkaException(e);
+ }
+ default:
+ throw new IllegalArgumentException("Unknown compression type: " + type);
+ }
+ } catch (IOException e) {
+ throw new KafkaException(e);
+ }
+ }
+
+ private interface ConstructorSupplier {
+ Constructor get() throws ClassNotFoundException, NoSuchMethodException;
+ }
+
+ // this code is based on Guava's @see{com.google.common.base.Suppliers.MemoizingSupplier}
+ private static class MemoizingConstructorSupplier {
+ final ConstructorSupplier delegate;
+ transient volatile boolean initialized;
+ transient Constructor value;
+
+ public MemoizingConstructorSupplier(ConstructorSupplier delegate) {
+ this.delegate = delegate;
+ }
+
+ public Constructor get() throws NoSuchMethodException, ClassNotFoundException {
+ if (!initialized) {
+ synchronized (this) {
+ if (!initialized) {
+ value = delegate.get();
+ initialized = true;
+ }
+ }
+ }
+ return value;
+ }
+ }
+
+ public static class RecordsInfo {
+ public final long maxTimestamp;
+ public final long shallowOffsetOfMaxTimestamp;
+
+ public RecordsInfo(long maxTimestamp,
+ long shallowOffsetOfMaxTimestamp) {
+ this.maxTimestamp = maxTimestamp;
+ this.shallowOffsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp;
+ }
+ }
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/Record.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/Record.java b/clients/src/main/java/org/apache/kafka/common/record/Record.java
index 09cb80d..0c0fa3c 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/Record.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/Record.java
@@ -16,11 +16,15 @@
*/
package org.apache.kafka.common.record;
-import java.nio.ByteBuffer;
-
+import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.utils.Crc32;
import org.apache.kafka.common.utils.Utils;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import static org.apache.kafka.common.utils.Utils.wrapNullable;
/**
* A record: a serialized key and value along with the associated CRC and other fields
@@ -53,7 +57,12 @@ public final class Record {
/**
* The amount of overhead bytes in a record
*/
- public static final int RECORD_OVERHEAD = HEADER_SIZE + TIMESTAMP_LENGTH + KEY_SIZE_LENGTH + VALUE_SIZE_LENGTH;
+ public static final int RECORD_OVERHEAD_V0 = HEADER_SIZE + KEY_SIZE_LENGTH + VALUE_SIZE_LENGTH;
+
+ /**
+ * The amount of overhead bytes in a record
+ */
+ public static final int RECORD_OVERHEAD_V1 = HEADER_SIZE + TIMESTAMP_LENGTH + KEY_SIZE_LENGTH + VALUE_SIZE_LENGTH;
/**
* The "magic" values
@@ -80,11 +89,6 @@ public final class Record {
public static final int TIMESTAMP_TYPE_ATTRIBUTE_OFFSET = 3;
/**
- * Compression code for uncompressed records
- */
- public static final int NO_COMPRESSION = 0;
-
- /**
* Timestamp value for records without a timestamp
*/
public static final long NO_TIMESTAMP = -1L;
@@ -94,155 +98,20 @@ public final class Record {
private final TimestampType wrapperRecordTimestampType;
public Record(ByteBuffer buffer) {
- this.buffer = buffer;
- this.wrapperRecordTimestamp = null;
- this.wrapperRecordTimestampType = null;
+ this(buffer, null, null);
}
- // Package private constructor for inner iteration.
- Record(ByteBuffer buffer, Long wrapperRecordTimestamp, TimestampType wrapperRecordTimestampType) {
+ public Record(ByteBuffer buffer, Long wrapperRecordTimestamp, TimestampType wrapperRecordTimestampType) {
this.buffer = buffer;
this.wrapperRecordTimestamp = wrapperRecordTimestamp;
this.wrapperRecordTimestampType = wrapperRecordTimestampType;
}
/**
- * A constructor to create a LogRecord. If the record's compression type is not none, then
- * its value payload should be already compressed with the specified type; the constructor
- * would always write the value payload as is and will not do the compression itself.
- *
- * @param timestamp The timestamp of the record
- * @param key The key of the record (null, if none)
- * @param value The record value
- * @param type The compression type used on the contents of the record (if any)
- * @param valueOffset The offset into the payload array used to extract payload
- * @param valueSize The size of the payload to use
- */
- public Record(long timestamp, byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
- this(ByteBuffer.allocate(recordSize(key == null ? 0 : key.length,
- value == null ? 0 : valueSize >= 0 ? valueSize : value.length - valueOffset)));
- write(this.buffer, timestamp, key, value, type, valueOffset, valueSize);
- this.buffer.rewind();
- }
-
- public Record(long timestamp, byte[] key, byte[] value, CompressionType type) {
- this(timestamp, key, value, type, 0, -1);
- }
-
- public Record(long timestamp, byte[] value, CompressionType type) {
- this(timestamp, null, value, type);
- }
-
- public Record(long timestamp, byte[] key, byte[] value) {
- this(timestamp, key, value, CompressionType.NONE);
- }
-
- public Record(long timestamp, byte[] value) {
- this(timestamp, null, value, CompressionType.NONE);
- }
-
- // Write a record to the buffer, if the record's compression type is none, then
- // its value payload should be already compressed with the specified type
- public static void write(ByteBuffer buffer, long timestamp, byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
- // construct the compressor with compression type none since this function will not do any
- //compression according to the input type, it will just write the record's payload as is
- Compressor compressor = new Compressor(buffer, CompressionType.NONE);
- try {
- compressor.putRecord(timestamp, key, value, type, valueOffset, valueSize);
- } finally {
- compressor.close();
- }
- }
-
- public static void write(Compressor compressor, long crc, byte attributes, long timestamp, byte[] key, byte[] value, int valueOffset, int valueSize) {
- // write crc
- compressor.putInt((int) (crc & 0xffffffffL));
- // write magic value
- compressor.putByte(CURRENT_MAGIC_VALUE);
- // write attributes
- compressor.putByte(attributes);
- // write timestamp
- compressor.putLong(timestamp);
- // write the key
- if (key == null) {
- compressor.putInt(-1);
- } else {
- compressor.putInt(key.length);
- compressor.put(key, 0, key.length);
- }
- // write the value
- if (value == null) {
- compressor.putInt(-1);
- } else {
- int size = valueSize >= 0 ? valueSize : (value.length - valueOffset);
- compressor.putInt(size);
- compressor.put(value, valueOffset, size);
- }
- }
-
- public static int recordSize(byte[] key, byte[] value) {
- return recordSize(key == null ? 0 : key.length, value == null ? 0 : value.length);
- }
-
- public static int recordSize(int keySize, int valueSize) {
- return CRC_LENGTH + MAGIC_LENGTH + ATTRIBUTE_LENGTH + TIMESTAMP_LENGTH + KEY_SIZE_LENGTH + keySize + VALUE_SIZE_LENGTH + valueSize;
- }
-
- public ByteBuffer buffer() {
- return this.buffer;
- }
-
- public static byte computeAttributes(CompressionType type) {
- byte attributes = 0;
- if (type.id > 0)
- attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & type.id));
- return attributes;
- }
-
- /**
- * Compute the checksum of the record from the record contents
- */
- public static long computeChecksum(ByteBuffer buffer, int position, int size) {
- Crc32 crc = new Crc32();
- crc.update(buffer.array(), buffer.arrayOffset() + position, size);
- return crc.getValue();
- }
-
- /**
- * Compute the checksum of the record from the attributes, key and value payloads
- */
- public static long computeChecksum(long timestamp, byte[] key, byte[] value, CompressionType type, int valueOffset, int valueSize) {
- Crc32 crc = new Crc32();
- crc.update(CURRENT_MAGIC_VALUE);
- byte attributes = 0;
- if (type.id > 0)
- attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & type.id));
- crc.update(attributes);
- crc.updateLong(timestamp);
- // update for the key
- if (key == null) {
- crc.updateInt(-1);
- } else {
- crc.updateInt(key.length);
- crc.update(key, 0, key.length);
- }
- // update for the value
- if (value == null) {
- crc.updateInt(-1);
- } else {
- int size = valueSize >= 0 ? valueSize : (value.length - valueOffset);
- crc.updateInt(size);
- crc.update(value, valueOffset, size);
- }
- return crc.getValue();
- }
-
-
- /**
* Compute the checksum of the record from the record contents
*/
public long computeChecksum() {
- return computeChecksum(buffer, MAGIC_OFFSET, buffer.limit() - MAGIC_OFFSET);
+ return Utils.computeChecksum(buffer, MAGIC_OFFSET, buffer.limit() - MAGIC_OFFSET);
}
/**
@@ -256,7 +125,15 @@ public final class Record {
* Returns true if the crc stored with the record matches the crc computed off the record contents
*/
public boolean isValid() {
- return size() >= CRC_LENGTH && checksum() == computeChecksum();
+ return sizeInBytes() >= CRC_LENGTH && checksum() == computeChecksum();
+ }
+
+ public Long wrapperRecordTimestamp() {
+ return wrapperRecordTimestamp;
+ }
+
+ public TimestampType wrapperRecordTimestampType() {
+ return wrapperRecordTimestampType;
}
/**
@@ -264,9 +141,9 @@ public final class Record {
*/
public void ensureValid() {
if (!isValid()) {
- if (size() < CRC_LENGTH)
+ if (sizeInBytes() < CRC_LENGTH)
throw new InvalidRecordException("Record is corrupt (crc could not be retrieved as the record is too "
- + "small, size = " + size() + ")");
+ + "small, size = " + sizeInBytes() + ")");
else
throw new InvalidRecordException("Record is corrupt (stored crc = " + checksum()
+ ", computed crc = " + computeChecksum() + ")");
@@ -274,14 +151,17 @@ public final class Record {
}
/**
- * The complete serialized size of this record in bytes (including crc, header attributes, etc)
+ * The complete serialized size of this record in bytes (including crc, header attributes, etc), but
+ * excluding the log overhead (offset and record size).
+ * @return the size in bytes
*/
- public int size() {
+ public int sizeInBytes() {
return buffer.limit();
}
/**
* The length of the key in bytes
+ * @return the size in bytes of the key (0 if the key is null)
*/
public int keySize() {
if (magic() == MAGIC_VALUE_V0)
@@ -292,6 +172,7 @@ public final class Record {
/**
* Does the record have a key?
+ * @return true if so, false otherwise
*/
public boolean hasKey() {
return keySize() >= 0;
@@ -309,13 +190,23 @@ public final class Record {
/**
* The length of the value in bytes
+ * @return the size in bytes of the value (0 if the value is null)
*/
public int valueSize() {
return buffer.getInt(valueSizeOffset());
}
/**
- * The magic version of this record
+ * Check whether the value field of this record is null.
+ * @return true if the value is null, false otherwise
+ */
+ public boolean hasNullValue() {
+ return valueSize() < 0;
+ }
+
+ /**
+ * The magic value (i.e. message format version) of this record
+ * @return the magic value
*/
public byte magic() {
return buffer.get(MAGIC_OFFSET);
@@ -323,6 +214,7 @@ public final class Record {
/**
* The attributes stored with this record
+ * @return the attributes
*/
public byte attributes() {
return buffer.get(ATTRIBUTES_OFFSET);
@@ -333,6 +225,8 @@ public final class Record {
* 1. wrapperRecordTimestampType = null and wrapperRecordTimestamp is null - Uncompressed message, timestamp is in the message.
* 2. wrapperRecordTimestampType = LOG_APPEND_TIME and WrapperRecordTimestamp is not null - Compressed message using LOG_APPEND_TIME
* 3. wrapperRecordTimestampType = CREATE_TIME and wrapperRecordTimestamp is not null - Compressed message using CREATE_TIME
+ *
+ * @return the timestamp as determined above
*/
public long timestamp() {
if (magic() == MAGIC_VALUE_V0)
@@ -349,6 +243,8 @@ public final class Record {
/**
* The timestamp of the message.
+ * @return the timstamp type or {@link TimestampType#NO_TIMESTAMP_TYPE} if the magic is 0 or the message has
+ * been up-converted.
*/
public TimestampType timestampType() {
if (magic() == 0)
@@ -366,36 +262,30 @@ public final class Record {
/**
* A ByteBuffer containing the value of this record
+ * @return the value or null if the value for this record is null
*/
public ByteBuffer value() {
- return sliceDelimited(valueSizeOffset());
+ return Utils.sizeDelimited(buffer, valueSizeOffset());
}
/**
* A ByteBuffer containing the message key
+ * @return the buffer or null if the key for this record is null
*/
public ByteBuffer key() {
if (magic() == MAGIC_VALUE_V0)
- return sliceDelimited(KEY_SIZE_OFFSET_V0);
+ return Utils.sizeDelimited(buffer, KEY_SIZE_OFFSET_V0);
else
- return sliceDelimited(KEY_SIZE_OFFSET_V1);
+ return Utils.sizeDelimited(buffer, KEY_SIZE_OFFSET_V1);
}
/**
- * Read a size-delimited byte buffer starting at the given offset
+ * Get the underlying buffer backing this record instance.
+ *
+ * @return the buffer
*/
- private ByteBuffer sliceDelimited(int start) {
- int size = buffer.getInt(start);
- if (size < 0) {
- return null;
- } else {
- ByteBuffer b = buffer.duplicate();
- b.position(start + 4);
- b = b.slice();
- b.limit(size);
- b.rewind();
- return b;
- }
+ public ByteBuffer buffer() {
+ return this.buffer;
}
public String toString() {
@@ -434,4 +324,316 @@ public final class Record {
return buffer.hashCode();
}
+ /**
+ * Get the size of this record if converted to the given format.
+ *
+ * @param toMagic The target magic version to convert to
+ * @return The size in bytes after conversion
+ */
+ public int convertedSize(byte toMagic) {
+ return recordSize(toMagic, Math.max(0, keySize()), Math.max(0, valueSize()));
+ }
+
+ /**
+ * Convert this record to another message format.
+ *
+ * @param toMagic The target magic version to convert to
+ * @return A new record instance with a freshly allocated ByteBuffer.
+ */
+ public Record convert(byte toMagic) {
+ if (toMagic == magic())
+ return this;
+
+ ByteBuffer buffer = ByteBuffer.allocate(convertedSize(toMagic));
+ TimestampType timestampType = wrapperRecordTimestampType != null ?
+ wrapperRecordTimestampType : TimestampType.forAttributes(attributes());
+ convertTo(buffer, toMagic, timestamp(), timestampType);
+ buffer.rewind();
+ return new Record(buffer);
+ }
+
+ private void convertTo(ByteBuffer buffer, byte toMagic, long timestamp, TimestampType timestampType) {
+ if (compressionType() != CompressionType.NONE)
+ throw new IllegalArgumentException("Cannot use convertTo for deep conversion");
+
+ write(buffer, toMagic, timestamp, key(), value(), CompressionType.NONE, timestampType);
+ }
+
+ /**
+ * Convert this record to another message format and write the converted data to the provided outputs stream.
+ *
+ * @param out The output stream to write the converted data to
+ * @param toMagic The target magic version for conversion
+ * @param timestamp The timestamp to use in the converted record (for up-conversion)
+ * @param timestampType The timestamp type to use in the converted record (for up-conversion)
+ * @throws IOException for any IO errors writing the converted record.
+ */
+ public void convertTo(DataOutputStream out, byte toMagic, long timestamp, TimestampType timestampType) throws IOException {
+ if (compressionType() != CompressionType.NONE)
+ throw new IllegalArgumentException("Cannot use convertTo for deep conversion");
+
+ write(out, toMagic, timestamp, key(), value(), CompressionType.NONE, timestampType);
+ }
+
+ /**
+ * Create a new record instance. If the record's compression type is not none, then
+ * its value payload should be already compressed with the specified type; the constructor
+ * would always write the value payload as is and will not do the compression itself.
+ *
+ * @param magic The magic value to use
+ * @param timestamp The timestamp of the record
+ * @param key The key of the record (null, if none)
+ * @param value The record value
+ * @param compressionType The compression type used on the contents of the record (if any)
+ * @param timestampType The timestamp type to be used for this record
+ */
+ public static Record create(byte magic,
+ long timestamp,
+ byte[] key,
+ byte[] value,
+ CompressionType compressionType,
+ TimestampType timestampType) {
+ int keySize = key == null ? 0 : key.length;
+ int valueSize = value == null ? 0 : value.length;
+ ByteBuffer buffer = ByteBuffer.allocate(recordSize(magic, keySize, valueSize));
+ write(buffer, magic, timestamp, wrapNullable(key), wrapNullable(value), compressionType, timestampType);
+ buffer.rewind();
+ return new Record(buffer);
+ }
+
+ public static Record create(long timestamp, byte[] key, byte[] value) {
+ return create(CURRENT_MAGIC_VALUE, timestamp, key, value, CompressionType.NONE, TimestampType.CREATE_TIME);
+ }
+
+ public static Record create(byte magic, long timestamp, byte[] key, byte[] value) {
+ return create(magic, timestamp, key, value, CompressionType.NONE, TimestampType.CREATE_TIME);
+ }
+
+ public static Record create(byte magic, TimestampType timestampType, long timestamp, byte[] key, byte[] value) {
+ return create(magic, timestamp, key, value, CompressionType.NONE, timestampType);
+ }
+
+ public static Record create(byte magic, long timestamp, byte[] value) {
+ return create(magic, timestamp, null, value, CompressionType.NONE, TimestampType.CREATE_TIME);
+ }
+
+ public static Record create(byte magic, byte[] key, byte[] value) {
+ return create(magic, NO_TIMESTAMP, key, value);
+ }
+
+ public static Record create(byte[] key, byte[] value) {
+ return create(NO_TIMESTAMP, key, value);
+ }
+
+ public static Record create(byte[] value) {
+ return create(CURRENT_MAGIC_VALUE, NO_TIMESTAMP, null, value, CompressionType.NONE, TimestampType.CREATE_TIME);
+ }
+
+ /**
+ * Write the header for a compressed record set in-place (i.e. assuming the compressed record data has already
+ * been written at the value offset in a wrapped record). This lets you dynamically create a compressed message
+ * set, and then go back later and fill in its size and CRC, which saves the need for copying to another buffer.
+ *
+ * @param buffer The buffer containing the compressed record data positioned at the first offset of the
+ * @param magic The magic value of the record set
+ * @param recordSize The size of the record (including record overhead)
+ * @param timestamp The timestamp of the wrapper record
+ * @param compressionType The compression type used
+ * @param timestampType The timestamp type of the wrapper record
+ */
+ public static void writeCompressedRecordHeader(ByteBuffer buffer,
+ byte magic,
+ int recordSize,
+ long timestamp,
+ CompressionType compressionType,
+ TimestampType timestampType) {
+ int recordPosition = buffer.position();
+ int valueSize = recordSize - recordOverhead(magic);
+
+ // write the record header with a null value (the key is always null for the wrapper)
+ write(buffer, magic, timestamp, null, null, compressionType, timestampType);
+
+ // now fill in the value size
+ buffer.putInt(recordPosition + keyOffset(magic), valueSize);
+
+ // compute and fill the crc from the beginning of the message
+ long crc = Utils.computeChecksum(buffer, recordPosition + MAGIC_OFFSET, recordSize - MAGIC_OFFSET);
+ Utils.writeUnsignedInt(buffer, recordPosition + CRC_OFFSET, crc);
+ }
+
+ private static void write(ByteBuffer buffer,
+ byte magic,
+ long timestamp,
+ ByteBuffer key,
+ ByteBuffer value,
+ CompressionType compressionType,
+ TimestampType timestampType) {
+ try {
+ ByteBufferOutputStream out = new ByteBufferOutputStream(buffer);
+ write(out, magic, timestamp, key, value, compressionType, timestampType);
+ } catch (IOException e) {
+ throw new KafkaException(e);
+ }
+ }
+
+ /**
+ * Write the record data with the given compression type and return the computed crc.
+ *
+ * @param out The output stream to write to
+ * @param magic The magic value to be used
+ * @param timestamp The timestamp of the record
+ * @param key The record key
+ * @param value The record value
+ * @param compressionType The compression type
+ * @param timestampType The timestamp type
+ * @return the computed CRC for this record.
+ * @throws IOException for any IO errors writing to the output stream.
+ */
+ public static long write(DataOutputStream out,
+ byte magic,
+ long timestamp,
+ byte[] key,
+ byte[] value,
+ CompressionType compressionType,
+ TimestampType timestampType) throws IOException {
+ return write(out, magic, timestamp, wrapNullable(key), wrapNullable(value), compressionType, timestampType);
+ }
+
+ private static long write(DataOutputStream out,
+ byte magic,
+ long timestamp,
+ ByteBuffer key,
+ ByteBuffer value,
+ CompressionType compressionType,
+ TimestampType timestampType) throws IOException {
+ byte attributes = computeAttributes(magic, compressionType, timestampType);
+ long crc = computeChecksum(magic, attributes, timestamp, key, value);
+ write(out, magic, crc, attributes, timestamp, key, value);
+ return crc;
+ }
+
+
+ /**
+ * Write a record using raw fields (without validation). This should only be used in testing.
+ */
+ public static void write(DataOutputStream out,
+ byte magic,
+ long crc,
+ byte attributes,
+ long timestamp,
+ byte[] key,
+ byte[] value) throws IOException {
+ write(out, magic, crc, attributes, timestamp, wrapNullable(key), wrapNullable(value));
+ }
+
+ // Write a record to the buffer, if the record's compression type is none, then
+ // its value payload should be already compressed with the specified type
+ private static void write(DataOutputStream out,
+ byte magic,
+ long crc,
+ byte attributes,
+ long timestamp,
+ ByteBuffer key,
+ ByteBuffer value) throws IOException {
+ if (magic != MAGIC_VALUE_V0 && magic != MAGIC_VALUE_V1)
+ throw new IllegalArgumentException("Invalid magic value " + magic);
+ if (timestamp < 0 && timestamp != NO_TIMESTAMP)
+ throw new IllegalArgumentException("Invalid message timestamp " + timestamp);
+
+ // write crc
+ out.writeInt((int) (crc & 0xffffffffL));
+ // write magic value
+ out.writeByte(magic);
+ // write attributes
+ out.writeByte(attributes);
+
+ // maybe write timestamp
+ if (magic > 0)
+ out.writeLong(timestamp);
+
+ // write the key
+ if (key == null) {
+ out.writeInt(-1);
+ } else {
+ int size = key.remaining();
+ out.writeInt(size);
+ out.write(key.array(), key.arrayOffset(), size);
+ }
+ // write the value
+ if (value == null) {
+ out.writeInt(-1);
+ } else {
+ int size = value.remaining();
+ out.writeInt(size);
+ out.write(value.array(), value.arrayOffset(), size);
+ }
+ }
+
+ public static int recordSize(byte[] key, byte[] value) {
+ return recordSize(CURRENT_MAGIC_VALUE, key, value);
+ }
+
+ public static int recordSize(byte magic, byte[] key, byte[] value) {
+ return recordSize(magic, key == null ? 0 : key.length, value == null ? 0 : value.length);
+ }
+
+ private static int recordSize(byte magic, int keySize, int valueSize) {
+ return recordOverhead(magic) + keySize + valueSize;
+ }
+
+ // visible only for testing
+ public static byte computeAttributes(byte magic, CompressionType type, TimestampType timestampType) {
+ byte attributes = 0;
+ if (type.id > 0)
+ attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & type.id));
+ if (magic > 0)
+ return timestampType.updateAttributes(attributes);
+ return attributes;
+ }
+
+ // visible only for testing
+ public static long computeChecksum(byte magic, byte attributes, long timestamp, byte[] key, byte[] value) {
+ return computeChecksum(magic, attributes, timestamp, wrapNullable(key), wrapNullable(value));
+ }
+
+ /**
+ * Compute the checksum of the record from the attributes, key and value payloads
+ */
+ private static long computeChecksum(byte magic, byte attributes, long timestamp, ByteBuffer key, ByteBuffer value) {
+ Crc32 crc = new Crc32();
+ crc.update(magic);
+ crc.update(attributes);
+ if (magic > 0)
+ crc.updateLong(timestamp);
+ // update for the key
+ if (key == null) {
+ crc.updateInt(-1);
+ } else {
+ int size = key.remaining();
+ crc.updateInt(size);
+ crc.update(key.array(), key.arrayOffset(), size);
+ }
+ // update for the value
+ if (value == null) {
+ crc.updateInt(-1);
+ } else {
+ int size = value.remaining();
+ crc.updateInt(size);
+ crc.update(value.array(), value.arrayOffset(), size);
+ }
+ return crc.getValue();
+ }
+
+ public static int recordOverhead(byte magic) {
+ if (magic == 0)
+ return RECORD_OVERHEAD_V0;
+ return RECORD_OVERHEAD_V1;
+ }
+
+ private static int keyOffset(byte magic) {
+ if (magic == 0)
+ return KEY_OFFSET_V0;
+ return KEY_OFFSET_V1;
+ }
+
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/Records.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/Records.java b/clients/src/main/java/org/apache/kafka/common/record/Records.java
index 3bc043f..823d2b7 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/Records.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/Records.java
@@ -18,32 +18,74 @@ package org.apache.kafka.common.record;
import java.io.IOException;
import java.nio.channels.GatheringByteChannel;
+import java.util.Iterator;
/**
- * A binary format which consists of a 4 byte size, an 8 byte offset, and the record bytes. See {@link MemoryRecords}
- * for the in-memory representation.
+ * Interface for accessing the records contained in a log. The log itself is represented as a sequence of log entries.
+ * Each log entry consists of an 8 byte offset, a 4 byte record size, and a "shallow" {@link Record record}.
+ * If the entry is not compressed, then each entry will have only the shallow record contained inside it. If it is
+ * compressed, the entry contains "deep" records, which are packed into the value field of the shallow record. To iterate
+ * over the shallow records, use {@link #shallowIterator()}; for the deep records, use {@link #deepIterator()}. Note
+ * that the deep iterator handles both compressed and non-compressed entries: if the entry is not compressed, the
+ * shallow record is returned; otherwise, the shallow record is decompressed and the deep entries are returned.
+ * See {@link MemoryRecords} for the in-memory representation and {@link FileRecords} for the on-disk representation.
*/
-public interface Records extends Iterable<LogEntry> {
+public interface Records {
- int SIZE_LENGTH = 4;
+ int OFFSET_OFFSET = 0;
int OFFSET_LENGTH = 8;
- int LOG_OVERHEAD = SIZE_LENGTH + OFFSET_LENGTH;
+ int SIZE_OFFSET = OFFSET_OFFSET + OFFSET_LENGTH;
+ int SIZE_LENGTH = 4;
+ int LOG_OVERHEAD = SIZE_OFFSET + SIZE_LENGTH;
/**
- * The size of these records in bytes
- * @return The size in bytes
+ * The size of these records in bytes.
+ * @return The size in bytes of the records
*/
int sizeInBytes();
/**
- * Write the messages in this set to the given channel starting at the given offset byte.
+ * Write the contents of this buffer to a channel.
* @param channel The channel to write to
- * @param position The position within this record set to begin writing from
+ * @param position The position in the buffer to write from
* @param length The number of bytes to write
- * @return The number of bytes written to the channel (which may be fewer than requested)
- * @throws IOException For any IO errors copying the
+ * @return The number of bytes written
+ * @throws IOException For any IO errors
*/
long writeTo(GatheringByteChannel channel, long position, int length) throws IOException;
+ /**
+ * Get the shallow log entries in this log buffer. Note that the signature allows subclasses
+ * to return a more specific log entry type. This enables optimizations such as in-place offset
+ * assignment (see {@link ByteBufferLogInputStream.ByteBufferLogEntry}), and partial reading of
+ * record data (see {@link FileLogInputStream.FileChannelLogEntry#magic()}.
+ * @return An iterator over the shallow entries of the log
+ */
+ Iterator<? extends LogEntry> shallowIterator();
+
+ /**
+ * Get the deep log entries (i.e. descend into compressed message sets). For the deep records,
+ * there are fewer options for optimization since the data must be decompressed before it can be
+ * returned. Hence there is little advantage in allowing subclasses to return a more specific type
+ * as we do for {@link #shallowIterator()}.
+ * @return An iterator over the deep entries of the log
+ */
+ Iterator<LogEntry> deepIterator();
+
+ /**
+ * Check whether all shallow entries in this buffer have a certain magic value.
+ * @param magic The magic value to check
+ * @return true if all shallow entries have a matching magic value, false otherwise
+ */
+ boolean hasMatchingShallowMagic(byte magic);
+
+
+ /**
+ * Convert all entries in this buffer to the format passed as a parameter. Note that this requires
+ * deep iteration since all of the deep records must also be converted to the desired format.
+ * @param toMagic The magic value to convert to
+ * @return A Records (which may or may not be the same instance)
+ */
+ Records toMessageFormat(byte toMagic);
}
[6/9] kafka git commit: KAFKA-4390;
Replace MessageSet usage with client-side alternatives
Posted by jg...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java b/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java
index 6482529..551d820 100644
--- a/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java
@@ -35,18 +35,22 @@ import org.junit.runners.Parameterized.Parameters;
@RunWith(value = Parameterized.class)
public class RecordTest {
+ private byte magic;
private long timestamp;
private ByteBuffer key;
private ByteBuffer value;
private CompressionType compression;
+ private TimestampType timestampType;
private Record record;
- public RecordTest(long timestamp, byte[] key, byte[] value, CompressionType compression) {
+ public RecordTest(byte magic, long timestamp, byte[] key, byte[] value, CompressionType compression) {
+ this.magic = magic;
this.timestamp = timestamp;
+ this.timestampType = TimestampType.CREATE_TIME;
this.key = key == null ? null : ByteBuffer.wrap(key);
this.value = value == null ? null : ByteBuffer.wrap(value);
this.compression = compression;
- this.record = new Record(timestamp, key, value, compression);
+ this.record = Record.create(magic, timestamp, key, value, compression, timestampType);
}
@Test
@@ -56,22 +60,33 @@ public class RecordTest {
assertEquals(key, record.key());
if (key != null)
assertEquals(key.limit(), record.keySize());
- assertEquals(Record.CURRENT_MAGIC_VALUE, record.magic());
+ assertEquals(magic, record.magic());
assertEquals(value, record.value());
if (value != null)
assertEquals(value.limit(), record.valueSize());
+ if (magic > 0) {
+ assertEquals(timestamp, record.timestamp());
+ assertEquals(timestampType, record.timestampType());
+ } else {
+ assertEquals(Record.NO_TIMESTAMP, record.timestamp());
+ assertEquals(TimestampType.NO_TIMESTAMP_TYPE, record.timestampType());
+ }
}
@Test
public void testChecksum() {
assertEquals(record.checksum(), record.computeChecksum());
+
+ byte attributes = Record.computeAttributes(magic, this.compression, TimestampType.CREATE_TIME);
assertEquals(record.checksum(), Record.computeChecksum(
- this.timestamp,
- this.key == null ? null : this.key.array(),
- this.value == null ? null : this.value.array(),
- this.compression, 0, -1));
+ magic,
+ attributes,
+ this.timestamp,
+ this.key == null ? null : this.key.array(),
+ this.value == null ? null : this.value.array()
+ ));
assertTrue(record.isValid());
- for (int i = Record.CRC_OFFSET + Record.CRC_LENGTH; i < record.size(); i++) {
+ for (int i = Record.CRC_OFFSET + Record.CRC_LENGTH; i < record.sizeInBytes(); i++) {
Record copy = copyOf(record);
copy.buffer().put(i, (byte) 69);
assertFalse(copy.isValid());
@@ -85,7 +100,7 @@ public class RecordTest {
}
private Record copyOf(Record record) {
- ByteBuffer buffer = ByteBuffer.allocate(record.size());
+ ByteBuffer buffer = ByteBuffer.allocate(record.sizeInBytes());
record.buffer().put(buffer);
buffer.rewind();
record.buffer().rewind();
@@ -101,12 +116,13 @@ public class RecordTest {
public static Collection<Object[]> data() {
byte[] payload = new byte[1000];
Arrays.fill(payload, (byte) 1);
- List<Object[]> values = new ArrayList<Object[]>();
- for (long timestamp : Arrays.asList(Record.NO_TIMESTAMP, 0L, 1L))
- for (byte[] key : Arrays.asList(null, "".getBytes(), "key".getBytes(), payload))
- for (byte[] value : Arrays.asList(null, "".getBytes(), "value".getBytes(), payload))
- for (CompressionType compression : CompressionType.values())
- values.add(new Object[] {timestamp, key, value, compression});
+ List<Object[]> values = new ArrayList<>();
+ for (byte magic : Arrays.asList(Record.MAGIC_VALUE_V0, Record.MAGIC_VALUE_V1))
+ for (long timestamp : Arrays.asList(Record.NO_TIMESTAMP, 0L, 1L))
+ for (byte[] key : Arrays.asList(null, "".getBytes(), "key".getBytes(), payload))
+ for (byte[] value : Arrays.asList(null, "".getBytes(), "value".getBytes(), payload))
+ for (CompressionType compression : CompressionType.values())
+ values.add(new Object[] {magic, timestamp, key, value, compression});
return values;
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java b/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java
index aabadfe..427c743 100644
--- a/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/record/SimpleRecordTest.java
@@ -20,35 +20,29 @@ import org.junit.Test;
import java.nio.ByteBuffer;
+import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
public class SimpleRecordTest {
/* This scenario can happen if the record size field is corrupt and we end up allocating a buffer that is too small */
- @Test
+ @Test(expected = InvalidRecordException.class)
public void testIsValidWithTooSmallBuffer() {
ByteBuffer buffer = ByteBuffer.allocate(2);
Record record = new Record(buffer);
assertFalse(record.isValid());
- try {
- record.ensureValid();
- fail("InvalidRecordException should have been thrown");
- } catch (InvalidRecordException e) { }
+ record.ensureValid();
}
- @Test
+ @Test(expected = InvalidRecordException.class)
public void testIsValidWithChecksumMismatch() {
ByteBuffer buffer = ByteBuffer.allocate(4);
// set checksum
buffer.putInt(2);
Record record = new Record(buffer);
assertFalse(record.isValid());
- try {
- record.ensureValid();
- fail("InvalidRecordException should have been thrown");
- } catch (InvalidRecordException e) { }
+ record.ensureValid();
}
@Test
@@ -63,4 +57,40 @@ public class SimpleRecordTest {
record.ensureValid();
}
+ @Test
+ public void testConvertFromV0ToV1() {
+ byte[][] keys = new byte[][] {"a".getBytes(), "".getBytes(), null, "b".getBytes()};
+ byte[][] values = new byte[][] {"1".getBytes(), "".getBytes(), "2".getBytes(), null};
+
+ for (int i = 0; i < keys.length; i++) {
+ Record record = Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, keys[i], values[i]);
+ Record converted = record.convert(Record.MAGIC_VALUE_V1);
+
+ assertEquals(Record.MAGIC_VALUE_V1, converted.magic());
+ assertEquals(Record.NO_TIMESTAMP, converted.timestamp());
+ assertEquals(record.key(), converted.key());
+ assertEquals(record.value(), converted.value());
+ assertTrue(record.isValid());
+ assertEquals(record.convertedSize(Record.MAGIC_VALUE_V1), converted.sizeInBytes());
+ }
+ }
+
+ @Test
+ public void testConvertFromV1ToV0() {
+ byte[][] keys = new byte[][] {"a".getBytes(), "".getBytes(), null, "b".getBytes()};
+ byte[][] values = new byte[][] {"1".getBytes(), "".getBytes(), "2".getBytes(), null};
+
+ for (int i = 0; i < keys.length; i++) {
+ Record record = Record.create(Record.MAGIC_VALUE_V1, System.currentTimeMillis(), keys[i], values[i]);
+ Record converted = record.convert(Record.MAGIC_VALUE_V0);
+
+ assertEquals(Record.MAGIC_VALUE_V0, converted.magic());
+ assertEquals(Record.NO_TIMESTAMP, converted.timestamp());
+ assertEquals(record.key(), converted.key());
+ assertEquals(record.value(), converted.value());
+ assertTrue(record.isValid());
+ assertEquals(record.convertedSize(Record.MAGIC_VALUE_V0), converted.sizeInBytes());
+ }
+ }
+
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/test/java/org/apache/kafka/common/record/TimestampTypeTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/record/TimestampTypeTest.java b/clients/src/test/java/org/apache/kafka/common/record/TimestampTypeTest.java
new file mode 100644
index 0000000..4759715
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/record/TimestampTypeTest.java
@@ -0,0 +1,37 @@
+/**
+ * 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.common.record;
+
+import org.junit.Test;
+
+import static org.junit.Assert.assertEquals;
+
+public class TimestampTypeTest {
+
+ @Test
+ public void toAndFromAttributesCreateTime() {
+ byte attributes = TimestampType.CREATE_TIME.updateAttributes((byte) 0);
+ assertEquals(TimestampType.CREATE_TIME, TimestampType.forAttributes(attributes));
+ }
+
+ @Test
+ public void toAndFromAttributesLogAppendTime() {
+ byte attributes = TimestampType.LOG_APPEND_TIME.updateAttributes((byte) 0);
+ assertEquals(TimestampType.LOG_APPEND_TIME, TimestampType.forAttributes(attributes));
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/test/java/org/apache/kafka/test/TestUtils.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/test/TestUtils.java b/clients/src/test/java/org/apache/kafka/test/TestUtils.java
index d3280e5..4e80b61 100644
--- a/clients/src/test/java/org/apache/kafka/test/TestUtils.java
+++ b/clients/src/test/java/org/apache/kafka/test/TestUtils.java
@@ -23,8 +23,10 @@ import org.apache.kafka.common.Node;
import org.apache.kafka.common.PartitionInfo;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.record.MemoryRecordsBuilder;
import org.apache.kafka.common.record.Record;
import org.apache.kafka.common.record.Records;
+import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.utils.Utils;
import javax.xml.bind.DatatypeConverter;
@@ -35,6 +37,7 @@ import java.nio.file.Files;
import java.nio.file.Path;
import java.util.ArrayList;
import java.util.Collections;
+import java.util.Iterator;
import java.util.List;
import java.util.Map;
import java.util.Properties;
@@ -46,6 +49,7 @@ import java.util.regex.Pattern;
import static java.util.Arrays.asList;
import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertNotNull;
import static org.junit.Assert.assertTrue;
import static org.junit.Assert.fail;
@@ -185,13 +189,13 @@ public class TestUtils {
public static ByteBuffer partitionRecordsBuffer(final long offset, final CompressionType compressionType, final Record... records) {
int bufferSize = 0;
for (final Record record : records)
- bufferSize += Records.LOG_OVERHEAD + record.size();
+ bufferSize += Records.LOG_OVERHEAD + record.sizeInBytes();
final ByteBuffer buffer = ByteBuffer.allocate(bufferSize);
- final MemoryRecords memoryRecords = MemoryRecords.emptyRecords(buffer, compressionType);
+ MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, compressionType, TimestampType.CREATE_TIME);
+ long nextOffset = offset;
for (final Record record : records)
- memoryRecords.append(offset, record);
- memoryRecords.close();
- return memoryRecords.buffer();
+ builder.append(nextOffset++, record);
+ return builder.build().buffer();
}
public static Properties producerConfig(final String bootstrapServers,
@@ -309,4 +313,22 @@ public class TestUtils {
fail(clusterId + " cannot be converted back to UUID.");
}
}
+
+ /**
+ * Throw an exception if the two iterators are of differing lengths or contain
+ * different messages on their Nth element
+ */
+ public static <T> void checkEquals(Iterator<T> s1, Iterator<T> s2) {
+ while (s1.hasNext() && s2.hasNext())
+ assertEquals(s1.next(), s2.next());
+ assertFalse("Iterators have uneven length--first has more", s1.hasNext());
+ assertFalse("Iterators have uneven length--second has more", s2.hasNext());
+ }
+
+ public static <T> List<T> toList(Iterator<T> iterator) {
+ List<T> res = new ArrayList<>();
+ while (iterator.hasNext())
+ res.add(iterator.next());
+ return res;
+ }
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/api/ApiVersion.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/api/ApiVersion.scala b/core/src/main/scala/kafka/api/ApiVersion.scala
index 895c1b1..4052639 100644
--- a/core/src/main/scala/kafka/api/ApiVersion.scala
+++ b/core/src/main/scala/kafka/api/ApiVersion.scala
@@ -17,7 +17,7 @@
package kafka.api
-import kafka.message.Message
+import org.apache.kafka.common.record.Record
/**
* This class contains the different Kafka versions.
@@ -87,54 +87,54 @@ sealed trait ApiVersion extends Ordered[ApiVersion] {
// Keep the IDs in order of versions
case object KAFKA_0_8_0 extends ApiVersion {
val version: String = "0.8.0.X"
- val messageFormatVersion: Byte = Message.MagicValue_V0
+ val messageFormatVersion: Byte = Record.MAGIC_VALUE_V0
val id: Int = 0
}
case object KAFKA_0_8_1 extends ApiVersion {
val version: String = "0.8.1.X"
- val messageFormatVersion: Byte = Message.MagicValue_V0
+ val messageFormatVersion: Byte = Record.MAGIC_VALUE_V0
val id: Int = 1
}
case object KAFKA_0_8_2 extends ApiVersion {
val version: String = "0.8.2.X"
- val messageFormatVersion: Byte = Message.MagicValue_V0
+ val messageFormatVersion: Byte = Record.MAGIC_VALUE_V0
val id: Int = 2
}
case object KAFKA_0_9_0 extends ApiVersion {
val version: String = "0.9.0.X"
- val messageFormatVersion: Byte = Message.MagicValue_V0
+ val messageFormatVersion: Byte = Record.MAGIC_VALUE_V0
val id: Int = 3
}
case object KAFKA_0_10_0_IV0 extends ApiVersion {
val version: String = "0.10.0-IV0"
- val messageFormatVersion: Byte = Message.MagicValue_V1
+ val messageFormatVersion: Byte = Record.MAGIC_VALUE_V1
val id: Int = 4
}
case object KAFKA_0_10_0_IV1 extends ApiVersion {
val version: String = "0.10.0-IV1"
- val messageFormatVersion: Byte = Message.MagicValue_V1
+ val messageFormatVersion: Byte = Record.MAGIC_VALUE_V1
val id: Int = 5
}
case object KAFKA_0_10_1_IV0 extends ApiVersion {
val version: String = "0.10.1-IV0"
- val messageFormatVersion: Byte = Message.MagicValue_V1
+ val messageFormatVersion: Byte = Record.MAGIC_VALUE_V1
val id: Int = 6
}
case object KAFKA_0_10_1_IV1 extends ApiVersion {
val version: String = "0.10.1-IV1"
- val messageFormatVersion: Byte = Message.MagicValue_V1
+ val messageFormatVersion: Byte = Record.MAGIC_VALUE_V1
val id: Int = 7
}
case object KAFKA_0_10_1_IV2 extends ApiVersion {
val version: String = "0.10.1-IV2"
- val messageFormatVersion: Byte = Message.MagicValue_V1
+ val messageFormatVersion: Byte = Record.MAGIC_VALUE_V1
val id: Int = 8
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/cluster/Partition.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/cluster/Partition.scala b/core/src/main/scala/kafka/cluster/Partition.scala
index 7e52a91..9eb92cd 100755
--- a/core/src/main/scala/kafka/cluster/Partition.scala
+++ b/core/src/main/scala/kafka/cluster/Partition.scala
@@ -25,7 +25,6 @@ import kafka.log.LogConfig
import kafka.server._
import kafka.metrics.KafkaMetricsGroup
import kafka.controller.KafkaController
-import kafka.message.ByteBufferMessageSet
import java.io.IOException
import java.util.concurrent.locks.ReentrantReadWriteLock
@@ -34,6 +33,7 @@ import org.apache.kafka.common.protocol.Errors
import scala.collection.JavaConverters._
import com.yammer.metrics.core.Gauge
+import org.apache.kafka.common.record.MemoryRecords
import org.apache.kafka.common.requests.PartitionState
import org.apache.kafka.common.utils.Time
@@ -190,7 +190,7 @@ class Partition(val topic: String,
allReplicas.foreach(replica => getOrCreateReplica(replica))
val newInSyncReplicas = partitionStateInfo.isr.asScala.map(r => getOrCreateReplica(r)).toSet
// remove assigned replicas that have been removed by the controller
- (assignedReplicas().map(_.brokerId) -- allReplicas).foreach(removeReplica(_))
+ (assignedReplicas().map(_.brokerId) -- allReplicas).foreach(removeReplica)
inSyncReplicas = newInSyncReplicas
leaderEpoch = partitionStateInfo.leaderEpoch
zkVersion = partitionStateInfo.zkVersion
@@ -440,7 +440,7 @@ class Partition(val topic: String,
laggingReplicas
}
- def appendMessagesToLeader(messages: ByteBufferMessageSet, requiredAcks: Int = 0) = {
+ def appendRecordsToLeader(records: MemoryRecords, requiredAcks: Int = 0) = {
val (info, leaderHWIncremented) = inReadLock(leaderIsrUpdateLock) {
val leaderReplicaOpt = leaderReplicaIfLocal()
leaderReplicaOpt match {
@@ -455,7 +455,7 @@ class Partition(val topic: String,
.format(topic, partitionId, inSyncSize, minIsr))
}
- val info = log.append(messages, assignOffsets = true)
+ val info = log.append(records, assignOffsets = true)
// probably unblock some follower fetch requests since log end offset has been updated
replicaManager.tryCompleteDelayedFetch(TopicPartitionOperationKey(this.topic, this.partitionId))
// we may need to increment high watermark since ISR could be down to 1
@@ -480,7 +480,7 @@ class Partition(val topic: String,
newLeaderAndIsr, controllerEpoch, zkVersion)
if(updateSucceeded) {
- replicaManager.recordIsrChange(new TopicAndPartition(topic, partitionId))
+ replicaManager.recordIsrChange(TopicAndPartition(topic, partitionId))
inSyncReplicas = newIsr
zkVersion = newVersion
trace("ISR updated to [%s] and zkVersion updated to [%d]".format(newIsr.mkString(","), zkVersion))
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala
index c47efb7..f702b9d 100644
--- a/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala
+++ b/core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala
@@ -17,14 +17,16 @@
package kafka.consumer
-import kafka.api.{OffsetRequest, Request, FetchRequestBuilder, FetchResponsePartitionData}
+import kafka.api.{FetchRequestBuilder, FetchResponsePartitionData, OffsetRequest, Request}
import kafka.cluster.BrokerEndPoint
import kafka.message.ByteBufferMessageSet
-import kafka.server.{PartitionFetchState, AbstractFetcherThread}
+import kafka.server.{AbstractFetcherThread, PartitionFetchState}
import kafka.common.{ErrorMapping, TopicAndPartition}
+
import scala.collection.Map
import ConsumerFetcherThread._
import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.record.MemoryRecords
class ConsumerFetcherThread(name: String,
val config: ConsumerConfig,
@@ -81,7 +83,7 @@ class ConsumerFetcherThread(name: String,
case OffsetRequest.LargestTimeString => OffsetRequest.LatestTime
case _ => OffsetRequest.LatestTime
}
- val topicAndPartition = new TopicAndPartition(topicPartition.topic, topicPartition.partition)
+ val topicAndPartition = TopicAndPartition(topicPartition.topic, topicPartition.partition)
val newOffset = simpleConsumer.earliestOrLatestOffset(topicAndPartition, startTimestamp, Request.OrdinaryConsumerId)
val pti = partitionMap(topicPartition)
pti.resetFetchOffset(newOffset)
@@ -123,7 +125,7 @@ object ConsumerFetcherThread {
class PartitionData(val underlying: FetchResponsePartitionData) extends AbstractFetcherThread.PartitionData {
def errorCode: Short = underlying.error
- def toByteBufferMessageSet: ByteBufferMessageSet = underlying.messages.asInstanceOf[ByteBufferMessageSet]
+ def toRecords: MemoryRecords = underlying.messages.asInstanceOf[ByteBufferMessageSet].asRecords
def highWatermark: Long = underlying.hw
def exception: Option[Throwable] =
if (errorCode == ErrorMapping.NoError) None else Some(ErrorMapping.exceptionFor(errorCode))
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
index 0c53345..db40482 100644
--- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
+++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
@@ -458,14 +458,14 @@ class GroupCoordinator(val brokerId: Int,
def handleFetchOffsets(groupId: String,
partitions: Seq[TopicPartition]): Map[TopicPartition, OffsetFetchResponse.PartitionData] = {
if (!isActive.get) {
- partitions.map { case topicPartition =>
+ partitions.map { topicPartition =>
(topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code))}.toMap
} else if (!isCoordinatorForGroup(groupId)) {
debug("Could not fetch offsets for group %s (not group coordinator).".format(groupId))
- partitions.map { case topicPartition =>
+ partitions.map { topicPartition =>
(topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.NOT_COORDINATOR_FOR_GROUP.code))}.toMap
} else if (isCoordinatorLoadingInProgress(groupId)) {
- partitions.map { case topicPartition =>
+ partitions.map { topicPartition =>
(topicPartition, new OffsetFetchResponse.PartitionData(OffsetFetchResponse.INVALID_OFFSET, "", Errors.GROUP_LOAD_IN_PROGRESS.code))}.toMap
} else {
// return offsets blindly regardless the current group state since the group may be using
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala
index e55bcaa..a97b527 100644
--- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala
+++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala
@@ -17,38 +17,31 @@
package kafka.coordinator
-import org.apache.kafka.common.TopicPartition
-import org.apache.kafka.common.protocol.Errors
-import org.apache.kafka.common.protocol.types.{ArrayOf, Field, Schema, Struct}
-import org.apache.kafka.common.protocol.types.Type.STRING
-import org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING
-import org.apache.kafka.common.protocol.types.Type.INT32
-import org.apache.kafka.common.protocol.types.Type.INT64
-import org.apache.kafka.common.protocol.types.Type.BYTES
-import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
-import org.apache.kafka.common.requests.OffsetFetchResponse
-import org.apache.kafka.common.utils.Utils
-import org.apache.kafka.common.utils.Time
-import org.apache.kafka.clients.consumer.ConsumerRecord
-import kafka.utils._
-import kafka.common._
-import kafka.message._
-import kafka.log.FileMessageSet
-import kafka.metrics.KafkaMetricsGroup
-import kafka.common.TopicAndPartition
-import kafka.common.MessageFormatter
-import kafka.server.ReplicaManager
-
-import scala.collection._
import java.io.PrintStream
import java.nio.ByteBuffer
-import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.TimeUnit
+import java.util.concurrent.atomic.AtomicBoolean
import java.util.concurrent.locks.ReentrantLock
import com.yammer.metrics.core.Gauge
import kafka.api.{ApiVersion, KAFKA_0_10_1_IV0}
+import kafka.common.{MessageFormatter, TopicAndPartition, _}
+import kafka.metrics.KafkaMetricsGroup
+import kafka.server.ReplicaManager
import kafka.utils.CoreUtils.inLock
+import kafka.utils._
+import org.apache.kafka.clients.consumer.ConsumerRecord
+import org.apache.kafka.common.TopicPartition
+import org.apache.kafka.common.protocol.Errors
+import org.apache.kafka.common.protocol.types.Type._
+import org.apache.kafka.common.protocol.types.{ArrayOf, Field, Schema, Struct}
+import org.apache.kafka.common.record._
+import org.apache.kafka.common.requests.OffsetFetchResponse
+import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
+import org.apache.kafka.common.utils.{Time, Utils}
+
+import scala.collection.JavaConverters._
+import scala.collection._
class GroupMetadataManager(val brokerId: Int,
val interBrokerProtocolVersion: ApiVersion,
@@ -57,6 +50,8 @@ class GroupMetadataManager(val brokerId: Int,
zkUtils: ZkUtils,
time: Time) extends Logging with KafkaMetricsGroup {
+ private val compressionType: CompressionType = CompressionType.forId(config.offsetsTopicCompressionCodec.codec)
+
private val groupMetadataCache = new Pool[String, GroupMetadata]
/* lock protecting access to loading and owned partition sets */
@@ -135,13 +130,11 @@ class GroupMetadataManager(val brokerId: Int,
}
}
-
def prepareStoreGroup(group: GroupMetadata,
groupAssignment: Map[String, Array[Byte]],
responseCallback: Errors => Unit): Option[DelayedStore] = {
- val magicValueAndTimestampOpt = getMessageFormatVersionAndTimestamp(partitionFor(group.groupId))
- magicValueAndTimestampOpt match {
- case Some((magicValue, timestamp)) =>
+ getMagicAndTimestamp(partitionFor(group.groupId)) match {
+ case Some((magicValue, timestampType, timestamp)) =>
val groupMetadataValueVersion = {
if (interBrokerProtocolVersion < KAFKA_0_10_1_IV0)
0.toShort
@@ -149,17 +142,12 @@ class GroupMetadataManager(val brokerId: Int,
GroupMetadataManager.CURRENT_GROUP_VALUE_SCHEMA_VERSION
}
- val message = new Message(
- key = GroupMetadataManager.groupMetadataKey(group.groupId),
- bytes = GroupMetadataManager.groupMetadataValue(group, groupAssignment, version = groupMetadataValueVersion),
- timestamp = timestamp,
- magicValue = magicValue)
+ val record = Record.create(magicValue, timestampType, timestamp,
+ GroupMetadataManager.groupMetadataKey(group.groupId),
+ GroupMetadataManager.groupMetadataValue(group, groupAssignment, version = groupMetadataValueVersion))
val groupMetadataPartition = new TopicPartition(Topic.GroupMetadataTopicName, partitionFor(group.groupId))
-
- val groupMetadataMessageSet = Map(groupMetadataPartition ->
- new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, message))
-
+ val groupMetadataRecords = Map(groupMetadataPartition -> MemoryRecords.withRecords(timestampType, compressionType, record))
val generationId = group.generationId
// set the callback function to insert the created group into cache after log append completed
@@ -212,7 +200,7 @@ class GroupMetadataManager(val brokerId: Int,
responseCallback(responseError)
}
- Some(DelayedStore(groupMetadataMessageSet, putCacheCallback))
+ Some(DelayedStore(groupMetadataRecords, putCacheCallback))
case None =>
responseCallback(Errors.NOT_COORDINATOR_FOR_GROUP)
@@ -222,11 +210,11 @@ class GroupMetadataManager(val brokerId: Int,
def store(delayedStore: DelayedStore) {
// call replica manager to append the group message
- replicaManager.appendMessages(
+ replicaManager.appendRecords(
config.offsetCommitTimeoutMs.toLong,
config.offsetCommitRequiredAcks,
true, // allow appending to internal offset topic
- delayedStore.messageSet,
+ delayedStore.partitionRecords,
delayedStore.callback)
}
@@ -244,22 +232,17 @@ class GroupMetadataManager(val brokerId: Int,
}
// construct the message set to append
- val magicValueAndTimestampOpt = getMessageFormatVersionAndTimestamp(partitionFor(group.groupId))
- magicValueAndTimestampOpt match {
- case Some((magicValue, timestamp)) =>
- val messages = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) =>
- new Message(
- key = GroupMetadataManager.offsetCommitKey(group.groupId, topicAndPartition.topic, topicAndPartition.partition),
- bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata),
- timestamp = timestamp,
- magicValue = magicValue
- )
+ getMagicAndTimestamp(partitionFor(group.groupId)) match {
+ case Some((magicValue, timestampType, timestamp)) =>
+ val records = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) =>
+ Record.create(magicValue, timestampType, timestamp,
+ GroupMetadataManager.offsetCommitKey(group.groupId, topicAndPartition.topic, topicAndPartition.partition),
+ GroupMetadataManager.offsetCommitValue(offsetAndMetadata))
}.toSeq
val offsetTopicPartition = new TopicPartition(Topic.GroupMetadataTopicName, partitionFor(group.groupId))
- val offsetsAndMetadataMessageSet = Map(offsetTopicPartition ->
- new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*))
+ val entries = Map(offsetTopicPartition -> MemoryRecords.withRecords(timestampType, compressionType, records:_*))
// set the callback function to insert offsets into cache after log append completed
def putCacheCallback(responseStatus: Map[TopicPartition, PartitionResponse]) {
@@ -330,7 +313,7 @@ class GroupMetadataManager(val brokerId: Int,
group.prepareOffsetCommit(offsetMetadata)
}
- Some(DelayedStore(offsetsAndMetadataMessageSet, putCacheCallback))
+ Some(DelayedStore(entries, putCacheCallback))
case None =>
val commitStatus = offsetMetadata.map { case (topicAndPartition, offsetAndMetadata) =>
@@ -412,28 +395,30 @@ class GroupMetadataManager(val brokerId: Int,
while (currOffset < getHighWatermark(offsetsPartition) && !shuttingDown.get()) {
buffer.clear()
- val messages = log.read(currOffset, config.loadBufferSize, minOneMessage = true).messageSet.asInstanceOf[FileMessageSet]
- messages.readInto(buffer, 0)
- val messageSet = new ByteBufferMessageSet(buffer)
- messageSet.foreach { msgAndOffset =>
- require(msgAndOffset.message.key != null, "Offset entry key should not be null")
- val baseKey = GroupMetadataManager.readMessageKey(msgAndOffset.message.key)
+ val fileRecords = log.read(currOffset, config.loadBufferSize, minOneMessage = true).records.asInstanceOf[FileRecords]
+ fileRecords.readInto(buffer, 0)
+
+ MemoryRecords.readableRecords(buffer).deepIterator.asScala.foreach { entry =>
+ val record = entry.record
+
+ require(record.hasKey, "Offset entry key should not be null")
+ val baseKey = GroupMetadataManager.readMessageKey(record.key)
if (baseKey.isInstanceOf[OffsetKey]) {
// load offset
val key = baseKey.key.asInstanceOf[GroupTopicPartition]
- if (msgAndOffset.message.payload == null) {
+ if (record.hasNullValue) {
loadedOffsets.remove(key)
removedOffsets.add(key)
} else {
- val value = GroupMetadataManager.readOffsetMessageValue(msgAndOffset.message.payload)
+ val value = GroupMetadataManager.readOffsetMessageValue(record.value)
loadedOffsets.put(key, value)
removedOffsets.remove(key)
}
} else {
// load group metadata
val groupId = baseKey.key.asInstanceOf[String]
- val groupMetadata = GroupMetadataManager.readGroupMessageValue(groupId, msgAndOffset.message.payload)
+ val groupMetadata = GroupMetadataManager.readGroupMessageValue(groupId, record.value)
if (groupMetadata != null) {
trace(s"Loaded group metadata for group ${groupMetadata.groupId} with generation ${groupMetadata.generationId}")
removedGroups.remove(groupId)
@@ -444,7 +429,7 @@ class GroupMetadataManager(val brokerId: Int,
}
}
- currOffset = msgAndOffset.nextOffset
+ currOffset = entry.nextOffset
}
}
@@ -467,8 +452,8 @@ class GroupMetadataManager(val brokerId: Int,
removedGroups.foreach { groupId =>
if (groupMetadataCache.contains(groupId))
- throw new IllegalStateException(s"Unexpected unload of active group ${groupId} while " +
- s"loading partition ${topicPartition}")
+ throw new IllegalStateException(s"Unexpected unload of active group $groupId while " +
+ s"loading partition $topicPartition")
}
if (!shuttingDown.get())
@@ -572,15 +557,15 @@ class GroupMetadataManager(val brokerId: Int,
}
val offsetsPartition = partitionFor(groupId)
- getMessageFormatVersionAndTimestamp(offsetsPartition) match {
- case Some((magicValue, timestamp)) =>
+ getMagicAndTimestamp(offsetsPartition) match {
+ case Some((magicValue, timestampType, timestamp)) =>
val partitionOpt = replicaManager.getPartition(Topic.GroupMetadataTopicName, offsetsPartition)
partitionOpt.foreach { partition =>
val appendPartition = TopicAndPartition(Topic.GroupMetadataTopicName, offsetsPartition)
val tombstones = expiredOffsets.map { case (topicPartition, offsetAndMetadata) =>
trace(s"Removing expired offset and metadata for $groupId, $topicPartition: $offsetAndMetadata")
val commitKey = GroupMetadataManager.offsetCommitKey(groupId, topicPartition.topic, topicPartition.partition)
- new Message(bytes = null, key = commitKey, timestamp = timestamp, magicValue = magicValue)
+ Record.create(magicValue, timestampType, timestamp, commitKey, null)
}.toBuffer
trace(s"Marked ${expiredOffsets.size} offsets in $appendPartition for deletion.")
@@ -590,8 +575,7 @@ class GroupMetadataManager(val brokerId: Int,
// Append the tombstone messages to the partition. It is okay if the replicas don't receive these (say,
// if we crash or leaders move) since the new leaders will still expire the consumers with heartbeat and
// retry removing this group.
- tombstones += new Message(bytes = null, key = GroupMetadataManager.groupMetadataKey(group.groupId),
- timestamp = timestamp, magicValue = magicValue)
+ tombstones += Record.create(magicValue, timestampType, timestamp, GroupMetadataManager.groupMetadataKey(group.groupId), null)
trace(s"Group $groupId removed from the metadata cache and marked for deletion in $appendPartition.")
}
@@ -599,7 +583,7 @@ class GroupMetadataManager(val brokerId: Int,
try {
// do not need to require acks since even if the tombstone is lost,
// it will be appended again in the next purge cycle
- partition.appendMessagesToLeader(new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, tombstones: _*))
+ partition.appendRecordsToLeader(MemoryRecords.withRecords(timestampType, compressionType, tombstones: _*))
offsetsRemoved += expiredOffsets.size
trace(s"Successfully appended ${tombstones.size} tombstones to $appendPartition for expired offsets and/or metadata for group $groupId")
} catch {
@@ -663,16 +647,11 @@ class GroupMetadataManager(val brokerId: Int,
* @param partition Partition of GroupMetadataTopic
* @return Option[(MessageFormatVersion, TimeStamp)] if replica is local, None otherwise
*/
- private def getMessageFormatVersionAndTimestamp(partition: Int): Option[(Byte, Long)] = {
+ private def getMagicAndTimestamp(partition: Int): Option[(Byte, TimestampType, Long)] = {
val groupMetadataTopicAndPartition = TopicAndPartition(Topic.GroupMetadataTopicName, partition)
- replicaManager.getMessageFormatVersion(groupMetadataTopicAndPartition).map { messageFormatVersion =>
- val timestamp = {
- if (messageFormatVersion == Message.MagicValue_V0)
- Message.NoTimestamp
- else
- time.milliseconds()
- }
- (messageFormatVersion, timestamp)
+ replicaManager.getMagicAndTimestampType(groupMetadataTopicAndPartition).map { case (messageFormatVersion, timestampType) =>
+ val timestamp = if (messageFormatVersion == Record.MAGIC_VALUE_V0) Record.NO_TIMESTAMP else time.milliseconds()
+ (messageFormatVersion, timestampType, timestamp)
}
}
@@ -964,7 +943,7 @@ object GroupMetadataManager {
* @return an offset-metadata object from the message
*/
def readOffsetMessageValue(buffer: ByteBuffer): OffsetAndMetadata = {
- if(buffer == null) { // tombstone
+ if (buffer == null) { // tombstone
null
} else {
val version = buffer.getShort
@@ -997,7 +976,7 @@ object GroupMetadataManager {
* @return a group metadata object from the message
*/
def readGroupMessageValue(groupId: String, buffer: ByteBuffer): GroupMetadata = {
- if(buffer == null) { // tombstone
+ if (buffer == null) { // tombstone
null
} else {
val version = buffer.getShort
@@ -1016,23 +995,22 @@ object GroupMetadataManager {
group.leaderId = value.get(LEADER_KEY).asInstanceOf[String]
group.protocol = value.get(PROTOCOL_KEY).asInstanceOf[String]
- memberMetadataArray.foreach {
- case memberMetadataObj =>
- val memberMetadata = memberMetadataObj.asInstanceOf[Struct]
- val memberId = memberMetadata.get(MEMBER_ID_KEY).asInstanceOf[String]
- val clientId = memberMetadata.get(CLIENT_ID_KEY).asInstanceOf[String]
- val clientHost = memberMetadata.get(CLIENT_HOST_KEY).asInstanceOf[String]
- val sessionTimeout = memberMetadata.get(SESSION_TIMEOUT_KEY).asInstanceOf[Int]
- val rebalanceTimeout = if (version == 0) sessionTimeout else memberMetadata.get(REBALANCE_TIMEOUT_KEY).asInstanceOf[Int]
+ memberMetadataArray.foreach { memberMetadataObj =>
+ val memberMetadata = memberMetadataObj.asInstanceOf[Struct]
+ val memberId = memberMetadata.get(MEMBER_ID_KEY).asInstanceOf[String]
+ val clientId = memberMetadata.get(CLIENT_ID_KEY).asInstanceOf[String]
+ val clientHost = memberMetadata.get(CLIENT_HOST_KEY).asInstanceOf[String]
+ val sessionTimeout = memberMetadata.get(SESSION_TIMEOUT_KEY).asInstanceOf[Int]
+ val rebalanceTimeout = if (version == 0) sessionTimeout else memberMetadata.get(REBALANCE_TIMEOUT_KEY).asInstanceOf[Int]
- val subscription = Utils.toArray(memberMetadata.get(SUBSCRIPTION_KEY).asInstanceOf[ByteBuffer])
+ val subscription = Utils.toArray(memberMetadata.get(SUBSCRIPTION_KEY).asInstanceOf[ByteBuffer])
- val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeout, sessionTimeout,
- protocolType, List((group.protocol, subscription)))
+ val member = new MemberMetadata(memberId, groupId, clientId, clientHost, rebalanceTimeout, sessionTimeout,
+ protocolType, List((group.protocol, subscription)))
- member.assignment = Utils.toArray(memberMetadata.get(ASSIGNMENT_KEY).asInstanceOf[ByteBuffer])
+ member.assignment = Utils.toArray(memberMetadata.get(ASSIGNMENT_KEY).asInstanceOf[ByteBuffer])
- group.add(memberId, member)
+ group.add(memberId, member)
}
group
@@ -1087,7 +1065,7 @@ object GroupMetadataManager {
}
-case class DelayedStore(messageSet: Map[TopicPartition, MessageSet],
+case class DelayedStore(partitionRecords: Map[TopicPartition, MemoryRecords],
callback: Map[TopicPartition, PartitionResponse] => Unit)
case class GroupTopicPartition(group: String, topicPartition: TopicPartition) {
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/log/FileMessageSet.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/FileMessageSet.scala b/core/src/main/scala/kafka/log/FileMessageSet.scala
deleted file mode 100755
index 506f5b9..0000000
--- a/core/src/main/scala/kafka/log/FileMessageSet.scala
+++ /dev/null
@@ -1,445 +0,0 @@
-/**
- * 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.log
-
-import java.io._
-import java.nio._
-import java.nio.channels._
-import java.util.concurrent.atomic._
-import java.util.concurrent.TimeUnit
-
-import kafka.utils._
-import kafka.message._
-import kafka.common.KafkaException
-import kafka.metrics.{KafkaMetricsGroup, KafkaTimer}
-import org.apache.kafka.common.errors.CorruptRecordException
-import org.apache.kafka.common.record.FileRecords
-import org.apache.kafka.common.utils.Utils
-
-import scala.collection.mutable.ArrayBuffer
-
-/**
- * An on-disk message set. An optional start and end position can be applied to the message set
- * which will allow slicing a subset of the file.
- * @param file The file name for the underlying log data
- * @param channel the underlying file channel used
- * @param start A lower bound on the absolute position in the file from which the message set begins
- * @param end The upper bound on the absolute position in the file at which the message set ends
- * @param isSlice Should the start and end parameters be used for slicing?
- */
-@nonthreadsafe
-class FileMessageSet private[kafka](@volatile var file: File,
- private[log] val channel: FileChannel,
- private[log] val start: Int,
- private[log] val end: Int,
- isSlice: Boolean) extends MessageSet {
- /* the size of the message set in bytes */
- private val _size =
- if(isSlice)
- new AtomicInteger(end - start) // don't check the file size if this is just a slice view
- else
- new AtomicInteger(math.min(channel.size.toInt, end) - start)
-
- /* if this is not a slice, update the file pointer to the end of the file */
- if (!isSlice)
- /* set the file position to the last byte in the file */
- channel.position(math.min(channel.size.toInt, end))
-
- /**
- * Create a file message set with no slicing.
- */
- def this(file: File, channel: FileChannel) =
- this(file, channel, start = 0, end = Int.MaxValue, isSlice = false)
-
- /**
- * Create a file message set with no slicing
- */
- def this(file: File) =
- this(file, FileMessageSet.openChannel(file, mutable = true))
-
- /**
- * Create a file message set with no slicing, and with initFileSize and preallocate.
- * For windows NTFS and some old LINUX file system, set preallocate to true and initFileSize
- * with one value (for example 512 * 1024 *1024 ) can improve the kafka produce performance.
- * If it's new file and preallocate is true, end will be set to 0. Otherwise set to Int.MaxValue.
- */
- def this(file: File, fileAlreadyExists: Boolean, initFileSize: Int, preallocate: Boolean) =
- this(file,
- channel = FileMessageSet.openChannel(file, mutable = true, fileAlreadyExists, initFileSize, preallocate),
- start = 0,
- end = if (!fileAlreadyExists && preallocate) 0 else Int.MaxValue,
- isSlice = false)
-
- /**
- * Create a file message set with mutable option
- */
- def this(file: File, mutable: Boolean) = this(file, FileMessageSet.openChannel(file, mutable))
-
- /**
- * Create a slice view of the file message set that begins and ends at the given byte offsets
- */
- def this(file: File, channel: FileChannel, start: Int, end: Int) =
- this(file, channel, start, end, isSlice = true)
-
- /**
- * Return a message set which is a view into this set starting from the given position and with the given size limit.
- *
- * If the size is beyond the end of the file, the end will be based on the size of the file at the time of the read.
- *
- * If this message set is already sliced, the position will be taken relative to that slicing.
- *
- * @param position The start position to begin the read from
- * @param size The number of bytes after the start position to include
- *
- * @return A sliced wrapper on this message set limited based on the given position and size
- */
- def read(position: Int, size: Int): FileMessageSet = {
- if(position < 0)
- throw new IllegalArgumentException("Invalid position: " + position)
- if(size < 0)
- throw new IllegalArgumentException("Invalid size: " + size)
- new FileMessageSet(file,
- channel,
- start = this.start + position,
- end = {
- // Handle the integer overflow
- if (this.start + position + size < 0)
- sizeInBytes()
- else
- math.min(this.start + position + size, sizeInBytes())
- })
- }
-
- override def asRecords: FileRecords = new FileRecords(file, channel, start, end, isSlice)
-
- /**
- * Search forward for the file position of the last offset that is greater than or equal to the target offset
- * and return its physical position and the size of the message (including log overhead) at the returned offset. If
- * no such offsets are found, return null.
- *
- * @param targetOffset The offset to search for.
- * @param startingPosition The starting position in the file to begin searching from.
- */
- def searchForOffsetWithSize(targetOffset: Long, startingPosition: Int): (OffsetPosition, Int) = {
- var position = startingPosition
- val buffer = ByteBuffer.allocate(MessageSet.LogOverhead)
- val size = sizeInBytes()
- while(position + MessageSet.LogOverhead < size) {
- buffer.rewind()
- channel.read(buffer, position)
- if(buffer.hasRemaining)
- throw new IllegalStateException("Failed to read complete buffer for targetOffset %d startPosition %d in %s"
- .format(targetOffset, startingPosition, file.getAbsolutePath))
- buffer.rewind()
- val offset = buffer.getLong()
- val messageSize = buffer.getInt()
- if (messageSize < Message.MinMessageOverhead)
- throw new IllegalStateException("Invalid message size: " + messageSize)
- if (offset >= targetOffset)
- return (OffsetPosition(offset, position), messageSize + MessageSet.LogOverhead)
- position += MessageSet.LogOverhead + messageSize
- }
- null
- }
-
- /**
- * Search forward for the message whose timestamp is greater than or equals to the target timestamp.
- *
- * @param targetTimestamp The timestamp to search for.
- * @param startingPosition The starting position to search.
- * @return The timestamp and offset of the message found. None, if no message is found.
- */
- def searchForTimestamp(targetTimestamp: Long, startingPosition: Int): Option[TimestampOffset] = {
- val messagesToSearch = read(startingPosition, sizeInBytes)
- for (messageAndOffset <- messagesToSearch) {
- val message = messageAndOffset.message
- if (message.timestamp >= targetTimestamp) {
- // We found a message
- message.compressionCodec match {
- case NoCompressionCodec =>
- return Some(TimestampOffset(messageAndOffset.message.timestamp, messageAndOffset.offset))
- case _ =>
- // Iterate over the inner messages to get the exact offset.
- for (innerMessageAndOffset <- ByteBufferMessageSet.deepIterator(messageAndOffset)) {
- val timestamp = innerMessageAndOffset.message.timestamp
- if (timestamp >= targetTimestamp)
- return Some(TimestampOffset(innerMessageAndOffset.message.timestamp, innerMessageAndOffset.offset))
- }
- throw new IllegalStateException(s"The message set (max timestamp = ${message.timestamp}, max offset = ${messageAndOffset.offset}" +
- s" should contain target timestamp $targetTimestamp but it does not.")
- }
- }
- }
- None
- }
-
- /**
- * Return the largest timestamp of the messages after a given position in this file message set.
- * @param startingPosition The starting position.
- * @return The largest timestamp of the messages after the given position.
- */
- def largestTimestampAfter(startingPosition: Int): TimestampOffset = {
- var maxTimestamp = Message.NoTimestamp
- var offsetOfMaxTimestamp = -1L
- val messagesToSearch = read(startingPosition, Int.MaxValue)
- for (messageAndOffset <- messagesToSearch) {
- if (messageAndOffset.message.timestamp > maxTimestamp) {
- maxTimestamp = messageAndOffset.message.timestamp
- offsetOfMaxTimestamp = messageAndOffset.offset
- }
- }
- TimestampOffset(maxTimestamp, offsetOfMaxTimestamp)
- }
-
- /**
- * This method is called before we write messages to the socket using zero-copy transfer. We need to
- * make sure all the messages in the message set have the expected magic value.
- *
- * @param expectedMagicValue the magic value expected
- * @return true if all messages have expected magic value, false otherwise
- */
- override def isMagicValueInAllWrapperMessages(expectedMagicValue: Byte): Boolean = {
- var location = start
- val offsetAndSizeBuffer = ByteBuffer.allocate(MessageSet.LogOverhead)
- val crcAndMagicByteBuffer = ByteBuffer.allocate(Message.CrcLength + Message.MagicLength)
- while (location < end) {
- offsetAndSizeBuffer.rewind()
- channel.read(offsetAndSizeBuffer, location)
- if (offsetAndSizeBuffer.hasRemaining)
- return true
- offsetAndSizeBuffer.rewind()
- offsetAndSizeBuffer.getLong // skip offset field
- val messageSize = offsetAndSizeBuffer.getInt
- if (messageSize < Message.MinMessageOverhead)
- throw new IllegalStateException("Invalid message size: " + messageSize)
- crcAndMagicByteBuffer.rewind()
- channel.read(crcAndMagicByteBuffer, location + MessageSet.LogOverhead)
- if (crcAndMagicByteBuffer.get(Message.MagicOffset) != expectedMagicValue)
- return false
- location += (MessageSet.LogOverhead + messageSize)
- }
- true
- }
-
- /**
- * Convert this message set to use the specified message format.
- */
- def toMessageFormat(toMagicValue: Byte): MessageSet = {
- val offsets = new ArrayBuffer[Long]
- val newMessages = new ArrayBuffer[Message]
- this.foreach { messageAndOffset =>
- val message = messageAndOffset.message
- if (message.compressionCodec == NoCompressionCodec) {
- newMessages += message.toFormatVersion(toMagicValue)
- offsets += messageAndOffset.offset
- } else {
- // File message set only has shallow iterator. We need to do deep iteration here if needed.
- val deepIter = ByteBufferMessageSet.deepIterator(messageAndOffset)
- for (innerMessageAndOffset <- deepIter) {
- newMessages += innerMessageAndOffset.message.toFormatVersion(toMagicValue)
- offsets += innerMessageAndOffset.offset
- }
- }
- }
-
- if (sizeInBytes > 0 && newMessages.isEmpty) {
- // This indicates that the message is too large. We just return all the bytes in the file message set.
- this
- } else {
- // We use the offset seq to assign offsets so the offset of the messages does not change.
- new ByteBufferMessageSet(
- compressionCodec = this.headOption.map(_.message.compressionCodec).getOrElse(NoCompressionCodec),
- offsetSeq = offsets,
- newMessages: _*)
- }
- }
-
- /**
- * Get a shallow iterator over the messages in the set.
- */
- override def iterator: Iterator[MessageAndOffset] = iterator(Int.MaxValue)
-
- /**
- * Get an iterator over the messages in the set. We only do shallow iteration here.
- * @param maxMessageSize A limit on allowable message size to avoid allocating unbounded memory.
- * If we encounter a message larger than this we throw an InvalidMessageException.
- * @return The iterator.
- */
- def iterator(maxMessageSize: Int): Iterator[MessageAndOffset] = {
- new IteratorTemplate[MessageAndOffset] {
- var location = start
- val sizeOffsetLength = 12
- val sizeOffsetBuffer = ByteBuffer.allocate(sizeOffsetLength)
-
- override def makeNext(): MessageAndOffset = {
- if(location + sizeOffsetLength >= end)
- return allDone()
-
- // read the size of the item
- sizeOffsetBuffer.rewind()
- channel.read(sizeOffsetBuffer, location)
- if(sizeOffsetBuffer.hasRemaining)
- return allDone()
-
- sizeOffsetBuffer.rewind()
- val offset = sizeOffsetBuffer.getLong()
- val size = sizeOffsetBuffer.getInt()
- if(size < Message.MinMessageOverhead || location + sizeOffsetLength + size > end)
- return allDone()
- if(size > maxMessageSize)
- throw new CorruptRecordException("Message size exceeds the largest allowable message size (%d).".format(maxMessageSize))
-
- // read the item itself
- val buffer = ByteBuffer.allocate(size)
- channel.read(buffer, location + sizeOffsetLength)
- if(buffer.hasRemaining)
- return allDone()
- buffer.rewind()
-
- // increment the location and return the item
- location += size + sizeOffsetLength
- MessageAndOffset(new Message(buffer), offset)
- }
- }
- }
-
- /**
- * The number of bytes taken up by this file set
- */
- def sizeInBytes(): Int = _size.get()
-
- /**
- * Append these messages to the message set
- */
- def append(messages: ByteBufferMessageSet) {
- val written = messages.writeFullyTo(channel)
- _size.getAndAdd(written)
- }
-
- /**
- * Commit all written data to the physical disk
- */
- def flush() = {
- channel.force(true)
- }
-
- /**
- * Close this message set
- */
- def close() {
- flush()
- trim()
- channel.close()
- }
-
- /**
- * Trim file when close or roll to next file
- */
- def trim() {
- truncateTo(sizeInBytes())
- }
-
- /**
- * Delete this message set from the filesystem
- * @return True iff this message set was deleted.
- */
- def delete(): Boolean = {
- CoreUtils.swallow(channel.close())
- file.delete()
- }
-
- /**
- * Truncate this file message set to the given size in bytes. Note that this API does no checking that the
- * given size falls on a valid message boundary.
- * In some versions of the JDK truncating to the same size as the file message set will cause an
- * update of the files mtime, so truncate is only performed if the targetSize is smaller than the
- * size of the underlying FileChannel.
- * It is expected that no other threads will do writes to the log when this function is called.
- * @param targetSize The size to truncate to. Must be between 0 and sizeInBytes.
- * @return The number of bytes truncated off
- */
- def truncateTo(targetSize: Int): Int = {
- val originalSize = sizeInBytes
- if(targetSize > originalSize || targetSize < 0)
- throw new KafkaException("Attempt to truncate log segment to " + targetSize + " bytes failed, " +
- " size of this log segment is " + originalSize + " bytes.")
- if (targetSize < channel.size.toInt) {
- channel.truncate(targetSize)
- channel.position(targetSize)
- _size.set(targetSize)
- }
- originalSize - targetSize
- }
-
- /**
- * Read from the underlying file into the buffer starting at the given position
- */
- def readInto(buffer: ByteBuffer, relativePosition: Int): ByteBuffer = {
- channel.read(buffer, relativePosition + this.start)
- buffer.flip()
- buffer
- }
-
- /**
- * Rename the file that backs this message set
- * @throws IOException if rename fails.
- */
- def renameTo(f: File) {
- try Utils.atomicMoveWithFallback(file.toPath, f.toPath)
- finally this.file = f
- }
-
-}
-
-object FileMessageSet extends Logging
-{
- //preserve the previous logger name after moving logger aspect from FileMessageSet to companion
- override val loggerName = classOf[FileMessageSet].getName
-
- /**
- * Open a channel for the given file
- * For windows NTFS and some old LINUX file system, set preallocate to true and initFileSize
- * with one value (for example 512 * 1025 *1024 ) can improve the kafka produce performance.
- * @param file File path
- * @param mutable mutable
- * @param fileAlreadyExists File already exists or not
- * @param initFileSize The size used for pre allocate file, for example 512 * 1025 *1024
- * @param preallocate Pre allocate file or not, gotten from configuration.
- */
- def openChannel(file: File, mutable: Boolean, fileAlreadyExists: Boolean = false, initFileSize: Int = 0, preallocate: Boolean = false): FileChannel = {
- if (mutable) {
- if (fileAlreadyExists)
- new RandomAccessFile(file, "rw").getChannel()
- else {
- if (preallocate) {
- val randomAccessFile = new RandomAccessFile(file, "rw")
- randomAccessFile.setLength(initFileSize)
- randomAccessFile.getChannel()
- }
- else
- new RandomAccessFile(file, "rw").getChannel()
- }
- }
- else
- new FileInputStream(file).getChannel()
- }
-}
-
-object LogFlushStats extends KafkaMetricsGroup {
- val logFlushTimer = new KafkaTimer(newTimer("LogFlushRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS))
-}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/log/Log.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala
index 6acc8d2..d58a066 100644
--- a/core/src/main/scala/kafka/log/Log.scala
+++ b/core/src/main/scala/kafka/log/Log.scala
@@ -19,7 +19,6 @@ package kafka.log
import kafka.api.KAFKA_0_10_0_IV0
import kafka.utils._
-import kafka.message._
import kafka.common._
import kafka.metrics.KafkaMetricsGroup
import kafka.server.{BrokerTopicStats, FetchDataInfo, LogOffsetMetadata}
@@ -29,16 +28,18 @@ import java.util.concurrent.atomic._
import java.text.NumberFormat
import org.apache.kafka.common.errors.{CorruptRecordException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException, UnsupportedForMessageFormatException}
-import org.apache.kafka.common.record.TimestampType
+import org.apache.kafka.common.record._
import org.apache.kafka.common.requests.ListOffsetRequest
import scala.collection.Seq
import scala.collection.JavaConverters._
import com.yammer.metrics.core.Gauge
import org.apache.kafka.common.utils.{Time, Utils}
+import kafka.message.{BrokerCompressionCodec, CompressionCodec, NoCompressionCodec}
object LogAppendInfo {
- val UnknownLogAppendInfo = LogAppendInfo(-1, -1, Message.NoTimestamp, -1L, Message.NoTimestamp, NoCompressionCodec, NoCompressionCodec, -1, -1, offsetsMonotonic = false)
+ val UnknownLogAppendInfo = LogAppendInfo(-1, -1, Record.NO_TIMESTAMP, -1L, Record.NO_TIMESTAMP,
+ NoCompressionCodec, NoCompressionCodec, -1, -1, offsetsMonotonic = false)
}
/**
@@ -243,7 +244,7 @@ class Log(@volatile var dir: File,
val index = new OffsetIndex(indexFile, baseOffset = startOffset, maxIndexSize = config.maxIndexSize)
val timeIndexFile = new File(CoreUtils.replaceSuffix(logFile.getPath, LogFileSuffix, TimeIndexFileSuffix) + SwapFileSuffix)
val timeIndex = new TimeIndex(timeIndexFile, baseOffset = startOffset, maxIndexSize = config.maxIndexSize)
- val swapSegment = new LogSegment(new FileMessageSet(file = swapFile),
+ val swapSegment = new LogSegment(FileRecords.open(swapFile),
index = index,
timeIndex = timeIndex,
baseOffset = startOffset,
@@ -338,20 +339,20 @@ class Log(@volatile var dir: File,
* This method will generally be responsible for assigning offsets to the messages,
* however if the assignOffsets=false flag is passed we will only check that the existing offsets are valid.
*
- * @param messages The message set to append
+ * @param records The log records to append
* @param assignOffsets Should the log assign offsets to this message set or blindly apply what it is given
* @throws KafkaStorageException If the append fails due to an I/O error.
* @return Information about the appended messages including the first and last offset.
*/
- def append(messages: ByteBufferMessageSet, assignOffsets: Boolean = true): LogAppendInfo = {
- val appendInfo = analyzeAndValidateMessageSet(messages)
+ def append(records: MemoryRecords, assignOffsets: Boolean = true): LogAppendInfo = {
+ val appendInfo = analyzeAndValidateRecords(records)
// if we have any valid messages, append them to the log
if (appendInfo.shallowCount == 0)
return appendInfo
// trim any invalid bytes or partial messages before appending it to the on-disk log
- var validMessages = trimInvalidBytes(messages, appendInfo)
+ var validRecords = trimInvalidBytes(records, appendInfo)
try {
// they are valid, insert them in the log
@@ -363,20 +364,21 @@ class Log(@volatile var dir: File,
appendInfo.firstOffset = offset.value
val now = time.milliseconds
val validateAndOffsetAssignResult = try {
- validMessages.validateMessagesAndAssignOffsets(offset,
- now,
- appendInfo.sourceCodec,
- appendInfo.targetCodec,
- config.compact,
- config.messageFormatVersion.messageFormatVersion,
- config.messageTimestampType,
- config.messageTimestampDifferenceMaxMs)
+ LogValidator.validateMessagesAndAssignOffsets(validRecords,
+ offset,
+ now,
+ appendInfo.sourceCodec,
+ appendInfo.targetCodec,
+ config.compact,
+ config.messageFormatVersion.messageFormatVersion,
+ config.messageTimestampType,
+ config.messageTimestampDifferenceMaxMs)
} catch {
case e: IOException => throw new KafkaException("Error in validating messages while appending to log '%s'".format(name), e)
}
- validMessages = validateAndOffsetAssignResult.validatedMessages
+ validRecords = validateAndOffsetAssignResult.validatedRecords
appendInfo.maxTimestamp = validateAndOffsetAssignResult.maxTimestamp
- appendInfo.offsetOfMaxTimestamp = validateAndOffsetAssignResult.offsetOfMaxTimestamp
+ appendInfo.offsetOfMaxTimestamp = validateAndOffsetAssignResult.shallowOffsetOfMaxTimestamp
appendInfo.lastOffset = offset.value - 1
if (config.messageTimestampType == TimestampType.LOG_APPEND_TIME)
appendInfo.logAppendTime = now
@@ -384,14 +386,14 @@ class Log(@volatile var dir: File,
// re-validate message sizes if there's a possibility that they have changed (due to re-compression or message
// format conversion)
if (validateAndOffsetAssignResult.messageSizeMaybeChanged) {
- for (messageAndOffset <- validMessages.shallowIterator) {
- if (MessageSet.entrySize(messageAndOffset.message) > config.maxMessageSize) {
+ for (logEntry <- validRecords.shallowIterator.asScala) {
+ if (logEntry.sizeInBytes > config.maxMessageSize) {
// we record the original message set size instead of the trimmed size
// to be consistent with pre-compression bytesRejectedRate recording
- BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(messages.sizeInBytes)
- BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(messages.sizeInBytes)
+ BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(records.sizeInBytes)
+ BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(records.sizeInBytes)
throw new RecordTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d."
- .format(MessageSet.entrySize(messageAndOffset.message), config.maxMessageSize))
+ .format(logEntry.sizeInBytes, config.maxMessageSize))
}
}
}
@@ -399,28 +401,27 @@ class Log(@volatile var dir: File,
} else {
// we are taking the offsets we are given
if (!appendInfo.offsetsMonotonic || appendInfo.firstOffset < nextOffsetMetadata.messageOffset)
- throw new IllegalArgumentException("Out of order offsets found in " + messages)
+ throw new IllegalArgumentException("Out of order offsets found in " + records.deepIterator.asScala.map(_.offset))
}
// check messages set size may be exceed config.segmentSize
- if (validMessages.sizeInBytes > config.segmentSize) {
+ if (validRecords.sizeInBytes > config.segmentSize) {
throw new RecordBatchTooLargeException("Message set size is %d bytes which exceeds the maximum configured segment size of %d."
- .format(validMessages.sizeInBytes, config.segmentSize))
+ .format(validRecords.sizeInBytes, config.segmentSize))
}
// maybe roll the log if this segment is full
- val segment = maybeRoll(messagesSize = validMessages.sizeInBytes,
- maxTimestampInMessages = appendInfo.maxTimestamp)
+ val segment = maybeRoll(messagesSize = validRecords.sizeInBytes, maxTimestampInMessages = appendInfo.maxTimestamp)
// now append to the log
segment.append(firstOffset = appendInfo.firstOffset, largestTimestamp = appendInfo.maxTimestamp,
- offsetOfLargestTimestamp = appendInfo.offsetOfMaxTimestamp, messages = validMessages)
+ shallowOffsetOfMaxTimestamp = appendInfo.offsetOfMaxTimestamp, records = validRecords)
// increment the log end offset
updateLogEndOffset(appendInfo.lastOffset + 1)
trace("Appended message set to log %s with first offset: %d, next offset: %d, and messages: %s"
- .format(this.name, appendInfo.firstOffset, nextOffsetMetadata.messageOffset, validMessages))
+ .format(this.name, appendInfo.firstOffset, nextOffsetMetadata.messageOffset, validRecords))
if (unflushedMessages >= config.flushInterval)
flush()
@@ -449,73 +450,74 @@ class Log(@volatile var dir: File,
* <li> Whether any compression codec is used (if many are used, then the last one is given)
* </ol>
*/
- private def analyzeAndValidateMessageSet(messages: ByteBufferMessageSet): LogAppendInfo = {
+ private def analyzeAndValidateRecords(records: MemoryRecords): LogAppendInfo = {
var shallowMessageCount = 0
var validBytesCount = 0
var firstOffset, lastOffset = -1L
var sourceCodec: CompressionCodec = NoCompressionCodec
var monotonic = true
- var maxTimestamp = Message.NoTimestamp
+ var maxTimestamp = Record.NO_TIMESTAMP
var offsetOfMaxTimestamp = -1L
- for(messageAndOffset <- messages.shallowIterator) {
+ for (entry <- records.shallowIterator.asScala) {
// update the first offset if on the first message
if(firstOffset < 0)
- firstOffset = messageAndOffset.offset
+ firstOffset = entry.offset
// check that offsets are monotonically increasing
- if(lastOffset >= messageAndOffset.offset)
+ if(lastOffset >= entry.offset)
monotonic = false
// update the last offset seen
- lastOffset = messageAndOffset.offset
+ lastOffset = entry.offset
- val m = messageAndOffset.message
+ val record = entry.record
// Check if the message sizes are valid.
- val messageSize = MessageSet.entrySize(m)
+ val messageSize = entry.sizeInBytes
if(messageSize > config.maxMessageSize) {
- BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(messages.sizeInBytes)
- BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(messages.sizeInBytes)
+ BrokerTopicStats.getBrokerTopicStats(topicAndPartition.topic).bytesRejectedRate.mark(records.sizeInBytes)
+ BrokerTopicStats.getBrokerAllTopicsStats.bytesRejectedRate.mark(records.sizeInBytes)
throw new RecordTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d."
.format(messageSize, config.maxMessageSize))
}
// check the validity of the message by checking CRC
- m.ensureValid()
- if (m.timestamp > maxTimestamp) {
- maxTimestamp = m.timestamp
+ record.ensureValid()
+ if (record.timestamp > maxTimestamp) {
+ maxTimestamp = record.timestamp
offsetOfMaxTimestamp = lastOffset
}
shallowMessageCount += 1
validBytesCount += messageSize
- val messageCodec = m.compressionCodec
- if(messageCodec != NoCompressionCodec)
+ val messageCodec = CompressionCodec.getCompressionCodec(record.compressionType.id)
+ if (messageCodec != NoCompressionCodec)
sourceCodec = messageCodec
}
// Apply broker-side compression if any
val targetCodec = BrokerCompressionCodec.getTargetCompressionCodec(config.compressionType, sourceCodec)
- LogAppendInfo(firstOffset, lastOffset, maxTimestamp, offsetOfMaxTimestamp, Message.NoTimestamp, sourceCodec, targetCodec, shallowMessageCount, validBytesCount, monotonic)
+ LogAppendInfo(firstOffset, lastOffset, maxTimestamp, offsetOfMaxTimestamp, Record.NO_TIMESTAMP, sourceCodec,
+ targetCodec, shallowMessageCount, validBytesCount, monotonic)
}
/**
* Trim any invalid bytes from the end of this message set (if there are any)
*
- * @param messages The message set to trim
+ * @param records The records to trim
* @param info The general information of the message set
* @return A trimmed message set. This may be the same as what was passed in or it may not.
*/
- private def trimInvalidBytes(messages: ByteBufferMessageSet, info: LogAppendInfo): ByteBufferMessageSet = {
- val messageSetValidBytes = info.validBytes
- if(messageSetValidBytes < 0)
- throw new CorruptRecordException("Illegal length of message set " + messageSetValidBytes + " Message set cannot be appended to log. Possible causes are corrupted produce requests")
- if(messageSetValidBytes == messages.sizeInBytes) {
- messages
+ private def trimInvalidBytes(records: MemoryRecords, info: LogAppendInfo): MemoryRecords = {
+ val validBytes = info.validBytes
+ if (validBytes < 0)
+ throw new CorruptRecordException("Illegal length of message set " + validBytes + " Message set cannot be appended to log. Possible causes are corrupted produce requests")
+ if (validBytes == records.sizeInBytes) {
+ records
} else {
// trim invalid bytes
- val validByteBuffer = messages.buffer.duplicate()
- validByteBuffer.limit(messageSetValidBytes)
- new ByteBufferMessageSet(validByteBuffer)
+ val validByteBuffer = records.buffer.duplicate()
+ validByteBuffer.limit(validBytes)
+ MemoryRecords.readableRecords(validByteBuffer)
}
}
@@ -538,7 +540,7 @@ class Log(@volatile var dir: File,
val currentNextOffsetMetadata = nextOffsetMetadata
val next = currentNextOffsetMetadata.messageOffset
if(startOffset == next)
- return FetchDataInfo(currentNextOffsetMetadata, MessageSet.Empty)
+ return FetchDataInfo(currentNextOffsetMetadata, MemoryRecords.EMPTY)
var entry = segments.floorEntry(startOffset)
@@ -578,7 +580,7 @@ class Log(@volatile var dir: File,
// okay we are beyond the end of the last segment with no data fetched although the start offset is in range,
// this can happen when all messages with offset larger than start offsets have been deleted.
// In this case, we will return the empty set with log end offset metadata
- FetchDataInfo(nextOffsetMetadata, MessageSet.Empty)
+ FetchDataInfo(nextOffsetMetadata, MemoryRecords.EMPTY)
}
/**
@@ -610,9 +612,9 @@ class Log(@volatile var dir: File,
val segmentsCopy = logSegments.toBuffer
// For the earliest and latest, we do not need to return the timestamp.
if (targetTimestamp == ListOffsetRequest.EARLIEST_TIMESTAMP)
- return Some(TimestampOffset(Message.NoTimestamp, segmentsCopy.head.baseOffset))
+ return Some(TimestampOffset(Record.NO_TIMESTAMP, segmentsCopy.head.baseOffset))
else if (targetTimestamp == ListOffsetRequest.LATEST_TIMESTAMP)
- return Some(TimestampOffset(Message.NoTimestamp, logEndOffset))
+ return Some(TimestampOffset(Record.NO_TIMESTAMP, logEndOffset))
val targetSeg = {
// Get all the segments whose largest timestamp is smaller than target timestamp
@@ -656,7 +658,7 @@ class Log(@volatile var dir: File,
if (segments.size == numToDelete)
roll()
// remove the segments for lookups
- deletable.foreach(deleteSegment(_))
+ deletable.foreach(deleteSegment)
}
numToDelete
}
@@ -865,7 +867,7 @@ class Log(@volatile var dir: File,
truncateFullyAndStartAt(targetOffset)
} else {
val deletable = logSegments.filter(segment => segment.baseOffset > targetOffset)
- deletable.foreach(deleteSegment(_))
+ deletable.foreach(deleteSegment)
activeSegment.truncateTo(targetOffset)
updateLogEndOffset(targetOffset)
this.recoveryPoint = math.min(targetOffset, this.recoveryPoint)
@@ -882,7 +884,7 @@ class Log(@volatile var dir: File,
debug("Truncate and start log '" + name + "' to " + newOffset)
lock synchronized {
val segmentsToDelete = logSegments.toList
- segmentsToDelete.foreach(deleteSegment(_))
+ segmentsToDelete.foreach(deleteSegment)
addSegment(new LogSegment(dir,
newOffset,
indexIntervalBytes = config.indexInterval,
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/log/LogCleaner.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala
index 4a76b0c..c5a73d5 100644
--- a/core/src/main/scala/kafka/log/LogCleaner.scala
+++ b/core/src/main/scala/kafka/log/LogCleaner.scala
@@ -17,20 +17,21 @@
package kafka.log
-import java.io.{DataOutputStream, File}
+import java.io.File
import java.nio._
import java.util.Date
import java.util.concurrent.{CountDownLatch, TimeUnit}
import com.yammer.metrics.core.Gauge
import kafka.common._
-import kafka.message._
import kafka.metrics.KafkaMetricsGroup
import kafka.utils._
+import org.apache.kafka.common.record.{FileRecords, LogEntry, MemoryRecords}
import org.apache.kafka.common.utils.Time
+import MemoryRecords.LogEntryFilter
-import scala.Iterable
import scala.collection._
+import JavaConverters._
/**
* The cleaner is responsible for removing obsolete records from logs which have the dedupe retention strategy.
@@ -390,10 +391,10 @@ private[log] class Cleaner(val id: Int,
val timeIndexFile = new File(segments.head.timeIndex.file.getPath + Log.CleanedFileSuffix)
indexFile.delete()
timeIndexFile.delete()
- val messages = new FileMessageSet(logFile, fileAlreadyExists = false, initFileSize = log.initFileSize(), preallocate = log.config.preallocate)
+ val records = FileRecords.open(logFile, false, log.initFileSize(), log.config.preallocate)
val index = new OffsetIndex(indexFile, segments.head.baseOffset, segments.head.index.maxIndexSize)
val timeIndex = new TimeIndex(timeIndexFile, segments.head.baseOffset, segments.head.timeIndex.maxIndexSize)
- val cleaned = new LogSegment(messages, index, timeIndex, segments.head.baseOffset, segments.head.indexIntervalBytes, log.config.randomSegmentJitter, time)
+ val cleaned = new LogSegment(records, index, timeIndex, segments.head.baseOffset, segments.head.indexIntervalBytes, log.config.randomSegmentJitter, time)
try {
// clean segments into the new destination segment
@@ -449,8 +450,12 @@ private[log] class Cleaner(val id: Int,
retainDeletes: Boolean,
maxLogMessageSize: Int,
stats: CleanerStats) {
- def shouldRetain(messageAndOffset: MessageAndOffset): Boolean =
- shouldRetainMessage(source, map, retainDeletes, messageAndOffset, stats)
+ def shouldRetainEntry(logEntry: LogEntry): Boolean =
+ shouldRetainMessage(source, map, retainDeletes, logEntry, stats)
+
+ class LogCleanerFilter extends LogEntryFilter {
+ def shouldRetain(logEntry: LogEntry): Boolean = shouldRetainEntry(logEntry)
+ }
var position = 0
while (position < source.log.sizeInBytes) {
@@ -460,10 +465,9 @@ private[log] class Cleaner(val id: Int,
writeBuffer.clear()
source.log.readInto(readBuffer, position)
- val messages = new ByteBufferMessageSet(readBuffer)
- throttler.maybeThrottle(messages.sizeInBytes)
- val result = messages.filterInto(writeBuffer, shouldRetain)
-
+ val records = MemoryRecords.readableRecords(readBuffer)
+ throttler.maybeThrottle(records.sizeInBytes)
+ val result = records.filterTo(new LogCleanerFilter, writeBuffer)
stats.readMessages(result.messagesRead, result.bytesRead)
stats.recopyMessages(result.messagesRetained, result.bytesRetained)
@@ -472,9 +476,10 @@ private[log] class Cleaner(val id: Int,
// if any messages are to be retained, write them out
if (writeBuffer.position > 0) {
writeBuffer.flip()
- val retained = new ByteBufferMessageSet(writeBuffer)
- dest.append(firstOffset = retained.head.offset, largestTimestamp = result.maxTimestamp,
- offsetOfLargestTimestamp = result.offsetOfMaxTimestamp, messages = retained)
+
+ val retained = MemoryRecords.readableRecords(writeBuffer)
+ dest.append(firstOffset = retained.deepIterator().next().offset, largestTimestamp = result.maxTimestamp,
+ shallowOffsetOfMaxTimestamp = result.shallowOffsetOfMaxTimestamp, records = retained)
throttler.maybeThrottle(writeBuffer.limit)
}
@@ -488,21 +493,22 @@ private[log] class Cleaner(val id: Int,
private def shouldRetainMessage(source: kafka.log.LogSegment,
map: kafka.log.OffsetMap,
retainDeletes: Boolean,
- entry: kafka.message.MessageAndOffset,
+ entry: LogEntry,
stats: CleanerStats): Boolean = {
val pastLatestOffset = entry.offset > map.latestOffset
if (pastLatestOffset)
return true
- val key = entry.message.key
- if (key != null) {
+
+ if (entry.record.hasKey) {
+ val key = entry.record.key
val foundOffset = map.get(key)
/* two cases in which we can get rid of a message:
* 1) if there exists a message with the same key but higher offset
* 2) if the message is a delete "tombstone" marker and enough time has passed
*/
val redundant = foundOffset >= 0 && entry.offset < foundOffset
- val obsoleteDelete = !retainDeletes && entry.message.isNull
+ val obsoleteDelete = !retainDeletes && entry.record.hasNullValue
!redundant && !obsoleteDelete
} else {
stats.invalidMessage()
@@ -620,12 +626,12 @@ private[log] class Cleaner(val id: Int,
checkDone(topicAndPartition)
readBuffer.clear()
segment.log.readInto(readBuffer, position)
- val messages = new ByteBufferMessageSet(readBuffer)
- throttler.maybeThrottle(messages.sizeInBytes)
+ val records = MemoryRecords.readableRecords(readBuffer)
+ throttler.maybeThrottle(records.sizeInBytes)
val startPosition = position
- for (entry <- messages) {
- val message = entry.message
+ for (entry <- records.deepIterator.asScala) {
+ val message = entry.record
if (message.hasKey && entry.offset >= start) {
if (map.size < maxDesiredMapSize)
map.put(message.key, entry.offset)
@@ -634,8 +640,9 @@ private[log] class Cleaner(val id: Int,
}
stats.indexMessagesRead(1)
}
- position += messages.validBytes
- stats.indexBytesRead(messages.validBytes)
+ val bytesRead = records.validBytes
+ position += bytesRead
+ stats.indexBytesRead(bytesRead)
// if we didn't read even one complete message, our read buffer may be too small
if(position == startPosition)
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/log/LogManager.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala
index ed79946..953fca4 100755
--- a/core/src/main/scala/kafka/log/LogManager.scala
+++ b/core/src/main/scala/kafka/log/LogManager.scala
@@ -440,7 +440,7 @@ class LogManager(val logDirs: Array[File],
removedLog.dir = renamedDir
// change the file pointers for log and index file
for (logSegment <- removedLog.logSegments) {
- logSegment.log.file = new File(renamedDir, logSegment.log.file.getName)
+ logSegment.log.setFile(new File(renamedDir, logSegment.log.file.getName))
logSegment.index.file = new File(renamedDir, logSegment.index.file.getName)
}
[5/9] kafka git commit: KAFKA-4390;
Replace MessageSet usage with client-side alternatives
Posted by jg...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/log/LogSegment.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala
index c5418e3..bd4eb68 100755
--- a/core/src/main/scala/kafka/log/LogSegment.scala
+++ b/core/src/main/scala/kafka/log/LogSegment.scala
@@ -16,15 +16,20 @@
*/
package kafka.log
-import kafka.message._
+import java.io.{File, IOException}
+import java.util.concurrent.TimeUnit
+
import kafka.common._
-import kafka.utils._
+import kafka.metrics.{KafkaMetricsGroup, KafkaTimer}
import kafka.server.{FetchDataInfo, LogOffsetMetadata}
+import kafka.utils._
import org.apache.kafka.common.errors.CorruptRecordException
+import org.apache.kafka.common.record.FileRecords.LogEntryPosition
+import org.apache.kafka.common.record._
import org.apache.kafka.common.utils.Time
+import scala.collection.JavaConverters._
import scala.math._
-import java.io.{File, IOException}
/**
* A segment of the log. Each segment has two components: a log and an index. The log is a FileMessageSet containing
@@ -42,7 +47,7 @@ import java.io.{File, IOException}
* @param time The time instance
*/
@nonthreadsafe
-class LogSegment(val log: FileMessageSet,
+class LogSegment(val log: FileRecords,
val index: OffsetIndex,
val timeIndex: TimeIndex,
val baseOffset: Long,
@@ -63,7 +68,7 @@ class LogSegment(val log: FileMessageSet,
@volatile private var offsetOfMaxTimestamp = timeIndex.lastEntry.offset
def this(dir: File, startOffset: Long, indexIntervalBytes: Int, maxIndexSize: Int, rollJitterMs: Long, time: Time, fileAlreadyExists: Boolean = false, initFileSize: Int = 0, preallocate: Boolean = false) =
- this(new FileMessageSet(file = Log.logFilename(dir, startOffset), fileAlreadyExists = fileAlreadyExists, initFileSize = initFileSize, preallocate = preallocate),
+ this(FileRecords.open(Log.logFilename(dir, startOffset), fileAlreadyExists, initFileSize, preallocate),
new OffsetIndex(Log.indexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize),
new TimeIndex(Log.timeIndexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize),
startOffset,
@@ -82,23 +87,25 @@ class LogSegment(val log: FileMessageSet,
*
* @param firstOffset The first offset in the message set.
* @param largestTimestamp The largest timestamp in the message set.
- * @param offsetOfLargestTimestamp The offset of the message that has the largest timestamp in the messages to append.
- * @param messages The messages to append.
+ * @param shallowOffsetOfMaxTimestamp The offset of the message that has the largest timestamp in the messages to append.
+ * @param records The log entries to append.
*/
@nonthreadsafe
- def append(firstOffset: Long, largestTimestamp: Long, offsetOfLargestTimestamp: Long, messages: ByteBufferMessageSet) {
- if (messages.sizeInBytes > 0) {
- trace("Inserting %d bytes at offset %d at position %d with largest timestamp %d at offset %d"
- .format(messages.sizeInBytes, firstOffset, log.sizeInBytes(), largestTimestamp, offsetOfLargestTimestamp))
+ def append(firstOffset: Long, largestTimestamp: Long, shallowOffsetOfMaxTimestamp: Long, records: MemoryRecords) {
+ if (records.sizeInBytes > 0) {
+ trace("Inserting %d bytes at offset %d at position %d with largest timestamp %d at shallow offset %d"
+ .format(records.sizeInBytes, firstOffset, log.sizeInBytes(), largestTimestamp, shallowOffsetOfMaxTimestamp))
val physicalPosition = log.sizeInBytes()
if (physicalPosition == 0)
rollingBasedTimestamp = Some(largestTimestamp)
// append the messages
- log.append(messages)
+ val appendedBytes = log.append(records)
+ trace(s"Appended $appendedBytes to ${log.file()} at offset $firstOffset")
+
// Update the in memory max timestamp and corresponding offset.
if (largestTimestamp > maxTimestampSoFar) {
maxTimestampSoFar = largestTimestamp
- offsetOfMaxTimestamp = offsetOfLargestTimestamp
+ offsetOfMaxTimestamp = shallowOffsetOfMaxTimestamp
}
// append an entry to the index (if needed)
if(bytesSinceLastIndexEntry > indexIntervalBytes) {
@@ -106,7 +113,7 @@ class LogSegment(val log: FileMessageSet,
timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp)
bytesSinceLastIndexEntry = 0
}
- bytesSinceLastIndexEntry += messages.sizeInBytes
+ bytesSinceLastIndexEntry += records.sizeInBytes
}
}
@@ -123,7 +130,7 @@ class LogSegment(val log: FileMessageSet,
* message or null if no message meets this criteria.
*/
@threadsafe
- private[log] def translateOffset(offset: Long, startingFilePosition: Int = 0): (OffsetPosition, Int) = {
+ private[log] def translateOffset(offset: Long, startingFilePosition: Int = 0): LogEntryPosition = {
val mapping = index.lookup(offset)
log.searchForOffsetWithSize(offset, max(mapping.position, startingFilePosition))
}
@@ -154,40 +161,40 @@ class LogSegment(val log: FileMessageSet,
if (startOffsetAndSize == null)
return null
- val (startPosition, messageSetSize) = startOffsetAndSize
- val offsetMetadata = new LogOffsetMetadata(startOffset, this.baseOffset, startPosition.position)
+ val startPosition = startOffsetAndSize.position.toInt
+ val offsetMetadata = new LogOffsetMetadata(startOffset, this.baseOffset, startPosition)
val adjustedMaxSize =
- if (minOneMessage) math.max(maxSize, messageSetSize)
+ if (minOneMessage) math.max(maxSize, startOffsetAndSize.size)
else maxSize
// return a log segment but with zero size in the case below
if (adjustedMaxSize == 0)
- return FetchDataInfo(offsetMetadata, MessageSet.Empty)
+ return FetchDataInfo(offsetMetadata, MemoryRecords.EMPTY)
// calculate the length of the message set to read based on whether or not they gave us a maxOffset
val length = maxOffset match {
case None =>
// no max offset, just read until the max position
- min((maxPosition - startPosition.position).toInt, adjustedMaxSize)
+ min((maxPosition - startPosition).toInt, adjustedMaxSize)
case Some(offset) =>
// there is a max offset, translate it to a file position and use that to calculate the max read size;
// when the leader of a partition changes, it's possible for the new leader's high watermark to be less than the
// true high watermark in the previous leader for a short window. In this window, if a consumer fetches on an
// offset between new leader's high watermark and the log end offset, we want to return an empty response.
if (offset < startOffset)
- return FetchDataInfo(offsetMetadata, MessageSet.Empty, firstMessageSetIncomplete = false)
- val mapping = translateOffset(offset, startPosition.position)
+ return FetchDataInfo(offsetMetadata, MemoryRecords.EMPTY, firstEntryIncomplete = false)
+ val mapping = translateOffset(offset, startPosition)
val endPosition =
if (mapping == null)
logSize // the max offset is off the end of the log, use the end of the file
else
- mapping._1.position
- min(min(maxPosition, endPosition) - startPosition.position, adjustedMaxSize).toInt
+ mapping.position
+ min(min(maxPosition, endPosition) - startPosition, adjustedMaxSize).toInt
}
- FetchDataInfo(offsetMetadata, log.read(startPosition.position, length),
- firstMessageSetIncomplete = adjustedMaxSize < messageSetSize)
+ FetchDataInfo(offsetMetadata, log.read(startPosition, length),
+ firstEntryIncomplete = adjustedMaxSize < startOffsetAndSize.size)
}
/**
@@ -205,16 +212,16 @@ class LogSegment(val log: FileMessageSet,
timeIndex.resize(timeIndex.maxIndexSize)
var validBytes = 0
var lastIndexEntry = 0
- val iter = log.iterator(maxMessageSize)
- maxTimestampSoFar = Message.NoTimestamp
+ val iter = log.shallowIterator(maxMessageSize)
+ maxTimestampSoFar = Record.NO_TIMESTAMP
try {
- while(iter.hasNext) {
- val entry = iter.next
- entry.message.ensureValid()
+ for (entry <- iter.asScala) {
+ val record = entry.record
+ record.ensureValid()
// The max timestamp should have been put in the outer message, so we don't need to iterate over the inner messages.
- if (entry.message.timestamp > maxTimestampSoFar) {
- maxTimestampSoFar = entry.message.timestamp
+ if (record.timestamp > maxTimestampSoFar) {
+ maxTimestampSoFar = record.timestamp
offsetOfMaxTimestamp = entry.offset
}
@@ -225,11 +232,12 @@ class LogSegment(val log: FileMessageSet,
timeIndex.maybeAppend(maxTimestampSoFar, offsetOfMaxTimestamp)
lastIndexEntry = validBytes
}
- validBytes += MessageSet.entrySize(entry.message)
+ validBytes += entry.sizeInBytes()
}
} catch {
case e: CorruptRecordException =>
- logger.warn("Found invalid messages in log segment %s at byte offset %d: %s.".format(log.file.getAbsolutePath, validBytes, e.getMessage))
+ logger.warn("Found invalid messages in log segment %s at byte offset %d: %s."
+ .format(log.file.getAbsolutePath, validBytes, e.getMessage))
}
val truncated = log.sizeInBytes - validBytes
log.truncateTo(validBytes)
@@ -276,8 +284,7 @@ class LogSegment(val log: FileMessageSet,
// after truncation, reset and allocate more space for the (new currently active) index
index.resize(index.maxIndexSize)
timeIndex.resize(timeIndex.maxIndexSize)
- val (offsetPosition, _) = mapping
- val bytesTruncated = log.truncateTo(offsetPosition.position)
+ val bytesTruncated = log.truncateTo(mapping.position.toInt)
if(log.sizeInBytes == 0) {
created = time.milliseconds
rollingBasedTimestamp = None
@@ -296,10 +303,10 @@ class LogSegment(val log: FileMessageSet,
@threadsafe
def nextOffset(): Long = {
val ms = read(index.lastOffset, None, log.sizeInBytes)
- if(ms == null) {
+ if (ms == null) {
baseOffset
} else {
- ms.messageSet.lastOption match {
+ ms.records.shallowIterator.asScala.toSeq.lastOption match {
case None => baseOffset
case Some(last) => last.nextOffset
}
@@ -360,9 +367,9 @@ class LogSegment(val log: FileMessageSet,
def timeWaitedForRoll(now: Long, messageTimestamp: Long) : Long = {
// Load the timestamp of the first message into memory
if (rollingBasedTimestamp.isEmpty) {
- val iter = log.iterator
+ val iter = log.shallowIterator
if (iter.hasNext)
- rollingBasedTimestamp = Some(iter.next.message.timestamp)
+ rollingBasedTimestamp = Some(iter.next.record.timestamp)
}
rollingBasedTimestamp match {
case Some(t) if t >= 0 => messageTimestamp - t
@@ -394,8 +401,11 @@ class LogSegment(val log: FileMessageSet,
// Get the index entry with a timestamp less than or equal to the target timestamp
val timestampOffset = timeIndex.lookup(timestamp)
val position = index.lookup(timestampOffset.offset).position
+
// Search the timestamp
- log.searchForTimestamp(timestamp, position)
+ Option(log.searchForTimestamp(timestamp, position)).map { timestampAndOffset =>
+ TimestampOffset(timestampAndOffset.timestamp, timestampAndOffset.offset)
+ }
}
/**
@@ -444,3 +454,7 @@ class LogSegment(val log: FileMessageSet,
timeIndex.file.setLastModified(ms)
}
}
+
+object LogFlushStats extends KafkaMetricsGroup {
+ val logFlushTimer = new KafkaTimer(newTimer("LogFlushRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS))
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/log/LogValidator.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/LogValidator.scala b/core/src/main/scala/kafka/log/LogValidator.scala
new file mode 100644
index 0000000..d9f27e4
--- /dev/null
+++ b/core/src/main/scala/kafka/log/LogValidator.scala
@@ -0,0 +1,239 @@
+/**
+ * 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.log
+
+import java.nio.ByteBuffer
+
+import kafka.common.LongRef
+import kafka.message.{CompressionCodec, InvalidMessageException, NoCompressionCodec}
+import org.apache.kafka.common.errors.InvalidTimestampException
+import org.apache.kafka.common.record._
+
+import scala.collection.mutable
+import scala.collection.JavaConverters._
+
+private[kafka] object LogValidator {
+
+ /**
+ * Update the offsets for this message set and do further validation on messages including:
+ * 1. Messages for compacted topics must have keys
+ * 2. When magic value = 1, inner messages of a compressed message set must have monotonically increasing offsets
+ * starting from 0.
+ * 3. When magic value = 1, validate and maybe overwrite timestamps of messages.
+ *
+ * This method will convert the messages in the following scenarios:
+ * A. Magic value of a message = 0 and messageFormatVersion is 1
+ * B. Magic value of a message = 1 and messageFormatVersion is 0
+ *
+ * If no format conversion or value overwriting is required for messages, this method will perform in-place
+ * operations and avoid re-compression.
+ *
+ * Returns a ValidationAndOffsetAssignResult containing the validated message set, maximum timestamp, the offset
+ * of the shallow message with the max timestamp and a boolean indicating whether the message sizes may have changed.
+ */
+ private[kafka] def validateMessagesAndAssignOffsets(records: MemoryRecords,
+ offsetCounter: LongRef,
+ now: Long,
+ sourceCodec: CompressionCodec,
+ targetCodec: CompressionCodec,
+ compactedTopic: Boolean = false,
+ messageFormatVersion: Byte = Record.CURRENT_MAGIC_VALUE,
+ messageTimestampType: TimestampType,
+ messageTimestampDiffMaxMs: Long): ValidationAndOffsetAssignResult = {
+ if (sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) {
+ // check the magic value
+ if (!records.hasMatchingShallowMagic(messageFormatVersion))
+ convertAndAssignOffsetsNonCompressed(records, offsetCounter, compactedTopic, now, messageTimestampType,
+ messageTimestampDiffMaxMs, messageFormatVersion)
+ else
+ // Do in-place validation, offset assignment and maybe set timestamp
+ assignOffsetsNonCompressed(records, offsetCounter, now, compactedTopic, messageTimestampType,
+ messageTimestampDiffMaxMs)
+ } else {
+ // Deal with compressed messages
+ // We cannot do in place assignment in one of the following situations:
+ // 1. Source and target compression codec are different
+ // 2. When magic value to use is 0 because offsets need to be overwritten
+ // 3. When magic value to use is above 0, but some fields of inner messages need to be overwritten.
+ // 4. Message format conversion is needed.
+
+ // No in place assignment situation 1 and 2
+ var inPlaceAssignment = sourceCodec == targetCodec && messageFormatVersion > Record.MAGIC_VALUE_V0
+
+ var maxTimestamp = Record.NO_TIMESTAMP
+ val expectedInnerOffset = new LongRef(0)
+ val validatedRecords = new mutable.ArrayBuffer[Record]
+
+ records.deepIterator(true).asScala.foreach { logEntry =>
+ val record = logEntry.record
+ validateKey(record, compactedTopic)
+
+ if (record.magic > Record.MAGIC_VALUE_V0 && messageFormatVersion > Record.MAGIC_VALUE_V0) {
+ // No in place assignment situation 3
+ // Validate the timestamp
+ validateTimestamp(record, now, messageTimestampType, messageTimestampDiffMaxMs)
+ // Check if we need to overwrite offset
+ if (logEntry.offset != expectedInnerOffset.getAndIncrement())
+ inPlaceAssignment = false
+ if (record.timestamp > maxTimestamp)
+ maxTimestamp = record.timestamp
+ }
+
+ if (sourceCodec != NoCompressionCodec && logEntry.isCompressed)
+ throw new InvalidMessageException("Compressed outer record should not have an inner record with a " +
+ s"compression attribute set: $record")
+
+ // No in place assignment situation 4
+ if (record.magic != messageFormatVersion)
+ inPlaceAssignment = false
+
+ validatedRecords += record.convert(messageFormatVersion)
+ }
+
+ if (!inPlaceAssignment) {
+ val entries = validatedRecords.map(record => LogEntry.create(offsetCounter.getAndIncrement(), record))
+ val builder = MemoryRecords.builderWithEntries(messageTimestampType, CompressionType.forId(targetCodec.codec),
+ now, entries.asJava)
+ builder.close()
+ val info = builder.info
+
+ ValidationAndOffsetAssignResult(
+ validatedRecords = builder.build(),
+ maxTimestamp = info.maxTimestamp,
+ shallowOffsetOfMaxTimestamp = info.shallowOffsetOfMaxTimestamp,
+ messageSizeMaybeChanged = true)
+ } else {
+ // ensure the inner messages are valid
+ validatedRecords.foreach(_.ensureValid)
+
+ // we can update the wrapper message only and write the compressed payload as is
+ val entry = records.shallowIterator.next()
+ val offset = offsetCounter.addAndGet(validatedRecords.size) - 1
+ entry.setOffset(offset)
+ if (messageTimestampType == TimestampType.CREATE_TIME)
+ entry.setCreateTime(maxTimestamp)
+ else if (messageTimestampType == TimestampType.LOG_APPEND_TIME)
+ entry.setLogAppendTime(now)
+
+ ValidationAndOffsetAssignResult(validatedRecords = records,
+ maxTimestamp = if (messageTimestampType == TimestampType.LOG_APPEND_TIME) now else maxTimestamp,
+ shallowOffsetOfMaxTimestamp = offset,
+ messageSizeMaybeChanged = false)
+ }
+ }
+ }
+
+ private def convertAndAssignOffsetsNonCompressed(records: MemoryRecords,
+ offsetCounter: LongRef,
+ compactedTopic: Boolean,
+ now: Long,
+ timestampType: TimestampType,
+ messageTimestampDiffMaxMs: Long,
+ toMagicValue: Byte): ValidationAndOffsetAssignResult = {
+ val sizeInBytesAfterConversion = records.shallowIterator.asScala.map { logEntry =>
+ logEntry.record.convertedSize(toMagicValue)
+ }.sum
+
+ val newBuffer = ByteBuffer.allocate(sizeInBytesAfterConversion)
+ val builder = MemoryRecords.builder(newBuffer, toMagicValue, CompressionType.NONE, timestampType,
+ offsetCounter.value, now)
+
+ records.shallowIterator.asScala.foreach { logEntry =>
+ val record = logEntry.record
+ validateKey(record, compactedTopic)
+ validateTimestamp(record, now, timestampType, messageTimestampDiffMaxMs)
+ builder.convertAndAppend(offsetCounter.getAndIncrement(), record)
+ }
+
+ builder.close()
+ val info = builder.info
+
+ ValidationAndOffsetAssignResult(
+ validatedRecords = builder.build(),
+ maxTimestamp = info.maxTimestamp,
+ shallowOffsetOfMaxTimestamp = info.shallowOffsetOfMaxTimestamp,
+ messageSizeMaybeChanged = true)
+ }
+
+ private def assignOffsetsNonCompressed(records: MemoryRecords,
+ offsetCounter: LongRef,
+ now: Long,
+ compactedTopic: Boolean,
+ timestampType: TimestampType,
+ timestampDiffMaxMs: Long): ValidationAndOffsetAssignResult = {
+ var maxTimestamp = Record.NO_TIMESTAMP
+ var offsetOfMaxTimestamp = -1L
+ val firstOffset = offsetCounter.value
+
+ for (entry <- records.shallowIterator.asScala) {
+ val record = entry.record
+ validateKey(record, compactedTopic)
+
+ val offset = offsetCounter.getAndIncrement()
+ entry.setOffset(offset)
+
+ if (record.magic > Record.MAGIC_VALUE_V0) {
+ validateTimestamp(record, now, timestampType, timestampDiffMaxMs)
+
+ if (timestampType == TimestampType.LOG_APPEND_TIME)
+ entry.setLogAppendTime(now)
+ else if (record.timestamp > maxTimestamp) {
+ maxTimestamp = record.timestamp
+ offsetOfMaxTimestamp = offset
+ }
+ }
+ }
+
+ if (timestampType == TimestampType.LOG_APPEND_TIME) {
+ maxTimestamp = now
+ offsetOfMaxTimestamp = firstOffset
+ }
+
+ ValidationAndOffsetAssignResult(
+ validatedRecords = records,
+ maxTimestamp = maxTimestamp,
+ shallowOffsetOfMaxTimestamp = offsetOfMaxTimestamp,
+ messageSizeMaybeChanged = false)
+ }
+
+ private def validateKey(record: Record, compactedTopic: Boolean) {
+ if (compactedTopic && !record.hasKey)
+ throw new InvalidMessageException("Compacted topic cannot accept message without key.")
+ }
+
+ /**
+ * This method validates the timestamps of a message.
+ * If the message is using create time, this method checks if it is within acceptable range.
+ */
+ private def validateTimestamp(record: Record,
+ now: Long,
+ timestampType: TimestampType,
+ timestampDiffMaxMs: Long) {
+ if (timestampType == TimestampType.CREATE_TIME && math.abs(record.timestamp - now) > timestampDiffMaxMs)
+ throw new InvalidTimestampException(s"Timestamp ${record.timestamp} of message is out of range. " +
+ s"The timestamp should be within [${now - timestampDiffMaxMs}, ${now + timestampDiffMaxMs}")
+ if (record.timestampType == TimestampType.LOG_APPEND_TIME)
+ throw new InvalidTimestampException(s"Invalid timestamp type in message $record. Producer should not set " +
+ s"timestamp type to LogAppendTime.")
+ }
+
+ case class ValidationAndOffsetAssignResult(validatedRecords: MemoryRecords,
+ maxTimestamp: Long,
+ shallowOffsetOfMaxTimestamp: Long,
+ messageSizeMaybeChanged: Boolean)
+
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/log/TimeIndex.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/log/TimeIndex.scala b/core/src/main/scala/kafka/log/TimeIndex.scala
index 7f24081..21c212f 100644
--- a/core/src/main/scala/kafka/log/TimeIndex.scala
+++ b/core/src/main/scala/kafka/log/TimeIndex.scala
@@ -21,9 +21,9 @@ import java.io.File
import java.nio.ByteBuffer
import kafka.common.InvalidOffsetException
-import kafka.message.Message
import kafka.utils.CoreUtils._
import kafka.utils.Logging
+import org.apache.kafka.common.record.Record
/**
* An index that maps from the timestamp to the logical offsets of the messages in a segment. This index might be
@@ -69,7 +69,7 @@ class TimeIndex(file: File,
def lastEntry: TimestampOffset = {
inLock(lock) {
_entries match {
- case 0 => TimestampOffset(Message.NoTimestamp, baseOffset)
+ case 0 => TimestampOffset(Record.NO_TIMESTAMP, baseOffset)
case s => parseEntry(mmap, s - 1).asInstanceOf[TimestampOffset]
}
}
@@ -145,7 +145,7 @@ class TimeIndex(file: File,
val idx = mmap.duplicate
val slot = indexSlotFor(idx, targetTimestamp, IndexSearchType.KEY)
if (slot == -1)
- TimestampOffset(Message.NoTimestamp, baseOffset)
+ TimestampOffset(Record.NO_TIMESTAMP, baseOffset)
else {
val entry = parseEntry(idx, slot).asInstanceOf[TimestampOffset]
TimestampOffset(entry.timestamp, entry.offset)
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala
index 096344d..2c8fef6 100644
--- a/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala
+++ b/core/src/main/scala/kafka/message/ByteBufferMessageSet.scala
@@ -17,20 +17,13 @@
package kafka.message
-import kafka.utils.{CoreUtils, IteratorTemplate, Logging}
-import kafka.common.{KafkaException, LongRef}
import java.nio.ByteBuffer
-import java.nio.channels._
-import java.io._
-import java.util.ArrayDeque
-import kafka.message.ByteBufferMessageSet.FilterResult
-import org.apache.kafka.common.errors.InvalidTimestampException
-import org.apache.kafka.common.record.{MemoryRecords, TimestampType}
-import org.apache.kafka.common.utils.Utils
+import kafka.common.LongRef
+import kafka.utils.Logging
+import org.apache.kafka.common.record._
-import scala.collection.mutable
-import scala.collection.mutable.ArrayBuffer
+import scala.collection.JavaConverters._
object ByteBufferMessageSet {
@@ -41,204 +34,19 @@ object ByteBufferMessageSet {
messages: Message*): ByteBuffer = {
if (messages.isEmpty)
MessageSet.Empty.buffer
- else if (compressionCodec == NoCompressionCodec) {
- val buffer = ByteBuffer.allocate(MessageSet.messageSetSize(messages))
- for (message <- messages) writeMessage(buffer, message, offsetAssigner.nextAbsoluteOffset())
- buffer.rewind()
- buffer
- } else {
+ else {
val magicAndTimestamp = wrapperMessageTimestamp match {
case Some(ts) => MagicAndTimestamp(messages.head.magic, ts)
case None => MessageSet.magicAndLargestTimestamp(messages)
}
- val (messageWriter, lastOffset) = writeCompressedMessages(compressionCodec, offsetAssigner, magicAndTimestamp,
- timestampType, messages)
- val buffer = ByteBuffer.allocate(messageWriter.size + MessageSet.LogOverhead)
- writeMessage(buffer, messageWriter, lastOffset)
- buffer.rewind()
- buffer
- }
- }
-
- /** Deep iterator that decompresses the message sets and adjusts timestamp and offset if needed. */
- def deepIterator(wrapperMessageAndOffset: MessageAndOffset, ensureMatchingMagic: Boolean = false): Iterator[MessageAndOffset] = {
-
- import Message._
-
- new IteratorTemplate[MessageAndOffset] {
-
- val MessageAndOffset(wrapperMessage, wrapperMessageOffset) = wrapperMessageAndOffset
-
- if (wrapperMessage.payload == null)
- throw new KafkaException(s"Message payload is null: $wrapperMessage")
-
- val wrapperMessageTimestampOpt: Option[Long] =
- if (wrapperMessage.magic > MagicValue_V0) Some(wrapperMessage.timestamp) else None
- val wrapperMessageTimestampTypeOpt: Option[TimestampType] =
- if (wrapperMessage.magic > MagicValue_V0) Some(wrapperMessage.timestampType) else None
-
- var lastInnerOffset = -1L
-
- val messageAndOffsets = {
- val inputStream = new ByteBufferBackedInputStream(wrapperMessage.payload)
- val compressed = try {
- new DataInputStream(CompressionFactory(wrapperMessage.compressionCodec, wrapperMessage.magic, inputStream))
- } catch {
- case ioe: IOException =>
- throw new InvalidMessageException(s"Failed to instantiate input stream compressed with ${wrapperMessage.compressionCodec}", ioe)
- }
-
- val innerMessageAndOffsets = new ArrayDeque[MessageAndOffset]()
- try {
- while (true)
- innerMessageAndOffsets.add(readMessageFromStream(compressed))
- } catch {
- case _: EOFException =>
- // we don't do anything at all here, because the finally
- // will close the compressed input stream, and we simply
- // want to return the innerMessageAndOffsets
- case ioe: IOException =>
- throw new InvalidMessageException(s"Error while reading message from stream compressed with ${wrapperMessage.compressionCodec}", ioe)
- } finally {
- CoreUtils.swallow(compressed.close())
- }
-
- innerMessageAndOffsets
- }
- private def readMessageFromStream(compressed: DataInputStream): MessageAndOffset = {
- val innerOffset = compressed.readLong()
- val recordSize = compressed.readInt()
-
- if (recordSize < MinMessageOverhead)
- throw new InvalidMessageException(s"Message found with corrupt size `$recordSize` in deep iterator")
-
- // read the record into an intermediate record buffer (i.e. extra copy needed)
- val bufferArray = new Array[Byte](recordSize)
- compressed.readFully(bufferArray, 0, recordSize)
- val buffer = ByteBuffer.wrap(bufferArray)
-
- // Override the timestamp if necessary
- val newMessage = new Message(buffer, wrapperMessageTimestampOpt, wrapperMessageTimestampTypeOpt)
-
- // Due to KAFKA-4298, it is possible for the inner and outer magic values to differ. We ignore
- // this and depend on the outer message in order to decide how to compute the respective offsets
- // for the inner messages
- if (ensureMatchingMagic && newMessage.magic != wrapperMessage.magic)
- throw new InvalidMessageException(s"Compressed message has magic value ${wrapperMessage.magic} " +
- s"but inner message has magic value ${newMessage.magic}")
-
- lastInnerOffset = innerOffset
- MessageAndOffset(newMessage, innerOffset)
- }
-
- override def makeNext(): MessageAndOffset = {
- messageAndOffsets.pollFirst() match {
- case null => allDone()
- case nextMessage@ MessageAndOffset(message, offset) =>
- if (wrapperMessage.magic > MagicValue_V0) {
- val relativeOffset = offset - lastInnerOffset
- val absoluteOffset = wrapperMessageOffset + relativeOffset
- MessageAndOffset(message, absoluteOffset)
- } else {
- nextMessage
- }
- }
- }
+ val entries = messages.map(message => LogEntry.create(offsetAssigner.nextAbsoluteOffset(), message.asRecord))
+ val builder = MemoryRecords.builderWithEntries(timestampType, CompressionType.forId(compressionCodec.codec),
+ magicAndTimestamp.timestamp, entries.asJava)
+ builder.build().buffer
}
}
- private def writeCompressedMessages(codec: CompressionCodec,
- offsetAssigner: OffsetAssigner,
- magicAndTimestamp: MagicAndTimestamp,
- timestampType: TimestampType,
- messages: Seq[Message]): (MessageWriter, Long) = {
- require(codec != NoCompressionCodec, s"compressionCodec must not be $NoCompressionCodec")
- require(messages.nonEmpty, "cannot write empty compressed message set")
-
- var offset = -1L
- val magic = magicAndTimestamp.magic
- val messageWriter = new MessageWriter(math.min(math.max(MessageSet.messageSetSize(messages) / 2, 1024), 1 << 16))
- messageWriter.write(
- codec = codec,
- timestamp = magicAndTimestamp.timestamp,
- timestampType = timestampType,
- magicValue = magic) { outputStream =>
-
- val output = new DataOutputStream(CompressionFactory(codec, magic, outputStream))
- try {
- for (message <- messages) {
- offset = offsetAssigner.nextAbsoluteOffset()
-
- if (message.magic != magicAndTimestamp.magic)
- throw new IllegalArgumentException("Messages in the message set must have same magic value")
-
- // Use inner offset if magic value is greater than 0
- val innerOffset = if (magicAndTimestamp.magic > Message.MagicValue_V0)
- offsetAssigner.toInnerOffset(offset)
- else
- offset
-
- output.writeLong(innerOffset)
- output.writeInt(message.size)
- output.write(message.buffer.array, message.buffer.arrayOffset, message.buffer.limit)
- }
- } finally {
- output.close()
- }
- }
-
- (messageWriter, offset)
- }
-
- private[kafka] def writeCompressedMessages(buffer: ByteBuffer,
- codec: CompressionCodec,
- messageAndOffsets: Seq[MessageAndOffset]): Int = {
- require(codec != NoCompressionCodec, s"compressionCodec must not be $NoCompressionCodec")
-
- if (messageAndOffsets.isEmpty)
- 0
- else {
- val messages = messageAndOffsets.map(_.message)
- val magicAndTimestamp = MessageSet.magicAndLargestTimestamp(messages)
-
- // ensure that we use the magic from the first message in the set when writing the wrapper
- // message in order to fix message sets corrupted by KAFKA-4298
- val magic = magicAndTimestamp.magic
-
- val firstMessageAndOffset = messageAndOffsets.head
- val firstAbsoluteOffset = firstMessageAndOffset.offset
- val offsetAssigner = OffsetAssigner(firstAbsoluteOffset, magic, messageAndOffsets)
- val timestampType = firstMessageAndOffset.message.timestampType
-
- val (messageWriter, lastOffset) = writeCompressedMessages(codec, offsetAssigner, magicAndTimestamp,
- timestampType, messages)
-
- writeMessage(buffer, messageWriter, lastOffset)
- messageWriter.size + MessageSet.LogOverhead
- }
- }
-
- private[kafka] def writeMessage(buffer: ByteBuffer, message: Message, offset: Long) {
- buffer.putLong(offset)
- buffer.putInt(message.size)
- buffer.put(message.buffer)
- message.buffer.rewind()
- }
-
- private[kafka] def writeMessage(buffer: ByteBuffer, messageWriter: MessageWriter, offset: Long) {
- buffer.putLong(offset)
- buffer.putInt(messageWriter.size)
- messageWriter.writeTo(buffer)
- }
-
-
- case class FilterResult(messagesRead: Int,
- bytesRead: Int,
- messagesRetained: Int,
- bytesRetained: Int,
- maxTimestamp: Long,
- offsetOfMaxTimestamp: Long)
}
private object OffsetAssigner {
@@ -246,9 +54,6 @@ private object OffsetAssigner {
def apply(offsetCounter: LongRef, size: Int): OffsetAssigner =
new OffsetAssigner(offsetCounter.value to offsetCounter.addAndGet(size))
- def apply(baseOffset: Long, magic: Byte, messageAndOffsets: Seq[MessageAndOffset]): OffsetAssigner =
- new OffsetAssigner(messageAndOffsets.map(_.offset))
-
}
private class OffsetAssigner(offsets: Seq[Long]) {
@@ -322,7 +127,6 @@ private class OffsetAssigner(offsets: Seq[Long]) {
*
*/
class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Logging {
- private var shallowValidByteCount = -1
private[kafka] def this(compressionCodec: CompressionCodec,
offsetCounter: LongRef,
@@ -354,33 +158,6 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
override def asRecords: MemoryRecords = MemoryRecords.readableRecords(buffer.duplicate())
- private def shallowValidBytes: Int = {
- if (shallowValidByteCount < 0) {
- this.shallowValidByteCount = this.internalIterator(isShallow = true).map { messageAndOffset =>
- MessageSet.entrySize(messageAndOffset.message)
- }.sum
- }
- shallowValidByteCount
- }
-
- /** Write the messages in this set to the given channel */
- def writeFullyTo(channel: GatheringByteChannel): Int = {
- buffer.mark()
- var written = 0
- while (written < sizeInBytes)
- written += channel.write(buffer)
- buffer.reset()
- written
- }
-
- override def isMagicValueInAllWrapperMessages(expectedMagicValue: Byte): Boolean = {
- for (messageAndOffset <- shallowIterator) {
- if (messageAndOffset.message.magic != expectedMagicValue)
- return false
- }
- true
- }
-
/** default iterator that iterates over decompressed messages */
override def iterator: Iterator[MessageAndOffset] = internalIterator()
@@ -388,365 +165,12 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
def shallowIterator: Iterator[MessageAndOffset] = internalIterator(isShallow = true)
/** When flag isShallow is set to be true, we do a shallow iteration: just traverse the first level of messages. **/
- private def internalIterator(isShallow: Boolean = false, ensureMatchingMagic: Boolean = false): Iterator[MessageAndOffset] = {
- new IteratorTemplate[MessageAndOffset] {
- var topIter = buffer.slice()
- var innerIter: Iterator[MessageAndOffset] = null
-
- def innerDone(): Boolean = (innerIter == null || !innerIter.hasNext)
-
- def makeNextOuter: MessageAndOffset = {
- // if there isn't at least an offset and size, we are done
- if (topIter.remaining < 12)
- return allDone()
- val offset = topIter.getLong()
- val size = topIter.getInt()
- if(size < Message.MinMessageOverhead)
- throw new InvalidMessageException("Message found with corrupt size (" + size + ") in shallow iterator")
-
- // we have an incomplete message
- if(topIter.remaining < size)
- return allDone()
-
- // read the current message and check correctness
- val message = topIter.slice()
- message.limit(size)
- topIter.position(topIter.position + size)
- val newMessage = new Message(message)
- if(isShallow) {
- MessageAndOffset(newMessage, offset)
- } else {
- newMessage.compressionCodec match {
- case NoCompressionCodec =>
- innerIter = null
- MessageAndOffset(newMessage, offset)
- case _ =>
- innerIter = ByteBufferMessageSet.deepIterator(MessageAndOffset(newMessage, offset), ensureMatchingMagic)
- if(!innerIter.hasNext)
- innerIter = null
- makeNext()
- }
- }
- }
-
- override def makeNext(): MessageAndOffset = {
- if(isShallow){
- makeNextOuter
- } else {
- if(innerDone())
- makeNextOuter
- else
- innerIter.next()
- }
- }
-
- }
- }
-
- def filterInto(buffer: ByteBuffer,
- filter: MessageAndOffset => Boolean): FilterResult = {
- var maxTimestamp = Message.NoTimestamp
- var offsetOfMaxTimestamp = -1L
- var messagesRead = 0
- var bytesRead = 0
- var messagesRetained = 0
- var bytesRetained = 0
-
- for (shallowMessageAndOffset <- shallowIterator) {
- val shallowMessage = shallowMessageAndOffset.message
- val shallowOffset = shallowMessageAndOffset.offset
- val size = MessageSet.entrySize(shallowMessageAndOffset.message)
-
- messagesRead += 1
- bytesRead += size
-
- if (shallowMessageAndOffset.message.compressionCodec == NoCompressionCodec) {
- if (filter(shallowMessageAndOffset)) {
- ByteBufferMessageSet.writeMessage(buffer, shallowMessage, shallowOffset)
- messagesRetained += 1
- bytesRetained += size
-
- if (shallowMessage.timestamp > maxTimestamp) {
- maxTimestamp = shallowMessage.timestamp
- offsetOfMaxTimestamp = shallowOffset
- }
- }
- messagesRead += 1
- } else {
- // We use the absolute offset to decide whether to retain the message or not (this is handled by the
- // deep iterator). Because of KAFKA-4298, we have to allow for the possibility that a previous version
- // corrupted the log by writing a compressed message set with a wrapper magic value not matching the magic
- // of the inner messages. This will be fixed as we recopy the messages to the destination segment.
-
- var writeOriginalMessageSet = true
- val retainedMessages = ArrayBuffer[MessageAndOffset]()
- val shallowMagic = shallowMessage.magic
-
- for (deepMessageAndOffset <- ByteBufferMessageSet.deepIterator(shallowMessageAndOffset)) {
- messagesRead += 1
- if (filter(deepMessageAndOffset)) {
- // Check for log corruption due to KAFKA-4298. If we find it, make sure that we overwrite
- // the corrupted entry with correct data.
- if (shallowMagic != deepMessageAndOffset.message.magic)
- writeOriginalMessageSet = false
-
- retainedMessages += deepMessageAndOffset
- // We need the max timestamp and last offset for time index
- if (deepMessageAndOffset.message.timestamp > maxTimestamp)
- maxTimestamp = deepMessageAndOffset.message.timestamp
- }
- else writeOriginalMessageSet = false
- }
- offsetOfMaxTimestamp = if (retainedMessages.nonEmpty) retainedMessages.last.offset else -1L
- // There are no messages compacted out and no message format conversion, write the original message set back
- if (writeOriginalMessageSet)
- ByteBufferMessageSet.writeMessage(buffer, shallowMessage, shallowOffset)
- else if (retainedMessages.nonEmpty) {
- val compressedSize = ByteBufferMessageSet.writeCompressedMessages(buffer, shallowMessage.compressionCodec, retainedMessages)
- messagesRetained += 1
- bytesRetained += compressedSize
- }
- }
- }
-
- FilterResult(messagesRead, bytesRead, messagesRetained, bytesRetained, maxTimestamp, offsetOfMaxTimestamp)
- }
-
- /**
- * Update the offsets for this message set and do further validation on messages including:
- * 1. Messages for compacted topics must have keys
- * 2. When magic value = 1, inner messages of a compressed message set must have monotonically increasing offsets
- * starting from 0.
- * 3. When magic value = 1, validate and maybe overwrite timestamps of messages.
- *
- * This method will convert the messages in the following scenarios:
- * A. Magic value of a message = 0 and messageFormatVersion is 1
- * B. Magic value of a message = 1 and messageFormatVersion is 0
- *
- * If no format conversion or value overwriting is required for messages, this method will perform in-place
- * operations and avoid re-compression.
- *
- * Returns a ValidationAndOffsetAssignResult containing the validated message set, maximum timestamp, the offset
- * of the shallow message with the max timestamp and a boolean indicating whether the message sizes may have changed.
- */
- private[kafka] def validateMessagesAndAssignOffsets(offsetCounter: LongRef,
- now: Long,
- sourceCodec: CompressionCodec,
- targetCodec: CompressionCodec,
- compactedTopic: Boolean = false,
- messageFormatVersion: Byte = Message.CurrentMagicValue,
- messageTimestampType: TimestampType,
- messageTimestampDiffMaxMs: Long): ValidationAndOffsetAssignResult = {
- if (sourceCodec == NoCompressionCodec && targetCodec == NoCompressionCodec) {
- // check the magic value
- if (!isMagicValueInAllWrapperMessages(messageFormatVersion))
- convertNonCompressedMessages(offsetCounter, compactedTopic, now, messageTimestampType, messageTimestampDiffMaxMs,
- messageFormatVersion)
- else
- // Do in-place validation, offset assignment and maybe set timestamp
- validateNonCompressedMessagesAndAssignOffsetInPlace(offsetCounter, now, compactedTopic, messageTimestampType,
- messageTimestampDiffMaxMs)
- } else {
- // Deal with compressed messages
- // We cannot do in place assignment in one of the following situations:
- // 1. Source and target compression codec are different
- // 2. When magic value to use is 0 because offsets need to be overwritten
- // 3. When magic value to use is above 0, but some fields of inner messages need to be overwritten.
- // 4. Message format conversion is needed.
-
- // No in place assignment situation 1 and 2
- var inPlaceAssignment = sourceCodec == targetCodec && messageFormatVersion > Message.MagicValue_V0
-
- var maxTimestamp = Message.NoTimestamp
- var offsetOfMaxTimestamp = -1L
- val expectedInnerOffset = new LongRef(0)
- val validatedMessages = new mutable.ArrayBuffer[Message]
-
- this.internalIterator(isShallow = false, ensureMatchingMagic = true).foreach { messageAndOffset =>
- val message = messageAndOffset.message
- validateMessageKey(message, compactedTopic)
-
- if (message.magic > Message.MagicValue_V0 && messageFormatVersion > Message.MagicValue_V0) {
- // No in place assignment situation 3
- // Validate the timestamp
- validateTimestamp(message, now, messageTimestampType, messageTimestampDiffMaxMs)
- // Check if we need to overwrite offset
- if (messageAndOffset.offset != expectedInnerOffset.getAndIncrement())
- inPlaceAssignment = false
- if (message.timestamp > maxTimestamp) {
- maxTimestamp = message.timestamp
- offsetOfMaxTimestamp = offsetCounter.value + expectedInnerOffset.value - 1
- }
- }
-
- if (sourceCodec != NoCompressionCodec && message.compressionCodec != NoCompressionCodec)
- throw new InvalidMessageException("Compressed outer message should not have an inner message with a " +
- s"compression attribute set: $message")
-
- // No in place assignment situation 4
- if (message.magic != messageFormatVersion)
- inPlaceAssignment = false
-
- validatedMessages += message.toFormatVersion(messageFormatVersion)
- }
-
- if (!inPlaceAssignment) {
- // Cannot do in place assignment.
- val (largestTimestampOfMessageSet, offsetOfMaxTimestampInMessageSet) = {
- if (messageFormatVersion == Message.MagicValue_V0)
- (Some(Message.NoTimestamp), -1L)
- else if (messageTimestampType == TimestampType.CREATE_TIME)
- (Some(maxTimestamp), {if (targetCodec == NoCompressionCodec) offsetOfMaxTimestamp else offsetCounter.value + validatedMessages.length - 1})
- else // Log append time
- (Some(now), {if (targetCodec == NoCompressionCodec) offsetCounter.value else offsetCounter.value + validatedMessages.length - 1})
- }
-
- ValidationAndOffsetAssignResult(validatedMessages = new ByteBufferMessageSet(compressionCodec = targetCodec,
- offsetCounter = offsetCounter,
- wrapperMessageTimestamp = largestTimestampOfMessageSet,
- timestampType = messageTimestampType,
- messages = validatedMessages: _*),
- maxTimestamp = largestTimestampOfMessageSet.get,
- offsetOfMaxTimestamp = offsetOfMaxTimestampInMessageSet,
- messageSizeMaybeChanged = true)
- } else {
- // Do not do re-compression but simply update the offset, timestamp and attributes field of the wrapper message.
- buffer.putLong(0, offsetCounter.addAndGet(validatedMessages.size) - 1)
- // validate the messages
- validatedMessages.foreach(_.ensureValid())
-
- var crcUpdateNeeded = true
- val timestampOffset = MessageSet.LogOverhead + Message.TimestampOffset
- val attributeOffset = MessageSet.LogOverhead + Message.AttributesOffset
- val timestamp = buffer.getLong(timestampOffset)
- val attributes = buffer.get(attributeOffset)
- buffer.putLong(timestampOffset, maxTimestamp)
- if (messageTimestampType == TimestampType.CREATE_TIME && timestamp == maxTimestamp)
- // We don't need to recompute crc if the timestamp is not updated.
- crcUpdateNeeded = false
- else if (messageTimestampType == TimestampType.LOG_APPEND_TIME) {
- // Set timestamp type and timestamp
- buffer.putLong(timestampOffset, now)
- buffer.put(attributeOffset, messageTimestampType.updateAttributes(attributes))
- }
-
- if (crcUpdateNeeded) {
- // need to recompute the crc value
- buffer.position(MessageSet.LogOverhead)
- val wrapperMessage = new Message(buffer.slice())
- Utils.writeUnsignedInt(buffer, MessageSet.LogOverhead + Message.CrcOffset, wrapperMessage.computeChecksum)
- }
- buffer.rewind()
- // For compressed messages,
- ValidationAndOffsetAssignResult(validatedMessages = this,
- maxTimestamp = buffer.getLong(timestampOffset),
- offsetOfMaxTimestamp = buffer.getLong(0),
- messageSizeMaybeChanged = false)
- }
- }
- }
-
- // We create this method to avoid a memory copy. It reads from the original message set and directly
- // writes the converted messages into new message set buffer. Hence we don't need to allocate memory for each
- // individual message during message format conversion.
- private def convertNonCompressedMessages(offsetCounter: LongRef,
- compactedTopic: Boolean,
- now: Long,
- timestampType: TimestampType,
- messageTimestampDiffMaxMs: Long,
- toMagicValue: Byte): ValidationAndOffsetAssignResult = {
- val sizeInBytesAfterConversion = shallowValidBytes + this.internalIterator(isShallow = true).map { messageAndOffset =>
- Message.headerSizeDiff(messageAndOffset.message.magic, toMagicValue)
- }.sum
- val newBuffer = ByteBuffer.allocate(sizeInBytesAfterConversion)
- var newMessagePosition = 0
- var maxTimestamp = Message.NoTimestamp
- var offsetOfMaxTimestamp = -1L
- this.internalIterator(isShallow = true).foreach { case MessageAndOffset(message, _) =>
- validateMessageKey(message, compactedTopic)
- validateTimestamp(message, now, timestampType, messageTimestampDiffMaxMs)
- newBuffer.position(newMessagePosition)
- newBuffer.putLong(offsetCounter.getAndIncrement())
- val newMessageSize = message.size + Message.headerSizeDiff(message.magic, toMagicValue)
- newBuffer.putInt(newMessageSize)
- val newMessageBuffer = newBuffer.slice()
- newMessageBuffer.limit(newMessageSize)
- message.convertToBuffer(toMagicValue, newMessageBuffer, now, timestampType)
- if (toMagicValue > Message.MagicValue_V0) {
- val timestamp = newMessageBuffer.getLong(Message.TimestampOffset)
- if (maxTimestamp < timestamp) {
- maxTimestamp = timestamp
- offsetOfMaxTimestamp = offsetCounter.value - 1
- }
- }
- newMessagePosition += MessageSet.LogOverhead + newMessageSize
- }
- newBuffer.rewind()
- new ValidationAndOffsetAssignResult(validatedMessages = new ByteBufferMessageSet(newBuffer),
- maxTimestamp = maxTimestamp,
- offsetOfMaxTimestamp = offsetOfMaxTimestamp,
- messageSizeMaybeChanged = true)
- }
-
- private def validateNonCompressedMessagesAndAssignOffsetInPlace(offsetCounter: LongRef,
- now: Long,
- compactedTopic: Boolean,
- timestampType: TimestampType,
- timestampDiffMaxMs: Long): ValidationAndOffsetAssignResult = {
- // do in-place validation and offset assignment
- var messagePosition = 0
- var maxTimestamp = Message.NoTimestamp
- var offsetOfMaxTimestamp = -1L
- buffer.mark()
- while (messagePosition < sizeInBytes - MessageSet.LogOverhead) {
- buffer.position(messagePosition)
- buffer.putLong(offsetCounter.getAndIncrement())
- val messageSize = buffer.getInt()
- val messageBuffer = buffer.slice()
- messageBuffer.limit(messageSize)
- val message = new Message(messageBuffer)
- validateMessageKey(message, compactedTopic)
- if (message.magic > Message.MagicValue_V0) {
- validateTimestamp(message, now, timestampType, timestampDiffMaxMs)
- if (timestampType == TimestampType.LOG_APPEND_TIME) {
- message.buffer.putLong(Message.TimestampOffset, now)
- message.buffer.put(Message.AttributesOffset, timestampType.updateAttributes(message.attributes))
- Utils.writeUnsignedInt(message.buffer, Message.CrcOffset, message.computeChecksum)
- }
- if (message.timestamp > maxTimestamp) {
- maxTimestamp = message.timestamp
- offsetOfMaxTimestamp = offsetCounter.value - 1
- }
- }
-
- messagePosition += MessageSet.LogOverhead + messageSize
- }
- buffer.reset()
- ValidationAndOffsetAssignResult(validatedMessages = this,
- maxTimestamp = maxTimestamp,
- offsetOfMaxTimestamp = offsetOfMaxTimestamp,
- messageSizeMaybeChanged = false)
- }
-
- private def validateMessageKey(message: Message, compactedTopic: Boolean) {
- if (compactedTopic && !message.hasKey)
- throw new InvalidMessageException("Compacted topic cannot accept message without key.")
- }
-
- /**
- * This method validates the timestamps of a message.
- * If the message is using create time, this method checks if it is within acceptable range.
- */
- private def validateTimestamp(message: Message,
- now: Long,
- timestampType: TimestampType,
- timestampDiffMaxMs: Long) {
- if (timestampType == TimestampType.CREATE_TIME && math.abs(message.timestamp - now) > timestampDiffMaxMs)
- throw new InvalidTimestampException(s"Timestamp ${message.timestamp} of message is out of range. " +
- s"The timestamp should be within [${now - timestampDiffMaxMs}, ${now + timestampDiffMaxMs}")
- if (message.timestampType == TimestampType.LOG_APPEND_TIME)
- throw new InvalidTimestampException(s"Invalid timestamp type in message $message. Producer should not set " +
- s"timestamp type to LogAppendTime.")
+ private def internalIterator(isShallow: Boolean = false): Iterator[MessageAndOffset] = {
+ val entries = if (isShallow)
+ asRecords.shallowIterator
+ else
+ asRecords.deepIterator
+ entries.asScala.map(MessageAndOffset.fromLogEntry)
}
/**
@@ -757,7 +181,7 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
/**
* The total number of bytes in this message set not including any partial, trailing messages
*/
- def validBytes: Int = shallowValidBytes
+ def validBytes: Int = asRecords.validBytes
/**
* Two message sets are equal if their respective byte buffers are equal
@@ -773,8 +197,3 @@ class ByteBufferMessageSet(val buffer: ByteBuffer) extends MessageSet with Loggi
override def hashCode: Int = buffer.hashCode
}
-
-case class ValidationAndOffsetAssignResult(validatedMessages: ByteBufferMessageSet,
- maxTimestamp: Long,
- offsetOfMaxTimestamp: Long,
- messageSizeMaybeChanged: Boolean)
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/message/Message.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/message/Message.scala b/core/src/main/scala/kafka/message/Message.scala
index bb91078..175b7e9 100755
--- a/core/src/main/scala/kafka/message/Message.scala
+++ b/core/src/main/scala/kafka/message/Message.scala
@@ -19,7 +19,7 @@ package kafka.message
import java.nio._
-import org.apache.kafka.common.record.TimestampType
+import org.apache.kafka.common.record.{Record, TimestampType}
import scala.math._
import kafka.utils._
@@ -98,6 +98,11 @@ object Message {
MessageHeaderSizeMap(toMagicValue) - MessageHeaderSizeMap(fromMagicValue)
+ def fromRecord(record: Record): Message = {
+ val wrapperTimestamp: Option[Long] = if (record.wrapperRecordTimestamp() == null) None else Some(record.wrapperRecordTimestamp())
+ val wrapperTimestampType = Option(record.wrapperRecordTimestampType())
+ new Message(record.buffer, wrapperTimestamp, wrapperTimestampType)
+ }
}
/**
@@ -134,6 +139,15 @@ class Message(val buffer: ByteBuffer,
import kafka.message.Message._
+ private[message] def asRecord: Record = {
+ wrapperMessageTimestamp match {
+ case None => new Record(buffer)
+ case Some(timestamp) =>
+ val timestampType = wrapperMessageTimestampType.orNull
+ new Record(buffer, timestamp, timestampType)
+ }
+ }
+
/**
* A constructor to create a Message
* @param bytes The payload of the message
@@ -328,52 +342,6 @@ class Message(val buffer: ByteBuffer,
def key: ByteBuffer = sliceDelimited(keySizeOffset)
/**
- * convert the message to specified format
- */
- def toFormatVersion(toMagicValue: Byte): Message = {
- if (magic == toMagicValue)
- this
- else {
- val byteBuffer = ByteBuffer.allocate(size + Message.headerSizeDiff(magic, toMagicValue))
- // Copy bytes from old messages to new message
- convertToBuffer(toMagicValue, byteBuffer, Message.NoTimestamp)
- new Message(byteBuffer)
- }
- }
-
- def convertToBuffer(toMagicValue: Byte,
- byteBuffer: ByteBuffer,
- now: Long,
- timestampType: TimestampType = wrapperMessageTimestampType.getOrElse(TimestampType.forAttributes(attributes))) {
- if (byteBuffer.remaining() < size + headerSizeDiff(magic, toMagicValue))
- throw new IndexOutOfBoundsException("The byte buffer does not have enough capacity to hold new message format " +
- s"version $toMagicValue")
- if (toMagicValue == Message.MagicValue_V1) {
- // Up-conversion, reserve CRC and update magic byte
- byteBuffer.position(Message.MagicOffset)
- byteBuffer.put(Message.MagicValue_V1)
- byteBuffer.put(timestampType.updateAttributes(attributes))
- // Up-conversion, insert the timestamp field
- if (timestampType == TimestampType.LOG_APPEND_TIME)
- byteBuffer.putLong(now)
- else
- byteBuffer.putLong(Message.NoTimestamp)
- byteBuffer.put(buffer.array(), buffer.arrayOffset() + Message.KeySizeOffset_V0, size - Message.KeySizeOffset_V0)
- } else {
- // Down-conversion, reserve CRC and update magic byte
- byteBuffer.position(Message.MagicOffset)
- byteBuffer.put(Message.MagicValue_V0)
- byteBuffer.put(TimestampType.CREATE_TIME.updateAttributes(attributes))
- // Down-conversion, skip the timestamp field
- byteBuffer.put(buffer.array(), buffer.arrayOffset() + Message.KeySizeOffset_V1, size - Message.KeySizeOffset_V1)
- }
- // update crc value
- val newMessage = new Message(byteBuffer)
- Utils.writeUnsignedInt(byteBuffer, Message.CrcOffset, newMessage.computeChecksum)
- byteBuffer.rewind()
- }
-
- /**
* Read a size-delimited byte buffer starting at the given offset
*/
private def sliceDelimited(start: Int): ByteBuffer = {
@@ -399,7 +367,7 @@ class Message(val buffer: ByteBuffer,
if (timestamp < 0 && timestamp != NoTimestamp)
throw new IllegalArgumentException(s"Invalid message timestamp $timestamp")
if (magic == MagicValue_V0 && timestamp != NoTimestamp)
- throw new IllegalArgumentException(s"Invalid timestamp $timestamp. Timestamp must be ${NoTimestamp} when magic = ${MagicValue_V0}")
+ throw new IllegalArgumentException(s"Invalid timestamp $timestamp. Timestamp must be $NoTimestamp when magic = $MagicValue_V0")
}
override def toString: String = {
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/message/MessageAndOffset.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/message/MessageAndOffset.scala b/core/src/main/scala/kafka/message/MessageAndOffset.scala
index fab6898..46630c6 100644
--- a/core/src/main/scala/kafka/message/MessageAndOffset.scala
+++ b/core/src/main/scala/kafka/message/MessageAndOffset.scala
@@ -17,6 +17,13 @@
package kafka.message
+import org.apache.kafka.common.record.LogEntry
+
+object MessageAndOffset {
+ def fromLogEntry(logEntry : LogEntry): MessageAndOffset = {
+ MessageAndOffset(Message.fromRecord(logEntry.record), logEntry.offset)
+ }
+}
case class MessageAndOffset(message: Message, offset: Long) {
@@ -28,9 +35,10 @@ case class MessageAndOffset(message: Message, offset: Long) {
/**
* We need to decompress the message, if required, to get the offset of the first uncompressed message.
*/
- def firstOffset: Long = message.compressionCodec match {
- case NoCompressionCodec => offset
- case _ => ByteBufferMessageSet.deepIterator(this).next().offset
+ def firstOffset: Long = toLogEntry.firstOffset
+
+ def toLogEntry: LogEntry = {
+ LogEntry.create(offset, message.asRecord)
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/message/MessageSet.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/message/MessageSet.scala b/core/src/main/scala/kafka/message/MessageSet.scala
index ffa27fa..a44a362 100644
--- a/core/src/main/scala/kafka/message/MessageSet.scala
+++ b/core/src/main/scala/kafka/message/MessageSet.scala
@@ -18,7 +18,6 @@
package kafka.message
import java.nio._
-import java.nio.channels._
import org.apache.kafka.common.record.Records
@@ -73,11 +72,6 @@ case class MagicAndTimestamp(magic: Byte, timestamp: Long)
abstract class MessageSet extends Iterable[MessageAndOffset] {
/**
- * Check if all the wrapper messages in the message set have the expected magic value
- */
- def isMagicValueInAllWrapperMessages(expectedMagicValue: Byte): Boolean
-
- /**
* Provides an iterator over the message/offset pairs in this set
*/
def iterator: Iterator[MessageAndOffset]
@@ -99,7 +93,7 @@ abstract class MessageSet extends Iterable[MessageAndOffset] {
override def toString: String = {
val builder = new StringBuilder()
builder.append(getClass.getSimpleName + "(")
- val iter = this.iterator
+ val iter = this.asRecords.shallowIterator()
var i = 0
while(iter.hasNext && i < 100) {
val message = iter.next
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
index b3a8751..f959ce2 100755
--- a/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
+++ b/core/src/main/scala/kafka/server/AbstractFetcherThread.scala
@@ -21,7 +21,6 @@ import java.util.concurrent.locks.ReentrantLock
import kafka.cluster.BrokerEndPoint
import kafka.consumer.PartitionTopicInfo
-import kafka.message.ByteBufferMessageSet
import kafka.utils.{DelayedItem, Pool, ShutdownableThread}
import kafka.common.{ClientIdAndBroker, KafkaException}
import kafka.metrics.KafkaMetricsGroup
@@ -38,6 +37,7 @@ import java.util.concurrent.atomic.AtomicLong
import com.yammer.metrics.core.Gauge
import org.apache.kafka.common.TopicPartition
import org.apache.kafka.common.internals.PartitionStates
+import org.apache.kafka.common.record.MemoryRecords
/**
* Abstract class for fetching data from multiple partitions from the same broker.
@@ -144,15 +144,15 @@ abstract class AbstractFetcherThread(name: String,
Errors.forCode(partitionData.errorCode) match {
case Errors.NONE =>
try {
- val messages = partitionData.toByteBufferMessageSet
- val newOffset = messages.shallowIterator.toSeq.lastOption.map(_.nextOffset).getOrElse(
+ val records = partitionData.toRecords
+ val newOffset = records.shallowIterator.asScala.toSeq.lastOption.map(_.nextOffset).getOrElse(
currentPartitionFetchState.offset)
fetcherLagStats.getAndMaybePut(topic, partitionId).lag = Math.max(0L, partitionData.highWatermark - newOffset)
// Once we hand off the partition data to the subclass, we can't mess with it any more in this thread
processPartitionData(topicPartition, currentPartitionFetchState.offset, partitionData)
- val validBytes = messages.validBytes
+ val validBytes = records.validBytes
if (validBytes > 0) {
// Update partitionStates only if there is no exception during processPartitionData
partitionStates.updateAndMoveToEnd(topicPartition, new PartitionFetchState(newOffset))
@@ -260,7 +260,7 @@ object AbstractFetcherThread {
trait PartitionData {
def errorCode: Short
def exception: Option[Throwable]
- def toByteBufferMessageSet: ByteBufferMessageSet
+ def toRecords: MemoryRecords
def highWatermark: Long
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/server/DelayedFetch.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/DelayedFetch.scala b/core/src/main/scala/kafka/server/DelayedFetch.scala
index 001051f..743c994 100644
--- a/core/src/main/scala/kafka/server/DelayedFetch.scala
+++ b/core/src/main/scala/kafka/server/DelayedFetch.scala
@@ -19,7 +19,6 @@ package kafka.server
import java.util.concurrent.TimeUnit
-import kafka.api.FetchResponsePartitionData
import kafka.common.TopicAndPartition
import kafka.metrics.KafkaMetricsGroup
import org.apache.kafka.common.TopicPartition
@@ -59,7 +58,7 @@ class DelayedFetch(delayMs: Long,
fetchMetadata: FetchMetadata,
replicaManager: ReplicaManager,
quota: ReplicaQuota,
- responseCallback: Seq[(TopicAndPartition, FetchResponsePartitionData)] => Unit)
+ responseCallback: Seq[(TopicAndPartition, FetchPartitionData)] => Unit)
extends DelayedOperation(delayMs) {
/**
@@ -152,7 +151,7 @@ class DelayedFetch(delayMs: Long,
)
val fetchPartitionData = logReadResults.map { case (tp, result) =>
- tp -> FetchResponsePartitionData(result.errorCode, result.hw, result.info.messageSet)
+ tp -> FetchPartitionData(result.errorCode, result.hw, result.info.records)
}
responseCallback(fetchPartitionData)
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/server/FetchDataInfo.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/FetchDataInfo.scala b/core/src/main/scala/kafka/server/FetchDataInfo.scala
index 9d6d437..acfb5b0 100644
--- a/core/src/main/scala/kafka/server/FetchDataInfo.scala
+++ b/core/src/main/scala/kafka/server/FetchDataInfo.scala
@@ -17,7 +17,8 @@
package kafka.server
-import kafka.message.MessageSet
+import org.apache.kafka.common.record.Records
-case class FetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata, messageSet: MessageSet,
- firstMessageSetIncomplete: Boolean = false)
+case class FetchDataInfo(fetchOffsetMetadata: LogOffsetMetadata,
+ records: Records,
+ firstEntryIncomplete: Boolean = false)
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/server/KafkaApis.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala
index 447e4e2..a4a1e2a 100644
--- a/core/src/main/scala/kafka/server/KafkaApis.scala
+++ b/core/src/main/scala/kafka/server/KafkaApis.scala
@@ -23,14 +23,13 @@ import java.util.{Collections, Properties}
import java.util
import kafka.admin.{AdminUtils, RackAwareMode}
-import kafka.api.{ControlledShutdownRequest, ControlledShutdownResponse, FetchResponsePartitionData}
+import kafka.api.{ControlledShutdownRequest, ControlledShutdownResponse}
import kafka.cluster.Partition
import kafka.server.QuotaFactory.{QuotaManagers, UnboundedQuota}
import kafka.common._
import kafka.controller.KafkaController
import kafka.coordinator.{GroupCoordinator, JoinGroupResult}
import kafka.log._
-import kafka.message.{ByteBufferMessageSet, Message}
import kafka.network._
import kafka.network.RequestChannel.{Response, Session}
import kafka.security.auth
@@ -39,7 +38,7 @@ import kafka.utils.{Logging, ZKGroupTopicDirs, ZkUtils}
import org.apache.kafka.common.errors.{ClusterAuthorizationException, NotLeaderForPartitionException, TopicExistsException, UnknownTopicOrPartitionException, UnsupportedForMessageFormatException}
import org.apache.kafka.common.metrics.Metrics
import org.apache.kafka.common.protocol.{ApiKeys, Errors, Protocol, SecurityProtocol}
-import org.apache.kafka.common.record.MemoryRecords
+import org.apache.kafka.common.record.{MemoryRecords, Record}
import org.apache.kafka.common.requests._
import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
import org.apache.kafka.common.utils.{Time, Utils}
@@ -359,9 +358,9 @@ class KafkaApis(val requestChannel: RequestChannel,
val mergedResponseStatus = responseStatus ++
unauthorizedForWriteRequestInfo.mapValues(_ =>
- new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED.code, -1, Message.NoTimestamp)) ++
+ new PartitionResponse(Errors.TOPIC_AUTHORIZATION_FAILED.code, -1, Record.NO_TIMESTAMP)) ++
nonExistingOrUnauthorizedForDescribeTopics.mapValues(_ =>
- new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION.code, -1, Message.NoTimestamp))
+ new PartitionResponse(Errors.UNKNOWN_TOPIC_OR_PARTITION.code, -1, Record.NO_TIMESTAMP))
var errorInResponse = false
@@ -422,17 +421,12 @@ class KafkaApis(val requestChannel: RequestChannel,
else {
val internalTopicsAllowed = request.header.clientId == AdminUtils.AdminClientId
- // Convert Records to ByteBufferMessageSet
- val authorizedMessagesPerPartition = authorizedRequestInfo.map {
- case (topicPartition, records) => (topicPartition, new ByteBufferMessageSet(records.buffer))
- }
-
// call the replica manager to append messages to the replicas
- replicaManager.appendMessages(
+ replicaManager.appendRecords(
produceRequest.timeout.toLong,
produceRequest.acks,
internalTopicsAllowed,
- authorizedMessagesPerPartition,
+ authorizedRequestInfo,
sendResponseCallback)
// if the request is put into the purgatory, it will have a held reference
@@ -467,7 +461,7 @@ class KafkaApis(val requestChannel: RequestChannel,
}
// the callback for sending a fetch response
- def sendResponseCallback(responsePartitionData: Seq[(TopicAndPartition, FetchResponsePartitionData)]) {
+ def sendResponseCallback(responsePartitionData: Seq[(TopicAndPartition, FetchPartitionData)]) {
val convertedPartitionData = {
// Need to down-convert message when consumer only takes magic value 0.
responsePartitionData.map { case (tp, data) =>
@@ -480,18 +474,18 @@ class KafkaApis(val requestChannel: RequestChannel,
// Please note that if the message format is changed from a higher version back to lower version this
// test might break because some messages in new message format can be delivered to consumers before 0.10.0.0
// without format down conversion.
- val convertedData = if (versionId <= 1 && replicaManager.getMessageFormatVersion(tp).exists(_ > Message.MagicValue_V0) &&
- !data.messages.isMagicValueInAllWrapperMessages(Message.MagicValue_V0)) {
+ val convertedData = if (versionId <= 1 && replicaManager.getMagicAndTimestampType(tp).exists(_._1 > Record.MAGIC_VALUE_V0) &&
+ !data.records.hasMatchingShallowMagic(Record.MAGIC_VALUE_V0)) {
trace(s"Down converting message to V0 for fetch request from $clientId")
- new FetchResponsePartitionData(data.error, data.hw, data.messages.asInstanceOf[FileMessageSet].toMessageFormat(Message.MagicValue_V0))
+ FetchPartitionData(data.error, data.hw, data.records.toMessageFormat(Record.MAGIC_VALUE_V0))
} else data
- val records = convertedData.messages.asRecords
- new TopicPartition(tp.topic, tp.partition) -> new FetchResponse.PartitionData(convertedData.error, convertedData.hw, records)
+ new TopicPartition(tp.topic, tp.partition) -> new FetchResponse.PartitionData(convertedData.error, convertedData.hw, convertedData.records)
}
}
val mergedPartitionData = convertedPartitionData ++ unauthorizedForReadPartitionData ++ nonExistingOrUnauthorizedForDescribePartitionData
+
val fetchedPartitionData = new util.LinkedHashMap[TopicPartition, FetchResponse.PartitionData]()
mergedPartitionData.foreach { case (topicPartition, data) =>
@@ -660,7 +654,7 @@ class KafkaApis(val requestChannel: RequestChannel,
val found = {
if (fromConsumer && timestamp == ListOffsetRequest.LATEST_TIMESTAMP)
- TimestampOffset(Message.NoTimestamp, localReplica.highWatermark.messageOffset)
+ TimestampOffset(Record.NO_TIMESTAMP, localReplica.highWatermark.messageOffset)
else {
def allowed(timestampOffset: TimestampOffset): Boolean =
!fromConsumer || timestampOffset.offset <= localReplica.highWatermark.messageOffset
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
----------------------------------------------------------------------
diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
index 84c3feb..54a2e05 100644
--- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
+++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala
@@ -23,7 +23,6 @@ import java.util
import kafka.admin.AdminUtils
import kafka.cluster.BrokerEndPoint
import kafka.log.LogConfig
-import kafka.message.ByteBufferMessageSet
import kafka.api.{KAFKA_0_10_0_IV0, KAFKA_0_10_1_IV1, KAFKA_0_10_1_IV2, KAFKA_0_9_0}
import kafka.common.{KafkaStorageException, TopicAndPartition}
import ReplicaFetcherThread._
@@ -119,19 +118,19 @@ class ReplicaFetcherThread(name: String,
val topic = topicPartition.topic
val partitionId = topicPartition.partition
val replica = replicaMgr.getReplica(topic, partitionId).get
- val messageSet = partitionData.toByteBufferMessageSet
+ val records = partitionData.toRecords
- maybeWarnIfMessageOversized(messageSet, topicPartition)
+ maybeWarnIfOversizedRecords(records, topicPartition)
if (fetchOffset != replica.logEndOffset.messageOffset)
throw new RuntimeException("Offset mismatch for partition %s: fetched offset = %d, log end offset = %d.".format(topicPartition, fetchOffset, replica.logEndOffset.messageOffset))
if (logger.isTraceEnabled)
trace("Follower %d has replica log end offset %d for partition %s. Received %d messages and leader hw %d"
- .format(replica.brokerId, replica.logEndOffset.messageOffset, topicPartition, messageSet.sizeInBytes, partitionData.highWatermark))
- replica.log.get.append(messageSet, assignOffsets = false)
+ .format(replica.brokerId, replica.logEndOffset.messageOffset, topicPartition, records.sizeInBytes, partitionData.highWatermark))
+ replica.log.get.append(records, assignOffsets = false)
if (logger.isTraceEnabled)
trace("Follower %d has replica log end offset %d after appending %d bytes of messages for partition %s"
- .format(replica.brokerId, replica.logEndOffset.messageOffset, messageSet.sizeInBytes, topicPartition))
+ .format(replica.brokerId, replica.logEndOffset.messageOffset, records.sizeInBytes, topicPartition))
val followerHighWatermark = replica.logEndOffset.messageOffset.min(partitionData.highWatermark)
// for the follower replica, we do not need to keep
// its segment base offset the physical position,
@@ -141,7 +140,7 @@ class ReplicaFetcherThread(name: String,
trace("Follower %d set replica high watermark for partition [%s,%d] to %s"
.format(replica.brokerId, topic, partitionId, followerHighWatermark))
if (quota.isThrottled(TopicAndPartition(topic, partitionId)))
- quota.record(messageSet.sizeInBytes)
+ quota.record(records.sizeInBytes)
} catch {
case e: KafkaStorageException =>
fatal(s"Disk error while replicating data for $topicPartition", e)
@@ -149,9 +148,9 @@ class ReplicaFetcherThread(name: String,
}
}
- def maybeWarnIfMessageOversized(messageSet: ByteBufferMessageSet, topicPartition: TopicPartition): Unit = {
+ def maybeWarnIfOversizedRecords(records: MemoryRecords, topicPartition: TopicPartition): Unit = {
// oversized messages don't cause replication to fail from fetch request version 3 (KIP-74)
- if (fetchRequestVersion <= 2 && messageSet.sizeInBytes > 0 && messageSet.validBytes <= 0)
+ if (fetchRequestVersion <= 2 && records.sizeInBytes > 0 && records.validBytes <= 0)
error(s"Replication is failing due to a message that is greater than replica.fetch.max.bytes for partition $topicPartition. " +
"This generally occurs when the max.message.bytes has been overridden to exceed this value and a suitably large " +
"message has also been sent. To fix this problem increase replica.fetch.max.bytes in your broker config to be " +
@@ -323,9 +322,8 @@ object ReplicaFetcherThread {
def errorCode: Short = underlying.errorCode
- def toByteBufferMessageSet: ByteBufferMessageSet = {
- val buffer = underlying.records.asInstanceOf[MemoryRecords].buffer
- new ByteBufferMessageSet(buffer)
+ def toRecords: MemoryRecords = {
+ underlying.records.asInstanceOf[MemoryRecords]
}
def highWatermark: Long = underlying.highWatermark
[9/9] kafka git commit: KAFKA-4390;
Replace MessageSet usage with client-side alternatives
Posted by jg...@apache.org.
KAFKA-4390; Replace MessageSet usage with client-side alternatives
Author: Jason Gustafson <ja...@confluent.io>
Reviewers: Ismael Juma <is...@juma.me.uk>, Guozhang Wang <wa...@gmail.com>, Jun Rao <ju...@gmail.com>
Closes #2140 from hachikuji/KAFKA4390
Project: http://git-wip-us.apache.org/repos/asf/kafka/repo
Commit: http://git-wip-us.apache.org/repos/asf/kafka/commit/67f1e5b9
Tree: http://git-wip-us.apache.org/repos/asf/kafka/tree/67f1e5b9
Diff: http://git-wip-us.apache.org/repos/asf/kafka/diff/67f1e5b9
Branch: refs/heads/trunk
Commit: 67f1e5b91bf073151ff57d5d656693e385726697
Parents: 6626b05
Author: Jason Gustafson <ja...@confluent.io>
Authored: Tue Dec 13 10:26:25 2016 -0800
Committer: Jason Gustafson <ja...@confluent.io>
Committed: Tue Dec 13 10:26:25 2016 -0800
----------------------------------------------------------------------
checkstyle/import-control.xml | 1 +
.../clients/consumer/internals/Fetcher.java | 7 +-
.../clients/producer/internals/BufferPool.java | 2 +-
.../producer/internals/RecordAccumulator.java | 30 +-
.../clients/producer/internals/RecordBatch.java | 58 +-
.../clients/producer/internals/Sender.java | 10 +-
.../kafka/common/record/AbstractRecords.java | 92 +++
.../common/record/ByteBufferInputStream.java | 37 +-
.../common/record/ByteBufferLogInputStream.java | 119 ++++
.../common/record/ByteBufferOutputStream.java | 54 +-
.../kafka/common/record/CompressionType.java | 2 +-
.../apache/kafka/common/record/Compressor.java | 332 ----------
.../kafka/common/record/FileLogInputStream.java | 166 +++++
.../apache/kafka/common/record/FileRecords.java | 465 ++++++++++++--
.../common/record/InvalidRecordException.java | 4 +-
.../apache/kafka/common/record/LogEntry.java | 151 ++++-
.../kafka/common/record/LogInputStream.java | 12 +-
.../kafka/common/record/MemoryRecords.java | 443 +++++++++-----
.../common/record/MemoryRecordsBuilder.java | 461 ++++++++++++++
.../org/apache/kafka/common/record/Record.java | 538 +++++++++++-----
.../org/apache/kafka/common/record/Records.java | 64 +-
.../kafka/common/record/RecordsIterator.java | 179 ++++--
.../kafka/common/record/TimestampType.java | 1 +
.../org/apache/kafka/common/utils/Utils.java | 51 ++
.../clients/consumer/KafkaConsumerTest.java | 7 +-
.../clients/consumer/internals/FetcherTest.java | 100 +--
.../internals/RecordAccumulatorTest.java | 51 +-
.../record/ByteBufferLogInputStreamTest.java | 110 ++++
.../kafka/common/record/FileRecordsTest.java | 410 +++++++++++++
.../common/record/MemoryRecordsBuilderTest.java | 253 ++++++++
.../kafka/common/record/MemoryRecordsTest.java | 194 +++++-
.../apache/kafka/common/record/RecordTest.java | 46 +-
.../kafka/common/record/SimpleRecordTest.java | 52 +-
.../kafka/common/record/TimestampTypeTest.java | 37 ++
.../java/org/apache/kafka/test/TestUtils.java | 32 +-
core/src/main/scala/kafka/api/ApiVersion.scala | 20 +-
.../main/scala/kafka/cluster/Partition.scala | 10 +-
.../kafka/consumer/ConsumerFetcherThread.scala | 10 +-
.../kafka/coordinator/GroupCoordinator.scala | 6 +-
.../coordinator/GroupMetadataManager.scala | 170 +++--
.../main/scala/kafka/log/FileMessageSet.scala | 445 --------------
core/src/main/scala/kafka/log/Log.scala | 128 ++--
core/src/main/scala/kafka/log/LogCleaner.scala | 55 +-
core/src/main/scala/kafka/log/LogManager.scala | 2 +-
core/src/main/scala/kafka/log/LogSegment.scala | 98 +--
.../src/main/scala/kafka/log/LogValidator.scala | 239 ++++++++
core/src/main/scala/kafka/log/TimeIndex.scala | 6 +-
.../kafka/message/ByteBufferMessageSet.scala | 613 +------------------
core/src/main/scala/kafka/message/Message.scala | 64 +-
.../scala/kafka/message/MessageAndOffset.scala | 14 +-
.../main/scala/kafka/message/MessageSet.scala | 8 +-
.../kafka/server/AbstractFetcherThread.scala | 10 +-
.../main/scala/kafka/server/DelayedFetch.scala | 5 +-
.../main/scala/kafka/server/FetchDataInfo.scala | 7 +-
.../src/main/scala/kafka/server/KafkaApis.scala | 32 +-
.../kafka/server/ReplicaFetcherThread.scala | 22 +-
.../scala/kafka/server/ReplicaManager.scala | 85 +--
.../scala/kafka/tools/DumpLogSegments.scala | 150 ++---
.../kafka/tools/ReplicaVerificationTool.scala | 52 +-
.../api/GroupCoordinatorIntegrationTest.scala | 6 +-
.../scala/kafka/tools/TestLogCleaning.scala | 19 +-
.../test/scala/other/kafka/StressTestLog.scala | 22 +-
.../other/kafka/TestLinearWriteSpeed.scala | 19 +-
.../unit/kafka/admin/DeleteTopicTest.scala | 2 +-
.../GroupCoordinatorResponseTest.scala | 30 +-
.../coordinator/GroupMetadataManagerTest.scala | 100 ++-
.../unit/kafka/log/BrokerCompressionTest.scala | 15 +-
.../unit/kafka/log/FileMessageSetTest.scala | 354 -----------
.../kafka/log/LogCleanerIntegrationTest.scala | 66 +-
.../log/LogCleanerLagIntegrationTest.scala | 24 +-
.../unit/kafka/log/LogCleanerManagerTest.scala | 29 +-
.../scala/unit/kafka/log/LogCleanerTest.scala | 275 ++++-----
.../scala/unit/kafka/log/LogManagerTest.scala | 22 +-
.../scala/unit/kafka/log/LogSegmentTest.scala | 136 ++--
.../src/test/scala/unit/kafka/log/LogTest.scala | 204 +++---
.../scala/unit/kafka/log/LogValidatorTest.scala | 395 ++++++++++++
.../kafka/message/BaseMessageSetTestCases.scala | 20 +-
.../message/ByteBufferMessageSetTest.scala | 348 -----------
.../kafka/message/MessageCompressionTest.scala | 2 +-
.../scala/unit/kafka/message/MessageTest.scala | 38 +-
.../server/AbstractFetcherThreadTest.scala | 32 +-
.../unit/kafka/server/FetchRequestTest.scala | 53 +-
.../unit/kafka/server/ISRExpirationTest.scala | 27 +-
.../scala/unit/kafka/server/LogOffsetTest.scala | 14 +-
.../unit/kafka/server/ProduceRequestTest.scala | 8 +-
.../kafka/server/ReplicaManagerQuotasTest.scala | 33 +-
.../unit/kafka/server/ReplicaManagerTest.scala | 28 +-
.../unit/kafka/server/SimpleFetchTest.scala | 21 +-
.../test/scala/unit/kafka/utils/TestUtils.scala | 19 +-
89 files changed, 5269 insertions(+), 3914 deletions(-)
----------------------------------------------------------------------
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/checkstyle/import-control.xml
----------------------------------------------------------------------
diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
index 8eebdb5..62cd77a 100644
--- a/checkstyle/import-control.xml
+++ b/checkstyle/import-control.xml
@@ -89,6 +89,7 @@
<allow pkg="net.jpountz" />
<allow pkg="org.apache.kafka.common.record" />
<allow pkg="org.apache.kafka.common.network" />
+ <allow pkg="org.apache.kafka.common.errors" />
</subpackage>
<subpackage name="requests">
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
index 6fb4229..3b9d49c 100644
--- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
+++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java
@@ -61,6 +61,7 @@ import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
+import java.util.Iterator;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Locale;
@@ -686,11 +687,13 @@ public class Fetcher<K, V> {
}
List<ConsumerRecord<K, V>> parsed = new ArrayList<>();
- for (LogEntry logEntry : partition.records) {
+ Iterator<LogEntry> deepIterator = partition.records.deepIterator();
+ while (deepIterator.hasNext()) {
+ LogEntry logEntry = deepIterator.next();
// Skip the messages earlier than current position.
if (logEntry.offset() >= position) {
parsed.add(parseRecord(tp, logEntry));
- bytes += logEntry.size();
+ bytes += logEntry.sizeInBytes();
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java
index b42b0ec..077215c 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java
@@ -198,7 +198,7 @@ public final class BufferPool {
* memory as free.
*
* @param buffer The buffer to return
- * @param size The size of the buffer to mark as deallocated, note that this maybe smaller than buffer.capacity
+ * @param size The size of the buffer to mark as deallocated, note that this may be smaller than buffer.capacity
* since the buffer may re-allocate itself during in-place compression
*/
public void deallocate(ByteBuffer buffer, int size) {
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
index fa1e513..06d39ec 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
@@ -27,8 +27,10 @@ import org.apache.kafka.common.metrics.Sensor;
import org.apache.kafka.common.metrics.stats.Rate;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.record.MemoryRecordsBuilder;
import org.apache.kafka.common.record.Record;
import org.apache.kafka.common.record.Records;
+import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.utils.CopyOnWriteMap;
import org.apache.kafka.common.utils.Time;
import org.apache.kafka.common.utils.Utils;
@@ -49,7 +51,7 @@ import java.util.concurrent.ConcurrentMap;
import java.util.concurrent.atomic.AtomicInteger;
/**
- * This class acts as a queue that accumulates records into {@link org.apache.kafka.common.record.MemoryRecords}
+ * This class acts as a queue that accumulates records into {@link MemoryRecords}
* instances to be sent to the server.
* <p>
* The accumulator uses a bounded amount of memory and append calls will block when that memory is exhausted, unless
@@ -77,7 +79,7 @@ public final class RecordAccumulator {
/**
* Create a new record accumulator
*
- * @param batchSize The size to use when allocating {@link org.apache.kafka.common.record.MemoryRecords} instances
+ * @param batchSize The size to use when allocating {@link MemoryRecords} instances
* @param totalSize The maximum memory the record accumulator can use.
* @param compression The compression codec for the records
* @param lingerMs An artificial delay time to add before declaring a records instance that isn't full ready for
@@ -190,13 +192,13 @@ public final class RecordAccumulator {
free.deallocate(buffer);
return appendResult;
}
- MemoryRecords records = MemoryRecords.emptyRecords(buffer, compression, this.batchSize);
- RecordBatch batch = new RecordBatch(tp, records, time.milliseconds());
+ MemoryRecordsBuilder recordsBuilder = MemoryRecords.builder(buffer, compression, TimestampType.CREATE_TIME, this.batchSize);
+ RecordBatch batch = new RecordBatch(tp, recordsBuilder, time.milliseconds());
FutureRecordMetadata future = Utils.notNull(batch.tryAppend(timestamp, key, value, callback, time.milliseconds()));
dq.addLast(batch);
incomplete.add(batch);
- return new RecordAppendResult(future, dq.size() > 1 || batch.records.isFull(), true);
+ return new RecordAppendResult(future, dq.size() > 1 || batch.isFull(), true);
}
} finally {
appendsInProgress.decrementAndGet();
@@ -212,9 +214,9 @@ public final class RecordAccumulator {
if (last != null) {
FutureRecordMetadata future = last.tryAppend(timestamp, key, value, callback, time.milliseconds());
if (future == null)
- last.records.close();
+ last.close();
else
- return new RecordAppendResult(future, deque.size() > 1 || last.records.isFull(), false);
+ return new RecordAppendResult(future, deque.size() > 1 || last.isFull(), false);
}
return null;
}
@@ -240,7 +242,7 @@ public final class RecordAccumulator {
Iterator<RecordBatch> batchIterator = dq.iterator();
while (batchIterator.hasNext()) {
RecordBatch batch = batchIterator.next();
- boolean isFull = batch != lastBatch || batch.records.isFull();
+ boolean isFull = batch != lastBatch || batch.isFull();
// check if the batch is expired
if (batch.maybeExpire(requestTimeout, retryBackoffMs, now, this.lingerMs, isFull)) {
expiredBatches.add(batch);
@@ -319,7 +321,7 @@ public final class RecordAccumulator {
long waitedTimeMs = nowMs - batch.lastAttemptMs;
long timeToWaitMs = backingOff ? retryBackoffMs : lingerMs;
long timeLeftMs = Math.max(timeToWaitMs - waitedTimeMs, 0);
- boolean full = deque.size() > 1 || batch.records.isFull();
+ boolean full = deque.size() > 1 || batch.isFull();
boolean expired = waitedTimeMs >= timeToWaitMs;
boolean sendable = full || expired || exhausted || closed || flushInProgress();
if (sendable && !backingOff) {
@@ -389,15 +391,15 @@ public final class RecordAccumulator {
boolean backoff = first.attempts > 0 && first.lastAttemptMs + retryBackoffMs > now;
// Only drain the batch if it is not during backoff period.
if (!backoff) {
- if (size + first.records.sizeInBytes() > maxSize && !ready.isEmpty()) {
+ if (size + first.sizeInBytes() > maxSize && !ready.isEmpty()) {
// there is a rare case that a single batch size is larger than the request size due
// to compression; in this case we will still eventually send this batch in a single
// request
break;
} else {
RecordBatch batch = deque.pollFirst();
- batch.records.close();
- size += batch.records.sizeInBytes();
+ batch.close();
+ size += batch.sizeInBytes();
ready.add(batch);
batch.drainedMs = now;
}
@@ -437,7 +439,7 @@ public final class RecordAccumulator {
*/
public void deallocate(RecordBatch batch) {
incomplete.remove(batch);
- free.deallocate(batch.records.buffer(), batch.records.initialCapacity());
+ free.deallocate(batch.buffer(), batch.initialCapacity());
}
/**
@@ -507,7 +509,7 @@ public final class RecordAccumulator {
Deque<RecordBatch> dq = getDeque(batch.topicPartition);
// Close the batch before aborting
synchronized (dq) {
- batch.records.close();
+ batch.close();
dq.remove(batch);
}
batch.done(-1L, Record.NO_TIMESTAMP, new IllegalStateException("Producer is closed forcefully."));
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java
index 6706bfd..e9ef441 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java
@@ -12,18 +12,20 @@
*/
package org.apache.kafka.clients.producer.internals;
-import java.util.ArrayList;
-import java.util.List;
-
import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.TopicPartition;
import org.apache.kafka.common.errors.TimeoutException;
import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.record.MemoryRecordsBuilder;
import org.apache.kafka.common.record.Record;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
/**
* A batch of records that is or will be sent.
*
@@ -39,21 +41,21 @@ public final class RecordBatch {
public final long createdMs;
public long drainedMs;
public long lastAttemptMs;
- public final MemoryRecords records;
public final TopicPartition topicPartition;
public final ProduceRequestResult produceFuture;
public long lastAppendTime;
private final List<Thunk> thunks;
private long offsetCounter = 0L;
private boolean retry;
+ private final MemoryRecordsBuilder recordsBuilder;
- public RecordBatch(TopicPartition tp, MemoryRecords records, long now) {
+ public RecordBatch(TopicPartition tp, MemoryRecordsBuilder recordsBuilder, long now) {
this.createdMs = now;
this.lastAttemptMs = now;
- this.records = records;
+ this.recordsBuilder = recordsBuilder;
this.topicPartition = tp;
this.produceFuture = new ProduceRequestResult();
- this.thunks = new ArrayList<Thunk>();
+ this.thunks = new ArrayList<>();
this.lastAppendTime = createdMs;
this.retry = false;
}
@@ -64,10 +66,10 @@ public final class RecordBatch {
* @return The RecordSend corresponding to this record or null if there isn't sufficient room.
*/
public FutureRecordMetadata tryAppend(long timestamp, byte[] key, byte[] value, Callback callback, long now) {
- if (!this.records.hasRoomFor(key, value)) {
+ if (!recordsBuilder.hasRoomFor(key, value)) {
return null;
} else {
- long checksum = this.records.append(offsetCounter++, timestamp, key, value);
+ long checksum = this.recordsBuilder.append(offsetCounter++, timestamp, key, value);
this.maxRecordSize = Math.max(this.maxRecordSize, Record.recordSize(key, value));
this.lastAppendTime = now;
FutureRecordMetadata future = new FutureRecordMetadata(this.produceFuture, this.recordCount,
@@ -94,9 +96,8 @@ public final class RecordBatch {
baseOffset,
exception);
// execute callbacks
- for (int i = 0; i < this.thunks.size(); i++) {
+ for (Thunk thunk : thunks) {
try {
- Thunk thunk = this.thunks.get(i);
if (exception == null) {
// If the timestamp returned by server is NoTimestamp, that means CreateTime is used. Otherwise LogAppendTime is used.
RecordMetadata metadata = new RecordMetadata(this.topicPartition, baseOffset, thunk.future.relativeOffset(),
@@ -156,7 +157,7 @@ public final class RecordBatch {
}
if (expire) {
- this.records.close();
+ close();
this.done(-1L, Record.NO_TIMESTAMP,
new TimeoutException("Expiring " + recordCount + " record(s) for " + topicPartition + " due to " + errorMessage));
}
@@ -177,4 +178,37 @@ public final class RecordBatch {
public void setRetry() {
this.retry = true;
}
+
+ public MemoryRecords records() {
+ return recordsBuilder.build();
+ }
+
+ public int sizeInBytes() {
+ return recordsBuilder.sizeInBytes();
+ }
+
+ public double compressionRate() {
+ return recordsBuilder.compressionRate();
+ }
+
+ public boolean isFull() {
+ return recordsBuilder.isFull();
+ }
+
+ public void close() {
+ recordsBuilder.close();
+ }
+
+ public ByteBuffer buffer() {
+ return recordsBuilder.buffer();
+ }
+
+ public int initialCapacity() {
+ return recordsBuilder.initialCapacity();
+ }
+
+ public boolean isWritable() {
+ return !recordsBuilder.isClosed();
+ }
+
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
index 7555b71..1f54c0b 100644
--- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
@@ -345,7 +345,7 @@ public class Sender implements Runnable {
final Map<TopicPartition, RecordBatch> recordsByPartition = new HashMap<>(batches.size());
for (RecordBatch batch : batches) {
TopicPartition tp = batch.topicPartition;
- produceRecordsByPartition.put(tp, batch.records);
+ produceRecordsByPartition.put(tp, batch.records());
recordsByPartition.put(tp, batch);
}
@@ -505,17 +505,17 @@ public class Sender implements Runnable {
// per-topic bytes send rate
String topicByteRateName = "topic." + topic + ".bytes";
Sensor topicByteRate = Utils.notNull(this.metrics.getSensor(topicByteRateName));
- topicByteRate.record(batch.records.sizeInBytes());
+ topicByteRate.record(batch.sizeInBytes());
// per-topic compression rate
String topicCompressionRateName = "topic." + topic + ".compression-rate";
Sensor topicCompressionRate = Utils.notNull(this.metrics.getSensor(topicCompressionRateName));
- topicCompressionRate.record(batch.records.compressionRate());
+ topicCompressionRate.record(batch.compressionRate());
// global metrics
- this.batchSizeSensor.record(batch.records.sizeInBytes(), now);
+ this.batchSizeSensor.record(batch.sizeInBytes(), now);
this.queueTimeSensor.record(batch.drainedMs - batch.createdMs, now);
- this.compressionRateSensor.record(batch.records.compressionRate());
+ this.compressionRateSensor.record(batch.compressionRate());
this.maxRecordSizeSensor.record(batch.maxRecordSize, now);
records += batch.recordCount;
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java b/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java
new file mode 100644
index 0000000..3794dc6
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/record/AbstractRecords.java
@@ -0,0 +1,92 @@
+/**
+ * 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.common.record;
+
+import org.apache.kafka.common.utils.AbstractIterator;
+
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+public abstract class AbstractRecords implements Records {
+
+ @Override
+ public boolean hasMatchingShallowMagic(byte magic) {
+ Iterator<? extends LogEntry> iterator = shallowIterator();
+ while (iterator.hasNext())
+ if (iterator.next().magic() != magic)
+ return false;
+ return true;
+ }
+
+ /**
+ * Convert this message set to use the specified message format.
+ */
+ @Override
+ public Records toMessageFormat(byte toMagic) {
+ List<LogEntry> converted = new ArrayList<>();
+ Iterator<LogEntry> deepIterator = deepIterator();
+ while (deepIterator.hasNext()) {
+ LogEntry entry = deepIterator.next();
+ converted.add(LogEntry.create(entry.offset(), entry.record().convert(toMagic)));
+ }
+
+ if (converted.isEmpty()) {
+ // This indicates that the message is too large, which indicates that the buffer is not large
+ // enough to hold a full log entry. We just return all the bytes in the file message set.
+ // Even though the message set does not have the right format version, we expect old clients
+ // to raise an error to the user after reading the message size and seeing that there
+ // are not enough available bytes in the response to read the full message.
+ return this;
+ } else {
+ // We use the first message to determine the compression type for the resulting message set.
+ // This could result in message sets which are either larger or smaller than the original size.
+ // For example, it could end up larger if most messages were previously compressed, but
+ // it just so happens that the first one is not. There is also some risk that this can
+ // cause some timestamp information to be lost (e.g. if the timestamp type was changed) since
+ // we are essentially merging multiple message sets. However, currently this method is only
+ // used for down-conversion, so we've ignored the problem.
+ CompressionType compressionType = shallowIterator().next().record().compressionType();
+ return MemoryRecords.withLogEntries(compressionType, converted);
+ }
+ }
+
+ public static int estimatedSize(CompressionType compressionType, Iterable<LogEntry> entries) {
+ int size = 0;
+ for (LogEntry entry : entries)
+ size += entry.sizeInBytes();
+ // NOTE: 1024 is the minimum block size for snappy encoding
+ return compressionType == CompressionType.NONE ? size : Math.min(Math.max(size / 2, 1024), 1 << 16);
+ }
+
+ /**
+ * Get an iterator over the deep records.
+ * @return An iterator over the records
+ */
+ public Iterator<Record> records() {
+ return new AbstractIterator<Record>() {
+ private final Iterator<? extends LogEntry> deepEntries = deepIterator();
+ @Override
+ protected Record makeNext() {
+ if (deepEntries.hasNext())
+ return deepEntries.next().record();
+ return allDone();
+ }
+ };
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java
index 84668a5..b25f949 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferInputStream.java
@@ -16,34 +16,41 @@
*/
package org.apache.kafka.common.record;
+import java.io.DataInputStream;
import java.io.InputStream;
import java.nio.ByteBuffer;
/**
* A byte buffer backed input inputStream
*/
-public class ByteBufferInputStream extends InputStream {
-
- private ByteBuffer buffer;
+public class ByteBufferInputStream extends DataInputStream {
public ByteBufferInputStream(ByteBuffer buffer) {
- this.buffer = buffer;
+ super(new UnderlyingInputStream(buffer));
}
- public int read() {
- if (!buffer.hasRemaining()) {
- return -1;
+ private static class UnderlyingInputStream extends InputStream {
+ private ByteBuffer buffer;
+
+ public UnderlyingInputStream(ByteBuffer buffer) {
+ this.buffer = buffer;
}
- return buffer.get() & 0xFF;
- }
- public int read(byte[] bytes, int off, int len) {
- if (!buffer.hasRemaining()) {
- return -1;
+ public int read() {
+ if (!buffer.hasRemaining()) {
+ return -1;
+ }
+ return buffer.get() & 0xFF;
}
- len = Math.min(len, buffer.remaining());
- buffer.get(bytes, off, len);
- return len;
+ public int read(byte[] bytes, int off, int len) {
+ if (!buffer.hasRemaining()) {
+ return -1;
+ }
+
+ len = Math.min(len, buffer.remaining());
+ buffer.get(bytes, off, len);
+ return len;
+ }
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/ByteBufferLogInputStream.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferLogInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferLogInputStream.java
new file mode 100644
index 0000000..ae0c91b
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferLogInputStream.java
@@ -0,0 +1,119 @@
+/**
+ * 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.common.record;
+
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.utils.Utils;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import static org.apache.kafka.common.record.Records.LOG_OVERHEAD;
+import static org.apache.kafka.common.record.Records.OFFSET_OFFSET;
+
+/**
+ * A byte buffer backed log input stream. This class avoids the need to copy records by returning
+ * slices from the underlying byte buffer.
+ */
+class ByteBufferLogInputStream implements LogInputStream<ByteBufferLogInputStream.ByteBufferLogEntry> {
+ private final ByteBuffer buffer;
+ private final int maxMessageSize;
+
+ ByteBufferLogInputStream(ByteBuffer buffer, int maxMessageSize) {
+ this.buffer = buffer;
+ this.maxMessageSize = maxMessageSize;
+ }
+
+ public ByteBufferLogEntry nextEntry() throws IOException {
+ int remaining = buffer.remaining();
+ if (remaining < LOG_OVERHEAD)
+ return null;
+
+ int recordSize = buffer.getInt(buffer.position() + Records.SIZE_OFFSET);
+ if (recordSize < Record.RECORD_OVERHEAD_V0)
+ throw new CorruptRecordException(String.format("Record size is less than the minimum record overhead (%d)", Record.RECORD_OVERHEAD_V0));
+ if (recordSize > maxMessageSize)
+ throw new CorruptRecordException(String.format("Record size exceeds the largest allowable message size (%d).", maxMessageSize));
+
+ int entrySize = recordSize + LOG_OVERHEAD;
+ if (remaining < entrySize)
+ return null;
+
+ ByteBuffer entrySlice = buffer.slice();
+ entrySlice.limit(entrySize);
+ buffer.position(buffer.position() + entrySize);
+ return new ByteBufferLogEntry(entrySlice);
+ }
+
+ public static class ByteBufferLogEntry extends LogEntry {
+ private final ByteBuffer buffer;
+ private final Record record;
+
+ private ByteBufferLogEntry(ByteBuffer buffer) {
+ this.buffer = buffer;
+ buffer.position(LOG_OVERHEAD);
+ this.record = new Record(buffer.slice());
+ buffer.position(OFFSET_OFFSET);
+ }
+
+ @Override
+ public long offset() {
+ return buffer.getLong(OFFSET_OFFSET);
+ }
+
+ @Override
+ public Record record() {
+ return record;
+ }
+
+ public void setOffset(long offset) {
+ buffer.putLong(OFFSET_OFFSET, offset);
+ }
+
+ public void setCreateTime(long timestamp) {
+ if (record.magic() == Record.MAGIC_VALUE_V0)
+ throw new IllegalArgumentException("Cannot set timestamp for a record with magic = 0");
+
+ long currentTimestamp = record.timestamp();
+ // We don't need to recompute crc if the timestamp is not updated.
+ if (record.timestampType() == TimestampType.CREATE_TIME && currentTimestamp == timestamp)
+ return;
+
+ byte attributes = record.attributes();
+ buffer.put(LOG_OVERHEAD + Record.ATTRIBUTES_OFFSET, TimestampType.CREATE_TIME.updateAttributes(attributes));
+ buffer.putLong(LOG_OVERHEAD + Record.TIMESTAMP_OFFSET, timestamp);
+ long crc = record.computeChecksum();
+ Utils.writeUnsignedInt(buffer, LOG_OVERHEAD + Record.CRC_OFFSET, crc);
+ }
+
+ public void setLogAppendTime(long timestamp) {
+ if (record.magic() == Record.MAGIC_VALUE_V0)
+ throw new IllegalArgumentException("Cannot set timestamp for a record with magic = 0");
+
+ byte attributes = record.attributes();
+ buffer.put(LOG_OVERHEAD + Record.ATTRIBUTES_OFFSET, TimestampType.LOG_APPEND_TIME.updateAttributes(attributes));
+ buffer.putLong(LOG_OVERHEAD + Record.TIMESTAMP_OFFSET, timestamp);
+ long crc = record.computeChecksum();
+ Utils.writeUnsignedInt(buffer, LOG_OVERHEAD + Record.CRC_OFFSET, crc);
+ }
+
+ public ByteBuffer buffer() {
+ return buffer;
+ }
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java
index 1c9fbaa..3fb7f49 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/ByteBufferOutputStream.java
@@ -16,42 +16,54 @@
*/
package org.apache.kafka.common.record;
+import java.io.DataOutputStream;
import java.io.OutputStream;
import java.nio.ByteBuffer;
/**
* A byte buffer backed output outputStream
*/
-public class ByteBufferOutputStream extends OutputStream {
+public class ByteBufferOutputStream extends DataOutputStream {
private static final float REALLOCATION_FACTOR = 1.1f;
- private ByteBuffer buffer;
-
public ByteBufferOutputStream(ByteBuffer buffer) {
- this.buffer = buffer;
+ super(new UnderlyingOutputStream(buffer));
}
- public void write(int b) {
- if (buffer.remaining() < 1)
- expandBuffer(buffer.capacity() + 1);
- buffer.put((byte) b);
+ public ByteBuffer buffer() {
+ return ((UnderlyingOutputStream) out).buffer;
}
- public void write(byte[] bytes, int off, int len) {
- if (buffer.remaining() < len)
- expandBuffer(buffer.capacity() + len);
- buffer.put(bytes, off, len);
- }
+ public static class UnderlyingOutputStream extends OutputStream {
+ private ByteBuffer buffer;
- public ByteBuffer buffer() {
- return buffer;
- }
+ public UnderlyingOutputStream(ByteBuffer buffer) {
+ this.buffer = buffer;
+ }
- private void expandBuffer(int size) {
- int expandSize = Math.max((int) (buffer.capacity() * REALLOCATION_FACTOR), size);
- ByteBuffer temp = ByteBuffer.allocate(expandSize);
- temp.put(buffer.array(), buffer.arrayOffset(), buffer.position());
- buffer = temp;
+ public void write(int b) {
+ if (buffer.remaining() < 1)
+ expandBuffer(buffer.capacity() + 1);
+ buffer.put((byte) b);
+ }
+
+ public void write(byte[] bytes, int off, int len) {
+ if (buffer.remaining() < len)
+ expandBuffer(buffer.capacity() + len);
+ buffer.put(bytes, off, len);
+ }
+
+ public ByteBuffer buffer() {
+ return buffer;
+ }
+
+ private void expandBuffer(int size) {
+ int expandSize = Math.max((int) (buffer.capacity() * REALLOCATION_FACTOR), size);
+ ByteBuffer temp = ByteBuffer.allocate(expandSize);
+ temp.put(buffer.array(), buffer.arrayOffset(), buffer.position());
+ buffer = temp;
+ }
}
+
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java b/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java
index 65a7e43..e1d4754 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java
@@ -26,7 +26,7 @@ public enum CompressionType {
public final String name;
public final float rate;
- private CompressionType(int id, String name, float rate) {
+ CompressionType(int id, String name, float rate) {
this.id = id;
this.name = name;
this.rate = rate;
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/Compressor.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/Compressor.java b/clients/src/main/java/org/apache/kafka/common/record/Compressor.java
deleted file mode 100644
index a806975..0000000
--- a/clients/src/main/java/org/apache/kafka/common/record/Compressor.java
+++ /dev/null
@@ -1,332 +0,0 @@
-/**
- * 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 org.apache.kafka.common.record;
-
-import java.lang.reflect.Constructor;
-import org.apache.kafka.common.KafkaException;
-import org.apache.kafka.common.utils.Utils;
-
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.zip.GZIPInputStream;
-import java.util.zip.GZIPOutputStream;
-
-public class Compressor {
-
- static private final float COMPRESSION_RATE_DAMPING_FACTOR = 0.9f;
- static private final float COMPRESSION_RATE_ESTIMATION_FACTOR = 1.05f;
- static private final int COMPRESSION_DEFAULT_BUFFER_SIZE = 1024;
-
- private static final float[] TYPE_TO_RATE;
-
- static {
- int maxTypeId = -1;
- for (CompressionType type : CompressionType.values())
- maxTypeId = Math.max(maxTypeId, type.id);
- TYPE_TO_RATE = new float[maxTypeId + 1];
- for (CompressionType type : CompressionType.values()) {
- TYPE_TO_RATE[type.id] = type.rate;
- }
- }
-
- // dynamically load the snappy and lz4 classes to avoid runtime dependency if we are not using compression
- // caching constructors to avoid invoking of Class.forName method for each batch
- private static MemoizingConstructorSupplier snappyOutputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
- @Override
- public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
- return Class.forName("org.xerial.snappy.SnappyOutputStream")
- .getConstructor(OutputStream.class, Integer.TYPE);
- }
- });
-
- private static MemoizingConstructorSupplier lz4OutputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
- @Override
- public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
- return Class.forName("org.apache.kafka.common.record.KafkaLZ4BlockOutputStream")
- .getConstructor(OutputStream.class);
- }
- });
-
- private static MemoizingConstructorSupplier snappyInputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
- @Override
- public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
- return Class.forName("org.xerial.snappy.SnappyInputStream")
- .getConstructor(InputStream.class);
- }
- });
-
- private static MemoizingConstructorSupplier lz4InputStreamSupplier = new MemoizingConstructorSupplier(new ConstructorSupplier() {
- @Override
- public Constructor get() throws ClassNotFoundException, NoSuchMethodException {
- return Class.forName("org.apache.kafka.common.record.KafkaLZ4BlockInputStream")
- .getConstructor(InputStream.class, Boolean.TYPE);
- }
- });
-
- private final CompressionType type;
- private final DataOutputStream appendStream;
- private final ByteBufferOutputStream bufferStream;
- private final int initPos;
-
- public long writtenUncompressed;
- public long numRecords;
- public float compressionRate;
- public long maxTimestamp;
-
- public Compressor(ByteBuffer buffer, CompressionType type) {
- this.type = type;
- this.initPos = buffer.position();
-
- this.numRecords = 0;
- this.writtenUncompressed = 0;
- this.compressionRate = 1;
- this.maxTimestamp = Record.NO_TIMESTAMP;
-
- if (type != CompressionType.NONE) {
- // for compressed records, leave space for the header and the shallow message metadata
- // and move the starting position to the value payload offset
- buffer.position(initPos + Records.LOG_OVERHEAD + Record.RECORD_OVERHEAD);
- }
-
- // create the stream
- bufferStream = new ByteBufferOutputStream(buffer);
- appendStream = wrapForOutput(bufferStream, type, COMPRESSION_DEFAULT_BUFFER_SIZE);
- }
-
- public ByteBuffer buffer() {
- return bufferStream.buffer();
- }
-
- public double compressionRate() {
- return compressionRate;
- }
-
- public void close() {
- try {
- appendStream.close();
- } catch (IOException e) {
- throw new KafkaException(e);
- }
-
- if (type != CompressionType.NONE) {
- ByteBuffer buffer = bufferStream.buffer();
- int pos = buffer.position();
- // write the header, for the end offset write as number of records - 1
- buffer.position(initPos);
- buffer.putLong(numRecords - 1);
- buffer.putInt(pos - initPos - Records.LOG_OVERHEAD);
- // write the shallow message (the crc and value size are not correct yet)
- Record.write(buffer, maxTimestamp, null, null, type, 0, -1);
- // compute the fill the value size
- int valueSize = pos - initPos - Records.LOG_OVERHEAD - Record.RECORD_OVERHEAD;
- buffer.putInt(initPos + Records.LOG_OVERHEAD + Record.KEY_OFFSET_V1, valueSize);
- // compute and fill the crc at the beginning of the message
- long crc = Record.computeChecksum(buffer,
- initPos + Records.LOG_OVERHEAD + Record.MAGIC_OFFSET,
- pos - initPos - Records.LOG_OVERHEAD - Record.MAGIC_OFFSET);
- Utils.writeUnsignedInt(buffer, initPos + Records.LOG_OVERHEAD + Record.CRC_OFFSET, crc);
- // reset the position
- buffer.position(pos);
-
- // update the compression ratio
- this.compressionRate = (float) buffer.position() / this.writtenUncompressed;
- TYPE_TO_RATE[type.id] = TYPE_TO_RATE[type.id] * COMPRESSION_RATE_DAMPING_FACTOR +
- compressionRate * (1 - COMPRESSION_RATE_DAMPING_FACTOR);
- }
- }
-
- // Note that for all the write operations below, IO exceptions should
- // never be thrown since the underlying ByteBufferOutputStream does not throw IOException;
- // therefore upon encountering this issue we just close the append stream.
-
- public void putLong(final long value) {
- try {
- appendStream.writeLong(value);
- } catch (IOException e) {
- throw new KafkaException("I/O exception when writing to the append stream, closing", e);
- }
- }
-
- public void putInt(final int value) {
- try {
- appendStream.writeInt(value);
- } catch (IOException e) {
- throw new KafkaException("I/O exception when writing to the append stream, closing", e);
- }
- }
-
- public void put(final ByteBuffer buffer) {
- try {
- appendStream.write(buffer.array(), buffer.arrayOffset(), buffer.limit());
- } catch (IOException e) {
- throw new KafkaException("I/O exception when writing to the append stream, closing", e);
- }
- }
-
- public void putByte(final byte value) {
- try {
- appendStream.write(value);
- } catch (IOException e) {
- throw new KafkaException("I/O exception when writing to the append stream, closing", e);
- }
- }
-
- public void put(final byte[] bytes, final int offset, final int len) {
- try {
- appendStream.write(bytes, offset, len);
- } catch (IOException e) {
- throw new KafkaException("I/O exception when writing to the append stream, closing", e);
- }
- }
-
- /**
- * @return CRC of the record
- */
- public long putRecord(long timestamp, byte[] key, byte[] value, CompressionType type,
- int valueOffset, int valueSize) {
- // put a record as un-compressed into the underlying stream
- long crc = Record.computeChecksum(timestamp, key, value, type, valueOffset, valueSize);
- byte attributes = Record.computeAttributes(type);
- putRecord(crc, attributes, timestamp, key, value, valueOffset, valueSize);
- return crc;
- }
-
- /**
- * Put a record as uncompressed into the underlying stream
- * @return CRC of the record
- */
- public long putRecord(long timestamp, byte[] key, byte[] value) {
- return putRecord(timestamp, key, value, CompressionType.NONE, 0, -1);
- }
-
- private void putRecord(final long crc, final byte attributes, final long timestamp, final byte[] key, final byte[] value, final int valueOffset, final int valueSize) {
- maxTimestamp = Math.max(maxTimestamp, timestamp);
- Record.write(this, crc, attributes, timestamp, key, value, valueOffset, valueSize);
- }
-
- public void recordWritten(int size) {
- numRecords += 1;
- writtenUncompressed += size;
- }
-
- public long numRecordsWritten() {
- return numRecords;
- }
-
- public long estimatedBytesWritten() {
- if (type == CompressionType.NONE) {
- return bufferStream.buffer().position();
- } else {
- // estimate the written bytes to the underlying byte buffer based on uncompressed written bytes
- return (long) (writtenUncompressed * TYPE_TO_RATE[type.id] * COMPRESSION_RATE_ESTIMATION_FACTOR);
- }
- }
-
- // the following two functions also need to be public since they are used in MemoryRecords.iteration
-
- public static DataOutputStream wrapForOutput(ByteBufferOutputStream buffer, CompressionType type, int bufferSize) {
- try {
- switch (type) {
- case NONE:
- return new DataOutputStream(buffer);
- case GZIP:
- return new DataOutputStream(new GZIPOutputStream(buffer, bufferSize));
- case SNAPPY:
- try {
- OutputStream stream = (OutputStream) snappyOutputStreamSupplier.get().newInstance(buffer, bufferSize);
- return new DataOutputStream(stream);
- } catch (Exception e) {
- throw new KafkaException(e);
- }
- case LZ4:
- try {
- OutputStream stream = (OutputStream) lz4OutputStreamSupplier.get().newInstance(buffer);
- return new DataOutputStream(stream);
- } catch (Exception e) {
- throw new KafkaException(e);
- }
- default:
- throw new IllegalArgumentException("Unknown compression type: " + type);
- }
- } catch (IOException e) {
- throw new KafkaException(e);
- }
- }
-
- public static DataInputStream wrapForInput(ByteBufferInputStream buffer, CompressionType type, byte messageVersion) {
- try {
- switch (type) {
- case NONE:
- return new DataInputStream(buffer);
- case GZIP:
- return new DataInputStream(new GZIPInputStream(buffer));
- case SNAPPY:
- try {
- InputStream stream = (InputStream) snappyInputStreamSupplier.get().newInstance(buffer);
- return new DataInputStream(stream);
- } catch (Exception e) {
- throw new KafkaException(e);
- }
- case LZ4:
- try {
- InputStream stream = (InputStream) lz4InputStreamSupplier.get().newInstance(buffer,
- messageVersion == Record.MAGIC_VALUE_V0);
- return new DataInputStream(stream);
- } catch (Exception e) {
- throw new KafkaException(e);
- }
- default:
- throw new IllegalArgumentException("Unknown compression type: " + type);
- }
- } catch (IOException e) {
- throw new KafkaException(e);
- }
- }
-
- private interface ConstructorSupplier {
- Constructor get() throws ClassNotFoundException, NoSuchMethodException;
- }
-
- // this code is based on Guava's @see{com.google.common.base.Suppliers.MemoizingSupplier}
- private static class MemoizingConstructorSupplier {
- final ConstructorSupplier delegate;
- transient volatile boolean initialized;
- transient Constructor value;
-
- public MemoizingConstructorSupplier(ConstructorSupplier delegate) {
- this.delegate = delegate;
- }
-
- public Constructor get() throws NoSuchMethodException, ClassNotFoundException {
- if (!initialized) {
- synchronized (this) {
- if (!initialized) {
- Constructor constructor = delegate.get();
- value = constructor;
- initialized = true;
- return constructor;
- }
- }
- }
- return value;
- }
- }
-}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java b/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java
new file mode 100644
index 0000000..ae393b0
--- /dev/null
+++ b/clients/src/main/java/org/apache/kafka/common/record/FileLogInputStream.java
@@ -0,0 +1,166 @@
+/**
+ * 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.common.record;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.errors.CorruptRecordException;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+
+/**
+ * A log input stream which is backed by a {@link FileChannel}.
+ */
+public class FileLogInputStream implements LogInputStream<FileLogInputStream.FileChannelLogEntry> {
+ private int position;
+ private final int end;
+ private final FileChannel channel;
+ private final int maxRecordSize;
+ private final ByteBuffer logHeaderBuffer = ByteBuffer.allocate(Records.LOG_OVERHEAD);
+
+ /**
+ * Create a new log input stream over the FileChannel
+ * @param channel Underlying FileChannel
+ * @param maxRecordSize Maximum size of records
+ * @param start Position in the file channel to start from
+ * @param end Position in the file channel not to read past
+ */
+ public FileLogInputStream(FileChannel channel,
+ int maxRecordSize,
+ int start,
+ int end) {
+ this.channel = channel;
+ this.maxRecordSize = maxRecordSize;
+ this.position = start;
+ this.end = end;
+ }
+
+ @Override
+ public FileChannelLogEntry nextEntry() throws IOException {
+ if (position + Records.LOG_OVERHEAD >= end)
+ return null;
+
+ logHeaderBuffer.rewind();
+ channel.read(logHeaderBuffer, position);
+ if (logHeaderBuffer.hasRemaining())
+ return null;
+
+ logHeaderBuffer.rewind();
+ long offset = logHeaderBuffer.getLong();
+ int size = logHeaderBuffer.getInt();
+
+ if (size < Record.RECORD_OVERHEAD_V0)
+ throw new CorruptRecordException(String.format("Record size is smaller than minimum record overhead (%d).", Record.RECORD_OVERHEAD_V0));
+
+ if (size > maxRecordSize)
+ throw new CorruptRecordException(String.format("Record size exceeds the largest allowable message size (%d).", maxRecordSize));
+
+ if (position + Records.LOG_OVERHEAD + size > end)
+ return null;
+
+ FileChannelLogEntry logEntry = new FileChannelLogEntry(offset, channel, position, size);
+ position += logEntry.sizeInBytes();
+ return logEntry;
+ }
+
+ /**
+ * Log entry backed by an underlying FileChannel. This allows iteration over the shallow log
+ * entries without needing to read the record data into memory until it is needed. The downside
+ * is that entries will generally no longer be readable when the underlying channel is closed.
+ */
+ public static class FileChannelLogEntry extends LogEntry {
+ private final long offset;
+ private final FileChannel channel;
+ private final int position;
+ private final int recordSize;
+ private Record record = null;
+
+ private FileChannelLogEntry(long offset,
+ FileChannel channel,
+ int position,
+ int recordSize) {
+ this.offset = offset;
+ this.channel = channel;
+ this.position = position;
+ this.recordSize = recordSize;
+ }
+
+ @Override
+ public long offset() {
+ return offset;
+ }
+
+ public int position() {
+ return position;
+ }
+
+ @Override
+ public byte magic() {
+ if (record != null)
+ return record.magic();
+
+ try {
+ byte[] magic = new byte[1];
+ ByteBuffer buf = ByteBuffer.wrap(magic);
+ channel.read(buf, position + Records.LOG_OVERHEAD + Record.MAGIC_OFFSET);
+ if (buf.hasRemaining())
+ throw new KafkaException("Failed to read magic byte from FileChannel " + channel);
+ return magic[0];
+ } catch (IOException e) {
+ throw new KafkaException(e);
+ }
+ }
+
+ /**
+ * Force load the record and its data (key and value) into memory.
+ * @return The resulting record
+ * @throws IOException for any IO errors reading from the underlying file
+ */
+ private Record loadRecord() throws IOException {
+ if (record != null)
+ return record;
+
+ ByteBuffer recordBuffer = ByteBuffer.allocate(recordSize);
+ channel.read(recordBuffer, position + Records.LOG_OVERHEAD);
+ if (recordBuffer.hasRemaining())
+ throw new IOException("Failed to read full record from channel " + channel);
+
+ recordBuffer.rewind();
+ record = new Record(recordBuffer);
+ return record;
+ }
+
+ @Override
+ public Record record() {
+ if (record != null)
+ return record;
+
+ try {
+ return loadRecord();
+ } catch (IOException e) {
+ throw new KafkaException(e);
+ }
+ }
+
+ @Override
+ public int sizeInBytes() {
+ return Records.LOG_OVERHEAD + recordSize;
+ }
+
+ }
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java
index bdae08d..faf61e9 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/FileRecords.java
@@ -18,22 +18,31 @@ package org.apache.kafka.common.record;
import org.apache.kafka.common.KafkaException;
import org.apache.kafka.common.network.TransportLayer;
+import org.apache.kafka.common.record.FileLogInputStream.FileChannelLogEntry;
+import org.apache.kafka.common.utils.Utils;
+import java.io.Closeable;
import java.io.File;
+import java.io.FileInputStream;
import java.io.IOException;
+import java.io.RandomAccessFile;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.nio.channels.GatheringByteChannel;
+import java.util.Iterator;
+import java.util.concurrent.atomic.AtomicInteger;
/**
- * File-backed record set.
+ * A {@link Records} implementation backed by a file. An optional start and end position can be applied to this
+ * instance to enable slicing a range of the log records.
*/
-public class FileRecords implements Records {
- private final File file;
+public class FileRecords extends AbstractRecords implements Closeable {
+ private final boolean isSlice;
private final FileChannel channel;
- private final long start;
- private final long end;
- private final long size;
+ private final int start;
+ private final int end;
+ private volatile File file;
+ private final AtomicInteger size;
public FileRecords(File file,
FileChannel channel,
@@ -44,83 +53,435 @@ public class FileRecords implements Records {
this.channel = channel;
this.start = start;
this.end = end;
+ this.isSlice = isSlice;
+ this.size = new AtomicInteger();
- if (isSlice)
- this.size = end - start;
- else
- this.size = Math.min(channel.size(), end) - start;
+ // set the initial size of the buffer
+ resize();
+ }
+
+ public void resize() throws IOException {
+ if (isSlice) {
+ size.set(end - start);
+ } else {
+ int limit = Math.min((int) channel.size(), end);
+ size.set(limit - start);
+
+ // if this is not a slice, update the file pointer to the end of the file
+ // set the file position to the last byte in the file
+ channel.position(limit);
+ }
}
@Override
public int sizeInBytes() {
- return (int) size;
+ return size.get();
+ }
+
+ /**
+ * Get the underlying file.
+ * @return The file
+ */
+ public File file() {
+ return file;
+ }
+
+ /**
+ * Get the underlying file channel.
+ * @return The file channel
+ */
+ public FileChannel channel() {
+ return channel;
+ }
+
+ /**
+ * Read log entries into a given buffer.
+ * @param buffer The buffer to write the entries to
+ * @param position Position in the buffer to read from
+ * @return The same buffer
+ * @throws IOException
+ */
+ public ByteBuffer readInto(ByteBuffer buffer, int position) throws IOException {
+ channel.read(buffer, position + this.start);
+ buffer.flip();
+ return buffer;
+ }
+
+ /**
+ * Return a slice of records from this instance, which is a view into this set starting from the given position
+ * and with the given size limit.
+ *
+ * If the size is beyond the end of the file, the end will be based on the size of the file at the time of the read.
+ *
+ * If this message set is already sliced, the position will be taken relative to that slicing.
+ *
+ * @param position The start position to begin the read from
+ * @param size The number of bytes after the start position to include
+ * @return A sliced wrapper on this message set limited based on the given position and size
+ */
+ public FileRecords read(int position, int size) throws IOException {
+ if (position < 0)
+ throw new IllegalArgumentException("Invalid position: " + position);
+ if (size < 0)
+ throw new IllegalArgumentException("Invalid size: " + size);
+
+ final int end;
+ if (this.start + position + size < 0)
+ end = sizeInBytes();
+ else
+ end = Math.min(this.start + position + size, sizeInBytes());
+ return new FileRecords(file, channel, this.start + position, end, true);
+ }
+
+ /**
+ * Append log entries to the buffer
+ * @param records The records to append
+ * @return the number of bytes written to the underlying file
+ */
+ public int append(MemoryRecords records) throws IOException {
+ int written = records.writeFullyTo(channel);
+ size.getAndAdd(written);
+ return written;
+ }
+
+ /**
+ * Commit all written data to the physical disk
+ */
+ public void flush() throws IOException {
+ channel.force(true);
+ }
+
+ /**
+ * Close this record set
+ */
+ public void close() throws IOException {
+ flush();
+ trim();
+ channel.close();
+ }
+
+ /**
+ * Delete this message set from the filesystem
+ * @return True iff this message set was deleted.
+ */
+ public boolean delete() {
+ Utils.closeQuietly(channel, "FileChannel");
+ return file.delete();
+ }
+
+ /**
+ * Trim file when close or roll to next file
+ */
+ public void trim() throws IOException {
+ truncateTo(sizeInBytes());
+ }
+
+ /**
+ * Update the file reference (to be used with caution since this does not reopen the file channel)
+ * @param file The new file to use
+ */
+ public void setFile(File file) {
+ this.file = file;
+ }
+
+ /**
+ * Rename the file that backs this message set
+ * @throws IOException if rename fails.
+ */
+ public void renameTo(File f) throws IOException {
+ try {
+ Utils.atomicMoveWithFallback(file.toPath(), f.toPath());
+ } finally {
+ this.file = f;
+ }
+ }
+
+ /**
+ * Truncate this file message set to the given size in bytes. Note that this API does no checking that the
+ * given size falls on a valid message boundary.
+ * In some versions of the JDK truncating to the same size as the file message set will cause an
+ * update of the files mtime, so truncate is only performed if the targetSize is smaller than the
+ * size of the underlying FileChannel.
+ * It is expected that no other threads will do writes to the log when this function is called.
+ * @param targetSize The size to truncate to. Must be between 0 and sizeInBytes.
+ * @return The number of bytes truncated off
+ */
+ public int truncateTo(int targetSize) throws IOException {
+ int originalSize = sizeInBytes();
+ if (targetSize > originalSize || targetSize < 0)
+ throw new KafkaException("Attempt to truncate log segment to " + targetSize + " bytes failed, " +
+ " size of this log segment is " + originalSize + " bytes.");
+ if (targetSize < (int) channel.size()) {
+ channel.truncate(targetSize);
+ channel.position(targetSize);
+ size.set(targetSize);
+ }
+ return originalSize - targetSize;
}
@Override
public long writeTo(GatheringByteChannel destChannel, long offset, int length) throws IOException {
long newSize = Math.min(channel.size(), end) - start;
- if (newSize < size)
+ if (newSize < size.get())
throw new KafkaException(String.format("Size of FileRecords %s has been truncated during write: old size %d, new size %d", file.getAbsolutePath(), size, newSize));
- if (offset > size)
- throw new KafkaException(String.format("The requested offset %d is out of range. The size of this FileRecords is %d.", offset, size));
-
long position = start + offset;
- long count = Math.min(length, this.size - offset);
+ long count = Math.min(length, size.get());
+ final long bytesTransferred;
if (destChannel instanceof TransportLayer) {
TransportLayer tl = (TransportLayer) destChannel;
- return tl.transferFrom(this.channel, position, count);
+ bytesTransferred = tl.transferFrom(channel, position, count);
} else {
- return this.channel.transferTo(position, count, destChannel);
+ bytesTransferred = channel.transferTo(position, count, destChannel);
+ }
+ return bytesTransferred;
+ }
+
+ /**
+ * Search forward for the file position of the last offset that is greater than or equal to the target offset
+ * and return its physical position and the size of the message (including log overhead) at the returned offset. If
+ * no such offsets are found, return null.
+ *
+ * @param targetOffset The offset to search for.
+ * @param startingPosition The starting position in the file to begin searching from.
+ */
+ public LogEntryPosition searchForOffsetWithSize(long targetOffset, int startingPosition) {
+ Iterator<FileChannelLogEntry> iterator = shallowIteratorFrom(Integer.MAX_VALUE, startingPosition);
+ while (iterator.hasNext()) {
+ FileChannelLogEntry entry = iterator.next();
+ long offset = entry.offset();
+ if (offset >= targetOffset)
+ return new LogEntryPosition(offset, entry.position(), entry.sizeInBytes());
+ }
+ return null;
+ }
+
+ /**
+ * Search forward for the message whose timestamp is greater than or equals to the target timestamp.
+ *
+ * @param targetTimestamp The timestamp to search for.
+ * @param startingPosition The starting position to search.
+ * @return The timestamp and offset of the message found. None, if no message is found.
+ */
+ public TimestampAndOffset searchForTimestamp(long targetTimestamp, int startingPosition) {
+ Iterator<FileChannelLogEntry> shallowIterator = shallowIteratorFrom(startingPosition);
+ while (shallowIterator.hasNext()) {
+ LogEntry shallowEntry = shallowIterator.next();
+ Record shallowRecord = shallowEntry.record();
+ if (shallowRecord.timestamp() >= targetTimestamp) {
+ // We found a message
+ for (LogEntry deepLogEntry : shallowEntry) {
+ long timestamp = deepLogEntry.record().timestamp();
+ if (timestamp >= targetTimestamp)
+ return new TimestampAndOffset(timestamp, deepLogEntry.offset());
+ }
+ throw new IllegalStateException(String.format("The message set (max timestamp = %s, max offset = %s" +
+ " should contain target timestamp %s, but does not.", shallowRecord.timestamp(),
+ shallowEntry.offset(), targetTimestamp));
+ }
+ }
+ return null;
+ }
+
+ /**
+ * Return the largest timestamp of the messages after a given position in this file message set.
+ * @param startingPosition The starting position.
+ * @return The largest timestamp of the messages after the given position.
+ */
+ public TimestampAndOffset largestTimestampAfter(int startingPosition) {
+ long maxTimestamp = Record.NO_TIMESTAMP;
+ long offsetOfMaxTimestamp = -1L;
+
+ Iterator<FileChannelLogEntry> shallowIterator = shallowIteratorFrom(startingPosition);
+ while (shallowIterator.hasNext()) {
+ LogEntry shallowEntry = shallowIterator.next();
+ long timestamp = shallowEntry.record().timestamp();
+ if (timestamp > maxTimestamp) {
+ maxTimestamp = timestamp;
+ offsetOfMaxTimestamp = shallowEntry.offset();
+ }
}
+ return new TimestampAndOffset(maxTimestamp, offsetOfMaxTimestamp);
+ }
+
+ /**
+ * Get an iterator over the shallow entries in the file. Note that the entries are
+ * backed by the open file channel. When the channel is closed (i.e. when this instance
+ * is closed), the entries will generally no longer be readable.
+ * @return An iterator over the shallow entries
+ */
+ @Override
+ public Iterator<FileChannelLogEntry> shallowIterator() {
+ return shallowIteratorFrom(start);
+ }
+
+ /**
+ * Get an iterator over the shallow entries, enforcing a maximum record size
+ * @param maxRecordSize The maximum allowable size of individual records (including compressed record sets)
+ * @return An iterator over the shallow entries
+ */
+ public Iterator<FileChannelLogEntry> shallowIterator(int maxRecordSize) {
+ return shallowIteratorFrom(maxRecordSize, start);
+ }
+
+ private Iterator<FileChannelLogEntry> shallowIteratorFrom(int start) {
+ return shallowIteratorFrom(Integer.MAX_VALUE, start);
+ }
+
+ private Iterator<FileChannelLogEntry> shallowIteratorFrom(int maxRecordSize, int start) {
+ final int end;
+ if (isSlice)
+ end = this.end;
+ else
+ end = this.sizeInBytes();
+ FileLogInputStream inputStream = new FileLogInputStream(channel, maxRecordSize, start, end);
+ return RecordsIterator.shallowIterator(inputStream);
}
@Override
- public RecordsIterator iterator() {
- return new RecordsIterator(new FileLogInputStream(channel, start, end), false);
+ public Iterator<LogEntry> deepIterator() {
+ final int end;
+ if (isSlice)
+ end = this.end;
+ else
+ end = this.sizeInBytes();
+ FileLogInputStream inputStream = new FileLogInputStream(channel, Integer.MAX_VALUE, start, end);
+ return new RecordsIterator(inputStream, false, false, Integer.MAX_VALUE);
+ }
+
+ public static FileRecords open(File file,
+ boolean mutable,
+ boolean fileAlreadyExists,
+ int initFileSize,
+ boolean preallocate) throws IOException {
+ FileChannel channel = openChannel(file, mutable, fileAlreadyExists, initFileSize, preallocate);
+ int end = (!fileAlreadyExists && preallocate) ? 0 : Integer.MAX_VALUE;
+ return new FileRecords(file, channel, 0, end, false);
+ }
+
+ public static FileRecords open(File file,
+ boolean fileAlreadyExists,
+ int initFileSize,
+ boolean preallocate) throws IOException {
+ return open(file, true, fileAlreadyExists, initFileSize, preallocate);
+ }
+
+ public static FileRecords open(File file, boolean mutable) throws IOException {
+ return open(file, mutable, false, 0, false);
+ }
+
+ public static FileRecords open(File file) throws IOException {
+ return open(file, true);
+ }
+
+ /**
+ * Open a channel for the given file
+ * For windows NTFS and some old LINUX file system, set preallocate to true and initFileSize
+ * with one value (for example 512 * 1025 *1024 ) can improve the kafka produce performance.
+ * @param file File path
+ * @param mutable mutable
+ * @param fileAlreadyExists File already exists or not
+ * @param initFileSize The size used for pre allocate file, for example 512 * 1025 *1024
+ * @param preallocate Pre allocate file or not, gotten from configuration.
+ */
+ private static FileChannel openChannel(File file,
+ boolean mutable,
+ boolean fileAlreadyExists,
+ int initFileSize,
+ boolean preallocate) throws IOException {
+ if (mutable) {
+ if (fileAlreadyExists) {
+ return new RandomAccessFile(file, "rw").getChannel();
+ } else {
+ if (preallocate) {
+ RandomAccessFile randomAccessFile = new RandomAccessFile(file, "rw");
+ randomAccessFile.setLength(initFileSize);
+ return randomAccessFile.getChannel();
+ } else {
+ return new RandomAccessFile(file, "rw").getChannel();
+ }
+ }
+ } else {
+ return new FileInputStream(file).getChannel();
+ }
}
- private static class FileLogInputStream implements LogInputStream {
- private long position;
- protected final long end;
- protected final FileChannel channel;
- private final ByteBuffer logHeaderBuffer = ByteBuffer.allocate(Records.LOG_OVERHEAD);
+ public static class LogEntryPosition {
+ public final long offset;
+ public final int position;
+ public final int size;
- public FileLogInputStream(FileChannel channel, long start, long end) {
- this.channel = channel;
- this.position = start;
- this.end = end;
+ public LogEntryPosition(long offset, int position, int size) {
+ this.offset = offset;
+ this.position = position;
+ this.size = size;
}
@Override
- public LogEntry nextEntry() throws IOException {
- if (position + Records.LOG_OVERHEAD >= end)
- return null;
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
- logHeaderBuffer.rewind();
- channel.read(logHeaderBuffer, position);
- if (logHeaderBuffer.hasRemaining())
- return null;
+ LogEntryPosition that = (LogEntryPosition) o;
- logHeaderBuffer.rewind();
- long offset = logHeaderBuffer.getLong();
- int size = logHeaderBuffer.getInt();
- if (size < 0)
- throw new IllegalStateException("Record with size " + size);
+ if (offset != that.offset) return false;
+ if (position != that.position) return false;
+ return size == that.size;
- if (position + Records.LOG_OVERHEAD + size > end)
- return null;
+ }
- ByteBuffer recordBuffer = ByteBuffer.allocate(size);
- channel.read(recordBuffer, position + Records.LOG_OVERHEAD);
- if (recordBuffer.hasRemaining())
- return null;
- recordBuffer.rewind();
+ @Override
+ public int hashCode() {
+ int result = (int) (offset ^ (offset >>> 32));
+ result = 31 * result + position;
+ result = 31 * result + size;
+ return result;
+ }
- Record record = new Record(recordBuffer);
- LogEntry logEntry = new LogEntry(offset, record);
- position += logEntry.size();
- return logEntry;
+ @Override
+ public String toString() {
+ return "LogEntryPosition(" +
+ "offset=" + offset +
+ ", position=" + position +
+ ", size=" + size +
+ ')';
}
}
+
+ public static class TimestampAndOffset {
+ public final long timestamp;
+ public final long offset;
+
+ public TimestampAndOffset(long timestamp, long offset) {
+ this.timestamp = timestamp;
+ this.offset = offset;
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || getClass() != o.getClass()) return false;
+
+ TimestampAndOffset that = (TimestampAndOffset) o;
+
+ if (timestamp != that.timestamp) return false;
+ return offset == that.offset;
+ }
+
+ @Override
+ public int hashCode() {
+ int result = (int) (timestamp ^ (timestamp >>> 32));
+ result = 31 * result + (int) (offset ^ (offset >>> 32));
+ return result;
+ }
+
+ @Override
+ public String toString() {
+ return "TimestampAndOffset(" +
+ "timestamp=" + timestamp +
+ ", offset=" + offset +
+ ')';
+ }
+ }
+
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/InvalidRecordException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/InvalidRecordException.java b/clients/src/main/java/org/apache/kafka/common/record/InvalidRecordException.java
index a1009ca..ee60713 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/InvalidRecordException.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/InvalidRecordException.java
@@ -16,9 +16,9 @@
*/
package org.apache.kafka.common.record;
-import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.errors.CorruptRecordException;
-public class InvalidRecordException extends KafkaException {
+public class InvalidRecordException extends CorruptRecordException {
private static final long serialVersionUID = 1;
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java b/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java
index 2e54b56..d2db356 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java
@@ -16,33 +16,156 @@
*/
package org.apache.kafka.common.record;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Collections;
+import java.util.Iterator;
+
+import static org.apache.kafka.common.record.Records.LOG_OVERHEAD;
+
/**
* An offset and record pair
*/
-public final class LogEntry {
+public abstract class LogEntry implements Iterable<LogEntry> {
- private final long offset;
- private final Record record;
+ /**
+ * Get the offset of this entry. Note that if this entry contains a compressed
+ * message set, then this offset will be the last offset of the nested entries
+ * @return the last offset contained in this entry
+ */
+ public abstract long offset();
- public LogEntry(long offset, Record record) {
- this.offset = offset;
- this.record = record;
+ /**
+ * Get the shallow record for this log entry.
+ * @return the shallow record
+ */
+ public abstract Record record();
+
+ /**
+ * Get the first offset of the records contained in this entry. Note that this
+ * generally requires deep iteration, which requires decompression, so this should
+ * be used with caution.
+ * @return The first offset contained in this entry
+ */
+ public long firstOffset() {
+ return iterator().next().offset();
}
- public long offset() {
- return this.offset;
+ /**
+ * Get the offset following this entry (i.e. the last offset contained in this entry plus one).
+ * @return the next consecutive offset following this entry
+ */
+ public long nextOffset() {
+ return offset() + 1;
}
- public Record record() {
- return this.record;
+ /**
+ * Get the message format version of this entry (i.e its magic value).
+ * @return the magic byte
+ */
+ public byte magic() {
+ return record().magic();
}
@Override
public String toString() {
- return "LogEntry(" + offset + ", " + record + ")";
+ return "LogEntry(" + offset() + ", " + record() + ")";
+ }
+
+ /**
+ * Get the size in bytes of this entry, including the size of the record and the log overhead.
+ * @return The size in bytes of this entry
+ */
+ public int sizeInBytes() {
+ return record().sizeInBytes() + LOG_OVERHEAD;
+ }
+
+ /**
+ * Check whether this entry contains a compressed message set.
+ * @return true if so, false otherwise
+ */
+ public boolean isCompressed() {
+ return record().compressionType() != CompressionType.NONE;
+ }
+
+ /**
+ * Write this entry into a buffer.
+ * @param buffer The buffer to write the entry to
+ */
+ public void writeTo(ByteBuffer buffer) {
+ writeHeader(buffer, offset(), record().sizeInBytes());
+ buffer.put(record().buffer().duplicate());
+ }
+
+ /**
+ * Get an iterator for the nested entries contained within this log entry. Note that
+ * if the entry is not compressed, then this method will return an iterator over the
+ * shallow entry only (i.e. this object).
+ * @return An iterator over the entries contained within this log entry
+ */
+ @Override
+ public Iterator<LogEntry> iterator() {
+ if (isCompressed())
+ return new RecordsIterator.DeepRecordsIterator(this, false, Integer.MAX_VALUE);
+ return Collections.singletonList(this).iterator();
+ }
+
+ @Override
+ public boolean equals(Object o) {
+ if (this == o) return true;
+ if (o == null || !(o instanceof LogEntry)) return false;
+
+ LogEntry that = (LogEntry) o;
+
+ if (offset() != that.offset()) return false;
+ Record thisRecord = record();
+ Record thatRecord = that.record();
+ return thisRecord != null ? thisRecord.equals(thatRecord) : thatRecord == null;
+ }
+
+ @Override
+ public int hashCode() {
+ long offset = offset();
+ Record record = record();
+ int result = (int) (offset ^ (offset >>> 32));
+ result = 31 * result + (record != null ? record.hashCode() : 0);
+ return result;
+ }
+
+ public static void writeHeader(ByteBuffer buffer, long offset, int size) {
+ buffer.putLong(offset);
+ buffer.putInt(size);
+ }
+
+ public static void writeHeader(DataOutputStream out, long offset, int size) throws IOException {
+ out.writeLong(offset);
+ out.writeInt(size);
}
-
- public int size() {
- return record.size() + Records.LOG_OVERHEAD;
+
+ private static class SimpleLogEntry extends LogEntry {
+ private final long offset;
+ private final Record record;
+
+ public SimpleLogEntry(long offset, Record record) {
+ this.offset = offset;
+ this.record = record;
+ }
+
+ @Override
+ public long offset() {
+ return offset;
+ }
+
+ @Override
+ public Record record() {
+ return record;
+ }
+
+ }
+
+ public static LogEntry create(long offset, Record record) {
+ return new SimpleLogEntry(offset, record);
}
+
}
[7/9] kafka git commit: KAFKA-4390;
Replace MessageSet usage with client-side alternatives
Posted by jg...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java b/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java
index 1bc8a65..4a678d5 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/RecordsIterator.java
@@ -17,6 +17,7 @@
package org.apache.kafka.common.record;
import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.errors.CorruptRecordException;
import org.apache.kafka.common.utils.AbstractIterator;
import org.apache.kafka.common.utils.Utils;
@@ -25,51 +26,58 @@ import java.io.EOFException;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.ArrayDeque;
+import java.util.Iterator;
+/**
+ * An iterator which handles both the shallow and deep iteration of record sets.
+ */
public class RecordsIterator extends AbstractIterator<LogEntry> {
- private final LogInputStream logStream;
private final boolean shallow;
+ private final boolean ensureMatchingMagic;
+ private final int masRecordSize;
+ private final ShallowRecordsIterator<?> shallowIter;
private DeepRecordsIterator innerIter;
- public RecordsIterator(LogInputStream logStream, boolean shallow) {
- this.logStream = logStream;
+ public RecordsIterator(LogInputStream<?> logInputStream,
+ boolean shallow,
+ boolean ensureMatchingMagic,
+ int masRecordSize) {
+ this.shallowIter = new ShallowRecordsIterator<>(logInputStream);
this.shallow = shallow;
+ this.ensureMatchingMagic = ensureMatchingMagic;
+ this.masRecordSize = masRecordSize;
}
- /*
- * Read the next record from the buffer.
- *
- * Note that in the compressed message set, each message value size is set as the size of the un-compressed
- * version of the message value, so when we do de-compression allocating an array of the specified size for
- * reading compressed value data is sufficient.
+ /**
+ * Get a shallow iterator over the given input stream.
+ * @param logInputStream The log input stream to read the entries from
+ * @param <T> The type of the log entry
+ * @return The shallow iterator.
*/
+ public static <T extends LogEntry> Iterator<T> shallowIterator(LogInputStream<T> logInputStream) {
+ return new ShallowRecordsIterator<>(logInputStream);
+ }
+
@Override
protected LogEntry makeNext() {
if (innerDone()) {
- try {
- LogEntry entry = logStream.nextEntry();
- // No more record to return.
- if (entry == null)
- return allDone();
-
- // decide whether to go shallow or deep iteration if it is compressed
- CompressionType compressionType = entry.record().compressionType();
- if (compressionType == CompressionType.NONE || shallow) {
- return entry;
- } else {
- // init the inner iterator with the value payload of the message,
- // which will de-compress the payload to a set of messages;
- // since we assume nested compression is not allowed, the deep iterator
- // would not try to further decompress underlying messages
- // There will be at least one element in the inner iterator, so we don't
- // need to call hasNext() here.
- innerIter = new DeepRecordsIterator(entry);
- return innerIter.next();
- }
- } catch (EOFException e) {
+ if (!shallowIter.hasNext())
return allDone();
- } catch (IOException e) {
- throw new KafkaException(e);
+
+ LogEntry entry = shallowIter.next();
+
+ // decide whether to go shallow or deep iteration if it is compressed
+ if (shallow || !entry.isCompressed()) {
+ return entry;
+ } else {
+ // init the inner iterator with the value payload of the message,
+ // which will de-compress the payload to a set of messages;
+ // since we assume nested compression is not allowed, the deep iterator
+ // would not try to further decompress underlying messages
+ // There will be at least one element in the inner iterator, so we don't
+ // need to call hasNext() here.
+ innerIter = new DeepRecordsIterator(entry, ensureMatchingMagic, masRecordSize);
+ return innerIter.next();
}
} else {
return innerIter.next();
@@ -80,38 +88,70 @@ public class RecordsIterator extends AbstractIterator<LogEntry> {
return innerIter == null || !innerIter.hasNext();
}
- private static class DataLogInputStream implements LogInputStream {
+ private static class DataLogInputStream implements LogInputStream<LogEntry> {
private final DataInputStream stream;
+ protected final int maxMessageSize;
- private DataLogInputStream(DataInputStream stream) {
+ DataLogInputStream(DataInputStream stream, int maxMessageSize) {
this.stream = stream;
+ this.maxMessageSize = maxMessageSize;
}
public LogEntry nextEntry() throws IOException {
- long offset = stream.readLong();
- int size = stream.readInt();
- if (size < 0)
- throw new IllegalStateException("Record with size " + size);
-
- byte[] recordBuffer = new byte[size];
- stream.readFully(recordBuffer, 0, size);
- ByteBuffer buf = ByteBuffer.wrap(recordBuffer);
- return new LogEntry(offset, new Record(buf));
+ try {
+ long offset = stream.readLong();
+ int size = stream.readInt();
+ if (size < Record.RECORD_OVERHEAD_V0)
+ throw new CorruptRecordException(String.format("Record size is less than the minimum record overhead (%d)", Record.RECORD_OVERHEAD_V0));
+ if (size > maxMessageSize)
+ throw new CorruptRecordException(String.format("Record size exceeds the largest allowable message size (%d).", maxMessageSize));
+
+ byte[] recordBuffer = new byte[size];
+ stream.readFully(recordBuffer, 0, size);
+ ByteBuffer buf = ByteBuffer.wrap(recordBuffer);
+ return LogEntry.create(offset, new Record(buf));
+ } catch (EOFException e) {
+ return null;
+ }
}
}
- private static class DeepRecordsIterator extends AbstractIterator<LogEntry> {
+ private static class ShallowRecordsIterator<T extends LogEntry> extends AbstractIterator<T> {
+ private final LogInputStream<T> logStream;
+
+ public ShallowRecordsIterator(LogInputStream<T> logStream) {
+ this.logStream = logStream;
+ }
+
+ @Override
+ protected T makeNext() {
+ try {
+ T entry = logStream.nextEntry();
+ if (entry == null)
+ return allDone();
+ return entry;
+ } catch (IOException e) {
+ throw new KafkaException(e);
+ }
+ }
+ }
+
+ public static class DeepRecordsIterator extends AbstractIterator<LogEntry> {
private final ArrayDeque<LogEntry> logEntries;
private final long absoluteBaseOffset;
+ private final byte wrapperMagic;
+
+ public DeepRecordsIterator(LogEntry wrapperEntry, boolean ensureMatchingMagic, int maxMessageSize) {
+ Record wrapperRecord = wrapperEntry.record();
+ this.wrapperMagic = wrapperRecord.magic();
- private DeepRecordsIterator(LogEntry entry) {
- CompressionType compressionType = entry.record().compressionType();
- ByteBuffer buffer = entry.record().value();
- DataInputStream stream = Compressor.wrapForInput(new ByteBufferInputStream(buffer), compressionType, entry.record().magic());
- LogInputStream logStream = new DataLogInputStream(stream);
+ CompressionType compressionType = wrapperRecord.compressionType();
+ ByteBuffer buffer = wrapperRecord.value();
+ DataInputStream stream = MemoryRecordsBuilder.wrapForInput(new ByteBufferInputStream(buffer), compressionType, wrapperRecord.magic());
+ LogInputStream logStream = new DataLogInputStream(stream, maxMessageSize);
- long wrapperRecordOffset = entry.offset();
- long wrapperRecordTimestamp = entry.record().timestamp();
+ long wrapperRecordOffset = wrapperEntry.offset();
+ long wrapperRecordTimestamp = wrapperRecord.timestamp();
this.logEntries = new ArrayDeque<>();
// If relative offset is used, we need to decompress the entire message first to compute
@@ -119,22 +159,27 @@ public class RecordsIterator extends AbstractIterator<LogEntry> {
// do the same for message format version 0
try {
while (true) {
- try {
- LogEntry logEntry = logStream.nextEntry();
- if (entry.record().magic() > Record.MAGIC_VALUE_V0) {
- Record recordWithTimestamp = new Record(
- logEntry.record().buffer(),
- wrapperRecordTimestamp,
- entry.record().timestampType()
- );
- logEntry = new LogEntry(logEntry.offset(), recordWithTimestamp);
- }
- logEntries.add(logEntry);
- } catch (EOFException e) {
+ LogEntry logEntry = logStream.nextEntry();
+ if (logEntry == null)
break;
+
+ Record record = logEntry.record();
+ byte magic = record.magic();
+
+ if (ensureMatchingMagic && magic != wrapperMagic)
+ throw new InvalidRecordException("Compressed message magic does not match wrapper magic");
+
+ if (magic > Record.MAGIC_VALUE_V0) {
+ Record recordWithTimestamp = new Record(
+ record.buffer(),
+ wrapperRecordTimestamp,
+ wrapperRecord.timestampType()
+ );
+ logEntry = LogEntry.create(logEntry.offset(), recordWithTimestamp);
}
+ logEntries.addLast(logEntry);
}
- if (entry.record().magic() > Record.MAGIC_VALUE_V0)
+ if (wrapperMagic > Record.MAGIC_VALUE_V0)
this.absoluteBaseOffset = wrapperRecordOffset - logEntries.getLast().offset();
else
this.absoluteBaseOffset = -1;
@@ -155,12 +200,10 @@ public class RecordsIterator extends AbstractIterator<LogEntry> {
// Convert offset to absolute offset if needed.
if (absoluteBaseOffset >= 0) {
long absoluteOffset = absoluteBaseOffset + entry.offset();
- entry = new LogEntry(absoluteOffset, entry.record());
+ entry = LogEntry.create(absoluteOffset, entry.record());
}
- // decide whether to go shallow or deep iteration if it is compressed
- CompressionType compression = entry.record().compressionType();
- if (compression != CompressionType.NONE)
+ if (entry.isCompressed())
throw new InvalidRecordException("Inner messages must not be compressed");
return entry;
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java b/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java
index 62fd814..55c966a 100644
--- a/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/TimestampType.java
@@ -27,6 +27,7 @@ public enum TimestampType {
public final int id;
public final String name;
+
TimestampType(int id, String name) {
this.id = id;
this.name = name;
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
index c3c1045..c5e6716 100755
--- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
@@ -266,6 +266,24 @@ public class Utils {
}
/**
+ * Convert a ByteBuffer to a nullable array.
+ * @param buffer The buffer to convert
+ * @return The resulting array or null if the buffer is null
+ */
+ public static byte[] toNullableArray(ByteBuffer buffer) {
+ return buffer == null ? null : toArray(buffer);
+ }
+
+ /**
+ * Wrap an array as a nullable ByteBuffer.
+ * @param array The nullable array to wrap
+ * @return The wrapping ByteBuffer or null if array is null
+ */
+ public static ByteBuffer wrapNullable(byte[] array) {
+ return array == null ? null : ByteBuffer.wrap(array);
+ }
+
+ /**
* Read a byte array from the given offset and size in the buffer
*/
public static byte[] toArray(ByteBuffer buffer, int offset, int size) {
@@ -733,4 +751,37 @@ public class Utils {
public static int longHashcode(long value) {
return (int) (value ^ (value >>> 32));
}
+
+ /**
+ * Read a size-delimited byte buffer starting at the given offset.
+ * @param buffer Buffer containing the size and data
+ * @param start Offset in the buffer to read from
+ * @return A slice of the buffer containing only the delimited data (excluding the size)
+ */
+ public static ByteBuffer sizeDelimited(ByteBuffer buffer, int start) {
+ int size = buffer.getInt(start);
+ if (size < 0) {
+ return null;
+ } else {
+ ByteBuffer b = buffer.duplicate();
+ b.position(start + 4);
+ b = b.slice();
+ b.limit(size);
+ b.rewind();
+ return b;
+ }
+ }
+
+ /**
+ * Compute the checksum of a range of data
+ * @param buffer Buffer containing the data to checksum
+ * @param start Offset in the buffer to read from
+ * @param size The number of bytes to include
+ */
+ public static long computeChecksum(ByteBuffer buffer, int start, int size) {
+ Crc32 crc = new Crc32();
+ crc.update(buffer.array(), buffer.arrayOffset() + start, size);
+ return crc.getValue();
+ }
+
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
index ad6c127..a4386f8 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
@@ -39,6 +39,8 @@ import org.apache.kafka.common.protocol.ApiKeys;
import org.apache.kafka.common.protocol.Errors;
import org.apache.kafka.common.record.CompressionType;
import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.record.MemoryRecordsBuilder;
+import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.requests.AbstractRequest;
import org.apache.kafka.common.requests.FetchResponse;
import org.apache.kafka.common.requests.FetchResponse.PartitionData;
@@ -1323,11 +1325,10 @@ public class KafkaConsumerTest {
TopicPartition partition = fetchEntry.getKey();
long fetchOffset = fetchEntry.getValue().offset;
int fetchCount = fetchEntry.getValue().count;
- MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
+ MemoryRecordsBuilder records = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
for (int i = 0; i < fetchCount; i++)
records.append(fetchOffset + i, 0L, ("key-" + i).getBytes(), ("value-" + i).getBytes());
- records.close();
- tpResponses.put(partition, new FetchResponse.PartitionData(Errors.NONE.code(), 0, records));
+ tpResponses.put(partition, new FetchResponse.PartitionData(Errors.NONE.code(), 0, records.build()));
}
return new FetchResponse(tpResponses, 0);
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
index 6d5896f..15075cb 100644
--- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java
@@ -37,10 +37,12 @@ import org.apache.kafka.common.errors.TopicAuthorizationException;
import org.apache.kafka.common.metrics.KafkaMetric;
import org.apache.kafka.common.metrics.Metrics;
import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.record.ByteBufferOutputStream;
import org.apache.kafka.common.record.CompressionType;
-import org.apache.kafka.common.record.Compressor;
import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.record.MemoryRecordsBuilder;
import org.apache.kafka.common.record.Record;
+import org.apache.kafka.common.record.TimestampType;
import org.apache.kafka.common.requests.AbstractRequest;
import org.apache.kafka.common.requests.FetchRequest;
import org.apache.kafka.common.requests.FetchResponse;
@@ -93,8 +95,8 @@ public class FetcherTest {
private static final double EPSILON = 0.0001;
private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100, 1000);
- private MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
- private MemoryRecords nextRecords = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
+ private MemoryRecords records;
+ private MemoryRecords nextRecords;
private Fetcher<byte[], byte[]> fetcher = createFetcher(subscriptions, metrics);
private Metrics fetcherMetrics = new Metrics(time);
private Fetcher<byte[], byte[]> fetcherNoAutoReset = createFetcher(subscriptionsNoAutoReset, fetcherMetrics);
@@ -104,14 +106,16 @@ public class FetcherTest {
metadata.update(cluster, time.milliseconds());
client.setNode(node);
- records.append(1L, 0L, "key".getBytes(), "value-1".getBytes());
- records.append(2L, 0L, "key".getBytes(), "value-2".getBytes());
- records.append(3L, 0L, "key".getBytes(), "value-3".getBytes());
- records.close();
+ MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
+ builder.append(1L, 0L, "key".getBytes(), "value-1".getBytes());
+ builder.append(2L, 0L, "key".getBytes(), "value-2".getBytes());
+ builder.append(3L, 0L, "key".getBytes(), "value-3".getBytes());
+ records = builder.build();
- nextRecords.append(4L, 0L, "key".getBytes(), "value-4".getBytes());
- nextRecords.append(5L, 0L, "key".getBytes(), "value-5".getBytes());
- nextRecords.close();
+ builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
+ builder.append(4L, 0L, "key".getBytes(), "value-4".getBytes());
+ builder.append(5L, 0L, "key".getBytes(), "value-5".getBytes());
+ nextRecords = builder.build();
}
@After
@@ -129,7 +133,7 @@ public class FetcherTest {
assertEquals(1, fetcher.sendFetches());
assertFalse(fetcher.hasCompletedFetches());
- client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0));
+ client.prepareResponse(fetchResponse(this.records, Errors.NONE.code(), 100L, 0));
consumerClient.poll(0);
assertTrue(fetcher.hasCompletedFetches());
@@ -154,7 +158,7 @@ public class FetcherTest {
assertEquals(1, fetcher.sendFetches());
assertFalse(fetcher.hasCompletedFetches());
- client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NOT_LEADER_FOR_PARTITION.code(), 100L, 0));
+ client.prepareResponse(fetchResponse(this.records, Errors.NOT_LEADER_FOR_PARTITION.code(), 100L, 0));
consumerClient.poll(0);
assertTrue(fetcher.hasCompletedFetches());
@@ -192,7 +196,7 @@ public class FetcherTest {
subscriptions.assignFromUser(singleton(tp));
subscriptions.seek(tp, 1);
- client.prepareResponse(matchesOffset(tp, 1), fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0));
+ client.prepareResponse(matchesOffset(tp, 1), fetchResponse(this.records, Errors.NONE.code(), 100L, 0));
assertEquals(1, fetcher.sendFetches());
consumerClient.poll(0);
@@ -206,29 +210,30 @@ public class FetcherTest {
}
@Test
- public void testParseInvalidRecord() {
+ public void testParseInvalidRecord() throws Exception {
ByteBuffer buffer = ByteBuffer.allocate(1024);
- Compressor compressor = new Compressor(buffer, CompressionType.NONE);
+ ByteBufferOutputStream out = new ByteBufferOutputStream(buffer);
+ byte magic = Record.CURRENT_MAGIC_VALUE;
byte[] key = "foo".getBytes();
byte[] value = "baz".getBytes();
long offset = 0;
long timestamp = 500L;
int size = Record.recordSize(key, value);
- long crc = Record.computeChecksum(timestamp, key, value, CompressionType.NONE, 0, -1);
+ byte attributes = Record.computeAttributes(magic, CompressionType.NONE, TimestampType.CREATE_TIME);
+ long crc = Record.computeChecksum(magic, attributes, timestamp, key, value);
// write one valid record
- compressor.putLong(offset);
- compressor.putInt(size);
- Record.write(compressor, crc, Record.computeAttributes(CompressionType.NONE), timestamp, key, value, 0, -1);
+ out.writeLong(offset);
+ out.writeInt(size);
+ Record.write(out, magic, crc, Record.computeAttributes(magic, CompressionType.NONE, TimestampType.CREATE_TIME), timestamp, key, value);
// and one invalid record (note the crc)
- compressor.putLong(offset);
- compressor.putInt(size);
- Record.write(compressor, crc + 1, Record.computeAttributes(CompressionType.NONE), timestamp, key, value, 0, -1);
+ out.writeLong(offset);
+ out.writeInt(size);
+ Record.write(out, magic, crc + 1, Record.computeAttributes(magic, CompressionType.NONE, TimestampType.CREATE_TIME), timestamp, key, value);
- compressor.close();
buffer.flip();
subscriptions.assignFromUser(singleton(tp));
@@ -236,7 +241,7 @@ public class FetcherTest {
// normal fetch
assertEquals(1, fetcher.sendFetches());
- client.prepareResponse(fetchResponse(buffer, Errors.NONE.code(), 100L, 0));
+ client.prepareResponse(fetchResponse(MemoryRecords.readableRecords(buffer), Errors.NONE.code(), 100L, 0));
consumerClient.poll(0);
try {
fetcher.fetchedRecords();
@@ -255,8 +260,8 @@ public class FetcherTest {
subscriptions.assignFromUser(singleton(tp));
subscriptions.seek(tp, 1);
- client.prepareResponse(matchesOffset(tp, 1), fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0));
- client.prepareResponse(matchesOffset(tp, 4), fetchResponse(this.nextRecords.buffer(), Errors.NONE.code(), 100L, 0));
+ client.prepareResponse(matchesOffset(tp, 1), fetchResponse(this.records, Errors.NONE.code(), 100L, 0));
+ client.prepareResponse(matchesOffset(tp, 4), fetchResponse(this.nextRecords, Errors.NONE.code(), 100L, 0));
assertEquals(1, fetcher.sendFetches());
consumerClient.poll(0);
@@ -287,11 +292,11 @@ public class FetcherTest {
// if we are fetching from a compacted topic, there may be gaps in the returned records
// this test verifies the fetcher updates the current fetched/consumed positions correctly for this case
- MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
- records.append(15L, 0L, "key".getBytes(), "value-1".getBytes());
- records.append(20L, 0L, "key".getBytes(), "value-2".getBytes());
- records.append(30L, 0L, "key".getBytes(), "value-3".getBytes());
- records.close();
+ MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
+ builder.append(15L, 0L, "key".getBytes(), "value-1".getBytes());
+ builder.append(20L, 0L, "key".getBytes(), "value-2".getBytes());
+ builder.append(30L, 0L, "key".getBytes(), "value-3".getBytes());
+ MemoryRecords records = builder.build();
List<ConsumerRecord<byte[], byte[]>> consumerRecords;
subscriptions.assignFromUser(singleton(tp));
@@ -299,7 +304,7 @@ public class FetcherTest {
// normal fetch
assertEquals(1, fetcher.sendFetches());
- client.prepareResponse(fetchResponse(records.buffer(), Errors.NONE.code(), 100L, 0));
+ client.prepareResponse(fetchResponse(records, Errors.NONE.code(), 100L, 0));
consumerClient.poll(0);
consumerRecords = fetcher.fetchedRecords().get(tp);
assertEquals(3, consumerRecords.size());
@@ -317,7 +322,7 @@ public class FetcherTest {
// resize the limit of the buffer to pretend it is only fetch-size large
assertEquals(1, fetcher.sendFetches());
- client.prepareResponse(fetchResponse(this.records.buffer(), Errors.TOPIC_AUTHORIZATION_FAILED.code(), 100L, 0));
+ client.prepareResponse(fetchResponse(this.records, Errors.TOPIC_AUTHORIZATION_FAILED.code(), 100L, 0));
consumerClient.poll(0);
try {
fetcher.fetchedRecords();
@@ -337,7 +342,7 @@ public class FetcherTest {
// Now the rebalance happens and fetch positions are cleared
subscriptions.assignFromSubscribed(singleton(tp));
- client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0));
+ client.prepareResponse(fetchResponse(this.records, Errors.NONE.code(), 100L, 0));
consumerClient.poll(0);
// The active fetch should be ignored since its position is no longer valid
@@ -352,7 +357,7 @@ public class FetcherTest {
assertEquals(1, fetcher.sendFetches());
subscriptions.pause(tp);
- client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0));
+ client.prepareResponse(fetchResponse(this.records, Errors.NONE.code(), 100L, 0));
consumerClient.poll(0);
assertNull(fetcher.fetchedRecords().get(tp));
}
@@ -373,7 +378,7 @@ public class FetcherTest {
subscriptions.seek(tp, 0);
assertEquals(1, fetcher.sendFetches());
- client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NOT_LEADER_FOR_PARTITION.code(), 100L, 0));
+ client.prepareResponse(fetchResponse(this.records, Errors.NOT_LEADER_FOR_PARTITION.code(), 100L, 0));
consumerClient.poll(0);
assertEquals(0, fetcher.fetchedRecords().size());
assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds()));
@@ -385,7 +390,7 @@ public class FetcherTest {
subscriptions.seek(tp, 0);
assertEquals(1, fetcher.sendFetches());
- client.prepareResponse(fetchResponse(this.records.buffer(), Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), 100L, 0));
+ client.prepareResponse(fetchResponse(this.records, Errors.UNKNOWN_TOPIC_OR_PARTITION.code(), 100L, 0));
consumerClient.poll(0);
assertEquals(0, fetcher.fetchedRecords().size());
assertEquals(0L, metadata.timeToNextUpdate(time.milliseconds()));
@@ -397,7 +402,7 @@ public class FetcherTest {
subscriptions.seek(tp, 0);
assertEquals(1, fetcher.sendFetches());
- client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
+ client.prepareResponse(fetchResponse(this.records, Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
consumerClient.poll(0);
assertEquals(0, fetcher.fetchedRecords().size());
assertTrue(subscriptions.isOffsetResetNeeded(tp));
@@ -412,7 +417,7 @@ public class FetcherTest {
subscriptions.seek(tp, 0);
assertEquals(1, fetcher.sendFetches());
- client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
+ client.prepareResponse(fetchResponse(this.records, Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
subscriptions.seek(tp, 1);
consumerClient.poll(0);
assertEquals(0, fetcher.fetchedRecords().size());
@@ -426,7 +431,7 @@ public class FetcherTest {
subscriptionsNoAutoReset.seek(tp, 0);
assertTrue(fetcherNoAutoReset.sendFetches() > 0);
- client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
+ client.prepareResponse(fetchResponse(this.records, Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
consumerClient.poll(0);
assertFalse(subscriptionsNoAutoReset.isOffsetResetNeeded(tp));
subscriptionsNoAutoReset.seek(tp, 2);
@@ -439,7 +444,7 @@ public class FetcherTest {
subscriptionsNoAutoReset.seek(tp, 0);
fetcherNoAutoReset.sendFetches();
- client.prepareResponse(fetchResponse(this.records.buffer(), Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
+ client.prepareResponse(fetchResponse(this.records, Errors.OFFSET_OUT_OF_RANGE.code(), 100L, 0));
consumerClient.poll(0);
assertFalse(subscriptionsNoAutoReset.isOffsetResetNeeded(tp));
@@ -459,7 +464,7 @@ public class FetcherTest {
subscriptions.seek(tp, 0);
assertEquals(1, fetcher.sendFetches());
- client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 0), true);
+ client.prepareResponse(fetchResponse(this.records, Errors.NONE.code(), 100L, 0), true);
consumerClient.poll(0);
assertEquals(0, fetcher.fetchedRecords().size());
@@ -611,14 +616,14 @@ public class FetcherTest {
// We need to make sure the message offset grows. Otherwise they will be considered as already consumed
// and filtered out by consumer.
if (i > 1) {
- this.records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE);
+ MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), CompressionType.NONE, TimestampType.CREATE_TIME);
for (int v = 0; v < 3; v++) {
- this.records.append((long) i * 3 + v, Record.NO_TIMESTAMP, "key".getBytes(), String.format("value-%d", v).getBytes());
+ builder.append((long) i * 3 + v, Record.NO_TIMESTAMP, "key".getBytes(), String.format("value-%d", v).getBytes());
}
- this.records.close();
+ this.records = builder.build();
}
assertEquals(1, fetcher.sendFetches());
- client.prepareResponse(fetchResponse(this.records.buffer(), Errors.NONE.code(), 100L, 100 * i));
+ client.prepareResponse(fetchResponse(this.records, Errors.NONE.code(), 100L, 100 * i));
consumerClient.poll(0);
records = fetcher.fetchedRecords().get(tp);
assertEquals(3, records.size());
@@ -722,8 +727,7 @@ public class FetcherTest {
return new ListOffsetResponse(allPartitionData, 1);
}
- private FetchResponse fetchResponse(ByteBuffer buffer, short error, long hw, int throttleTime) {
- MemoryRecords records = MemoryRecords.readableRecords(buffer);
+ private FetchResponse fetchResponse(MemoryRecords records, short error, long hw, int throttleTime) {
return new FetchResponse(Collections.singletonMap(tp, new FetchResponse.PartitionData(error, hw, records)), throttleTime);
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java
index 28521e8..4f25bdf 100644
--- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java
@@ -12,23 +12,6 @@
*/
package org.apache.kafka.clients.producer.internals;
-import static java.util.Arrays.asList;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
-
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Deque;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Set;
-import java.util.concurrent.atomic.AtomicInteger;
-
import org.apache.kafka.clients.producer.Callback;
import org.apache.kafka.clients.producer.RecordMetadata;
import org.apache.kafka.common.Cluster;
@@ -45,6 +28,23 @@ import org.apache.kafka.common.utils.Time;
import org.junit.After;
import org.junit.Test;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import static java.util.Arrays.asList;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
public class RecordAccumulatorTest {
private String topic = "test";
@@ -84,7 +84,7 @@ public class RecordAccumulatorTest {
accum.append(tp1, 0L, key, value, null, maxBlockTimeMs);
Deque<RecordBatch> partitionBatches = accum.batches().get(tp1);
assertEquals(1, partitionBatches.size());
- assertTrue(partitionBatches.peekFirst().records.isWritable());
+ assertTrue(partitionBatches.peekFirst().isWritable());
assertEquals("No partitions should be ready.", 0, accum.ready(cluster, now).readyNodes.size());
}
@@ -93,15 +93,15 @@ public class RecordAccumulatorTest {
Deque<RecordBatch> partitionBatches = accum.batches().get(tp1);
assertEquals(2, partitionBatches.size());
Iterator<RecordBatch> partitionBatchesIterator = partitionBatches.iterator();
- assertFalse(partitionBatchesIterator.next().records.isWritable());
- assertTrue(partitionBatchesIterator.next().records.isWritable());
+ assertFalse(partitionBatchesIterator.next().isWritable());
+ assertTrue(partitionBatchesIterator.next().isWritable());
assertEquals("Our partition's leader should be ready", Collections.singleton(node1), accum.ready(cluster, time.milliseconds()).readyNodes);
List<RecordBatch> batches = accum.drain(cluster, Collections.singleton(node1), Integer.MAX_VALUE, 0).get(node1.id());
assertEquals(1, batches.size());
RecordBatch batch = batches.get(0);
- Iterator<LogEntry> iter = batch.records.iterator();
+ Iterator<LogEntry> iter = batch.records().deepIterator();
for (int i = 0; i < appends; i++) {
LogEntry entry = iter.next();
assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key());
@@ -130,7 +130,7 @@ public class RecordAccumulatorTest {
assertEquals(1, batches.size());
RecordBatch batch = batches.get(0);
- Iterator<LogEntry> iter = batch.records.iterator();
+ Iterator<LogEntry> iter = batch.records().deepIterator();
LogEntry entry = iter.next();
assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key());
assertEquals("Values should match", ByteBuffer.wrap(value), entry.record().value());
@@ -159,7 +159,7 @@ public class RecordAccumulatorTest {
final int msgs = 10000;
final int numParts = 2;
final RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 0L, 100L, metrics, time);
- List<Thread> threads = new ArrayList<Thread>();
+ List<Thread> threads = new ArrayList<>();
for (int i = 0; i < numThreads; i++) {
threads.add(new Thread() {
public void run() {
@@ -182,8 +182,11 @@ public class RecordAccumulatorTest {
List<RecordBatch> batches = accum.drain(cluster, nodes, 5 * 1024, 0).get(node1.id());
if (batches != null) {
for (RecordBatch batch : batches) {
- for (LogEntry entry : batch.records)
+ Iterator<LogEntry> deepEntries = batch.records().deepIterator();
+ while (deepEntries.hasNext()) {
+ deepEntries.next();
read++;
+ }
accum.deallocate(batch);
}
}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/test/java/org/apache/kafka/common/record/ByteBufferLogInputStreamTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/record/ByteBufferLogInputStreamTest.java b/clients/src/test/java/org/apache/kafka/common/record/ByteBufferLogInputStreamTest.java
new file mode 100644
index 0000000..62e8a05
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/record/ByteBufferLogInputStreamTest.java
@@ -0,0 +1,110 @@
+/**
+ * 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.common.record;
+
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class ByteBufferLogInputStreamTest {
+
+ @Test
+ public void iteratorIgnoresIncompleteEntries() {
+ ByteBuffer buffer = ByteBuffer.allocate(2048);
+ MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V1, CompressionType.NONE, TimestampType.CREATE_TIME, 0L);
+ builder.append(0L, 15L, "a".getBytes(), "1".getBytes());
+ builder.append(1L, 20L, "b".getBytes(), "2".getBytes());
+
+ ByteBuffer recordsBuffer = builder.build().buffer();
+ recordsBuffer.limit(recordsBuffer.limit() - 5);
+
+ Iterator<ByteBufferLogInputStream.ByteBufferLogEntry> iterator = MemoryRecords.readableRecords(recordsBuffer).shallowIterator();
+ assertTrue(iterator.hasNext());
+ ByteBufferLogInputStream.ByteBufferLogEntry first = iterator.next();
+ assertEquals(0L, first.offset());
+
+ assertFalse(iterator.hasNext());
+ }
+
+ @Test
+ public void testSetCreateTimeV1() {
+ ByteBuffer buffer = ByteBuffer.allocate(2048);
+ MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V1, CompressionType.NONE, TimestampType.CREATE_TIME, 0L);
+ builder.append(0L, 15L, "a".getBytes(), "1".getBytes());
+ Iterator<ByteBufferLogInputStream.ByteBufferLogEntry> iterator = builder.build().shallowIterator();
+
+ assertTrue(iterator.hasNext());
+ ByteBufferLogInputStream.ByteBufferLogEntry entry = iterator.next();
+
+ long createTimeMs = 20L;
+ entry.setCreateTime(createTimeMs);
+
+ assertEquals(TimestampType.CREATE_TIME, entry.record().timestampType());
+ assertEquals(createTimeMs, entry.record().timestamp());
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testSetCreateTimeNotAllowedV0() {
+ ByteBuffer buffer = ByteBuffer.allocate(2048);
+ MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V0, CompressionType.NONE, TimestampType.CREATE_TIME, 0L);
+ builder.append(0L, 15L, "a".getBytes(), "1".getBytes());
+ Iterator<ByteBufferLogInputStream.ByteBufferLogEntry> iterator = builder.build().shallowIterator();
+
+ assertTrue(iterator.hasNext());
+ ByteBufferLogInputStream.ByteBufferLogEntry entry = iterator.next();
+
+ long createTimeMs = 20L;
+ entry.setCreateTime(createTimeMs);
+ }
+
+ @Test
+ public void testSetLogAppendTimeV1() {
+ ByteBuffer buffer = ByteBuffer.allocate(2048);
+ MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V1, CompressionType.NONE, TimestampType.CREATE_TIME, 0L);
+ builder.append(0L, 15L, "a".getBytes(), "1".getBytes());
+ Iterator<ByteBufferLogInputStream.ByteBufferLogEntry> iterator = builder.build().shallowIterator();
+
+ assertTrue(iterator.hasNext());
+ ByteBufferLogInputStream.ByteBufferLogEntry entry = iterator.next();
+
+ long logAppendTime = 20L;
+ entry.setLogAppendTime(logAppendTime);
+
+ assertEquals(TimestampType.LOG_APPEND_TIME, entry.record().timestampType());
+ assertEquals(logAppendTime, entry.record().timestamp());
+ }
+
+ @Test(expected = IllegalArgumentException.class)
+ public void testSetLogAppendTimeNotAllowedV0() {
+ ByteBuffer buffer = ByteBuffer.allocate(2048);
+ MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, Record.MAGIC_VALUE_V0, CompressionType.NONE, TimestampType.CREATE_TIME, 0L);
+ builder.append(0L, 15L, "a".getBytes(), "1".getBytes());
+ Iterator<ByteBufferLogInputStream.ByteBufferLogEntry> iterator = builder.build().shallowIterator();
+
+ assertTrue(iterator.hasNext());
+ ByteBufferLogInputStream.ByteBufferLogEntry entry = iterator.next();
+
+ long logAppendTime = 20L;
+ entry.setLogAppendTime(logAppendTime);
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java
new file mode 100644
index 0000000..7e2c256
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/record/FileRecordsTest.java
@@ -0,0 +1,410 @@
+/**
+ * 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.common.record;
+
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.test.TestUtils;
+import org.easymock.EasyMock;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.FileChannel;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.apache.kafka.test.TestUtils.tempFile;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class FileRecordsTest {
+
+ private Record[] records = new Record[] {
+ Record.create("abcd".getBytes()),
+ Record.create("efgh".getBytes()),
+ Record.create("ijkl".getBytes())
+ };
+ private FileRecords fileRecords;
+
+ @Before
+ public void setup() throws IOException {
+ this.fileRecords = createFileRecords(records);
+ }
+
+ /**
+ * Test that the cached size variable matches the actual file size as we append messages
+ */
+ @Test
+ public void testFileSize() throws IOException {
+ assertEquals(fileRecords.channel().size(), fileRecords.sizeInBytes());
+ for (int i = 0; i < 20; i++) {
+ fileRecords.append(MemoryRecords.withRecords(Record.create("abcd".getBytes())));
+ assertEquals(fileRecords.channel().size(), fileRecords.sizeInBytes());
+ }
+ }
+
+ /**
+ * Test that adding invalid bytes to the end of the log doesn't break iteration
+ */
+ @Test
+ public void testIterationOverPartialAndTruncation() throws IOException {
+ testPartialWrite(0, fileRecords);
+ testPartialWrite(2, fileRecords);
+ testPartialWrite(4, fileRecords);
+ testPartialWrite(5, fileRecords);
+ testPartialWrite(6, fileRecords);
+ }
+
+ private void testPartialWrite(int size, FileRecords fileRecords) throws IOException {
+ ByteBuffer buffer = ByteBuffer.allocate(size);
+ for (int i = 0; i < size; i++)
+ buffer.put((byte) 0);
+
+ buffer.rewind();
+
+ fileRecords.channel().write(buffer);
+
+ // appending those bytes should not change the contents
+ TestUtils.checkEquals(Arrays.asList(records).iterator(), fileRecords.records());
+ }
+
+ /**
+ * Iterating over the file does file reads but shouldn't change the position of the underlying FileChannel.
+ */
+ @Test
+ public void testIterationDoesntChangePosition() throws IOException {
+ long position = fileRecords.channel().position();
+ TestUtils.checkEquals(Arrays.asList(records).iterator(), fileRecords.records());
+ assertEquals(position, fileRecords.channel().position());
+ }
+
+ /**
+ * Test a simple append and read.
+ */
+ @Test
+ public void testRead() throws IOException {
+ FileRecords read = fileRecords.read(0, fileRecords.sizeInBytes());
+ TestUtils.checkEquals(fileRecords.shallowIterator(), read.shallowIterator());
+
+ List<LogEntry> items = shallowEntries(read);
+ LogEntry second = items.get(1);
+
+ read = fileRecords.read(second.sizeInBytes(), fileRecords.sizeInBytes());
+ assertEquals("Try a read starting from the second message",
+ items.subList(1, 3), shallowEntries(read));
+
+ read = fileRecords.read(second.sizeInBytes(), second.sizeInBytes());
+ assertEquals("Try a read of a single message starting from the second message",
+ Collections.singletonList(second), shallowEntries(read));
+ }
+
+ /**
+ * Test the MessageSet.searchFor API.
+ */
+ @Test
+ public void testSearch() throws IOException {
+ // append a new message with a high offset
+ Record lastMessage = Record.create("test".getBytes());
+ fileRecords.append(MemoryRecords.withRecords(50L, lastMessage));
+
+ List<LogEntry> entries = shallowEntries(fileRecords);
+ int position = 0;
+
+ int message1Size = entries.get(0).sizeInBytes();
+ assertEquals("Should be able to find the first message by its offset",
+ new FileRecords.LogEntryPosition(0L, position, message1Size),
+ fileRecords.searchForOffsetWithSize(0, 0));
+ position += message1Size;
+
+ int message2Size = entries.get(1).sizeInBytes();
+ assertEquals("Should be able to find second message when starting from 0",
+ new FileRecords.LogEntryPosition(1L, position, message2Size),
+ fileRecords.searchForOffsetWithSize(1, 0));
+ assertEquals("Should be able to find second message starting from its offset",
+ new FileRecords.LogEntryPosition(1L, position, message2Size),
+ fileRecords.searchForOffsetWithSize(1, position));
+ position += message2Size + entries.get(2).sizeInBytes();
+
+ int message4Size = entries.get(3).sizeInBytes();
+ assertEquals("Should be able to find fourth message from a non-existant offset",
+ new FileRecords.LogEntryPosition(50L, position, message4Size),
+ fileRecords.searchForOffsetWithSize(3, position));
+ assertEquals("Should be able to find fourth message by correct offset",
+ new FileRecords.LogEntryPosition(50L, position, message4Size),
+ fileRecords.searchForOffsetWithSize(50, position));
+ }
+
+ /**
+ * Test that the message set iterator obeys start and end slicing
+ */
+ @Test
+ public void testIteratorWithLimits() throws IOException {
+ LogEntry entry = shallowEntries(fileRecords).get(1);
+ int start = fileRecords.searchForOffsetWithSize(1, 0).position;
+ int size = entry.sizeInBytes();
+ FileRecords slice = fileRecords.read(start, size);
+ assertEquals(Collections.singletonList(entry), shallowEntries(slice));
+ FileRecords slice2 = fileRecords.read(start, size - 1);
+ assertEquals(Collections.emptyList(), shallowEntries(slice2));
+ }
+
+ /**
+ * Test the truncateTo method lops off messages and appropriately updates the size
+ */
+ @Test
+ public void testTruncate() throws IOException {
+ LogEntry entry = shallowEntries(fileRecords).get(0);
+ int end = fileRecords.searchForOffsetWithSize(1, 0).position;
+ fileRecords.truncateTo(end);
+ assertEquals(Collections.singletonList(entry), shallowEntries(fileRecords));
+ assertEquals(entry.sizeInBytes(), fileRecords.sizeInBytes());
+ }
+
+ /**
+ * Test that truncateTo only calls truncate on the FileChannel if the size of the
+ * FileChannel is bigger than the target size. This is important because some JVMs
+ * change the mtime of the file, even if truncate should do nothing.
+ */
+ @Test
+ public void testTruncateNotCalledIfSizeIsSameAsTargetSize() throws IOException {
+ FileChannel channelMock = EasyMock.createMock(FileChannel.class);
+
+ EasyMock.expect(channelMock.size()).andReturn(42L).atLeastOnce();
+ EasyMock.expect(channelMock.position(42L)).andReturn(null);
+ EasyMock.replay(channelMock);
+
+ FileRecords fileRecords = new FileRecords(tempFile(), channelMock, 0, Integer.MAX_VALUE, false);
+ fileRecords.truncateTo(42);
+
+ EasyMock.verify(channelMock);
+ }
+
+ /**
+ * Expect a KafkaException if targetSize is bigger than the size of
+ * the FileRecords.
+ */
+ @Test
+ public void testTruncateNotCalledIfSizeIsBiggerThanTargetSize() throws IOException {
+ FileChannel channelMock = EasyMock.createMock(FileChannel.class);
+
+ EasyMock.expect(channelMock.size()).andReturn(42L).atLeastOnce();
+ EasyMock.expect(channelMock.position(42L)).andReturn(null);
+ EasyMock.replay(channelMock);
+
+ FileRecords fileRecords = new FileRecords(tempFile(), channelMock, 0, Integer.MAX_VALUE, false);
+
+ try {
+ fileRecords.truncateTo(43);
+ fail("Should throw KafkaException");
+ } catch (KafkaException e) {
+ // expected
+ }
+
+ EasyMock.verify(channelMock);
+ }
+
+ /**
+ * see #testTruncateNotCalledIfSizeIsSameAsTargetSize
+ */
+ @Test
+ public void testTruncateIfSizeIsDifferentToTargetSize() throws IOException {
+ FileChannel channelMock = EasyMock.createMock(FileChannel.class);
+
+ EasyMock.expect(channelMock.size()).andReturn(42L).atLeastOnce();
+ EasyMock.expect(channelMock.position(42L)).andReturn(null).once();
+ EasyMock.expect(channelMock.truncate(23L)).andReturn(null).once();
+ EasyMock.expect(channelMock.position(23L)).andReturn(null).once();
+ EasyMock.replay(channelMock);
+
+ FileRecords fileRecords = new FileRecords(tempFile(), channelMock, 0, Integer.MAX_VALUE, false);
+ fileRecords.truncateTo(23);
+
+ EasyMock.verify(channelMock);
+ }
+
+ /**
+ * Test the new FileRecords with pre allocate as true
+ */
+ @Test
+ public void testPreallocateTrue() throws IOException {
+ File temp = tempFile();
+ FileRecords fileRecords = FileRecords.open(temp, false, 512 * 1024 * 1024, true);
+ long position = fileRecords.channel().position();
+ int size = fileRecords.sizeInBytes();
+ assertEquals(0, position);
+ assertEquals(0, size);
+ assertEquals(512 * 1024 * 1024, temp.length());
+ }
+
+ /**
+ * Test the new FileRecords with pre allocate as false
+ */
+ @Test
+ public void testPreallocateFalse() throws IOException {
+ File temp = tempFile();
+ FileRecords set = FileRecords.open(temp, false, 512 * 1024 * 1024, false);
+ long position = set.channel().position();
+ int size = set.sizeInBytes();
+ assertEquals(0, position);
+ assertEquals(0, size);
+ assertEquals(0, temp.length());
+ }
+
+ /**
+ * Test the new FileRecords with pre allocate as true and file has been clearly shut down, the file will be truncate to end of valid data.
+ */
+ @Test
+ public void testPreallocateClearShutdown() throws IOException {
+ File temp = tempFile();
+ FileRecords set = FileRecords.open(temp, false, 512 * 1024 * 1024, true);
+ set.append(MemoryRecords.withRecords(records));
+
+ int oldPosition = (int) set.channel().position();
+ int oldSize = set.sizeInBytes();
+ assertEquals(fileRecords.sizeInBytes(), oldPosition);
+ assertEquals(fileRecords.sizeInBytes(), oldSize);
+ set.close();
+
+ File tempReopen = new File(temp.getAbsolutePath());
+ FileRecords setReopen = FileRecords.open(tempReopen, true, 512 * 1024 * 1024, true);
+ int position = (int) setReopen.channel().position();
+ int size = setReopen.sizeInBytes();
+
+ assertEquals(oldPosition, position);
+ assertEquals(oldPosition, size);
+ assertEquals(oldPosition, tempReopen.length());
+ }
+
+ @Test
+ public void testFormatConversionWithPartialMessage() throws IOException {
+ LogEntry entry = shallowEntries(fileRecords).get(1);
+ int start = fileRecords.searchForOffsetWithSize(1, 0).position;
+ int size = entry.sizeInBytes();
+ FileRecords slice = fileRecords.read(start, size - 1);
+ Records messageV0 = slice.toMessageFormat(Record.MAGIC_VALUE_V0);
+ assertTrue("No message should be there", shallowEntries(messageV0).isEmpty());
+ assertEquals("There should be " + (size - 1) + " bytes", size - 1, messageV0.sizeInBytes());
+ }
+
+ @Test
+ public void testConvertNonCompressedToMagic1() throws IOException {
+ List<LogEntry> entries = Arrays.asList(
+ LogEntry.create(0L, Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, "k1".getBytes(), "hello".getBytes())),
+ LogEntry.create(2L, Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, "k2".getBytes(), "goodbye".getBytes())));
+ MemoryRecords records = MemoryRecords.withLogEntries(CompressionType.NONE, entries);
+
+ // Up conversion. In reality we only do down conversion, but up conversion should work as well.
+ // up conversion for non-compressed messages
+ try (FileRecords fileRecords = FileRecords.open(tempFile())) {
+ fileRecords.append(records);
+ fileRecords.flush();
+ Records convertedRecords = fileRecords.toMessageFormat(Record.MAGIC_VALUE_V1);
+ verifyConvertedMessageSet(entries, convertedRecords, Record.MAGIC_VALUE_V1);
+ }
+ }
+
+ @Test
+ public void testConvertCompressedToMagic1() throws IOException {
+ List<LogEntry> entries = Arrays.asList(
+ LogEntry.create(0L, Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, "k1".getBytes(), "hello".getBytes())),
+ LogEntry.create(2L, Record.create(Record.MAGIC_VALUE_V0, Record.NO_TIMESTAMP, "k2".getBytes(), "goodbye".getBytes())));
+ MemoryRecords records = MemoryRecords.withLogEntries(CompressionType.GZIP, entries);
+
+ // up conversion for compressed messages
+ try (FileRecords fileRecords = FileRecords.open(tempFile())) {
+ fileRecords.append(records);
+ fileRecords.flush();
+ Records convertedRecords = fileRecords.toMessageFormat(Record.MAGIC_VALUE_V1);
+ verifyConvertedMessageSet(entries, convertedRecords, Record.MAGIC_VALUE_V1);
+ }
+ }
+
+ @Test
+ public void testConvertNonCompressedToMagic0() throws IOException {
+ List<LogEntry> entries = Arrays.asList(
+ LogEntry.create(0L, Record.create(Record.MAGIC_VALUE_V1, 1L, "k1".getBytes(), "hello".getBytes())),
+ LogEntry.create(2L, Record.create(Record.MAGIC_VALUE_V1, 2L, "k2".getBytes(), "goodbye".getBytes())));
+ MemoryRecords records = MemoryRecords.withLogEntries(CompressionType.NONE, entries);
+
+ // down conversion for non-compressed messages
+ try (FileRecords fileRecords = FileRecords.open(tempFile())) {
+ fileRecords.append(records);
+ fileRecords.flush();
+ Records convertedRecords = fileRecords.toMessageFormat(Record.MAGIC_VALUE_V0);
+ verifyConvertedMessageSet(entries, convertedRecords, Record.MAGIC_VALUE_V0);
+ }
+ }
+
+ @Test
+ public void testConvertCompressedToMagic0() throws IOException {
+ List<LogEntry> entries = Arrays.asList(
+ LogEntry.create(0L, Record.create(Record.MAGIC_VALUE_V1, 1L, "k1".getBytes(), "hello".getBytes())),
+ LogEntry.create(2L, Record.create(Record.MAGIC_VALUE_V1, 2L, "k2".getBytes(), "goodbye".getBytes())));
+ MemoryRecords records = MemoryRecords.withLogEntries(CompressionType.GZIP, entries);
+
+ // down conversion for compressed messages
+ try (FileRecords fileRecords = FileRecords.open(tempFile())) {
+ fileRecords.append(records);
+ fileRecords.flush();
+ Records convertedRecords = fileRecords.toMessageFormat(Record.MAGIC_VALUE_V0);
+ verifyConvertedMessageSet(entries, convertedRecords, Record.MAGIC_VALUE_V0);
+ }
+ }
+
+ private void verifyConvertedMessageSet(List<LogEntry> initialEntries, Records convertedRecords, byte magicByte) {
+ int i = 0;
+ for (LogEntry logEntry : deepEntries(convertedRecords)) {
+ assertEquals("magic byte should be " + magicByte, magicByte, logEntry.record().magic());
+ assertEquals("offset should not change", initialEntries.get(i).offset(), logEntry.offset());
+ assertEquals("key should not change", initialEntries.get(i).record().key(), logEntry.record().key());
+ assertEquals("payload should not change", initialEntries.get(i).record().value(), logEntry.record().value());
+ i += 1;
+ }
+ }
+
+ private static List<LogEntry> shallowEntries(Records buffer) {
+ List<LogEntry> entries = new ArrayList<>();
+ Iterator<? extends LogEntry> iterator = buffer.shallowIterator();
+ while (iterator.hasNext())
+ entries.add(iterator.next());
+ return entries;
+ }
+
+ private static List<LogEntry> deepEntries(Records buffer) {
+ List<LogEntry> entries = new ArrayList<>();
+ Iterator<? extends LogEntry> iterator = buffer.shallowIterator();
+ while (iterator.hasNext()) {
+ for (LogEntry deepEntry : iterator.next())
+ entries.add(deepEntry);
+ }
+ return entries;
+ }
+
+ private FileRecords createFileRecords(Record ... records) throws IOException {
+ FileRecords fileRecords = FileRecords.open(tempFile());
+ fileRecords.append(MemoryRecords.withRecords(records));
+ fileRecords.flush();
+ return fileRecords;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java
new file mode 100644
index 0000000..40fa212
--- /dev/null
+++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsBuilderTest.java
@@ -0,0 +1,253 @@
+/**
+ * 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.common.record;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+@RunWith(value = Parameterized.class)
+public class MemoryRecordsBuilderTest {
+
+ private final CompressionType compressionType;
+ private final int bufferOffset;
+
+ public MemoryRecordsBuilderTest(int bufferOffset, CompressionType compressionType) {
+ this.bufferOffset = bufferOffset;
+ this.compressionType = compressionType;
+ }
+
+ @Test
+ public void testCompressionRateV0() {
+ ByteBuffer buffer = ByteBuffer.allocate(1024);
+ buffer.position(bufferOffset);
+
+ Record[] records = new Record[] {
+ Record.create(Record.MAGIC_VALUE_V0, 0L, "a".getBytes(), "1".getBytes()),
+ Record.create(Record.MAGIC_VALUE_V0, 1L, "b".getBytes(), "2".getBytes()),
+ Record.create(Record.MAGIC_VALUE_V0, 2L, "c".getBytes(), "3".getBytes()),
+ };
+
+ MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V0, compressionType,
+ TimestampType.CREATE_TIME, 0L, 0L, buffer.capacity());
+
+ int uncompressedSize = 0;
+ long offset = 0L;
+ for (Record record : records) {
+ uncompressedSize += record.sizeInBytes() + Records.LOG_OVERHEAD;
+ builder.append(offset++, record);
+ }
+
+ MemoryRecords built = builder.build();
+ if (compressionType == CompressionType.NONE) {
+ assertEquals(1.0, builder.compressionRate(), 0.00001);
+ } else {
+ int compressedSize = built.sizeInBytes() - Records.LOG_OVERHEAD - Record.RECORD_OVERHEAD_V0;
+ double computedCompressionRate = (double) compressedSize / uncompressedSize;
+ assertEquals(computedCompressionRate, builder.compressionRate(), 0.00001);
+ }
+ }
+
+ @Test
+ public void testCompressionRateV1() {
+ ByteBuffer buffer = ByteBuffer.allocate(1024);
+ buffer.position(bufferOffset);
+
+ Record[] records = new Record[] {
+ Record.create(Record.MAGIC_VALUE_V1, 0L, "a".getBytes(), "1".getBytes()),
+ Record.create(Record.MAGIC_VALUE_V1, 1L, "b".getBytes(), "2".getBytes()),
+ Record.create(Record.MAGIC_VALUE_V1, 2L, "c".getBytes(), "3".getBytes()),
+ };
+
+ MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V1, compressionType,
+ TimestampType.CREATE_TIME, 0L, 0L, buffer.capacity());
+
+ int uncompressedSize = 0;
+ long offset = 0L;
+ for (Record record : records) {
+ uncompressedSize += record.sizeInBytes() + Records.LOG_OVERHEAD;
+ builder.append(offset++, record);
+ }
+
+ MemoryRecords built = builder.build();
+ if (compressionType == CompressionType.NONE) {
+ assertEquals(1.0, builder.compressionRate(), 0.00001);
+ } else {
+ int compressedSize = built.sizeInBytes() - Records.LOG_OVERHEAD - Record.RECORD_OVERHEAD_V1;
+ double computedCompressionRate = (double) compressedSize / uncompressedSize;
+ assertEquals(computedCompressionRate, builder.compressionRate(), 0.00001);
+ }
+ }
+
+ @Test
+ public void buildUsingLogAppendTime() {
+ ByteBuffer buffer = ByteBuffer.allocate(1024);
+ buffer.position(bufferOffset);
+
+ long logAppendTime = System.currentTimeMillis();
+ MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V1, compressionType,
+ TimestampType.LOG_APPEND_TIME, 0L, logAppendTime, buffer.capacity());
+ builder.append(0L, 0L, "a".getBytes(), "1".getBytes());
+ builder.append(1L, 0L, "b".getBytes(), "2".getBytes());
+ builder.append(2L, 0L, "c".getBytes(), "3".getBytes());
+ MemoryRecords records = builder.build();
+
+ MemoryRecordsBuilder.RecordsInfo info = builder.info();
+ assertEquals(logAppendTime, info.maxTimestamp);
+
+ assertEquals(2L, info.shallowOffsetOfMaxTimestamp);
+
+ Iterator<Record> iterator = records.records();
+ while (iterator.hasNext()) {
+ Record record = iterator.next();
+ assertEquals(TimestampType.LOG_APPEND_TIME, record.timestampType());
+ assertEquals(logAppendTime, record.timestamp());
+ }
+ }
+
+ @Test
+ public void convertUsingLogAppendTime() {
+ ByteBuffer buffer = ByteBuffer.allocate(1024);
+ buffer.position(bufferOffset);
+
+ long logAppendTime = System.currentTimeMillis();
+ MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V1, compressionType,
+ TimestampType.LOG_APPEND_TIME, 0L, logAppendTime, buffer.capacity());
+
+ builder.convertAndAppend(0L, Record.create(Record.MAGIC_VALUE_V0, 0L, "a".getBytes(), "1".getBytes()));
+ builder.convertAndAppend(1L, Record.create(Record.MAGIC_VALUE_V0, 0L, "b".getBytes(), "2".getBytes()));
+ builder.convertAndAppend(2L, Record.create(Record.MAGIC_VALUE_V0, 0L, "c".getBytes(), "3".getBytes()));
+ MemoryRecords records = builder.build();
+
+ MemoryRecordsBuilder.RecordsInfo info = builder.info();
+ assertEquals(logAppendTime, info.maxTimestamp);
+
+ assertEquals(2L, info.shallowOffsetOfMaxTimestamp);
+
+ Iterator<Record> iterator = records.records();
+ while (iterator.hasNext()) {
+ Record record = iterator.next();
+ assertEquals(TimestampType.LOG_APPEND_TIME, record.timestampType());
+ assertEquals(logAppendTime, record.timestamp());
+ }
+ }
+
+ @Test
+ public void buildUsingCreateTime() {
+ ByteBuffer buffer = ByteBuffer.allocate(1024);
+ buffer.position(bufferOffset);
+
+ long logAppendTime = System.currentTimeMillis();
+ MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V1, compressionType,
+ TimestampType.CREATE_TIME, 0L, logAppendTime, buffer.capacity());
+ builder.append(0L, 0L, "a".getBytes(), "1".getBytes());
+ builder.append(1L, 2L, "b".getBytes(), "2".getBytes());
+ builder.append(2L, 1L, "c".getBytes(), "3".getBytes());
+ MemoryRecords records = builder.build();
+
+ MemoryRecordsBuilder.RecordsInfo info = builder.info();
+ assertEquals(2L, info.maxTimestamp);
+
+ if (compressionType == CompressionType.NONE)
+ assertEquals(1L, info.shallowOffsetOfMaxTimestamp);
+ else
+ assertEquals(2L, info.shallowOffsetOfMaxTimestamp);
+
+ Iterator<Record> iterator = records.records();
+ int i = 0;
+ long[] expectedTimestamps = new long[] {0L, 2L, 1L};
+ while (iterator.hasNext()) {
+ Record record = iterator.next();
+ assertEquals(TimestampType.CREATE_TIME, record.timestampType());
+ assertEquals(expectedTimestamps[i++], record.timestamp());
+ }
+ }
+
+ @Test
+ public void writePastLimit() {
+ ByteBuffer buffer = ByteBuffer.allocate(64);
+ buffer.position(bufferOffset);
+
+ long logAppendTime = System.currentTimeMillis();
+ MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V1, compressionType,
+ TimestampType.CREATE_TIME, 0L, logAppendTime, buffer.capacity());
+ builder.append(0L, 0L, "a".getBytes(), "1".getBytes());
+ builder.append(1L, 1L, "b".getBytes(), "2".getBytes());
+
+ assertFalse(builder.hasRoomFor("c".getBytes(), "3".getBytes()));
+ builder.append(2L, 2L, "c".getBytes(), "3".getBytes());
+ MemoryRecords records = builder.build();
+
+ MemoryRecordsBuilder.RecordsInfo info = builder.info();
+ assertEquals(2L, info.maxTimestamp);
+ assertEquals(2L, info.shallowOffsetOfMaxTimestamp);
+
+ Iterator<Record> iterator = records.records();
+ long i = 0L;
+ while (iterator.hasNext()) {
+ Record record = iterator.next();
+ assertEquals(TimestampType.CREATE_TIME, record.timestampType());
+ assertEquals(i++, record.timestamp());
+ }
+ }
+
+ @Test
+ public void convertUsingCreateTime() {
+ ByteBuffer buffer = ByteBuffer.allocate(1024);
+ buffer.position(bufferOffset);
+
+ long logAppendTime = System.currentTimeMillis();
+ MemoryRecordsBuilder builder = new MemoryRecordsBuilder(buffer, Record.MAGIC_VALUE_V1, compressionType,
+ TimestampType.CREATE_TIME, 0L, logAppendTime, buffer.capacity());
+
+ builder.convertAndAppend(0L, Record.create(Record.MAGIC_VALUE_V0, 0L, "a".getBytes(), "1".getBytes()));
+ builder.convertAndAppend(0L, Record.create(Record.MAGIC_VALUE_V0, 0L, "b".getBytes(), "2".getBytes()));
+ builder.convertAndAppend(0L, Record.create(Record.MAGIC_VALUE_V0, 0L, "c".getBytes(), "3".getBytes()));
+ MemoryRecords records = builder.build();
+
+ MemoryRecordsBuilder.RecordsInfo info = builder.info();
+ assertEquals(Record.NO_TIMESTAMP, info.maxTimestamp);
+ assertEquals(0L, info.shallowOffsetOfMaxTimestamp);
+
+ Iterator<Record> iterator = records.records();
+ while (iterator.hasNext()) {
+ Record record = iterator.next();
+ assertEquals(TimestampType.CREATE_TIME, record.timestampType());
+ assertEquals(Record.NO_TIMESTAMP, record.timestamp());
+ }
+ }
+
+ @Parameterized.Parameters
+ public static Collection<Object[]> data() {
+ List<Object[]> values = new ArrayList<>();
+ for (int bufferOffset : Arrays.asList(0, 15))
+ for (CompressionType compressionType : CompressionType.values())
+ values.add(new Object[] {bufferOffset, compressionType});
+ return values;
+ }
+
+}
http://git-wip-us.apache.org/repos/asf/kafka/blob/67f1e5b9/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java
index b1117f1..ef0fbeb 100644
--- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java
@@ -16,53 +16,64 @@
*/
package org.apache.kafka.common.record;
-import static org.apache.kafka.common.utils.Utils.toArray;
-import static org.junit.Assert.assertEquals;
-import static org.junit.Assert.assertFalse;
-import static org.junit.Assert.assertTrue;
+import org.apache.kafka.test.TestUtils;
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
import java.nio.ByteBuffer;
import java.util.ArrayList;
-import java.util.Arrays;
import java.util.Collection;
import java.util.Iterator;
import java.util.List;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-import org.junit.runners.Parameterized;
+import static java.util.Arrays.asList;
+import static org.apache.kafka.common.utils.Utils.toNullableArray;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
@RunWith(value = Parameterized.class)
public class MemoryRecordsTest {
private CompressionType compression;
+ private byte magic;
+ private long firstOffset;
- public MemoryRecordsTest(CompressionType compression) {
+ public MemoryRecordsTest(byte magic, long firstOffset, CompressionType compression) {
+ this.magic = magic;
this.compression = compression;
+ this.firstOffset = firstOffset;
}
@Test
public void testIterator() {
- MemoryRecords recs1 = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), compression);
- MemoryRecords recs2 = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), compression);
- List<Record> list = Arrays.asList(new Record(0L, "a".getBytes(), "1".getBytes()),
- new Record(0L, "b".getBytes(), "2".getBytes()),
- new Record(0L, "c".getBytes(), "3".getBytes()));
+ MemoryRecordsBuilder builder1 = MemoryRecords.builder(ByteBuffer.allocate(1024), magic, compression, TimestampType.CREATE_TIME, firstOffset);
+ MemoryRecordsBuilder builder2 = MemoryRecords.builder(ByteBuffer.allocate(1024), magic, compression, TimestampType.CREATE_TIME, firstOffset);
+ List<Record> list = asList(
+ Record.create(magic, 1L, "a".getBytes(), "1".getBytes()),
+ Record.create(magic, 2L, "b".getBytes(), "2".getBytes()),
+ Record.create(magic, 3L, "c".getBytes(), "3".getBytes()),
+ Record.create(magic, 4L, null, "4".getBytes()),
+ Record.create(magic, 5L, "e".getBytes(), null),
+ Record.create(magic, 6L, null, null));
+
for (int i = 0; i < list.size(); i++) {
Record r = list.get(i);
- recs1.append(i, r);
- recs2.append(i, 0L, toArray(r.key()), toArray(r.value()));
+ builder1.append(firstOffset + i, r);
+ builder2.append(firstOffset + i, i + 1, toNullableArray(r.key()), toNullableArray(r.value()));
}
- recs1.close();
- recs2.close();
+
+ MemoryRecords recs1 = builder1.build();
+ MemoryRecords recs2 = builder2.build();
for (int iteration = 0; iteration < 2; iteration++) {
- for (MemoryRecords recs : Arrays.asList(recs1, recs2)) {
- Iterator<LogEntry> iter = recs.iterator();
+ for (MemoryRecords recs : asList(recs1, recs2)) {
+ Iterator<LogEntry> iter = recs.deepIterator();
for (int i = 0; i < list.size(); i++) {
assertTrue(iter.hasNext());
LogEntry entry = iter.next();
- assertEquals((long) i, entry.offset());
+ assertEquals(firstOffset + i, entry.offset());
assertEquals(list.get(i), entry.record());
entry.record().ensureValid();
}
@@ -73,20 +84,145 @@ public class MemoryRecordsTest {
@Test
public void testHasRoomForMethod() {
- MemoryRecords recs1 = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), compression);
- recs1.append(0, new Record(0L, "a".getBytes(), "1".getBytes()));
+ MemoryRecordsBuilder builder = MemoryRecords.builder(ByteBuffer.allocate(1024), magic, compression, TimestampType.CREATE_TIME);
+ builder.append(0, Record.create(magic, 0L, "a".getBytes(), "1".getBytes()));
+
+ assertTrue(builder.hasRoomFor("b".getBytes(), "2".getBytes()));
+ builder.close();
+ assertFalse(builder.hasRoomFor("b".getBytes(), "2".getBytes()));
+ }
+
+ @Test
+ public void testFilterTo() {
+ ByteBuffer buffer = ByteBuffer.allocate(2048);
+ MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME);
+ builder.append(0L, 10L, null, "a".getBytes());
+ builder.close();
+
+ builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, 1L);
+ builder.append(1L, 11L, "1".getBytes(), "b".getBytes());
+ builder.append(2L, 12L, null, "c".getBytes());
+ builder.close();
- assertTrue(recs1.hasRoomFor("b".getBytes(), "2".getBytes()));
- recs1.close();
- assertFalse(recs1.hasRoomFor("b".getBytes(), "2".getBytes()));
+ builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, 3L);
+ builder.append(3L, 13L, null, "d".getBytes());
+ builder.append(4L, 20L, "4".getBytes(), "e".getBytes());
+ builder.append(5L, 15L, "5".getBytes(), "f".getBytes());
+ builder.close();
+
+ builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.CREATE_TIME, 6L);
+ builder.append(6L, 16L, "6".getBytes(), "g".getBytes());
+ builder.close();
+
+ buffer.flip();
+
+ ByteBuffer filtered = ByteBuffer.allocate(2048);
+ MemoryRecords.FilterResult result = MemoryRecords.readableRecords(buffer).filterTo(new RetainNonNullKeysFilter(), filtered);
+
+ filtered.flip();
+
+ assertEquals(7, result.messagesRead);
+ assertEquals(4, result.messagesRetained);
+ assertEquals(buffer.limit(), result.bytesRead);
+ assertEquals(filtered.limit(), result.bytesRetained);
+ if (magic > 0) {
+ assertEquals(20L, result.maxTimestamp);
+ if (compression == CompressionType.NONE)
+ assertEquals(4L, result.shallowOffsetOfMaxTimestamp);
+ else
+ assertEquals(5L, result.shallowOffsetOfMaxTimestamp);
+ }
+ MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered);
+
+ List<ByteBufferLogInputStream.ByteBufferLogEntry> shallowEntries = TestUtils.toList(filteredRecords.shallowIterator());
+ List<Long> expectedOffsets = compression == CompressionType.NONE ? asList(1L, 4L, 5L, 6L) : asList(1L, 5L, 6L);
+ assertEquals(expectedOffsets.size(), shallowEntries.size());
+
+ for (int i = 0; i < expectedOffsets.size(); i++) {
+ LogEntry shallowEntry = shallowEntries.get(i);
+ assertEquals(expectedOffsets.get(i).longValue(), shallowEntry.offset());
+ assertEquals(magic, shallowEntry.record().magic());
+ assertEquals(compression, shallowEntry.record().compressionType());
+ assertEquals(magic == Record.MAGIC_VALUE_V0 ? TimestampType.NO_TIMESTAMP_TYPE : TimestampType.CREATE_TIME,
+ shallowEntry.record().timestampType());
+ }
+
+ List<LogEntry> deepEntries = TestUtils.toList(filteredRecords.deepIterator());
+ assertEquals(4, deepEntries.size());
+
+ LogEntry first = deepEntries.get(0);
+ assertEquals(1L, first.offset());
+ assertEquals(Record.create(magic, 11L, "1".getBytes(), "b".getBytes()), first.record());
+
+ LogEntry second = deepEntries.get(1);
+ assertEquals(4L, second.offset());
+ assertEquals(Record.create(magic, 20L, "4".getBytes(), "e".getBytes()), second.record());
+
+ LogEntry third = deepEntries.get(2);
+ assertEquals(5L, third.offset());
+ assertEquals(Record.create(magic, 15L, "5".getBytes(), "f".getBytes()), third.record());
+
+ LogEntry fourth = deepEntries.get(3);
+ assertEquals(6L, fourth.offset());
+ assertEquals(Record.create(magic, 16L, "6".getBytes(), "g".getBytes()), fourth.record());
+ }
+
+ @Test
+ public void testFilterToPreservesLogAppendTime() {
+ long logAppendTime = System.currentTimeMillis();
+
+ ByteBuffer buffer = ByteBuffer.allocate(2048);
+ MemoryRecordsBuilder builder = MemoryRecords.builder(buffer, magic, compression,
+ TimestampType.LOG_APPEND_TIME, 0L, logAppendTime);
+ builder.append(0L, 10L, null, "a".getBytes());
+ builder.close();
+
+ builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.LOG_APPEND_TIME, 1L, logAppendTime);
+ builder.append(1L, 11L, "1".getBytes(), "b".getBytes());
+ builder.append(2L, 12L, null, "c".getBytes());
+ builder.close();
+
+ builder = MemoryRecords.builder(buffer, magic, compression, TimestampType.LOG_APPEND_TIME, 3L, logAppendTime);
+ builder.append(3L, 13L, null, "d".getBytes());
+ builder.append(4L, 14L, "4".getBytes(), "e".getBytes());
+ builder.append(5L, 15L, "5".getBytes(), "f".getBytes());
+ builder.close();
+
+ buffer.flip();
+
+ ByteBuffer filtered = ByteBuffer.allocate(2048);
+ MemoryRecords.readableRecords(buffer).filterTo(new RetainNonNullKeysFilter(), filtered);
+
+ filtered.flip();
+ MemoryRecords filteredRecords = MemoryRecords.readableRecords(filtered);
+
+ List<ByteBufferLogInputStream.ByteBufferLogEntry> shallowEntries = TestUtils.toList(filteredRecords.shallowIterator());
+ assertEquals(compression == CompressionType.NONE ? 3 : 2, shallowEntries.size());
+
+ for (LogEntry shallowEntry : shallowEntries) {
+ assertEquals(compression, shallowEntry.record().compressionType());
+ if (magic > Record.MAGIC_VALUE_V0) {
+ assertEquals(TimestampType.LOG_APPEND_TIME, shallowEntry.record().timestampType());
+ assertEquals(logAppendTime, shallowEntry.record().timestamp());
+ }
+ }
}
@Parameterized.Parameters
public static Collection<Object[]> data() {
- List<Object[]> values = new ArrayList<Object[]>();
- for (CompressionType type: CompressionType.values())
- values.add(new Object[] {type});
+ List<Object[]> values = new ArrayList<>();
+ for (long firstOffset : asList(0L, 57L))
+ for (byte magic : asList(Record.MAGIC_VALUE_V0, Record.MAGIC_VALUE_V1))
+ for (CompressionType type: CompressionType.values())
+ values.add(new Object[] {magic, firstOffset, type});
return values;
}
+
+ private static class RetainNonNullKeysFilter implements MemoryRecords.LogEntryFilter {
+ @Override
+ public boolean shouldRetain(LogEntry entry) {
+ return entry.record().hasKey();
+ }
+ }
}