diff --git core/src/main/scala/kafka/Kafka.scala core/src/main/scala/kafka/Kafka.scala index 355948e..0fe6471 100644 --- core/src/main/scala/kafka/Kafka.scala +++ core/src/main/scala/kafka/Kafka.scala @@ -24,7 +24,6 @@ import utils.{Utils, Logging} object Kafka extends Logging { def main(args: Array[String]): Unit = { - if (args.length != 1) { println("USAGE: java [options] %s server.properties".format(classOf[KafkaServer].getSimpleName())) System.exit(1) diff --git core/src/main/scala/kafka/api/FetchResponse.scala core/src/main/scala/kafka/api/FetchResponse.scala index 556e31d..41dcb10 100644 --- core/src/main/scala/kafka/api/FetchResponse.scala +++ core/src/main/scala/kafka/api/FetchResponse.scala @@ -34,7 +34,7 @@ object PartitionData { val messageSetBuffer = buffer.slice() messageSetBuffer.limit(messageSetSize) buffer.position(buffer.position + messageSetSize) - new PartitionData(partition, error, initialOffset, hw, new ByteBufferMessageSet(messageSetBuffer, initialOffset)) + new PartitionData(partition, error, initialOffset, hw, new ByteBufferMessageSet(messageSetBuffer)) } val headerSize = @@ -50,6 +50,7 @@ case class PartitionData(partition: Int, error: Short = ErrorMapping.NoError, in val sizeInBytes = PartitionData.headerSize + messages.sizeInBytes.intValue() def this(partition: Int, messages: MessageSet) = this(partition, ErrorMapping.NoError, 0L, -1L, messages) + } // SENDS diff --git core/src/main/scala/kafka/api/ProducerRequest.scala core/src/main/scala/kafka/api/ProducerRequest.scala index 38458ba..3c79f46 100644 --- core/src/main/scala/kafka/api/ProducerRequest.scala +++ core/src/main/scala/kafka/api/ProducerRequest.scala @@ -53,12 +53,13 @@ object ProducerRequest { } } -case class ProducerRequest( versionId: Short = ProducerRequest.CurrentVersion, - correlationId: Int, - clientId: String, - requiredAcks: Short, - ackTimeoutMs: Int, - data: Map[TopicAndPartition, PartitionData]) + +case class ProducerRequest(versionId: Short = ProducerRequest.CurrentVersion, + correlationId: Int, + clientId: String, + requiredAcks: Short, + ackTimeoutMs: Int, + data: Map[TopicAndPartition, PartitionData]) extends RequestOrResponse(Some(RequestKeys.ProduceKey)) { /** @@ -88,10 +89,11 @@ case class ProducerRequest( versionId: Short = ProducerRequest.CurrentVersion, buffer.putInt(topicAndPartitionData.size) //the number of partitions topicAndPartitionData.foreach(partitionAndData => { val partitionData = partitionAndData._2 + val bytes = partitionData.messages.asInstanceOf[ByteBufferMessageSet].buffer buffer.putInt(partitionData.partition) - buffer.putInt(partitionData.messages.asInstanceOf[ByteBufferMessageSet].buffer.limit) - buffer.put(partitionData.messages.asInstanceOf[ByteBufferMessageSet].buffer) - partitionData.messages.asInstanceOf[ByteBufferMessageSet].buffer.rewind + buffer.putInt(bytes.limit) + buffer.put(bytes) + bytes.rewind }) } } diff --git core/src/main/scala/kafka/api/ProducerResponse.scala core/src/main/scala/kafka/api/ProducerResponse.scala index ceb36e1..6de9c93 100644 --- core/src/main/scala/kafka/api/ProducerResponse.scala +++ core/src/main/scala/kafka/api/ProducerResponse.scala @@ -43,7 +43,7 @@ object ProducerResponse { } } -case class ProducerResponseStatus(error: Short, nextOffset: Long) +case class ProducerResponseStatus(error: Short, offset: Long) case class ProducerResponse(versionId: Short, diff --git core/src/main/scala/kafka/cluster/Partition.scala core/src/main/scala/kafka/cluster/Partition.scala index 3aa6eab..41ed288 100644 --- core/src/main/scala/kafka/cluster/Partition.scala +++ core/src/main/scala/kafka/cluster/Partition.scala @@ -43,7 +43,7 @@ class Partition(val topic: String, private val leaderISRUpdateLock = new Object private var zkVersion: Int = LeaderAndIsr.initialZKVersion private var leaderEpoch: Int = LeaderAndIsr.initialLeaderEpoch - 1 - this.logIdent = "Partition [%s, %d] on broker %d, ".format(topic, partitionId, localBrokerId) + this.logIdent = "Partition [%s, %d] on broker %d: ".format(topic, partitionId, localBrokerId) private def isReplicaLocal(replicaId: Int) : Boolean = (replicaId == localBrokerId) @@ -116,7 +116,7 @@ class Partition(val topic: String, def makeLeaderOrFollower(topic: String, partitionId: Int, leaderAndISR: LeaderAndIsr, isMakingLeader: Boolean): Boolean = { leaderISRUpdateLock synchronized { if (leaderEpoch >= leaderAndISR.leaderEpoch){ - info("Current leaderEpoch [%d] is larger or equal to the requested leaderEpoch [%d], discard the become %s request" + info("Current leader epoch [%d] is larger or equal to the requested leader epoch [%d], discard the become %s request" .format(leaderEpoch, leaderAndISR.leaderEpoch, if(isMakingLeader) "leader" else "follower")) return false } @@ -183,7 +183,7 @@ class Partition(val topic: String, def updateLeaderHWAndMaybeExpandISR(replicaId: Int, offset: Long) { leaderISRUpdateLock synchronized { - debug("Recording follower %d position %d for topic %s partition %d".format(replicaId, offset, topic, partitionId)) + debug("Recording follower %d position %d for topic %s partition %d.".format(replicaId, offset, topic, partitionId)) val replica = getOrCreateReplica(replicaId) replica.logEndOffset = offset @@ -195,7 +195,7 @@ class Partition(val topic: String, if (!inSyncReplicas.contains(replica) && replica.logEndOffset >= leaderHW) { // expand ISR val newInSyncReplicas = inSyncReplicas + replica - info("Expanding ISR for topic %s partition %d to %s".format(topic, partitionId, newInSyncReplicas.map(_.brokerId).mkString(","))) + info("Expanding ISR for topic %s partition %d to %s".format(topic, partitionId, newInSyncReplicas.map(_.brokerId).mkString(", "))) // update ISR in ZK and cache updateISR(newInSyncReplicas) replicaManager.isrExpandRate.mark() @@ -289,7 +289,7 @@ class Partition(val topic: String, } private def updateISR(newISR: Set[Replica]) { - info("Updated ISR for topic %s partition %d to %s".format(topic, partitionId, newISR.mkString(","))) + info("Updated ISR for topic %s partition %d to %s".format(topic, partitionId, newISR.mkString(", "))) val newLeaderAndISR = new LeaderAndIsr(localBrokerId, leaderEpoch, newISR.map(r => r.brokerId).toList, zkVersion) val (updateSucceeded, newVersion) = ZkUtils.conditionalUpdatePersistentPath(zkClient, ZkUtils.getTopicPartitionLeaderAndIsrPath(topic, partitionId), newLeaderAndISR.toString, zkVersion) diff --git core/src/main/scala/kafka/common/ErrorMapping.scala core/src/main/scala/kafka/common/ErrorMapping.scala index ba6f352..fc06e9e 100644 --- core/src/main/scala/kafka/common/ErrorMapping.scala +++ core/src/main/scala/kafka/common/ErrorMapping.scala @@ -67,11 +67,3 @@ object ErrorMapping { def exceptionFor(code: Short) : Throwable = codeToException(code).newInstance() } - -class InvalidTopicException(message: String) extends RuntimeException(message) { - def this() = this(null) -} - -class MessageSizeTooLargeException(message: String) extends RuntimeException(message) { - def this() = this(null) -} diff --git core/src/main/scala/kafka/common/InvalidTopicException.scala core/src/main/scala/kafka/common/InvalidTopicException.scala new file mode 100644 index 0000000..59f8874 --- /dev/null +++ core/src/main/scala/kafka/common/InvalidTopicException.scala @@ -0,0 +1,22 @@ +/** + * 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.common + +class InvalidTopicException(message: String) extends RuntimeException(message) { + def this() = this(null) +} \ No newline at end of file diff --git core/src/main/scala/kafka/common/MessageSizeTooLargeException.scala core/src/main/scala/kafka/common/MessageSizeTooLargeException.scala new file mode 100644 index 0000000..2d18324 --- /dev/null +++ core/src/main/scala/kafka/common/MessageSizeTooLargeException.scala @@ -0,0 +1,22 @@ +/** + * 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.common + +class MessageSizeTooLargeException(message: String) extends RuntimeException(message) { + def this() = this(null) +} \ No newline at end of file diff --git core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala index 8dae7e4..028e216 100644 --- core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala +++ core/src/main/scala/kafka/consumer/ConsumerFetcherThread.scala @@ -28,16 +28,20 @@ class ConsumerFetcherThread(name: String, val config: ConsumerConfig, sourceBroker: Broker, val consumerFetcherManager: ConsumerFetcherManager) - extends AbstractFetcherThread(name = name, sourceBroker = sourceBroker, socketTimeout = config.socketTimeoutMs, - socketBufferSize = config.socketBufferSize, fetchSize = config.fetchSize, - fetcherBrokerId = Request.NonFollowerId, maxWait = config.maxFetchWaitMs, - minBytes = config.minFetchBytes) { + extends AbstractFetcherThread(name = name, + sourceBroker = sourceBroker, + socketTimeout = config.socketTimeoutMs, + socketBufferSize = config.socketBufferSize, + fetchSize = config.fetchSize, + fetcherBrokerId = Request.NonFollowerId, + maxWait = config.maxFetchWaitMs, + minBytes = config.minFetchBytes) { // process fetched data def processPartitionData(topic: String, fetchOffset: Long, partitionData: PartitionData) { val pti = consumerFetcherManager.getPartitionTopicInfo((topic, partitionData.partition)) if (pti.getFetchOffset != fetchOffset) - throw new RuntimeException("offset doesn't match for topic %s partition: %d pti offset: %d fetch ofset: %d" + throw new RuntimeException("Offset doesn't match for topic %s partition: %d pti offset: %d fetch offset: %d" .format(topic, partitionData.partition, pti.getFetchOffset, fetchOffset)) pti.enqueue(partitionData.messages.asInstanceOf[ByteBufferMessageSet]) } diff --git core/src/main/scala/kafka/consumer/ConsumerIterator.scala core/src/main/scala/kafka/consumer/ConsumerIterator.scala index f287368..f49d94b 100644 --- core/src/main/scala/kafka/consumer/ConsumerIterator.scala +++ core/src/main/scala/kafka/consumer/ConsumerIterator.scala @@ -22,7 +22,7 @@ import java.util.concurrent.{TimeUnit, BlockingQueue} import kafka.serializer.Decoder import java.util.concurrent.atomic.AtomicReference import kafka.message.{MessageAndOffset, MessageAndMetadata} -import kafka.common.KafkaException +import kafka.common.{KafkaException, MessageSizeTooLargeException} /** @@ -82,9 +82,15 @@ class ConsumerIterator[T](private val channel: BlockingQueue[FetchedDataChunk], else currentDataChunk.messages.iterator current.set(localCurrent) } + // if we just updated the current chunk and it is empty that means the fetch size is too small! + if(currentDataChunk.messages.validBytes == 0) + throw new MessageSizeTooLargeException("The broker contains a message larger than the maximum fetch size of this consumer. " + + "Increase the fetch size, or decrease the maximum message size the broker will allow.") } val item = localCurrent.next() - consumedOffset = item.offset + consumedOffset = item.nextOffset + + item.message.ensureValid() // validate checksum of message to ensure it is valid new MessageAndMetadata(decoder.toEvent(item.message), currentTopicInfo.topic) } diff --git core/src/main/scala/kafka/consumer/FetchedDataChunk.scala core/src/main/scala/kafka/consumer/FetchedDataChunk.scala index ea90d18..4845fcd 100644 --- core/src/main/scala/kafka/consumer/FetchedDataChunk.scala +++ core/src/main/scala/kafka/consumer/FetchedDataChunk.scala @@ -19,6 +19,6 @@ package kafka.consumer import kafka.message.ByteBufferMessageSet -private[consumer] class FetchedDataChunk(val messages: ByteBufferMessageSet, - val topicInfo: PartitionTopicInfo, - val fetchOffset: Long) +case class FetchedDataChunk(messages: ByteBufferMessageSet, + topicInfo: PartitionTopicInfo, + fetchOffset: Long) diff --git core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala index 3281ca9..fa07c64 100644 --- core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala +++ core/src/main/scala/kafka/consumer/PartitionTopicInfo.scala @@ -22,13 +22,13 @@ import java.util.concurrent.atomic._ import kafka.message._ import kafka.utils.Logging -private[consumer] class PartitionTopicInfo(val topic: String, - val brokerId: Int, - val partitionId: Int, - private val chunkQueue: BlockingQueue[FetchedDataChunk], - private val consumedOffset: AtomicLong, - private val fetchedOffset: AtomicLong, - private val fetchSize: AtomicInteger) extends Logging { +class PartitionTopicInfo(val topic: String, + val brokerId: Int, + val partitionId: Int, + private val chunkQueue: BlockingQueue[FetchedDataChunk], + private val consumedOffset: AtomicLong, + private val fetchedOffset: AtomicLong, + private val fetchSize: AtomicInteger) extends Logging { debug("initial consumer offset of " + this + " is " + consumedOffset.get) debug("initial fetch offset of " + this + " is " + fetchedOffset.get) @@ -53,15 +53,26 @@ private[consumer] class PartitionTopicInfo(val topic: String, def enqueue(messages: ByteBufferMessageSet) { val size = messages.validBytes if(size > 0) { - // update fetched offset to the compressed data chunk size, not the decompressed message set size + val next = nextOffset(messages) trace("Updating fetch offset = " + fetchedOffset.get + " with size = " + size) chunkQueue.put(new FetchedDataChunk(messages, this, fetchedOffset.get)) - val newOffset = fetchedOffset.addAndGet(size) - debug("updated fetch offset of ( %s ) to %d".format(this, newOffset)) + fetchedOffset.set(next) + debug("updated fetch offset of ( %s ) to %d".format(this, next)) ConsumerTopicStat.getConsumerTopicStat(topic).byteRate.mark(size) ConsumerTopicStat.getConsumerAllTopicStat().byteRate.mark(size) } } + + /** + * Get the next fetch offset after this message set + */ + private def nextOffset(messages: ByteBufferMessageSet): Long = { + var nextOffset = -1L + val iter = messages.shallowIterator + while(iter.hasNext) + nextOffset = iter.next.nextOffset + nextOffset + } override def toString(): String = topic + ":" + partitionId.toString + ": fetched offset = " + fetchedOffset.get + ": consumed offset = " + consumedOffset.get diff --git core/src/main/scala/kafka/consumer/SimpleConsumer.scala core/src/main/scala/kafka/consumer/SimpleConsumer.scala index 5c1af6f..728a0d8 100644 --- core/src/main/scala/kafka/consumer/SimpleConsumer.scala +++ core/src/main/scala/kafka/consumer/SimpleConsumer.scala @@ -28,10 +28,10 @@ import kafka.metrics.{KafkaTimer, KafkaMetricsGroup} * A consumer of kafka messages */ @threadsafe -class SimpleConsumer( val host: String, - val port: Int, - val soTimeout: Int, - val bufferSize: Int ) extends Logging { +class SimpleConsumer(val host: String, + val port: Int, + val soTimeout: Int, + val bufferSize: Int) extends Logging { private val lock = new Object() private val blockingChannel = new BlockingChannel(host, port, bufferSize, BlockingChannel.UseDefaultBufferSize, soTimeout) diff --git core/src/main/scala/kafka/consumer/storage/MemoryOffsetStorage.scala core/src/main/scala/kafka/consumer/storage/MemoryOffsetStorage.scala deleted file mode 100644 index d6ce868..0000000 --- core/src/main/scala/kafka/consumer/storage/MemoryOffsetStorage.scala +++ /dev/null @@ -1,44 +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.consumer.storage - -import java.util.concurrent._ -import java.util.concurrent.atomic._ -import java.util.concurrent.locks._ - -class MemoryOffsetStorage extends OffsetStorage { - - val offsetAndLock = new ConcurrentHashMap[(Int, String), (AtomicLong, Lock)] - - def reserve(node: Int, topic: String): Long = { - val key = (node, topic) - if(!offsetAndLock.containsKey(key)) - offsetAndLock.putIfAbsent(key, (new AtomicLong(0), new ReentrantLock)) - val (offset, lock) = offsetAndLock.get(key) - lock.lock - offset.get - } - - def commit(node: Int, topic: String, offset: Long) = { - val (highwater, lock) = offsetAndLock.get((node, topic)) - highwater.set(offset) - lock.unlock - offset - } - -} diff --git core/src/main/scala/kafka/consumer/storage/OffsetStorage.scala core/src/main/scala/kafka/consumer/storage/OffsetStorage.scala deleted file mode 100644 index f9c9467..0000000 --- core/src/main/scala/kafka/consumer/storage/OffsetStorage.scala +++ /dev/null @@ -1,40 +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.consumer.storage - - -/** - * A method for storing offsets for the consumer. - * This is used to track the progress of the consumer in the stream. - */ -trait OffsetStorage { - - /** - * Reserve a range of the length given by increment. - * @param increment The size to reserver - * @return The range reserved - */ - def reserve(node: Int, topic: String): Long - - /** - * Update the offset to the new offset - * @param offset The new offset - */ - def commit(node: Int, topic: String, offset: Long) - -} diff --git core/src/main/scala/kafka/consumer/storage/OracleOffsetStorage.scala core/src/main/scala/kafka/consumer/storage/OracleOffsetStorage.scala deleted file mode 100644 index a48e7a8..0000000 --- core/src/main/scala/kafka/consumer/storage/OracleOffsetStorage.scala +++ /dev/null @@ -1,155 +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.consumer.storage.sql - -import java.sql._ -import kafka.utils._ -import kafka.consumer.storage.OffsetStorage -import kafka.common.KafkaException - -/** - * An offset storage implementation that uses an oracle database to save offsets - */ -@nonthreadsafe -class OracleOffsetStorage(val connection: Connection) extends OffsetStorage with Logging { - - private val lock = new Object - connection.setAutoCommit(false) - - def reserve(node: Int, topic: String): Long = { - /* try to get and lock the offset, if it isn't there, make it */ - val maybeOffset = selectExistingOffset(connection, node, topic) - val offset = maybeOffset match { - case Some(offset) => offset - case None => { - maybeInsertZeroOffset(connection, node, topic) - selectExistingOffset(connection, node, topic).get - } - } - - debug("Reserved node " + node + " for topic '" + topic + " offset " + offset) - - offset - } - - def commit(node: Int, topic: String, offset: Long) { - var success = false - try { - updateOffset(connection, node, topic, offset) - success = true - } finally { - commitOrRollback(connection, success) - } - debug("Updated node " + node + " for topic '" + topic + "' to " + offset) - } - - def close() { - Utils.swallow(logger.error, connection.close()) - } - - /** - * Attempt to update an existing entry in the table if there isn't already one there - * @return true iff the row didn't already exist - */ - private def maybeInsertZeroOffset(connection: Connection, node: Int, topic: String): Boolean = { - val stmt = connection.prepareStatement( - """insert into kafka_offsets (node, topic, offset) - select ?, ?, 0 from dual where not exists - (select null from kafka_offsets where node = ? and topic = ?)""") - stmt.setInt(1, node) - stmt.setString(2, topic) - stmt.setInt(3, node) - stmt.setString(4, topic) - val updated = stmt.executeUpdate() - if(updated > 1) - throw new KafkaException("More than one key updated by primary key!") - else - updated == 1 - } - - /** - * Attempt to update an existing entry in the table - * @return true iff we updated an entry - */ - private def selectExistingOffset(connection: Connection, node: Int, topic: String): Option[Long] = { - val stmt = connection.prepareStatement( - """select offset from kafka_offsets - where node = ? and topic = ? - for update""") - var results: ResultSet = null - try { - stmt.setInt(1, node) - stmt.setString(2, topic) - results = stmt.executeQuery() - if(!results.next()) { - None - } else { - val offset = results.getLong("offset") - if(results.next()) - throw new KafkaException("More than one entry for primary key!") - Some(offset) - } - } finally { - close(stmt) - close(results) - } - } - - private def updateOffset(connection: Connection, - node: Int, - topic: String, - newOffset: Long): Unit = { - val stmt = connection.prepareStatement("update kafka_offsets set offset = ? where node = ? and topic = ?") - try { - stmt.setLong(1, newOffset) - stmt.setInt(2, node) - stmt.setString(3, topic) - val updated = stmt.executeUpdate() - if(updated != 1) - throw new KafkaException("Unexpected number of keys updated: " + updated) - } finally { - close(stmt) - } - } - - - private def commitOrRollback(connection: Connection, commit: Boolean) { - if(connection != null) { - if(commit) - Utils.swallow(logger.error, connection.commit()) - else - Utils.swallow(logger.error, connection.rollback()) - } - } - - private def close(rs: ResultSet) { - if(rs != null) - Utils.swallow(logger.error, rs.close()) - } - - private def close(stmt: PreparedStatement) { - if(stmt != null) - Utils.swallow(logger.error, stmt.close()) - } - - private def close(connection: Connection) { - if(connection != null) - Utils.swallow(logger.error, connection.close()) - } - -} diff --git core/src/main/scala/kafka/javaapi/Implicits.scala core/src/main/scala/kafka/javaapi/Implicits.scala index 307ca68..cf82b38 100644 --- core/src/main/scala/kafka/javaapi/Implicits.scala +++ core/src/main/scala/kafka/javaapi/Implicits.scala @@ -21,8 +21,9 @@ import kafka.utils.Logging private[javaapi] object Implicits extends Logging { implicit def scalaMessageSetToJavaMessageSet(messageSet: kafka.message.ByteBufferMessageSet): - kafka.javaapi.message.ByteBufferMessageSet = - new kafka.javaapi.message.ByteBufferMessageSet(messageSet) + kafka.javaapi.message.ByteBufferMessageSet = { + new kafka.javaapi.message.ByteBufferMessageSet(messageSet.buffer) + } implicit def toJavaFetchResponse(response: kafka.api.FetchResponse): kafka.javaapi.FetchResponse = new kafka.javaapi.FetchResponse(response) diff --git core/src/main/scala/kafka/javaapi/TopicMetadata.scala core/src/main/scala/kafka/javaapi/TopicMetadata.scala index 7640d8d..97b6dcd 100644 --- core/src/main/scala/kafka/javaapi/TopicMetadata.scala +++ core/src/main/scala/kafka/javaapi/TopicMetadata.scala @@ -22,11 +22,11 @@ import scala.collection.JavaConversions.asList private[javaapi] object MetadataListImplicits { implicit def toJavaTopicMetadataList(topicMetadataSeq: Seq[kafka.api.TopicMetadata]): java.util.List[kafka.javaapi.TopicMetadata] = - topicMetadataSeq.map(new kafka.javaapi.TopicMetadata(_)) + asList(topicMetadataSeq.map(new kafka.javaapi.TopicMetadata(_))) implicit def toPartitionMetadataList(partitionMetadataSeq: Seq[kafka.api.PartitionMetadata]): java.util.List[kafka.javaapi.PartitionMetadata] = - partitionMetadataSeq.map(new kafka.javaapi.PartitionMetadata(_)) + asList(partitionMetadataSeq.map(new kafka.javaapi.PartitionMetadata(_))) } class TopicMetadata(private val underlying: kafka.api.TopicMetadata) { @@ -51,9 +51,9 @@ class PartitionMetadata(private val underlying: kafka.api.PartitionMetadata) { underlying.leader } - def replicas: java.util.List[Broker] = underlying.replicas + def replicas: java.util.List[Broker] = asList(underlying.replicas) - def isr: java.util.List[Broker] = underlying.isr + def isr: java.util.List[Broker] = asList(underlying.isr) def errorCode: Short = underlying.errorCode diff --git core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala index 04aaa1f..ad98b75 100644 --- core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala +++ core/src/main/scala/kafka/javaapi/TopicMetadataRequest.scala @@ -18,13 +18,13 @@ package kafka.javaapi import kafka.api._ import java.nio.ByteBuffer -import scala.collection.JavaConversions.asBuffer +import scala.collection.JavaConversions class TopicMetadataRequest(val versionId: Short, val clientId: String, val topics: java.util.List[String]) extends RequestOrResponse(Some(kafka.api.RequestKeys.MetadataKey)) { val underlying: kafka.api.TopicMetadataRequest = - new kafka.api.TopicMetadataRequest(versionId, clientId, topics) + new kafka.api.TopicMetadataRequest(versionId, clientId, JavaConversions.asBuffer(topics)) def this(topics: java.util.List[String]) = this(kafka.api.TopicMetadataRequest.CurrentVersion, kafka.api.TopicMetadataRequest.DefaultClientId, topics) diff --git core/src/main/scala/kafka/javaapi/message/ByteBufferMessageSet.scala core/src/main/scala/kafka/javaapi/message/ByteBufferMessageSet.scala index b9344c2..6eaee09 100644 --- core/src/main/scala/kafka/javaapi/message/ByteBufferMessageSet.scala +++ core/src/main/scala/kafka/javaapi/message/ByteBufferMessageSet.scala @@ -16,9 +16,21 @@ */ package kafka.javaapi.message +import java.util.concurrent.atomic.AtomicLong +import scala.reflect.BeanProperty +import java.nio.ByteBuffer import kafka.message._ -class ByteBufferMessageSet(private val underlying: kafka.message.ByteBufferMessageSet) extends MessageSet { +class ByteBufferMessageSet(@BeanProperty val buffer: ByteBuffer) extends MessageSet { + private val underlying: kafka.message.ByteBufferMessageSet = new kafka.message.ByteBufferMessageSet(buffer) + + def this(compressionCodec: CompressionCodec, messages: java.util.List[Message]) { + this(new kafka.message.ByteBufferMessageSet(compressionCodec, new AtomicLong(0), scala.collection.JavaConversions.asBuffer(messages): _*).buffer) + } + + def this(messages: java.util.List[Message]) { + this(NoCompressionCodec, messages) + } def validBytes: Long = underlying.validBytes @@ -41,12 +53,11 @@ class ByteBufferMessageSet(private val underlying: kafka.message.ByteBufferMessa override def equals(other: Any): Boolean = { other match { - case that: ByteBufferMessageSet => - underlying.equals(that.underlying) + case that: ByteBufferMessageSet => buffer.equals(that.buffer) case _ => false } } - override def hashCode: Int = underlying.hashCode + override def hashCode: Int = buffer.hashCode } diff --git core/src/main/scala/kafka/log/FileMessageSet.scala core/src/main/scala/kafka/log/FileMessageSet.scala new file mode 100644 index 0000000..bc188d9 --- /dev/null +++ core/src/main/scala/kafka/log/FileMessageSet.scala @@ -0,0 +1,204 @@ +/** + * 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 kafka.utils._ +import kafka.message._ +import kafka.common.KafkaException +import java.util.concurrent.TimeUnit +import kafka.metrics.{KafkaTimer, KafkaMetricsGroup} + +/** + * An on-disk message set. The set can be opened either mutably or immutably. Mutation attempts + * will fail on an immutable message set. An optional limit and start position can be applied to the message set + * which will control the position in the file at which the set begins + */ +@nonthreadsafe +class FileMessageSet private[kafka](val file: File, + private[log] val channel: FileChannel, + private[log] val start: Long, // the starting position in the file + private[log] val limit: Long, // the length (may be less than the file length) + val mutable: Boolean) extends MessageSet with Logging { + + private val setSize = new AtomicLong() + + if(mutable) { + if(limit < Long.MaxValue || start > 0) + throw new KafkaException("Attempt to open a mutable message set with a view or offset, which is not allowed.") + + setSize.set(channel.size()) + channel.position(channel.size) + } else { + setSize.set(scala.math.min(channel.size(), limit) - start) + } + + /** + * Create a file message set with no limit or offset + */ + def this(file: File, channel: FileChannel, mutable: Boolean) = + this(file, channel, 0, Long.MaxValue, mutable) + + /** + * Create a file message set with no limit or offset + */ + def this(file: File, mutable: Boolean) = + this(file, Utils.openChannel(file, mutable), mutable) + + /** + * Return a message set which is a view into this set starting from the given position and with the given size limit. + */ + def read(position: Long, size: Long): FileMessageSet = { + new FileMessageSet(file, channel, this.start + position, scala.math.min(this.start + position + size, sizeInBytes()), + false) + } + + /** + * Search forward for the file position of the last offset that is great than or equal to the target offset + * and return its physical position. If no such offsets are found, return null. + */ + private[log] def searchFor(targetOffset: Long, startingPosition: Int): OffsetPosition = { + var position = startingPosition + val buffer = ByteBuffer.allocate(12) + val size = setSize.get() + while(position + 12 < size) { + buffer.rewind() + channel.read(buffer, position) + if(buffer.hasRemaining) + throw new IllegalStateException("Failed to read complete buffer.") + buffer.rewind() + val offset = buffer.getLong() + if(offset >= targetOffset) + return OffsetPosition(offset, position) + val messageSize = buffer.getInt() + position += MessageSet.LogOverhead + messageSize + } + null + } + + /** + * Write some of this set to the given channel, return the amount written + */ + def writeTo(destChannel: GatheringByteChannel, writePosition: Long, size: Long): Long = + channel.transferTo(start + writePosition, scala.math.min(size, sizeInBytes), destChannel) + + /** + * Get an iterator over the messages in the set + */ + override def iterator: Iterator[MessageAndOffset] = { + new IteratorTemplate[MessageAndOffset] { + var location = start + + override def makeNext(): MessageAndOffset = { + // read the size of the item + val sizeOffsetBuffer = ByteBuffer.allocate(12) + channel.read(sizeOffsetBuffer, location) + if(sizeOffsetBuffer.hasRemaining) + return allDone() + + sizeOffsetBuffer.rewind() + val offset = sizeOffsetBuffer.getLong() + val size = sizeOffsetBuffer.getInt() + if (size < Message.MinHeaderSize) + return allDone() + + // read the item itself + val buffer = ByteBuffer.allocate(size) + channel.read(buffer, location + 12) + if(buffer.hasRemaining) + return allDone() + buffer.rewind() + + // increment the location and return the item + location += size + 12 + new MessageAndOffset(new Message(buffer), offset) + } + } + } + + /** + * The number of bytes taken up by this file set + */ + def sizeInBytes(): Long = setSize.get() + + def checkMutable(): Unit = { + if(!mutable) + throw new KafkaException("Attempt to invoke mutation on immutable message set.") + } + + /** + * Append this message to the message set + */ + def append(messages: MessageSet): Unit = { + checkMutable() + var written = 0L + while(written < messages.sizeInBytes) + written += messages.writeTo(channel, 0, messages.sizeInBytes) + setSize.getAndAdd(written) + } + + /** + * Commit all written data to the physical disk + */ + def flush() = { + checkMutable() + LogFlushStats.logFlushTimer.time { + channel.force(true) + } + } + + /** + * Close this message set + */ + def close() { + if(mutable) + flush() + channel.close() + } + + /** + * Delete this message set from the filesystem + */ + def delete(): Boolean = { + Utils.swallow(channel.close()) + file.delete() + } + + /** + * Truncate this file message set to the given size. Note that this API does no checking that the + * given size falls on a valid byte offset. + */ + def truncateTo(targetSize: Long) = { + checkMutable() + if(targetSize > sizeInBytes()) + throw new KafkaException("Attempt to truncate log segment to %d bytes failed since the current ".format(targetSize) + + " size of this log segment is only %d bytes".format(sizeInBytes())) + channel.truncate(targetSize) + channel.position(targetSize) + setSize.set(targetSize) + } + +} + +object LogFlushStats extends KafkaMetricsGroup { + val logFlushTimer = new KafkaTimer(newTimer("LogFlushRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS)) +} diff --git core/src/main/scala/kafka/log/Log.scala core/src/main/scala/kafka/log/Log.scala index 1c01df8..e5c9e87 100644 --- core/src/main/scala/kafka/log/Log.scala +++ core/src/main/scala/kafka/log/Log.scala @@ -22,48 +22,42 @@ import java.io.{IOException, File} import java.util.{Comparator, Collections, ArrayList} import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong, AtomicInteger} import kafka.utils._ +import scala.math._ import java.text.NumberFormat import kafka.server.BrokerTopicStat -import kafka.message.{ByteBufferMessageSet, MessageSet, InvalidMessageException, FileMessageSet} -import kafka.common.{KafkaException, InvalidMessageSizeException, OffsetOutOfRangeException} +import kafka.message._ +import kafka.common.{KafkaException, InvalidMessageSizeException, OffsetOutOfRangeException, MessageSizeTooLargeException} import kafka.metrics.KafkaMetricsGroup import com.yammer.metrics.core.Gauge object Log { - val FileSuffix = ".kafka" + val LogFileSuffix = ".log" + val IndexFileSuffix = ".index" /** - * Find a given range object in a list of ranges by a value in that range. Does a binary search over the ranges - * but instead of checking for equality looks within the range. Takes the array size as an option in case - * the array grows while searching happens - * - * TODO: This should move into SegmentList.scala + * Search for the greatest range with start <= the target value. */ def findRange[T <: Range](ranges: Array[T], value: Long, arraySize: Int): Option[T] = { if(ranges.size < 1) return None // check out of bounds - if(value < ranges(0).start || value > ranges(arraySize - 1).start + ranges(arraySize - 1).size) - throw new OffsetOutOfRangeException("offset " + value + " is out of range") - - // check at the end - if (value == ranges(arraySize - 1).start + ranges(arraySize - 1).size) + if(value < ranges(0).start) return None var low = 0 var high = arraySize - 1 - while(low <= high) { - val mid = (high + low) / 2 + while(low < high) { + val mid = ceil((high + low) / 2.0).toInt val found = ranges(mid) - if(found.contains(value)) + if(found.start == value) return Some(found) else if (value < found.start) high = mid - 1 else - low = mid + 1 + low = mid } - None + Some(ranges(low)) } def findRange[T <: Range](ranges: Array[T], value: Long): Option[T] = @@ -73,13 +67,19 @@ object Log { * Make log segment file name from offset bytes. All this does is pad out the offset number with zeros * so that ls sorts the files numerically */ - def nameFromOffset(offset: Long): String = { + def filenamePrefixFromOffset(offset: Long): String = { val nf = NumberFormat.getInstance() nf.setMinimumIntegerDigits(20) nf.setMaximumFractionDigits(0) nf.setGroupingUsed(false) - nf.format(offset) + FileSuffix + nf.format(offset) } + + def logFilename(dir: File, offset: Long) = + new File(dir, filenamePrefixFromOffset(offset) + LogFileSuffix) + + def indexFilename(dir: File, offset: Long) = + new File(dir, filenamePrefixFromOffset(offset) + IndexFileSuffix) def getEmptyOffsets(timestamp: Long): Seq[Long] = if (timestamp == OffsetRequest.LatestTime || timestamp == OffsetRequest.EarliestTime) @@ -89,52 +89,29 @@ object Log { /** - * A segment file in the log directory. Each log segment consists of an open message set, a start offset and a size - */ -class LogSegment(val file: File, val messageSet: FileMessageSet, val start: Long, time: Time) extends Range { - var firstAppendTime: Option[Long] = None - @volatile var deleted = false - /* Return the size in bytes of this log segment */ - def size: Long = messageSet.sizeInBytes() - /* Return the absolute end offset of this log segment */ - def absoluteEndOffset: Long = start + messageSet.sizeInBytes() - - def updateFirstAppendTime() { - if (firstAppendTime.isEmpty) - firstAppendTime = Some(time.milliseconds) - } - - def append(messages: ByteBufferMessageSet) { - if (messages.sizeInBytes > 0) { - messageSet.append(messages) - updateFirstAppendTime() - } - } - - override def toString() = "(file=" + file + ", start=" + start + ", size=" + size + ")" - - /** - * Truncate this log segment upto absolute offset value. Since the offset specified is absolute, to compute the amount - * of data to be deleted, we have to compute the offset relative to start of the log segment - * @param offset Absolute offset for this partition - */ - def truncateTo(offset: Long) = { - messageSet.truncateTo(offset - start) - } -} - - -/** - * An append-only log for storing messages. + * An append-only log for storing messages. + * + * The log is a sequence of LogSegments, each with a base offset denoting the first message in the segment. + * + * New log segments are created according to a configurable policy that controls the size in bytes or time interval + * for a given segment. + * */ @threadsafe -private[kafka] class Log( val dir: File, val maxLogFileSize: Long, val maxMessageSize: Int, val flushInterval: Int, - val rollIntervalMs: Long, val needRecovery: Boolean, time: Time, - brokerId: Int = 0) extends Logging with KafkaMetricsGroup { +private[kafka] class Log(val dir: File, + val maxLogFileSize: Long, + val maxMessageSize: Int, + val flushInterval: Int, + val rollIntervalMs: Long, + val needsRecovery: Boolean, + val maxIndexSize: Int = (10*1024*1024), + val indexIntervalBytes: Int = 4096, + time: Time = SystemTime, + brokerId: Int = 0) extends Logging with KafkaMetricsGroup { this.logIdent = "[Kafka Log on Broker " + brokerId + "], " import kafka.log.Log._ - + /* A lock that guards all modifications to the log */ private val lock = new Object @@ -144,22 +121,17 @@ private[kafka] class Log( val dir: File, val maxLogFileSize: Long, val maxMessag /* last time it was flushed */ private val lastflushedTime = new AtomicLong(System.currentTimeMillis) - /* The actual segments of the log */ + /* the actual segments of the log */ private[log] val segments: SegmentList[LogSegment] = loadSegments() + + /* Calculate the offset of the next message */ + private var nextOffset: AtomicLong = new AtomicLong(segments.view.last.nextOffset()) - newGauge( - name + "-" + "NumLogSegments", - new Gauge[Int] { - def value() = numberOfSegments - } - ) + newGauge(name + "-" + "NumLogSegments", + new Gauge[Int] { def value() = numberOfSegments }) - newGauge( - name + "-" + "LogEndOffset", - new Gauge[Long] { - def value() = logEndOffset - } - ) + newGauge(name + "-" + "LogEndOffset", + new Gauge[Long] { def value() = logEndOffset }) /* The name of this log */ def name = dir.getName() @@ -170,21 +142,29 @@ private[kafka] class Log( val dir: File, val maxLogFileSize: Long, val maxMessag val logSegments = new ArrayList[LogSegment] val ls = dir.listFiles() if(ls != null) { - for(file <- ls if file.isFile && file.toString.endsWith(FileSuffix)) { + for(file <- ls if file.isFile && file.toString.endsWith(LogFileSuffix)) { if(!file.canRead) throw new IOException("Could not read file " + file) val filename = file.getName() - val start = filename.substring(0, filename.length - FileSuffix.length).toLong - val messageSet = new FileMessageSet(file, false) - logSegments.add(new LogSegment(file, messageSet, start, time)) + val start = filename.substring(0, filename.length - LogFileSuffix.length).toLong + // TODO: we should ideally rebuild any missing index files, instead of erroring out + if(!Log.indexFilename(dir, start).exists) + throw new IllegalStateException("Found log file with no corresponding index file.") + logSegments.add(new LogSegment(dir = dir, + startOffset = start, + mutable = false, + indexIntervalBytes = indexIntervalBytes, + maxIndexSize = maxIndexSize)) } } if(logSegments.size == 0) { // no existing segments, create a new mutable segment - val newFile = new File(dir, nameFromOffset(0)) - val set = new FileMessageSet(newFile, true) - logSegments.add(new LogSegment(newFile, set, 0, time)) + logSegments.add(new LogSegment(dir = dir, + startOffset = 0, + mutable = true, + indexIntervalBytes = indexIntervalBytes, + maxIndexSize = maxIndexSize)) } else { // there is at least one existing segment, validate and recover them/it // sort segments into ascending order for fast searching @@ -195,30 +175,46 @@ private[kafka] class Log( val dir: File, val maxLogFileSize: Long, val maxMessag else 1 } }) - validateSegments(logSegments) //make the final section mutable and run recovery on it if necessary val last = logSegments.remove(logSegments.size - 1) - last.messageSet.close() - info("Loading the last segment " + last.file.getAbsolutePath() + " in mutable mode, recovery " + needRecovery) - val mutable = new LogSegment(last.file, new FileMessageSet(last.file, true, new AtomicBoolean(needRecovery)), last.start, time) - logSegments.add(mutable) + last.close() + val mutableSegment = new LogSegment(dir = dir, + startOffset = last.start, + mutable = true, + indexIntervalBytes = indexIntervalBytes, + maxIndexSize = maxIndexSize) + if(needsRecovery) + recoverSegment(mutableSegment) + logSegments.add(mutableSegment) } new SegmentList(logSegments.toArray(new Array[LogSegment](logSegments.size))) } - + /** - * Check that the ranges and sizes add up, otherwise we have lost some data somewhere + * Run recovery on the given segment. This will rebuild the index from the log file and lop off any invalid bytes from the end of the log. */ - private def validateSegments(segments: ArrayList[LogSegment]) { - lock synchronized { - for(i <- 0 until segments.size - 1) { - val curr = segments.get(i) - val next = segments.get(i+1) - if(curr.start + curr.size != next.start) - throw new KafkaException("The following segments don't validate: " + curr.file.getAbsolutePath() + ", " + next.file.getAbsolutePath()) + private def recoverSegment(segment: LogSegment) { + segment.index.truncateTo(0) + var validBytes = 0 + var lastIndexEntry = 0 + val iter = segment.messageSet.iterator + try { + while(iter.hasNext) { + val entry = iter.next + entry.message.ensureValid() + if(validBytes - lastIndexEntry > indexIntervalBytes) + segment.index.append(entry.offset, validBytes) + validBytes += MessageSet.entrySize(entry.message) } + } catch { + case e: InvalidMessageException => + logger.warn("Found invalid messages in log " + name) } + val truncated = segment.messageSet.sizeInBytes - validBytes + if(truncated > 0) + warn("Truncated " + truncated + " invalid bytes from the log " + name + ".") + segment.messageSet.truncateTo(validBytes) } /** @@ -233,7 +229,7 @@ private[kafka] class Log( val dir: File, val maxLogFileSize: Long, val maxMessag debug("Closing log " + name) lock synchronized { for(seg <- segments.view) { - info("Closing log segment " + seg.file.getAbsolutePath) + debug("Closing log segment " + seg.start) seg.messageSet.close() } } @@ -241,61 +237,106 @@ private[kafka] class Log( val dir: File, val maxLogFileSize: Long, val maxMessag /** * Append this message set to the active segment of the log, rolling over to a fresh segment if necessary. - * Returns the offset at which the messages are written. + * Returns a tuple containing (first_offset, last_offset) for the newly appended of the message set, + * or (-1,-1) if the message set is empty */ - def append(messages: ByteBufferMessageSet): Unit = { - // validate the messages - messages.verifyMessageSize(maxMessageSize) - var numberOfMessages = 0 - for(messageAndOffset <- messages) { - if(!messageAndOffset.message.isValid) - throw new InvalidMessageException() - numberOfMessages += 1; + def append(messages: ByteBufferMessageSet): (Long, Long) = { + // check that all messages are valid and see if we have any compressed messages + var messageCount = 0 + var codec: CompressionCodec = NoCompressionCodec + for(messageAndOffset <- messages.shallowIterator) { + val m = messageAndOffset.message + m.ensureValid() + if(MessageSet.entrySize(m) > maxMessageSize) + throw new MessageSizeTooLargeException("Message size is %d bytes which exceeds the maximum configured message size of %d.".format(MessageSet.entrySize(m), maxMessageSize)) + messageCount += 1; + val messageCodec = m.compressionCodec + if(messageCodec != NoCompressionCodec) + codec = messageCodec } - BrokerTopicStat.getBrokerTopicStat(topicName).messagesInRate.mark(numberOfMessages) - BrokerTopicStat.getBrokerAllTopicStat.messagesInRate.mark(numberOfMessages) - - // truncate the message set's buffer upto validbytes, before appending it to the on-disk log - val validByteBuffer = messages.buffer.duplicate() + // if we have any valid messages, append them to the log + if(messageCount == 0) { + (-1L, -1L) + } else { + BrokerTopicStat.getBrokerTopicStat(topicName).messagesInRate.mark(messageCount) + BrokerTopicStat.getBrokerAllTopicStat.messagesInRate.mark(messageCount) + + // trim any invalid bytes or partial messages before appending it to the on-disk log + var validMessages = trimInvalidBytes(messages) + + // they are valid, insert them in the log + lock synchronized { + try { + val firstOffset = nextOffset.get + + // maybe roll the log + val segment = maybeRoll(segments.view.last) + + // assign offsets to the messages + validMessages = validMessages.assignOffsets(nextOffset, codec) + + trace("Appending message set to " + this.name + ": " + validMessages) + + // now append to the log + segment.append(firstOffset, validMessages) + val lastOffset = nextOffset.get - 1 + + // maybe flush the log and index + maybeFlush(messageCount) + + // return the offset at which the messages were appended + (firstOffset, lastOffset) + } catch { + case e: IOException => + fatal("Halting due to unrecoverable I/O error while handling producer request", e) + Runtime.getRuntime.halt(1) + null + } + } + } + } + + /** + * Trim any invalid bytes from the end of this message set (if there are any) + */ + def trimInvalidBytes(messages: ByteBufferMessageSet): ByteBufferMessageSet = { val messageSetValidBytes = messages.validBytes if(messageSetValidBytes > Int.MaxValue || messageSetValidBytes < 0) throw new InvalidMessageSizeException("Illegal length of message set " + messageSetValidBytes + " Message set cannot be appended to log. Possible causes are corrupted produce requests") - - validByteBuffer.limit(messageSetValidBytes.asInstanceOf[Int]) - val validMessages = new ByteBufferMessageSet(validByteBuffer) - - // they are valid, insert them in the log - lock synchronized { - try { - var segment = segments.view.last - maybeRoll(segment) - segment = segments.view.last - segment.append(validMessages) - maybeFlush(numberOfMessages) - } - catch { - case e: IOException => - fatal("Halting due to unrecoverable I/O error while handling producer request", e) - Runtime.getRuntime.halt(1) - case e2 => throw e2 - } + if(messageSetValidBytes == messages.sizeInBytes) { + messages + } else { + // trim invalid bytes + val validByteBuffer = messages.buffer.duplicate() + validByteBuffer.limit(messageSetValidBytes.asInstanceOf[Int]) + new ByteBufferMessageSet(validByteBuffer) } } /** - * Read from the log file at the given offset + * Read a message set from the log. + * startOffset - The logical offset to begin reading at + * maxLength - The maximum number of bytes to read + * maxOffset - The maximum logical offset to include in the resulting message set */ - def read(offset: Long, length: Int): MessageSet = { - trace("Reading %d bytes from offset %d in log %s of length %s bytes".format(length, offset, name, size)) + def read(startOffset: Long, maxLength: Int, maxOffset: Option[Long] = None): MessageSet = { + trace("Reading %d bytes from offset %d in log %s of length %d bytes".format(maxLength, startOffset, name, size)) val view = segments.view - Log.findRange(view, offset, view.length) match { - case Some(segment) => - if(length <= 0) - MessageSet.Empty - else - segment.messageSet.read((offset - segment.start), length) - case _ => MessageSet.Empty + + // check if the offset is valid and in range + val first = view.head.start + val next = nextOffset.get + if(startOffset == next) + return MessageSet.Empty + else if(startOffset > next || startOffset < first) + throw new OffsetOutOfRangeException("Request for offset %d but we only have log segments in the range %d to %d.".format(startOffset, first, next)) + + // Do the read on the segment with a base offset less than the target offset + // TODO: to handle sparse offsets, we need to skip to the next segment if this read doesn't find anything + Log.findRange(view, startOffset, view.length) match { + case None => throw new OffsetOutOfRangeException("Offset is earlier than the earliest offset") + case Some(segment) => segment.read(startOffset, maxLength, maxOffset) } } @@ -316,7 +357,7 @@ private[kafka] class Log( val dir: File, val maxLogFileSize: Long, val maxMessag else { // If the last segment to be deleted is empty and we roll the log, the new segment will have the same // file name. So simply reuse the last segment and reset the modified time. - view(numToDelete - 1).file.setLastModified(time.milliseconds) + view(numToDelete - 1).messageSet.file.setLastModified(time.milliseconds) numToDelete -=1 } } @@ -330,58 +371,75 @@ private[kafka] class Log( val dir: File, val maxLogFileSize: Long, val maxMessag def size: Long = segments.view.foldLeft(0L)(_ + _.size) /** - * Get the absolute offset of the last message in the log + * Get the offset of the next message that will be appended */ - def logEndOffset: Long = segments.view.last.start + segments.view.last.size + def logEndOffset: Long = nextOffset.get /** * Roll the log over if necessary */ - private def maybeRoll(segment: LogSegment) { + private def maybeRoll(segment: LogSegment): LogSegment = { if ((segment.messageSet.sizeInBytes > maxLogFileSize) || - ((segment.firstAppendTime.isDefined) && (time.milliseconds - segment.firstAppendTime.get > rollIntervalMs))) + ((segment.firstAppendTime.isDefined) && (time.milliseconds - segment.firstAppendTime.get > rollIntervalMs)) || + segment.index.isFull) roll() + else + segment } - private def rollSegment(newOffset: Long) { - val newFile = new File(dir, nameFromOffset(newOffset)) - if (newFile.exists) { - warn("newly rolled logsegment " + newFile.getName + " already exists; deleting it first") - newFile.delete() + /** + * Create a new segment and make it active, and return it + */ + def roll(): LogSegment = { + lock synchronized { + flush() + rollToOffset(logEndOffset) } - debug("Rolling log '" + name + "' to " + newFile.getName()) - segments.append(new LogSegment(newFile, new FileMessageSet(newFile, true), newOffset, time)) } - + /** - * Create a new segment and make it active + * Roll the log over to the given new offset value */ - def roll() { - lock synchronized { - flush - rollSegment(logEndOffset) + private def rollToOffset(newOffset: Long): LogSegment = { + val logFile = logFilename(dir, newOffset) + val indexFile = indexFilename(dir, newOffset) + for(file <- List(logFile, indexFile); if file.exists) { + warn("Newly rolled segment file " + file.getName + " already exists; deleting it first") + file.delete() } + debug("Rolling log '" + name + "' to " + logFile.getName + " and " + indexFile.getName) + segments.view.lastOption match { + case Some(segment) => segment.index.makeReadOnly() + case None => + } + val segment = new LogSegment(dir, + startOffset = newOffset, + mutable = true, + indexIntervalBytes = indexIntervalBytes, + maxIndexSize = maxIndexSize) + segments.append(segment) + segment } /** * Flush the log if necessary */ private def maybeFlush(numberOfMessages : Int) { - if(unflushed.addAndGet(numberOfMessages) >= flushInterval) { + if(unflushed.addAndGet(numberOfMessages) >= flushInterval) flush() - } } /** * Flush this log file to the physical disk */ def flush() : Unit = { - if (unflushed.get == 0) return + if (unflushed.get == 0) + return lock synchronized { debug("Flushing log '" + name + "' last flushed: " + getLastFlushedTime + " current time: " + time.milliseconds) - segments.view.last.messageSet.flush() + segments.view.last.flush() unflushed.set(0) lastflushedTime.set(time.milliseconds) } @@ -390,15 +448,15 @@ private[kafka] class Log( val dir: File, val maxLogFileSize: Long, val maxMessag def getOffsetsBefore(timestamp: Long, maxNumOffsets: Int): Seq[Long] = { val segsArray = segments.view var offsetTimeArray: Array[(Long, Long)] = null - if (segsArray.last.size > 0) + if(segsArray.last.size > 0) offsetTimeArray = new Array[(Long, Long)](segsArray.length + 1) else offsetTimeArray = new Array[(Long, Long)](segsArray.length) - for (i <- 0 until segsArray.length) - offsetTimeArray(i) = (segsArray(i).start, segsArray(i).file.lastModified) - if (segsArray.last.size > 0) - offsetTimeArray(segsArray.length) = (segsArray.last.start + segsArray.last.messageSet.sizeInBytes(), time.milliseconds) + for(i <- 0 until segsArray.length) + offsetTimeArray(i) = (segsArray(i).start, segsArray(i).messageSet.file.lastModified) + if(segsArray.last.size > 0) + offsetTimeArray(segsArray.length) = (logEndOffset, time.milliseconds) var startIndex = -1 timestamp match { @@ -420,7 +478,7 @@ private[kafka] class Log( val dir: File, val maxLogFileSize: Long, val maxMessag val retSize = maxNumOffsets.min(startIndex + 1) val ret = new Array[Long](retSize) - for (j <- 0 until retSize) { + for(j <- 0 until retSize) { ret(j) = offsetTimeArray(startIndex)._1 startIndex -= 1 } @@ -428,63 +486,81 @@ private[kafka] class Log( val dir: File, val maxLogFileSize: Long, val maxMessag ret.toSeq.sortBy(- _) } - /** - * Truncate all segments in the log and start a new segment on a new offset - */ - def truncateAndStartWithNewOffset(newOffset: Long) { - lock synchronized { - val deletedSegments = segments.trunc(segments.view.size) - rollSegment(newOffset) - deleteSegments(deletedSegments) - } + def delete(): Unit = { + deleteSegments(segments.contents.get()) + Utils.rm(dir) } + /* Attempts to delete all provided segments from a log and returns how many it was able to */ def deleteSegments(segments: Seq[LogSegment]): Int = { var total = 0 for(segment <- segments) { - info("Deleting log segment " + segment.file.getName() + " from " + name) - swallow(segment.messageSet.close()) - if(!segment.file.delete()) { - warn("Delete failed.") + info("Deleting log segment " + segment.start + " from " + name) + if(!segment.messageSet.delete() || !segment.index.delete()) { + warn("Delete of log segment " + segment.start + " failed.") } else { total += 1 } } total } - + def truncateTo(targetOffset: Long) { + if(targetOffset < 0) + throw new IllegalArgumentException("Cannot truncate to a negative offset (%d).".format(targetOffset)) lock synchronized { val segmentsToBeDeleted = segments.view.filter(segment => segment.start > targetOffset) val viewSize = segments.view.size val numSegmentsDeleted = deleteSegments(segmentsToBeDeleted) - /* We should not hit this error because segments.view is locked in markedDeletedWhile() */ if(numSegmentsDeleted != segmentsToBeDeleted.size) - error("Failed to delete some segments during log recovery during truncateTo(" + targetOffset +")") - + error("Failed to delete some segments when attempting to truncate to offset " + targetOffset +")") if (numSegmentsDeleted == viewSize) { segments.trunc(segments.view.size) - rollSegment(targetOffset) + rollToOffset(targetOffset) + this.nextOffset.set(targetOffset) } else { - // find the log segment that has this hw - val segmentToBeTruncated = - segments.view.find(segment => targetOffset >= segment.start && targetOffset < segment.absoluteEndOffset) - segmentToBeTruncated match { - case Some(segment) => - val truncatedSegmentIndex = segments.view.indexOf(segment) - segments.truncLast(truncatedSegmentIndex) - segment.truncateTo(targetOffset) - info("Truncated log segment %s to target offset %d".format(segment.file.getAbsolutePath, targetOffset)) - case None => - if(targetOffset > segments.view.last.absoluteEndOffset) - error("Target offset %d cannot be greater than the last message offset %d in the log %s". - format(targetOffset, segments.view.last.absoluteEndOffset, segments.view.last.file.getAbsolutePath)) + if(targetOffset > logEndOffset) { + error("Target offset %d cannot be greater than the last message offset %d in the log %s". + format(targetOffset, logEndOffset, segments.view.last.messageSet.file.getAbsolutePath)) + } else { + // find the log segment that has this hw + val segmentToBeTruncated = findRange(segments.view, targetOffset) + segmentToBeTruncated match { + case Some(segment) => + val truncatedSegmentIndex = segments.view.indexOf(segment) + segments.truncLast(truncatedSegmentIndex) + segment.truncateTo(targetOffset) + this.nextOffset.set(targetOffset) + info("Truncated log segment %s to target offset %d".format(segments.view.last.messageSet.file.getAbsolutePath, targetOffset)) + case None => // nothing to do + } } } } } + + /** + * Truncate all segments in the log and start a new segment on a new offset + */ + def truncateAndStartWithNewOffset(newOffset: Long) { + lock synchronized { + val deletedSegments = segments.trunc(segments.view.size) + val logFile = logFilename(dir, newOffset) + val indexFile = indexFilename(dir, newOffset) + debug("Truncate and start log '" + name + "' to " + newOffset) + val log = new FileMessageSet(file = logFile, mutable = true) + val index = new OffsetIndex(file = indexFile, baseOffset = newOffset, mutable = true, maxIndexSize = maxIndexSize) + segments.append(new LogSegment(dir, + newOffset, + mutable = true, + indexIntervalBytes = indexIntervalBytes, + maxIndexSize = maxIndexSize)) + deleteSegments(deletedSegments) + this.nextOffset.set(newOffset) + } + } def topicName():String = { name.substring(0, name.lastIndexOf("-")) diff --git core/src/main/scala/kafka/log/LogManager.scala core/src/main/scala/kafka/log/LogManager.scala index 2315a2a..78ce9b6 100644 --- core/src/main/scala/kafka/log/LogManager.scala +++ core/src/main/scala/kafka/log/LogManager.scala @@ -46,7 +46,7 @@ private[kafka] class LogManager(val config: KafkaConfig, private val logRetentionSizeMap = config.logRetentionSizeMap private val logRetentionMsMap = config.logRetentionHoursMap.map(e => (e._1, e._2 * 60 * 60 * 1000L)) // convert hours to ms private val logRollMsMap = config.logRollHoursMap.map(e => (e._1, e._2 * 60 * 60 * 1000L)) - this.logIdent = "[Log Manager on Broker " + config.brokerId + "], " + this.logIdent = "[Log Manager on Broker " + config.brokerId + "] " /* Initialize a log for each subdirectory of the main log directory */ private val logs = new Pool[String, Pool[Int, Log]]() @@ -69,7 +69,16 @@ private[kafka] class LogManager(val config: KafkaConfig, val topic = Utils.getTopicPartition(dir.getName)._1 val rollIntervalMs = logRollMsMap.get(topic).getOrElse(this.logRollDefaultIntervalMs) val maxLogFileSize = logFileSizeMap.get(topic).getOrElse(config.logFileSize) - val log = new Log(dir, maxLogFileSize, config.maxMessageSize, flushInterval, rollIntervalMs, needRecovery, time, config.brokerId) + val log = new Log(dir, + maxLogFileSize, + config.maxMessageSize, + flushInterval, + rollIntervalMs, + needRecovery, + config.logIndexMaxSizeBytes, + config.logIndexIntervalBytes, + time, + config.brokerId) val topicPartition = Utils.getTopicPartition(dir.getName) logs.putIfNotExists(topicPartition._1, new Pool[Int, Log]()) val parts = logs.get(topicPartition._1) @@ -88,7 +97,7 @@ private[kafka] class LogManager(val config: KafkaConfig, scheduler.scheduleWithRate(cleanupLogs, "kafka-logcleaner-", 60 * 1000, logCleanupIntervalMs, false) info("Starting log flusher every " + config.flushSchedulerThreadRate + " ms with the following overrides " + logFlushIntervals) - scheduler.scheduleWithRate(flushAllLogs, "kafka-logflusher-", + scheduler.scheduleWithRate(flushDirtyLogs, "kafka-logflusher-", config.flushSchedulerThreadRate, config.flushSchedulerThreadRate, false) } } @@ -103,7 +112,7 @@ private[kafka] class LogManager(val config: KafkaConfig, d.mkdirs() val rollIntervalMs = logRollMsMap.get(topic).getOrElse(this.logRollDefaultIntervalMs) val maxLogFileSize = logFileSizeMap.get(topic).getOrElse(config.logFileSize) - new Log(d, maxLogFileSize, config.maxMessageSize, flushInterval, rollIntervalMs, false, time, config.brokerId) + new Log(d, maxLogFileSize, config.maxMessageSize, flushInterval, rollIntervalMs, needsRecovery = false, config.logIndexMaxSizeBytes, config.logIndexIntervalBytes, time, config.brokerId) } } @@ -162,7 +171,7 @@ private[kafka] class LogManager(val config: KafkaConfig, val startMs = time.milliseconds val topic = Utils.getTopicPartition(log.name)._1 val logCleanupThresholdMs = logRetentionMsMap.get(topic).getOrElse(this.logCleanupDefaultAgeMs) - val toBeDeleted = log.markDeletedWhile(startMs - _.file.lastModified > logCleanupThresholdMs) + val toBeDeleted = log.markDeletedWhile(startMs - _.messageSet.file.lastModified > logCleanupThresholdMs) val total = log.deleteSegments(toBeDeleted) total } @@ -208,9 +217,9 @@ private[kafka] class LogManager(val config: KafkaConfig, * Close all the logs */ def shutdown() { - info("shut down") + debug("Shutting down.") allLogs.foreach(_.close()) - info("shutted down completedly") + debug("Shutdown complete.") } /** @@ -218,21 +227,22 @@ private[kafka] class LogManager(val config: KafkaConfig, */ def allLogs() = logs.values.flatMap(_.values) - private def flushAllLogs() = { - debug("Flushing the high watermark of all logs") - for (log <- allLogs) - { - try{ + /** + * Flush any log which has exceeded its flush interval and has unwritten messages. + */ + private def flushDirtyLogs() = { + debug("Checking for dirty logs to flush...") + for (log <- allLogs) { + try { val timeSinceLastFlush = System.currentTimeMillis - log.getLastFlushedTime var logFlushInterval = config.defaultFlushIntervalMs if(logFlushIntervals.contains(log.topicName)) logFlushInterval = logFlushIntervals(log.topicName) debug(log.topicName + " flush interval " + logFlushInterval + - " last flushed " + log.getLastFlushedTime + " timesincelastFlush: " + timeSinceLastFlush) + " last flushed " + log.getLastFlushedTime + " time since last flush: " + timeSinceLastFlush) if(timeSinceLastFlush >= logFlushInterval) log.flush - } - catch { + } catch { case e => error("Error flushing topic " + log.topicName, e) e match { diff --git core/src/main/scala/kafka/log/LogSegment.scala core/src/main/scala/kafka/log/LogSegment.scala new file mode 100644 index 0000000..d9e884c --- /dev/null +++ core/src/main/scala/kafka/log/LogSegment.scala @@ -0,0 +1,150 @@ +package kafka.log + +import scala.math._ +import java.io.File +import kafka.common._ +import kafka.message._ +import kafka.utils.{Utils, Range, Time, SystemTime, nonthreadsafe} + +/** + * A segment of the log. Each segment has two components: a log and an index. The log is a FileMessageSet containing + * the actual messages. The index is an OffsetIndex that maps from logical offsets to physical file positions. Each + * segment has a base offset which is an offset <= the least offset of any message in this segment and > any offset in + * any previous segment. + * + * A segment with a base offset of [base_offset] would be stored in two files, a [base_offset].index and a [base_offset].log file. + */ +@nonthreadsafe +class LogSegment(val messageSet: FileMessageSet, + val index: OffsetIndex, + val start: Long, + val indexIntervalBytes: Int, + time: Time) extends Range { + + var firstAppendTime: Option[Long] = None + + /* the number of bytes since we last added an entry in the offset index */ + var bytesSinceLastIndexEntry = 0 + + @volatile var deleted = false + + def this(dir: File, startOffset: Long, mutable: Boolean, indexIntervalBytes: Int, maxIndexSize: Int) = + this(new FileMessageSet(file = Log.logFilename(dir, startOffset), mutable = mutable), + new OffsetIndex(file = Log.indexFilename(dir, startOffset), baseOffset = startOffset, mutable = mutable, maxIndexSize = maxIndexSize), + startOffset, + indexIntervalBytes, + SystemTime) + + /* Return the size in bytes of this log segment */ + def size: Long = messageSet.sizeInBytes() + + def updateFirstAppendTime() { + if (firstAppendTime.isEmpty) + firstAppendTime = Some(time.milliseconds) + } + + /** + * Append the given messages starting with the given offset. Add + * an entry to the index if needed. + * + * It is assumed this method is being called from within a lock + */ + def append(offset: Long, messages: ByteBufferMessageSet) { + if (messages.sizeInBytes > 0) { + // append an entry to the index (if needed) + if(bytesSinceLastIndexEntry > indexIntervalBytes) { + index.append(offset, messageSet.sizeInBytes().toInt) + this.bytesSinceLastIndexEntry = 0 + } + // append the messages + messageSet.append(messages) + updateFirstAppendTime() + this.bytesSinceLastIndexEntry += messages.sizeInBytes.toInt + } + } + + /** + * Find the physical file position for the least offset >= the given offset. If no offset is found + * that meets this criteria before the end of the log, return null. + */ + def translateOffset(offset: Long): OffsetPosition = { + val mapping = index.lookup(offset) + messageSet.searchFor(offset, mapping.position) + } + + /** + * Read a message set from this segment beginning with the first offset + * greater than or equal to the startOffset. The message set will include + * no more than maxSize bytes and will end before maxOffset if a maxOffset is specified. + */ + def read(startOffset: Long, maxSize: Int, maxOffset: Option[Long]): MessageSet = { + if(maxSize <= 0) + return MessageSet.Empty + + val startPosition = translateOffset(startOffset) + + // if the start position is already off the end of the log, return MessageSet.Empty + if(startPosition == null) + return MessageSet.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 use the max size they gave unmolested + maxSize + case Some(offset) => { + // there is a max offset, translate it to a file position and use that to calculate the max read size + val mapping = translateOffset(offset) + val endPosition = + if(mapping == null) + messageSet.sizeInBytes().toInt // the max offset is off the end of the log, use the end of the file + else + mapping.position + min(endPosition - startPosition.position, maxSize) + } + } + messageSet.read(startPosition.position, length) + } + + override def toString() = "LogSegment(start=" + start + ", size=" + size + ")" + + /** + * Truncate off all index and log entries with offsets greater than or equal to the current offset. + */ + def truncateTo(offset: Long) { + val mapping = translateOffset(offset) + if(mapping == null) + return + index.truncateTo(offset) + messageSet.truncateTo(mapping.position) + } + + /** + * Calculate the offset that would be used for the next message to be append to this segment. + * Not that this is expensive. + */ + def nextOffset(): Long = { + val ms = read(index.lastOffset, messageSet.sizeInBytes.toInt, None) + ms.lastOption match { + case None => start + case Some(last) => last.nextOffset + } + } + + /** + * Flush this log segment to disk + */ + def flush() { + messageSet.flush() + index.flush() + } + + /** + * Close this log segment + */ + def close() { + Utils.swallow(index.close) + Utils.swallow(messageSet.close) + } +} \ No newline at end of file diff --git core/src/main/scala/kafka/log/OffsetIndex.scala core/src/main/scala/kafka/log/OffsetIndex.scala new file mode 100644 index 0000000..93488a7 --- /dev/null +++ core/src/main/scala/kafka/log/OffsetIndex.scala @@ -0,0 +1,249 @@ +/** + * 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 scala.math._ +import java.io._ +import java.nio._ +import java.nio.channels._ +import java.util.concurrent.atomic._ +import kafka.utils._ + +/** + * An index that maps logical offsets to physical file locations for a particular log segment. This index may be sparse: + * that is it may not hold an entry for all messages in the log. + * + * The index is stored in a file that is pre-allocated to hold a fixed maximum number of 8-byte entries. + * + * The index supports lookups against a memory-map of this file. These lookups are done using a simple binary search variant + * to locate the offset/location pair for the greatest offset less than or equal to the target offset. + * + * Index files can be opened in two ways: either as an empty, mutable index that allows appends or + * an immutable read-only index file that has previously been populated. The makeReadOnly method will turn a mutable file into an + * immutable one and truncate off any extra bytes. This is done when the index file is rolled over. + * + * No attempt is made to checksum the contents of this file, in the event of a crash it is rebuilt. + * + * The file format is a series of entries. The physical format is a 4 byte "relative" offset and a 4 byte file location for the + * message with that offset. The offset stored is relative to the base offset of the index file. So, for example, + * if the base offset was 50, then the offset 55 would be stored as 5. Using relative offsets in this way let's us use + * only 4 bytes for the offset. + * + * The frequency of entries is up to the user of this class. + * + * All external APIs translate from relative offsets to full offsets, so users of this class do not interact with the internal + * storage format. + */ +class OffsetIndex(val file: File, val baseOffset: Long, var mutable: Boolean, maxIndexSize: Int = -1) extends Logging { + + /* the memory mapping */ + private var mmap: MappedByteBuffer = + { + val raf = new RandomAccessFile(file, "rw") + try { + if(mutable) { + /* if mutable create and memory map a new sparse file */ + if(maxIndexSize < 8) + throw new IllegalArgumentException("Invalid max index size: " + maxIndexSize) + file.createNewFile() + /* round size to the nearest multiple of 8 */ + val size = 8*(maxIndexSize/8) + raf.setLength(size) + val idx = raf.getChannel.map(FileChannel.MapMode.READ_WRITE, 0, raf.length()) + idx.position(0).asInstanceOf[MappedByteBuffer] + } else { + /* if not mutable, just mmap what they gave us */ + val len = raf.length() + if(len < 0 || len % 8 != 0) + throw new IllegalStateException("Index file " + file.getName + " is corrupt, found " + len + + " bytes which is not positive or not a multiple of 8.") + raf.getChannel.map(FileChannel.MapMode.READ_ONLY, 0, len) + } + } finally { + Utils.swallow(raf.close()) + } + } + + /* the maximum number of entries this index can hold */ + val maxEntries = mmap.limit / 8 + + /* the number of entries in the index */ + private var size = if(mutable) new AtomicInteger(0) else new AtomicInteger(mmap.limit / 8) + + /* the last offset in the index */ + var lastOffset = if(mutable) baseOffset else readLastOffset() + + /** + * The last logical offset written to the index + */ + private def readLastOffset(): Long = { + val offset = + size.get match { + case 0 => 0 + case s => logical(this.mmap, s-1) + } + baseOffset + offset + } + + /** + * Find the largest offset less than or equal to the given targetOffset + * and return a pair holding this logical offset and it's corresponding physical file position. + * If the target offset is smaller than the least entry in the index (or the index is empty), + * the pair (baseOffset, 0) is returned. + */ + def lookup(targetOffset: Long): OffsetPosition = { + if(entries == 0) + return OffsetPosition(baseOffset, 0) + val idx = mmap.duplicate + val slot = indexSlotFor(idx, targetOffset) + if(slot == -1) + OffsetPosition(baseOffset, 0) + else + OffsetPosition(baseOffset + logical(idx, slot), physical(idx, slot)) + } + + /** + * Find the slot in which the largest offset less than or equal to the given + * target offset is stored. + * Return -1 if the least entry in the index is larger than the target offset + */ + private def indexSlotFor(idx: ByteBuffer, targetOffset: Long): Int = { + // we only store the difference from the baseoffset so calculate that + val relativeOffset = targetOffset - baseOffset + + // check if the target offset is smaller than the least offset + if(logical(idx, 0) > relativeOffset) + return -1 + + // binary search for the entry + var lo = 0 + var hi = entries-1 + while(lo < hi) { + val mid = ceil((hi + lo) / 2.0).toInt + val found = logical(idx, mid) + if(found == relativeOffset) + return mid + else if(found < relativeOffset) + lo = mid + else + hi = mid - 1 + } + return lo + } + + /* return the nth logical offset relative to the base offset */ + private def logical(buffer: ByteBuffer, n: Int): Int = buffer.getInt(n * 8) + + /* return the nth physical offset */ + private def physical(buffer: ByteBuffer, n: Int): Int = buffer.getInt(n * 8 + 4) + + /** + * Get the nth offset mapping from the index + */ + def entry(n: Int): OffsetPosition = { + if(n >= entries) + throw new IllegalArgumentException("Attempt to fetch the %dth entry from an index of size %d.".format(n, entries)) + val idx = mmap.duplicate + OffsetPosition(logical(idx, n), physical(idx, n)) + } + + /** + * Append entry for the given offset/location pair to the index. This entry must have a larger offset than all subsequent entries. + */ + def append(logicalOffset: Long, position: Int) { + this synchronized { + if(!mutable) + throw new IllegalStateException("Attempt to append to an immutable offset index " + file.getName) + if(isFull) + throw new IllegalStateException("Attempt to append to a full index (size = " + size + ").") + if(size.get > 0 && logicalOffset <= lastOffset) + throw new IllegalArgumentException("Attempt to append an offset (" + logicalOffset + ") no larger than the last offset appended (" + lastOffset + ").") + debug("Adding index entry %d => %d to %s.".format(logicalOffset, position, file.getName)) + this.mmap.putInt((logicalOffset - baseOffset).toInt) + this.mmap.putInt(position) + this.size.incrementAndGet() + this.lastOffset = logicalOffset + } + } + + /** + * True iff there are no more slots available in this index + */ + def isFull: Boolean = entries >= this.maxEntries + + /** + * Remove all entries from the index which have an offset greater than or equal to the given offset + */ + def truncateTo(offset: Long) { + this synchronized { + val idx = mmap.duplicate + val slot = indexSlotFor(idx, offset) + if(slot > 0) { + val found = logical(idx, slot) + val newEntries = if(found == offset) slot else slot + 1 + this.size.set(newEntries) + mmap.position(this.size.get * 8) + this.lastOffset = readLastOffset + } + } + } + + /** + * Make this segment read-only, flush any unsaved changes, and truncate any excess bytes + */ + def makeReadOnly() { + this synchronized { + mutable = false + flush() + val raf = new RandomAccessFile(file, "rws") + try { + val newLength = entries * 8 + raf.setLength(newLength) + this.mmap = raf.getChannel().map(FileChannel.MapMode.READ_ONLY, 0, newLength) + } finally { + Utils.swallow(raf.close()) + } + } + } + + /** + * Flush the data in the index to disk + */ + def flush() { + this synchronized { + mmap.force() + } + } + + /** + * Delete this index file + */ + def delete(): Boolean = { + this.file.delete() + } + + /** The number of entries in this index */ + def entries() = size.get + + /** Close the index */ + def close() { + if(mutable) + makeReadOnly() + } + +} \ No newline at end of file diff --git core/src/main/scala/kafka/log/OffsetPosition.scala core/src/main/scala/kafka/log/OffsetPosition.scala new file mode 100644 index 0000000..6cefde4 --- /dev/null +++ core/src/main/scala/kafka/log/OffsetPosition.scala @@ -0,0 +1,25 @@ +/** + * 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 + +/** + * The mapping between a logical log offset and the physical position + * in some log file of the beginning of the message set entry with the + * given offset. + */ +case class OffsetPosition(val offset: Long, val position: Int) \ No newline at end of file diff --git core/src/main/scala/kafka/log/SegmentList.scala core/src/main/scala/kafka/log/SegmentList.scala index 872a04f..d7c744a 100644 --- core/src/main/scala/kafka/log/SegmentList.scala +++ core/src/main/scala/kafka/log/SegmentList.scala @@ -72,8 +72,8 @@ private[log] class SegmentList[T](seq: Seq[T])(implicit m: ClassManifest[T]) { * Delete the items from position (newEnd + 1) until end of list */ def truncLast(newEnd: Int): Seq[T] = { - if (newEnd < 0 || newEnd > contents.get().length-1) - throw new KafkaException("End index must be positive and less than segment list size."); + if (newEnd < 0 || newEnd >= contents.get().length) + throw new KafkaException("Attempt to truncate segment list of length %d to %d.".format(contents.get().size, newEnd)); var deleted: Array[T] = null val curr = contents.get() if (curr.length > 0) { diff --git core/src/main/scala/kafka/message/ByteBufferMessageSet.scala core/src/main/scala/kafka/message/ByteBufferMessageSet.scala index cca60bb..35d6cbd 100644 --- core/src/main/scala/kafka/message/ByteBufferMessageSet.scala +++ core/src/main/scala/kafka/message/ByteBufferMessageSet.scala @@ -17,12 +17,72 @@ package kafka.message +import scala.reflect.BeanProperty import kafka.utils.Logging import java.nio.ByteBuffer import java.nio.channels._ +import java.io.{InputStream, ByteArrayInputStream, ByteArrayOutputStream, DataOutputStream, IOException} +import java.util.zip._ +import java.util.concurrent.atomic.AtomicLong import kafka.utils.IteratorTemplate import kafka.common.{MessageSizeTooLargeException, InvalidMessageSizeException} +object ByteBufferMessageSet { + + private def create(offsetCounter: AtomicLong, compressionCodec: CompressionCodec, messages: Message*): ByteBuffer = { + if(messages.size == 0) { + return MessageSet.Empty.buffer + } else if(compressionCodec == NoCompressionCodec) { + val buffer = ByteBuffer.allocate(MessageSet.messageSetSize(messages)) + for(message <- messages) + writeMessage(buffer, message, offsetCounter.getAndIncrement) + buffer.rewind() + buffer + } else { + val byteArrayStream = new ByteArrayOutputStream(MessageSet.messageSetSize(messages)) + val output = new DataOutputStream(CompressionFactory(compressionCodec, byteArrayStream)) + var offset = -1L + for(message <- messages) { + offset = offsetCounter.getAndIncrement + output.writeLong(offset) + output.writeInt(message.size) + output.write(message.buffer.array, message.buffer.arrayOffset, message.buffer.limit) + } + output.close() + + val bytes = byteArrayStream.toByteArray + val message = new Message(bytes, compressionCodec) + val buffer = ByteBuffer.allocate(message.size + MessageSet.LogOverhead) + writeMessage(buffer, message, offset) + buffer.rewind() + buffer + } + } + + def decompress(message: Message): ByteBufferMessageSet = { + val outputStream: ByteArrayOutputStream = new ByteArrayOutputStream + val inputStream: InputStream = new ByteBufferBackedInputStream(message.payload) + val intermediateBuffer = new Array[Byte](1024) + val compressed = CompressionFactory(message.compressionCodec, inputStream) + Stream.continually(compressed.read(intermediateBuffer)).takeWhile(_ > 0).foreach { dataRead => + outputStream.write(intermediateBuffer, 0, dataRead) + } + compressed.close() + + val outputBuffer = ByteBuffer.allocate(outputStream.size) + outputBuffer.put(outputStream.toByteArray) + outputBuffer.rewind + new ByteBufferMessageSet(outputBuffer) + } + + private def writeMessage(buffer: ByteBuffer, message: Message, offset: Long) { + buffer.putLong(offset) + buffer.putInt(message.size) + buffer.put(message.buffer) + message.buffer.rewind() + } +} + /** * A sequence of messages stored in a byte buffer * @@ -33,31 +93,33 @@ import kafka.common.{MessageSizeTooLargeException, InvalidMessageSizeException} * Option 2: Give it a list of messages along with instructions relating to serialization format. Producers will use this method. * */ -class ByteBufferMessageSet(val buffer: ByteBuffer, val initialOffset: Long = 0L) extends MessageSet with Logging { +class ByteBufferMessageSet(@BeanProperty val buffer: ByteBuffer) extends MessageSet with Logging { private var shallowValidByteCount = -1L if(sizeInBytes > Int.MaxValue) throw new InvalidMessageSizeException("Message set cannot be larger than " + Int.MaxValue) def this(compressionCodec: CompressionCodec, messages: Message*) { - this(MessageSet.createByteBuffer(compressionCodec, messages:_*), 0L) + this(ByteBufferMessageSet.create(new AtomicLong(0), compressionCodec, messages:_*)) + } + + def this(compressionCodec: CompressionCodec, offsetCounter: AtomicLong, messages: Message*) { + this(ByteBufferMessageSet.create(offsetCounter, compressionCodec, messages:_*)) } def this(messages: Message*) { - this(NoCompressionCodec, messages: _*) + this(NoCompressionCodec, new AtomicLong(0), messages: _*) } - def validBytes: Long = shallowValidBytes - private def shallowValidBytes: Long = { if(shallowValidByteCount < 0) { + this.shallowValidByteCount = 0 val iter = this.internalIterator(true) while(iter.hasNext) { val messageAndOffset = iter.next - shallowValidByteCount = messageAndOffset.offset + shallowValidByteCount += MessageSet.entrySize(messageAndOffset.message) } } - if(shallowValidByteCount < initialOffset) 0 - else (shallowValidByteCount - initialOffset) + shallowValidByteCount } /** Write the messages in this set to the given channel */ @@ -74,71 +136,44 @@ class ByteBufferMessageSet(val buffer: ByteBuffer, val initialOffset: Long = 0L) /** iterator over compressed messages without decompressing */ def shallowIterator: Iterator[MessageAndOffset] = internalIterator(true) - def verifyMessageSize(maxMessageSize: Int){ - var shallowIter = internalIterator(true) - while(shallowIter.hasNext){ - var messageAndOffset = shallowIter.next - val payloadSize = messageAndOffset.message.payloadSize - if ( payloadSize > maxMessageSize) - throw new MessageSizeTooLargeException("payload size of " + payloadSize + " larger than " + maxMessageSize) - } - } - - /** When flag isShallow is set to be true, we do a shallow iteration: just traverse the first level of messages. This is used in verifyMessageSize() function **/ + /** 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): Iterator[MessageAndOffset] = { new IteratorTemplate[MessageAndOffset] { var topIter = buffer.slice() - var currValidBytes = initialOffset - var innerIter:Iterator[MessageAndOffset] = null - var lastMessageSize = 0L + var innerIter: Iterator[MessageAndOffset] = null - def innerDone():Boolean = (innerIter==null || !innerIter.hasNext) + def innerDone():Boolean = (innerIter == null || !innerIter.hasNext) def makeNextOuter: MessageAndOffset = { - if (topIter.remaining < 4) { + // 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() - lastMessageSize = size - - trace("Remaining bytes in iterator = " + topIter.remaining) - trace("size of data = " + size) - - if(size < 0 || topIter.remaining < size) { - if (currValidBytes == initialOffset || size < 0) - throw new InvalidMessageSizeException("invalid message size: " + size + " only received bytes: " + - topIter.remaining + " at " + currValidBytes + "( possible causes (1) a single message larger than " + - "the fetch size; (2) log corruption )") + if(size < 0) + throw new InvalidMessageException("Message found with corrupt size (" + size + ")") + + // 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(!newMessage.isValid) - throw new InvalidMessageException("message is invalid, compression codec: " + newMessage.compressionCodec - + " size: " + size + " curr offset: " + currValidBytes + " init offset: " + initialOffset) - if(isShallow){ - currValidBytes += 4 + size - trace("shallow iterator currValidBytes = " + currValidBytes) - new MessageAndOffset(newMessage, currValidBytes) - } - else{ + if(isShallow) { + new MessageAndOffset(newMessage, offset) + } else { newMessage.compressionCodec match { case NoCompressionCodec => - debug("Message is uncompressed. Valid byte count = %d".format(currValidBytes)) innerIter = null - currValidBytes += 4 + size - trace("currValidBytes = " + currValidBytes) - new MessageAndOffset(newMessage, currValidBytes) + new MessageAndOffset(newMessage, offset) case _ => - debug("Message is compressed. Valid byte count = %d".format(currValidBytes)) - innerIter = CompressionUtils.decompress(newMessage).internalIterator() - if (!innerIter.hasNext) { - currValidBytes += 4 + lastMessageSize + innerIter = ByteBufferMessageSet.decompress(newMessage).internalIterator() + if(!innerIter.hasNext) innerIter = null - } makeNext() } } @@ -147,50 +182,62 @@ class ByteBufferMessageSet(val buffer: ByteBuffer, val initialOffset: Long = 0L) override def makeNext(): MessageAndOffset = { if(isShallow){ makeNextOuter - } - else{ - val isInnerDone = innerDone() - debug("makeNext() in internalIterator: innerDone = " + isInnerDone) - isInnerDone match { - case true => makeNextOuter - case false => { - val messageAndOffset = innerIter.next - if (!innerIter.hasNext) - currValidBytes += 4 + lastMessageSize - new MessageAndOffset(messageAndOffset.message, currValidBytes) - } - } + } else { + if(innerDone()) + makeNextOuter + else + innerIter.next } } + } } - - def sizeInBytes: Long = buffer.limit - override def toString: String = { - val builder = new StringBuilder() - builder.append("ByteBufferMessageSet(") - for(message <- this) { - builder.append(message) - builder.append(", ") + /** + * Update the offsets for this message set. This method attempts to do an in-place conversion + * if there is no compression, but otherwise recopies the messages + */ + private[kafka] def assignOffsets(offsetCounter: AtomicLong, codec: CompressionCodec): ByteBufferMessageSet = { + if(codec == NoCompressionCodec) { + // do an in-place conversion + var position = 0 + buffer.mark() + while(position < sizeInBytes - MessageSet.LogOverhead) { + buffer.position(position) + buffer.putLong(offsetCounter.getAndIncrement()) + position += MessageSet.LogOverhead + buffer.getInt() + } + buffer.reset() + this + } else { + // messages are compressed, crack open the messageset and recompress with correct offset + val messages = this.internalIterator(isShallow = false).map(_.message) + new ByteBufferMessageSet(compressionCodec = codec, offsetCounter = offsetCounter, messages = messages.toBuffer:_*) } - builder.append(")") - builder.toString } + + + /** + * The total number of bytes in this message set, including any partial trailing messages + */ + def sizeInBytes: Long = buffer.limit + + /** + * The total number of bytes in this message set not including any partial, trailing messages + */ + def validBytes: Long = shallowValidBytes + /** + * Two message sets are equal if their respective byte buffers are equal + */ override def equals(other: Any): Boolean = { other match { - case that: ByteBufferMessageSet => - buffer.equals(that.buffer) && initialOffset == that.initialOffset + case that: ByteBufferMessageSet => + buffer.equals(that.buffer) case _ => false } } - override def hashCode: Int = { - var hash = 17 - hash = hash * 31 + buffer.hashCode - hash = hash * 31 + initialOffset.hashCode - hash - } + override def hashCode: Int = buffer.hashCode } diff --git core/src/main/scala/kafka/message/CompressionFactory.scala core/src/main/scala/kafka/message/CompressionFactory.scala new file mode 100644 index 0000000..53cfdcf --- /dev/null +++ core/src/main/scala/kafka/message/CompressionFactory.scala @@ -0,0 +1,55 @@ +/** + * 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.message + +import java.io.OutputStream +import java.io.ByteArrayOutputStream +import java.util.zip.GZIPOutputStream +import java.util.zip.GZIPInputStream +import java.io.IOException +import java.io.InputStream +import java.nio.ByteBuffer +import java.util.concurrent.atomic.AtomicLong +import kafka.utils._ + +object CompressionFactory { + + def apply(compressionCodec: CompressionCodec, stream: OutputStream): OutputStream = { + compressionCodec match { + case DefaultCompressionCodec => new GZIPOutputStream(stream) + case GZIPCompressionCodec => new GZIPOutputStream(stream) + case SnappyCompressionCodec => + import org.xerial.snappy.SnappyOutputStream + new SnappyOutputStream(stream) + case _ => + throw new kafka.common.UnknownCodecException("Unknown Codec: " + compressionCodec) + } + } + + def apply(compressionCodec: CompressionCodec, stream: InputStream): InputStream = { + compressionCodec match { + case DefaultCompressionCodec => new GZIPInputStream(stream) + case GZIPCompressionCodec => new GZIPInputStream(stream) + case SnappyCompressionCodec => + import org.xerial.snappy.SnappyInputStream + new SnappyInputStream(stream) + case _ => + throw new kafka.common.UnknownCodecException("Unknown Codec: " + compressionCodec) + } + } +} diff --git core/src/main/scala/kafka/message/CompressionUtils.scala core/src/main/scala/kafka/message/CompressionUtils.scala deleted file mode 100644 index 607ca77..0000000 --- core/src/main/scala/kafka/message/CompressionUtils.scala +++ /dev/null @@ -1,160 +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.message - -import java.io.ByteArrayOutputStream -import java.io.IOException -import java.io.InputStream -import java.nio.ByteBuffer -import kafka.utils._ - -abstract sealed class CompressionFacade(inputStream: InputStream, outputStream: ByteArrayOutputStream) { - def close() = { - if (inputStream != null) inputStream.close() - if (outputStream != null) outputStream.close() - } - def read(a: Array[Byte]): Int - def write(a: Array[Byte]) -} - -class GZIPCompression(inputStream: InputStream, outputStream: ByteArrayOutputStream) extends CompressionFacade(inputStream,outputStream) { - import java.util.zip.GZIPInputStream - import java.util.zip.GZIPOutputStream - val gzipIn:GZIPInputStream = if (inputStream == null) null else new GZIPInputStream(inputStream) - val gzipOut:GZIPOutputStream = if (outputStream == null) null else new GZIPOutputStream(outputStream) - - override def close() { - if (gzipIn != null) gzipIn.close() - if (gzipOut != null) gzipOut.close() - super.close() - } - - override def write(a: Array[Byte]) = { - gzipOut.write(a) - } - - override def read(a: Array[Byte]): Int = { - gzipIn.read(a) - } -} - -class SnappyCompression(inputStream: InputStream,outputStream: ByteArrayOutputStream) extends CompressionFacade(inputStream,outputStream) { - import org.xerial.snappy.SnappyInputStream - import org.xerial.snappy.SnappyOutputStream - - val snappyIn:SnappyInputStream = if (inputStream == null) null else new SnappyInputStream(inputStream) - val snappyOut:SnappyOutputStream = if (outputStream == null) null else new SnappyOutputStream(outputStream) - - override def close() = { - if (snappyIn != null) snappyIn.close() - if (snappyOut != null) snappyOut.close() - super.close() - } - - override def write(a: Array[Byte]) = { - snappyOut.write(a) - } - - override def read(a: Array[Byte]): Int = { - snappyIn.read(a) - } - -} - -object CompressionFactory { - def apply(compressionCodec: CompressionCodec, stream: ByteArrayOutputStream): CompressionFacade = compressionCodec match { - case GZIPCompressionCodec => new GZIPCompression(null,stream) - case SnappyCompressionCodec => new SnappyCompression(null,stream) - case _ => - throw new kafka.common.UnknownCodecException("Unknown Codec: " + compressionCodec) - } - def apply(compressionCodec: CompressionCodec, stream: InputStream): CompressionFacade = compressionCodec match { - case GZIPCompressionCodec => new GZIPCompression(stream,null) - case SnappyCompressionCodec => new SnappyCompression(stream,null) - case _ => - throw new kafka.common.UnknownCodecException("Unknown Codec: " + compressionCodec) - } -} - -object CompressionUtils extends Logging{ - - //specify the codec which is the default when DefaultCompressionCodec is used - private var defaultCodec: CompressionCodec = GZIPCompressionCodec - - def compress(messages: Iterable[Message], compressionCodec: CompressionCodec = DefaultCompressionCodec):Message = { - val outputStream:ByteArrayOutputStream = new ByteArrayOutputStream() - - debug("Allocating message byte buffer of size = " + MessageSet.messageSetSize(messages)) - - var cf: CompressionFacade = null - - if (compressionCodec == DefaultCompressionCodec) - cf = CompressionFactory(defaultCodec,outputStream) - else - cf = CompressionFactory(compressionCodec,outputStream) - - val messageByteBuffer = ByteBuffer.allocate(MessageSet.messageSetSize(messages)) - messages.foreach(m => m.serializeTo(messageByteBuffer)) - messageByteBuffer.rewind - - try { - cf.write(messageByteBuffer.array) - } catch { - case e: IOException => error("Error while writing to the GZIP output stream", e) - cf.close() - throw e - } finally { - cf.close() - } - - val oneCompressedMessage:Message = new Message(outputStream.toByteArray, compressionCodec) - oneCompressedMessage - } - - def decompress(message: Message): ByteBufferMessageSet = { - val outputStream:ByteArrayOutputStream = new ByteArrayOutputStream - val inputStream:InputStream = new ByteBufferBackedInputStream(message.payload) - - val intermediateBuffer = new Array[Byte](1024) - - var cf: CompressionFacade = null - - if (message.compressionCodec == DefaultCompressionCodec) - cf = CompressionFactory(defaultCodec,inputStream) - else - cf = CompressionFactory(message.compressionCodec,inputStream) - - try { - Stream.continually(cf.read(intermediateBuffer)).takeWhile(_ > 0).foreach { dataRead => - outputStream.write(intermediateBuffer, 0, dataRead) - } - }catch { - case e: IOException => error("Error while reading from the GZIP input stream", e) - cf.close() - throw e - } finally { - cf.close() - } - - val outputBuffer = ByteBuffer.allocate(outputStream.size) - outputBuffer.put(outputStream.toByteArray) - outputBuffer.rewind - val outputByteArray = outputStream.toByteArray - new ByteBufferMessageSet(outputBuffer) - } -} diff --git core/src/main/scala/kafka/message/FileMessageSet.scala core/src/main/scala/kafka/message/FileMessageSet.scala deleted file mode 100644 index 2b48495..0000000 --- core/src/main/scala/kafka/message/FileMessageSet.scala +++ /dev/null @@ -1,245 +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.message - -import java.io._ -import java.nio._ -import java.nio.channels._ -import java.util.concurrent.atomic._ - -import kafka.utils._ -import kafka.common.KafkaException -import java.util.concurrent.TimeUnit -import kafka.metrics.{KafkaTimer, KafkaMetricsGroup} - -/** - * An on-disk message set. The set can be opened either mutably or immutably. Mutation attempts - * will fail on an immutable message set. An optional limit and offset can be applied to the message set - * which will control the offset into the file and the effective length into the file from which - * messages will be read - */ -@nonthreadsafe -class FileMessageSet private[kafka](private[message] val channel: FileChannel, - private[message] val offset: Long, - private[message] val limit: Long, - val mutable: Boolean, - val needRecover: AtomicBoolean) extends MessageSet with Logging { - - private val setSize = new AtomicLong() - - if(mutable) { - if(limit < Long.MaxValue || offset > 0) - throw new KafkaException("Attempt to open a mutable message set with a view or offset, which is not allowed.") - - if (needRecover.get) { - // set the file position to the end of the file for appending messages - val startMs = System.currentTimeMillis - val truncated = recover() - info("Recovery succeeded in " + (System.currentTimeMillis - startMs) / 1000 + - " seconds. " + truncated + " bytes truncated. New log size is " + sizeInBytes() + " bytes") - } - else { - setSize.set(channel.size()) - channel.position(channel.size) - } - } else { - setSize.set(scala.math.min(channel.size(), limit) - offset) - } - - /** - * Create a file message set with no limit or offset - */ - def this(channel: FileChannel, mutable: Boolean) = - this(channel, 0, Long.MaxValue, mutable, new AtomicBoolean(false)) - - /** - * Create a file message set with no limit or offset - */ - def this(file: File, mutable: Boolean) = - this(Utils.openChannel(file, mutable), mutable) - - /** - * Create a file message set with no limit or offset - */ - def this(channel: FileChannel, mutable: Boolean, needRecover: AtomicBoolean) = - this(channel, 0, Long.MaxValue, mutable, needRecover) - - /** - * Create a file message set with no limit or offset - */ - def this(file: File, mutable: Boolean, needRecover: AtomicBoolean) = - this(Utils.openChannel(file, mutable), mutable, needRecover) - - - /** - * Return a message set which is a view into this set starting from the given offset and with the given size limit. - */ - def read(readOffset: Long, size: Long): FileMessageSet = { - new FileMessageSet(channel, this.offset + readOffset, scala.math.min(this.offset + readOffset + size, sizeInBytes()), - false, new AtomicBoolean(false)) - } - - /** - * Write some of this set to the given channel, return the ammount written - */ - def writeTo(destChannel: GatheringByteChannel, writeOffset: Long, size: Long): Long = - channel.transferTo(offset + writeOffset, scala.math.min(size, sizeInBytes), destChannel) - - /** - * Get an iterator over the messages in the set - */ - override def iterator: Iterator[MessageAndOffset] = { - new IteratorTemplate[MessageAndOffset] { - var location = offset - - override def makeNext(): MessageAndOffset = { - // read the size of the item - val sizeBuffer = ByteBuffer.allocate(4) - channel.read(sizeBuffer, location) - if(sizeBuffer.hasRemaining) - return allDone() - - sizeBuffer.rewind() - val size: Int = sizeBuffer.getInt() - if (size < Message.MinHeaderSize) - return allDone() - - // read the item itself - val buffer = ByteBuffer.allocate(size) - channel.read(buffer, location + 4) - if(buffer.hasRemaining) - return allDone() - buffer.rewind() - - // increment the location and return the item - location += size + 4 - new MessageAndOffset(new Message(buffer), location) - } - } - } - - /** - * The number of bytes taken up by this file set - */ - def sizeInBytes(): Long = setSize.get() - - def checkMutable(): Unit = { - if(!mutable) - throw new KafkaException("Attempt to invoke mutation on immutable message set.") - } - - /** - * Append this message to the message set - */ - def append(messages: MessageSet): Unit = { - checkMutable() - var written = 0L - while(written < messages.sizeInBytes) - written += messages.writeTo(channel, 0, messages.sizeInBytes) - setSize.getAndAdd(written) - } - - /** - * Commit all written data to the physical disk - */ - def flush() = { - checkMutable() - LogFlushStats.logFlushTimer.time { - channel.force(true) - } - } - - /** - * Close this message set - */ - def close() = { - if(mutable) - flush() - channel.close() - } - - /** - * Recover log up to the last complete entry. Truncate off any bytes from any incomplete messages written - */ - def recover(): Long = { - checkMutable() - val len = channel.size - val buffer = ByteBuffer.allocate(4) - var validUpTo: Long = 0 - var next = 0L - do { - next = validateMessage(channel, validUpTo, len, buffer) - if(next >= 0) - validUpTo = next - } while(next >= 0) - truncateTo(validUpTo) - needRecover.set(false) - len - validUpTo - } - - def truncateTo(targetSize: Long) = { - if(targetSize > sizeInBytes()) - throw new KafkaException("Attempt to truncate log segment to %d bytes failed since the current ".format(targetSize) + - " size of this log segment is only %d bytes".format(sizeInBytes())) - channel.truncate(targetSize) - setSize.set(targetSize) - /* This should not be necessary, but fixes bug 6191269 on some OSs. */ - channel.position(targetSize) - } - - /** - * Read, validate, and discard a single message, returning the next valid offset, and - * the message being validated - */ - private def validateMessage(channel: FileChannel, start: Long, len: Long, buffer: ByteBuffer): Long = { - buffer.rewind() - var read = channel.read(buffer, start) - if(read < 4) - return -1 - - // check that we have sufficient bytes left in the file - val size = buffer.getInt(0) - if (size < Message.MinHeaderSize) - return -1 - - val next = start + 4 + size - if(next > len) - return -1 - - // read the message - val messageBuffer = ByteBuffer.allocate(size) - var curr = start + 4 - while(messageBuffer.hasRemaining) { - read = channel.read(messageBuffer, curr) - if(read < 0) - throw new KafkaException("File size changed during recovery!") - else - curr += read - } - messageBuffer.rewind() - val message = new Message(messageBuffer) - if(!message.isValid) - return -1 - else - next - } -} - -object LogFlushStats extends KafkaMetricsGroup { - val logFlushTimer = new KafkaTimer(newTimer("LogFlushRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS)) -} diff --git core/src/main/scala/kafka/message/Message.scala core/src/main/scala/kafka/message/Message.scala index c7ba4f9..aff46e4 100644 --- core/src/main/scala/kafka/message/Message.scala +++ core/src/main/scala/kafka/message/Message.scala @@ -18,145 +18,214 @@ package kafka.message import java.nio._ +import scala.math._ import kafka.utils._ -import kafka.common.UnknownMagicByteException /** - * Message byte offsets + * Constants related to messages */ object Message { - val MagicVersion: Byte = 1 - val CurrentMagicValue: Byte = 1 - val MagicOffset = 0 - val MagicLength = 1 - val AttributeOffset = MagicOffset + MagicLength - val AttributeLength = 1 + /** - * Specifies the mask for the compression code. 2 bits to hold the compression codec. - * 0 is reserved to indicate no compression + * The current offset and size for all the fixed-length fields */ - val CompressionCodeMask: Int = 0x03 // - - - val NoCompression:Int = 0 - + val CrcOffset = 0 + val CrcLength = 4 + val MagicOffset = CrcOffset + CrcLength + val MagicLength = 1 + val AttributesOffset = MagicOffset + MagicLength + val AttributesLength = 1 + val KeySizeOffset = AttributesOffset + AttributesLength + val KeySizeLength = 4 + val KeyOffset = KeySizeOffset + KeySizeLength + val MessageOverhead = KeyOffset + /** - * Computes the CRC value based on the magic byte - * @param magic Specifies the magic byte value. The only value allowed currently is 1. + * The minimum valid size for the message header */ - def crcOffset(magic: Byte): Int = magic match { - case MagicVersion => AttributeOffset + AttributeLength - case _ => throw new UnknownMagicByteException("Magic byte value of %d is unknown".format(magic)) - } + val MinHeaderSize = CrcLength + MagicLength + AttributesLength + KeySizeLength - val CrcLength = 4 - /** - * Computes the offset to the message payload based on the magic byte - * @param magic Specifies the magic byte value. The only value allowed currently is 1. + * The current "magic" value */ - def payloadOffset(magic: Byte): Int = crcOffset(magic) + CrcLength + val CurrentMagicValue: Byte = 2 /** - * Computes the size of the message header based on the magic byte - * @param magic Specifies the magic byte value. The only value allowed currently is 1. + * Specifies the mask for the compression code. 2 bits to hold the compression codec. + * 0 is reserved to indicate no compression */ - def headerSize(magic: Byte): Int = payloadOffset(magic) + val CompressionCodeMask: Int = 0x03 /** - * Size of the header for magic byte 1. This is the minimum size of any message header. + * Compression code for uncompressed messages */ - val MinHeaderSize = headerSize(1); + val NoCompression: Int = 0 + } /** * A message. The format of an N byte message is the following: * - * 1. 1 byte "magic" identifier to allow format changes, whose value is 1 currently - * - * 2. 1 byte "attributes" identifier to allow annotations on the message independent of the version (e.g. compression enabled, type of codec used) - * - * 3. 4 byte CRC32 of the payload - * - * 4. N - 6 byte payload + * 1. 4 byte CRC32 of the message + * 2. 1 byte "magic" identifier to allow format changes, value is 2 currently + * 3. 1 byte "attributes" identifier to allow annotations on the message independent of the version (e.g. compression enabled, type of codec used) + * 4. 4 byte key length, containing length K + * 5. K byte key + * 6. (N - K - 10) byte payload * + * Default constructor wraps an existing ByteBuffer with the Message object with no change to the contents. */ class Message(val buffer: ByteBuffer) { import kafka.message.Message._ - - - private def this(checksum: Long, bytes: Array[Byte], offset: Int, size: Int, compressionCodec: CompressionCodec) = { - this(ByteBuffer.allocate(Message.headerSize(Message.CurrentMagicValue) + size)) - buffer.put(CurrentMagicValue) + + /** + * A constructor to create a Message + * @param bytes The payload of the message + * @param compressionCodec The compression codec used on the contents of the message (if any) + * @param key The key of the message (null, if none) + * @param payloadOffset The offset into the payload array used to extract payload + * @param payloadSize The size of the payload to use + */ + def this(bytes: Array[Byte], + key: Array[Byte], + codec: CompressionCodec, + payloadOffset: Int, + payloadSize: Int) = { + this(ByteBuffer.allocate(Message.CrcLength + + Message.MagicLength + + Message.AttributesLength + + Message.KeySizeLength + + (if(key == null) 0 else key.length) + + (if(payloadSize >= 0) payloadSize else bytes.length - payloadOffset))) + // skip crc, we will fill that in at the end + buffer.put(MagicOffset, CurrentMagicValue) var attributes:Byte = 0 - if (compressionCodec.codec > 0) { - attributes = (attributes | (Message.CompressionCodeMask & compressionCodec.codec)).toByte + if (codec.codec > 0) + attributes = (attributes | (CompressionCodeMask & codec.codec)).toByte + buffer.put(AttributesOffset, attributes) + if(key == null) { + buffer.putInt(KeySizeOffset, -1) + buffer.position(KeyOffset) + } else { + buffer.putInt(KeySizeOffset, key.length) + buffer.position(KeyOffset) + buffer.put(key, 0, key.length) } - buffer.put(attributes) - Utils.putUnsignedInt(buffer, checksum) - buffer.put(bytes, offset, size) + buffer.put(bytes, payloadOffset, if(payloadSize >= 0) payloadSize else bytes.length - payloadOffset) buffer.rewind() + + // now compute the checksum and fill it in + Utils.putUnsignedInt(buffer, CrcOffset, computeChecksum) } - - def this(checksum:Long, bytes:Array[Byte]) = this(checksum, bytes, 0, bytes.length, NoCompressionCodec) - - def this(bytes: Array[Byte], offset: Int, size: Int, compressionCodec: CompressionCodec) = { - //Note: we're not crc-ing the attributes header, so we're susceptible to bit-flipping there - this(Utils.crc32(bytes, offset, size), bytes, offset, size, compressionCodec) + + def this(bytes: Array[Byte], key: Array[Byte], codec: CompressionCodec) = + this(bytes = bytes, key = key, codec = codec, payloadOffset = 0, payloadSize = -1) + + def this(bytes: Array[Byte], codec: CompressionCodec) = + this(bytes = bytes, key = null, codec = codec) + + def this(bytes: Array[Byte], key: Array[Byte]) = + this(bytes = bytes, key = key, codec = NoCompressionCodec) + + def this(bytes: Array[Byte]) = + this(bytes = bytes, key = null, codec = NoCompressionCodec) + + /** + * Compute the checksum of the message from the message contents + */ + def computeChecksum(): Long = + Utils.crc32(buffer.array, buffer.arrayOffset + MagicOffset, buffer.limit - MagicOffset) + + /** + * Retrieve the previously computed CRC for this message + */ + def checksum: Long = Utils.getUnsignedInt(buffer, CrcOffset) + + /** + * Returns true if the crc stored with the message matches the crc computed off the message contents + */ + def isValid: Boolean = checksum == computeChecksum + + /** + * Throw an InvalidMessageException if isValid is false for this message + */ + def ensureValid() { + if(!isValid) + throw new InvalidMessageException("Message is corrupt (stored crc = " + checksum + ", computed crc = " + computeChecksum() + ")") } - - def this(bytes: Array[Byte], compressionCodec: CompressionCodec) = this(bytes, 0, bytes.length, compressionCodec) - - def this(bytes: Array[Byte], offset: Int, size: Int) = this(bytes, offset, size, NoCompressionCodec) - - def this(bytes: Array[Byte]) = this(bytes, 0, bytes.length, NoCompressionCodec) + /** + * The complete serialized size of this message in bytes (including crc, header attributes, etc) + */ def size: Int = buffer.limit - def payloadSize: Int = size - headerSize(magic) + /** + * The length of the key in bytes + */ + def keySize: Int = buffer.getInt(Message.KeySizeOffset) + + /** + * Does the message have a key? + */ + def hasKey: Boolean = keySize >= 0 + + /** + * The length of the message value in bytes + */ + def payloadSize: Int = size - KeyOffset - max(0, keySize) + /** + * The magic version of this message + */ def magic: Byte = buffer.get(MagicOffset) - def attributes: Byte = buffer.get(AttributeOffset) + /** + * The attributes stored with this message + */ + def attributes: Byte = buffer.get(AttributesOffset) - def compressionCodec:CompressionCodec = { - magic match { - case 0 => NoCompressionCodec - case 1 => CompressionCodec.getCompressionCodec(buffer.get(AttributeOffset) & CompressionCodeMask) - case _ => throw new RuntimeException("Invalid magic byte " + magic) - } - - } - - def checksum: Long = Utils.getUnsignedInt(buffer, crcOffset(magic)) + /** + * The compression codec used with this message + */ + def compressionCodec: CompressionCodec = + CompressionCodec.getCompressionCodec(buffer.get(AttributesOffset) & CompressionCodeMask) + /** + * A ByteBuffer containing the content of the message + */ def payload: ByteBuffer = { var payload = buffer.duplicate - payload.position(headerSize(magic)) + payload.position(KeyOffset + max(keySize, 0)) payload = payload.slice() payload.limit(payloadSize) payload.rewind() payload } - def isValid: Boolean = - checksum == Utils.crc32(buffer.array, buffer.position + buffer.arrayOffset + payloadOffset(magic), payloadSize) - - def serializedSize: Int = 4 /* int size*/ + buffer.limit - - def serializeTo(serBuffer:ByteBuffer) = { - serBuffer.putInt(buffer.limit) - serBuffer.put(buffer.duplicate) + /** + * A ByteBuffer containing the message key + */ + def key: ByteBuffer = { + val s = keySize + if(s < 0) { + null + } else { + var key = buffer.duplicate + key.position(KeyOffset) + key = key.slice() + key.limit(s) + key.rewind() + key + } } override def toString(): String = - "message(magic = %d, attributes = %d, crc = %d, payload = %s)".format(magic, attributes, checksum, payload) + "Message(magic = %d, attributes = %d, crc = %d, key = %s, payload = %s)".format(magic, attributes, checksum, key, payload) override def equals(any: Any): Boolean = { any match { - case that: Message => size == that.size && attributes == that.attributes && checksum == that.checksum && - payload == that.payload && magic == that.magic + case that: Message => this.buffer.equals(that.buffer) case _ => false } } diff --git core/src/main/scala/kafka/message/MessageAndOffset.scala core/src/main/scala/kafka/message/MessageAndOffset.scala index d769fc6..51edf9f 100644 --- core/src/main/scala/kafka/message/MessageAndOffset.scala +++ core/src/main/scala/kafka/message/MessageAndOffset.scala @@ -18,5 +18,11 @@ package kafka.message -case class MessageAndOffset(message: Message, offset: Long) +case class MessageAndOffset(message: Message, offset: Long) { + + /** + * Compute the offset of the next message in the log + */ + def nextOffset: Long = offset + 1 +} diff --git core/src/main/scala/kafka/message/MessageSet.scala core/src/main/scala/kafka/message/MessageSet.scala index b53c60b..8c75bfd 100644 --- core/src/main/scala/kafka/message/MessageSet.scala +++ core/src/main/scala/kafka/message/MessageSet.scala @@ -24,9 +24,11 @@ import java.nio.channels._ * Message set helper functions */ object MessageSet { - - val LogOverhead = 4 - val Empty: MessageSet = new ByteBufferMessageSet(ByteBuffer.allocate(0)) + + val MessageSizeLength = 4 + val OffsetLength = 8 + val LogOverhead = MessageSizeLength + OffsetLength + val Empty = new ByteBufferMessageSet(ByteBuffer.allocate(0)) /** * The size of a message set containing the given messages @@ -52,37 +54,15 @@ object MessageSet { */ def entrySize(message: Message): Int = LogOverhead + message.size - def createByteBuffer(compressionCodec: CompressionCodec, messages: Message*): ByteBuffer = - compressionCodec match { - case NoCompressionCodec => - val buffer = ByteBuffer.allocate(MessageSet.messageSetSize(messages)) - for (message <- messages) { - message.serializeTo(buffer) - } - buffer.rewind - buffer - case _ => - messages.size match { - case 0 => - val buffer = ByteBuffer.allocate(MessageSet.messageSetSize(messages)) - buffer.rewind - buffer - case _ => - val message = CompressionUtils.compress(messages, compressionCodec) - val buffer = ByteBuffer.allocate(message.serializedSize) - message.serializeTo(buffer) - buffer.rewind - buffer - } - } } /** - * A set of messages. A message set has a fixed serialized form, though the container - * for the bytes could be either in-memory or on disk. A The format of each message is + * A set of messages with offsets. A message set has a fixed serialized form, though the container + * for the bytes could be either in-memory or on disk. The format of each message is * as follows: + * 8 byte message offset number * 4 byte size containing an integer N - * N message bytes as described in the message class + * N message bytes as described in the Message class */ abstract class MessageSet extends Iterable[MessageAndOffset] { @@ -92,7 +72,7 @@ abstract class MessageSet extends Iterable[MessageAndOffset] { def writeTo(channel: GatheringByteChannel, offset: Long, maxSize: Long): Long /** - * Provides an iterator over the messages in this set + * Provides an iterator over the message/offset pairs in this set */ def iterator: Iterator[MessageAndOffset] @@ -110,5 +90,19 @@ abstract class MessageSet extends Iterable[MessageAndOffset] { if(!messageAndOffset.message.isValid) throw new InvalidMessageException } + + /** + * Print this message set's contents + */ + override def toString: String = { + val builder = new StringBuilder() + builder.append(getClass.getSimpleName + "(") + for(message <- this) { + builder.append(message) + builder.append(", ") + } + builder.append(")") + builder.toString + } } diff --git core/src/main/scala/kafka/producer/ConsoleProducer.scala core/src/main/scala/kafka/producer/ConsoleProducer.scala index 589923c..4a8b296 100644 --- core/src/main/scala/kafka/producer/ConsoleProducer.scala +++ core/src/main/scala/kafka/producer/ConsoleProducer.scala @@ -88,7 +88,8 @@ object ConsoleProducer { val props = new Properties() props.put("broker.list", brokerList) - props.put("compression.codec", DefaultCompressionCodec.codec.toString) + val codec = if(compress) DefaultCompressionCodec.codec else NoCompressionCodec.codec + props.put("compression.codec", codec.toString) props.put("producer.type", if(sync) "sync" else "async") if(options.has(batchSizeOpt)) props.put("batch.size", batchSize.toString) diff --git core/src/main/scala/kafka/server/AbstractFetcherThread.scala core/src/main/scala/kafka/server/AbstractFetcherThread.scala index fbab2db..119d0f1 100644 --- core/src/main/scala/kafka/server/AbstractFetcherThread.scala +++ core/src/main/scala/kafka/server/AbstractFetcherThread.scala @@ -22,6 +22,7 @@ import kafka.consumer.SimpleConsumer import kafka.common.{TopicAndPartition, ErrorMapping} import collection.mutable import kafka.message.ByteBufferMessageSet +import kafka.message.MessageAndOffset import kafka.api.{FetchResponse, PartitionData, FetchRequestBuilder} import kafka.metrics.KafkaMetricsGroup import com.yammer.metrics.core.Gauge @@ -36,12 +37,13 @@ import java.util.concurrent.TimeUnit abstract class AbstractFetcherThread(name: String, sourceBroker: Broker, socketTimeout: Int, socketBufferSize: Int, fetchSize: Int, fetcherBrokerId: Int = -1, maxWait: Int = 0, minBytes: Int = 1) extends ShutdownableThread(name) { + private val fetchMap = new mutable.HashMap[TopicAndPartition, Long] // a (topic, partition) -> offset map private val fetchMapLock = new Object val simpleConsumer = new SimpleConsumer(sourceBroker.host, sourceBroker.port, socketTimeout, socketBufferSize) val fetcherMetrics = FetcherStat.getFetcherStat(name + "-" + sourceBroker.id) - // callbacks to be defined in subclass + /* callbacks to be defined in subclass */ // process fetched data def processPartitionData(topic: String, fetchOffset: Long, partitionData: PartitionData) @@ -101,8 +103,12 @@ abstract class AbstractFetcherThread(name: String, sourceBroker: Broker, socket partitionData.error match { case ErrorMapping.NoError => processPartitionData(topic, currentOffset.get, partitionData) - val validBytes = partitionData.messages.asInstanceOf[ByteBufferMessageSet].validBytes - val newOffset = currentOffset.get + partitionData.messages.asInstanceOf[ByteBufferMessageSet].validBytes + val messages = partitionData.messages.asInstanceOf[ByteBufferMessageSet] + val validBytes = messages.validBytes + val newOffset = messages.lastOption match { + case Some(m: MessageAndOffset) => m.nextOffset + case None => currentOffset.get + } fetchMap.put(topicAndPartition, newOffset) FetcherLagMetrics.getFetcherLagMetrics(topic, partitionId).lag = partitionData.hw - newOffset fetcherMetrics.byteRate.mark(validBytes) @@ -122,7 +128,7 @@ abstract class AbstractFetcherThread(name: String, sourceBroker: Broker, socket } } - if (partitionsWithError.size > 0) { + if(partitionsWithError.size > 0) { debug("handling partitions with error for %s".format(partitionsWithError)) handlePartitionsWithErrors(partitionsWithError) } diff --git core/src/main/scala/kafka/server/KafkaApis.scala core/src/main/scala/kafka/server/KafkaApis.scala index 4fe8248..b1fda8b 100644 --- core/src/main/scala/kafka/server/KafkaApis.scala +++ core/src/main/scala/kafka/server/KafkaApis.scala @@ -29,6 +29,7 @@ import mutable.HashMap import scala.math._ import kafka.network.RequestChannel.Response import java.util.concurrent.TimeUnit +import java.util.concurrent.atomic._ import kafka.metrics.KafkaMetricsGroup import org.I0Itec.zkclient.ZkClient import kafka.common._ @@ -98,7 +99,7 @@ class KafkaApis(val requestChannel: RequestChannel, */ def maybeUnblockDelayedFetchRequests(topic: String, partitionData: PartitionData) { val partition = partitionData.partition - val satisfied = fetchRequestPurgatory.update(RequestKey(topic, partition), null) + val satisfied = fetchRequestPurgatory.update(RequestKey(topic, partition), partitionData) trace("Producer request to (%s-%d) unblocked %d fetch requests.".format(topic, partition, satisfied.size)) // send any newly unblocked responses @@ -119,26 +120,30 @@ class KafkaApis(val requestChannel: RequestChannel, requestLogger.trace("Handling producer request " + request.toString) trace("Handling producer request " + request.toString) - val localProduceResponse = produceToLocalLog(produceRequest) + val localProduceResults = appendToLocalLog(produceRequest.data) debug("Produce to local log in %d ms".format(SystemTime.milliseconds - sTime)) - val numPartitionsInError = localProduceResponse.status.count(_._2.error != ErrorMapping.NoError) + val numPartitionsInError = localProduceResults.count(_.error.isDefined) produceRequest.data.foreach(partitionAndData => maybeUnblockDelayedFetchRequests(partitionAndData._1.topic, partitionAndData._2)) if (produceRequest.requiredAcks == 0 || produceRequest.requiredAcks == 1 || produceRequest.numPartitions <= 0 || - numPartitionsInError == produceRequest.numPartitions) - requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(localProduceResponse))) - else { + numPartitionsInError == produceRequest.numPartitions) { + val statuses = localProduceResults.map(r => r.key -> ProducerResponseStatus(r.errorCode, r.start)).toMap + val response = ProducerResponse(produceRequest.versionId, produceRequest.correlationId, statuses) + requestChannel.sendResponse(new RequestChannel.Response(request, new BoundedByteBufferSend(response))) + } else { // create a list of (topic, partition) pairs to use as keys for this delayed request val producerRequestKeys = produceRequest.data.keys.map( topicAndPartition => new RequestKey(topicAndPartition)).toSeq - - val delayedProduce = new DelayedProduce( - producerRequestKeys, request, localProduceResponse, - produceRequest, produceRequest.ackTimeoutMs.toLong) + val statuses = localProduceResults.map(r => r.key -> ProducerResponseStatus(r.errorCode, r.end + 1)).toMap + val delayedProduce = new DelayedProduce(producerRequestKeys, + request, + statuses, + produceRequest, + produceRequest.ackTimeoutMs.toLong) producerRequestPurgatory.watch(delayedProduce) /* @@ -155,47 +160,51 @@ class KafkaApis(val requestChannel: RequestChannel, satisfiedProduceRequests.foreach(_.respond()) } } + + case class ProduceResult(key: TopicAndPartition, start: Long, end: Long, error: Option[Throwable] = None) { + def this(key: TopicAndPartition, throwable: Throwable) = + this(key, -1L, -1L, Some(throwable)) + + def errorCode = error match { + case None => ErrorMapping.NoError + case Some(error) => ErrorMapping.codeFor(error.getClass.asInstanceOf[Class[Throwable]]) + } + } /** * Helper method for handling a parsed producer request */ - private def produceToLocalLog(request: ProducerRequest): ProducerResponse = { - trace("Produce [%s] to local log ".format(request.toString)) - - val localErrorsAndOffsets = request.data.map (topicAndPartitionData => { - val (topic, partitionData) = (topicAndPartitionData._1.topic, topicAndPartitionData._2) - BrokerTopicStat.getBrokerTopicStat(topic).bytesInRate.mark(partitionData.messages.sizeInBytes) + private def appendToLocalLog(messages: Map[TopicAndPartition, PartitionData]): Iterable[ProduceResult] = { + trace("Append [%s] to local log ".format(messages.toString)) + messages.map (data => { + val (key, partitionData) = data + BrokerTopicStat.getBrokerTopicStat(key.topic).bytesInRate.mark(partitionData.messages.sizeInBytes) BrokerTopicStat.getBrokerAllTopicStat.bytesInRate.mark(partitionData.messages.sizeInBytes) - + try { - val localReplica = replicaManager.getLeaderReplicaIfLocal(topic, partitionData.partition) + val localReplica = replicaManager.getLeaderReplicaIfLocal(key.topic, key.partition) val log = localReplica.log.get - log.append(partitionData.messages.asInstanceOf[ByteBufferMessageSet]) + val (start, end) = log.append(partitionData.messages.asInstanceOf[ByteBufferMessageSet]) // we may need to increment high watermark since ISR could be down to 1 localReplica.partition.maybeIncrementLeaderHW(localReplica) - val responseStatus = ProducerResponseStatus(ErrorMapping.NoError, log.logEndOffset) - trace("%d bytes written to logs, nextAppendOffset = %d" - .format(partitionData.messages.sizeInBytes, responseStatus.nextOffset)) - (TopicAndPartition(topic, partitionData.partition), responseStatus) + trace("%d bytes written to log %s-%d beginning at offset %d and ending at offset %d" + .format(partitionData.messages.sizeInBytes, key.topic, key.partition, start, end)) + ProduceResult(key, start, end) } catch { case e: Throwable => - BrokerTopicStat.getBrokerTopicStat(topic).failedProduceRequestRate.mark() + BrokerTopicStat.getBrokerTopicStat(key.topic).failedProduceRequestRate.mark() BrokerTopicStat.getBrokerAllTopicStat.failedProduceRequestRate.mark() - error("Error processing ProducerRequest on %s:%d".format(topic, partitionData.partition), e) + error("Error processing ProducerRequest on %s:%d".format(key.topic, key.partition), e) e match { case _: IOException => fatal("Halting due to unrecoverable I/O error while handling producer request: " + e.getMessage, e) // compiler requires scala.sys.exit (not System.exit). exit(1) - case _ => - val (error, offset) = (ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]), -1L) - (TopicAndPartition(topic, partitionData.partition), ProducerResponseStatus(error, offset)) + null + case _ => new ProduceResult(key, e) } - } - } - ) - - ProducerResponse(request.versionId, request.correlationId, localErrorsAndOffsets) + } + }) } /** @@ -208,7 +217,7 @@ class KafkaApis(val requestChannel: RequestChannel, trace("Handling fetch request " + fetchRequest.toString) if(fetchRequest.isFromFollower) { - maybeUpdatePartitionHW(fetchRequest) + maybeUpdatePartitionHw(fetchRequest) // after updating HW, some delayed produce requests may be unblocked var satisfiedProduceRequests = new mutable.ArrayBuffer[DelayedProduce] fetchRequest.requestInfo.foreach { @@ -221,61 +230,24 @@ class KafkaApis(val requestChannel: RequestChannel, satisfiedProduceRequests.foreach(_.respond()) } - // if there are enough bytes available right now we can answer the request, otherwise we have to punt - val availableBytes = availableFetchBytes(fetchRequest) + val dataRead = readMessageSets(fetchRequest) + val bytesReadable = dataRead.values.map(_.messages.sizeInBytes).sum if(fetchRequest.maxWait <= 0 || - availableBytes >= fetchRequest.minBytes || + bytesReadable >= fetchRequest.minBytes || fetchRequest.numPartitions <= 0) { - val topicData = readMessageSets(fetchRequest) - debug("Returning fetch response %s for fetch request with correlation id %d".format( - topicData.values.map(_.error).mkString(","), fetchRequest.correlationId)) - val response = FetchResponse(FetchRequest.CurrentVersion, fetchRequest.correlationId, topicData) + debug("Returning fetch response %s for fetch request with correlation id %d".format(dataRead.values.map(_.error).mkString(","), fetchRequest.correlationId)) + val response = new FetchResponse(FetchRequest.CurrentVersion, fetchRequest.correlationId, dataRead) requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(response))) } else { debug("Putting fetch request into purgatory") // create a list of (topic, partition) pairs to use as keys for this delayed request val delayedFetchKeys = fetchRequest.requestInfo.keys.toSeq.map(new RequestKey(_)) - val delayedFetch = new DelayedFetch(delayedFetchKeys, request, fetchRequest, fetchRequest.maxWait) + val delayedFetch = new DelayedFetch(delayedFetchKeys, request, fetchRequest, fetchRequest.maxWait, bytesReadable) fetchRequestPurgatory.watch(delayedFetch) } } - /** - * Calculate the number of available bytes for the given fetch request - */ - private def availableFetchBytes(fetchRequest: FetchRequest): Long = { - val totalBytes = fetchRequest.requestInfo.foldLeft(0L)((folded, curr) => { - folded + - { - val (topic, partition) = (curr._1.topic, curr._1.partition) - val (offset, fetchSize) = (curr._2.offset, curr._2.fetchSize) - debug("Fetching log for topic %s partition %d".format(topic, partition)) - try { - val leader = replicaManager.getLeaderReplicaIfLocal(topic, partition) - val end = if (!fetchRequest.isFromFollower) { - leader.highWatermark - } else { - leader.logEndOffset - } - val available = max(0, end - offset) - math.min(fetchSize, available) - } catch { - case e: UnknownTopicOrPartitionException => - info("Invalid partition %d in fetch request from client %s." - .format(partition, fetchRequest.clientId)) - 0 - case e => - warn("Error determining available fetch bytes for topic %s partition %s on broker %s for client %s" - .format(topic, partition, brokerId, fetchRequest.clientId), e) - 0 - } - } - }) - trace(totalBytes + " available bytes for fetch request.") - totalBytes - } - - private def maybeUpdatePartitionHW(fetchRequest: FetchRequest) { + private def maybeUpdatePartitionHw(fetchRequest: FetchRequest) { debug("Maybe update partition HW due to fetch request: %s ".format(fetchRequest)) fetchRequest.requestInfo.foreach(info => { val (topic, partition, offset) = (info._1.topic, info._1.partition, info._2.offset) @@ -291,21 +263,21 @@ class KafkaApis(val requestChannel: RequestChannel, val isFetchFromFollower = fetchRequest.isFromFollower fetchRequest.requestInfo.map { case (TopicAndPartition(topic, partition), PartitionFetchInfo(offset, fetchSize)) => - val partitionData = try { - val (messages, highWatermark) = readMessageSet(topic, partition, offset, fetchSize, isFetchFromFollower) - BrokerTopicStat.getBrokerTopicStat(topic).bytesOutRate.mark(messages.sizeInBytes) - BrokerTopicStat.getBrokerAllTopicStat.bytesOutRate.mark(messages.sizeInBytes) - if (!isFetchFromFollower) { - new PartitionData(partition, ErrorMapping.NoError, offset, highWatermark, messages) - } else { - debug("Leader %d for topic %s partition %d received fetch request from follower %d" + val partitionData = + try { + val (messages, highWatermark) = readMessageSet(topic, partition, offset, fetchSize, isFetchFromFollower) + BrokerTopicStat.getBrokerTopicStat(topic).bytesOutRate.mark(messages.sizeInBytes) + BrokerTopicStat.getBrokerAllTopicStat.bytesOutRate.mark(messages.sizeInBytes) + if (!isFetchFromFollower) { + new PartitionData(partition, ErrorMapping.NoError, offset, highWatermark, messages) + } else { + debug("Leader %d for topic %s partition %d received fetch request from follower %d" .format(brokerId, topic, partition, fetchRequest.replicaId)) - debug("Leader %d returning %d messages for topic %s partition %d to follower %d" + debug("Leader %d returning %d messages for topic %s partition %d to follower %d" .format(brokerId, messages.sizeInBytes, topic, partition, fetchRequest.replicaId)) - new PartitionData(partition, ErrorMapping.NoError, offset, highWatermark, messages) - } - } - catch { + new PartitionData(partition, ErrorMapping.NoError, offset, highWatermark, messages) + } + } catch { case t: Throwable => BrokerTopicStat.getBrokerTopicStat(topic).failedFetchRequestRate.mark() BrokerTopicStat.getBrokerAllTopicStat.failedFetchRequestRate.mark() @@ -320,19 +292,20 @@ class KafkaApis(val requestChannel: RequestChannel, /** * Read from a single topic/partition at the given offset upto maxSize bytes */ - private def readMessageSet(topic: String, partition: Int, offset: Long, - maxSize: Int, fromFollower: Boolean): (MessageSet, Long) = { + private def readMessageSet(topic: String, + partition: Int, + offset: Long, + maxSize: Int, + fromFollower: Boolean): (MessageSet, Long) = { // check if the current broker is the leader for the partitions val leader = replicaManager.getLeaderReplicaIfLocal(topic, partition) trace("Fetching log segment for topic, partition, offset, size = " + (topic, partition, offset, maxSize)) - val actualSize = if (!fromFollower) { - min(leader.highWatermark - offset, maxSize).toInt - } else { - maxSize - } val messages = leader.log match { case Some(log) => - log.read(offset, actualSize) + if(fromFollower) + log.read(startOffset = offset, maxLength = maxSize, maxOffset = None) + else + log.read(startOffset = offset, maxLength = maxSize, maxOffset = Some(leader.highWatermark)) case None => error("Leader for topic %s partition %d on broker %d does not have a local log".format(topic, partition, brokerId)) MessageSet.Empty @@ -454,21 +427,24 @@ class KafkaApis(val requestChannel: RequestChannel, /** * A delayed fetch request */ - class DelayedFetch(keys: Seq[RequestKey], request: RequestChannel.Request, val fetch: FetchRequest, delayMs: Long) - extends DelayedRequest(keys, request, delayMs) + class DelayedFetch(keys: Seq[RequestKey], request: RequestChannel.Request, val fetch: FetchRequest, delayMs: Long, initialSize: Long) + extends DelayedRequest(keys, request, delayMs) { + val bytesAccumulated = new AtomicLong(initialSize) + } /** * A holding pen for fetch requests waiting to be satisfied */ - class FetchRequestPurgatory(requestChannel: RequestChannel) extends RequestPurgatory[DelayedFetch, Null](brokerId) { - + class FetchRequestPurgatory(requestChannel: RequestChannel) extends RequestPurgatory[DelayedFetch, PartitionData](brokerId) { this.logIdent = "[FetchRequestPurgatory-%d] ".format(brokerId) /** * A fetch request is satisfied when it has accumulated enough data to meet the min_bytes field */ - def checkSatisfied(n: Null, delayedFetch: DelayedFetch): Boolean = - availableFetchBytes(delayedFetch.fetch) >= delayedFetch.fetch.minBytes + def checkSatisfied(partitionData: PartitionData, delayedFetch: DelayedFetch): Boolean = { + val accumulatedSize = delayedFetch.bytesAccumulated.addAndGet(partitionData.messages.sizeInBytes) + accumulatedSize >= delayedFetch.fetch.minBytes + } /** * When a request expires just answer it with whatever data is present @@ -484,12 +460,11 @@ class KafkaApis(val requestChannel: RequestChannel, class DelayedProduce(keys: Seq[RequestKey], request: RequestChannel.Request, - localProduceResponse: ProducerResponse, + initialErrorsAndOffsets: Map[TopicAndPartition, ProducerResponseStatus], val produce: ProducerRequest, delayMs: Long) extends DelayedRequest(keys, request, delayMs) with Logging { - private val initialErrorsAndOffsets = localProduceResponse.status /** * Map of (topic, partition) -> partition status * The values in this map don't need to be synchronized since updates to the @@ -503,9 +478,9 @@ class KafkaApis(val requestChannel: RequestChannel, val (acksPending, error, nextOffset) = if (producerResponseStatus.error == ErrorMapping.NoError) { // Timeout error state will be cleared when requiredAcks are received - (true, ErrorMapping.RequestTimedOutCode, producerResponseStatus.nextOffset) + (true, ErrorMapping.RequestTimedOutCode, producerResponseStatus.offset) } - else (false, producerResponseStatus.error, producerResponseStatus.nextOffset) + else (false, producerResponseStatus.error, producerResponseStatus.offset) val initialStatus = PartitionStatus(acksPending, error, nextOffset) trace("Initial partition status for %s = %s".format(requestKey.keyLabel, initialStatus)) @@ -584,8 +559,7 @@ class KafkaApis(val requestChannel: RequestChannel, /** * A holding pen for produce requests waiting to be satisfied. */ - private [kafka] class ProducerRequestPurgatory extends RequestPurgatory[DelayedProduce, RequestKey](brokerId) { - + private [kafka] class ProducerRequestPurgatory extends RequestPurgatory[DelayedProduce, RequestKey] { this.logIdent = "[ProducerRequestPurgatory-%d] ".format(brokerId) protected def checkSatisfied(followerFetchRequestKey: RequestKey, diff --git core/src/main/scala/kafka/server/KafkaConfig.scala core/src/main/scala/kafka/server/KafkaConfig.scala index b6bab26..4252c89 100644 --- core/src/main/scala/kafka/server/KafkaConfig.scala +++ core/src/main/scala/kafka/server/KafkaConfig.scala @@ -33,16 +33,32 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro } def verify() = props.verify() + + /*********** General Configuration ***********/ + + /* the broker id for this server */ + val brokerId: Int = props.getIntInRange("brokerid", (0, Int.MaxValue)) + /* the maximum size of message that the server can receive */ + val maxMessageSize = props.getIntInRange("max.message.size", 1000000, (0, Int.MaxValue)) + + /* the number of network threads that the server uses for handling network requests */ + val numNetworkThreads = props.getIntInRange("network.threads", 3, (1, Int.MaxValue)) + + /* the number of io threads that the server uses for carrying out network requests */ + val numIoThreads = props.getIntInRange("io.threads", 8, (1, Int.MaxValue)) + + /* the number of queued requests allowed before blocking the network threads */ + val numQueuedRequests = props.getIntInRange("max.queued.requests", 500, (1, Int.MaxValue)) + + /*********** Socket Server Configuration ***********/ + /* the port to listen and accept connections on */ val port: Int = props.getInt("port", 6667) /* hostname of broker. If not set, will pick up from the value returned from getLocalHost. If there are multiple interfaces getLocalHost may not be what you want. */ val hostName: String = props.getString("hostname", InetAddress.getLocalHost.getHostAddress) - /* the broker id for this server */ - val brokerId: Int = props.getIntInRange("brokerid", (0, Int.MaxValue)) - /* the SO_SNDBUFF buffer of the socket sever sockets */ val socketSendBuffer: Int = props.getInt("socket.send.buffer", 100*1024) @@ -51,18 +67,8 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* the maximum number of bytes in a socket request */ val maxSocketRequestSize: Int = props.getIntInRange("max.socket.request.bytes", 100*1024*1024, (1, Int.MaxValue)) - - /* the maximum size of message that the server can receive */ - val maxMessageSize = props.getIntInRange("max.message.size", 1000000, (0, Int.MaxValue)) - /* the number of network threads that the server uses for handling network requests */ - val numNetworkThreads = props.getIntInRange("network.threads", 3, (1, Int.MaxValue)) - - /* the number of io threads that the server uses for carrying out network requests */ - val numIoThreads = props.getIntInRange("io.threads", 8, (1, Int.MaxValue)) - - /* the number of queued requests allowed before blocking the network threads */ - val numQueuedRequests = props.getIntInRange("max.queued.requests", 500, (1, Int.MaxValue)) + /*********** Log Configuration ***********/ /* the default number of log partitions per topic */ val numPartitions = props.getIntInRange("num.partitions", 1, (1, Int.MaxValue)) @@ -96,6 +102,12 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* the frequency in minutes that the log cleaner checks whether any log is eligible for deletion */ val logCleanupIntervalMinutes = props.getIntInRange("log.cleanup.interval.mins", 10, (1, Int.MaxValue)) + + /* the maximum size in bytes of the offset index */ + val logIndexMaxSizeBytes = props.getIntInRange("log.index.max.size", 10*1024*1024, (4, Int.MaxValue)) + + /* the interval with which we add an entry to the offset index */ + val logIndexIntervalBytes = props.getIntInRange("log.index.interval.bytes", 4096, (0, Int.MaxValue)) /* the number of messages accumulated on a log partition before messages are flushed to disk */ val flushInterval = props.getIntInRange("log.flush.interval", 500, (1, Int.MaxValue)) @@ -112,9 +124,7 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* enable auto creation of topic on the server */ val autoCreateTopics = props.getBoolean("auto.create.topics", true) - /** - * Following properties are relevant to Kafka replication - */ + /*********** Replication configuration ***********/ /* the socket timeout for controller-to-broker channels */ val controllerSocketTimeoutMs = props.getInt("controller.socket.timeout.ms", 30000) @@ -122,7 +132,6 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro /* the buffer size for controller-to-broker-channels */ val controllerMessageQueueSize= props.getInt("controller.message.queue.size", 10) - /* default replication factors for automatically created topics */ val defaultReplicationFactor = props.getInt("default.replication.factor", 1) @@ -134,25 +143,22 @@ class KafkaConfig private (val props: VerifiableProperties) extends ZKConfig(pro val replicaMaxLagBytes = props.getLong("replica.max.lag.bytes", 4000) - /** - * Config options relevant to a follower for a replica - */ - /** the socket timeout for network requests */ + /* the socket timeout for network requests */ val replicaSocketTimeoutMs = props.getInt("replica.socket.timeout.ms", ConsumerConfig.SocketTimeout) - /** the socket receive buffer for network requests */ + /* the socket receive buffer for network requests */ val replicaSocketBufferSize = props.getInt("replica.socket.buffersize", ConsumerConfig.SocketBufferSize) - /** the number of byes of messages to attempt to fetch */ + /* the number of byes of messages to attempt to fetch */ val replicaFetchSize = props.getInt("replica.fetch.size", ConsumerConfig.FetchSize) - /** max wait time for each fetcher request issued by follower replicas*/ + /* max wait time for each fetcher request issued by follower replicas*/ val replicaMaxWaitTimeMs = props.getInt("replica.fetch.wait.time.ms", 500) - /** minimum bytes expected for each fetch response. If not enough bytes, wait up to replicaMaxWaitTimeMs */ + /* minimum bytes expected for each fetch response. If not enough bytes, wait up to replicaMaxWaitTimeMs */ val replicaMinBytes = props.getInt("replica.fetch.min.bytes", 4096) /* number of fetcher threads used to replicate messages from a source broker. - * Increasing this value can increase the degree of I/O parallelism in the follower broker. */ + * Increasing this value can increase the degree of I/O parallelism in the follower broker. */ val numReplicaFetchers = props.getInt("replica.fetchers", 1) } diff --git core/src/main/scala/kafka/server/KafkaServer.scala core/src/main/scala/kafka/server/KafkaServer.scala index 885e884..cc83198 100644 --- core/src/main/scala/kafka/server/KafkaServer.scala +++ core/src/main/scala/kafka/server/KafkaServer.scala @@ -114,11 +114,13 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime) extends Logg if(requestHandlerPool != null) requestHandlerPool.shutdown() kafkaScheduler.shutdown() - apis.close() - kafkaZookeeper.shutdown() + if(apis != null) + apis.close() + if(kafkaZookeeper != null) + kafkaZookeeper.shutdown() if(replicaManager != null) replicaManager.shutdown() - if (socketServer != null) + if(socketServer != null) socketServer.shutdown() if(logManager != null) logManager.shutdown() diff --git core/src/main/scala/kafka/server/ReplicaFetcherThread.scala core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 64cc2e8..4349cde 100644 --- core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -36,11 +36,11 @@ class ReplicaFetcherThread(name:String, sourceBroker: Broker, brokerConfig: Kafk val messageSet = partitionData.messages.asInstanceOf[ByteBufferMessageSet] if (fetchOffset != replica.logEndOffset) - throw new RuntimeException("offset mismatch: fetchOffset=%d, logEndOffset=%d".format(fetchOffset, replica.logEndOffset)) + throw new RuntimeException("Offset mismatch: fetched offset = %d, log end offset = %d.".format(fetchOffset, replica.logEndOffset)) trace("Follower %d has replica log end offset %d. Received %d messages and leader hw %d".format(replica.brokerId, replica.logEndOffset, messageSet.sizeInBytes, partitionData.hw)) replica.log.get.append(messageSet) - trace("Follower %d has replica log end offset %d after appending %d messages" + trace("Follower %d has replica log end offset %d after appending %d bytes of messages" .format(replica.brokerId, replica.logEndOffset, messageSet.sizeInBytes)) val followerHighWatermark = replica.logEndOffset.min(partitionData.hw) replica.highWatermark = followerHighWatermark diff --git core/src/main/scala/kafka/tools/DumpLogSegments.scala core/src/main/scala/kafka/tools/DumpLogSegments.scala index 986700b..529de9c 100644 --- core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -19,37 +19,58 @@ package kafka.tools import java.io._ import kafka.message._ +import kafka.log._ import kafka.utils._ object DumpLogSegments { def main(args: Array[String]) { - var isNoPrint = false; - for(arg <- args) - if ("-noprint".compareToIgnoreCase(arg) == 0) - isNoPrint = true; + val print = args.contains("--print") + val files = args.filter(_ != "--print") - for(arg <- args) { - if (! ("-noprint".compareToIgnoreCase(arg) == 0) ) { - val file = new File(arg) + for(arg <- files) { + val file = new File(arg) + if(file.getName.endsWith(Log.LogFileSuffix)) { println("Dumping " + file) - val startOffset = file.getName().split("\\.")(0).toLong - var offset = 0L - println("Starting offset: " + startOffset) - val messageSet = new FileMessageSet(file, false) - for(messageAndOffset <- messageSet) { - val msg = messageAndOffset.message - println("offset: " + (startOffset + offset) + " isvalid: " + msg.isValid + - " payloadsize: " + msg.payloadSize + " magic: " + msg.magic + " compresscodec: " + msg.compressionCodec) - if (!isNoPrint) - println("payload:\t" + Utils.toString(messageAndOffset.message.payload, "UTF-8")) - offset = messageAndOffset.offset - } - val endOffset = startOffset + offset - println("Tail of the log is at offset: " + endOffset) - if (messageSet.sizeInBytes != endOffset) - println("Log corrupted from " + endOffset + " to " + messageSet.sizeInBytes + "!!!") + dumpLog(file, print) + } else if(file.getName.endsWith(Log.IndexFileSuffix)){ + println("Dumping " + file) + dumpIndex(file) + } + } + } + + /* print out the contents of the index */ + def dumpIndex(file: File) { + val startOffset = file.getName().split("\\.")(0).toLong + val index = new OffsetIndex(file = file, baseOffset = startOffset, mutable = false) + for(i <- 0 until index.entries) { + val entry = index.entry(i) + // since it is a sparse file, in the event of a crash there may be many zero entries, stop if we see one + if(entry.offset <= startOffset) + return + println("offset: %d position: %d".format(entry.offset, entry.position)) + } + } + + /* print out the contents of the log */ + def dumpLog(file: File, printContents: Boolean) { + val startOffset = file.getName().split("\\.")(0).toLong + println("Starting offset: " + startOffset) + val messageSet = new FileMessageSet(file, false) + for(messageAndOffset <- messageSet) { + val msg = messageAndOffset.message + print("offset: " + messageAndOffset.offset + " isvalid: " + msg.isValid + + " payloadsize: " + msg.payloadSize + " magic: " + msg.magic + + " compresscodec: " + msg.compressionCodec + " crc: " + msg.checksum) + if(msg.hasKey) + print(" keysize: " + msg.keySize) + if(printContents) { + if(msg.hasKey) + print(" key: " + Utils.toString(messageAndOffset.message.payload, "UTF-8")) + print(" payload: " + Utils.toString(messageAndOffset.message.payload, "UTF-8")) } + println() } } diff --git core/src/main/scala/kafka/tools/SimpleConsumerShell.scala core/src/main/scala/kafka/tools/SimpleConsumerShell.scala index dc04180..f5f9dde 100644 --- core/src/main/scala/kafka/tools/SimpleConsumerShell.scala +++ core/src/main/scala/kafka/tools/SimpleConsumerShell.scala @@ -108,7 +108,7 @@ object SimpleConsumerShell extends Logging { for(messageAndOffset <- messageSet) { if(printMessages) info("consumed: " + Utils.toString(messageAndOffset.message.payload, "UTF-8")) - offset = messageAndOffset.offset + offset = messageAndOffset.nextOffset if(printOffsets) info("next offset = " + offset) consumed += 1 diff --git core/src/main/scala/kafka/utils/ZkUtils.scala core/src/main/scala/kafka/utils/ZkUtils.scala index 66332a4..9556b6e 100644 --- core/src/main/scala/kafka/utils/ZkUtils.scala +++ core/src/main/scala/kafka/utils/ZkUtils.scala @@ -295,11 +295,11 @@ object ZkUtils extends Logging { def conditionalUpdatePersistentPath(client: ZkClient, path: String, data: String, expectVersion: Int): (Boolean, Int) = { try { val stat = client.writeData(path, data, expectVersion) - info("Conditional update the zkPath %s with expected version %d succeed and return the new version: %d".format(path, expectVersion, stat.getVersion)) + debug("Conditional update to the zookeeper path %s with expected version %d succeeded and returned the new version: %d".format(path, expectVersion, stat.getVersion)) (true, stat.getVersion) } catch { case e: Exception => - info("Conditional update the zkPath %s with expected version %d failed".format(path, expectVersion)) + debug("Conditional update to the zookeeper path %s with expected version %d failed".format(path, expectVersion), e) (false, -1) } } diff --git core/src/test/scala/other/kafka/TestLogPerformance.scala core/src/test/scala/other/kafka/TestLogPerformance.scala index 625f03f..75c33e0 100644 --- core/src/test/scala/other/kafka/TestLogPerformance.scala +++ core/src/test/scala/other/kafka/TestLogPerformance.scala @@ -33,7 +33,7 @@ object TestLogPerformance { val props = TestUtils.createBrokerConfig(0, -1) val config = new KafkaConfig(props) val dir = TestUtils.tempDir() - val log = new Log(dir, 50*1024*1024, config.maxMessageSize, 5000000, config.logRollHours*60*60*1000L, false, SystemTime) + val log = new Log(dir, 50*1024*1024, config.maxMessageSize, 5000000, config.logRollHours*60*60*1000L, needsRecovery = false, time = SystemTime) val bytes = new Array[Byte](messageSize) new java.util.Random().nextBytes(bytes) val message = new Message(bytes) diff --git core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala new file mode 100644 index 0000000..0b02f2a --- /dev/null +++ core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -0,0 +1,225 @@ +/** + * 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.api + +import org.junit._ +import org.scalatest.junit.JUnitSuite +import junit.framework.Assert._ +import java.nio.ByteBuffer +import kafka.api._ +import kafka.message.{Message, ByteBufferMessageSet} +import kafka.cluster.Broker +import collection.mutable._ +import kafka.common.{TopicAndPartition, ErrorMapping} + + +object SerializationTestUtils{ + private val topic1 = "test1" + private val topic2 = "test2" + private val leader1 = 0 + private val isr1 = List(0, 1, 2) + private val leader2 = 0 + private val isr2 = List(0, 2, 3) + private val partitionData0 = new PartitionData(0, new ByteBufferMessageSet(new Message("first message".getBytes))) + private val partitionData1 = new PartitionData(1, new ByteBufferMessageSet(new Message("second message".getBytes))) + private val partitionData2 = new PartitionData(2, new ByteBufferMessageSet(new Message("third message".getBytes))) + private val partitionData3 = new PartitionData(3, new ByteBufferMessageSet(new Message("fourth message".getBytes))) + private val partitionDataArray = Array(partitionData0, partitionData1, partitionData2, partitionData3) + + private val topicData = { + val groupedData = Array(topic1, topic2).flatMap(topic => + partitionDataArray.map(partitionData => + (TopicAndPartition(topic, partitionData.partition), partitionData))) + collection.immutable.Map(groupedData:_*) + } + + private val requestInfos = collection.immutable.Map( + TopicAndPartition(topic1, 0) -> PartitionFetchInfo(1000, 100), + TopicAndPartition(topic1, 1) -> PartitionFetchInfo(2000, 100), + TopicAndPartition(topic1, 2) -> PartitionFetchInfo(3000, 100), + TopicAndPartition(topic1, 3) -> PartitionFetchInfo(4000, 100), + TopicAndPartition(topic2, 0) -> PartitionFetchInfo(1000, 100), + TopicAndPartition(topic2, 1) -> PartitionFetchInfo(2000, 100), + TopicAndPartition(topic2, 2) -> PartitionFetchInfo(3000, 100), + TopicAndPartition(topic2, 3) -> PartitionFetchInfo(4000, 100) + ) + + private val partitionMetaData0 = new PartitionMetadata(0, Some(new Broker(0, "creator", "localhost", 1011)), collection.immutable.Seq.empty) + private val partitionMetaData1 = new PartitionMetadata(1, Some(new Broker(0, "creator", "localhost", 1011)), collection.immutable.Seq.empty) + private val partitionMetaData2 = new PartitionMetadata(2, Some(new Broker(0, "creator", "localhost", 1011)), collection.immutable.Seq.empty) + private val partitionMetaData3 = new PartitionMetadata(3, Some(new Broker(0, "creator", "localhost", 1011)), collection.immutable.Seq.empty) + private val partitionMetaDataSeq = Seq(partitionMetaData0, partitionMetaData1, partitionMetaData2, partitionMetaData3) + private val topicmetaData1 = new TopicMetadata(topic1, partitionMetaDataSeq) + private val topicmetaData2 = new TopicMetadata(topic2, partitionMetaDataSeq) + + def createTestLeaderAndISRRequest() : LeaderAndIsrRequest = { + val leaderAndISR1 = new LeaderAndIsr(leader1, 1, isr1, 1) + val leaderAndISR2 = new LeaderAndIsr(leader2, 1, isr2, 2) + val map = Map(((topic1, 0), leaderAndISR1), + ((topic2, 0), leaderAndISR2)) + new LeaderAndIsrRequest(map) + } + + def createTestLeaderAndISRResponse() : LeaderAndISRResponse = { + val responseMap = Map(((topic1, 0), ErrorMapping.NoError), + ((topic2, 0), ErrorMapping.NoError)) + new LeaderAndISRResponse(1, responseMap) + } + + def createTestStopReplicaRequest() : StopReplicaRequest = { + new StopReplicaRequest(Set((topic1, 0), (topic2, 0))) + } + + def createTestStopReplicaResponse() : StopReplicaResponse = { + val responseMap = Map(((topic1, 0), ErrorMapping.NoError), + ((topic2, 0), ErrorMapping.NoError)) + new StopReplicaResponse(1, responseMap) + } + + def createTestProducerRequest: ProducerRequest = { + new ProducerRequest(1, "client 1", 0, 1000, topicData) + } + + def createTestProducerResponse: ProducerResponse = + ProducerResponse(1, 1, Map( + TopicAndPartition(topic1, 0) -> ProducerResponseStatus(0.toShort, 10001), + TopicAndPartition(topic2, 0) -> ProducerResponseStatus(0.toShort, 20001) + )) + + def createTestFetchRequest: FetchRequest = { + new FetchRequest(requestInfo = requestInfos) + } + + def createTestFetchResponse: FetchResponse = { + FetchResponse(1, 1, topicData) + } + + def createTestOffsetRequest = new OffsetRequest( + collection.immutable.Map(TopicAndPartition(topic1, 1) -> PartitionOffsetRequestInfo(1000, 200)), + replicaId = 0 + ) + + def createTestOffsetResponse: OffsetResponse = { + new OffsetResponse(OffsetRequest.CurrentVersion, collection.immutable.Map( + TopicAndPartition(topic1, 1) -> PartitionOffsetsResponse(ErrorMapping.NoError, Seq(1000l, 2000l, 3000l, 4000l))) + ) + } + + def createTestTopicMetadataRequest: TopicMetadataRequest = { + new TopicMetadataRequest(1, "client 1", Seq(topic1, topic2)) + } + + def createTestTopicMetadataResponse: TopicMetadataResponse = { + new TopicMetadataResponse(1, Seq(topicmetaData1, topicmetaData2)) + } +} + +class RequestResponseSerializationTest extends JUnitSuite { + private val leaderAndISRRequest = SerializationTestUtils.createTestLeaderAndISRRequest + private val leaderAndISRResponse = SerializationTestUtils.createTestLeaderAndISRResponse + private val stopReplicaRequest = SerializationTestUtils.createTestStopReplicaRequest + private val stopReplicaResponse = SerializationTestUtils.createTestStopReplicaResponse + private val producerRequest = SerializationTestUtils.createTestProducerRequest + private val producerResponse = SerializationTestUtils.createTestProducerResponse + private val fetchRequest = SerializationTestUtils.createTestFetchRequest + private val offsetRequest = SerializationTestUtils.createTestOffsetRequest + private val offsetResponse = SerializationTestUtils.createTestOffsetResponse + private val topicMetadataRequest = SerializationTestUtils.createTestTopicMetadataRequest + private val topicMetadataResponse = SerializationTestUtils.createTestTopicMetadataResponse + + + @Test + def testSerializationAndDeserialization() { + var buffer: ByteBuffer = ByteBuffer.allocate(leaderAndISRRequest.sizeInBytes()) + leaderAndISRRequest.writeTo(buffer) + buffer.rewind() + val deserializedLeaderAndISRRequest = LeaderAndIsrRequest.readFrom(buffer) + assertEquals("The original and deserialzed leaderAndISRRequest should be the same", leaderAndISRRequest, + deserializedLeaderAndISRRequest) + + buffer = ByteBuffer.allocate(leaderAndISRResponse.sizeInBytes()) + leaderAndISRResponse.writeTo(buffer) + buffer.rewind() + val deserializedLeaderAndISRResponse = LeaderAndISRResponse.readFrom(buffer) + assertEquals("The original and deserialzed leaderAndISRResponse should be the same", leaderAndISRResponse, + deserializedLeaderAndISRResponse) + + buffer = ByteBuffer.allocate(stopReplicaRequest.sizeInBytes()) + stopReplicaRequest.writeTo(buffer) + buffer.rewind() + val deserializedStopReplicaRequest = StopReplicaRequest.readFrom(buffer) + assertEquals("The original and deserialzed stopReplicaRequest should be the same", stopReplicaRequest, + deserializedStopReplicaRequest) + + buffer = ByteBuffer.allocate(stopReplicaResponse.sizeInBytes()) + stopReplicaResponse.writeTo(buffer) + buffer.rewind() + val deserializedStopReplicaResponse = StopReplicaResponse.readFrom(buffer) + assertEquals("The original and deserialzed stopReplicaResponse should be the same", stopReplicaResponse, + deserializedStopReplicaResponse) + + buffer = ByteBuffer.allocate(producerRequest.sizeInBytes) + producerRequest.writeTo(buffer) + buffer.rewind() + val deserializedProducerRequest = ProducerRequest.readFrom(buffer) + assertEquals("The original and deserialzed producerRequest should be the same", producerRequest, + deserializedProducerRequest) + + buffer = ByteBuffer.allocate(producerResponse.sizeInBytes) + producerResponse.writeTo(buffer) + buffer.rewind() + val deserializedProducerResponse = ProducerResponse.readFrom(buffer) + assertEquals("The original and deserialzed producerResponse should be the same: [%s], [%s]".format(producerResponse, deserializedProducerResponse), producerResponse, + deserializedProducerResponse) + + buffer = ByteBuffer.allocate(fetchRequest.sizeInBytes) + fetchRequest.writeTo(buffer) + buffer.rewind() + val deserializedFetchRequest = FetchRequest.readFrom(buffer) + assertEquals("The original and deserialzed fetchRequest should be the same", fetchRequest, + deserializedFetchRequest) + + buffer = ByteBuffer.allocate(offsetRequest.sizeInBytes) + offsetRequest.writeTo(buffer) + buffer.rewind() + val deserializedOffsetRequest = OffsetRequest.readFrom(buffer) + assertEquals("The original and deserialzed offsetRequest should be the same", offsetRequest, + deserializedOffsetRequest) + + buffer = ByteBuffer.allocate(offsetResponse.sizeInBytes) + offsetResponse.writeTo(buffer) + buffer.rewind() + val deserializedOffsetResponse = OffsetResponse.readFrom(buffer) + assertEquals("The original and deserialzed offsetResponse should be the same", offsetResponse, + deserializedOffsetResponse) + + buffer = ByteBuffer.allocate(topicMetadataRequest.sizeInBytes()) + topicMetadataRequest.writeTo(buffer) + buffer.rewind() + val deserializedTopicMetadataRequest = TopicMetadataRequest.readFrom(buffer) + assertEquals("The original and deserialzed topicMetadataRequest should be the same", topicMetadataRequest, + deserializedTopicMetadataRequest) + + buffer = ByteBuffer.allocate(topicMetadataResponse.sizeInBytes) + topicMetadataResponse.writeTo(buffer) + buffer.rewind() + val deserializedTopicMetadataResponse = TopicMetadataResponse.readFrom(buffer) + assertEquals("The original and deserialzed topicMetadataResponse should be the same", topicMetadataResponse, + deserializedTopicMetadataResponse) + } +} diff --git core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala index 0cfa89c..b52b159 100644 --- core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala +++ core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala @@ -348,7 +348,7 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar val receivedMessages1 = getMessages(nMessages, topicMessageStreams1) assertEquals(nMessages, receivedMessages1.size) - assertEquals(sentMessages1, receivedMessages1) + assertEquals(sentMessages1.sortWith((s,t) => s.checksum < t.checksum), receivedMessages1) } def sendMessagesToBrokerPartition(config: KafkaConfig, topic: String, partition: Int, numMessages: Int, compression: CompressionCodec = NoCompressionCodec): List[Message] = { @@ -395,10 +395,10 @@ class ZookeeperConsumerConnectorTest extends JUnit3Suite with KafkaServerTestHar def getMessages(nMessagesPerThread: Int, topicMessageStreams: Map[String,List[KafkaStream[Message]]]): List[Message]= { var messages: List[Message] = Nil - for ((topic, messageStreams) <- topicMessageStreams) { + for((topic, messageStreams) <- topicMessageStreams) { for (messageStream <- messageStreams) { val iterator = messageStream.iterator - for (i <- 0 until nMessagesPerThread) { + for(i <- 0 until nMessagesPerThread) { assertTrue(iterator.hasNext) val message = iterator.next.message messages ::= message diff --git core/src/test/scala/unit/kafka/integration/FetcherTest.scala core/src/test/scala/unit/kafka/integration/FetcherTest.scala index 6362e2b..4af7a52 100644 --- core/src/test/scala/unit/kafka/integration/FetcherTest.scala +++ core/src/test/scala/unit/kafka/integration/FetcherTest.scala @@ -15,7 +15,7 @@ * limitations under the License. */ -package kafka.consumer +package kafka.integration import java.util.concurrent._ import java.util.concurrent.atomic._ @@ -26,7 +26,7 @@ import kafka.cluster._ import kafka.message._ import kafka.server._ import org.scalatest.junit.JUnit3Suite -import kafka.integration.KafkaServerTestHarness +import kafka.consumer._ import kafka.producer.{ProducerData, Producer} import kafka.utils.TestUtils._ import kafka.utils.TestUtils diff --git core/src/test/scala/unit/kafka/integration/LazyInitProducerTest.scala core/src/test/scala/unit/kafka/integration/LazyInitProducerTest.scala index 868bf02..e460d63 100644 --- core/src/test/scala/unit/kafka/integration/LazyInitProducerTest.scala +++ core/src/test/scala/unit/kafka/integration/LazyInitProducerTest.scala @@ -103,7 +103,7 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness // send some invalid offsets val builder = new FetchRequestBuilder() - for( (topic, offset) <- topicOffsets ) + for((topic, offset) <- topicOffsets) builder.addFetch(topic, offset, -1, 10000) val request = builder.build() @@ -113,8 +113,7 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness ErrorMapping.maybeThrowException(pd.error) fail("Expected an OffsetOutOfRangeException exception to be thrown") } catch { - case e: OffsetOutOfRangeException => - + case e: OffsetOutOfRangeException => // this is good } }) } diff --git core/src/test/scala/unit/kafka/integration/LogCorruptionTest.scala core/src/test/scala/unit/kafka/integration/LogCorruptionTest.scala deleted file mode 100644 index d1cd847..0000000 --- core/src/test/scala/unit/kafka/integration/LogCorruptionTest.scala +++ /dev/null @@ -1,100 +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 kafka.server.KafkaConfig -import java.io.File -import java.nio.ByteBuffer -import kafka.api.FetchRequestBuilder -import kafka.common.InvalidMessageSizeException -import kafka.consumer.{ZookeeperConsumerConnector, ConsumerConfig} -import kafka.integration.{KafkaServerTestHarness, ProducerConsumerTestHarness} -import kafka.message.Message -import kafka.utils.{Utils, TestUtils} -import org.scalatest.junit.JUnit3Suite -import org.apache.log4j.{Logger, Level} -import kafka.producer.ProducerData - -class LogCorruptionTest extends JUnit3Suite with ProducerConsumerTestHarness with KafkaServerTestHarness { - val port = TestUtils.choosePort - val props = TestUtils.createBrokerConfig(0, port) - val config = new KafkaConfig(props) { - override val hostName = "localhost" - } - val configs = List(config) - val topic = "test" - val partition = 0 - - def testMessageSizeTooLarge() { - val requestHandlerLogger = Logger.getLogger(classOf[kafka.server.KafkaRequestHandler]) - - requestHandlerLogger.setLevel(Level.FATAL) - - // send some messages - val producerData = new ProducerData[String, Message](topic, topic, List(new Message("hello".getBytes()))) - - producer.send(producerData) - - // corrupt the file on disk - val logFile = new File(config.logDir + File.separator + topic + "-" + partition, Log.nameFromOffset(0)) - val byteBuffer = ByteBuffer.allocate(4) - byteBuffer.putInt(1000) // wrong message size - byteBuffer.rewind() - val channel = Utils.openChannel(logFile, true) - channel.write(byteBuffer) - channel.force(true) - channel.close - - // test SimpleConsumer - val response = consumer.fetch(new FetchRequestBuilder().addFetch(topic, partition, 0, 10000).build()) - try { - for (msg <- response.messageSet(topic, partition)) - fail("shouldn't reach here in SimpleConsumer since log file is corrupted.") - fail("shouldn't reach here in SimpleConsumer since log file is corrupted.") - } catch { - case e: InvalidMessageSizeException => "This is good" - } - - val response2 = consumer.fetch(new FetchRequestBuilder().addFetch(topic, partition, 0, 10000).build()) - try { - for (msg <- response2.messageSet(topic, partition)) - fail("shouldn't reach here in SimpleConsumer since log file is corrupted.") - fail("shouldn't reach here in SimpleConsumer since log file is corrupted.") - } catch { - case e: InvalidMessageSizeException => "This is good" - } - - // test ZookeeperConsumer - val consumerConfig1 = new ConsumerConfig( - TestUtils.createConsumerProperties(zkConnect, "group1", "consumer1", 10000)) - val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1) - val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Predef.Map(topic -> 1)) - try { - for ((topic, messageStreams) <- topicMessageStreams1) - for (message <- messageStreams(0)) - fail("shouldn't reach here in ZookeeperConsumer since log file is corrupted.") - fail("shouldn't reach here in ZookeeperConsumer since log file is corrupted.") - } catch { - case e: InvalidMessageSizeException => "This is good" - case e: Exception => "This is not bad too !" - } - - zkConsumerConnector1.shutdown - requestHandlerLogger.setLevel(Level.ERROR) - } -} diff --git core/src/test/scala/unit/kafka/javaapi/message/ByteBufferMessageSetTest.scala core/src/test/scala/unit/kafka/javaapi/message/ByteBufferMessageSetTest.scala index ac767c7..383fcef 100644 --- core/src/test/scala/unit/kafka/javaapi/message/ByteBufferMessageSetTest.scala +++ core/src/test/scala/unit/kafka/javaapi/message/ByteBufferMessageSetTest.scala @@ -22,8 +22,9 @@ import org.junit.Test import kafka.message.{DefaultCompressionCodec, CompressionCodec, NoCompressionCodec, Message} class ByteBufferMessageSetTest extends kafka.javaapi.message.BaseMessageSetTestCases { + override def createMessageSet(messages: Seq[Message], compressed: CompressionCodec = NoCompressionCodec): ByteBufferMessageSet = - new ByteBufferMessageSet(new kafka.message.ByteBufferMessageSet(compressed, messages: _*)) + new ByteBufferMessageSet(new kafka.message.ByteBufferMessageSet(compressed, messages: _*).buffer) val msgSeq: Seq[Message] = Seq(new Message("hello".getBytes()), new Message("there".getBytes())) diff --git core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala new file mode 100644 index 0000000..f06e537 --- /dev/null +++ core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala @@ -0,0 +1,107 @@ +/** + * 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._ +import java.util.concurrent.atomic._ +import junit.framework.Assert._ +import kafka.utils.TestUtils._ +import kafka.message._ +import org.junit.Test + +class FileMessageSetTest extends BaseMessageSetTestCases { + + val messageSet = createMessageSet(messages) + + def createMessageSet(messages: Seq[Message]): FileMessageSet = { + val set = new FileMessageSet(tempFile(), true) + set.append(new ByteBufferMessageSet(NoCompressionCodec, messages: _*)) + set.flush() + set + } + + @Test + def testFileSize() { + assertEquals(messageSet.channel.size, messageSet.sizeInBytes) + messageSet.append(singleMessageSet("abcd".getBytes())) + assertEquals(messageSet.channel.size, messageSet.sizeInBytes) + } + + @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) + val originalPosition = messageSet.channel.position + for(i <- 0 until size) + buffer.put(0.asInstanceOf[Byte]) + buffer.rewind() + messageSet.channel.write(buffer) + // appending those bytes should not change the contents + checkEquals(messages.iterator, messageSet.map(m => m.message).iterator) + } + + @Test + def testIterationDoesntChangePosition() { + val position = messageSet.channel.position + checkEquals(messages.iterator, messageSet.map(m => m.message).iterator) + assertEquals(position, messageSet.channel.position) + } + + @Test + def testRead() { + val read = messageSet.read(0, messageSet.sizeInBytes) + checkEquals(messageSet.iterator, read.iterator) + val items = read.iterator.toList + val sec = items.tail.head + val read2 = messageSet.read(MessageSet.entrySize(sec.message), messageSet.sizeInBytes) + checkEquals(items.tail.iterator, read2.iterator) + } + + @Test + def testSearch() { + // append a new message with a high offset + val lastMessage = new Message("test".getBytes) + messageSet.append(new ByteBufferMessageSet(NoCompressionCodec, new AtomicLong(50), lastMessage)) + var physicalOffset = 0 + assertEquals("Should be able to find the first message by its offset", + OffsetPosition(0L, physicalOffset), + messageSet.searchFor(0, 0)) + physicalOffset += MessageSet.entrySize(messageSet.head.message) + assertEquals("Should be able to find second message when starting from 0", + OffsetPosition(1L, physicalOffset), + messageSet.searchFor(1, 0)) + assertEquals("Should be able to find second message starting from its offset", + OffsetPosition(1L, physicalOffset), + messageSet.searchFor(1, physicalOffset)) + physicalOffset += MessageSet.entrySize(messageSet.tail.head.message) + assertEquals("Should be able to find third message from a non-existant offset", + OffsetPosition(50L, physicalOffset), + messageSet.searchFor(3, physicalOffset)) + assertEquals("Should be able to find third message by correct offset", + OffsetPosition(50L, physicalOffset), + messageSet.searchFor(50, physicalOffset)) + } + +} diff --git core/src/test/scala/unit/kafka/log/LogManagerTest.scala core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 6bf7221..cf304b5 100644 --- core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -27,7 +27,7 @@ import kafka.admin.CreateTopicCommand import kafka.server.KafkaConfig import kafka.utils._ -class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness { +class LogManagerTest extends JUnit3Suite { val time: MockTime = new MockTime() val maxRollInterval = 100 @@ -35,15 +35,13 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness { var logDir: File = null var logManager: LogManager = null var config:KafkaConfig = null - val zookeeperConnect = TestZKUtils.zookeeperConnect val name = "kafka" val veryLargeLogFlushInterval = 10000000L val scheduler = new KafkaScheduler(2) override def setUp() { super.setUp() - val props = TestUtils.createBrokerConfig(0, -1) - config = new KafkaConfig(props) { + config = new KafkaConfig(TestUtils.createBrokerConfig(0, -1)) { override val logFileSize = 1024 override val flushInterval = 100 } @@ -51,11 +49,6 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness { logManager = new LogManager(config, scheduler, time, maxRollInterval, veryLargeLogFlushInterval, maxLogAge, false) logManager.startup logDir = logManager.logDir - - TestUtils.createBrokersInZk(zkClient, List(config.brokerId)) - - // setup brokers in zookeeper as owners of partitions for this test - CreateTopicCommand.createTopic(zkClient, name, 3, 1, "0,0,0") } override def tearDown() { @@ -87,8 +80,8 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness { var offset = 0L for(i <- 0 until 1000) { var set = TestUtils.singleMessageSet("test".getBytes()) - log.append(set) - offset += set.sizeInBytes + val (start, end) = log.append(set) + offset = end } log.flush @@ -96,12 +89,12 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness { // update the last modified time of all log segments val logSegments = log.segments.view - logSegments.foreach(s => s.file.setLastModified(time.currentMs)) + logSegments.foreach(s => s.messageSet.file.setLastModified(time.currentMs)) time.currentMs += maxLogAge + 3000 logManager.cleanupLogs() assertEquals("Now there should only be only one segment.", 1, log.numberOfSegments) - assertEquals("Should get empty fetch off new log.", 0L, log.read(offset, 1024).sizeInBytes) + assertEquals("Should get empty fetch off new log.", 0L, log.read(offset+1, 1024).sizeInBytes) try { log.read(0, 1024) fail("Should get exception from fetching earlier.") @@ -135,8 +128,8 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness { // add a bunch of messages that should be larger than the retentionSize for(i <- 0 until 1000) { val set = TestUtils.singleMessageSet("test".getBytes()) - log.append(set) - offset += set.sizeInBytes + val (start, end) = log.append(set) + offset = start } // flush to make sure it's written to disk log.flush @@ -147,7 +140,7 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness { // this cleanup shouldn't find any expired segments but should delete some to reduce size logManager.cleanupLogs() assertEquals("Now there should be exactly 6 segments", 6, log.numberOfSegments) - assertEquals("Should get empty fetch off new log.", 0L, log.read(offset, 1024).sizeInBytes) + assertEquals("Should get empty fetch off new log.", 0L, log.read(offset + 1, 1024).sizeInBytes) try { log.read(0, 1024) fail("Should get exception from fetching earlier.") @@ -175,8 +168,8 @@ class LogManagerTest extends JUnit3Suite with ZooKeeperTestHarness { var set = TestUtils.singleMessageSet("test".getBytes()) log.append(set) } - + println("now = " + System.currentTimeMillis + " last flush = " + log.getLastFlushedTime) assertTrue("The last flush time has to be within defaultflushInterval of current time ", - (System.currentTimeMillis - log.getLastFlushedTime) < 100) + (System.currentTimeMillis - log.getLastFlushedTime) < 150) } } diff --git core/src/test/scala/unit/kafka/log/LogOffsetTest.scala core/src/test/scala/unit/kafka/log/LogOffsetTest.scala index fd248ee..c39656f 100644 --- core/src/test/scala/unit/kafka/log/LogOffsetTest.scala +++ core/src/test/scala/unit/kafka/log/LogOffsetTest.scala @@ -92,7 +92,7 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { log.flush() val offsets = log.getOffsetsBefore(OffsetRequest.LatestTime, 10) - assertEquals(Seq(240L, 216L, 108L, 0L), offsets) + assertEquals(Seq(20L, 15L, 10L, 5L, 0L), offsets) waitUntilTrue(() => isLeaderLocalOnBroker(topic, part, server), 1000) val topicAndPartition = TopicAndPartition(topic, part) @@ -101,7 +101,7 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { replicaId = 0) val consumerOffsets = simpleConsumer.getOffsetsBefore(offsetRequest).partitionErrorAndOffsets(topicAndPartition).offsets - assertEquals(Seq(240L, 216L, 108L, 0L), consumerOffsets) + assertEquals(Seq(20L, 15L, 10L, 5L, 0L), consumerOffsets) // try to fetch using latest offset val fetchResponse = simpleConsumer.fetch( @@ -157,14 +157,14 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness { val now = time.milliseconds val offsets = log.getOffsetsBefore(now, 10) - assertEquals(Seq(240L, 216L, 108L, 0L), offsets) + assertEquals(Seq(20L, 15L, 10L, 5L, 0L), offsets) waitUntilTrue(() => isLeaderLocalOnBroker(topic, part, server), 1000) val topicAndPartition = TopicAndPartition(topic, part) val offsetRequest = OffsetRequest(Map(topicAndPartition -> PartitionOffsetRequestInfo(now, 10)), replicaId = 0) val consumerOffsets = simpleConsumer.getOffsetsBefore(offsetRequest).partitionErrorAndOffsets(topicAndPartition).offsets - assertEquals(Seq(240L, 216L, 108L, 0L), consumerOffsets) + assertEquals(Seq(20L, 15L, 10L, 5L, 0L), consumerOffsets) } @Test diff --git core/src/test/scala/unit/kafka/log/LogSegmentTest.scala core/src/test/scala/unit/kafka/log/LogSegmentTest.scala new file mode 100644 index 0000000..6383a90 --- /dev/null +++ core/src/test/scala/unit/kafka/log/LogSegmentTest.scala @@ -0,0 +1,104 @@ +package kafka.log + +import junit.framework.Assert._ +import java.util.concurrent.atomic._ +import org.junit.{Test, Before, After} +import org.scalatest.junit.JUnit3Suite +import kafka.utils.TestUtils +import kafka.message._ +import kafka.utils.SystemTime +import scala.collection._ + +class LogSegmentTest extends JUnit3Suite { + + val segments = mutable.ArrayBuffer[LogSegment]() + + def createSegment(offset: Long): LogSegment = { + val msFile = TestUtils.tempFile() + val ms = new FileMessageSet(msFile, true) + val idxFile = TestUtils.tempFile() + val idx = new OffsetIndex(idxFile, offset, true, 100) + val seg = new LogSegment(ms, idx, offset, 10, SystemTime) + segments += seg + seg + } + + def messages(offset: Long, messages: String*): ByteBufferMessageSet = { + new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, + offsetCounter = new AtomicLong(offset), + messages = messages.map(s => new Message(s.getBytes)):_*) + } + + @After + def teardown() { + for(seg <- segments) { + seg.index.delete() + seg.messageSet.delete() + } + } + + @Test + def testReadOnEmptySegment() { + val seg = createSegment(40) + val read = seg.read(startOffset = 40, maxSize = 300, maxOffset = None) + assertEquals(0, read.size) + } + + @Test + def testReadBeforeFirstOffset() { + val seg = createSegment(40) + val ms = messages(50, "hello", "there", "little", "bee") + seg.append(50, ms) + val read = seg.read(startOffset = 41, maxSize = 300, maxOffset = None) + assertEquals(ms.toList, read.toList) + } + + @Test + def testReadSingleMessage() { + val seg = createSegment(40) + val ms = messages(50, "hello", "there") + seg.append(50, ms) + val read = seg.read(startOffset = 41, maxSize = 200, maxOffset = Some(50)) + assertEquals(new Message("hello".getBytes), read.head.message) + } + + @Test + def testReadAfterLast() { + val seg = createSegment(40) + val ms = messages(50, "hello", "there") + seg.append(50, ms) + val read = seg.read(startOffset = 52, maxSize = 200, maxOffset = None) + assertEquals(0, read.size) + } + + @Test + def testReadFromGap() { + val seg = createSegment(40) + val ms = messages(50, "hello", "there") + seg.append(50, ms) + val ms2 = messages(60, "alpha", "beta") + seg.append(60, ms2) + val read = seg.read(startOffset = 55, maxSize = 200, maxOffset = None) + assertEquals(ms2.toList, read.toList) + } + + @Test + def testTruncate() { + val seg = createSegment(40) + val ms = messages(50, "hello", "there", "you") + seg.append(50, ms) + seg.truncateTo(51) + val read = seg.read(50, maxSize = 1000, None) + assertEquals(1, read.size) + assertEquals(ms.head, read.head) + } + + @Test + def testNextOffsetCalculation() { + val seg = createSegment(40) + assertEquals(40, seg.nextOffset) + seg.append(50, messages(50, "hello", "there", "you")) + assertEquals(53, seg.nextOffset()) + } + +} \ No newline at end of file diff --git core/src/test/scala/unit/kafka/log/LogTest.scala core/src/test/scala/unit/kafka/log/LogTest.scala index afda54b..ba367ca 100644 --- core/src/test/scala/unit/kafka/log/LogTest.scala +++ core/src/test/scala/unit/kafka/log/LogTest.scala @@ -22,7 +22,7 @@ import java.util.ArrayList import junit.framework.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.{After, Before, Test} -import kafka.message.{NoCompressionCodec, ByteBufferMessageSet, Message} +import kafka.message._ import kafka.common.{MessageSizeTooLargeException, KafkaException, OffsetOutOfRangeException} import kafka.utils._ import scala.Some @@ -46,9 +46,11 @@ class LogTest extends JUnitSuite { Utils.rm(logDir) } - def createEmptyLogs(dir: File, offsets: Int*) = { - for(offset <- offsets) - new File(dir, Integer.toString(offset) + Log.FileSuffix).createNewFile() + def createEmptyLogs(dir: File, offsets: Int*) { + for(offset <- offsets) { + Log.logFilename(dir, offset).createNewFile() + Log.indexFilename(dir, offset).createNewFile() + } } /** Test that the size and time based log segment rollout works. */ @@ -59,7 +61,7 @@ class LogTest extends JUnitSuite { val time: MockTime = new MockTime() // create a log - val log = new Log(logDir, 1000, config.maxMessageSize, 1000, rollMs, false, time) + val log = new Log(logDir, 1000, config.maxMessageSize, 1000, rollMs, needsRecovery = false, time = time) time.currentMs += rollMs + 1 // segment age is less than its limit @@ -76,12 +78,12 @@ class LogTest extends JUnitSuite { time.currentMs += rollMs + 1 val blank = Array[Message]() - log.append(new ByteBufferMessageSet(blank:_*)) + log.append(new ByteBufferMessageSet(new Message("blah".getBytes))) assertEquals("There should be exactly 3 segments.", 3, log.numberOfSegments) time.currentMs += rollMs + 1 // the last segment expired in age, but was blank. So new segment should not be generated - log.append(set) + log.append(new ByteBufferMessageSet()) assertEquals("There should be exactly 3 segments.", 3, log.numberOfSegments) } @@ -93,7 +95,7 @@ class LogTest extends JUnitSuite { val logFileSize = msgPerSeg * (setSize - 1).asInstanceOf[Int] // each segment will be 10 messages // create a log - val log = new Log(logDir, logFileSize, config.maxMessageSize, 1000, 10000, false, time) + val log = new Log(logDir, logFileSize, config.maxMessageSize, 1000, 10000, needsRecovery = false, time = time) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) // segments expire in size @@ -106,23 +108,12 @@ class LogTest extends JUnitSuite { @Test def testLoadEmptyLog() { createEmptyLogs(logDir, 0) - new Log(logDir, 1024, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, false, time) - } - - @Test - def testLoadInvalidLogsFails() { - createEmptyLogs(logDir, 0, 15) - try { - new Log(logDir, 1024, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, false, time) - fail("Allowed load of corrupt logs without complaint.") - } catch { - case e: KafkaException => "This is good" - } + new Log(logDir, 1024, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time) } @Test def testAppendAndRead() { - val log = new Log(logDir, 1024, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, false, time) + val log = new Log(logDir, 1024, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time) val message = new Message(Integer.toString(42).getBytes()) for(i <- 0 until 10) log.append(new ByteBufferMessageSet(NoCompressionCodec, message)) @@ -139,7 +130,7 @@ class LogTest extends JUnitSuite { @Test def testReadOutOfRange() { createEmptyLogs(logDir, 1024) - val log = new Log(logDir, 1024, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, false, time) + val log = new Log(logDir, 1024, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time) assertEquals("Reading just beyond end of log should produce 0 byte read.", 0L, log.read(1024, 1000).sizeInBytes) try { log.read(0, 1024) @@ -159,95 +150,96 @@ class LogTest extends JUnitSuite { @Test def testLogRolls() { /* create a multipart log with 100 messages */ - val log = new Log(logDir, 100, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, false, time) + val log = new Log(logDir, 100, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time) val numMessages = 100 - for(i <- 0 until numMessages) - log.append(TestUtils.singleMessageSet(Integer.toString(i).getBytes())) + val messageSets = (0 until numMessages).map(i => TestUtils.singleMessageSet(i.toString.getBytes)) + val offsets = messageSets.map(log.append(_)._1) log.flush - /* now do successive reads and iterate over the resulting message sets counting the messages - * we should find exact 100 messages. - */ - var reads = 0 - var current = 0 + /* do successive reads to ensure all our messages are there */ var offset = 0L - var readOffset = 0L - while(current < numMessages) { - val messages = log.read(readOffset, 1024*1024) - readOffset += messages.last.offset - current += messages.size - if(reads > 2*numMessages) - fail("Too many read attempts.") - reads += 1 + for(i <- 0 until numMessages) { + val messages = log.read(offset, 1024*1024) + assertEquals("Offsets not equal", offset, messages.head.offset) + assertEquals("Messages not equal at offset " + offset, messageSets(i).head.message, messages.head.message) + offset = messages.head.offset + 1 } - assertEquals("We did not find all the messages we put in", numMessages, current) + val lastRead = log.read(startOffset = numMessages, maxLength = 1024*1024, maxOffset = Some(numMessages + 1)) + assertEquals("Should be no more messages", 0, lastRead.size) + } + + /** Test the case where we have compressed batches of messages */ + @Test + def testCompressedMessages() { + /* this log should roll after every messageset */ + val log = new Log(logDir, 10, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, 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))) + + def read(offset: Int) = ByteBufferMessageSet.decompress(log.read(offset, 4096).head.message) + + /* 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).head.offset) + assertEquals("Read at offset 1 should produce 0", 0, read(1).head.offset) + assertEquals("Read at offset 2 should produce 2", 2, read(2).head.offset) + assertEquals("Read at offset 3 should produce 2", 2, read(3).head.offset) } @Test def testFindSegment() { assertEquals("Search in empty segments list should find nothing", None, Log.findRange(makeRanges(), 45)) - assertEquals("Search in segment list just outside the range of the last segment should find nothing", - None, Log.findRange(makeRanges(5, 9, 12), 12)) - try { - Log.findRange(makeRanges(35), 36) - fail("expect exception") - } - catch { - case e: OffsetOutOfRangeException => "this is good" - } - - try { - Log.findRange(makeRanges(35,35), 36) - } - catch { - case e: OffsetOutOfRangeException => "this is good" - } - + assertEquals("Search in segment list just outside the range of the last segment should find last segment", + 9, Log.findRange(makeRanges(5, 9, 12), 12).get.start) + assertEquals("Search in segment list far outside the range of the last segment should find last segment", + 9, Log.findRange(makeRanges(5, 9, 12), 100).get.start) + assertEquals("Search in segment list far outside the range of the last segment should find last segment", + None, Log.findRange(makeRanges(5, 9, 12), -1)) assertContains(makeRanges(5, 9, 12), 11) assertContains(makeRanges(5), 4) assertContains(makeRanges(5,8), 5) assertContains(makeRanges(5,8), 6) } - /** Test corner cases of rolling logs */ @Test - def testEdgeLogRolls() { - { - // first test a log segment starting at 0 - val log = new Log(logDir, 100, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, false, time) - val curOffset = log.logEndOffset - assertEquals(curOffset, 0) - - // time goes by; the log file is deleted - log.markDeletedWhile(_ => true) - - // we now have a new log; the starting offset of the new log should remain 0 - assertEquals(curOffset, log.logEndOffset) - } - - { - // second test an empty log segment starting at none-zero - val log = new Log(logDir, 100, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, false, time) - val numMessages = 1 - for(i <- 0 until numMessages) - log.append(TestUtils.singleMessageSet(Integer.toString(i).getBytes())) + def testEdgeLogRollsStartingAtZero() { + // first test a log segment starting at 0 + val log = new Log(logDir, 100, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time) + val curOffset = log.logEndOffset + assertEquals(curOffset, 0) + + // time goes by; the log file is deleted + log.markDeletedWhile(_ => true) + + // we now have a new log; the starting offset of the new log should remain 0 + assertEquals(curOffset, log.logEndOffset) + log.delete() + } - val curOffset = log.logEndOffset - // time goes by; the log file is deleted - log.markDeletedWhile(_ => true) + @Test + def testEdgeLogRollsStartingAtNonZero() { + // second test an empty log segment starting at non-zero + val log = new Log(logDir, 100, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time) + val numMessages = 1 + for(i <- 0 until numMessages) + log.append(TestUtils.singleMessageSet(i.toString.getBytes)) + val curOffset = log.logEndOffset + + // time goes by; the log file is deleted + log.markDeletedWhile(_ => true) - // we now have a new log - assertEquals(curOffset, log.logEndOffset) + // we now have a new log + assertEquals(curOffset, log.logEndOffset) - // time goes by; the log file (which is empty) is deleted again - val deletedSegments = log.markDeletedWhile(_ => true) + // time goes by; the log file (which is empty) is deleted again + val deletedSegments = log.markDeletedWhile(_ => true) - // we shouldn't delete the last empty log segment. - assertTrue("We shouldn't delete the last empty log segment", deletedSegments.size == 0) + // we shouldn't delete the last empty log segment. + assertTrue("We shouldn't delete the last empty log segment", deletedSegments.size == 0) - // we now have a new log - assertEquals(curOffset, log.logEndOffset) - } + // we now have a new log + assertEquals(curOffset, log.logEndOffset) } @Test @@ -256,27 +248,38 @@ class LogTest extends JUnitSuite { val second = new ByteBufferMessageSet(NoCompressionCodec, new Message("change".getBytes())) // append messages to log - val log = new Log(logDir, 100, 5, 1000, config.logRollHours*60*60*1000L, false, time) - - var ret = - try { - log.append(first) - true - } - catch { - case e: MessageSizeTooLargeException => false - } - assert(ret, "First messageset should pass.") - - ret = - try { - log.append(second) - false - } - catch { - case e:MessageSizeTooLargeException => true - } - assert(ret, "Second message set should throw MessageSizeTooLargeException.") + val maxMessageSize = second.sizeInBytes - 1 + val log = new Log(logDir, 100, maxMessageSize.toInt, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time) + + // should be able to append the small message + log.append(first) + + try { + log.append(second) + fail("Second message set should throw MessageSizeTooLargeException.") + } catch { + case e:MessageSizeTooLargeException => // this is good + } + } + + @Test + def testLogRecoversToCorrectOffset() { + val numMessages = 100 + var log = new Log(logDir, 100, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time) + for(i <- 0 until numMessages) + log.append(TestUtils.singleMessageSet(i.toString.getBytes)) + assertEquals("After appending %d messages to an empty log, the log end offset should be %d".format(numMessages, numMessages), numMessages, log.logEndOffset) + log.close() + + // test non-recovery case + log = new Log(logDir, 100, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = false, time = time) + assertEquals("Should have %d messages when log is reopened w/o recovery".format(numMessages), numMessages, log.logEndOffset) + log.close() + + // test + log = new Log(logDir, 100, config.maxMessageSize, 1000, config.logRollHours*60*60*1000L, needsRecovery = true, time = time) + assertEquals("Should have %d messages when log is reopened with recovery".format(numMessages), numMessages, log.logEndOffset) + log.close() } @Test @@ -287,14 +290,15 @@ class LogTest extends JUnitSuite { val logFileSize = msgPerSeg * (setSize - 1).asInstanceOf[Int] // each segment will be 10 messages // create a log - val log = new Log(logDir, logFileSize, config.maxMessageSize, 1000, 10000, false, time) + val log = new Log(logDir, logFileSize, config.maxMessageSize, 1000, 10000, needsRecovery = false, time = time) assertEquals("There should be exactly 1 segment.", 1, log.numberOfSegments) - for (i<- 1 to msgPerSeg) { + for (i<- 1 to msgPerSeg) log.append(set) - } + assertEquals("There should be exactly 1 segments.", 1, log.numberOfSegments) - + assertEquals("Log end offset should be equal to number of messages", msgPerSeg, log.logEndOffset) + val lastOffset = log.logEndOffset val size = log.size log.truncateTo(log.logEndOffset) // keep the entire log @@ -303,29 +307,29 @@ class LogTest extends JUnitSuite { log.truncateTo(log.logEndOffset + 1) // try to truncate beyond lastOffset assertEquals("Should not change offset but should log error", lastOffset, log.logEndOffset) assertEquals("Should not change log size", size, log.size) - log.truncateTo(log.logEndOffset - 10) // truncate somewhere in between - assertEquals("Should change offset", lastOffset, log.logEndOffset + 10) - assertEquals("Should change log size", size, log.size + 10) - log.truncateTo(log.logEndOffset - log.size) // truncate the entire log - assertEquals("Should change offset", log.logEndOffset, lastOffset - size) - assertEquals("Should change log size", log.size, 0) - - for (i<- 1 to msgPerSeg) { + log.truncateTo(msgPerSeg/2) // truncate somewhere in between + assertEquals("Should change offset", log.logEndOffset, msgPerSeg/2) + assertTrue("Should change log size", log.size < size) + log.truncateTo(0) // truncate the entire log + assertEquals("Should change offset", 0, log.logEndOffset) + assertEquals("Should change log size", 0, log.size) + + for (i<- 1 to msgPerSeg) log.append(set) - } + assertEquals("Should be back to original offset", log.logEndOffset, lastOffset) assertEquals("Should be back to original size", log.size, size) - log.truncateAndStartWithNewOffset(log.logEndOffset - (msgPerSeg - 1)*setSize) - assertEquals("Should change offset", log.logEndOffset, lastOffset - (msgPerSeg - 1)*setSize) + log.truncateAndStartWithNewOffset(log.logEndOffset - (msgPerSeg - 1)) + assertEquals("Should change offset", log.logEndOffset, lastOffset - (msgPerSeg - 1)) assertEquals("Should change log size", log.size, 0) - for (i<- 1 to msgPerSeg) { + for (i<- 1 to msgPerSeg) log.append(set) - } - assertEquals("Should be ahead of to original offset", log.logEndOffset, lastOffset + setSize) + + assertTrue("Should be ahead of to original offset", log.logEndOffset > msgPerSeg) assertEquals("log size should be same as before", size, log.size) - log.truncateTo(log.logEndOffset - log.size - setSize) // truncate before first start offset in the log - assertEquals("Should change offset", log.logEndOffset, lastOffset - size) + log.truncateTo(0) // truncate before first start offset in the log + assertEquals("Should change offset", 0, log.logEndOffset) assertEquals("Should change log size", log.size, 0) } diff --git core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala new file mode 100644 index 0000000..9266afb --- /dev/null +++ core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala @@ -0,0 +1,180 @@ +/** + * 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 junit.framework.Assert._ +import java.util.{Collections, Arrays} +import org.junit._ +import org.scalatest.junit.JUnitSuite +import scala.collection._ +import scala.util.Random +import kafka.utils._ + + /* + * Test cases: + * - empty index + * - first value + * - last value + * - non-present value + * - present value + * - random values + * - test immutability + * - test truncate + * - test lookup outside bounds + * - Extreme values in append + * - what value do we return if falls off the end? + */ +class OffsetIndexTest extends JUnitSuite { + + var idx: OffsetIndex = null + val maxEntries = 30 + + @Before + def setup() { + this.idx = new OffsetIndex(file = TestUtils.tempFile(), baseOffset = 45L, mutable = true, maxIndexSize = 30 * 8) + } + + @After + def teardown() { + if(this.idx != null) + this.idx.file.delete() + } + + @Test + def randomLookupTest() { + assertEquals("Not present value should return physical offset 0.", OffsetPosition(idx.baseOffset, 0), idx.lookup(92L)) + + // append some random values + val base = idx.baseOffset.toInt + 1 + val size = idx.maxEntries + val vals: Seq[(Long, Int)] = monotonicSeq(base, size).map(_.toLong).zip(monotonicSeq(0, size)) + vals.foreach{x => idx.append(x._1, x._2)} + + // should be able to find all those values + for((logical, physical) <- vals) + assertEquals("Should be able to find values that are present.", OffsetPosition(logical, physical), idx.lookup(logical)) + + // for non-present values we should find the offset of the largest value less than or equal to this + val valMap = new immutable.TreeMap[Long, (Long, Int)]() ++ vals.map(p => (p._1, p)) + val offsets = (idx.baseOffset until vals.last._1.toInt).toArray + Collections.shuffle(Arrays.asList(offsets)) + for(offset <- offsets.take(30)) { + val rightAnswer = + if(offset < valMap.firstKey) + OffsetPosition(idx.baseOffset, 0) + else + OffsetPosition(valMap.to(offset).last._1, valMap.to(offset).last._2._2) + assertEquals("The index should give the same answer as the sorted map", rightAnswer, idx.lookup(offset)) + } + } + + @Test + def lookupExtremeCases() { + assertEquals("Lookup on empty file", OffsetPosition(idx.baseOffset, 0), idx.lookup(idx.baseOffset)) + for(i <- 0 until idx.maxEntries) + idx.append(idx.baseOffset + i + 1, i) + // check first and last entry + assertEquals(OffsetPosition(idx.baseOffset, 0), idx.lookup(idx.baseOffset)) + assertEquals(OffsetPosition(idx.baseOffset + idx.maxEntries, idx.maxEntries - 1), idx.lookup(idx.baseOffset + idx.maxEntries)) + } + + @Test + def appendTooMany() { + for(i <- 0 until idx.maxEntries) { + val offset = idx.baseOffset + i + 1 + idx.append(offset, i) + } + assertWriteFails("Append should fail on a full index", idx, idx.maxEntries + 1, classOf[IllegalStateException]) + } + + + @Test + def testReadOnly() { + /* add some random values */ + val vals = List((49, 1), (52, 2), (55, 3)) + for((logical, physical) <- vals) + idx.append(logical, physical) + + idx.makeReadOnly() + + assertEquals("File length should just contain added entries.", vals.size * 8L, idx.file.length()) + assertEquals("Last offset field should be initialized", vals.last._1, idx.lastOffset) + + for((logical, physical) <- vals) + assertEquals("Should still be able to find everything.", OffsetPosition(logical, physical), idx.lookup(logical)) + + assertWriteFails("Append should fail on read-only index", idx, 60, classOf[IllegalStateException]) + } + + @Test(expected = classOf[IllegalArgumentException]) + def appendOutOfOrder() { + idx.append(51, 0) + idx.append(50, 1) + } + + @Test + def reopenAsReadonly() { + val first = OffsetPosition(51, 0) + val sec = OffsetPosition(52, 1) + idx.append(first.offset, first.position) + idx.append(sec.offset, sec.position) + idx.close() + val idxRo = new OffsetIndex(file = idx.file, baseOffset = idx.baseOffset, mutable = false) + assertEquals(first, idxRo.lookup(first.offset)) + assertEquals(sec, idxRo.lookup(sec.offset)) + assertWriteFails("Append should fail on read-only index", idxRo, 53, classOf[IllegalStateException]) + } + + @Test + def truncate() { + val idx = new OffsetIndex(file = TestUtils.tempFile(), baseOffset = 0L, mutable = true, maxIndexSize = 10 * 8) + for(i <- 1 until 10) + idx.append(i, i) + idx.truncateTo(5) + assertEquals("4 should be the last entry in the index", OffsetPosition(4, 4), idx.lookup(10)) + assertEquals("4 should be the last entry in the index", 4, idx.lastOffset) + } + + def assertWriteFails[T](message: String, idx: OffsetIndex, offset: Int, klass: Class[T]) { + try { + idx.append(offset, 1) + fail(message) + } catch { + case e: Exception => assertEquals("Got an unexpected exception.", klass, e.getClass) + } + } + + def makeIndex(baseOffset: Long, mutable: Boolean, vals: Seq[(Long, Int)]): OffsetIndex = { + val idx = new OffsetIndex(file = TestUtils.tempFile, baseOffset = baseOffset, mutable = mutable, maxIndexSize = 2 * vals.size * 8) + for ((logical, physical) <- vals) + idx.append(logical, physical) + idx + } + + def monotonicSeq(base: Int, len: Int): Seq[Int] = { + val rand = new Random(1L) + val vals = new mutable.ArrayBuffer[Int](len) + var last = base + for (i <- 0 until len) { + last += rand.nextInt(15) + 1 + vals += last + } + vals + } +} \ No newline at end of file diff --git core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala index a6dc642..c436f3d 100644 --- core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala +++ core/src/test/scala/unit/kafka/message/BaseMessageSetTestCases.scala @@ -17,8 +17,10 @@ package kafka.message +import java.io.RandomAccessFile import junit.framework.Assert._ import kafka.utils.TestUtils._ +import kafka.log.FileMessageSet import org.scalatest.junit.JUnitSuite import org.junit.Test @@ -59,11 +61,15 @@ trait BaseMessageSetTestCases extends JUnitSuite { } def testWriteToWithMessageSet(set: MessageSet) { - val channel = tempChannel() - val written = set.writeTo(channel, 0, 1024) - assertEquals("Expect to write the number of bytes in the set.", set.sizeInBytes, written) - val newSet = new FileMessageSet(channel, false) - checkEquals(set.iterator, newSet.iterator) + // do the write twice to ensure the message set is restored to its orginal state + for(i <- List(0,1)) { + val file = tempFile() + val channel = new RandomAccessFile(file, "rw").getChannel() + val written = set.writeTo(channel, 0, 1024) + assertEquals("Expect to write the number of bytes in the set.", set.sizeInBytes, written) + val newSet = new FileMessageSet(file, channel, false) + checkEquals(set.iterator, newSet.iterator) + } } } diff --git core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala index b098b02..8ad1944 100644 --- core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala +++ core/src/test/scala/unit/kafka/message/ByteBufferMessageSetTest.scala @@ -18,6 +18,7 @@ package kafka.message import java.nio._ +import java.util.concurrent.atomic.AtomicLong import junit.framework.Assert._ import org.junit.Test import kafka.utils.TestUtils @@ -27,24 +28,6 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { override def createMessageSet(messages: Seq[Message]): ByteBufferMessageSet = new ByteBufferMessageSet(NoCompressionCodec, messages: _*) - - @Test - def testSmallFetchSize() { - // create a ByteBufferMessageSet that doesn't contain a full message - // iterating it should get an InvalidMessageSizeException - val messages = new ByteBufferMessageSet(NoCompressionCodec, new Message("01234567890123456789".getBytes())) - val buffer = messages.buffer.slice - buffer.limit(10) - val messageSetWithNoFullMessage = new ByteBufferMessageSet(buffer = buffer, initialOffset = 1000) - try { - for (message <- messageSetWithNoFullMessage) - fail("shouldn't see any message") - } - catch { - case e: InvalidMessageSizeException => //this is expected - case e2 => fail("shouldn't see any other exceptions") - } - } @Test def testValidBytes() { @@ -104,8 +87,6 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(messageSet.iterator)) //make sure ByteBufferMessageSet is re-iterable. TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(messageSet.iterator)) - //make sure the last offset after iteration is correct - assertEquals("offset of last message not expected", messageSet.last.offset, messageSet.buffer.limit) //make sure shallow iterator is the same as deep iterator TestUtils.checkEquals[Message](TestUtils.getMessageIterator(messageSet.shallowIterator), @@ -118,9 +99,6 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(messageSet.iterator)) //make sure ByteBufferMessageSet is re-iterable. TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(messageSet.iterator)) - //make sure the last offset after iteration is correct - assertEquals("offset of last message not expected", messageSet.last.offset, messageSet.buffer.limit) - verifyShallowIterator(messageSet) } @@ -137,9 +115,6 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(mixedMessageSet.iterator)) //make sure ByteBufferMessageSet is re-iterable. TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(mixedMessageSet.iterator)) - //make sure the last offset after iteration is correct - assertEquals("offset of last message not expected", mixedMessageSet.last.offset, mixedMessageSet.buffer.limit) - //make sure shallow iterator is the same as deep iterator TestUtils.checkEquals[Message](TestUtils.getMessageIterator(mixedMessageSet.shallowIterator), TestUtils.getMessageIterator(mixedMessageSet.iterator)) @@ -158,17 +133,41 @@ class ByteBufferMessageSetTest extends BaseMessageSetTestCases { TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(mixedMessageSet.iterator)) //make sure ByteBufferMessageSet is re-iterable. TestUtils.checkEquals[Message](messageList.iterator, TestUtils.getMessageIterator(mixedMessageSet.iterator)) - //make sure the last offset after iteration is correct - assertEquals("offset of last message not expected", mixedMessageSet.last.offset, mixedMessageSet.buffer.limit) - verifyShallowIterator(mixedMessageSet) } } + + @Test + def testOffsetAssignment() { + val messages = new ByteBufferMessageSet(NoCompressionCodec, + new Message("hello".getBytes), + new Message("there".getBytes), + new Message("beautiful".getBytes)) + val compressedMessages = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec, + messages = messages.map(_.message).toBuffer:_*) + // check uncompressed offsets + checkOffsets(messages, 0) + var offset = 1234567 + checkOffsets(messages.assignOffsets(new AtomicLong(offset), NoCompressionCodec), offset) + + // check compressed messages + checkOffsets(compressedMessages, 0) + checkOffsets(compressedMessages.assignOffsets(new AtomicLong(offset), DefaultCompressionCodec), offset) + } + + /* check that offsets are assigned based on byte offset from the given base offset */ + def checkOffsets(messages: ByteBufferMessageSet, baseOffset: Long) { + var offset = baseOffset + for(entry <- messages) { + assertEquals("Unexpected offset in message set iterator", offset, entry.offset) + offset += 1 + } + } def verifyShallowIterator(messageSet: ByteBufferMessageSet) { - //make sure the offsets returned by a shallow iterator is a subset of that of a deep iterator - val shallowOffsets = messageSet.shallowIterator.map(msgAndOff => msgAndOff.offset).toSet - val deepOffsets = messageSet.iterator.map(msgAndOff => msgAndOff.offset).toSet - assertTrue(shallowOffsets.subsetOf(deepOffsets)) + //make sure the offsets returned by a shallow iterator is a subset of that of a deep iterator + val shallowOffsets = messageSet.shallowIterator.map(msgAndOff => msgAndOff.offset).toSet + val deepOffsets = messageSet.iterator.map(msgAndOff => msgAndOff.offset).toSet + assertTrue(shallowOffsets.subsetOf(deepOffsets)) } } diff --git core/src/test/scala/unit/kafka/message/CompressionUtilsTest.scala core/src/test/scala/unit/kafka/message/CompressionUtilsTest.scala deleted file mode 100644 index df96603..0000000 --- core/src/test/scala/unit/kafka/message/CompressionUtilsTest.scala +++ /dev/null @@ -1,75 +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.message - -import kafka.utils.TestUtils -import org.scalatest.junit.JUnitSuite -import org.junit.Test -import junit.framework.Assert._ - -class CompressionUtilTest extends JUnitSuite { - - - @Test - def testSimpleCompressDecompress() { - - val messages = List[Message](new Message("hi there".getBytes), new Message("I am fine".getBytes), new Message("I am not so well today".getBytes)) - - val message = CompressionUtils.compress(messages) - - val decompressedMessages = CompressionUtils.decompress(message) - - TestUtils.checkLength(decompressedMessages.iterator,3) - - TestUtils.checkEquals(messages.iterator, TestUtils.getMessageIterator(decompressedMessages.iterator)) - } - - @Test - def testComplexCompressDecompress() { - - val messages = List[Message](new Message("hi there".getBytes), new Message("I am fine".getBytes), new Message("I am not so well today".getBytes)) - - val message = CompressionUtils.compress(messages.slice(0, 2)) - - val complexMessages = List[Message](message):::messages.slice(2,3) - - val complexMessage = CompressionUtils.compress(complexMessages) - - val decompressedMessages = CompressionUtils.decompress(complexMessage) - - TestUtils.checkLength(TestUtils.getMessageIterator(decompressedMessages.iterator),3) - - TestUtils.checkEquals(messages.iterator, TestUtils.getMessageIterator(decompressedMessages.iterator)) - } - - @Test - def testSnappyCompressDecompressExplicit() { - - val messages = List[Message](new Message("hi there".getBytes), new Message("I am fine".getBytes), new Message("I am not so well today".getBytes)) - - val message = CompressionUtils.compress(messages,SnappyCompressionCodec) - - assertEquals(message.compressionCodec,SnappyCompressionCodec) - - val decompressedMessages = CompressionUtils.decompress(message) - - TestUtils.checkLength(decompressedMessages.iterator,3) - - TestUtils.checkEquals(messages.iterator, TestUtils.getMessageIterator(decompressedMessages.iterator)) - } -} diff --git core/src/test/scala/unit/kafka/message/FileMessageSetTest.scala core/src/test/scala/unit/kafka/message/FileMessageSetTest.scala deleted file mode 100644 index a683963..0000000 --- core/src/test/scala/unit/kafka/message/FileMessageSetTest.scala +++ /dev/null @@ -1,84 +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.message - -import java.nio._ -import junit.framework.Assert._ -import kafka.utils.TestUtils._ -import org.junit.Test - -class FileMessageSetTest extends BaseMessageSetTestCases { - - val messageSet = createMessageSet(messages) - - def createMessageSet(messages: Seq[Message]): FileMessageSet = { - val set = new FileMessageSet(tempFile(), true) - set.append(new ByteBufferMessageSet(NoCompressionCodec, messages: _*)) - set.flush() - set - } - - @Test - def testFileSize() { - assertEquals(messageSet.channel.size, messageSet.sizeInBytes) - messageSet.append(singleMessageSet("abcd".getBytes())) - assertEquals(messageSet.channel.size, messageSet.sizeInBytes) - } - - @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) - val originalPosition = messageSet.channel.position - for(i <- 0 until size) - buffer.put(0.asInstanceOf[Byte]) - buffer.rewind() - messageSet.channel.write(buffer) - // appending those bytes should not change the contents - checkEquals(messages.iterator, messageSet.map(m => m.message).iterator) - assertEquals("Unexpected number of bytes truncated", size.longValue, messageSet.recover()) - assertEquals("File pointer should now be at the end of the file.", originalPosition, messageSet.channel.position) - // nor should recovery change the contents - checkEquals(messages.iterator, messageSet.map(m => m.message).iterator) - } - - @Test - def testIterationDoesntChangePosition() { - val position = messageSet.channel.position - checkEquals(messages.iterator, messageSet.map(m => m.message).iterator) - assertEquals(position, messageSet.channel.position) - } - - @Test - def testRead() { - val read = messageSet.read(0, messageSet.sizeInBytes) - checkEquals(messageSet.iterator, read.iterator) - val items = read.iterator.toList - val first = items.head - val read2 = messageSet.read(first.offset, messageSet.sizeInBytes) - checkEquals(items.tail.iterator, read2.iterator) - } - -} diff --git core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala new file mode 100644 index 0000000..8b5f38a --- /dev/null +++ core/src/test/scala/unit/kafka/message/MessageCompressionTest.scala @@ -0,0 +1,65 @@ +/** + * 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.message + +import java.io.ByteArrayOutputStream +import java.util.concurrent.atomic.AtomicLong +import scala.collection._ +import kafka.utils.TestUtils +import org.scalatest.junit.JUnitSuite +import org.junit._ +import junit.framework.Assert._ + +class MessageCompressionTest extends JUnitSuite { + + @Test + def testSimpleCompressDecompress() { + val codecs = mutable.ArrayBuffer[CompressionCodec](GZIPCompressionCodec) + if(isSnappyAvailable) + codecs += SnappyCompressionCodec + for(codec <- codecs) + testSimpleCompressDecompress(codec) + } + + def testSimpleCompressDecompress(compressionCodec: CompressionCodec) { + val messages = List[Message](new Message("hi there".getBytes), new Message("I am fine".getBytes), new Message("I am not so well today".getBytes)) + val messageSet = new ByteBufferMessageSet(compressionCodec = compressionCodec, messages = messages:_*) + assertEquals(compressionCodec, messageSet.shallowIterator.next.message.compressionCodec) + val decompressed = messageSet.iterator.map(_.message).toList + assertEquals(messages, decompressed) + } + + @Test + def testComplexCompressDecompress() { + val messages = List(new Message("hi there".getBytes), new Message("I am fine".getBytes), new Message("I am not so well today".getBytes)) + val message = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec, messages = messages.slice(0, 2):_*) + val complexMessages = List(message.shallowIterator.next.message):::messages.slice(2,3) + val complexMessage = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec, messages = complexMessages:_*) + val decompressedMessages = complexMessage.iterator.map(_.message).toList + assertEquals(messages, decompressedMessages) + } + + def isSnappyAvailable(): Boolean = { + try { + val snappy = new org.xerial.snappy.SnappyOutputStream(new ByteArrayOutputStream()) + true + } catch { + case e: UnsatisfiedLinkError => false + } + } +} diff --git core/src/test/scala/unit/kafka/message/MessageTest.scala core/src/test/scala/unit/kafka/message/MessageTest.scala index 4e3184c..fc75dd6 100644 --- core/src/test/scala/unit/kafka/message/MessageTest.scala +++ core/src/test/scala/unit/kafka/message/MessageTest.scala @@ -19,52 +19,78 @@ package kafka.message import java.util._ import java.nio._ +import scala.collection._ import junit.framework.Assert._ import org.scalatest.junit.JUnitSuite import org.junit.{Before, Test} +import org.junit.runner.RunWith +import org.junit.runners.Parameterized +import org.junit.runners.Parameterized.Parameters import kafka.utils.TestUtils +import kafka.utils.Utils + +case class MessageTestVal(val key: Array[Byte], + val payload: Array[Byte], + val codec: CompressionCodec, + val message: Message) class MessageTest extends JUnitSuite { - var message: Message = null - val payload = "some bytes".getBytes() - + var messages = new mutable.ArrayBuffer[MessageTestVal]() + @Before def setUp(): Unit = { - message = new Message(payload) + val keys = Array(null, "key".getBytes, "".getBytes) + val vals = Array("value".getBytes, "".getBytes) + val codecs = Array(NoCompressionCodec, GZIPCompressionCodec) + for(k <- keys; v <- vals; codec <- codecs) + messages += new MessageTestVal(k, v, codec, new Message(v, k, codec)) } @Test def testFieldValues = { - TestUtils.checkEquals(ByteBuffer.wrap(payload), message.payload) - assertEquals(Message.CurrentMagicValue, message.magic) - assertEquals(69L, new Message(69, "hello".getBytes()).checksum) + for(v <- messages) { + TestUtils.checkEquals(ByteBuffer.wrap(v.payload), v.message.payload) + assertEquals(Message.CurrentMagicValue, v.message.magic) + if(v.message.hasKey) + TestUtils.checkEquals(ByteBuffer.wrap(v.key), v.message.key) + else + assertEquals(null, v.message.key) + assertEquals(v.codec, v.message.compressionCodec) + } } @Test def testChecksum() { - assertTrue("Auto-computed checksum should be valid", message.isValid) - val badChecksum = message.checksum + 1 % Int.MaxValue - val invalid = new Message(badChecksum, payload) - assertEquals("Message should return written checksum", badChecksum, invalid.checksum) - assertFalse("Message with invalid checksum should be invalid", invalid.isValid) + for(v <- messages) { + assertTrue("Auto-computed checksum should be valid", v.message.isValid) + // garble checksum + val badChecksum: Int = (v.message.checksum + 1 % Int.MaxValue).toInt + Utils.putUnsignedInt(v.message.buffer, Message.CrcOffset, badChecksum) + assertFalse("Message with invalid checksum should be invalid", v.message.isValid) + } } @Test def testEquality() = { - assertFalse("Should not equal null", message.equals(null)) - assertFalse("Should not equal a random string", message.equals("asdf")) - assertTrue("Should equal itself", message.equals(message)) - val copy = new Message(message.checksum, payload) - assertTrue("Should equal another message with the same content.", message.equals(copy)) + 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)) + val copy = new Message(bytes = v.payload, key = v.key, codec = v.codec) + assertTrue("Should equal another message with the same content.", v.message.equals(copy)) + } } @Test def testIsHashable() = { // this is silly, but why not - val m = new HashMap[Message,Boolean]() - m.put(message, true) - assertNotNull(m.get(message)) + val m = new HashMap[Message, Message]() + for(v <- messages) + m.put(v.message, v.message) + for(v <- messages) + assertEquals(v.message, m.get(v.message)) } } + diff --git core/src/test/scala/unit/kafka/metrics/KafkaTimerTest.scala core/src/test/scala/unit/kafka/metrics/KafkaTimerTest.scala index 019e515..fe5bc09 100644 --- core/src/test/scala/unit/kafka/metrics/KafkaTimerTest.scala +++ core/src/test/scala/unit/kafka/metrics/KafkaTimerTest.scala @@ -1,4 +1,4 @@ -package unit.kafka.metrics +package kafka.metrics /** * Licensed to the Apache Software Foundation (ASF) under one or more @@ -21,7 +21,6 @@ import org.junit.Test import org.scalatest.junit.JUnit3Suite import java.util.concurrent.TimeUnit import junit.framework.Assert._ -import kafka.metrics.KafkaTimer import com.yammer.metrics.core.{MetricsRegistry, Clock} class KafkaTimerTest extends JUnit3Suite { diff --git core/src/test/scala/unit/kafka/network/RpcDataSerializationTest.scala core/src/test/scala/unit/kafka/network/RpcDataSerializationTest.scala deleted file mode 100644 index c463763..0000000 --- core/src/test/scala/unit/kafka/network/RpcDataSerializationTest.scala +++ /dev/null @@ -1,225 +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.network - -import org.junit._ -import org.scalatest.junit.JUnitSuite -import junit.framework.Assert._ -import java.nio.ByteBuffer -import kafka.api._ -import kafka.message.{Message, ByteBufferMessageSet} -import kafka.cluster.Broker -import collection.mutable._ -import kafka.common.{TopicAndPartition, ErrorMapping} - - -object RpcDataSerializationTestUtils{ - private val topic1 = "test1" - private val topic2 = "test2" - private val leader1 = 0 - private val isr1 = List(0, 1, 2) - private val leader2 = 0 - private val isr2 = List(0, 2, 3) - private val partitionData0 = new PartitionData(0, new ByteBufferMessageSet(new Message("first message".getBytes))) - private val partitionData1 = new PartitionData(1, new ByteBufferMessageSet(new Message("second message".getBytes))) - private val partitionData2 = new PartitionData(2, new ByteBufferMessageSet(new Message("third message".getBytes))) - private val partitionData3 = new PartitionData(3, new ByteBufferMessageSet(new Message("fourth message".getBytes))) - private val partitionDataArray = Array(partitionData0, partitionData1, partitionData2, partitionData3) - - private val topicData = { - val groupedData = Array(topic1, topic2).flatMap(topic => - partitionDataArray.map(partitionData => - (TopicAndPartition(topic, partitionData.partition), partitionData))) - collection.immutable.Map(groupedData:_*) - } - - private val requestInfos = collection.immutable.Map( - TopicAndPartition(topic1, 0) -> PartitionFetchInfo(1000, 100), - TopicAndPartition(topic1, 1) -> PartitionFetchInfo(2000, 100), - TopicAndPartition(topic1, 2) -> PartitionFetchInfo(3000, 100), - TopicAndPartition(topic1, 3) -> PartitionFetchInfo(4000, 100), - TopicAndPartition(topic2, 0) -> PartitionFetchInfo(1000, 100), - TopicAndPartition(topic2, 1) -> PartitionFetchInfo(2000, 100), - TopicAndPartition(topic2, 2) -> PartitionFetchInfo(3000, 100), - TopicAndPartition(topic2, 3) -> PartitionFetchInfo(4000, 100) - ) - - private val partitionMetaData0 = new PartitionMetadata(0, Some(new Broker(0, "creator", "localhost", 1011)), collection.immutable.Seq.empty) - private val partitionMetaData1 = new PartitionMetadata(1, Some(new Broker(0, "creator", "localhost", 1011)), collection.immutable.Seq.empty) - private val partitionMetaData2 = new PartitionMetadata(2, Some(new Broker(0, "creator", "localhost", 1011)), collection.immutable.Seq.empty) - private val partitionMetaData3 = new PartitionMetadata(3, Some(new Broker(0, "creator", "localhost", 1011)), collection.immutable.Seq.empty) - private val partitionMetaDataSeq = Seq(partitionMetaData0, partitionMetaData1, partitionMetaData2, partitionMetaData3) - private val topicmetaData1 = new TopicMetadata(topic1, partitionMetaDataSeq) - private val topicmetaData2 = new TopicMetadata(topic2, partitionMetaDataSeq) - - def createTestLeaderAndISRRequest() : LeaderAndIsrRequest = { - val leaderAndISR1 = new LeaderAndIsr(leader1, 1, isr1, 1) - val leaderAndISR2 = new LeaderAndIsr(leader2, 1, isr2, 2) - val map = Map(((topic1, 0), leaderAndISR1), - ((topic2, 0), leaderAndISR2)) - new LeaderAndIsrRequest(map) - } - - def createTestLeaderAndISRResponse() : LeaderAndISRResponse = { - val responseMap = Map(((topic1, 0), ErrorMapping.NoError), - ((topic2, 0), ErrorMapping.NoError)) - new LeaderAndISRResponse(1, responseMap) - } - - def createTestStopReplicaRequest() : StopReplicaRequest = { - new StopReplicaRequest(Set((topic1, 0), (topic2, 0))) - } - - def createTestStopReplicaResponse() : StopReplicaResponse = { - val responseMap = Map(((topic1, 0), ErrorMapping.NoError), - ((topic2, 0), ErrorMapping.NoError)) - new StopReplicaResponse(1, responseMap) - } - - def createTestProducerRequest: ProducerRequest = { - new ProducerRequest(1, "client 1", 0, 1000, topicData) - } - - def createTestProducerResponse: ProducerResponse = - ProducerResponse(1, 1, Map( - TopicAndPartition(topic1, 0) -> ProducerResponseStatus(0.toShort, 10001), - TopicAndPartition(topic2, 0) -> ProducerResponseStatus(0.toShort, 20001) - )) - - def createTestFetchRequest: FetchRequest = { - new FetchRequest(requestInfo = requestInfos) - } - - def createTestFetchResponse: FetchResponse = { - FetchResponse(1, 1, topicData) - } - - def createTestOffsetRequest = new OffsetRequest( - collection.immutable.Map(TopicAndPartition(topic1, 1) -> PartitionOffsetRequestInfo(1000, 200)), - replicaId = 0 - ) - - def createTestOffsetResponse: OffsetResponse = { - new OffsetResponse(OffsetRequest.CurrentVersion, collection.immutable.Map( - TopicAndPartition(topic1, 1) -> PartitionOffsetsResponse(ErrorMapping.NoError, Seq(1000l, 2000l, 3000l, 4000l))) - ) - } - - def createTestTopicMetadataRequest: TopicMetadataRequest = { - new TopicMetadataRequest(1, "client 1", Seq(topic1, topic2)) - } - - def createTestTopicMetadataResponse: TopicMetadataResponse = { - new TopicMetadataResponse(1, Seq(topicmetaData1, topicmetaData2)) - } -} - -class RpcDataSerializationTest extends JUnitSuite { - private val leaderAndISRRequest = RpcDataSerializationTestUtils.createTestLeaderAndISRRequest - private val leaderAndISRResponse = RpcDataSerializationTestUtils.createTestLeaderAndISRResponse - private val stopReplicaRequest = RpcDataSerializationTestUtils.createTestStopReplicaRequest - private val stopReplicaResponse = RpcDataSerializationTestUtils.createTestStopReplicaResponse - private val producerRequest = RpcDataSerializationTestUtils.createTestProducerRequest - private val producerResponse = RpcDataSerializationTestUtils.createTestProducerResponse - private val fetchRequest = RpcDataSerializationTestUtils.createTestFetchRequest - private val offsetRequest = RpcDataSerializationTestUtils.createTestOffsetRequest - private val offsetResponse = RpcDataSerializationTestUtils.createTestOffsetResponse - private val topicMetadataRequest = RpcDataSerializationTestUtils.createTestTopicMetadataRequest - private val topicMetadataResponse = RpcDataSerializationTestUtils.createTestTopicMetadataResponse - - - @Test - def testSerializationAndDeserialization() { - var buffer: ByteBuffer = ByteBuffer.allocate(leaderAndISRRequest.sizeInBytes()) - leaderAndISRRequest.writeTo(buffer) - buffer.rewind() - val deserializedLeaderAndISRRequest = LeaderAndIsrRequest.readFrom(buffer) - assertEquals("The original and deserialzed leaderAndISRRequest should be the same", leaderAndISRRequest, - deserializedLeaderAndISRRequest) - - buffer = ByteBuffer.allocate(leaderAndISRResponse.sizeInBytes()) - leaderAndISRResponse.writeTo(buffer) - buffer.rewind() - val deserializedLeaderAndISRResponse = LeaderAndISRResponse.readFrom(buffer) - assertEquals("The original and deserialzed leaderAndISRResponse should be the same", leaderAndISRResponse, - deserializedLeaderAndISRResponse) - - buffer = ByteBuffer.allocate(stopReplicaRequest.sizeInBytes()) - stopReplicaRequest.writeTo(buffer) - buffer.rewind() - val deserializedStopReplicaRequest = StopReplicaRequest.readFrom(buffer) - assertEquals("The original and deserialzed stopReplicaRequest should be the same", stopReplicaRequest, - deserializedStopReplicaRequest) - - buffer = ByteBuffer.allocate(stopReplicaResponse.sizeInBytes()) - stopReplicaResponse.writeTo(buffer) - buffer.rewind() - val deserializedStopReplicaResponse = StopReplicaResponse.readFrom(buffer) - assertEquals("The original and deserialzed stopReplicaResponse should be the same", stopReplicaResponse, - deserializedStopReplicaResponse) - - buffer = ByteBuffer.allocate(producerRequest.sizeInBytes) - producerRequest.writeTo(buffer) - buffer.rewind() - val deserializedProducerRequest = ProducerRequest.readFrom(buffer) - assertEquals("The original and deserialzed producerRequest should be the same", producerRequest, - deserializedProducerRequest) - - buffer = ByteBuffer.allocate(producerResponse.sizeInBytes) - producerResponse.writeTo(buffer) - buffer.rewind() - val deserializedProducerResponse = ProducerResponse.readFrom(buffer) - assertEquals("The original and deserialzed producerResponse should be the same: [%s], [%s]".format(producerResponse, deserializedProducerResponse), producerResponse, - deserializedProducerResponse) - - buffer = ByteBuffer.allocate(fetchRequest.sizeInBytes) - fetchRequest.writeTo(buffer) - buffer.rewind() - val deserializedFetchRequest = FetchRequest.readFrom(buffer) - assertEquals("The original and deserialzed fetchRequest should be the same", fetchRequest, - deserializedFetchRequest) - - buffer = ByteBuffer.allocate(offsetRequest.sizeInBytes) - offsetRequest.writeTo(buffer) - buffer.rewind() - val deserializedOffsetRequest = OffsetRequest.readFrom(buffer) - assertEquals("The original and deserialzed offsetRequest should be the same", offsetRequest, - deserializedOffsetRequest) - - buffer = ByteBuffer.allocate(offsetResponse.sizeInBytes) - offsetResponse.writeTo(buffer) - buffer.rewind() - val deserializedOffsetResponse = OffsetResponse.readFrom(buffer) - assertEquals("The original and deserialzed offsetResponse should be the same", offsetResponse, - deserializedOffsetResponse) - - buffer = ByteBuffer.allocate(topicMetadataRequest.sizeInBytes()) - topicMetadataRequest.writeTo(buffer) - buffer.rewind() - val deserializedTopicMetadataRequest = TopicMetadataRequest.readFrom(buffer) - assertEquals("The original and deserialzed topicMetadataRequest should be the same", topicMetadataRequest, - deserializedTopicMetadataRequest) - - buffer = ByteBuffer.allocate(topicMetadataResponse.sizeInBytes) - topicMetadataResponse.writeTo(buffer) - buffer.rewind() - val deserializedTopicMetadataResponse = TopicMetadataResponse.readFrom(buffer) - assertEquals("The original and deserialzed topicMetadataResponse should be the same", topicMetadataResponse, - deserializedTopicMetadataResponse) - } -} diff --git core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala index aba070e..1726fdd 100644 --- core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala +++ core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala @@ -19,6 +19,7 @@ package kafka.producer import java.util.{LinkedList, Properties} import java.util.concurrent.LinkedBlockingQueue +import java.io.IOException import junit.framework.Assert._ import org.easymock.EasyMock import org.junit.Test @@ -149,7 +150,7 @@ class AsyncProducerTest extends JUnit3Suite { for (producerData <- producerDataList) queue.put(producerData) - Thread.sleep(queueExpirationTime + 10) + Thread.sleep(queueExpirationTime + 100) EasyMock.verify(mockHandler) producerSendThread.shutdown } @@ -354,6 +355,7 @@ class AsyncProducerTest extends JUnit3Suite { @Test def testBrokerListAndAsync() { + return val props = new Properties() props.put("serializer.class", "kafka.serializer.StringEncoder") props.put("producer.type", "async") @@ -401,7 +403,6 @@ class AsyncProducerTest extends JUnit3Suite { val topicPartitionInfos = new collection.mutable.HashMap[String, TopicMetadata] topicPartitionInfos.put("topic1", topic1Metadata) - val msgs = TestUtils.getMsgStrings(2) // produce request for topic1 and partitions 0 and 1. Let the first request fail @@ -432,16 +433,10 @@ class AsyncProducerTest extends JUnit3Suite { encoder = new StringEncoder, producerPool = producerPool, topicPartitionInfos) - try { - val data = List( - new ProducerData[Int,String](topic1, 0, msgs), - new ProducerData[Int,String](topic1, 1, msgs) - ) - handler.handle(data) - handler.close() - } catch { - case e: Exception => fail("Not expected", e) - } + val data = List(new ProducerData[Int,String](topic1, 0, msgs), + new ProducerData[Int,String](topic1, 1, msgs)) + handler.handle(data) + handler.close() EasyMock.verify(mockSyncProducer) EasyMock.verify(producerPool) diff --git core/src/test/scala/unit/kafka/producer/ProducerTest.scala core/src/test/scala/unit/kafka/producer/ProducerTest.scala index 8486a8b..2077ec1 100644 --- core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -170,19 +170,14 @@ class ProducerTest extends JUnit3Suite with ZooKeeperTestHarness with Logging{ val messageSet = if(leader == server1.config.brokerId) { val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build()) - response1.messageSet("new-topic", 0).iterator + response1.messageSet("new-topic", 0).iterator.toBuffer }else { val response2 = consumer2.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build()) - response2.messageSet("new-topic", 0).iterator + response2.messageSet("new-topic", 0).iterator.toBuffer } - assertTrue("Message set should have 1 message", messageSet.hasNext) - - assertEquals(new Message("test1".getBytes), messageSet.next.message) - assertTrue("Message set should have 1 message", messageSet.hasNext) - assertEquals(new Message("test2".getBytes), messageSet.next.message) - if (messageSet.hasNext) - fail("Message set should not have any more messages, but received a message of %s" - .format(Utils.toString(messageSet.next.message.payload, "UTF-8"))) + assertEquals("Should have fetched 2 messages", 2, messageSet.size) + assertEquals(new Message("test1".getBytes), messageSet(0).message) + assertEquals(new Message("test2".getBytes), messageSet(1).message) producer1.close() try { diff --git core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index 4eb36c8..e459377 100644 --- core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -21,10 +21,11 @@ import java.net.SocketTimeoutException import java.util.Properties import junit.framework.Assert import kafka.admin.CreateTopicCommand +import kafka.common.{ErrorMapping} import kafka.integration.KafkaServerTestHarness -import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet} +import kafka.message._ import kafka.server.KafkaConfig -import kafka.utils.{TestZKUtils, SystemTime, TestUtils} +import kafka.utils._ import org.junit.Test import org.scalatest.junit.JUnit3Suite import kafka.api.{ProducerResponseStatus, PartitionData} @@ -105,21 +106,22 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { CreateTopicCommand.createTopic(zkClient, "test", 1, 1) TestUtils.waitUntilLeaderIsElectedOrChanged(zkClient, "test", 0, 500) - val message1 = new Message(new Array[Byte](1000001)) + val message1 = new Message(new Array[Byte](configs(0).maxMessageSize + 1)) val messageSet1 = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = message1) val response1 = producer.send(TestUtils.produceRequest("test", 0, messageSet1)) Assert.assertEquals(1, response1.status.count(_._2.error != ErrorMapping.NoError)) Assert.assertEquals(ErrorMapping.MessageSizeTooLargeCode, response1.status(TopicAndPartition("test", 0)).error) - Assert.assertEquals(-1L, response1.status(TopicAndPartition("test", 0)).nextOffset) + Assert.assertEquals(-1L, response1.status(TopicAndPartition("test", 0)).offset) - val message2 = new Message(new Array[Byte](1000000)) + val safeSize = configs(0).maxMessageSize - Message.MessageOverhead - MessageSet.LogOverhead - 1 + val message2 = new Message(new Array[Byte](safeSize)) val messageSet2 = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = message2) val response2 = producer.send(TestUtils.produceRequest("test", 0, messageSet2)) Assert.assertEquals(1, response1.status.count(_._2.error != ErrorMapping.NoError)) Assert.assertEquals(ErrorMapping.NoError, response2.status(TopicAndPartition("test", 0)).error) - Assert.assertEquals(messageSet2.sizeInBytes, response2.status(TopicAndPartition("test", 0)).nextOffset) + Assert.assertEquals(0, response2.status(TopicAndPartition("test", 0)).offset) } @Test @@ -163,13 +165,13 @@ class SyncProducerTest extends JUnit3Suite with KafkaServerTestHarness { // the first and last message should have been accepted by broker Assert.assertEquals(ErrorMapping.NoError, response2.status(TopicAndPartition("topic1", 0)).error) Assert.assertEquals(ErrorMapping.NoError, response2.status(TopicAndPartition("topic3", 0)).error) - Assert.assertEquals(messages.sizeInBytes, response2.status(TopicAndPartition("topic1", 0)).nextOffset) - Assert.assertEquals(messages.sizeInBytes, response2.status(TopicAndPartition("topic3", 0)).nextOffset) + Assert.assertEquals(0, response2.status(TopicAndPartition("topic1", 0)).offset) + Assert.assertEquals(0, response2.status(TopicAndPartition("topic3", 0)).offset) // the middle message should have been rejected because broker doesn't lead partition Assert.assertEquals(ErrorMapping.UnknownTopicOrPartitionCode.toShort, response2.status(TopicAndPartition("topic2", 0)).error) - Assert.assertEquals(-1, response2.status(TopicAndPartition("topic2", 0)).nextOffset) + Assert.assertEquals(-1, response2.status(TopicAndPartition("topic2", 0)).offset) } @Test diff --git core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala index d4cb295..fcbe4e1 100644 --- core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala +++ core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala @@ -22,7 +22,7 @@ import kafka.admin.CreateTopicCommand import kafka.utils.TestUtils._ import kafka.utils.{Utils, TestUtils} import kafka.zk.ZooKeeperTestHarness -import kafka.message.Message +import kafka.message.{Message, MessageSet, ByteBufferMessageSet} import kafka.producer.{ProducerConfig, ProducerData, Producer} class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { @@ -36,20 +36,13 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { val topic = "new-topic" val partitionId = 0 - val brokerId1 = 0 - val brokerId2 = 1 - - val port1 = TestUtils.choosePort() - val port2 = TestUtils.choosePort() - var server1: KafkaServer = null var server2: KafkaServer = null val configProps1 = configs.head val configProps2 = configs.last - val sent1 = List(new Message("hello".getBytes()), new Message("there".getBytes())) - val sent2 = List( new Message("more".getBytes()), new Message("messages".getBytes())) + val message = new Message("hello".getBytes()) var producer: Producer[Int, Message] = null var hwFile1: HighwaterMarkCheckpoint = new HighwaterMarkCheckpoint(configProps1.logDir) @@ -76,18 +69,20 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { // NOTE: this is to avoid transient test failures assertTrue("Leader could be broker 0 or broker 1", (leader.getOrElse(-1) == 0) || (leader.getOrElse(-1) == 1)) - sendMessages(2) + val numMessages = 2L + sendMessages(numMessages.toInt) - // give some time for the follower 1 to record leader HW of 60 - assertTrue("Failed to update highwatermark for follower after 1000 ms", TestUtils.waitUntilTrue(() => - server2.replicaManager.getReplica(topic, 0).get.highWatermark == 60L, 1000)) + // give some time for the follower 1 to record leader HW + assertTrue("Failed to update highwatermark for follower after 1000 ms", + TestUtils.waitUntilTrue(() => + server2.replicaManager.getReplica(topic, 0).get.highWatermark == numMessages, 10000)) servers.foreach(server => server.replicaManager.checkpointHighWatermarks()) producer.close() val leaderHW = hwFile1.read(topic, 0) - assertEquals(60L, leaderHW) + assertEquals(numMessages, leaderHW) val followerHW = hwFile2.read(topic, 0) - assertEquals(60L, followerHW) + assertEquals(numMessages, followerHW) servers.foreach(server => { server.shutdown(); Utils.rm(server.config.logDir)}) } @@ -110,14 +105,16 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { assertTrue("Leader should get elected", leader.isDefined) // NOTE: this is to avoid transient test failures assertTrue("Leader could be broker 0 or broker 1", (leader.getOrElse(-1) == 0) || (leader.getOrElse(-1) == 1)) - + assertEquals(0L, hwFile1.read(topic, 0)) - sendMessages() + sendMessages(1) + Thread.sleep(1000) + var hw = 1L // kill the server hosting the preferred replica server1.shutdown() - assertEquals(30L, hwFile1.read(topic, 0)) + assertEquals(hw, hwFile1.read(topic, 0)) // check if leader moves to the other server leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500, leader) @@ -130,25 +127,27 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { assertTrue("Leader must remain on broker 1, in case of zookeeper session expiration it can move to broker 0", leader.isDefined && (leader.get == 0 || leader.get == 1)) - assertEquals(30L, hwFile1.read(topic, 0)) + assertEquals(hw, hwFile1.read(topic, 0)) // since server 2 was never shut down, the hw value of 30 is probably not checkpointed to disk yet server2.shutdown() - assertEquals(30L, hwFile2.read(topic, 0)) + assertEquals(hw, hwFile2.read(topic, 0)) server2.startup() leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500, leader) assertTrue("Leader must remain on broker 0, in case of zookeeper session expiration it can move to broker 1", leader.isDefined && (leader.get == 0 || leader.get == 1)) - sendMessages() + sendMessages(1) + hw += 1 + // give some time for follower 1 to record leader HW of 60 assertTrue("Failed to update highwatermark for follower after 1000 ms", TestUtils.waitUntilTrue(() => - server2.replicaManager.getReplica(topic, 0).get.highWatermark == 60L, 1000)) + server2.replicaManager.getReplica(topic, 0).get.highWatermark == hw, 2000)) // shutdown the servers to allow the hw to be checkpointed servers.foreach(server => server.shutdown()) producer.close() - assertEquals(60L, hwFile1.read(topic, 0)) - assertEquals(60L, hwFile2.read(topic, 0)) + assertEquals(hw, hwFile1.read(topic, 0)) + assertEquals(hw, hwFile2.read(topic, 0)) servers.foreach(server => Utils.rm(server.config.logDir)) } @@ -183,16 +182,17 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { // NOTE: this is to avoid transient test failures assertTrue("Leader could be broker 0 or broker 1", (leader.getOrElse(-1) == 0) || (leader.getOrElse(-1) == 1)) sendMessages(20) + var hw = 20L // give some time for follower 1 to record leader HW of 600 assertTrue("Failed to update highwatermark for follower after 1000 ms", TestUtils.waitUntilTrue(() => - server2.replicaManager.getReplica(topic, 0).get.highWatermark == 600L, 1000)) + server2.replicaManager.getReplica(topic, 0).get.highWatermark == hw, 1000)) // shutdown the servers to allow the hw to be checkpointed servers.foreach(server => server.shutdown()) producer.close() val leaderHW = hwFile1.read(topic, 0) - assertEquals(600L, leaderHW) + assertEquals(hw, leaderHW) val followerHW = hwFile2.read(topic, 0) - assertEquals(600L, followerHW) + assertEquals(hw, followerHW) servers.foreach(server => Utils.rm(server.config.logDir)) } @@ -228,43 +228,46 @@ class LogRecoveryTest extends JUnit3Suite with ZooKeeperTestHarness { assertTrue("Leader could be broker 0 or broker 1", (leader.getOrElse(-1) == 0) || (leader.getOrElse(-1) == 1)) sendMessages(2) + var hw = 2L + // allow some time for the follower to get the leader HW assertTrue("Failed to update highwatermark for follower after 1000 ms", TestUtils.waitUntilTrue(() => - server2.replicaManager.getReplica(topic, 0).get.highWatermark == 60L, 1000)) + server2.replicaManager.getReplica(topic, 0).get.highWatermark == hw, 1000)) // kill the server hosting the preferred replica server1.shutdown() server2.shutdown() - assertEquals(60L, hwFile1.read(topic, 0)) - assertEquals(60L, hwFile2.read(topic, 0)) + assertEquals(hw, hwFile1.read(topic, 0)) + assertEquals(hw, hwFile2.read(topic, 0)) server2.startup() // check if leader moves to the other server leader = waitUntilLeaderIsElectedOrChanged(zkClient, topic, partitionId, 500, leader) assertEquals("Leader must move to broker 1", 1, leader.getOrElse(-1)) - assertEquals(60L, hwFile1.read(topic, 0)) + assertEquals(hw, hwFile1.read(topic, 0)) // bring the preferred replica back server1.startup() - assertEquals(60L, hwFile1.read(topic, 0)) - assertEquals(60L, hwFile2.read(topic, 0)) + assertEquals(hw, hwFile1.read(topic, 0)) + assertEquals(hw, hwFile2.read(topic, 0)) sendMessages(2) + hw += 2 + // allow some time for the follower to get the leader HW assertTrue("Failed to update highwatermark for follower after 1000 ms", TestUtils.waitUntilTrue(() => - server1.replicaManager.getReplica(topic, 0).get.highWatermark == 120L, 1000)) + server1.replicaManager.getReplica(topic, 0).get.highWatermark == hw, 1000)) // shutdown the servers to allow the hw to be checkpointed servers.foreach(server => server.shutdown()) producer.close() - assertEquals(120L, hwFile1.read(topic, 0)) - assertEquals(120L, hwFile2.read(topic, 0)) + assertEquals(hw, hwFile1.read(topic, 0)) + assertEquals(hw, hwFile2.read(topic, 0)) servers.foreach(server => Utils.rm(server.config.logDir)) } - private def sendMessages(numMessages: Int = 1) { - for(i <- 0 until numMessages) { - producer.send(new ProducerData[Int, Message](topic, 0, sent1)) - } + private def sendMessages(n: Int = 1) { + for(i <- 0 until n) + producer.send(new ProducerData[Int, Message](topic, 0, message)) } } diff --git core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala index 6a12a95..749c0ae 100644 --- core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala +++ core/src/test/scala/unit/kafka/server/ReplicaFetchTest.scala @@ -60,7 +60,7 @@ class ReplicaFetchTest extends JUnit3Suite with ZooKeeperTestHarness { new ProducerData[String, String](topic2, testMessageList2)) producer.close() - def condition(): Boolean = { + def logsMatch(): Boolean = { var result = true for (topic <- List(topic1, topic2)) { val expectedOffset = brokers.head.getLogManager().getLog(topic, partition).get.logEndOffset @@ -69,6 +69,6 @@ class ReplicaFetchTest extends JUnit3Suite with ZooKeeperTestHarness { } result } - assertTrue("broker logs should be identical", waitUntilTrue(condition, 6000)) + assertTrue("Broker logs should be identical", waitUntilTrue(logsMatch, 6000)) } } \ No newline at end of file diff --git core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index 79fa5a3..d738010 100644 --- core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -80,7 +80,7 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { fetchedMessage = fetched.messageSet(topic, 0) } TestUtils.checkEquals(sent1.iterator, fetchedMessage.map(m => m.message).iterator) - val newOffset = fetchedMessage.validBytes + val newOffset = fetchedMessage.last.nextOffset // send some more messages producer.send(new ProducerData[Int, Message](topic, 0, sent2)) diff --git core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala index 66ecd60..c675f1f 100644 --- core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala +++ core/src/test/scala/unit/kafka/server/SimpleFetchTest.scala @@ -54,6 +54,7 @@ class SimpleFetchTest extends JUnit3Suite { val time = new MockTime val leo = 20 val hw = 5 + val fetchSize = 100 val messages = new Message("test-message".getBytes()) val zkClient = EasyMock.createMock(classOf[ZkClient]) @@ -61,7 +62,7 @@ class SimpleFetchTest extends JUnit3Suite { val log = EasyMock.createMock(classOf[kafka.log.Log]) EasyMock.expect(log.logEndOffset).andReturn(leo).anyTimes() - EasyMock.expect(log.read(0, hw)).andReturn(new ByteBufferMessageSet(messages)) + EasyMock.expect(log.read(0, fetchSize, Some(hw))).andReturn(new ByteBufferMessageSet(messages)) EasyMock.replay(log) val logManager = EasyMock.createMock(classOf[kafka.log.LogManager]) @@ -92,9 +93,9 @@ class SimpleFetchTest extends JUnit3Suite { // This request (from a follower) wants to read up to 2*HW but should only get back up to HW bytes into the log val goodFetch = new FetchRequestBuilder() - .replicaId(Request.NonFollowerId) - .addFetch(topic, partitionId, 0, hw*2) - .build() + .replicaId(Request.NonFollowerId) + .addFetch(topic, partitionId, 0, fetchSize) + .build() val goodFetchBB = TestUtils.createRequestByteBuffer(goodFetch) // send the request @@ -156,7 +157,7 @@ class SimpleFetchTest extends JUnit3Suite { val log = EasyMock.createMock(classOf[kafka.log.Log]) EasyMock.expect(log.logEndOffset).andReturn(leo).anyTimes() - EasyMock.expect(log.read(followerLEO, Integer.MAX_VALUE)).andReturn(new ByteBufferMessageSet(messages)) + EasyMock.expect(log.read(followerLEO, Integer.MAX_VALUE, None)).andReturn(new ByteBufferMessageSet(messages)) EasyMock.replay(log) val logManager = EasyMock.createMock(classOf[kafka.log.LogManager]) diff --git core/src/test/scala/unit/kafka/utils/TestUtils.scala core/src/test/scala/unit/kafka/utils/TestUtils.scala index 0e47daf..dc72fbe 100644 --- core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -155,8 +155,8 @@ object TestUtils extends Logging { * Wrap the message in a message set * @param payload The bytes of the message */ - def singleMessageSet(payload: Array[Byte]) = - new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = new Message(payload)) + def singleMessageSet(payload: Array[Byte], codec: CompressionCodec = NoCompressionCodec) = + new ByteBufferMessageSet(compressionCodec = codec, messages = new Message(payload)) /** * Generate an array of random bytes @@ -426,7 +426,29 @@ object TestUtils extends Logging { leaderLock.unlock() } } + + /** + * Execute the given block. If it throws an assert error, retry. Repeat + * until no error is thrown or the time limit ellapses + */ + def retry(waitTime: Long, block: () => Unit) { + val startTime = System.currentTimeMillis() + while(true) { + try { + block() + } catch { + case e: AssertionError => + if(System.currentTimeMillis - startTime > waitTime) + throw e + else + Thread.sleep(100) + } + } + } + /** + * Wait until the given condition is true or the given wait time ellapses + */ def waitUntilTrue(condition: () => Boolean, waitTime: Long): Boolean = { val startTime = System.currentTimeMillis() while (true) {