new consumer request format; patched by Prashanth Menon; reviewed by Jun Rao and Jay Kreps; KAFKA-240

git-svn-id: https://svn.apache.org/repos/asf/incubator/kafka/branches/0.8@1243407 13f79535-47bb-0310-9956-ffa450edef68
This commit is contained in:
Jun Rao 2012-02-13 03:58:37 +00:00
parent 21da30456f
commit a5fb217293
37 changed files with 1038 additions and 804 deletions

View File

@ -16,27 +16,26 @@
*/ */
package kafka.etl; package kafka.etl;
import java.io.IOException;
import java.net.URI;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
import java.util.zip.CRC32;
import kafka.api.FetchRequest; import kafka.api.FetchRequest;
import kafka.javaapi.MultiFetchResponse; import kafka.api.FetchRequestBuilder;
import kafka.api.OffsetRequest; import kafka.api.OffsetRequest;
import kafka.common.ErrorMapping; import kafka.common.ErrorMapping;
import kafka.javaapi.FetchResponse;
import kafka.javaapi.consumer.SimpleConsumer; import kafka.javaapi.consumer.SimpleConsumer;
import kafka.javaapi.message.ByteBufferMessageSet; import kafka.javaapi.message.ByteBufferMessageSet;
import kafka.message.Message; import kafka.javaapi.message.MessageSet;
import kafka.message.MessageAndOffset; import kafka.message.MessageAndOffset;
import kafka.message.MessageSet;
import org.apache.hadoop.io.BytesWritable; import org.apache.hadoop.io.BytesWritable;
import org.apache.hadoop.mapred.JobConf; import org.apache.hadoop.mapred.JobConf;
import org.apache.hadoop.mapred.OutputCollector; import org.apache.hadoop.mapred.OutputCollector;
import org.apache.hadoop.mapred.Reporter; import org.apache.hadoop.mapred.Reporter;
import org.apache.hadoop.mapred.lib.MultipleOutputs; import org.apache.hadoop.mapred.lib.MultipleOutputs;
import java.io.IOException;
import java.net.URI;
import java.nio.ByteBuffer; import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Iterator;
@SuppressWarnings({ "deprecation"}) @SuppressWarnings({ "deprecation"})
public class KafkaETLContext { public class KafkaETLContext {
@ -59,7 +58,8 @@ public class KafkaETLContext {
protected long _offset = Long.MAX_VALUE; /*current offset*/ protected long _offset = Long.MAX_VALUE; /*current offset*/
protected long _count; /*current count*/ protected long _count; /*current count*/
protected MultiFetchResponse _response = null; /*fetch response*/ protected int requestId = 0; /* the id of the next fetch request */
protected FetchResponse _response = null; /*fetch response*/
protected Iterator<MessageAndOffset> _messageIt = null; /*message iterator*/ protected Iterator<MessageAndOffset> _messageIt = null; /*message iterator*/
protected Iterator<ByteBufferMessageSet> _respIterator = null; protected Iterator<ByteBufferMessageSet> _respIterator = null;
protected int _retry = 0; protected int _retry = 0;
@ -149,15 +149,19 @@ public class KafkaETLContext {
public boolean fetchMore () throws IOException { public boolean fetchMore () throws IOException {
if (!hasMore()) return false; if (!hasMore()) return false;
FetchRequest fetchRequest = FetchRequest fetchRequest = new FetchRequestBuilder()
new FetchRequest(_request.getTopic(), _request.getPartition(), _offset, _bufferSize); .correlationId(requestId)
List<FetchRequest> array = new ArrayList<FetchRequest>(); .clientId(_request.clientId())
array.add(fetchRequest); .addFetch(_request.getTopic(), _request.getPartition(), _offset, _bufferSize)
.build();
long tempTime = System.currentTimeMillis(); long tempTime = System.currentTimeMillis();
_response = _consumer.multifetch(array); _response = _consumer.fetch(fetchRequest);
if(_response != null) if(_response != null) {
_respIterator = _response.iterator(); _respIterator = new ArrayList<ByteBufferMessageSet>(){{
add((ByteBufferMessageSet) _response.messageSet(_request.getTopic(), _request.getPartition()));
}}.iterator();
}
_requestTime += (System.currentTimeMillis() - tempTime); _requestTime += (System.currentTimeMillis() - tempTime);
return true; return true;

View File

@ -29,6 +29,7 @@ public class KafkaETLRequest {
URI _uri; URI _uri;
int _partition; int _partition;
long _offset = DEFAULT_OFFSET; long _offset = DEFAULT_OFFSET;
String _clientId = "KafkaHadoopETL";
public KafkaETLRequest() { public KafkaETLRequest() {
@ -86,8 +87,8 @@ public class KafkaETLRequest {
public String getTopic() { return _topic; } public String getTopic() { return _topic; }
public URI getURI () { return _uri; } public URI getURI () { return _uri; }
public int getPartition() { return _partition; } public int getPartition() { return _partition; }
public long getOffset() { return _offset; } public long getOffset() { return _offset; }
public String clientId() { return _clientId; }
public boolean isValidOffset() { public boolean isValidOffset() {
return _offset >= 0; return _offset >= 0;

View File

@ -20,32 +20,149 @@ package kafka.api
import java.nio._ import java.nio._
import kafka.network._ import kafka.network._
import kafka.utils._ import kafka.utils._
import scala.collection.mutable.{HashMap, Buffer, ListBuffer}
object FetchRequest { object OffsetDetail {
def readFrom(buffer: ByteBuffer): FetchRequest = { def readFrom(buffer: ByteBuffer): OffsetDetail = {
val topic = Utils.readShortString(buffer, "UTF-8") val topic = Utils.readShortString(buffer, "UTF-8")
val partition = buffer.getInt()
val offset = buffer.getLong() val partitionsCount = buffer.getInt
val size = buffer.getInt() val partitions = new Array[Int](partitionsCount)
new FetchRequest(topic, partition, offset, size) for (i <- 0 until partitions.length)
} partitions(i) = buffer.getInt
val offsetsCount = buffer.getInt
val offsets = new Array[Long](offsetsCount)
for (i <- 0 until offsets.length)
offsets(i) = buffer.getLong
val fetchesCount = buffer.getInt
val fetchSizes = new Array[Int](fetchesCount)
for (i <- 0 until fetchSizes.length)
fetchSizes(i) = buffer.getInt
new OffsetDetail(topic, partitions, offsets, fetchSizes)
} }
class FetchRequest(val topic: String, }
val partition: Int,
val offset: Long, case class OffsetDetail(topic: String, partitions: Seq[Int], offsets: Seq[Long], fetchSizes: Seq[Int]) {
val maxSize: Int) extends Request(RequestKeys.Fetch) {
def writeTo(buffer: ByteBuffer) { def writeTo(buffer: ByteBuffer) {
Utils.writeShortString(buffer, topic) Utils.writeShortString(buffer, topic, "UTF-8")
buffer.putInt(partition)
buffer.putLong(offset) if(partitions.size > Int.MaxValue || offsets.size > Int.MaxValue || fetchSizes.size > Int.MaxValue)
buffer.putInt(maxSize) throw new IllegalArgumentException("Number of fetches in FetchRequest exceeds " + Int.MaxValue + ".")
buffer.putInt(partitions.length)
partitions.foreach(buffer.putInt(_))
buffer.putInt(offsets.length)
offsets.foreach(buffer.putLong(_))
buffer.putInt(fetchSizes.length)
fetchSizes.foreach(buffer.putInt(_))
} }
def sizeInBytes(): Int = 2 + topic.length + 4 + 8 + 4 def sizeInBytes(): Int = {
2 + topic.length() + // topic string
override def toString(): String= "FetchRequest(topic:" + topic + ", part:" + partition +" offset:" + offset + partitions.foldLeft(4)((s, _) => s + 4) + // each request partition (int)
" maxSize:" + maxSize + ")" offsets.foldLeft(4)((s, _) => s + 8) + // each request offset (long)
fetchSizes.foldLeft(4)((s,_) => s + 4) // each request fetch size
}
}
object FetchRequest {
val CurrentVersion = 1.shortValue()
def readFrom(buffer: ByteBuffer): FetchRequest = {
val versionId = buffer.getShort
val correlationId = buffer.getInt
val clientId = Utils.readShortString(buffer, "UTF-8")
val replicaId = buffer.getInt
val maxWait = buffer.getInt
val minBytes = buffer.getInt
val offsetsCount = buffer.getInt
val offsetInfo = new Array[OffsetDetail](offsetsCount)
for(i <- 0 until offsetInfo.length)
offsetInfo(i) = OffsetDetail.readFrom(buffer)
new FetchRequest(versionId, correlationId, clientId, replicaId, maxWait, minBytes, offsetInfo)
}
}
case class FetchRequest( versionId: Short,
correlationId: Int,
clientId: String,
replicaId: Int,
maxWait: Int,
minBytes: Int,
offsetInfo: Seq[OffsetDetail] ) extends Request(RequestKeys.Fetch) {
def writeTo(buffer: ByteBuffer) {
buffer.putShort(versionId)
buffer.putInt(correlationId)
Utils.writeShortString(buffer, clientId, "UTF-8")
buffer.putInt(replicaId)
buffer.putInt(maxWait)
buffer.putInt(minBytes)
buffer.putInt(offsetInfo.size)
for(topicDetail <- offsetInfo) {
topicDetail.writeTo(buffer)
}
}
def sizeInBytes: Int = 2 + 4 + (2 + clientId.length()) + 4 + 4 + 4 + offsetInfo.foldLeft(4)(_ + _.sizeInBytes())
}
class FetchRequestBuilder() {
private var correlationId = -1
private val versionId = FetchRequest.CurrentVersion
private var clientId = ""
private var replicaId = -1 // sensible default
private var maxWait = -1 // sensible default
private var minBytes = -1 // sensible default
private val requestMap = new HashMap[String, Tuple3[Buffer[Int], Buffer[Long], Buffer[Int]]]
def addFetch(topic: String, partition: Int, offset: Long, fetchSize: Int) = {
val topicData = requestMap.getOrElseUpdate(topic, (ListBuffer[Int](), ListBuffer[Long](), ListBuffer[Int]()))
topicData._1.append(partition)
topicData._2.append(offset)
topicData._3.append(fetchSize)
this
}
def correlationId(correlationId: Int): FetchRequestBuilder = {
this.correlationId = correlationId
this
}
def clientId(clientId: String): FetchRequestBuilder = {
this.clientId = clientId
this
}
def replicaId(replicaId: Int): FetchRequestBuilder = {
this.replicaId = replicaId
this
}
def maxWait(maxWait: Int): FetchRequestBuilder = {
this.maxWait = maxWait
this
}
def minBytes(minBytes: Int): FetchRequestBuilder = {
this.minBytes = minBytes
this
}
def build() = {
val offsetDetails = requestMap.map{ topicData =>
new OffsetDetail(topicData._1, topicData._2._1.toArray, topicData._2._2.toArray, topicData._2._3.toArray)
}
new FetchRequest(versionId, correlationId, clientId, replicaId, maxWait, minBytes, offsetDetails.toArray[OffsetDetail])
}
} }

View File

@ -0,0 +1,199 @@
/**
* 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 java.nio.ByteBuffer
import java.nio.channels.GatheringByteChannel
import kafka.common.ErrorMapping
import kafka.message.{MessageSet, ByteBufferMessageSet}
import kafka.network.{MultiSend, Send}
import kafka.utils.Utils
object PartitionData {
def readFrom(buffer: ByteBuffer): PartitionData = {
val partition = buffer.getInt
val error = buffer.getInt
val initialOffset = buffer.getLong
val messageSetSize = buffer.getInt
val messageSetBuffer = buffer.slice()
messageSetBuffer.limit(messageSetSize)
buffer.position(buffer.position + messageSetSize)
new PartitionData(partition, error, initialOffset, new ByteBufferMessageSet(messageSetBuffer, initialOffset, error))
}
}
case class PartitionData(partition: Int, error: Int = ErrorMapping.NoError, initialOffset:Long = 0L, messages: MessageSet) {
val sizeInBytes = 4 + 4 + 8 + 4 + messages.sizeInBytes.intValue()
}
object TopicData {
def readFrom(buffer: ByteBuffer): TopicData = {
val topic = Utils.readShortString(buffer, "UTF-8")
val partitionCount = buffer.getInt
val partitions = new Array[PartitionData](partitionCount)
for(i <- 0 until partitions.length)
partitions(i) = PartitionData.readFrom(buffer)
new TopicData(topic, partitions.sortBy(_.partition))
}
def findPartition(data: Array[PartitionData], partition: Int): Option[PartitionData] = {
if(data == null || data.size == 0)
return None
var (low, high) = (0, data.size-1)
while(low <= high) {
val mid = (low + high) / 2
val found = data(mid)
if(found.partition == partition)
return Some(found)
else if(partition < found.partition)
high = mid - 1
else
low = mid + 1
}
None
}
}
case class TopicData(topic: String, partitionData: Array[PartitionData]) {
val sizeInBytes = 2 + topic.length + partitionData.foldLeft(4)(_ + _.sizeInBytes)
}
object FetchResponse {
def readFrom(buffer: ByteBuffer): FetchResponse = {
val versionId = buffer.getShort
val correlationId = buffer.getInt
val dataCount = buffer.getInt
val data = new Array[TopicData](dataCount)
for(i <- 0 until data.length)
data(i) = TopicData.readFrom(buffer)
new FetchResponse(versionId, correlationId, data)
}
}
case class FetchResponse(versionId: Short, correlationId: Int, data: Array[TopicData]) {
val sizeInBytes = 2 + 4 + data.foldLeft(4)(_ + _.sizeInBytes)
lazy val topicMap = data.groupBy(_.topic).mapValues(_.head)
def messageSet(topic: String, partition: Int): ByteBufferMessageSet = {
val messageSet = topicMap.get(topic) match {
case Some(topicData) =>
TopicData.findPartition(topicData.partitionData, partition).map(_.messages).getOrElse(MessageSet.Empty)
case None =>
MessageSet.Empty
}
messageSet.asInstanceOf[ByteBufferMessageSet]
}
}
// SENDS
class PartitionDataSend(val partitionData: PartitionData) extends Send {
private val messageSize = partitionData.messages.sizeInBytes
private var messagesSentSize = 0L
private val buffer = ByteBuffer.allocate(20)
buffer.putInt(partitionData.partition)
buffer.putInt(partitionData.error)
buffer.putLong(partitionData.initialOffset)
buffer.putInt(partitionData.messages.sizeInBytes.intValue())
buffer.rewind()
def complete = !buffer.hasRemaining && messagesSentSize >= messageSize
def writeTo(channel: GatheringByteChannel): Int = {
var written = 0
if(buffer.hasRemaining)
written += channel.write(buffer)
if(!buffer.hasRemaining && messagesSentSize < messageSize) {
val bytesSent = partitionData.messages.writeTo(channel, messagesSentSize, messageSize - messagesSentSize).toInt
messagesSentSize += bytesSent
written += bytesSent
}
written
}
}
class TopicDataSend(val topicData: TopicData) extends Send {
val size = topicData.sizeInBytes
var sent = 0
private val buffer = ByteBuffer.allocate(2 + topicData.topic.length() + 4)
Utils.writeShortString(buffer, topicData.topic, "UTF-8")
buffer.putInt(topicData.partitionData.length)
buffer.rewind()
val sends = new MultiSend(topicData.partitionData.map(new PartitionDataSend(_)).toList) {
val expectedBytesToWrite = topicData.partitionData.foldLeft(0)(_ + _.sizeInBytes)
}
def complete = sent >= size
def writeTo(channel: GatheringByteChannel): Int = {
expectIncomplete()
var written = 0
if(buffer.hasRemaining)
written += channel.write(buffer)
if(!buffer.hasRemaining && !sends.complete) {
written += sends.writeCompletely(channel)
}
sent += written
written
}
}
class FetchResponseSend(val fetchResponse: FetchResponse,
val errorCode: Int = ErrorMapping.NoError) extends Send {
private val size = fetchResponse.sizeInBytes
private var sent = 0
private val buffer = ByteBuffer.allocate(16)
buffer.putInt(size + 2)
buffer.putShort(errorCode.shortValue())
buffer.putShort(fetchResponse.versionId)
buffer.putInt(fetchResponse.correlationId)
buffer.putInt(fetchResponse.data.length)
buffer.rewind()
val sends = new MultiSend(fetchResponse.data.map(new TopicDataSend(_)).toList) {
val expectedBytesToWrite = fetchResponse.data.foldLeft(0)(_ + _.sizeInBytes)
}
def complete = sent >= sendSize
def writeTo(channel: GatheringByteChannel):Int = {
expectIncomplete()
var written = 0
if(buffer.hasRemaining)
written += channel.write(buffer)
if(!buffer.hasRemaining && !sends.complete) {
written += sends.writeCompletely(channel)
}
sent += written
written
}
def sendSize = 4 + 2 + fetchResponse.sizeInBytes
}

View File

@ -1,58 +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.api
import java.nio._
import kafka.network._
object MultiFetchRequest {
def readFrom(buffer: ByteBuffer): MultiFetchRequest = {
val count = buffer.getShort
val fetches = new Array[FetchRequest](count)
for(i <- 0 until fetches.length)
fetches(i) = FetchRequest.readFrom(buffer)
new MultiFetchRequest(fetches)
}
}
class MultiFetchRequest(val fetches: Array[FetchRequest]) extends Request(RequestKeys.MultiFetch) {
def writeTo(buffer: ByteBuffer) {
if(fetches.length > Short.MaxValue)
throw new IllegalArgumentException("Number of requests in MultiFetchRequest exceeds " + Short.MaxValue + ".")
buffer.putShort(fetches.length.toShort)
for(fetch <- fetches)
fetch.writeTo(buffer)
}
def sizeInBytes: Int = {
var size = 2
for(fetch <- fetches)
size += fetch.sizeInBytes
size
}
override def toString(): String = {
val buffer = new StringBuffer
for(fetch <- fetches) {
buffer.append(fetch.toString)
buffer.append(",")
}
buffer.toString
}
}

View File

@ -1,52 +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.api
import java.nio._
import collection.mutable
import kafka.utils.IteratorTemplate
import kafka.message._
class MultiFetchResponse(val buffer: ByteBuffer, val numSets: Int, val offsets: Array[Long]) extends Iterable[ByteBufferMessageSet] {
private val messageSets = new mutable.ListBuffer[ByteBufferMessageSet]
for(i <- 0 until numSets) {
val size = buffer.getInt()
val errorCode: Int = buffer.getShort()
val copy = buffer.slice()
val payloadSize = size - 2
copy.limit(payloadSize)
buffer.position(buffer.position + payloadSize)
messageSets += new ByteBufferMessageSet(copy, offsets(i), errorCode)
}
def iterator : Iterator[ByteBufferMessageSet] = {
new IteratorTemplate[ByteBufferMessageSet] {
val iter = messageSets.iterator
override def makeNext(): ByteBufferMessageSet = {
if(iter.hasNext)
iter.next
else
return allDone
}
}
}
override def toString() = this.messageSets.toString
}

View File

@ -20,8 +20,7 @@ package kafka.api
object RequestKeys { object RequestKeys {
val Produce: Short = 0 val Produce: Short = 0
val Fetch: Short = 1 val Fetch: Short = 1
val MultiFetch: Short = 2 val MultiProduce: Short = 2
val MultiProduce: Short = 3 val Offsets: Short = 3
val Offsets: Short = 4 val TopicMetadata: Short = 4
val TopicMetadata: Short = 5
} }

View File

@ -50,8 +50,7 @@ class ConsumerConfig(props: Properties) extends ZKConfig(props) {
/** consumer id: generated automatically if not set. /** consumer id: generated automatically if not set.
* Set this explicitly for only testing purpose. */ * Set this explicitly for only testing purpose. */
val consumerId: Option[String] = /** TODO: can be written better in scala 2.8 */ val consumerId: Option[String] = Option(Utils.getString(props, "consumerid", null))
if (Utils.getString(props, "consumerid", null) != null) Some(Utils.getString(props, "consumerid")) else None
/** the socket timeout for network requests */ /** the socket timeout for network requests */
val socketTimeoutMs = Utils.getInt(props, "socket.timeout.ms", SocketTimeout) val socketTimeoutMs = Utils.getInt(props, "socket.timeout.ms", SocketTimeout)

View File

@ -17,13 +17,14 @@
package kafka.consumer package kafka.consumer
import java.util.concurrent.CountDownLatch
import kafka.common.ErrorMapping
import kafka.cluster.{Partition, Broker}
import kafka.api.{OffsetRequest, FetchRequest}
import org.I0Itec.zkclient.ZkClient
import kafka.utils._
import java.io.IOException import java.io.IOException
import java.util.concurrent.CountDownLatch
import kafka.api.{FetchRequestBuilder, OffsetRequest}
import kafka.cluster.{Partition, Broker}
import kafka.common.ErrorMapping
import kafka.message.ByteBufferMessageSet
import kafka.utils._
import org.I0Itec.zkclient.ZkClient
class FetcherRunnable(val name: String, class FetcherRunnable(val name: String,
val zkClient : ZkClient, val zkClient : ZkClient,
@ -50,18 +51,26 @@ class FetcherRunnable(val name: String,
info(name + " start fetching topic: " + infopti.topic + " part: " + infopti.partition.partId + " offset: " info(name + " start fetching topic: " + infopti.topic + " part: " + infopti.partition.partId + " offset: "
+ infopti.getFetchOffset + " from " + broker.host + ":" + broker.port) + infopti.getFetchOffset + " from " + broker.host + ":" + broker.port)
var reqId = 0
try { try {
while (!stopped) { while (!stopped) {
val fetches = partitionTopicInfos.map(info => // TODO: fix up the max wait and min bytes
new FetchRequest(info.topic, info.partition.partId, info.getFetchOffset, config.fetchSize)) val builder = new FetchRequestBuilder().
correlationId(reqId).
clientId(config.consumerId.getOrElse(name)).
maxWait(0).
minBytes(0)
partitionTopicInfos.foreach(pti =>
builder.addFetch(pti.topic, pti.partition.partId, pti.getFetchOffset(), config.fetchSize)
)
trace("fetch request: " + fetches.toString) val fetchRequest = builder.build()
trace("fetch request: " + fetchRequest)
val response = simpleConsumer.multifetch(fetches : _*) val response = simpleConsumer.fetch(fetchRequest)
var read = 0L var read = 0L
for(infopti <- partitionTopicInfos) {
for((messages, infopti) <- response.zip(partitionTopicInfos)) { val messages = response.messageSet(infopti.topic, infopti.partition.partId).asInstanceOf[ByteBufferMessageSet]
try { try {
var done = false var done = false
if(messages.getErrorCode == ErrorMapping.OffsetOutOfRangeCode) { if(messages.getErrorCode == ErrorMapping.OffsetOutOfRangeCode) {
@ -76,8 +85,7 @@ class FetcherRunnable(val name: String,
} }
if (!done) if (!done)
read += infopti.enqueue(messages, infopti.getFetchOffset) read += infopti.enqueue(messages, infopti.getFetchOffset)
} } catch {
catch {
case e1: IOException => case e1: IOException =>
// something is wrong with the socket, re-throw the exception to stop the fetcher // something is wrong with the socket, re-throw the exception to stop the fetcher
throw e1 throw e1
@ -91,6 +99,7 @@ class FetcherRunnable(val name: String,
throw e2 throw e2
} }
} }
reqId = if(reqId == Int.MaxValue) 0 else reqId + 1
trace("fetched bytes: " + read) trace("fetched bytes: " + read)
if(read == 0) { if(read == 0) {
@ -98,8 +107,7 @@ class FetcherRunnable(val name: String,
Thread.sleep(config.fetcherBackoffMs) Thread.sleep(config.fetcherBackoffMs)
} }
} }
} } catch {
catch {
case e => case e =>
if (stopped) if (stopped)
info("FecherRunnable " + this + " interrupted") info("FecherRunnable " + this + " interrupted")

View File

@ -20,7 +20,6 @@ package kafka.consumer
import java.net._ import java.net._
import java.nio.channels._ import java.nio.channels._
import kafka.api._ import kafka.api._
import kafka.message._
import kafka.network._ import kafka.network._
import kafka.utils._ import kafka.utils._
@ -72,7 +71,7 @@ class SimpleConsumer(val host: String,
* @param request specifies the topic name, topic partition, starting byte offset, maximum bytes to be fetched. * @param request specifies the topic name, topic partition, starting byte offset, maximum bytes to be fetched.
* @return a set of fetched messages * @return a set of fetched messages
*/ */
def fetch(request: FetchRequest): ByteBufferMessageSet = { def fetch(request: FetchRequest): FetchResponse = {
lock synchronized { lock synchronized {
val startTime = SystemTime.nanoseconds val startTime = SystemTime.nanoseconds
getOrMakeConnection() getOrMakeConnection()
@ -93,46 +92,14 @@ class SimpleConsumer(val host: String,
} }
case e => throw e case e => throw e
} }
val fetchResponse = FetchResponse.readFrom(response._1.buffer)
val fetchedSize = fetchResponse.sizeInBytes
val endTime = SystemTime.nanoseconds val endTime = SystemTime.nanoseconds
SimpleConsumerStats.recordFetchRequest(endTime - startTime) SimpleConsumerStats.recordFetchRequest(endTime - startTime)
SimpleConsumerStats.recordConsumptionThroughput(response._1.buffer.limit) SimpleConsumerStats.recordConsumptionThroughput(fetchedSize)
new ByteBufferMessageSet(response._1.buffer, request.offset, response._2)
}
}
/** fetchResponse
* Combine multiple fetch requests in one call.
*
* @param fetches a sequence of fetch requests.
* @return a sequence of fetch responses
*/
def multifetch(fetches: FetchRequest*): MultiFetchResponse = {
lock synchronized {
val startTime = SystemTime.nanoseconds
getOrMakeConnection()
var response: Tuple2[Receive,Int] = null
try {
sendRequest(new MultiFetchRequest(fetches.toArray))
response = getResponse
} catch {
case e : java.io.IOException =>
info("Reconnect in multifetch due to socket error: ", e)
// retry once
try {
channel = connect
sendRequest(new MultiFetchRequest(fetches.toArray))
response = getResponse
}catch {
case ioe: java.io.IOException => channel = null; throw ioe;
}
case e => throw e
}
val endTime = SystemTime.nanoseconds
SimpleConsumerStats.recordFetchRequest(endTime - startTime)
SimpleConsumerStats.recordConsumptionThroughput(response._1.buffer.limit)
// error code will be set on individual messageset inside MultiFetchResponse
new MultiFetchResponse(response._1.buffer, fetches.length, fetches.toArray.map(f => f.offset))
} }
} }

View File

@ -32,8 +32,7 @@ private[kafka] object TopicCount extends Logging {
case Some(m) => topMap = m.asInstanceOf[Map[String,Int]] case Some(m) => topMap = m.asInstanceOf[Map[String,Int]]
case None => throw new RuntimeException("error constructing TopicCount : " + jsonString) case None => throw new RuntimeException("error constructing TopicCount : " + jsonString)
} }
} } catch {
catch {
case e => case e =>
error("error parsing consumer json string " + jsonString, e) error("error parsing consumer json string " + jsonString, e)
throw e throw e
@ -46,8 +45,7 @@ private[kafka] object TopicCount extends Logging {
private[kafka] class TopicCount(val consumerIdString: String, val topicCountMap: Map[String, Int]) { private[kafka] class TopicCount(val consumerIdString: String, val topicCountMap: Map[String, Int]) {
def getConsumerThreadIdsPerTopic() def getConsumerThreadIdsPerTopic(): Map[String, Set[String]] = {
: Map[String, Set[String]] = {
val consumerThreadIdsPerTopicMap = new mutable.HashMap[String, Set[String]]() val consumerThreadIdsPerTopicMap = new mutable.HashMap[String, Set[String]]()
for ((topic, nConsumers) <- topicCountMap) { for ((topic, nConsumers) <- topicCountMap) {
val consumerSet = new mutable.HashSet[String] val consumerSet = new mutable.HashSet[String]

View File

@ -105,8 +105,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
def this(config: ConsumerConfig) = this(config, true) def this(config: ConsumerConfig) = this(config, true)
def createMessageStreams[T](topicCountMap: Map[String,Int], def createMessageStreams[T](topicCountMap: Map[String,Int], decoder: Decoder[T])
decoder: Decoder[T])
: Map[String,List[KafkaMessageStream[T]]] = { : Map[String,List[KafkaMessageStream[T]]] = {
consume(topicCountMap, decoder) consume(topicCountMap, decoder)
} }
@ -138,8 +137,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
zkClient.close() zkClient.close()
zkClient = null zkClient = null
} }
} } catch {
catch {
case e => case e =>
fatal("error during consumer connector shutdown", e) fatal("error during consumer connector shutdown", e)
} }
@ -147,8 +145,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
} }
} }
def consume[T](topicCountMap: scala.collection.Map[String,Int], def consume[T](topicCountMap: scala.collection.Map[String,Int], decoder: Decoder[T])
decoder: Decoder[T])
: Map[String,List[KafkaMessageStream[T]]] = { : Map[String,List[KafkaMessageStream[T]]] = {
debug("entering consume ") debug("entering consume ")
if (topicCountMap == null) if (topicCountMap == null)
@ -159,12 +156,12 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
var consumerUuid : String = null var consumerUuid : String = null
config.consumerId match { config.consumerId match {
case Some(consumerId) // for testing only case Some(consumerId) => // for testing only
=> consumerUuid = consumerId consumerUuid = consumerId
case None // generate unique consumerId automatically case None => // generate unique consumerId automatically
=> val uuid = UUID.randomUUID() val uuid = UUID.randomUUID()
consumerUuid = "%s-%d-%s".format( consumerUuid = "%s-%d-%s".format( InetAddress.getLocalHost.getHostName,
InetAddress.getLocalHost.getHostName, System.currentTimeMillis, System.currentTimeMillis,
uuid.getMostSignificantBits().toHexString.substring(0,8) ) uuid.getMostSignificantBits().toHexString.substring(0,8) )
} }
val consumerIdString = config.groupId + "_" + consumerUuid val consumerIdString = config.groupId + "_" + consumerUuid
@ -243,8 +240,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
try { try {
updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + info.partition.name, updatePersistentPath(zkClient, topicDirs.consumerOffsetDir + "/" + info.partition.name,
newOffset.toString) newOffset.toString)
} } catch {
catch {
case t: Throwable => case t: Throwable =>
// log it and let it go // log it and let it go
warn("exception during commitOffsets", t) warn("exception during commitOffsets", t)
@ -321,8 +317,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
ConsumerConfig.SocketBufferSize) ConsumerConfig.SocketBufferSize)
val offsets = simpleConsumer.getOffsetsBefore(topic, partitionId, earliestOrLatest, 1) val offsets = simpleConsumer.getOffsetsBefore(topic, partitionId, earliestOrLatest, 1)
producedOffset = offsets(0) producedOffset = offsets(0)
} } catch {
catch {
case e => case e =>
error("error in earliestOrLatestOffset() ", e) error("error in earliestOrLatestOffset() ", e)
} }
@ -419,8 +414,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
val cluster = getCluster(zkClient) val cluster = getCluster(zkClient)
try { try {
done = rebalance(cluster) done = rebalance(cluster)
} } catch {
catch {
case e => case e =>
/** occasionally, we may hit a ZK exception because the ZK state is changing while we are iterating. /** occasionally, we may hit a ZK exception because the ZK state is changing while we are iterating.
* For example, a ZK node can disappear between the time we get all children and the time we try to get * For example, a ZK node can disappear between the time we get all children and the time we try to get
@ -611,8 +605,7 @@ private[kafka] class ZookeeperConsumerConnector(val config: ConsumerConfig,
createEphemeralPathExpectConflict(zkClient, partitionOwnerPath, consumerThreadId) createEphemeralPathExpectConflict(zkClient, partitionOwnerPath, consumerThreadId)
info(consumerThreadId + " successfully owned partition " + partition + " for topic " + topic) info(consumerThreadId + " successfully owned partition " + partition + " for topic " + topic)
true true
} } catch {
catch {
case e: ZkNodeExistsException => case e: ZkNodeExistsException =>
// The node hasn't been deleted by the original owner. So wait a bit and retry. // The node hasn't been deleted by the original owner. So wait a bit and retry.
info("waiting for the partition ownership to be deleted: " + partition) info("waiting for the partition ownership to be deleted: " + partition)

View File

@ -0,0 +1,33 @@
/**
* 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.javaapi
import kafka.api.TopicData
class FetchResponse( val versionId: Short,
val correlationId: Int,
val data: Array[TopicData] ) {
private val underlying = new kafka.api.FetchResponse(versionId, correlationId, data)
def messageSet(topic: String, partition: Int): kafka.javaapi.message.ByteBufferMessageSet = {
import Implicits._
underlying.messageSet(topic, partition)
}
}

View File

@ -28,9 +28,6 @@ private[javaapi] object Implicits extends Logging {
messageSet.getErrorCode) messageSet.getErrorCode)
} }
implicit def toMultiFetchResponse(response: kafka.javaapi.MultiFetchResponse): kafka.api.MultiFetchResponse = implicit def toJavaFetchResponse(response: kafka.api.FetchResponse): kafka.javaapi.FetchResponse =
response.underlying new kafka.javaapi.FetchResponse(response.versionId, response.correlationId, response.data)
implicit def toJavaMultiFetchResponse(response: kafka.api.MultiFetchResponse): kafka.javaapi.MultiFetchResponse =
new kafka.javaapi.MultiFetchResponse(response.buffer, response.numSets, response.offsets)
} }

View File

@ -1,45 +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.javaapi
import kafka.utils.IteratorTemplate
import java.nio.ByteBuffer
import message.ByteBufferMessageSet
class MultiFetchResponse(buffer: ByteBuffer, numSets: Int, offsets: Array[Long]) extends java.lang.Iterable[ByteBufferMessageSet] {
val underlyingBuffer = ByteBuffer.wrap(buffer.array)
// this has the side effect of setting the initial position of buffer correctly
val errorCode = underlyingBuffer.getShort
import Implicits._
val underlying = new kafka.api.MultiFetchResponse(underlyingBuffer, numSets, offsets)
override def toString() = underlying.toString
def iterator : java.util.Iterator[ByteBufferMessageSet] = {
new IteratorTemplate[ByteBufferMessageSet] {
val iter = underlying.iterator
override def makeNext(): ByteBufferMessageSet = {
if(iter.hasNext)
iter.next
else
return allDone
}
}
}
}

View File

@ -17,10 +17,9 @@
package kafka.javaapi.consumer package kafka.javaapi.consumer
import kafka.utils.threadsafe
import kafka.javaapi.message.ByteBufferMessageSet
import kafka.javaapi.MultiFetchResponse
import kafka.api.FetchRequest import kafka.api.FetchRequest
import kafka.javaapi.FetchResponse
import kafka.utils.threadsafe
/** /**
* A consumer of kafka messages * A consumer of kafka messages
@ -38,23 +37,11 @@ class SimpleConsumer(val host: String,
* @param request specifies the topic name, topic partition, starting byte offset, maximum bytes to be fetched. * @param request specifies the topic name, topic partition, starting byte offset, maximum bytes to be fetched.
* @return a set of fetched messages * @return a set of fetched messages
*/ */
def fetch(request: FetchRequest): ByteBufferMessageSet = { def fetch(request: FetchRequest): FetchResponse = {
import kafka.javaapi.Implicits._ import kafka.javaapi.Implicits._
underlying.fetch(request) underlying.fetch(request)
} }
/**
* Combine multiple fetch requests in one call.
*
* @param fetches a sequence of fetch requests.
* @return a sequence of fetch responses
*/
def multifetch(fetches: java.util.List[FetchRequest]): MultiFetchResponse = {
import scala.collection.JavaConversions._
import kafka.javaapi.Implicits._
underlying.multifetch(asBuffer(fetches): _*)
}
/** /**
* Get a list of valid offsets (up to maxSize) before the given time. * Get a list of valid offsets (up to maxSize) before the given time.
* The result is a list of offsets, in descending order. * The result is a list of offsets, in descending order.

View File

@ -28,7 +28,7 @@ import kafka.utils._
@nonthreadsafe @nonthreadsafe
private[kafka] class BoundedByteBufferReceive(val maxSize: Int) extends Receive with Logging { private[kafka] class BoundedByteBufferReceive(val maxSize: Int) extends Receive with Logging {
private val sizeBuffer: ByteBuffer = ByteBuffer.allocate(4) private val sizeBuffer = ByteBuffer.allocate(4)
private var contentBuffer: ByteBuffer = null private var contentBuffer: ByteBuffer = null
def this() = this(Int.MaxValue) def this() = this(Int.MaxValue)
@ -78,12 +78,10 @@ private[kafka] class BoundedByteBufferReceive(val maxSize: Int) extends Receive
var buffer: ByteBuffer = null var buffer: ByteBuffer = null
try { try {
buffer = ByteBuffer.allocate(size) buffer = ByteBuffer.allocate(size)
} } catch {
catch { case e: OutOfMemoryError =>
case e: OutOfMemoryError => {
error("OOME with size " + size, e) error("OOME with size " + size, e)
throw e throw e
}
case e2 => case e2 =>
throw e2 throw e2
} }

View File

@ -21,8 +21,8 @@ import java.util.concurrent._
object RequestChannel { object RequestChannel {
val AllDone = new Request(1, 2, null, 0) val AllDone = new Request(1, 2, null, 0)
case class Request(val processor: Int, requestKey: Any, request: Receive, start: Long) case class Request(processor: Int, requestKey: Any, request: Receive, start: Long)
case class Response(val processor: Int, requestKey: Any, response: Send, start: Long, ellapsed: Long) case class Response(processor: Int, requestKey: Any, response: Send, start: Long, elapsed: Long)
} }
class RequestChannel(val numProcessors: Int, val queueSize: Int) { class RequestChannel(val numProcessors: Int, val queueSize: Int) {

View File

@ -50,7 +50,7 @@ class SocketServerStats(val monitorDurationNs: Long, val time: Time) extends Soc
requestTypeId match { requestTypeId match {
case r if r == RequestKeys.Produce || r == RequestKeys.MultiProduce => case r if r == RequestKeys.Produce || r == RequestKeys.MultiProduce =>
produceTimeStats.recordRequestMetric(durationNs) produceTimeStats.recordRequestMetric(durationNs)
case r if r == RequestKeys.Fetch || r == RequestKeys.MultiFetch => case r if r == RequestKeys.Fetch =>
fetchTimeStats.recordRequestMetric(durationNs) fetchTimeStats.recordRequestMetric(durationNs)
case _ => /* not collecting; let go */ case _ => /* not collecting; let go */
} }

View File

@ -68,12 +68,13 @@ trait Send extends Transmission {
def writeTo(channel: GatheringByteChannel): Int def writeTo(channel: GatheringByteChannel): Int
def writeCompletely(channel: GatheringByteChannel): Int = { def writeCompletely(channel: GatheringByteChannel): Int = {
var written = 0 var totalWritten = 0
while(!complete) { while(!complete) {
written = writeTo(channel) val written = writeTo(channel)
trace(written + " bytes written.") trace(written + " bytes written.")
totalWritten += written
} }
written totalWritten
} }
} }
@ -99,9 +100,9 @@ abstract class MultiSend[S <: Send](val sends: List[S]) extends Send {
if (current == Nil) { if (current == Nil) {
if (totalWritten != expectedBytesToWrite) if (totalWritten != expectedBytesToWrite)
error("mismatch in sending bytes over socket; expected: " + expectedBytesToWrite + " actual: " + totalWritten) error("mismatch in sending bytes over socket; expected: " + expectedBytesToWrite + " actual: " + totalWritten)
return true true
} } else {
else false
return false }
} }
} }

View File

@ -17,17 +17,17 @@
package kafka.server package kafka.server
import org.apache.log4j.Logger import java.io.IOException
import kafka.log._ import java.lang.IllegalStateException
import kafka.network._ import kafka.admin.{CreateTopicCommand, AdminUtils}
import kafka.message._
import kafka.api._ import kafka.api._
import kafka.common.ErrorMapping import kafka.common.ErrorMapping
import java.io.IOException import kafka.log._
import kafka.message._
import kafka.network._
import kafka.utils.{SystemTime, Logging} import kafka.utils.{SystemTime, Logging}
import collection.mutable.ListBuffer import org.apache.log4j.Logger
import kafka.admin.{CreateTopicCommand, AdminUtils} import scala.collection.mutable.ListBuffer
import java.lang.IllegalStateException
/** /**
* Logic to handle the various Kafka requests * Logic to handle the various Kafka requests
@ -41,7 +41,6 @@ class KafkaApis(val logManager: LogManager) extends Logging {
apiId match { apiId match {
case RequestKeys.Produce => handleProducerRequest(receive) case RequestKeys.Produce => handleProducerRequest(receive)
case RequestKeys.Fetch => handleFetchRequest(receive) case RequestKeys.Fetch => handleFetchRequest(receive)
case RequestKeys.MultiFetch => handleMultiFetchRequest(receive)
case RequestKeys.MultiProduce => handleMultiProducerRequest(receive) case RequestKeys.MultiProduce => handleMultiProducerRequest(receive)
case RequestKeys.Offsets => handleOffsetRequest(receive) case RequestKeys.Offsets => handleOffsetRequest(receive)
case RequestKeys.TopicMetadata => handleTopicMetadataRequest(receive) case RequestKeys.TopicMetadata => handleTopicMetadataRequest(receive)
@ -92,34 +91,37 @@ class KafkaApis(val logManager: LogManager) extends Logging {
val fetchRequest = FetchRequest.readFrom(request.buffer) val fetchRequest = FetchRequest.readFrom(request.buffer)
if(requestLogger.isTraceEnabled) if(requestLogger.isTraceEnabled)
requestLogger.trace("Fetch request " + fetchRequest.toString) requestLogger.trace("Fetch request " + fetchRequest.toString)
Some(readMessageSet(fetchRequest))
val fetchedData = new ListBuffer[TopicData]()
var error: Int = ErrorMapping.NoError
for(offsetDetail <- fetchRequest.offsetInfo) {
val info = new ListBuffer[PartitionData]()
val topic = offsetDetail.topic
val (partitions, offsets, fetchSizes) = (offsetDetail.partitions, offsetDetail.offsets, offsetDetail.fetchSizes)
for( (partition, offset, fetchSize) <- (partitions, offsets, fetchSizes).zipped.map((_,_,_)) ) {
val partitionInfo = readMessageSet(topic, partition, offset, fetchSize) match {
case Left(err) => error = err; new PartitionData(partition, err, offset, MessageSet.Empty)
case Right(messages) => new PartitionData(partition, ErrorMapping.NoError, offset, messages)
}
info.append(partitionInfo)
}
fetchedData.append(new TopicData(topic, info.toArray))
}
val response = new FetchResponse(FetchRequest.CurrentVersion, fetchRequest.correlationId, fetchedData.toArray )
Some(new FetchResponseSend(response, error))
} }
def handleMultiFetchRequest(request: Receive): Option[Send] = { private def readMessageSet(topic: String, partition: Int, offset: Long, maxSize: Int): Either[Int, MessageSet] = {
val multiFetchRequest = MultiFetchRequest.readFrom(request.buffer) var response: Either[Int, MessageSet] = null
if(requestLogger.isTraceEnabled)
requestLogger.trace("Multifetch request")
multiFetchRequest.fetches.foreach(req => requestLogger.trace(req.toString))
var responses = multiFetchRequest.fetches.map(fetch =>
readMessageSet(fetch)).toList
Some(new MultiMessageSetSend(responses))
}
private def readMessageSet(fetchRequest: FetchRequest): MessageSetSend = {
var response: MessageSetSend = null
try { try {
trace("Fetching log segment for topic = " + fetchRequest.topic + " and partition = " + fetchRequest.partition) trace("Fetching log segment for topic, partition, offset, size = " + (topic, partition, offset, maxSize))
val log = logManager.getLog(fetchRequest.topic, fetchRequest.partition) val log = logManager.getLog(topic, partition)
if (log != null) response = Right(if(log != null) log.read(offset, maxSize) else MessageSet.Empty)
response = new MessageSetSend(log.read(fetchRequest.offset, fetchRequest.maxSize)) } catch {
else
response = new MessageSetSend()
}
catch {
case e => case e =>
error("error when processing request " + fetchRequest, e) error("error when processing request " + (topic, partition, offset, maxSize), e)
response=new MessageSetSend(MessageSet.Empty, ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]])) response = Left(ErrorMapping.codeFor(e.getClass.asInstanceOf[Class[Throwable]]))
} }
response response
} }

View File

@ -37,7 +37,7 @@ class KafkaRequestHandler(val requestChannel: RequestChannel, val handle: (Recei
requestKey = req.requestKey, requestKey = req.requestKey,
response = send, response = send,
start = req.start, start = req.start,
ellapsed = -1) elapsed = -1)
requestChannel.sendResponse(resp) requestChannel.sendResponse(resp)
trace("Processor " + Thread.currentThread.getName + " sent response " + resp) trace("Processor " + Thread.currentThread.getName + " sent response " + resp)
} }

View File

@ -19,7 +19,7 @@ package kafka.tools
import java.net.URI import java.net.URI
import joptsimple._ import joptsimple._
import kafka.api.FetchRequest import kafka.api.FetchRequestBuilder
import kafka.utils._ import kafka.utils._
import kafka.consumer._ import kafka.consumer._
@ -54,6 +54,11 @@ object SimpleConsumerShell extends Logging {
.describedAs("fetchsize") .describedAs("fetchsize")
.ofType(classOf[java.lang.Integer]) .ofType(classOf[java.lang.Integer])
.defaultsTo(1000000) .defaultsTo(1000000)
val clientIdOpt = parser.accepts("clientId", "The ID of this client.")
.withOptionalArg
.describedAs("clientId")
.ofType(classOf[String])
.defaultsTo("SimpleConsumerShell")
val printOffsetOpt = parser.accepts("print-offsets", "Print the offsets returned by the iterator") val printOffsetOpt = parser.accepts("print-offsets", "Print the offsets returned by the iterator")
.withOptionalArg .withOptionalArg
.describedAs("print offsets") .describedAs("print offsets")
@ -79,7 +84,8 @@ object SimpleConsumerShell extends Logging {
val topic = options.valueOf(topicOpt) val topic = options.valueOf(topicOpt)
val partition = options.valueOf(partitionOpt).intValue val partition = options.valueOf(partitionOpt).intValue
val startingOffset = options.valueOf(offsetOpt).longValue val startingOffset = options.valueOf(offsetOpt).longValue
val fetchsize = options.valueOf(fetchsizeOpt).intValue val fetchSize = options.valueOf(fetchsizeOpt).intValue
val clientId = options.valueOf(clientIdOpt).toString
val printOffsets = if(options.has(printOffsetOpt)) true else false val printOffsets = if(options.has(printOffsetOpt)) true else false
val printMessages = if(options.has(printMessageOpt)) true else false val printMessages = if(options.has(printMessageOpt)) true else false
@ -87,14 +93,19 @@ object SimpleConsumerShell extends Logging {
val consumer = new SimpleConsumer(url.getHost, url.getPort, 10000, 64*1024) val consumer = new SimpleConsumer(url.getHost, url.getPort, 10000, 64*1024)
val thread = Utils.newThread("kafka-consumer", new Runnable() { val thread = Utils.newThread("kafka-consumer", new Runnable() {
def run() { def run() {
var reqId = 0
var offset = startingOffset var offset = startingOffset
while(true) { while(true) {
val fetchRequest = new FetchRequest(topic, partition, offset, fetchsize) val fetchRequest = new FetchRequestBuilder()
val messageSets = consumer.multifetch(fetchRequest) .correlationId(reqId)
for (messages <- messageSets) { .clientId(clientId)
debug("multi fetched " + messages.sizeInBytes + " bytes from offset " + offset) .addFetch(topic, partition, offset, fetchSize)
.build()
val fetchResponse = consumer.fetch(fetchRequest)
val messageSet = fetchResponse.messageSet(topic, partition)
debug("multi fetched " + messageSet.sizeInBytes + " bytes from offset " + offset)
var consumed = 0 var consumed = 0
for(messageAndOffset <- messages) { for(messageAndOffset <- messageSet) {
if(printMessages) if(printMessages)
info("consumed: " + Utils.toString(messageAndOffset.message.payload, "UTF-8")) info("consumed: " + Utils.toString(messageAndOffset.message.payload, "UTF-8"))
offset = messageAndOffset.offset offset = messageAndOffset.offset
@ -102,7 +113,7 @@ object SimpleConsumerShell extends Logging {
info("next offset = " + offset) info("next offset = " + offset)
consumed += 1 consumed += 1
} }
} reqId += 1
} }
} }
}, false); }, false);

View File

@ -17,15 +17,17 @@
package kafka.integration package kafka.integration
import kafka.server.KafkaConfig
import org.scalatest.junit.JUnit3Suite
import org.apache.log4j.Logger
import java.util.Properties
import kafka.consumer.SimpleConsumer
import kafka.api.{OffsetRequest, FetchRequest}
import junit.framework.Assert._ import junit.framework.Assert._
import java.util.Properties
import kafka.api.{FetchRequestBuilder, OffsetRequest}
import kafka.consumer.SimpleConsumer
import kafka.server.KafkaConfig
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.log4j.Logger
import org.scalatest.junit.JUnit3Suite
class BackwardsCompatibilityTest extends JUnit3Suite with KafkaServerTestHarness { class BackwardsCompatibilityTest extends JUnit3Suite with KafkaServerTestHarness {
val topic = "MagicByte0" val topic = "MagicByte0"
@ -62,9 +64,10 @@ class BackwardsCompatibilityTest extends JUnit3Suite with KafkaServerTestHarness
var messageCount: Int = 0 var messageCount: Int = 0
while(fetchOffset < lastOffset(0)) { while(fetchOffset < lastOffset(0)) {
val fetched = simpleConsumer.fetch(new FetchRequest(topic, 0, fetchOffset, 10000)) val fetched = simpleConsumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, fetchOffset, 10000).build())
fetched.foreach(m => fetchOffset = m.offset) val fetchedMessages = fetched.messageSet(topic, 0)
messageCount += fetched.size fetchedMessages.foreach(m => fetchOffset = m.offset)
messageCount += fetchedMessages.size
} }
assertEquals(100, messageCount) assertEquals(100, messageCount)
} }

View File

@ -17,15 +17,15 @@
package kafka.integration package kafka.integration
import scala.collection._ import kafka.api.{FetchRequestBuilder, ProducerRequest}
import kafka.common.OffsetOutOfRangeException import kafka.common.OffsetOutOfRangeException
import kafka.api.{ProducerRequest, FetchRequest} import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet}
import kafka.server.{KafkaRequestHandler, KafkaServer, KafkaConfig} import kafka.server.{KafkaRequestHandler, KafkaServer, KafkaConfig}
import kafka.utils.{TestUtils, Utils}
import kafka.zk.ZooKeeperTestHarness
import org.apache.log4j.{Level, Logger} import org.apache.log4j.{Level, Logger}
import org.scalatest.junit.JUnit3Suite import org.scalatest.junit.JUnit3Suite
import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet} import scala.collection._
import kafka.zk.ZooKeeperTestHarness
import kafka.utils.{TestUtils, Utils}
/** /**
* End to end tests of the primitive apis against a local server * End to end tests of the primitive apis against a local server
@ -65,54 +65,60 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness
new Message("hello".getBytes()), new Message("there".getBytes())) new Message("hello".getBytes()), new Message("there".getBytes()))
producer.send(topic, sent) producer.send(topic, sent)
sent.getBuffer.rewind sent.getBuffer.rewind
var fetched: ByteBufferMessageSet = null
while(fetched == null || fetched.validBytes == 0) var fetchedMessage: ByteBufferMessageSet = null
fetched = consumer.fetch(new FetchRequest(topic, 0, 0, 10000)) while(fetchedMessage == null || fetchedMessage.validBytes == 0) {
TestUtils.checkEquals(sent.iterator, fetched.iterator) val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build())
fetchedMessage = fetched.messageSet(topic, 0)
}
TestUtils.checkEquals(sent.iterator, fetchedMessage.iterator)
// send an invalid offset // send an invalid offset
try { try {
val fetchedWithError = consumer.fetch(new FetchRequest(topic, 0, -1, 10000)) val fetchedWithError = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, -1, 10000).build())
fetchedWithError.iterator fetchedWithError.messageSet(topic, 0).iterator
fail("Expected an OffsetOutOfRangeException exception to be thrown") fail("Expected an OffsetOutOfRangeException exception to be thrown")
} } catch {
catch {
case e: OffsetOutOfRangeException => case e: OffsetOutOfRangeException =>
} }
} }
def testProduceAndMultiFetch() { def testProduceAndMultiFetch() {
// send some messages // send some messages, with non-ordered topics
val topics = List("test1", "test2", "test3"); val topicOffsets = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
{ {
val messages = new mutable.HashMap[String, ByteBufferMessageSet] val messages = new mutable.HashMap[String, ByteBufferMessageSet]
val fetches = new mutable.ArrayBuffer[FetchRequest] val builder = new FetchRequestBuilder()
for(topic <- topics) { for( (topic, offset) <- topicOffsets) {
val set = new ByteBufferMessageSet(NoCompressionCodec, val set = new ByteBufferMessageSet(NoCompressionCodec,
new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
messages += topic -> set
producer.send(topic, set) producer.send(topic, set)
set.getBuffer.rewind set.getBuffer.rewind
fetches += new FetchRequest(topic, 0, 0, 10000) messages += topic -> set
builder.addFetch(topic, offset, 0, 10000)
} }
// wait a bit for produced message to be available // wait a bit for produced message to be available
Thread.sleep(200) Thread.sleep(200)
val response = consumer.multifetch(fetches: _*) val request = builder.build()
for((topic, resp) <- topics.zip(response.toList)) val response = consumer.fetch(request)
TestUtils.checkEquals(messages(topic).iterator, resp.iterator) for( (topic, offset) <- topicOffsets) {
val fetched = response.messageSet(topic, offset)
TestUtils.checkEquals(messages(topic).iterator, fetched.iterator)
}
} }
{ {
// send some invalid offsets // send some invalid offsets
val fetches = new mutable.ArrayBuffer[FetchRequest] val builder = new FetchRequestBuilder()
for(topic <- topics) for( (topic, offset) <- topicOffsets )
fetches += new FetchRequest(topic, 0, -1, 10000) builder.addFetch(topic, offset, -1, 10000)
val responses = consumer.multifetch(fetches: _*) val request = builder.build()
for(resp <- responses) { val responses = consumer.fetch(request)
for( (topic, offset) <- topicOffsets ) {
try { try {
resp.iterator responses.messageSet(topic, offset).iterator
fail("Expected an OffsetOutOfRangeException exception to be thrown") fail("Expected an OffsetOutOfRangeException exception to be thrown")
} catch { } catch {
case e: OffsetOutOfRangeException => case e: OffsetOutOfRangeException =>
@ -125,14 +131,14 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness
// send some messages // send some messages
val topics = List("test1", "test2", "test3"); val topics = List("test1", "test2", "test3");
val messages = new mutable.HashMap[String, ByteBufferMessageSet] val messages = new mutable.HashMap[String, ByteBufferMessageSet]
val fetches = new mutable.ArrayBuffer[FetchRequest] val builder = new FetchRequestBuilder()
var produceList: List[ProducerRequest] = Nil var produceList: List[ProducerRequest] = Nil
for(topic <- topics) { for(topic <- topics) {
val set = new ByteBufferMessageSet(NoCompressionCodec, val set = new ByteBufferMessageSet(NoCompressionCodec,
new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
messages += topic -> set messages += topic -> set
produceList ::= new ProducerRequest(topic, 0, set) produceList ::= new ProducerRequest(topic, 0, set)
fetches += new FetchRequest(topic, 0, 0, 10000) builder.addFetch(topic, 0, 0, 10000)
} }
producer.multiSend(produceList.toArray) producer.multiSend(produceList.toArray)
@ -141,23 +147,26 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness
// wait a bit for produced message to be available // wait a bit for produced message to be available
Thread.sleep(200) Thread.sleep(200)
val response = consumer.multifetch(fetches: _*) val request = builder.build()
for((topic, resp) <- topics.zip(response.toList)) val response = consumer.fetch(request)
TestUtils.checkEquals(messages(topic).iterator, resp.iterator) for(topic <- topics) {
val fetched = response.messageSet(topic, 0)
TestUtils.checkEquals(messages(topic).iterator, fetched.iterator)
}
} }
def testMultiProduceResend() { def testMultiProduceResend() {
// send some messages // send some messages
val topics = List("test1", "test2", "test3"); val topics = List("test1", "test2", "test3");
val messages = new mutable.HashMap[String, ByteBufferMessageSet] val messages = new mutable.HashMap[String, ByteBufferMessageSet]
val fetches = new mutable.ArrayBuffer[FetchRequest] val builder = new FetchRequestBuilder()
var produceList: List[ProducerRequest] = Nil var produceList: List[ProducerRequest] = Nil
for(topic <- topics) { for(topic <- topics) {
val set = new ByteBufferMessageSet(NoCompressionCodec, val set = new ByteBufferMessageSet(NoCompressionCodec,
new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
messages += topic -> set messages += topic -> set
produceList ::= new ProducerRequest(topic, 0, set) produceList ::= new ProducerRequest(topic, 0, set)
fetches += new FetchRequest(topic, 0, 0, 10000) builder.addFetch(topic, 0, 0, 10000)
} }
producer.multiSend(produceList.toArray) producer.multiSend(produceList.toArray)
@ -169,11 +178,13 @@ class LazyInitProducerTest extends JUnit3Suite with ProducerConsumerTestHarness
// wait a bit for produced message to be available // wait a bit for produced message to be available
Thread.sleep(750) Thread.sleep(750)
val response = consumer.multifetch(fetches: _*) val request = builder.build()
for((topic, resp) <- topics.zip(response.toList)) val response = consumer.fetch(request)
for(topic <- topics) {
val topicMessages = response.messageSet(topic, 0)
TestUtils.checkEquals(TestUtils.stackedIterator(messages(topic).map(m => m.message).iterator, TestUtils.checkEquals(TestUtils.stackedIterator(messages(topic).map(m => m.message).iterator,
messages(topic).map(m => m.message).iterator), messages(topic).map(m => m.message).iterator),
resp.map(m => m.message).iterator) topicMessages.iterator.map(_.message))
// TestUtils.checkEquals(TestUtils.stackedIterator(messages(topic).iterator, messages(topic).iterator), resp.iterator) }
} }
} }

View File

@ -20,16 +20,14 @@ package kafka.log
import kafka.server.KafkaConfig import kafka.server.KafkaConfig
import java.io.File import java.io.File
import java.nio.ByteBuffer import java.nio.ByteBuffer
import kafka.utils.Utils import kafka.api.FetchRequestBuilder
import kafka.api.FetchRequest
import kafka.common.InvalidMessageSizeException import kafka.common.InvalidMessageSizeException
import kafka.utils.TestUtils
import kafka.consumer.{ZookeeperConsumerConnector, ConsumerConfig} import kafka.consumer.{ZookeeperConsumerConnector, ConsumerConfig}
import org.scalatest.junit.JUnit3Suite import kafka.integration.{KafkaServerTestHarness, ProducerConsumerTestHarness}
import kafka.integration.ProducerConsumerTestHarness
import kafka.integration.KafkaServerTestHarness
import org.apache.log4j.{Logger, Level}
import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet} import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet}
import kafka.utils.{Utils, TestUtils}
import org.scalatest.junit.JUnit3Suite
import org.apache.log4j.{Logger, Level}
class LogCorruptionTest extends JUnit3Suite with ProducerConsumerTestHarness with KafkaServerTestHarness { class LogCorruptionTest extends JUnit3Suite with ProducerConsumerTestHarness with KafkaServerTestHarness {
val port = TestUtils.choosePort val port = TestUtils.choosePort
@ -65,23 +63,21 @@ class LogCorruptionTest extends JUnit3Suite with ProducerConsumerTestHarness wit
Thread.sleep(500) Thread.sleep(500)
// test SimpleConsumer // test SimpleConsumer
val messageSet = consumer.fetch(new FetchRequest(topic, partition, 0, 10000)) val response = consumer.fetch(new FetchRequestBuilder().addFetch(topic, partition, 0, 10000).build())
try { try {
for (msg <- messageSet) 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.")
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 {
catch {
case e: InvalidMessageSizeException => "This is good" case e: InvalidMessageSizeException => "This is good"
} }
val messageSet2 = consumer.fetch(new FetchRequest(topic, partition, 0, 10000)) val response2 = consumer.fetch(new FetchRequestBuilder().addFetch(topic, partition, 0, 10000).build())
try { try {
for (msg <- messageSet2) 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.")
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 {
catch {
case e: InvalidMessageSizeException => println("This is good") case e: InvalidMessageSizeException => println("This is good")
} }
@ -95,8 +91,7 @@ class LogCorruptionTest extends JUnit3Suite with ProducerConsumerTestHarness wit
for (message <- messageStreams(0)) 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.")
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 {
catch {
case e: InvalidMessageSizeException => "This is good" case e: InvalidMessageSizeException => "This is good"
case e: Exception => "This is not bad too !" case e: Exception => "This is not bad too !"
} }

View File

@ -18,18 +18,19 @@
package kafka.integration package kafka.integration
import scala.collection._ import scala.collection._
import junit.framework.Assert._ import java.io.File
import kafka.api.{ProducerRequest, FetchRequest}
import kafka.common.{OffsetOutOfRangeException, InvalidPartitionException}
import kafka.server.{KafkaRequestHandler, KafkaConfig}
import org.apache.log4j.{Level, Logger}
import org.scalatest.junit.JUnit3Suite
import java.util.Properties import java.util.Properties
import junit.framework.Assert._
import kafka.common.{ErrorMapping, OffsetOutOfRangeException, InvalidPartitionException}
import kafka.message.{DefaultCompressionCodec, NoCompressionCodec, Message, ByteBufferMessageSet}
import kafka.producer.{ProducerData, Producer, ProducerConfig} import kafka.producer.{ProducerData, Producer, ProducerConfig}
import kafka.serializer.StringDecoder import kafka.serializer.StringDecoder
import kafka.message.{DefaultCompressionCodec, NoCompressionCodec, Message, ByteBufferMessageSet} import kafka.server.{KafkaRequestHandler, KafkaConfig}
import java.io.File
import kafka.utils.TestUtils import kafka.utils.TestUtils
import org.apache.log4j.{Level, Logger}
import org.scalatest.junit.JUnit3Suite
import java.nio.ByteBuffer
import kafka.api.{FetchRequest, FetchRequestBuilder, ProducerRequest}
/** /**
* End to end tests of the primitive apis against a local server * End to end tests of the primitive apis against a local server
@ -44,6 +45,23 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
val configs = List(config) val configs = List(config)
val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler])
def testFetchRequestCanProperlySerialize() {
val request = new FetchRequestBuilder()
.correlationId(100)
.clientId("test-client")
.maxWait(10001)
.minBytes(4444)
.addFetch("topic1", 0, 0, 10000)
.addFetch("topic2", 1, 1024, 9999)
.addFetch("topic1", 1, 256, 444)
.build()
val serializedBuffer = ByteBuffer.allocate(request.sizeInBytes)
request.writeTo(serializedBuffer)
serializedBuffer.rewind()
val deserializedRequest = FetchRequest.readFrom(serializedBuffer)
assertEquals(request, deserializedRequest)
}
def testDefaultEncoderProducerAndFetch() { def testDefaultEncoderProducerAndFetch() {
val topic = "test-topic" val topic = "test-topic"
val props = new Properties() val props = new Properties()
@ -55,10 +73,18 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
stringProducer1.send(new ProducerData[String, String](topic, Array("test-message"))) stringProducer1.send(new ProducerData[String, String](topic, Array("test-message")))
Thread.sleep(200) Thread.sleep(200)
var fetched = consumer.fetch(new FetchRequest(topic, 0, 0, 10000)) val request = new FetchRequestBuilder()
assertTrue(fetched.iterator.hasNext) .correlationId(100)
.clientId("test-client")
.addFetch(topic, 0, 0, 10000)
.build()
val fetched = consumer.fetch(request)
assertEquals("Returned correlationId doesn't match that in request.", 100, fetched.correlationId)
val fetchedMessageAndOffset = fetched.iterator.next val messageSet = fetched.messageSet(topic, 0)
assertTrue(messageSet.iterator.hasNext)
val fetchedMessageAndOffset = messageSet.head
val stringDecoder = new StringDecoder val stringDecoder = new StringDecoder
val fetchedStringMessage = stringDecoder.toEvent(fetchedMessageAndOffset.message) val fetchedStringMessage = stringDecoder.toEvent(fetchedMessageAndOffset.message)
assertEquals("test-message", fetchedStringMessage) assertEquals("test-message", fetchedStringMessage)
@ -76,10 +102,11 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
stringProducer1.send(new ProducerData[String, String](topic, Array("test-message"))) stringProducer1.send(new ProducerData[String, String](topic, Array("test-message")))
Thread.sleep(200) Thread.sleep(200)
var fetched = consumer.fetch(new FetchRequest(topic, 0, 0, 10000)) var fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build())
assertTrue(fetched.iterator.hasNext) val messageSet = fetched.messageSet(topic, 0)
assertTrue(messageSet.iterator.hasNext)
val fetchedMessageAndOffset = fetched.iterator.next val fetchedMessageAndOffset = messageSet.head
val stringDecoder = new StringDecoder val stringDecoder = new StringDecoder
val fetchedStringMessage = stringDecoder.toEvent(fetchedMessageAndOffset.message) val fetchedStringMessage = stringDecoder.toEvent(fetchedMessageAndOffset.message)
assertEquals("test-message", fetchedStringMessage) assertEquals("test-message", fetchedStringMessage)
@ -87,24 +114,27 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
def testProduceAndMultiFetch() { def testProduceAndMultiFetch() {
// send some messages // send some messages
val topics = List("test1", "test2", "test3"); val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
{ {
val messages = new mutable.HashMap[String, ByteBufferMessageSet] val messages = new mutable.HashMap[String, ByteBufferMessageSet]
val fetches = new mutable.ArrayBuffer[FetchRequest] val builder = new FetchRequestBuilder()
for(topic <- topics) { for( (topic, partition) <- topics) {
val set = new ByteBufferMessageSet(NoCompressionCodec, val set = new ByteBufferMessageSet(NoCompressionCodec,
new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
messages += topic -> set messages += topic -> set
producer.send(topic, set) producer.send(topic, set)
set.getBuffer.rewind set.getBuffer.rewind
fetches += new FetchRequest(topic, 0, 0, 10000) builder.addFetch(topic, partition, 0, 10000)
} }
// wait a bit for produced message to be available // wait a bit for produced message to be available
Thread.sleep(700) Thread.sleep(700)
val response = consumer.multifetch(fetches: _*) val request = builder.build()
for((topic, resp) <- topics.zip(response.toList)) val response = consumer.fetch(request)
TestUtils.checkEquals(messages(topic).iterator, resp.iterator) for( (topic, partition) <- topics) {
val fetched = response.messageSet(topic, partition)
TestUtils.checkEquals(messages(topic).iterator, fetched.iterator)
}
} }
// temporarily set request handler logger to a higher level // temporarily set request handler logger to a higher level
@ -112,34 +142,34 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
{ {
// send some invalid offsets // send some invalid offsets
val fetches = new mutable.ArrayBuffer[FetchRequest] val builder = new FetchRequestBuilder()
for(topic <- topics) for( (topic, partition) <- topics)
fetches += new FetchRequest(topic, 0, -1, 10000) builder.addFetch(topic, partition, -1, 10000)
try { try {
val responses = consumer.multifetch(fetches: _*) val request = builder.build()
for(resp <- responses) val response = consumer.fetch(request)
resp.iterator for( (topic, partition) <- topics)
fail("expect exception") response.messageSet(topic, partition).iterator
} fail("Expected exception when fetching message with invalid offset")
catch { } catch {
case e: OffsetOutOfRangeException => "this is good" case e: OffsetOutOfRangeException => "this is good"
} }
} }
{ {
// send some invalid partitions // send some invalid partitions
val fetches = new mutable.ArrayBuffer[FetchRequest] val builder = new FetchRequestBuilder()
for(topic <- topics) for( (topic, partition) <- topics)
fetches += new FetchRequest(topic, -1, 0, 10000) builder.addFetch(topic, -1, 0, 10000)
try { try {
val responses = consumer.multifetch(fetches: _*) val request = builder.build()
for(resp <- responses) val response = consumer.fetch(request)
resp.iterator for( (topic, partition) <- topics)
fail("expect exception") response.messageSet(topic, -1).iterator
} fail("Expected exception when fetching message with invalid partition")
catch { } catch {
case e: InvalidPartitionException => "this is good" case e: InvalidPartitionException => "this is good"
} }
} }
@ -150,24 +180,27 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
def testProduceAndMultiFetchWithCompression() { def testProduceAndMultiFetchWithCompression() {
// send some messages // send some messages
val topics = List("test1", "test2", "test3"); val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
{ {
val messages = new mutable.HashMap[String, ByteBufferMessageSet] val messages = new mutable.HashMap[String, ByteBufferMessageSet]
val fetches = new mutable.ArrayBuffer[FetchRequest] val builder = new FetchRequestBuilder()
for(topic <- topics) { for( (topic, partition) <- topics) {
val set = new ByteBufferMessageSet(DefaultCompressionCodec, val set = new ByteBufferMessageSet(DefaultCompressionCodec,
new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
messages += topic -> set messages += topic -> set
producer.send(topic, set) producer.send(topic, set)
set.getBuffer.rewind set.getBuffer.rewind
fetches += new FetchRequest(topic, 0, 0, 10000) builder.addFetch(topic, partition, 0, 10000)
} }
// wait a bit for produced message to be available // wait a bit for produced message to be available
Thread.sleep(200) Thread.sleep(200)
val response = consumer.multifetch(fetches: _*) val request = builder.build()
for((topic, resp) <- topics.zip(response.toList)) val response = consumer.fetch(request)
TestUtils.checkEquals(messages(topic).iterator, resp.iterator) for( (topic, partition) <- topics) {
val fetched = response.messageSet(topic, partition)
TestUtils.checkEquals(messages(topic).iterator, fetched.iterator)
}
} }
// temporarily set request handler logger to a higher level // temporarily set request handler logger to a higher level
@ -175,34 +208,34 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
{ {
// send some invalid offsets // send some invalid offsets
val fetches = new mutable.ArrayBuffer[FetchRequest] val builder = new FetchRequestBuilder()
for(topic <- topics) for( (topic, partition) <- topics)
fetches += new FetchRequest(topic, 0, -1, 10000) builder.addFetch(topic, partition, -1, 10000)
try { try {
val responses = consumer.multifetch(fetches: _*) val request = builder.build()
for(resp <- responses) val response = consumer.fetch(request)
resp.iterator for( (topic, partition) <- topics)
fail("expect exception") response.messageSet(topic, partition).iterator
} fail("Expected exception when fetching message with invalid offset")
catch { } catch {
case e: OffsetOutOfRangeException => "this is good" case e: OffsetOutOfRangeException => "this is good"
} }
} }
{ {
// send some invalid partitions // send some invalid partitions
val fetches = new mutable.ArrayBuffer[FetchRequest] val builder = new FetchRequestBuilder()
for(topic <- topics) for( (topic, _) <- topics)
fetches += new FetchRequest(topic, -1, 0, 10000) builder.addFetch(topic, -1, 0, 10000)
try { try {
val responses = consumer.multifetch(fetches: _*) val request = builder.build()
for(resp <- responses) val response = consumer.fetch(request)
resp.iterator for( (topic, _) <- topics)
fail("expect exception") response.messageSet(topic, -1).iterator
} fail("Expected exception when fetching message with invalid partition")
catch { } catch {
case e: InvalidPartitionException => "this is good" case e: InvalidPartitionException => "this is good"
} }
} }
@ -213,16 +246,16 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
def testMultiProduce() { def testMultiProduce() {
// send some messages // send some messages
val topics = List("test1", "test2", "test3"); val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
val messages = new mutable.HashMap[String, ByteBufferMessageSet] val messages = new mutable.HashMap[String, ByteBufferMessageSet]
val fetches = new mutable.ArrayBuffer[FetchRequest] val builder = new FetchRequestBuilder()
var produceList: List[ProducerRequest] = Nil var produceList: List[ProducerRequest] = Nil
for(topic <- topics) { for( (topic, partition) <- topics) {
val set = new ByteBufferMessageSet(NoCompressionCodec, val set = new ByteBufferMessageSet(NoCompressionCodec,
new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
messages += topic -> set messages += topic -> set
produceList ::= new ProducerRequest(topic, 0, set) produceList ::= new ProducerRequest(topic, 0, set)
fetches += new FetchRequest(topic, 0, 0, 10000) builder.addFetch(topic, partition, 0, 10000)
} }
producer.multiSend(produceList.toArray) producer.multiSend(produceList.toArray)
@ -231,23 +264,26 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
// wait a bit for produced message to be available // wait a bit for produced message to be available
Thread.sleep(200) Thread.sleep(200)
val response = consumer.multifetch(fetches: _*) val request = builder.build()
for((topic, resp) <- topics.zip(response.toList)) val response = consumer.fetch(request)
TestUtils.checkEquals(messages(topic).iterator, resp.iterator) for( (topic, partition) <- topics) {
val fetched = response.messageSet(topic, partition)
TestUtils.checkEquals(messages(topic).iterator, fetched.iterator)
}
} }
def testMultiProduceWithCompression() { def testMultiProduceWithCompression() {
// send some messages // send some messages
val topics = List("test1", "test2", "test3"); val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
val messages = new mutable.HashMap[String, ByteBufferMessageSet] val messages = new mutable.HashMap[String, ByteBufferMessageSet]
val fetches = new mutable.ArrayBuffer[FetchRequest] val builder = new FetchRequestBuilder()
var produceList: List[ProducerRequest] = Nil var produceList: List[ProducerRequest] = Nil
for(topic <- topics) { for( (topic, partition) <- topics) {
val set = new ByteBufferMessageSet(DefaultCompressionCodec, val set = new ByteBufferMessageSet(DefaultCompressionCodec,
new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes)) new Message(("a_" + topic).getBytes), new Message(("b_" + topic).getBytes))
messages += topic -> set messages += topic -> set
produceList ::= new ProducerRequest(topic, 0, set) produceList ::= new ProducerRequest(topic, 0, set)
fetches += new FetchRequest(topic, 0, 0, 10000) builder.addFetch(topic, partition, 0, 10000)
} }
producer.multiSend(produceList.toArray) producer.multiSend(produceList.toArray)
@ -256,15 +292,18 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
// wait a bit for produced message to be available // wait a bit for produced message to be available
Thread.sleep(200) Thread.sleep(200)
val response = consumer.multifetch(fetches: _*) val request = builder.build()
for((topic, resp) <- topics.zip(response.toList)) val response = consumer.fetch(request)
TestUtils.checkEquals(messages(topic).iterator, resp.iterator) for( (topic, partition) <- topics) {
val fetched = response.messageSet(topic, 0)
TestUtils.checkEquals(messages(topic).iterator, fetched.iterator)
}
} }
def testConsumerNotExistTopic() { def testConsumerNotExistTopic() {
val newTopic = "new-topic" val newTopic = "new-topic"
val messageSetIter = consumer.fetch(new FetchRequest(newTopic, 0, 0, 10000)).iterator val fetchResponse = consumer.fetch(new FetchRequestBuilder().addFetch(newTopic, 0, 0, 10000).build())
assertTrue(messageSetIter.hasNext == false) assertFalse(fetchResponse.messageSet(newTopic, 0).iterator.hasNext)
val logFile = new File(config.logDir, newTopic + "-0") val logFile = new File(config.logDir, newTopic + "-0")
assertTrue(!logFile.exists) assertTrue(!logFile.exists)
} }

View File

@ -18,15 +18,15 @@
package kafka.javaapi.integration package kafka.javaapi.integration
import scala.collection._ import scala.collection._
import kafka.api.FetchRequest import kafka.api.FetchRequestBuilder
import kafka.common.{InvalidPartitionException, OffsetOutOfRangeException} import kafka.common.{InvalidPartitionException, OffsetOutOfRangeException}
import kafka.javaapi.ProducerRequest
import kafka.javaapi.message.ByteBufferMessageSet
import kafka.message.{DefaultCompressionCodec, NoCompressionCodec, Message}
import kafka.server.{KafkaRequestHandler, KafkaConfig} import kafka.server.{KafkaRequestHandler, KafkaConfig}
import kafka.utils.TestUtils
import org.apache.log4j.{Level, Logger} import org.apache.log4j.{Level, Logger}
import org.scalatest.junit.JUnit3Suite import org.scalatest.junit.JUnit3Suite
import kafka.javaapi.message.ByteBufferMessageSet
import kafka.javaapi.ProducerRequest
import kafka.message.{DefaultCompressionCodec, NoCompressionCodec, Message}
import kafka.utils.TestUtils
/** /**
* End to end tests of the primitive apis against a local server * End to end tests of the primitive apis against a local server
@ -44,38 +44,41 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
val topic = "test" val topic = "test"
// send an empty messageset first // send an empty messageset first
val sent2 = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, val sent2 = new ByteBufferMessageSet(NoCompressionCodec, getMessageList(Seq.empty[Message]: _*))
messages = getMessageList(Seq.empty[Message]: _*))
producer.send(topic, sent2) producer.send(topic, sent2)
Thread.sleep(200) Thread.sleep(200)
sent2.getBuffer.rewind sent2.getBuffer.rewind
var fetched2 = consumer.fetch(new FetchRequest(topic, 0, 0, 10000)) val fetched2 = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build())
TestUtils.checkEquals(sent2.iterator, fetched2.iterator) val fetchedMessage2 = fetched2.messageSet(topic, 0)
TestUtils.checkEquals(sent2.iterator, fetchedMessage2.iterator)
// send some messages // send some messages
val sent3 = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, val sent3 = new ByteBufferMessageSet(NoCompressionCodec,
messages = getMessageList(new Message("hello".getBytes()), getMessageList(
new Message("there".getBytes()))) new Message("hello".getBytes()),new Message("there".getBytes())))
producer.send(topic, sent3) producer.send(topic, sent3)
Thread.sleep(200) Thread.sleep(200)
sent3.getBuffer.rewind sent3.getBuffer.rewind
var fetched3: ByteBufferMessageSet = null var messageSet: ByteBufferMessageSet = null
while(fetched3 == null || fetched3.validBytes == 0) while(messageSet == null || messageSet.validBytes == 0) {
fetched3 = consumer.fetch(new FetchRequest(topic, 0, 0, 10000)) val fetched3 = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build())
TestUtils.checkEquals(sent3.iterator, fetched3.iterator) messageSet = fetched3.messageSet(topic, 0).asInstanceOf[ByteBufferMessageSet]
}
TestUtils.checkEquals(sent3.iterator, messageSet.iterator)
// temporarily set request handler logger to a higher level // temporarily set request handler logger to a higher level
requestHandlerLogger.setLevel(Level.FATAL) requestHandlerLogger.setLevel(Level.FATAL)
// send an invalid offset // send an invalid offset
try { try {
val fetchedWithError = consumer.fetch(new FetchRequest(topic, 0, -1, 10000)) val fetchedWithError = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, -1, 10000).build())
fetchedWithError.iterator val messageWithError = fetchedWithError.messageSet(topic, 0)
fail("expect exception") messageWithError.iterator
} fail("Fetch with invalid offset should throw an exception when iterating over response")
catch { } catch {
case e: OffsetOutOfRangeException => "this is good" case e: OffsetOutOfRangeException => "this is good"
} }
@ -88,38 +91,41 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
val topic = "test" val topic = "test"
// send an empty messageset first // send an empty messageset first
val sent2 = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec, val sent2 = new ByteBufferMessageSet(DefaultCompressionCodec, getMessageList(Seq.empty[Message]: _*))
messages = getMessageList(Seq.empty[Message]: _*))
producer.send(topic, sent2) producer.send(topic, sent2)
Thread.sleep(200) Thread.sleep(200)
sent2.getBuffer.rewind sent2.getBuffer.rewind
var fetched2 = consumer.fetch(new FetchRequest(topic, 0, 0, 10000)) val fetched2 = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build())
TestUtils.checkEquals(sent2.iterator, fetched2.iterator) val message2 = fetched2.messageSet(topic, 0)
TestUtils.checkEquals(sent2.iterator, message2.iterator)
// send some messages // send some messages
val sent3 = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec, val sent3 = new ByteBufferMessageSet( DefaultCompressionCodec,
messages = getMessageList(new Message("hello".getBytes()), getMessageList(
new Message("there".getBytes()))) new Message("hello".getBytes()),new Message("there".getBytes())))
producer.send(topic, sent3) producer.send(topic, sent3)
Thread.sleep(200) Thread.sleep(200)
sent3.getBuffer.rewind sent3.getBuffer.rewind
var fetched3: ByteBufferMessageSet = null var fetchedMessage: ByteBufferMessageSet = null
while(fetched3 == null || fetched3.validBytes == 0) while(fetchedMessage == null || fetchedMessage.validBytes == 0) {
fetched3 = consumer.fetch(new FetchRequest(topic, 0, 0, 10000)) val fetched3 = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build())
TestUtils.checkEquals(sent3.iterator, fetched3.iterator) fetchedMessage = fetched3.messageSet(topic, 0).asInstanceOf[ByteBufferMessageSet]
}
TestUtils.checkEquals(sent3.iterator, fetchedMessage.iterator)
// temporarily set request handler logger to a higher level // temporarily set request handler logger to a higher level
requestHandlerLogger.setLevel(Level.FATAL) requestHandlerLogger.setLevel(Level.FATAL)
// send an invalid offset // send an invalid offset
try { try {
val fetchedWithError = consumer.fetch(new FetchRequest(topic, 0, -1, 10000)) val fetchedWithError = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, -1, 10000).build())
fetchedWithError.iterator val messageWithError = fetchedWithError.messageSet(topic, 0)
fail("expect exception") messageWithError.iterator
} fail("Fetch with invalid offset should throw an exception when iterating over response")
catch { } catch {
case e: OffsetOutOfRangeException => "this is good" case e: OffsetOutOfRangeException => "this is good"
} }
@ -129,31 +135,27 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
def testProduceAndMultiFetch() { def testProduceAndMultiFetch() {
// send some messages // send some messages
val topics = List("test1", "test2", "test3"); val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
{ {
val messages = new mutable.HashMap[String, ByteBufferMessageSet] val messages = new mutable.HashMap[String, ByteBufferMessageSet]
val fetches = new mutable.ArrayBuffer[FetchRequest] val builder = new FetchRequestBuilder()
for(topic <- topics) { for( (topic, partition) <- topics) {
val set = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, val set = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec,
messages = getMessageList(new Message(("a_" + topic).getBytes), messages = getMessageList(new Message(("a_" + topic).getBytes),
new Message(("b_" + topic).getBytes))) new Message(("b_" + topic).getBytes)))
messages += topic -> set messages += topic -> set
producer.send(topic, set) producer.send(topic, set)
set.getBuffer.rewind set.getBuffer.rewind
fetches += new FetchRequest(topic, 0, 0, 10000) builder.addFetch(topic, partition, 0, 10000)
} }
// wait a bit for produced message to be available // wait a bit for produced message to be available
Thread.sleep(200) Thread.sleep(200)
val response = consumer.multifetch(getFetchRequestList(fetches: _*)) val request = builder.build()
val iter = response.iterator val response = consumer.fetch(request)
for(topic <- topics) { for( (topic, partition) <- topics) {
if (iter.hasNext) { val messageSet = response.messageSet(topic, partition)
val resp = iter.next TestUtils.checkEquals(messages(topic).iterator, messageSet.iterator)
TestUtils.checkEquals(messages(topic).iterator, resp.iterator)
}
else
fail("fewer responses than expected")
} }
} }
@ -162,39 +164,43 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
{ {
// send some invalid offsets // send some invalid offsets
val fetches = new mutable.ArrayBuffer[FetchRequest] val builder = new FetchRequestBuilder()
for(topic <- topics) for( (topic, partition) <- topics)
fetches += new FetchRequest(topic, 0, -1, 10000) builder.addFetch(topic, partition, -1, 10000)
val request = builder.build()
val response = consumer.fetch(request)
for( (topic, partition) <- topics) {
try { try {
val responses = consumer.multifetch(getFetchRequestList(fetches: _*)) val iter = response.messageSet(topic, partition).iterator
val iter = responses.iterator
while (iter.hasNext) while (iter.hasNext)
iter.next.iterator iter.next
fail("expect exception") fail("MessageSet for invalid offset should throw exception")
} } catch {
catch {
case e: OffsetOutOfRangeException => "this is good" case e: OffsetOutOfRangeException => "this is good"
} }
} }
}
{ {
// send some invalid partitions // send some invalid partitions
val fetches = new mutable.ArrayBuffer[FetchRequest] val builder = new FetchRequestBuilder()
for(topic <- topics) for( (topic, _) <- topics)
fetches += new FetchRequest(topic, -1, 0, 10000) builder.addFetch(topic, -1, 0, 10000)
val request = builder.build()
val response = consumer.fetch(request)
for( (topic, _) <- topics) {
try { try {
val responses = consumer.multifetch(getFetchRequestList(fetches: _*)) val iter = response.messageSet(topic, -1).iterator
val iter = responses.iterator
while (iter.hasNext) while (iter.hasNext)
iter.next.iterator iter.next
fail("expect exception") fail("MessageSet for invalid partition should throw exception")
} } catch {
catch {
case e: InvalidPartitionException => "this is good" case e: InvalidPartitionException => "this is good"
} }
} }
}
// restore set request handler logger to a higher level // restore set request handler logger to a higher level
requestHandlerLogger.setLevel(Level.ERROR) requestHandlerLogger.setLevel(Level.ERROR)
@ -202,72 +208,76 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
def testProduceAndMultiFetchWithCompression() { def testProduceAndMultiFetchWithCompression() {
// send some messages // send some messages
val topics = List("test1", "test2", "test3"); val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
{ {
val messages = new mutable.HashMap[String, ByteBufferMessageSet] val messages = new mutable.HashMap[String, ByteBufferMessageSet]
val fetches = new mutable.ArrayBuffer[FetchRequest] val builder = new FetchRequestBuilder()
for(topic <- topics) { for( (topic, partition) <- topics) {
val set = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec, val set = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec,
messages = getMessageList(new Message(("a_" + topic).getBytes), messages = getMessageList(new Message(("a_" + topic).getBytes),
new Message(("b_" + topic).getBytes))) new Message(("b_" + topic).getBytes)))
messages += topic -> set messages += topic -> set
producer.send(topic, set) producer.send(topic, set)
set.getBuffer.rewind set.getBuffer.rewind
fetches += new FetchRequest(topic, 0, 0, 10000) builder.addFetch(topic, partition, 0, 10000)
} }
// wait a bit for produced message to be available // wait a bit for produced message to be available
Thread.sleep(200) Thread.sleep(200)
val response = consumer.multifetch(getFetchRequestList(fetches: _*)) val request = builder.build()
val iter = response.iterator val response = consumer.fetch(request)
for(topic <- topics) { for( (topic, partition) <- topics) {
val iter = response.messageSet(topic, partition).iterator
if (iter.hasNext) { if (iter.hasNext) {
val resp = iter.next TestUtils.checkEquals(messages(topic).iterator, iter)
TestUtils.checkEquals(messages(topic).iterator, resp.iterator) } else {
}
else
fail("fewer responses than expected") fail("fewer responses than expected")
} }
} }
}
// temporarily set request handler logger to a higher level // temporarily set request handler logger to a higher level
requestHandlerLogger.setLevel(Level.FATAL) requestHandlerLogger.setLevel(Level.FATAL)
{ {
// send some invalid offsets // send some invalid offsets
val fetches = new mutable.ArrayBuffer[FetchRequest] val builder = new FetchRequestBuilder()
for(topic <- topics) for( (topic, partition) <- topics)
fetches += new FetchRequest(topic, 0, -1, 10000) builder.addFetch(topic, partition, -1, 10000)
val request = builder.build()
val response = consumer.fetch(request)
for( (topic, partition) <- topics) {
try { try {
val responses = consumer.multifetch(getFetchRequestList(fetches: _*)) val iter = response.messageSet(topic, partition).iterator
val iter = responses.iterator
while (iter.hasNext) while (iter.hasNext)
iter.next.iterator iter.next
fail("expect exception") fail("Expected exception when fetching invalid offset")
} } catch {
catch {
case e: OffsetOutOfRangeException => "this is good" case e: OffsetOutOfRangeException => "this is good"
} }
} }
}
{ {
// send some invalid partitions // send some invalid partitions
val fetches = new mutable.ArrayBuffer[FetchRequest] val builder = new FetchRequestBuilder()
for(topic <- topics) for( (topic, _) <- topics)
fetches += new FetchRequest(topic, -1, 0, 10000) builder.addFetch(topic, -1, 0, 10000)
val request = builder.build()
val response = consumer.fetch(request)
for( (topic, _) <- topics) {
try { try {
val responses = consumer.multifetch(getFetchRequestList(fetches: _*)) val iter = response.messageSet(topic, -1).iterator
val iter = responses.iterator
while (iter.hasNext) while (iter.hasNext)
iter.next.iterator iter.next
fail("expect exception") fail("Expected exception when fetching invalid partition")
} } catch {
catch {
case e: InvalidPartitionException => "this is good" case e: InvalidPartitionException => "this is good"
} }
} }
}
// restore set request handler logger to a higher level // restore set request handler logger to a higher level
requestHandlerLogger.setLevel(Level.ERROR) requestHandlerLogger.setLevel(Level.ERROR)
@ -275,79 +285,75 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
def testProduceAndMultiFetchJava() { def testProduceAndMultiFetchJava() {
// send some messages // send some messages
val topics = List("test1", "test2", "test3"); val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
{ {
val messages = new mutable.HashMap[String, ByteBufferMessageSet] val messages = new mutable.HashMap[String, ByteBufferMessageSet]
val fetches : java.util.ArrayList[FetchRequest] = new java.util.ArrayList[FetchRequest] val builder = new FetchRequestBuilder()
for(topic <- topics) { for( (topic, partition) <- topics) {
val set = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, val set = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec,
messages = getMessageList(new Message(("a_" + topic).getBytes), messages = getMessageList(new Message(("a_" + topic).getBytes),
new Message(("b_" + topic).getBytes))) new Message(("b_" + topic).getBytes)))
messages += topic -> set messages += topic -> set
producer.send(topic, set) producer.send(topic, set)
set.getBuffer.rewind set.getBuffer.rewind
fetches.add(new FetchRequest(topic, 0, 0, 10000)) builder.addFetch(topic, partition, 0, 10000)
} }
// wait a bit for produced message to be available // wait a bit for produced message to be available
Thread.sleep(200) Thread.sleep(200)
val response = consumer.multifetch(fetches) val request = builder.build()
val iter = response.iterator val response = consumer.fetch(request)
for(topic <- topics) { for( (topic, partition) <- topics) {
val iter = response.messageSet(topic, partition).iterator
if (iter.hasNext) { if (iter.hasNext) {
val resp = iter.next TestUtils.checkEquals(messages(topic).iterator, iter)
TestUtils.checkEquals(messages(topic).iterator, resp.iterator) } else {
}
else
fail("fewer responses than expected") fail("fewer responses than expected")
} }
} }
} }
}
def testProduceAndMultiFetchJavaWithCompression() { def testProduceAndMultiFetchJavaWithCompression() {
// send some messages // send some messages
val topics = List("test1", "test2", "test3"); val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
{ {
val messages = new mutable.HashMap[String, ByteBufferMessageSet] val messages = new mutable.HashMap[String, ByteBufferMessageSet]
val fetches : java.util.ArrayList[FetchRequest] = new java.util.ArrayList[FetchRequest] val builder = new FetchRequestBuilder()
for(topic <- topics) { for( (topic, partition) <- topics) {
val set = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec, val set = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec,
messages = getMessageList(new Message(("a_" + topic).getBytes), messages = getMessageList(new Message(("a_" + topic).getBytes),
new Message(("b_" + topic).getBytes))) new Message(("b_" + topic).getBytes)))
messages += topic -> set messages += topic -> set
producer.send(topic, set) producer.send(topic, set)
set.getBuffer.rewind set.getBuffer.rewind
fetches.add(new FetchRequest(topic, 0, 0, 10000)) builder.addFetch(topic, partition, 0, 10000)
} }
// wait a bit for produced message to be available // wait a bit for produced message to be available
Thread.sleep(200) Thread.sleep(200)
val response = consumer.multifetch(fetches) val request = builder.build()
val iter = response.iterator val response = consumer.fetch(request)
for(topic <- topics) { for( (topic, partition) <- topics) {
if (iter.hasNext) { val iter = response.messageSet(topic, partition).iterator
val resp = iter.next TestUtils.checkEquals(messages(topic).iterator, iter)
TestUtils.checkEquals(messages(topic).iterator, resp.iterator)
}
else
fail("fewer responses than expected")
} }
} }
} }
def testMultiProduce() { def testMultiProduce() {
// send some messages // send some messages
val topics = List("test1", "test2", "test3"); val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
val messages = new mutable.HashMap[String, ByteBufferMessageSet] val messages = new mutable.HashMap[String, ByteBufferMessageSet]
val fetches = new mutable.ArrayBuffer[FetchRequest] val builder = new FetchRequestBuilder()
var produceList: List[ProducerRequest] = Nil var produceList: List[ProducerRequest] = Nil
for(topic <- topics) { for( (topic, partition) <- topics) {
val set = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, val set = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec,
messages = getMessageList(new Message(("a_" + topic).getBytes), messages = getMessageList(new Message(("a_" + topic).getBytes),
new Message(("b_" + topic).getBytes))) new Message(("b_" + topic).getBytes)))
messages += topic -> set messages += topic -> set
produceList ::= new ProducerRequest(topic, 0, set) produceList ::= new ProducerRequest(topic, 0, set)
fetches += new FetchRequest(topic, 0, 0, 10000) builder.addFetch(topic, partition, 0, 10000)
} }
producer.multiSend(produceList.toArray) producer.multiSend(produceList.toArray)
@ -356,31 +362,31 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
// wait a bit for produced message to be available // wait a bit for produced message to be available
Thread.sleep(200) Thread.sleep(200)
val response = consumer.multifetch(getFetchRequestList(fetches: _*)) val request = builder.build()
val iter = response.iterator val response = consumer.fetch(request)
for(topic <- topics) { for( (topic, partition) <- topics) {
val iter = response.messageSet(topic, partition).iterator
if (iter.hasNext) { if (iter.hasNext) {
val resp = iter.next TestUtils.checkEquals(messages(topic).iterator, iter)
TestUtils.checkEquals(messages(topic).iterator, resp.iterator) } else {
}
else
fail("fewer responses than expected") fail("fewer responses than expected")
} }
} }
}
def testMultiProduceWithCompression() { def testMultiProduceWithCompression() {
// send some messages // send some messages
val topics = List("test1", "test2", "test3"); val topics = List(("test4", 0), ("test1", 0), ("test2", 0), ("test3", 0));
val messages = new mutable.HashMap[String, ByteBufferMessageSet] val messages = new mutable.HashMap[String, ByteBufferMessageSet]
val fetches = new mutable.ArrayBuffer[FetchRequest] val builder = new FetchRequestBuilder()
var produceList: List[ProducerRequest] = Nil var produceList: List[ProducerRequest] = Nil
for(topic <- topics) { for( (topic, partition) <- topics) {
val set = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec, val set = new ByteBufferMessageSet(compressionCodec = DefaultCompressionCodec,
messages = getMessageList(new Message(("a_" + topic).getBytes), messages = getMessageList(new Message(("a_" + topic).getBytes),
new Message(("b_" + topic).getBytes))) new Message(("b_" + topic).getBytes)))
messages += topic -> set messages += topic -> set
produceList ::= new ProducerRequest(topic, 0, set) produceList ::= new ProducerRequest(topic, 0, set)
fetches += new FetchRequest(topic, 0, 0, 10000) builder.addFetch(topic, partition, 0, 10000)
} }
producer.multiSend(produceList.toArray) producer.multiSend(produceList.toArray)
@ -389,27 +395,21 @@ class PrimitiveApiTest extends JUnit3Suite with ProducerConsumerTestHarness with
// wait a bit for produced message to be available // wait a bit for produced message to be available
Thread.sleep(200) Thread.sleep(200)
val response = consumer.multifetch(getFetchRequestList(fetches: _*)) val request = builder.build()
val iter = response.iterator val response = consumer.fetch(request)
for(topic <- topics) { for( (topic, partition) <- topics) {
val iter = response.messageSet(topic, partition).iterator
if (iter.hasNext) { if (iter.hasNext) {
val resp = iter.next TestUtils.checkEquals(messages(topic).iterator, iter)
TestUtils.checkEquals(messages(topic).iterator, resp.iterator) } else {
}
else
fail("fewer responses than expected") fail("fewer responses than expected")
} }
} }
}
private def getMessageList(messages: Message*): java.util.List[Message] = { private def getMessageList(messages: Message*): java.util.List[Message] = {
val messageList = new java.util.ArrayList[Message]() val messageList = new java.util.ArrayList[Message]()
messages.foreach(m => messageList.add(m)) messages.foreach(m => messageList.add(m))
messageList messageList
} }
private def getFetchRequestList(fetches: FetchRequest*): java.util.List[FetchRequest] = {
val fetchReqs = new java.util.ArrayList[FetchRequest]()
fetches.foreach(f => fetchReqs.add(f))
fetchReqs
}
} }

View File

@ -22,7 +22,6 @@ import kafka.utils._
import kafka.server.{KafkaConfig, KafkaServer} import kafka.server.{KafkaConfig, KafkaServer}
import junit.framework.Assert._ import junit.framework.Assert._
import java.util.{Random, Properties} import java.util.{Random, Properties}
import kafka.api.{FetchRequest, OffsetRequest}
import collection.mutable.WrappedArray import collection.mutable.WrappedArray
import kafka.consumer.SimpleConsumer import kafka.consumer.SimpleConsumer
import org.junit.{After, Before, Test} import org.junit.{After, Before, Test}
@ -30,6 +29,7 @@ import kafka.message.{NoCompressionCodec, ByteBufferMessageSet, Message}
import org.apache.log4j._ import org.apache.log4j._
import kafka.zk.ZooKeeperTestHarness import kafka.zk.ZooKeeperTestHarness
import org.scalatest.junit.JUnit3Suite import org.scalatest.junit.JUnit3Suite
import kafka.api.{FetchRequestBuilder, OffsetRequest}
object LogOffsetTest { object LogOffsetTest {
val random = new Random() val random = new Random()
@ -66,9 +66,8 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness {
@Test @Test
def testEmptyLogs() { def testEmptyLogs() {
val messageSet: ByteBufferMessageSet = simpleConsumer.fetch( val fetchResponse = simpleConsumer.fetch(new FetchRequestBuilder().addFetch("test", 0, 0, 300 * 1024).build())
new FetchRequest("test", 0, 0, 300 * 1024)) assertFalse(fetchResponse.messageSet("test", 0).iterator.hasNext)
assertFalse(messageSet.iterator.hasNext)
val name = "test" val name = "test"
val logFile = new File(logDir, name + "-0") val logFile = new File(logDir, name + "-0")
@ -119,9 +118,9 @@ class LogOffsetTest extends JUnit3Suite with ZooKeeperTestHarness {
assertTrue((Array(240L, 216L, 108L, 0L): WrappedArray[Long]) == (consumerOffsets: WrappedArray[Long])) assertTrue((Array(240L, 216L, 108L, 0L): WrappedArray[Long]) == (consumerOffsets: WrappedArray[Long]))
// try to fetch using latest offset // try to fetch using latest offset
val messageSet: ByteBufferMessageSet = simpleConsumer.fetch( val fetchResponse = simpleConsumer.fetch(
new FetchRequest(topic, 0, consumerOffsets.head, 300 * 1024)) new FetchRequestBuilder().addFetch(topic, 0, consumerOffsets.head, 300 * 1024).build())
assertFalse(messageSet.iterator.hasNext) assertFalse(fetchResponse.messageSet(topic, 0).iterator.hasNext)
} }
@Test @Test

View File

@ -17,23 +17,21 @@
package kafka.log4j package kafka.log4j
import org.apache.log4j.spi.LoggingEvent
import org.apache.log4j.{PropertyConfigurator, Logger}
import java.util.Properties import java.util.Properties
import java.io.File import java.io.File
import kafka.consumer.SimpleConsumer
import kafka.server.{KafkaConfig, KafkaServer}
import kafka.utils.TestZKUtils
import kafka.zk.EmbeddedZookeeper
import junit.framework.Assert._ import junit.framework.Assert._
import kafka.api.FetchRequest import kafka.api.FetchRequestBuilder
import kafka.serializer.Encoder import kafka.consumer.SimpleConsumer
import kafka.message.Message import kafka.message.Message
import kafka.producer.async.MissingConfigException import kafka.producer.async.MissingConfigException
import kafka.serializer.Encoder
import kafka.server.{KafkaConfig, KafkaServer}
import kafka.utils.{TestUtils, TestZKUtils, Utils, Logging}
import kafka.zk.{EmbeddedZookeeper, ZooKeeperTestHarness}
import org.apache.log4j.spi.LoggingEvent
import org.apache.log4j.{PropertyConfigurator, Logger}
import org.junit.{After, Before, Test} import org.junit.{After, Before, Test}
import org.scalatest.junit.JUnit3Suite import org.scalatest.junit.JUnit3Suite
import kafka.zk.ZooKeeperTestHarness
import kafka.utils.{TestUtils, Utils, Logging}
class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with Logging { class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with Logging {
@ -172,10 +170,10 @@ class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with
Thread.sleep(2500) Thread.sleep(2500)
var offset = 0L var offset = 0L
val messages = simpleConsumerBl.fetch(new FetchRequest("test-topic", 0, offset, 1024*1024)) val response = simpleConsumerBl.fetch(new FetchRequestBuilder().addFetch("test-topic", 0, offset, 1024*1024).build())
val fetchedMessage = response.messageSet("test-topic", 0)
var count = 0 var count = 0
for(message <- messages) { for(message <- fetchedMessage) {
count = count + 1 count = count + 1
offset += message.offset offset += message.offset
} }
@ -192,14 +190,16 @@ class KafkaLog4jAppenderTest extends JUnit3Suite with ZooKeeperTestHarness with
Thread.sleep(500) Thread.sleep(500)
val messages = simpleConsumerZk.fetch(new FetchRequest("test-topic", 0, 0L, 1024*1024)) val response = simpleConsumerZk.fetch(new FetchRequestBuilder().addFetch("test-topic", 0, 0L, 1024*1024).build())
val fetchMessage = response.messageSet("test-topic", 0)
var count = 0 var count = 0
for(message <- messages) { for(message <- fetchMessage) {
count = count + 1 count = count + 1
} }
val messagesFromOtherBroker = simpleConsumerBl.fetch(new FetchRequest("test-topic", 0, 0L, 1024*1024)) val response2 = simpleConsumerBl.fetch(new FetchRequestBuilder().addFetch("test-topic", 0, 0L, 1024*1024).build())
val messagesFromOtherBroker = response2.messageSet("test-topic", 0)
for(message <- messagesFromOtherBroker) { for(message <- messagesFromOtherBroker) {
count = count + 1 count = count + 1

View File

@ -17,18 +17,18 @@
package kafka.producer package kafka.producer
import org.apache.log4j.{Logger, Level}
import kafka.zk.EmbeddedZookeeper
import org.junit.{After, Before, Test}
import junit.framework.Assert._ import junit.framework.Assert._
import org.scalatest.junit.JUnitSuite import java.util.Properties
import kafka.utils.{TestUtils, TestZKUtils, Utils} import kafka.api.FetchRequestBuilder
import kafka.api.FetchRequest import kafka.consumer.SimpleConsumer
import kafka.message.Message import kafka.message.Message
import kafka.serializer.Encoder import kafka.serializer.Encoder
import kafka.consumer.SimpleConsumer
import java.util.Properties
import kafka.server.{KafkaRequestHandler, KafkaServer, KafkaConfig} import kafka.server.{KafkaRequestHandler, KafkaServer, KafkaConfig}
import kafka.utils.{TestUtils, TestZKUtils, Utils}
import kafka.zk.EmbeddedZookeeper
import org.apache.log4j.{Logger, Level}
import org.junit.{After, Before, Test}
import org.scalatest.junit.JUnitSuite
class ProducerTest extends JUnitSuite { class ProducerTest extends JUnitSuite {
private val topic = "test-topic" private val topic = "test-topic"
@ -106,12 +106,14 @@ class ProducerTest extends JUnitSuite {
producer.send(new ProducerData[String, String]("new-topic", "test", Array("test1"))) producer.send(new ProducerData[String, String]("new-topic", "test", Array("test1")))
Thread.sleep(100) Thread.sleep(100)
// cross check if brokers got the messages // cross check if brokers got the messages
val messageSet1 = consumer1.fetch(new FetchRequest("new-topic", 0, 0, 10000)).iterator val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build())
assertTrue("Message set should have 1 message", messageSet1.hasNext) val messageSet1 = response1.messageSet("new-topic", 0)
assertEquals(new Message("test1".getBytes), messageSet1.next.message) assertTrue("Message set should have 1 message", messageSet1.iterator.hasNext)
val messageSet2 = consumer2.fetch(new FetchRequest("new-topic", 0, 0, 10000)).iterator assertEquals(new Message("test1".getBytes), messageSet1.head.message)
assertTrue("Message set should have 1 message", messageSet2.hasNext) val response2 = consumer2.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build())
assertEquals(new Message("test1".getBytes), messageSet2.next.message) val messageSet2 = response2.messageSet("new-topic", 0)
assertTrue("Message set should have 1 message", messageSet2.iterator.hasNext)
assertEquals(new Message("test1".getBytes), messageSet2.head.message)
} catch { } catch {
case e: Exception => fail("Not expected", e) case e: Exception => fail("Not expected", e)
} }
@ -142,11 +144,12 @@ class ProducerTest extends JUnitSuite {
producer.send(new ProducerData[String, String]("new-topic", "test", Array("test1"))) producer.send(new ProducerData[String, String]("new-topic", "test", Array("test1")))
Thread.sleep(100) Thread.sleep(100)
// cross check if brokers got the messages // cross check if brokers got the messages
val messageSet1 = consumer1.fetch(new FetchRequest("new-topic", 0, 0, 10000)).iterator val response1 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build())
assertTrue("Message set should have 1 message", messageSet1.hasNext) val messageSet1Iter = response1.messageSet("new-topic", 0).iterator
assertEquals(new Message("test1".getBytes), messageSet1.next.message) assertTrue("Message set should have 1 message", messageSet1Iter.hasNext)
assertTrue("Message set should have another message", messageSet1.hasNext) assertEquals(new Message("test1".getBytes), messageSet1Iter.next.message)
assertEquals(new Message("test1".getBytes), messageSet1.next.message) assertTrue("Message set should have another message", messageSet1Iter.hasNext)
assertEquals(new Message("test1".getBytes), messageSet1Iter.next.message)
} catch { } catch {
case e: Exception => fail("Not expected") case e: Exception => fail("Not expected")
} }
@ -174,9 +177,10 @@ class ProducerTest extends JUnitSuite {
producer.send(new ProducerData[String, String]("new-topic", "test", Array("test"))) producer.send(new ProducerData[String, String]("new-topic", "test", Array("test")))
Thread.sleep(100) Thread.sleep(100)
// cross check if brokers got the messages // cross check if brokers got the messages
val messageSet1 = consumer2.fetch(new FetchRequest("new-topic", 0, 0, 10000)).iterator val response1 = consumer2.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build())
assertTrue("Message set should have 1 message", messageSet1.hasNext) val messageSet1 = response1.messageSet("new-topic", 0)
assertEquals(new Message("test".getBytes), messageSet1.next.message) assertTrue("Message set should have 1 message", messageSet1.iterator.hasNext)
assertEquals(new Message("test".getBytes), messageSet1.head.message)
// shutdown server2 // shutdown server2
server2.shutdown server2.shutdown
@ -197,9 +201,10 @@ class ProducerTest extends JUnitSuite {
Thread.sleep(100) Thread.sleep(100)
// cross check if brokers got the messages // cross check if brokers got the messages
val messageSet2 = consumer1.fetch(new FetchRequest("new-topic", 0, 0, 10000)).iterator val response2 = consumer1.fetch(new FetchRequestBuilder().addFetch("new-topic", 0, 0, 10000).build())
assertTrue("Message set should have 1 message", messageSet2.hasNext) val messageSet2 = response2.messageSet("new-topic", 0)
assertEquals(new Message("test".getBytes), messageSet2.next.message) assertTrue("Message set should have 1 message", messageSet2.iterator.hasNext)
assertEquals(new Message("test".getBytes), messageSet2.head.message)
} catch { } catch {
case e: Exception => fail("Not expected", e) case e: Exception => fail("Not expected", e)

View File

@ -17,7 +17,6 @@
package kafka.server package kafka.server
import java.io.File import java.io.File
import kafka.api.FetchRequest
import kafka.producer.{SyncProducer, SyncProducerConfig} import kafka.producer.{SyncProducer, SyncProducerConfig}
import kafka.consumer.SimpleConsumer import kafka.consumer.SimpleConsumer
import java.util.Properties import java.util.Properties
@ -27,6 +26,7 @@ import kafka.message.{NoCompressionCodec, Message, ByteBufferMessageSet}
import org.scalatest.junit.JUnit3Suite import org.scalatest.junit.JUnit3Suite
import kafka.zk.ZooKeeperTestHarness import kafka.zk.ZooKeeperTestHarness
import kafka.utils.{TestUtils, Utils} import kafka.utils.{TestUtils, Utils}
import kafka.api.{FetchResponse, FetchRequestBuilder, FetchRequest}
class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness { class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness {
val port = TestUtils.choosePort val port = TestUtils.choosePort
@ -82,11 +82,13 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness {
server.startup() server.startup()
// bring the server back again and read the messages // bring the server back again and read the messages
var fetched: ByteBufferMessageSet = null var fetchedMessage: ByteBufferMessageSet = null
while(fetched == null || fetched.validBytes == 0) while(fetchedMessage == null || fetchedMessage.validBytes == 0) {
fetched = consumer.fetch(new FetchRequest(topic, 0, 0, 10000)) val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).build())
TestUtils.checkEquals(sent1.iterator, fetched.iterator) fetchedMessage = fetched.messageSet(topic, 0)
val newOffset = fetched.validBytes }
TestUtils.checkEquals(sent1.iterator, fetchedMessage.iterator)
val newOffset = fetchedMessage.validBytes
// send some more messages // send some more messages
producer.send(topic, sent2) producer.send(topic, sent2)
@ -94,10 +96,12 @@ class ServerShutdownTest extends JUnit3Suite with ZooKeeperTestHarness {
Thread.sleep(200) Thread.sleep(200)
fetched = null fetchedMessage = null
while(fetched == null || fetched.validBytes == 0) while(fetchedMessage == null || fetchedMessage.validBytes == 0) {
fetched = consumer.fetch(new FetchRequest(topic, 0, newOffset, 10000)) val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, newOffset, 10000).build())
TestUtils.checkEquals(sent2.map(m => m.message).iterator, fetched.map(m => m.message).iterator) fetchedMessage = fetched.messageSet(topic, 0)
}
TestUtils.checkEquals(sent2.map(m => m.message).iterator, fetchedMessage.map(m => m.message).iterator)
server.shutdown() server.shutdown()
Utils.rm(server.config.logDir) Utils.rm(server.config.logDir)

View File

@ -187,22 +187,20 @@ object TestUtils {
assertEquals(expected.next, actual.next) assertEquals(expected.next, actual.next)
} }
if (expected.hasNext) // check if the expected iterator is longer
{ if (expected.hasNext) {
var length1 = length; var length1 = length;
while (expected.hasNext) while (expected.hasNext) {
{
expected.next expected.next
length1 += 1 length1 += 1
} }
assertFalse("Iterators have uneven length-- first has more: "+length1 + " > " + length, true); assertFalse("Iterators have uneven length-- first has more: "+length1 + " > " + length, true);
} }
if (actual.hasNext) // check if the actual iterator was longer
{ if (actual.hasNext) {
var length2 = length; var length2 = length;
while (actual.hasNext) while (actual.hasNext) {
{
actual.next actual.next
length2 += 1 length2 += 1
} }

View File

@ -77,7 +77,7 @@ class ZKLoadBalanceTest extends JUnit3Suite with ZooKeeperTestHarness {
// wait a bit to make sure rebalancing logic is triggered // wait a bit to make sure rebalancing logic is triggered
Thread.sleep(1000) Thread.sleep(1500)
// check Partition Owner Registry // check Partition Owner Registry
val actual_3 = getZKChildrenValues(dirs.consumerOwnerDir) val actual_3 = getZKChildrenValues(dirs.consumerOwnerDir)
val expected_3 = List( ("200-0", "group1_consumer1-0"), val expected_3 = List( ("200-0", "group1_consumer1-0"),

View File

@ -28,4 +28,5 @@ public interface KafkaProperties
final static int reconnectInterval = 10000; final static int reconnectInterval = 10000;
final static String topic2 = "topic2"; final static String topic2 = "topic2";
final static String topic3 = "topic3"; final static String topic3 = "topic3";
final static String clientId = "SimpleConsumerDemoClient";
} }

View File

@ -16,71 +16,76 @@
*/ */
package kafka.examples; package kafka.examples;
import java.util.ArrayList; import kafka.api.FetchRequest;
import java.util.List; import kafka.api.FetchRequestBuilder;
import kafka.javaapi.FetchResponse;
import kafka.javaapi.MultiFetchResponse;
import kafka.javaapi.consumer.SimpleConsumer; import kafka.javaapi.consumer.SimpleConsumer;
import kafka.javaapi.message.ByteBufferMessageSet; import kafka.javaapi.message.ByteBufferMessageSet;
import kafka.javaapi.message.MessageSet;
import kafka.message.MessageAndOffset; import kafka.message.MessageAndOffset;
import scala.collection.Iterator;
import kafka.api.FetchRequest; import java.util.ArrayList;
import kafka.message.Message; import java.util.HashMap;
import java.util.List;
import java.util.Map;
public class SimpleConsumerDemo public class SimpleConsumerDemo {
{
private static void printMessages(ByteBufferMessageSet messageSet) private static void printMessages(ByteBufferMessageSet messageSet) {
{
for (MessageAndOffset messageAndOffset : messageSet) { for (MessageAndOffset messageAndOffset : messageSet) {
System.out.println(ExampleUtils.getMessage(messageAndOffset.message())); System.out.println(ExampleUtils.getMessage(messageAndOffset.message()));
} }
} }
private static void generateData() private static void generateData() {
{
Producer producer2 = new Producer(KafkaProperties.topic2); Producer producer2 = new Producer(KafkaProperties.topic2);
producer2.start(); producer2.start();
Producer producer3 = new Producer(KafkaProperties.topic3); Producer producer3 = new Producer(KafkaProperties.topic3);
producer3.start(); producer3.start();
try try {
{
Thread.sleep(1000); Thread.sleep(1000);
} } catch (InterruptedException e) {
catch (InterruptedException e)
{
e.printStackTrace(); e.printStackTrace();
} }
} }
public static void main(String[] args) public static void main(String[] args) {
{
generateData(); generateData();
SimpleConsumer simpleConsumer = new SimpleConsumer(KafkaProperties.kafkaServerURL, SimpleConsumer simpleConsumer = new SimpleConsumer(KafkaProperties.kafkaServerURL,
KafkaProperties.kafkaServerPort, KafkaProperties.kafkaServerPort,
KafkaProperties.connectionTimeOut, KafkaProperties.connectionTimeOut,
KafkaProperties.kafkaProducerBufferSize); KafkaProperties.kafkaProducerBufferSize);
System.out.println("Testing single fetch"); System.out.println("Testing single fetch");
FetchRequest req = new FetchRequest(KafkaProperties.topic2, 0, 0L, 100); FetchRequest req = new FetchRequestBuilder()
ByteBufferMessageSet messageSet = simpleConsumer.fetch(req); .correlationId(0)
printMessages(messageSet); .clientId(KafkaProperties.clientId)
.addFetch(KafkaProperties.topic2, 0, 0L, 100)
.build();
FetchResponse fetchResponse = simpleConsumer.fetch(req);
printMessages((ByteBufferMessageSet) fetchResponse.messageSet(KafkaProperties.topic2, 0));
System.out.println("Testing single multi-fetch"); System.out.println("Testing single multi-fetch");
req = new FetchRequest(KafkaProperties.topic2, 0, 0L, 100); Map<String, List<Integer>> topicMap = new HashMap<String, List<Integer>>() {{
List<FetchRequest> list = new ArrayList<FetchRequest>(); put(KafkaProperties.topic2, new ArrayList<Integer>(){{ add(0); }});
list.add(req); put(KafkaProperties.topic3, new ArrayList<Integer>(){{ add(0); }});
req = new FetchRequest(KafkaProperties.topic3, 0, 0L, 100); }};
list.add(req); req = new FetchRequestBuilder()
MultiFetchResponse response = simpleConsumer.multifetch(list); .correlationId(0)
.clientId(KafkaProperties.clientId)
.addFetch(KafkaProperties.topic2, 0, 0L, 100)
.addFetch(KafkaProperties.topic3, 0, 0L, 100)
.build();
fetchResponse = simpleConsumer.fetch(req);
int fetchReq = 0; int fetchReq = 0;
for (ByteBufferMessageSet resMessageSet : response ) for ( Map.Entry<String, List<Integer>> entry : topicMap.entrySet() ) {
{ String topic = entry.getKey();
for ( Integer offset : entry.getValue()) {
System.out.println("Response from fetch request no: " + ++fetchReq); System.out.println("Response from fetch request no: " + ++fetchReq);
printMessages(resMessageSet); printMessages((ByteBufferMessageSet) fetchResponse.messageSet(topic, offset));
}
} }
} }
} }

View File

@ -18,13 +18,12 @@
package kafka.perf package kafka.perf
import java.net.URI import java.net.URI
import joptsimple._
import kafka.utils._
import kafka.server._
import kafka.consumer.SimpleConsumer
import org.apache.log4j.Logger
import kafka.api.{OffsetRequest, FetchRequest}
import java.text.SimpleDateFormat import java.text.SimpleDateFormat
import kafka.api.{FetchRequestBuilder, OffsetRequest}
import kafka.consumer.SimpleConsumer
import kafka.utils._
import org.apache.log4j.Logger
import kafka.message.ByteBufferMessageSet
/** /**
* Performance test for the simple consumer * Performance test for the simple consumer
@ -56,12 +55,20 @@ object SimpleConsumerPerformance {
var lastReportTime: Long = startMs var lastReportTime: Long = startMs
var lastBytesRead = 0L var lastBytesRead = 0L
var lastMessagesRead = 0L var lastMessagesRead = 0L
var reqId = 0
while(!done) { while(!done) {
val messages = consumer.fetch(new FetchRequest(config.topic, config.partition, offset, config.fetchSize)) // TODO: add in the maxWait and minBytes for performance
val request = new FetchRequestBuilder()
.correlationId(reqId)
.clientId(config.clientId)
.addFetch(config.topic, config.partition, offset, config.fetchSize)
.build()
val fetchResponse = consumer.fetch(request)
var messagesRead = 0 var messagesRead = 0
var bytesRead = 0 var bytesRead = 0
val messageSet = fetchResponse.messageSet(config.topic, config.partition)
for(message <- messages) { for (message <- messageSet) {
messagesRead += 1 messagesRead += 1
bytesRead += message.message.payloadSize bytesRead += message.message.payloadSize
} }
@ -69,7 +76,8 @@ object SimpleConsumerPerformance {
if(messagesRead == 0 || totalMessagesRead > config.numMessages) if(messagesRead == 0 || totalMessagesRead > config.numMessages)
done = true done = true
else else
offset += messages.validBytes // we only did one fetch so we find the offset for the first (head) messageset
offset += messageSet.validBytes
totalBytesRead += bytesRead totalBytesRead += bytesRead
totalMessagesRead += messagesRead totalMessagesRead += messagesRead
@ -89,6 +97,7 @@ object SimpleConsumerPerformance {
lastMessagesRead = totalMessagesRead lastMessagesRead = totalMessagesRead
consumedInterval = 0 consumedInterval = 0
} }
reqId += 1
} }
val reportTime = System.currentTimeMillis val reportTime = System.currentTimeMillis
val elapsed = (reportTime - startMs) / 1000.0 val elapsed = (reportTime - startMs) / 1000.0
@ -119,6 +128,11 @@ object SimpleConsumerPerformance {
.describedAs("bytes") .describedAs("bytes")
.ofType(classOf[java.lang.Integer]) .ofType(classOf[java.lang.Integer])
.defaultsTo(1024*1024) .defaultsTo(1024*1024)
val clientIdOpt = parser.accepts("clientId", "The ID of this client.")
.withOptionalArg
.describedAs("clientId")
.ofType(classOf[String])
.defaultsTo("SimpleConsumerPerformanceClient")
val options = parser.parse(args : _*) val options = parser.parse(args : _*)
@ -139,5 +153,6 @@ object SimpleConsumerPerformance {
val showDetailedStats = options.has(showDetailedStatsOpt) val showDetailedStats = options.has(showDetailedStatsOpt)
val dateFormat = new SimpleDateFormat(options.valueOf(dateFormatOpt)) val dateFormat = new SimpleDateFormat(options.valueOf(dateFormatOpt))
val hideHeader = options.has(hideHeaderOpt) val hideHeader = options.has(hideHeaderOpt)
val clientId = options.valueOf(clientIdOpt).toString
} }
} }