You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@kafka.apache.org by jk...@apache.org on 2014/01/28 20:16:27 UTC

[1/7] KAFKA-1227 New producer!

Updated Branches:
  refs/heads/trunk 26a02c32d -> 269d16d3c


http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/test/java/kafka/test/TestUtils.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/kafka/test/TestUtils.java b/clients/src/test/java/kafka/test/TestUtils.java
new file mode 100644
index 0000000..a2ef3a2
--- /dev/null
+++ b/clients/src/test/java/kafka/test/TestUtils.java
@@ -0,0 +1,73 @@
+package kafka.test;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.ServerSocket;
+import java.util.Random;
+
+/**
+ * Helper functions for writing unit tests
+ */
+public class TestUtils {
+
+    public static File IO_TMP_DIR = new File(System.getProperty("java.io.tmpdir"));
+
+    public static String LETTERS = "ABCDEFGHIJKLMNOPQRSTUVWXYZabcdefghijklmnopqrstuvwxyz";
+    public static String DIGITS = "0123456789";
+    public static String LETTERS_AND_DIGITS = LETTERS + DIGITS;
+
+    /* A consistent random number generator to make tests repeatable */
+    public static final Random seededRandom = new Random(192348092834L);
+    public static final Random random = new Random();
+
+    /**
+     * Choose a number of random available ports
+     */
+    public static int[] choosePorts(int count) {
+        try {
+            ServerSocket[] sockets = new ServerSocket[count];
+            int[] ports = new int[count];
+            for (int i = 0; i < count; i++) {
+                sockets[i] = new ServerSocket(0);
+                ports[i] = sockets[i].getLocalPort();
+            }
+            for (int i = 0; i < count; i++)
+                sockets[i].close();
+            return ports;
+        } catch (IOException e) {
+            throw new RuntimeException(e);
+        }
+    }
+
+    /**
+     * Choose an available port
+     */
+    public static int choosePort() {
+        return choosePorts(1)[0];
+    }
+
+    /**
+     * Generate an array of random bytes
+     * 
+     * @param numBytes The size of the array
+     */
+    public static byte[] randomBytes(int size) {
+        byte[] bytes = new byte[size];
+        seededRandom.nextBytes(bytes);
+        return bytes;
+    }
+
+    /**
+     * Generate a random string of letters and digits of the given length
+     * 
+     * @param len The length of the string
+     * @return The random string
+     */
+    public static String randomString(int len) {
+        StringBuilder b = new StringBuilder();
+        for (int i = 0; i < len; i++)
+            b.append(LETTERS_AND_DIGITS.charAt(seededRandom.nextInt(LETTERS_AND_DIGITS.length())));
+        return b.toString();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/project/Build.scala
----------------------------------------------------------------------
diff --git a/project/Build.scala b/project/Build.scala
index 098e874..ddcfc41 100644
--- a/project/Build.scala
+++ b/project/Build.scala
@@ -147,5 +147,6 @@ object KafkaBuild extends Build {
   lazy val contrib        = Project(id = "contrib", base = file("contrib")).aggregate(hadoopProducer, hadoopConsumer).settings(commonSettings :_*)
   lazy val hadoopProducer = Project(id = "hadoop-producer", base = file("contrib/hadoop-producer")).settings(hadoopSettings ++ commonSettings: _*) dependsOn (core)
   lazy val hadoopConsumer = Project(id = "hadoop-consumer", base = file("contrib/hadoop-consumer")).settings(hadoopSettings ++ commonSettings: _*) dependsOn (core)
+  lazy val clients = Project(id = "kafka-clients", base = file("clients"))
 
 }


[6/7] KAFKA-1227 New producer!

Posted by jk...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/clients/producer/internals/Sender.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/kafka/clients/producer/internals/Sender.java
new file mode 100644
index 0000000..effeb9c
--- /dev/null
+++ b/clients/src/main/java/kafka/clients/producer/internals/Sender.java
@@ -0,0 +1,503 @@
+package kafka.clients.producer.internals;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import kafka.common.Cluster;
+import kafka.common.Node;
+import kafka.common.TopicPartition;
+import kafka.common.errors.NetworkException;
+import kafka.common.network.NetworkReceive;
+import kafka.common.network.NetworkSend;
+import kafka.common.network.Selectable;
+import kafka.common.protocol.ApiKeys;
+import kafka.common.protocol.Errors;
+import kafka.common.protocol.ProtoUtils;
+import kafka.common.protocol.types.Struct;
+import kafka.common.requests.RequestHeader;
+import kafka.common.requests.RequestSend;
+import kafka.common.requests.ResponseHeader;
+import kafka.common.utils.Time;
+
+/**
+ * The background thread that handles the sending of produce requests to the Kafka cluster. This thread makes metadata
+ * requests to renew its view of the cluster and then sends produce requests to the appropriate nodes.
+ */
+public class Sender implements Runnable {
+
+    private final Map<Integer, NodeState> nodeState;
+    private final RecordAccumulator accumulator;
+    private final Selectable selector;
+    private final String clientId;
+    private final int maxRequestSize;
+    private final long reconnectBackoffMs;
+    private final short acks;
+    private final int requestTimeout;
+    private final InFlightRequests inFlightRequests;
+    private final Metadata metadata;
+    private final Time time;
+    private int correlation;
+    private boolean metadataFetchInProgress;
+    private volatile boolean running;
+
+    public Sender(Selectable selector,
+                  Metadata metadata,
+                  RecordAccumulator accumulator,
+                  String clientId,
+                  int maxRequestSize,
+                  long reconnectBackoffMs,
+                  short acks,
+                  int requestTimeout,
+                  Time time) {
+        this.nodeState = new HashMap<Integer, NodeState>();
+        this.accumulator = accumulator;
+        this.selector = selector;
+        this.maxRequestSize = maxRequestSize;
+        this.reconnectBackoffMs = reconnectBackoffMs;
+        this.metadata = metadata;
+        this.clientId = clientId;
+        this.running = true;
+        this.requestTimeout = requestTimeout;
+        this.acks = acks;
+        this.inFlightRequests = new InFlightRequests();
+        this.correlation = 0;
+        this.metadataFetchInProgress = false;
+        this.time = time;
+    }
+
+    /**
+     * The main run loop for the sender thread
+     */
+    public void run() {
+        // main loop, runs until close is called
+        while (running) {
+            try {
+                run(time.milliseconds());
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
+
+        // send anything left in the accumulator
+        int unsent = 0;
+        do {
+            try {
+                unsent = run(time.milliseconds());
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        } while (unsent > 0);
+
+        // close all the connections
+        this.selector.close();
+    }
+
+    /**
+     * Run a single iteration of sending
+     * 
+     * @param now The current time
+     * @return The total number of topic/partitions that had data ready (regardless of what we actually sent)
+     */
+    public int run(long now) {
+        Cluster cluster = metadata.fetch();
+        // get the list of partitions with data ready to send
+        List<TopicPartition> ready = this.accumulator.ready(now);
+
+        // prune the list of ready topics to eliminate any that we aren't ready to send yet
+        List<TopicPartition> sendable = processReadyPartitions(cluster, ready, now);
+
+        // should we update our metadata?
+        List<NetworkSend> sends = new ArrayList<NetworkSend>(sendable.size());
+        InFlightRequest metadataReq = maybeMetadataRequest(cluster, now);
+        if (metadataReq != null) {
+            sends.add(metadataReq.request);
+            this.inFlightRequests.add(metadataReq);
+        }
+
+        // create produce requests
+        List<RecordBatch> batches = this.accumulator.drain(sendable, this.maxRequestSize);
+        List<InFlightRequest> requests = collate(cluster, batches);
+        for (int i = 0; i < requests.size(); i++) {
+            InFlightRequest request = requests.get(i);
+            this.inFlightRequests.add(request);
+            sends.add(request.request);
+        }
+
+        // do the I/O
+        try {
+            this.selector.poll(5L, sends);
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+
+        // handle responses, connections, and disconnections
+        handleSends(this.selector.completedSends());
+        handleResponses(this.selector.completedReceives(), now);
+        handleDisconnects(this.selector.disconnected());
+        handleConnects(this.selector.connected());
+
+        return ready.size();
+    }
+
+    private InFlightRequest maybeMetadataRequest(Cluster cluster, long now) {
+        if (this.metadataFetchInProgress || !metadata.needsUpdate(now))
+            return null;
+        Node node = cluster.nextNode();
+        NodeState state = nodeState.get(node.id());
+        if (state == null || (state.state == ConnectionState.DISCONNECTED && now - state.lastConnectAttempt > this.reconnectBackoffMs)) {
+            // we don't have a connection to this node right now, make one
+            initiateConnect(node, now);
+            return null;
+        } else if (state.state == ConnectionState.CONNECTED) {
+            this.metadataFetchInProgress = true;
+            return metadataRequest(node.id(), metadata.topics());
+        } else {
+            return null;
+        }
+    }
+
+    /**
+     * Start closing the sender (won't actually complete until all data is sent out)
+     */
+    public void initiateClose() {
+        this.running = false;
+        this.accumulator.close();
+    }
+
+    /**
+     * Process the set of topic-partitions with data ready to send. If we have a connection to the appropriate node, add
+     * it to the returned set. For any partitions we have no connection to either make one, fetch the appropriate
+     * metdata to be able to do so
+     */
+    private List<TopicPartition> processReadyPartitions(Cluster cluster, List<TopicPartition> ready, long now) {
+        List<TopicPartition> sendable = new ArrayList<TopicPartition>(ready.size());
+        for (TopicPartition tp : ready) {
+            Node node = cluster.leaderFor(tp);
+            if (node == null) {
+                // we don't know about this topic/partition or it has no leader, re-fetch metadata
+                metadata.forceUpdate();
+            } else {
+                NodeState state = nodeState.get(node.id());
+                // TODO: encapsulate this logic somehow
+                if (state == null || (state.state == ConnectionState.DISCONNECTED && now - state.lastConnectAttempt > this.reconnectBackoffMs)) {
+                    // we don't have a connection to this node right now, make one
+                    initiateConnect(node, now);
+                } else if (state.state == ConnectionState.CONNECTED && inFlightRequests.canSendMore(node.id())) {
+                    sendable.add(tp);
+                }
+            }
+        }
+        return sendable;
+    }
+
+    /**
+     * Initiate a connection to the given node
+     */
+    private void initiateConnect(Node node, long now) {
+        try {
+            selector.connect(node.id(), new InetSocketAddress(node.host(), node.port()), 64 * 1024 * 1024, 64 * 1024 * 1024); // TODO
+                                                                                                                              // socket
+                                                                                                                              // buffers
+            nodeState.put(node.id(), new NodeState(ConnectionState.CONNECTING, now));
+        } catch (IOException e) {
+            /* attempt failed, we'll try again after the backoff */
+            nodeState.put(node.id(), new NodeState(ConnectionState.DISCONNECTED, now));
+            /* maybe the problem is our metadata, update it */
+            metadata.forceUpdate();
+        }
+    }
+
+    /**
+     * Handle any closed connections
+     */
+    private void handleDisconnects(List<Integer> disconnects) {
+        for (int node : disconnects) {
+            for (InFlightRequest request : this.inFlightRequests.clearAll(node)) {
+                if (request.batches != null) {
+                    for (RecordBatch batch : request.batches.values())
+                        batch.done(-1L, new NetworkException("The server disconnected unexpectedly without sending a response."));
+                    this.accumulator.deallocate(request.batches.values());
+                }
+                NodeState state = this.nodeState.get(request.request.destination());
+                if (state != null)
+                    state.state = ConnectionState.DISCONNECTED;
+            }
+        }
+    }
+
+    /**
+     * Record any connections that completed in our node state
+     */
+    private void handleConnects(List<Integer> connects) {
+        for (Integer id : connects)
+            this.nodeState.get(id).state = ConnectionState.CONNECTED;
+    }
+
+    /**
+     * Process completed sends
+     */
+    public void handleSends(List<NetworkSend> sends) {
+        /* if acks = 0 then the request is satisfied once sent */
+        for (NetworkSend send : sends) {
+            Deque<InFlightRequest> requests = this.inFlightRequests.requestQueue(send.destination());
+            InFlightRequest request = requests.peekFirst();
+            if (!request.expectResponse) {
+                requests.pollFirst();
+                if (request.request.header().apiKey() == ApiKeys.PRODUCE.id) {
+                    for (RecordBatch batch : request.batches.values())
+                        batch.done(-1L, Errors.NONE.exception());
+                    this.accumulator.deallocate(request.batches.values());
+                }
+            }
+        }
+    }
+
+    /**
+     * Handle responses from the server
+     */
+    private void handleResponses(List<NetworkReceive> receives, long now) {
+        for (NetworkReceive receive : receives) {
+            int source = receive.source();
+            InFlightRequest req = inFlightRequests.nextCompleted(source);
+            ResponseHeader header = ResponseHeader.parse(receive.payload());
+            short apiKey = req.request.header().apiKey();
+            Struct body = (Struct) ProtoUtils.currentResponseSchema(apiKey).read(receive.payload());
+            correlate(req.request.header(), header);
+            if (req.request.header().apiKey() == ApiKeys.PRODUCE.id)
+                handleProduceResponse(req, body);
+            else if (req.request.header().apiKey() == ApiKeys.METADATA.id)
+                handleMetadataResponse(body, now);
+            else
+                throw new IllegalStateException("Unexpected response type: " + req.request.header().apiKey());
+        }
+    }
+
+    private void handleMetadataResponse(Struct body, long now) {
+        this.metadataFetchInProgress = false;
+        this.metadata.update(ProtoUtils.parseMetadataResponse(body), now);
+    }
+
+    /**
+     * Handle a produce response
+     */
+    private void handleProduceResponse(InFlightRequest request, Struct response) {
+        for (Object topicResponse : (Object[]) response.get("responses")) {
+            Struct topicRespStruct = (Struct) topicResponse;
+            String topic = (String) topicRespStruct.get("topic");
+            for (Object partResponse : (Object[]) topicRespStruct.get("partition_responses")) {
+                Struct partRespStruct = (Struct) partResponse;
+                int partition = (Integer) partRespStruct.get("partition");
+                short errorCode = (Short) partRespStruct.get("error_code");
+                long offset = (Long) partRespStruct.get("base_offset");
+                RecordBatch batch = request.batches.get(new TopicPartition(topic, partition));
+                batch.done(offset, Errors.forCode(errorCode).exception());
+            }
+        }
+        this.accumulator.deallocate(request.batches.values());
+    }
+
+    /**
+     * Validate that the response corresponds to the request we expect or else explode
+     */
+    private void correlate(RequestHeader requestHeader, ResponseHeader responseHeader) {
+        if (requestHeader.correlationId() != responseHeader.correlationId())
+            throw new IllegalStateException("Correlation id for response (" + responseHeader.correlationId()
+                                            + ") does not match request ("
+                                            + requestHeader.correlationId()
+                                            + ")");
+    }
+
+    /**
+     * Create a metadata request for the given topics
+     */
+    private InFlightRequest metadataRequest(int node, Set<String> topics) {
+        String[] ts = new String[topics.size()];
+        topics.toArray(ts);
+        Struct body = new Struct(ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id));
+        body.set("topics", topics.toArray());
+        RequestSend send = new RequestSend(node, new RequestHeader(ApiKeys.METADATA.id, clientId, correlation++), body);
+        return new InFlightRequest(true, send, null);
+    }
+
+    /**
+     * Collate the record batches into a list of produce requests on a per-node basis
+     */
+    private List<InFlightRequest> collate(Cluster cluster, List<RecordBatch> batches) {
+        Map<Integer, List<RecordBatch>> collated = new HashMap<Integer, List<RecordBatch>>();
+        for (RecordBatch batch : batches) {
+            Node node = cluster.leaderFor(batch.topicPartition);
+            List<RecordBatch> found = collated.get(node.id());
+            if (found == null) {
+                found = new ArrayList<RecordBatch>();
+                collated.put(node.id(), found);
+            }
+            found.add(batch);
+        }
+        List<InFlightRequest> requests = new ArrayList<InFlightRequest>(collated.size());
+        for (Map.Entry<Integer, List<RecordBatch>> entry : collated.entrySet())
+            requests.add(produceRequest(entry.getKey(), acks, requestTimeout, entry.getValue()));
+        return requests;
+    }
+
+    /**
+     * Create a produce request from the given record batches
+     */
+    private InFlightRequest produceRequest(int destination, short acks, int timeout, List<RecordBatch> batches) {
+        Map<TopicPartition, RecordBatch> batchesByPartition = new HashMap<TopicPartition, RecordBatch>();
+        Map<String, List<RecordBatch>> batchesByTopic = new HashMap<String, List<RecordBatch>>();
+        for (RecordBatch batch : batches) {
+            batchesByPartition.put(batch.topicPartition, batch);
+            List<RecordBatch> found = batchesByTopic.get(batch.topicPartition.topic());
+            if (found == null) {
+                found = new ArrayList<RecordBatch>();
+                batchesByTopic.put(batch.topicPartition.topic(), found);
+            }
+            found.add(batch);
+        }
+        Struct produce = new Struct(ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id));
+        produce.set("acks", acks);
+        produce.set("timeout", timeout);
+        List<Struct> topicDatas = new ArrayList<Struct>(batchesByTopic.size());
+        for (Map.Entry<String, List<RecordBatch>> entry : batchesByTopic.entrySet()) {
+            Struct topicData = produce.instance("topic_data");
+            topicData.set("topic", entry.getKey());
+            List<RecordBatch> parts = entry.getValue();
+            Object[] partitionData = new Object[parts.size()];
+            for (int i = 0; i < parts.size(); i++) {
+                ByteBuffer buffer = parts.get(i).records.buffer();
+                buffer.flip();
+                Struct part = topicData.instance("data")
+                                       .set("partition", parts.get(i).topicPartition.partition())
+                                       .set("message_set", buffer);
+                partitionData[i] = part;
+            }
+            topicData.set("data", partitionData);
+            topicDatas.add(topicData);
+        }
+        produce.set("topic_data", topicDatas.toArray());
+
+        RequestHeader header = new RequestHeader(ApiKeys.PRODUCE.id, clientId, correlation++);
+        RequestSend send = new RequestSend(destination, header, produce);
+        return new InFlightRequest(acks != 0, send, batchesByPartition);
+    }
+
+    /**
+     * Wake up the selector associated with this send thread
+     */
+    public void wakeup() {
+        this.selector.wakeup();
+    }
+
+    /**
+     * The states of a node connection
+     */
+    private static enum ConnectionState {
+        DISCONNECTED, CONNECTING, CONNECTED
+    }
+
+    /**
+     * The state of a node
+     */
+    private static final class NodeState {
+        private ConnectionState state;
+        private long lastConnectAttempt;
+
+        public NodeState(ConnectionState state, long lastConnectAttempt) {
+            this.state = state;
+            this.lastConnectAttempt = lastConnectAttempt;
+        }
+
+        public String toString() {
+            return "NodeState(" + state + ", " + lastConnectAttempt + ")";
+        }
+    }
+
+    /**
+     * An request that hasn't been fully processed yet
+     */
+    private static final class InFlightRequest {
+        public boolean expectResponse;
+        public Map<TopicPartition, RecordBatch> batches;
+        public RequestSend request;
+
+        /**
+         * @param expectResponse Should we expect a response message or is this request complete once it is sent?
+         * @param request The request
+         * @param batches The record batches contained in the request if it is a produce request
+         */
+        public InFlightRequest(boolean expectResponse, RequestSend request, Map<TopicPartition, RecordBatch> batches) {
+            this.batches = batches;
+            this.request = request;
+            this.expectResponse = expectResponse;
+        }
+    }
+
+    /**
+     * A set of outstanding request queues for each node that have not yet received responses
+     */
+    private static final class InFlightRequests {
+        private final Map<Integer, Deque<InFlightRequest>> requests = new HashMap<Integer, Deque<InFlightRequest>>();
+
+        /**
+         * Add the given request to the queue for the node it was directed to
+         */
+        public void add(InFlightRequest request) {
+            Deque<InFlightRequest> reqs = this.requests.get(request.request.destination());
+            if (reqs == null) {
+                reqs = new ArrayDeque<InFlightRequest>();
+                this.requests.put(request.request.destination(), reqs);
+            }
+            reqs.addFirst(request);
+        }
+
+        public Deque<InFlightRequest> requestQueue(int node) {
+            Deque<InFlightRequest> reqs = requests.get(node);
+            if (reqs == null || reqs.isEmpty())
+                throw new IllegalStateException("Response from server for which there are no in-flight requests.");
+            return reqs;
+        }
+
+        /**
+         * Get the oldest request (the one that that will be completed next) for the given node
+         */
+        public InFlightRequest nextCompleted(int node) {
+            return requestQueue(node).pollLast();
+        }
+
+        /**
+         * Can we send more requests to this node?
+         * 
+         * @param node Node in question
+         * @return true iff we have no requests still being sent to the given node
+         */
+        public boolean canSendMore(int node) {
+            Deque<InFlightRequest> queue = requests.get(node);
+            return queue == null || queue.isEmpty() || queue.peekFirst().request.complete();
+        }
+
+        /**
+         * Clear out all the in-flight requests for the given node and return them
+         * 
+         * @param node The node
+         * @return All the in-flight requests for that node that have been removed
+         */
+        public Iterable<InFlightRequest> clearAll(int node) {
+            Deque<InFlightRequest> reqs = requests.get(node);
+            if (reqs == null) {
+                return Collections.emptyList();
+            } else {
+                return requests.remove(node);
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/clients/tools/ProducerPerformance.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/clients/tools/ProducerPerformance.java b/clients/src/main/java/kafka/clients/tools/ProducerPerformance.java
new file mode 100644
index 0000000..7331b73
--- /dev/null
+++ b/clients/src/main/java/kafka/clients/tools/ProducerPerformance.java
@@ -0,0 +1,70 @@
+package kafka.clients.tools;
+
+import java.util.Arrays;
+import java.util.Properties;
+
+import kafka.clients.producer.Callback;
+import kafka.clients.producer.KafkaProducer;
+import kafka.clients.producer.ProducerConfig;
+import kafka.clients.producer.ProducerRecord;
+import kafka.clients.producer.RecordSend;
+import kafka.common.ByteSerialization;
+
+public class ProducerPerformance {
+
+    public static void main(String[] args) throws Exception {
+        if (args.length != 3) {
+            System.err.println("USAGE: java " + ProducerPerformance.class.getName() + " url num_messages message_size");
+            System.exit(1);
+        }
+        String url = args[0];
+        int numMessages = Integer.parseInt(args[1]);
+        int messageSize = Integer.parseInt(args[2]);
+        Properties props = new Properties();
+        props.setProperty(ProducerConfig.REQUIRED_ACKS_CONFIG, "1");
+        props.setProperty(ProducerConfig.BROKER_LIST_CONFIG, url);
+        props.setProperty(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG, Integer.toString(5 * 1000));
+        props.setProperty(ProducerConfig.REQUEST_TIMEOUT_CONFIG, Integer.toString(Integer.MAX_VALUE));
+        props.setProperty(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteSerialization.class.getName());
+        props.setProperty(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteSerialization.class.getName());
+
+        KafkaProducer producer = new KafkaProducer(props);
+        Callback callback = new Callback() {
+            public void onCompletion(RecordSend send) {
+                try {
+                    send.offset();
+                } catch (Exception e) {
+                    e.printStackTrace();
+                }
+            }
+        };
+        byte[] payload = new byte[messageSize];
+        Arrays.fill(payload, (byte) 1);
+        ProducerRecord record = new ProducerRecord("test", payload);
+        long start = System.currentTimeMillis();
+        long maxLatency = -1L;
+        long totalLatency = 0;
+        int reportingInterval = 1000000;
+        for (int i = 0; i < numMessages; i++) {
+            long sendStart = System.currentTimeMillis();
+            producer.send(record, null);
+            long sendEllapsed = System.currentTimeMillis() - sendStart;
+            maxLatency = Math.max(maxLatency, sendEllapsed);
+            totalLatency += sendEllapsed;
+            if (i % reportingInterval == 0) {
+                System.out.printf("%d  max latency = %d ms, avg latency = %.5f\n",
+                                  i,
+                                  maxLatency,
+                                  (totalLatency / (double) reportingInterval));
+                totalLatency = 0L;
+                maxLatency = -1L;
+            }
+        }
+        long ellapsed = System.currentTimeMillis() - start;
+        double msgsSec = 1000.0 * numMessages / (double) ellapsed;
+        double mbSec = msgsSec * messageSize / (1024.0 * 1024.0);
+        System.out.printf("%d messages sent in %d ms ms. %.2f messages per second (%.2f mb/sec).", numMessages, ellapsed, msgsSec, mbSec);
+        producer.close();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/ByteSerialization.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/ByteSerialization.java b/clients/src/main/java/kafka/common/ByteSerialization.java
new file mode 100644
index 0000000..eca69f1
--- /dev/null
+++ b/clients/src/main/java/kafka/common/ByteSerialization.java
@@ -0,0 +1,18 @@
+package kafka.common;
+
+/**
+ * A serialization implementation that just retains the provided byte array unchanged
+ */
+public class ByteSerialization implements Serializer, Deserializer {
+
+    @Override
+    public Object fromBytes(byte[] bytes) {
+        return bytes;
+    }
+
+    @Override
+    public byte[] toBytes(Object o) {
+        return (byte[]) o;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/Cluster.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/Cluster.java b/clients/src/main/java/kafka/common/Cluster.java
new file mode 100644
index 0000000..d0acd8d
--- /dev/null
+++ b/clients/src/main/java/kafka/common/Cluster.java
@@ -0,0 +1,102 @@
+package kafka.common;
+
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import kafka.common.utils.Utils;
+
+/**
+ * A representation of the nodes, topics, and partitions in the Kafka cluster
+ */
+public final class Cluster {
+
+    private final AtomicInteger counter = new AtomicInteger(0);
+    private final List<Node> nodes;
+    private final Map<Integer, Node> nodesById;
+    private final Map<TopicPartition, PartitionInfo> partitionsByTopicPartition;
+    private final Map<String, List<PartitionInfo>> partitionsByTopic;
+
+    /**
+     * Create a new cluster with the given nodes and partitions
+     * @param nodes The nodes in the cluster
+     * @param partitions Information about a subset of the topic-partitions this cluster hosts
+     */
+    public Cluster(Collection<Node> nodes, Collection<PartitionInfo> partitions) {
+        this.nodes = new ArrayList<Node>(nodes);
+        this.nodesById = new HashMap<Integer, Node>(this.nodes.size());
+        this.partitionsByTopicPartition = new HashMap<TopicPartition, PartitionInfo>(partitions.size());
+        this.partitionsByTopic = new HashMap<String, List<PartitionInfo>>(partitions.size());
+
+        Collections.shuffle(this.nodes);
+        for (Node n : this.nodes)
+            this.nodesById.put(n.id(), n);
+        for (PartitionInfo p : partitions)
+            this.partitionsByTopicPartition.put(new TopicPartition(p.topic(), p.partition()), p);
+        for (PartitionInfo p : partitions) {
+            if (!this.partitionsByTopic.containsKey(p.topic()))
+                this.partitionsByTopic.put(p.topic(), new ArrayList<PartitionInfo>());
+            List<PartitionInfo> ps = this.partitionsByTopic.get(p.topic());
+            ps.add(p);
+        }
+    }
+
+    /**
+     * Create an empty cluster instance with no nodes and no topic-partitions.
+     */
+    public static Cluster empty() {
+        return new Cluster(new ArrayList<Node>(0), new ArrayList<PartitionInfo>(0));
+    }
+
+    /**
+     * Create a "bootstrap" cluster using the given list of host/ports
+     * @param addresses The addresses
+     * @return A cluster for these hosts/ports
+     */
+    public static Cluster bootstrap(List<InetSocketAddress> addresses) {
+        List<Node> nodes = new ArrayList<Node>();
+        int nodeId = Integer.MIN_VALUE;
+        for (InetSocketAddress address : addresses)
+            nodes.add(new Node(nodeId++, address.getHostName(), address.getPort()));
+        return new Cluster(nodes, new ArrayList<PartitionInfo>(0));
+    }
+
+    /**
+     * Get the current leader for the given topic-partition
+     * @param topicPartition The topic and partition we want to know the leader for
+     * @return The node that is the leader for this topic-partition, or null if there is currently no leader
+     */
+    public Node leaderFor(TopicPartition topicPartition) {
+        PartitionInfo info = partitionsByTopicPartition.get(topicPartition);
+        if (info == null)
+            return null;
+        else
+            return nodesById.get(info.leader());
+    }
+
+    /**
+     * Get the list of partitions for this topic
+     * @param topic The topic name
+     * @return A list of partitions
+     */
+    public List<PartitionInfo> partitionsFor(String topic) {
+        return this.partitionsByTopic.get(topic);
+    }
+
+    /**
+     * Round-robin over the nodes in this cluster
+     */
+    public Node nextNode() {
+        int size = nodes.size();
+        if (size == 0)
+            throw new IllegalStateException("No known nodes.");
+        int idx = Utils.abs(counter.getAndIncrement()) % size;
+        return this.nodes.get(idx);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/Configurable.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/Configurable.java b/clients/src/main/java/kafka/common/Configurable.java
new file mode 100644
index 0000000..1af9dd4
--- /dev/null
+++ b/clients/src/main/java/kafka/common/Configurable.java
@@ -0,0 +1,15 @@
+package kafka.common;
+
+import java.util.Map;
+
+/**
+ * A Mix-in style interface for classes that are instantiated by reflection and need to take configuration parameters
+ */
+public interface Configurable {
+
+    /**
+     * Configure this class with the given key-value pairs
+     */
+    public void configure(Map<String, ?> configs);
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/Deserializer.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/Deserializer.java b/clients/src/main/java/kafka/common/Deserializer.java
new file mode 100644
index 0000000..ad2e784
--- /dev/null
+++ b/clients/src/main/java/kafka/common/Deserializer.java
@@ -0,0 +1,18 @@
+package kafka.common;
+
+/**
+ * A class that controls how an object is turned into bytes. Classes implementing this interface will generally be
+ * instantiated by the framework.
+ * <p>
+ * An implementation that requires special configuration parameters can implement {@link Configurable}
+ */
+public interface Deserializer {
+
+    /**
+     * Map a byte[] to an object
+     * @param bytes The bytes for the object (can be null)
+     * @return The deserialized object (can return null)
+     */
+    public Object fromBytes(byte[] bytes);
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/KafkaException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/KafkaException.java b/clients/src/main/java/kafka/common/KafkaException.java
new file mode 100644
index 0000000..7182cac
--- /dev/null
+++ b/clients/src/main/java/kafka/common/KafkaException.java
@@ -0,0 +1,26 @@
+package kafka.common;
+
+/**
+ * The base class of all other Kafka exceptions
+ */
+public class KafkaException extends RuntimeException {
+
+    private final static long serialVersionUID = 1L;
+
+    public KafkaException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public KafkaException(String message) {
+        super(message);
+    }
+
+    public KafkaException(Throwable cause) {
+        super(cause);
+    }
+
+    public KafkaException() {
+        super();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/Metric.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/Metric.java b/clients/src/main/java/kafka/common/Metric.java
new file mode 100644
index 0000000..c29e331
--- /dev/null
+++ b/clients/src/main/java/kafka/common/Metric.java
@@ -0,0 +1,23 @@
+package kafka.common;
+
+/**
+ * A numerical metric tracked for monitoring purposes
+ */
+public interface Metric {
+
+    /**
+     * A unique name for this metric
+     */
+    public String name();
+
+    /**
+     * A description of what is measured...this will be "" if no description was given
+     */
+    public String description();
+
+    /**
+     * The value of the metric
+     */
+    public double value();
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/Node.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/Node.java b/clients/src/main/java/kafka/common/Node.java
new file mode 100644
index 0000000..81fc907
--- /dev/null
+++ b/clients/src/main/java/kafka/common/Node.java
@@ -0,0 +1,76 @@
+package kafka.common;
+
+/**
+ * Information about a Kafka node
+ */
+public class Node {
+
+    private final int id;
+    private final String host;
+    private final int port;
+
+    public Node(int id, String host, int port) {
+        super();
+        this.id = id;
+        this.host = host;
+        this.port = port;
+    }
+
+    /**
+     * The node id of this node
+     */
+    public int id() {
+        return id;
+    }
+
+    /**
+     * The host name for this node
+     */
+    public String host() {
+        return host;
+    }
+
+    /**
+     * The port for this node
+     */
+    public int port() {
+        return port;
+    }
+
+    @Override
+    public int hashCode() {
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + ((host == null) ? 0 : host.hashCode());
+        result = prime * result + id;
+        result = prime * result + port;
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        Node other = (Node) obj;
+        if (host == null) {
+            if (other.host != null)
+                return false;
+        } else if (!host.equals(other.host))
+            return false;
+        if (id != other.id)
+            return false;
+        if (port != other.port)
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return "Node(" + id + ", " + host + ", " + port + ")";
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/PartitionInfo.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/PartitionInfo.java b/clients/src/main/java/kafka/common/PartitionInfo.java
new file mode 100644
index 0000000..f3f08dd
--- /dev/null
+++ b/clients/src/main/java/kafka/common/PartitionInfo.java
@@ -0,0 +1,58 @@
+package kafka.common;
+
+/**
+ * Information about a topic-partition.
+ */
+public class PartitionInfo {
+
+    private final String topic;
+    private final int partition;
+    private final int leader;
+    private final int[] replicas;
+    private final int[] inSyncReplicas;
+
+    public PartitionInfo(String topic, int partition, int leader, int[] replicas, int[] inSyncReplicas) {
+        this.topic = topic;
+        this.partition = partition;
+        this.leader = leader;
+        this.replicas = replicas;
+        this.inSyncReplicas = inSyncReplicas;
+    }
+
+    /**
+     * The topic name
+     */
+    public String topic() {
+        return topic;
+    }
+
+    /**
+     * The partition id
+     */
+    public int partition() {
+        return partition;
+    }
+
+    /**
+     * The node id of the node currently acting as a leader for this partition or -1 if there is no leader
+     */
+    public int leader() {
+        return leader;
+    }
+
+    /**
+     * The complete set of replicas for this partition regardless of whether they are alive or up-to-date
+     */
+    public int[] replicas() {
+        return replicas;
+    }
+
+    /**
+     * The subset of the replicas that are in sync, that is caught-up to the leader and ready to take over as leader if
+     * the leader should fail
+     */
+    public int[] inSyncReplicas() {
+        return inSyncReplicas;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/Serializer.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/Serializer.java b/clients/src/main/java/kafka/common/Serializer.java
new file mode 100644
index 0000000..63353d8
--- /dev/null
+++ b/clients/src/main/java/kafka/common/Serializer.java
@@ -0,0 +1,21 @@
+package kafka.common;
+
+/**
+ * A class that controls how an object is turned into bytes. Classes implementing this interface will generally be
+ * instantiated by the framework.
+ * <p>
+ * An implementation should handle null inputs.
+ * <p>
+ * An implementation that requires special configuration parameters can implement {@link Configurable}
+ */
+public interface Serializer {
+
+    /**
+     * Translate an object into bytes. The serializer must handle null inputs, and will generally just return null in
+     * this case.
+     * @param o The object to serialize, can be null
+     * @return The serialized bytes for the object or null
+     */
+    public byte[] toBytes(Object o);
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/StringSerialization.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/StringSerialization.java b/clients/src/main/java/kafka/common/StringSerialization.java
new file mode 100644
index 0000000..c0ed5ca
--- /dev/null
+++ b/clients/src/main/java/kafka/common/StringSerialization.java
@@ -0,0 +1,58 @@
+package kafka.common;
+
+import java.io.UnsupportedEncodingException;
+import java.util.Map;
+
+/**
+ * A serializer and deserializer for strings.
+ * <p>
+ * This class accepts a configuration parameter string.encoding which can take the string name of any supported
+ * encoding. If no encoding is specified the default will be UTF-8.
+ */
+public class StringSerialization implements Serializer, Deserializer, Configurable {
+
+    private final static String ENCODING_CONFIG = "string.encoding";
+
+    private String encoding;
+
+    public StringSerialization(String encoding) {
+        super();
+        this.encoding = encoding;
+    }
+
+    public StringSerialization() {
+        this("UTF8");
+    }
+
+    public void configure(Map<String, ?> configs) {
+        if (configs.containsKey(ENCODING_CONFIG))
+            this.encoding = (String) configs.get(ENCODING_CONFIG);
+    }
+
+    @Override
+    public Object fromBytes(byte[] bytes) {
+        if (bytes == null) {
+            return null;
+        } else {
+            try {
+                return new String(bytes, encoding);
+            } catch (UnsupportedEncodingException e) {
+                throw new KafkaException(e);
+            }
+        }
+    }
+
+    @Override
+    public byte[] toBytes(Object o) {
+        if (o == null) {
+            return null;
+        } else {
+            try {
+                return ((String) o).getBytes(encoding);
+            } catch (UnsupportedEncodingException e) {
+                throw new KafkaException(e);
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/TopicPartition.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/TopicPartition.java b/clients/src/main/java/kafka/common/TopicPartition.java
new file mode 100644
index 0000000..e7be96c
--- /dev/null
+++ b/clients/src/main/java/kafka/common/TopicPartition.java
@@ -0,0 +1,61 @@
+package kafka.common;
+
+/**
+ * A topic name and partition number
+ */
+public final class TopicPartition {
+
+    private int hash = 0;
+    private final int partition;
+    private final String topic;
+
+    public TopicPartition(String topic, int partition) {
+        this.partition = partition;
+        this.topic = topic;
+    }
+
+    public int partition() {
+        return partition;
+    }
+
+    public String topic() {
+        return topic;
+    }
+
+    @Override
+    public int hashCode() {
+        if (hash != 0)
+            return hash;
+        final int prime = 31;
+        int result = 1;
+        result = prime * result + partition;
+        result = prime * result + ((topic == null) ? 0 : topic.hashCode());
+        this.hash = result;
+        return result;
+    }
+
+    @Override
+    public boolean equals(Object obj) {
+        if (this == obj)
+            return true;
+        if (obj == null)
+            return false;
+        if (getClass() != obj.getClass())
+            return false;
+        TopicPartition other = (TopicPartition) obj;
+        if (partition != other.partition)
+            return false;
+        if (topic == null) {
+            if (other.topic != null)
+                return false;
+        } else if (!topic.equals(other.topic))
+            return false;
+        return true;
+    }
+
+    @Override
+    public String toString() {
+        return topic + "-" + partition;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/config/AbstractConfig.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/config/AbstractConfig.java b/clients/src/main/java/kafka/common/config/AbstractConfig.java
new file mode 100644
index 0000000..5db302d
--- /dev/null
+++ b/clients/src/main/java/kafka/common/config/AbstractConfig.java
@@ -0,0 +1,93 @@
+package kafka.common.config;
+
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import kafka.common.Configurable;
+import kafka.common.KafkaException;
+import kafka.common.utils.Utils;
+
+/**
+ * A convenient base class for configurations to extend.
+ * <p>
+ * This class holds both the original configuration that was provided as well as the parsed
+ */
+public class AbstractConfig {
+
+    private final Set<String> used;
+    private final Map<String, Object> values;
+    private final Map<String, ?> originals;
+
+    @SuppressWarnings("unchecked")
+    public AbstractConfig(ConfigDef definition, Map<?, ?> originals) {
+        /* check that all the keys are really strings */
+        for (Object key : originals.keySet())
+            if (!(key instanceof String))
+                throw new ConfigException(key.toString(), originals.get(key), "Key must be a string.");
+        this.originals = (Map<String, ?>) originals;
+        this.values = definition.parse(this.originals);
+        this.used = Collections.synchronizedSet(new HashSet<String>());
+    }
+
+    protected Object get(String key) {
+        if (!values.containsKey(key))
+            throw new ConfigException(String.format("Unknown configuration '%s'", key));
+        used.add(key);
+        return values.get(key);
+    }
+
+    public int getInt(String key) {
+        return (Integer) get(key);
+    }
+
+    public Long getLong(String key) {
+        return (Long) get(key);
+    }
+
+    @SuppressWarnings("unchecked")
+    public List<String> getList(String key) {
+        return (List<String>) get(key);
+    }
+
+    public boolean getBoolean(String key) {
+        return (Boolean) get(key);
+    }
+
+    public String getString(String key) {
+        return (String) get(key);
+    }
+
+    public Class<?> getClass(String key) {
+        return (Class<?>) get(key);
+    }
+
+    public Set<String> unused() {
+        Set<String> keys = new HashSet<String>(originals.keySet());
+        keys.remove(used);
+        return keys;
+    }
+
+    /**
+     * Get a configured instance of the give class specified by the given configuration key. If the object implements
+     * Configurable configure it using the configuration.
+     * 
+     * @param key The configuration key for the class
+     * @param t The interface the class should implement
+     * @return A configured instance of the class
+     */
+    public <T> T getConfiguredInstance(String key, Class<T> t) {
+        Class<?> c = getClass(key);
+        if (c == null)
+            return null;
+        Object o = Utils.newInstance(c);
+        if (!t.isInstance(o))
+            throw new KafkaException(c.getName() + " is not an instance of " + t.getName());
+        if (o instanceof Configurable)
+            ((Configurable) o).configure(this.originals);
+        return t.cast(o);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/config/ConfigDef.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/config/ConfigDef.java b/clients/src/main/java/kafka/common/config/ConfigDef.java
new file mode 100644
index 0000000..2507c9c
--- /dev/null
+++ b/clients/src/main/java/kafka/common/config/ConfigDef.java
@@ -0,0 +1,253 @@
+package kafka.common.config;
+
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * This class is used for specifying the set of expected configurations, their type, their defaults, their
+ * documentation, and any special validation logic used for checking the correctness of the values the user provides.
+ * <p>
+ * Usage of this class looks something like this:
+ * 
+ * <pre>
+ * ConfigDef defs = new ConfigDef();
+ * defs.define(&quot;config_name&quot;, Type.STRING, &quot;default string value&quot;, &quot;This configuration is used for blah blah blah.&quot;);
+ * defs.define(&quot;another_config_name&quot;, Type.INT, 42, Range.atLeast(0), &quot;More documentation on this config&quot;);
+ * 
+ * Properties props = new Properties();
+ * props.setProperty(&quot;config_name&quot;, &quot;some value&quot;);
+ * Map&lt;String, Object&gt; configs = defs.parse(props);
+ * 
+ * String someConfig = (String) configs.get(&quot;config_name&quot;); // will return &quot;some value&quot;
+ * int anotherConfig = (Integer) configs.get(&quot;another_config_name&quot;); // will return default value of 42
+ * </pre>
+ * 
+ * This class can be used stand-alone or in combination with {@link AbstractConfig} which provides some additional
+ * functionality for accessing configs.
+ */
+public class ConfigDef {
+
+    private static final Object NO_DEFAULT_VALUE = new Object();
+
+    private final Map<String, ConfigKey> configKeys = new HashMap<String, ConfigKey>();
+
+    /**
+     * Define a new configuration
+     * @param name The name of the config parameter
+     * @param type The type of the config
+     * @param defaultValue The default value to use if this config isn't present
+     * @param validator A validator to use in checking the correctness of the config
+     * @param documentation The documentation string for the config
+     * @return This ConfigDef so you can chain calls
+     */
+    public ConfigDef define(String name, Type type, Object defaultValue, Validator validator, String documentation) {
+        if (configKeys.containsKey(name))
+            throw new ConfigException("Configuration " + name + " is defined twice.");
+        Object parsedDefault = defaultValue == NO_DEFAULT_VALUE ? NO_DEFAULT_VALUE : parseType(name, defaultValue, type);
+        configKeys.put(name, new ConfigKey(name, type, parsedDefault, validator, documentation));
+        return this;
+    }
+
+    /**
+     * Define a new configuration with no special validation logic
+     * @param name The name of the config parameter
+     * @param type The type of the config
+     * @param defaultValue The default value to use if this config isn't present
+     * @param documentation The documentation string for the config
+     * @return This ConfigDef so you can chain calls
+     */
+    public ConfigDef define(String name, Type type, Object defaultValue, String documentation) {
+        return define(name, type, defaultValue, null, documentation);
+    }
+
+    /**
+     * Define a required parameter with no default value
+     * @param name The name of the config parameter
+     * @param type The type of the config
+     * @param validator A validator to use in checking the correctness of the config
+     * @param documentation The documentation string for the config
+     * @return This ConfigDef so you can chain calls
+     */
+    public ConfigDef define(String name, Type type, Validator validator, String documentation) {
+        return define(name, type, NO_DEFAULT_VALUE, validator, documentation);
+    }
+
+    /**
+     * Define a required parameter with no default value and no special validation logic
+     * @param name The name of the config parameter
+     * @param type The type of the config
+     * @param documentation The documentation string for the config
+     * @return This ConfigDef so you can chain calls
+     */
+    public ConfigDef define(String name, Type type, String documentation) {
+        return define(name, type, NO_DEFAULT_VALUE, null, documentation);
+    }
+
+    /**
+     * Parse and validate configs against this configuration definition. The input is a map of configs. It is expected
+     * that the keys of the map are strings, but the values can either be strings or they may already be of the
+     * appropriate type (int, string, etc). This will work equally well with either java.util.Properties instances or a
+     * programmatically constructed map.
+     * @param props The configs to parse and validate
+     * @return Parsed and validated configs. The key will be the config name and the value will be the value parsed into
+     *         the appropriate type (int, string, etc)
+     */
+    public Map<String, Object> parse(Map<?, ?> props) {
+        /* parse all known keys */
+        Map<String, Object> values = new HashMap<String, Object>();
+        for (ConfigKey key : configKeys.values()) {
+            Object value;
+            if (props.containsKey(key.name))
+                value = parseType(key.name, props.get(key.name), key.type);
+            else if (key.defaultValue == NO_DEFAULT_VALUE)
+                throw new ConfigException("Missing required configuration \"" + key.name + "\" which has no default value.");
+            else
+                value = key.defaultValue;
+            values.put(key.name, value);
+        }
+        return values;
+    }
+
+    /**
+     * Parse a value according to its expected type.
+     * @param name The config name
+     * @param value The config value
+     * @param type The expected type
+     * @return The parsed object
+     */
+    private Object parseType(String name, Object value, Type type) {
+        try {
+            String trimmed = null;
+            if (value instanceof String)
+                trimmed = ((String) value).trim();
+            switch (type) {
+                case BOOLEAN:
+                    if (value instanceof String)
+                        return Boolean.parseBoolean(trimmed);
+                    else if (value instanceof Boolean)
+                        return value;
+                    else
+                        throw new ConfigException(name, value, "Expected value to be either true or false");
+                case STRING:
+                    if (value instanceof String)
+                        return trimmed;
+                    else
+                        throw new ConfigException(name, value, "Expected value to be a string, but it was a " + value.getClass().getName());
+                case INT:
+                    if (value instanceof Integer) {
+                        return (Integer) value;
+                    } else if (value instanceof String) {
+                        return Integer.parseInt(trimmed);
+                    } else {
+                        throw new ConfigException(name, value, "Expected value to be an number.");
+                    }
+                case LONG:
+                    if (value instanceof Integer)
+                        return ((Integer) value).longValue();
+                    if (value instanceof Long)
+                        return (Long) value;
+                    else if (value instanceof String)
+                        return Long.parseLong(trimmed);
+                    else
+                        throw new ConfigException(name, value, "Expected value to be an number.");
+                case DOUBLE:
+                    if (value instanceof Number)
+                        return ((Number) value).doubleValue();
+                    else if (value instanceof String)
+                        return Double.parseDouble(trimmed);
+                    else
+                        throw new ConfigException(name, value, "Expected value to be an number.");
+                case LIST:
+                    if (value instanceof List)
+                        return (List<?>) value;
+                    else if (value instanceof String)
+                        return Arrays.asList(trimmed.split("\\s*,\\s*", -1));
+                    else
+                        throw new ConfigException(name, value, "Expected a comma seperated list.");
+                case CLASS:
+                    if (value instanceof Class)
+                        return (Class<?>) value;
+                    else if (value instanceof String)
+                        return Class.forName(trimmed);
+                    else
+                        throw new ConfigException(name, value, "Expected a Class instance or class name.");
+                default:
+                    throw new IllegalStateException("Unknown type.");
+            }
+        } catch (NumberFormatException e) {
+            throw new ConfigException(name, value, "Not a number of type " + type);
+        } catch (ClassNotFoundException e) {
+            throw new ConfigException(name, value, "Class " + value + " could not be found.");
+        }
+    }
+
+    /**
+     * The config types
+     */
+    public enum Type {
+        BOOLEAN, STRING, INT, LONG, DOUBLE, LIST, CLASS;
+    }
+
+    /**
+     * Validation logic the user may provide
+     */
+    public interface Validator {
+        public void ensureValid(String name, Object o);
+    }
+
+    /**
+     * Validation logic for numeric ranges
+     */
+    public static class Range implements Validator {
+        private final Number min;
+        private final Number max;
+
+        private Range(Number min, Number max) {
+            this.min = min;
+            this.max = max;
+        }
+
+        /**
+         * A numeric range that checks only the lower bound
+         * @param min The minimum acceptable value
+         */
+        public static Range atLeast(Number min) {
+            return new Range(min, Double.MAX_VALUE);
+        }
+
+        /**
+         * A numeric range that checks both the upper and lower bound
+         */
+        public static Range between(Number min, Number max) {
+            return new Range(min, max);
+        }
+
+        public void ensureValid(String name, Object o) {
+            Number n = (Number) o;
+            if (n.doubleValue() < min.doubleValue() || n.doubleValue() > max.doubleValue())
+                throw new ConfigException(name, o, "Value must be in the range [" + min + ", " + max + "]");
+        }
+    }
+
+    private static class ConfigKey {
+        public final String name;
+        public final Type type;
+        public final String documentation;
+        public final Object defaultValue;
+        public final Validator validator;
+
+        public ConfigKey(String name, Type type, Object defaultValue, Validator validator, String documentation) {
+            super();
+            this.name = name;
+            this.type = type;
+            this.defaultValue = defaultValue;
+            this.validator = validator;
+            if (this.validator != null)
+                this.validator.ensureValid(name, defaultValue);
+            this.documentation = documentation;
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/config/ConfigException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/config/ConfigException.java b/clients/src/main/java/kafka/common/config/ConfigException.java
new file mode 100644
index 0000000..fad141e
--- /dev/null
+++ b/clients/src/main/java/kafka/common/config/ConfigException.java
@@ -0,0 +1,24 @@
+package kafka.common.config;
+
+import kafka.common.KafkaException;
+
+/**
+ * Thrown if the user supplies an invalid configuration
+ */
+public class ConfigException extends KafkaException {
+
+    private static final long serialVersionUID = 1L;
+
+    public ConfigException(String message) {
+        super(message);
+    }
+
+    public ConfigException(String name, Object value) {
+        this(name, value, null);
+    }
+
+    public ConfigException(String name, Object value, String message) {
+        super("Invalid value " + value + " for configuration " + name + (message == null ? "" : ": " + message));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/errors/ApiException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/errors/ApiException.java b/clients/src/main/java/kafka/common/errors/ApiException.java
new file mode 100644
index 0000000..28f5411
--- /dev/null
+++ b/clients/src/main/java/kafka/common/errors/ApiException.java
@@ -0,0 +1,35 @@
+package kafka.common.errors;
+
+import kafka.common.KafkaException;
+
+/**
+ * Any API exception that is part of the public protocol and should be a subclass of this class and be part of this
+ * package.
+ */
+public abstract class ApiException extends KafkaException {
+
+    private static final long serialVersionUID = 1L;
+
+    public ApiException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public ApiException(String message) {
+        super(message);
+    }
+
+    public ApiException(Throwable cause) {
+        super(cause);
+    }
+
+    public ApiException() {
+        super();
+    }
+
+    /* avoid the expensive and useless stack trace for api exceptions */
+    @Override
+    public Throwable fillInStackTrace() {
+        return this;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/errors/CorruptMessageException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/errors/CorruptMessageException.java b/clients/src/main/java/kafka/common/errors/CorruptMessageException.java
new file mode 100644
index 0000000..faf6234
--- /dev/null
+++ b/clients/src/main/java/kafka/common/errors/CorruptMessageException.java
@@ -0,0 +1,23 @@
+package kafka.common.errors;
+
+public class CorruptMessageException extends ApiException {
+
+    private static final long serialVersionUID = 1L;
+
+    public CorruptMessageException() {
+        super("This message has failed it's CRC checksum or is otherwise corrupt.");
+    }
+
+    public CorruptMessageException(String message) {
+        super(message);
+    }
+
+    public CorruptMessageException(Throwable cause) {
+        super(cause);
+    }
+
+    public CorruptMessageException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/errors/LeaderNotAvailableException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/errors/LeaderNotAvailableException.java b/clients/src/main/java/kafka/common/errors/LeaderNotAvailableException.java
new file mode 100644
index 0000000..d7b86f8
--- /dev/null
+++ b/clients/src/main/java/kafka/common/errors/LeaderNotAvailableException.java
@@ -0,0 +1,19 @@
+package kafka.common.errors;
+
+public class LeaderNotAvailableException extends RetryableException {
+
+    private static final long serialVersionUID = 1L;
+
+    public LeaderNotAvailableException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public LeaderNotAvailableException(String message) {
+        super(message);
+    }
+
+    public LeaderNotAvailableException(Throwable cause) {
+        super(cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/errors/MessageTooLargeException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/errors/MessageTooLargeException.java b/clients/src/main/java/kafka/common/errors/MessageTooLargeException.java
new file mode 100644
index 0000000..7417906
--- /dev/null
+++ b/clients/src/main/java/kafka/common/errors/MessageTooLargeException.java
@@ -0,0 +1,23 @@
+package kafka.common.errors;
+
+public class MessageTooLargeException extends ApiException {
+
+    private static final long serialVersionUID = 1L;
+
+    public MessageTooLargeException() {
+        super();
+    }
+
+    public MessageTooLargeException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public MessageTooLargeException(String message) {
+        super(message);
+    }
+
+    public MessageTooLargeException(Throwable cause) {
+        super(cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/errors/NetworkException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/errors/NetworkException.java b/clients/src/main/java/kafka/common/errors/NetworkException.java
new file mode 100644
index 0000000..daedbf4
--- /dev/null
+++ b/clients/src/main/java/kafka/common/errors/NetworkException.java
@@ -0,0 +1,23 @@
+package kafka.common.errors;
+
+public class NetworkException extends ApiException {
+
+    private static final long serialVersionUID = 1L;
+
+    public NetworkException() {
+        super();
+    }
+
+    public NetworkException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public NetworkException(String message) {
+        super(message);
+    }
+
+    public NetworkException(Throwable cause) {
+        super(cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/errors/NotLeaderForPartitionException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/errors/NotLeaderForPartitionException.java b/clients/src/main/java/kafka/common/errors/NotLeaderForPartitionException.java
new file mode 100644
index 0000000..5d750fd
--- /dev/null
+++ b/clients/src/main/java/kafka/common/errors/NotLeaderForPartitionException.java
@@ -0,0 +1,23 @@
+package kafka.common.errors;
+
+public class NotLeaderForPartitionException extends RetryableException {
+
+    private static final long serialVersionUID = 1L;
+
+    public NotLeaderForPartitionException() {
+        super();
+    }
+
+    public NotLeaderForPartitionException(String message) {
+        super(message);
+    }
+
+    public NotLeaderForPartitionException(Throwable cause) {
+        super(cause);
+    }
+
+    public NotLeaderForPartitionException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/errors/OffsetMetadataTooLarge.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/errors/OffsetMetadataTooLarge.java b/clients/src/main/java/kafka/common/errors/OffsetMetadataTooLarge.java
new file mode 100644
index 0000000..ab9cd62
--- /dev/null
+++ b/clients/src/main/java/kafka/common/errors/OffsetMetadataTooLarge.java
@@ -0,0 +1,22 @@
+package kafka.common.errors;
+
+public class OffsetMetadataTooLarge extends ApiException {
+
+    private static final long serialVersionUID = 1L;
+
+    public OffsetMetadataTooLarge() {
+    }
+
+    public OffsetMetadataTooLarge(String message) {
+        super(message);
+    }
+
+    public OffsetMetadataTooLarge(Throwable cause) {
+        super(cause);
+    }
+
+    public OffsetMetadataTooLarge(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/errors/OffsetOutOfRangeException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/errors/OffsetOutOfRangeException.java b/clients/src/main/java/kafka/common/errors/OffsetOutOfRangeException.java
new file mode 100644
index 0000000..93210cd
--- /dev/null
+++ b/clients/src/main/java/kafka/common/errors/OffsetOutOfRangeException.java
@@ -0,0 +1,22 @@
+package kafka.common.errors;
+
+public class OffsetOutOfRangeException extends ApiException {
+
+    private static final long serialVersionUID = 1L;
+
+    public OffsetOutOfRangeException() {
+    }
+
+    public OffsetOutOfRangeException(String message) {
+        super(message);
+    }
+
+    public OffsetOutOfRangeException(Throwable cause) {
+        super(cause);
+    }
+
+    public OffsetOutOfRangeException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/errors/RetryableException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/errors/RetryableException.java b/clients/src/main/java/kafka/common/errors/RetryableException.java
new file mode 100644
index 0000000..5aa8684
--- /dev/null
+++ b/clients/src/main/java/kafka/common/errors/RetryableException.java
@@ -0,0 +1,31 @@
+package kafka.common.errors;
+
+/**
+ * A retryable exception is an exception that is safe to retry. To be retryable an exception should be
+ * <ol>
+ * <li>Transient, there is no point retrying a error due to a non-existant topic or message too large
+ * <li>Idempotent, the exception is known to not change any state on the server
+ * </ol>
+ * A client may choose to retry any request they like, but exceptions extending this class are always safe and sane to
+ * retry.
+ */
+public abstract class RetryableException extends ApiException {
+
+    private static final long serialVersionUID = 1L;
+
+    public RetryableException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public RetryableException(String message) {
+        super(message);
+    }
+
+    public RetryableException(Throwable cause) {
+        super(cause);
+    }
+
+    public RetryableException() {
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/errors/TimeoutException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/errors/TimeoutException.java b/clients/src/main/java/kafka/common/errors/TimeoutException.java
new file mode 100644
index 0000000..da27a98
--- /dev/null
+++ b/clients/src/main/java/kafka/common/errors/TimeoutException.java
@@ -0,0 +1,23 @@
+package kafka.common.errors;
+
+public class TimeoutException extends ApiException {
+
+    private static final long serialVersionUID = 1L;
+
+    public TimeoutException() {
+        super();
+    }
+
+    public TimeoutException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+    public TimeoutException(String message) {
+        super(message);
+    }
+
+    public TimeoutException(Throwable cause) {
+        super(cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/errors/UnknownServerException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/errors/UnknownServerException.java b/clients/src/main/java/kafka/common/errors/UnknownServerException.java
new file mode 100644
index 0000000..d0b56d6
--- /dev/null
+++ b/clients/src/main/java/kafka/common/errors/UnknownServerException.java
@@ -0,0 +1,22 @@
+package kafka.common.errors;
+
+public class UnknownServerException extends ApiException {
+
+    private static final long serialVersionUID = 1L;
+
+    public UnknownServerException() {
+    }
+
+    public UnknownServerException(String message) {
+        super(message);
+    }
+
+    public UnknownServerException(Throwable cause) {
+        super(cause);
+    }
+
+    public UnknownServerException(String message, Throwable cause) {
+        super(message, cause);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/errors/UnknownTopicOrPartitionException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/errors/UnknownTopicOrPartitionException.java b/clients/src/main/java/kafka/common/errors/UnknownTopicOrPartitionException.java
new file mode 100644
index 0000000..5c1ca12
--- /dev/null
+++ b/clients/src/main/java/kafka/common/errors/UnknownTopicOrPartitionException.java
@@ -0,0 +1,22 @@
+package kafka.common.errors;
+
+public class UnknownTopicOrPartitionException extends ApiException {
+
+    private static final long serialVersionUID = 1L;
+
+    public UnknownTopicOrPartitionException() {
+    }
+
+    public UnknownTopicOrPartitionException(String message) {
+        super(message);
+    }
+
+    public UnknownTopicOrPartitionException(Throwable throwable) {
+        super(throwable);
+    }
+
+    public UnknownTopicOrPartitionException(String message, Throwable throwable) {
+        super(message, throwable);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/CompoundStat.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/CompoundStat.java b/clients/src/main/java/kafka/common/metrics/CompoundStat.java
new file mode 100644
index 0000000..5541e32
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/CompoundStat.java
@@ -0,0 +1,40 @@
+package kafka.common.metrics;
+
+import java.util.List;
+
+/**
+ * A compound stat is a stat where a single measurement and associated data structure feeds many metrics. This is the
+ * example for a histogram which has many associated percentiles.
+ */
+public interface CompoundStat extends Stat {
+
+    public List<NamedMeasurable> stats();
+
+    public static class NamedMeasurable {
+
+        private final String name;
+        private final String description;
+        private final Measurable stat;
+
+        public NamedMeasurable(String name, String description, Measurable stat) {
+            super();
+            this.name = name;
+            this.description = description;
+            this.stat = stat;
+        }
+
+        public String name() {
+            return name;
+        }
+
+        public String description() {
+            return description;
+        }
+
+        public Measurable stat() {
+            return stat;
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/JmxReporter.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/JmxReporter.java b/clients/src/main/java/kafka/common/metrics/JmxReporter.java
new file mode 100644
index 0000000..a0cee01
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/JmxReporter.java
@@ -0,0 +1,184 @@
+package kafka.common.metrics;
+
+import java.lang.management.ManagementFactory;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import javax.management.Attribute;
+import javax.management.AttributeList;
+import javax.management.AttributeNotFoundException;
+import javax.management.DynamicMBean;
+import javax.management.InvalidAttributeValueException;
+import javax.management.JMException;
+import javax.management.MBeanAttributeInfo;
+import javax.management.MBeanException;
+import javax.management.MBeanInfo;
+import javax.management.MBeanServer;
+import javax.management.MalformedObjectNameException;
+import javax.management.ObjectName;
+import javax.management.ReflectionException;
+
+import kafka.common.KafkaException;
+
+/**
+ * Register metrics in JMX as dynamic mbeans based on the metric names
+ */
+public class JmxReporter implements MetricsReporter {
+
+    private final String prefix;
+    private final Map<String, KafkaMbean> mbeans = new HashMap<String, KafkaMbean>();
+
+    public JmxReporter() {
+        this("");
+    }
+
+    /**
+     * Create a JMX reporter that prefixes all metrics with the given string.
+     */
+    public JmxReporter(String prefix) {
+        this.prefix = prefix;
+    }
+
+    @Override
+    public synchronized void init(List<KafkaMetric> metrics) {
+        for (KafkaMetric metric : metrics)
+            addAttribute(metric);
+        for (KafkaMbean mbean : mbeans.values())
+            reregister(mbean);
+
+    }
+
+    @Override
+    public synchronized void metricChange(KafkaMetric metric) {
+        KafkaMbean mbean = addAttribute(metric);
+        reregister(mbean);
+    }
+
+    private KafkaMbean addAttribute(KafkaMetric metric) {
+        try {
+            String[] names = split(prefix + metric.name());
+            String qualifiedName = names[0] + "." + names[1];
+            if (!this.mbeans.containsKey(qualifiedName))
+                mbeans.put(qualifiedName, new KafkaMbean(names[0], names[1]));
+            KafkaMbean mbean = this.mbeans.get(qualifiedName);
+            mbean.setAttribute(names[2], metric);
+            return mbean;
+        } catch (JMException e) {
+            throw new KafkaException("Error creating mbean attribute " + metric.name(), e);
+        }
+    }
+
+    public synchronized void close() {
+        for (KafkaMbean mbean : this.mbeans.values())
+            unregister(mbean);
+
+    }
+
+    private void unregister(KafkaMbean mbean) {
+        MBeanServer server = ManagementFactory.getPlatformMBeanServer();
+        try {
+            if (server.isRegistered(mbean.name()))
+                server.unregisterMBean(mbean.name());
+        } catch (JMException e) {
+            throw new KafkaException("Error unregistering mbean", e);
+        }
+    }
+
+    private void reregister(KafkaMbean mbean) {
+        unregister(mbean);
+        try {
+            ManagementFactory.getPlatformMBeanServer().registerMBean(mbean, mbean.name());
+        } catch (JMException e) {
+            throw new KafkaException("Error registering mbean " + mbean.name(), e);
+        }
+    }
+
+    private String[] split(String name) {
+        int attributeStart = name.lastIndexOf('.');
+        if (attributeStart < 0)
+            throw new IllegalArgumentException("No MBean name in metric name: " + name);
+        String attributeName = name.substring(attributeStart + 1, name.length());
+        String remainder = name.substring(0, attributeStart);
+        int beanStart = remainder.lastIndexOf('.');
+        if (beanStart < 0)
+            return new String[] { "", remainder, attributeName };
+        String packageName = remainder.substring(0, beanStart);
+        String beanName = remainder.substring(beanStart + 1, remainder.length());
+        return new String[] { packageName, beanName, attributeName };
+    }
+
+    private static class KafkaMbean implements DynamicMBean {
+        private final String beanName;
+        private final ObjectName objectName;
+        private final Map<String, KafkaMetric> metrics;
+
+        public KafkaMbean(String packageName, String beanName) throws MalformedObjectNameException {
+            this.beanName = beanName;
+            this.metrics = new HashMap<String, KafkaMetric>();
+            this.objectName = new ObjectName(packageName + ":type=" + beanName);
+        }
+
+        public ObjectName name() {
+            return objectName;
+        }
+
+        public void setAttribute(String name, KafkaMetric metric) {
+            this.metrics.put(name, metric);
+        }
+
+        @Override
+        public Object getAttribute(String name) throws AttributeNotFoundException, MBeanException, ReflectionException {
+            if (this.metrics.containsKey(name))
+                return this.metrics.get(name).value();
+            else
+                throw new AttributeNotFoundException("Could not find attribute " + name);
+        }
+
+        @Override
+        public AttributeList getAttributes(String[] names) {
+            try {
+                AttributeList list = new AttributeList();
+                for (String name : names)
+                    list.add(new Attribute(name, getAttribute(name)));
+                return list;
+            } catch (Exception e) {
+                e.printStackTrace();
+                return new AttributeList();
+            }
+        }
+
+        @Override
+        public MBeanInfo getMBeanInfo() {
+            MBeanAttributeInfo[] attrs = new MBeanAttributeInfo[metrics.size()];
+            int i = 0;
+            for (Map.Entry<String, KafkaMetric> entry : this.metrics.entrySet()) {
+                String attribute = entry.getKey();
+                KafkaMetric metric = entry.getValue();
+                attrs[i] = new MBeanAttributeInfo(attribute, double.class.getName(), metric.description(), true, false, false);
+                i += 1;
+            }
+            return new MBeanInfo(beanName, "", attrs, null, null, null);
+        }
+
+        @Override
+        public Object invoke(String name, Object[] params, String[] sig) throws MBeanException, ReflectionException {
+            throw new UnsupportedOperationException("Set not allowed.");
+        }
+
+        @Override
+        public void setAttribute(Attribute attribute) throws AttributeNotFoundException,
+                                                     InvalidAttributeValueException,
+                                                     MBeanException,
+                                                     ReflectionException {
+            throw new UnsupportedOperationException("Set not allowed.");
+        }
+
+        @Override
+        public AttributeList setAttributes(AttributeList list) {
+            throw new UnsupportedOperationException("Set not allowed.");
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/KafkaMetric.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/KafkaMetric.java b/clients/src/main/java/kafka/common/metrics/KafkaMetric.java
new file mode 100644
index 0000000..33212b0
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/KafkaMetric.java
@@ -0,0 +1,55 @@
+package kafka.common.metrics;
+
+import kafka.common.Metric;
+import kafka.common.utils.Time;
+
+public final class KafkaMetric implements Metric {
+
+    private final String name;
+    private final String description;
+    private final Object lock;
+    private final Time time;
+    private final Measurable measurable;
+    private MetricConfig config;
+
+    KafkaMetric(Object lock, String name, String description, Measurable measurable, MetricConfig config, Time time) {
+        super();
+        this.name = name;
+        this.description = description;
+        this.lock = lock;
+        this.measurable = measurable;
+        this.config = config;
+        this.time = time;
+    }
+
+    MetricConfig config() {
+        return this.config;
+    }
+
+    @Override
+    public String name() {
+        return this.name;
+    }
+
+    @Override
+    public String description() {
+        return this.description;
+    }
+
+    @Override
+    public double value() {
+        synchronized (this.lock) {
+            return value(time.nanoseconds());
+        }
+    }
+
+    double value(long time) {
+        return this.measurable.measure(config, time);
+    }
+
+    public void config(MetricConfig config) {
+        synchronized (lock) {
+            this.config = config;
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/Measurable.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/Measurable.java b/clients/src/main/java/kafka/common/metrics/Measurable.java
new file mode 100644
index 0000000..f5511ea
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/Measurable.java
@@ -0,0 +1,16 @@
+package kafka.common.metrics;
+
+/**
+ * A measurable quantity that can be registered as a metric
+ */
+public interface Measurable {
+
+    /**
+     * Measure this quantity and return the result as a double
+     * @param config The configuration for this metric
+     * @param now The time the measurement is being taken
+     * @return The measured value
+     */
+    public double measure(MetricConfig config, long now);
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/MeasurableStat.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/MeasurableStat.java b/clients/src/main/java/kafka/common/metrics/MeasurableStat.java
new file mode 100644
index 0000000..74d3bb4
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/MeasurableStat.java
@@ -0,0 +1,10 @@
+package kafka.common.metrics;
+
+/**
+ * A MeasurableStat is a {@link Stat} that is also {@link Measurable} (i.e. can produce a single floating point value).
+ * This is the interface used for most of the simple statistics such as {@link kafka.common.metrics.stats.Avg},
+ * {@link kafka.common.metrics.stats.Max}, {@link kafka.common.metrics.stats.Count}, etc.
+ */
+public interface MeasurableStat extends Stat, Measurable {
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/MetricConfig.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/MetricConfig.java b/clients/src/main/java/kafka/common/metrics/MetricConfig.java
new file mode 100644
index 0000000..92f67f0
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/MetricConfig.java
@@ -0,0 +1,71 @@
+package kafka.common.metrics;
+
+import java.util.concurrent.TimeUnit;
+
+/**
+ * Configuration values for metrics
+ */
+public class MetricConfig {
+
+    private Quota quota;
+    private int samples;
+    private long eventWindow;
+    private long timeWindowNs;
+    private TimeUnit unit;
+
+    public MetricConfig() {
+        super();
+        this.quota = null;
+        this.samples = 2;
+        this.eventWindow = Long.MAX_VALUE;
+        this.timeWindowNs = TimeUnit.NANOSECONDS.convert(30, TimeUnit.SECONDS);
+        this.unit = TimeUnit.SECONDS;
+    }
+
+    public Quota quota() {
+        return this.quota;
+    }
+
+    public MetricConfig quota(Quota quota) {
+        this.quota = quota;
+        return this;
+    }
+
+    public long eventWindow() {
+        return eventWindow;
+    }
+
+    public MetricConfig eventWindow(long window) {
+        this.eventWindow = window;
+        return this;
+    }
+
+    public long timeWindowNs() {
+        return timeWindowNs;
+    }
+
+    public MetricConfig timeWindow(long window, TimeUnit unit) {
+        this.timeWindowNs = TimeUnit.NANOSECONDS.convert(window, unit);
+        return this;
+    }
+
+    public int samples() {
+        return this.samples;
+    }
+
+    public MetricConfig samples(int samples) {
+        if (samples < 1)
+            throw new IllegalArgumentException("The number of samples must be at least 1.");
+        this.samples = samples;
+        return this;
+    }
+
+    public TimeUnit timeUnit() {
+        return unit;
+    }
+
+    public MetricConfig timeUnit(TimeUnit unit) {
+        this.unit = unit;
+        return this;
+    }
+}


[3/7] KAFKA-1227 New producer!

Posted by jk...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/utils/Crc32.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/utils/Crc32.java b/clients/src/main/java/kafka/common/utils/Crc32.java
new file mode 100644
index 0000000..d929c8d
--- /dev/null
+++ b/clients/src/main/java/kafka/common/utils/Crc32.java
@@ -0,0 +1,2169 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE
+ * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the
+ * License. You may obtain a copy of the License at
+ * 
+ * http://www.apache.org/licenses/LICENSE-2.0
+ * 
+ * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on
+ * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the
+ * specific language governing permissions and limitations under the License.
+ */
+package kafka.common.utils;
+
+import java.util.zip.Checksum;
+
+/**
+ * This class was taken from Hadoop org.apache.hadoop.util.PureJavaCrc32
+ * 
+ * A pure-java implementation of the CRC32 checksum that uses the same polynomial as the built-in native CRC32.
+ * 
+ * This is to avoid the JNI overhead for certain uses of Checksumming where many small pieces of data are checksummed in
+ * succession.
+ * 
+ * The current version is ~10x to 1.8x as fast as Sun's native java.util.zip.CRC32 in Java 1.6
+ * 
+ * @see java.util.zip.CRC32
+ */
+public class Crc32 implements Checksum {
+
+    /** the current CRC value, bit-flipped */
+    private int crc;
+
+    /** Create a new PureJavaCrc32 object. */
+    public Crc32() {
+        reset();
+    }
+
+    @Override
+    public long getValue() {
+        return (~crc) & 0xffffffffL;
+    }
+
+    @Override
+    public void reset() {
+        crc = 0xffffffff;
+    }
+
+    @Override
+    public void update(byte[] b, int off, int len) {
+        int localCrc = crc;
+
+        while (len > 7) {
+            final int c0 = (b[off + 0] ^ localCrc) & 0xff;
+            final int c1 = (b[off + 1] ^ (localCrc >>>= 8)) & 0xff;
+            final int c2 = (b[off + 2] ^ (localCrc >>>= 8)) & 0xff;
+            final int c3 = (b[off + 3] ^ (localCrc >>>= 8)) & 0xff;
+            localCrc = (T[T8_7_start + c0] ^ T[T8_6_start + c1]) ^ (T[T8_5_start + c2] ^ T[T8_4_start + c3]);
+
+            final int c4 = b[off + 4] & 0xff;
+            final int c5 = b[off + 5] & 0xff;
+            final int c6 = b[off + 6] & 0xff;
+            final int c7 = b[off + 7] & 0xff;
+
+            localCrc ^= (T[T8_3_start + c4] ^ T[T8_2_start + c5]) ^ (T[T8_1_start + c6] ^ T[T8_0_start + c7]);
+
+            off += 8;
+            len -= 8;
+        }
+
+        /* loop unroll - duff's device style */
+        switch (len) {
+            case 7:
+                localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)];
+            case 6:
+                localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)];
+            case 5:
+                localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)];
+            case 4:
+                localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)];
+            case 3:
+                localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)];
+            case 2:
+                localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)];
+            case 1:
+                localCrc = (localCrc >>> 8) ^ T[T8_0_start + ((localCrc ^ b[off++]) & 0xff)];
+            default:
+                /* nothing */
+        }
+
+        // Publish crc out to object
+        crc = localCrc;
+    }
+
+    @Override
+    final public void update(int b) {
+        crc = (crc >>> 8) ^ T[T8_0_start + ((crc ^ b) & 0xff)];
+    }
+
+    /*
+     * CRC-32 lookup tables generated by the polynomial 0xEDB88320. See also TestPureJavaCrc32.Table.
+     */
+    private static final int T8_0_start = 0 * 256;
+    private static final int T8_1_start = 1 * 256;
+    private static final int T8_2_start = 2 * 256;
+    private static final int T8_3_start = 3 * 256;
+    private static final int T8_4_start = 4 * 256;
+    private static final int T8_5_start = 5 * 256;
+    private static final int T8_6_start = 6 * 256;
+    private static final int T8_7_start = 7 * 256;
+
+    private static final int[] T = new int[] {
+                                              /* T8_0 */
+                                              0x00000000,
+                                              0x77073096,
+                                              0xEE0E612C,
+                                              0x990951BA,
+                                              0x076DC419,
+                                              0x706AF48F,
+                                              0xE963A535,
+                                              0x9E6495A3,
+                                              0x0EDB8832,
+                                              0x79DCB8A4,
+                                              0xE0D5E91E,
+                                              0x97D2D988,
+                                              0x09B64C2B,
+                                              0x7EB17CBD,
+                                              0xE7B82D07,
+                                              0x90BF1D91,
+                                              0x1DB71064,
+                                              0x6AB020F2,
+                                              0xF3B97148,
+                                              0x84BE41DE,
+                                              0x1ADAD47D,
+                                              0x6DDDE4EB,
+                                              0xF4D4B551,
+                                              0x83D385C7,
+                                              0x136C9856,
+                                              0x646BA8C0,
+                                              0xFD62F97A,
+                                              0x8A65C9EC,
+                                              0x14015C4F,
+                                              0x63066CD9,
+                                              0xFA0F3D63,
+                                              0x8D080DF5,
+                                              0x3B6E20C8,
+                                              0x4C69105E,
+                                              0xD56041E4,
+                                              0xA2677172,
+                                              0x3C03E4D1,
+                                              0x4B04D447,
+                                              0xD20D85FD,
+                                              0xA50AB56B,
+                                              0x35B5A8FA,
+                                              0x42B2986C,
+                                              0xDBBBC9D6,
+                                              0xACBCF940,
+                                              0x32D86CE3,
+                                              0x45DF5C75,
+                                              0xDCD60DCF,
+                                              0xABD13D59,
+                                              0x26D930AC,
+                                              0x51DE003A,
+                                              0xC8D75180,
+                                              0xBFD06116,
+                                              0x21B4F4B5,
+                                              0x56B3C423,
+                                              0xCFBA9599,
+                                              0xB8BDA50F,
+                                              0x2802B89E,
+                                              0x5F058808,
+                                              0xC60CD9B2,
+                                              0xB10BE924,
+                                              0x2F6F7C87,
+                                              0x58684C11,
+                                              0xC1611DAB,
+                                              0xB6662D3D,
+                                              0x76DC4190,
+                                              0x01DB7106,
+                                              0x98D220BC,
+                                              0xEFD5102A,
+                                              0x71B18589,
+                                              0x06B6B51F,
+                                              0x9FBFE4A5,
+                                              0xE8B8D433,
+                                              0x7807C9A2,
+                                              0x0F00F934,
+                                              0x9609A88E,
+                                              0xE10E9818,
+                                              0x7F6A0DBB,
+                                              0x086D3D2D,
+                                              0x91646C97,
+                                              0xE6635C01,
+                                              0x6B6B51F4,
+                                              0x1C6C6162,
+                                              0x856530D8,
+                                              0xF262004E,
+                                              0x6C0695ED,
+                                              0x1B01A57B,
+                                              0x8208F4C1,
+                                              0xF50FC457,
+                                              0x65B0D9C6,
+                                              0x12B7E950,
+                                              0x8BBEB8EA,
+                                              0xFCB9887C,
+                                              0x62DD1DDF,
+                                              0x15DA2D49,
+                                              0x8CD37CF3,
+                                              0xFBD44C65,
+                                              0x4DB26158,
+                                              0x3AB551CE,
+                                              0xA3BC0074,
+                                              0xD4BB30E2,
+                                              0x4ADFA541,
+                                              0x3DD895D7,
+                                              0xA4D1C46D,
+                                              0xD3D6F4FB,
+                                              0x4369E96A,
+                                              0x346ED9FC,
+                                              0xAD678846,
+                                              0xDA60B8D0,
+                                              0x44042D73,
+                                              0x33031DE5,
+                                              0xAA0A4C5F,
+                                              0xDD0D7CC9,
+                                              0x5005713C,
+                                              0x270241AA,
+                                              0xBE0B1010,
+                                              0xC90C2086,
+                                              0x5768B525,
+                                              0x206F85B3,
+                                              0xB966D409,
+                                              0xCE61E49F,
+                                              0x5EDEF90E,
+                                              0x29D9C998,
+                                              0xB0D09822,
+                                              0xC7D7A8B4,
+                                              0x59B33D17,
+                                              0x2EB40D81,
+                                              0xB7BD5C3B,
+                                              0xC0BA6CAD,
+                                              0xEDB88320,
+                                              0x9ABFB3B6,
+                                              0x03B6E20C,
+                                              0x74B1D29A,
+                                              0xEAD54739,
+                                              0x9DD277AF,
+                                              0x04DB2615,
+                                              0x73DC1683,
+                                              0xE3630B12,
+                                              0x94643B84,
+                                              0x0D6D6A3E,
+                                              0x7A6A5AA8,
+                                              0xE40ECF0B,
+                                              0x9309FF9D,
+                                              0x0A00AE27,
+                                              0x7D079EB1,
+                                              0xF00F9344,
+                                              0x8708A3D2,
+                                              0x1E01F268,
+                                              0x6906C2FE,
+                                              0xF762575D,
+                                              0x806567CB,
+                                              0x196C3671,
+                                              0x6E6B06E7,
+                                              0xFED41B76,
+                                              0x89D32BE0,
+                                              0x10DA7A5A,
+                                              0x67DD4ACC,
+                                              0xF9B9DF6F,
+                                              0x8EBEEFF9,
+                                              0x17B7BE43,
+                                              0x60B08ED5,
+                                              0xD6D6A3E8,
+                                              0xA1D1937E,
+                                              0x38D8C2C4,
+                                              0x4FDFF252,
+                                              0xD1BB67F1,
+                                              0xA6BC5767,
+                                              0x3FB506DD,
+                                              0x48B2364B,
+                                              0xD80D2BDA,
+                                              0xAF0A1B4C,
+                                              0x36034AF6,
+                                              0x41047A60,
+                                              0xDF60EFC3,
+                                              0xA867DF55,
+                                              0x316E8EEF,
+                                              0x4669BE79,
+                                              0xCB61B38C,
+                                              0xBC66831A,
+                                              0x256FD2A0,
+                                              0x5268E236,
+                                              0xCC0C7795,
+                                              0xBB0B4703,
+                                              0x220216B9,
+                                              0x5505262F,
+                                              0xC5BA3BBE,
+                                              0xB2BD0B28,
+                                              0x2BB45A92,
+                                              0x5CB36A04,
+                                              0xC2D7FFA7,
+                                              0xB5D0CF31,
+                                              0x2CD99E8B,
+                                              0x5BDEAE1D,
+                                              0x9B64C2B0,
+                                              0xEC63F226,
+                                              0x756AA39C,
+                                              0x026D930A,
+                                              0x9C0906A9,
+                                              0xEB0E363F,
+                                              0x72076785,
+                                              0x05005713,
+                                              0x95BF4A82,
+                                              0xE2B87A14,
+                                              0x7BB12BAE,
+                                              0x0CB61B38,
+                                              0x92D28E9B,
+                                              0xE5D5BE0D,
+                                              0x7CDCEFB7,
+                                              0x0BDBDF21,
+                                              0x86D3D2D4,
+                                              0xF1D4E242,
+                                              0x68DDB3F8,
+                                              0x1FDA836E,
+                                              0x81BE16CD,
+                                              0xF6B9265B,
+                                              0x6FB077E1,
+                                              0x18B74777,
+                                              0x88085AE6,
+                                              0xFF0F6A70,
+                                              0x66063BCA,
+                                              0x11010B5C,
+                                              0x8F659EFF,
+                                              0xF862AE69,
+                                              0x616BFFD3,
+                                              0x166CCF45,
+                                              0xA00AE278,
+                                              0xD70DD2EE,
+                                              0x4E048354,
+                                              0x3903B3C2,
+                                              0xA7672661,
+                                              0xD06016F7,
+                                              0x4969474D,
+                                              0x3E6E77DB,
+                                              0xAED16A4A,
+                                              0xD9D65ADC,
+                                              0x40DF0B66,
+                                              0x37D83BF0,
+                                              0xA9BCAE53,
+                                              0xDEBB9EC5,
+                                              0x47B2CF7F,
+                                              0x30B5FFE9,
+                                              0xBDBDF21C,
+                                              0xCABAC28A,
+                                              0x53B39330,
+                                              0x24B4A3A6,
+                                              0xBAD03605,
+                                              0xCDD70693,
+                                              0x54DE5729,
+                                              0x23D967BF,
+                                              0xB3667A2E,
+                                              0xC4614AB8,
+                                              0x5D681B02,
+                                              0x2A6F2B94,
+                                              0xB40BBE37,
+                                              0xC30C8EA1,
+                                              0x5A05DF1B,
+                                              0x2D02EF8D,
+                                              /* T8_1 */
+                                              0x00000000,
+                                              0x191B3141,
+                                              0x32366282,
+                                              0x2B2D53C3,
+                                              0x646CC504,
+                                              0x7D77F445,
+                                              0x565AA786,
+                                              0x4F4196C7,
+                                              0xC8D98A08,
+                                              0xD1C2BB49,
+                                              0xFAEFE88A,
+                                              0xE3F4D9CB,
+                                              0xACB54F0C,
+                                              0xB5AE7E4D,
+                                              0x9E832D8E,
+                                              0x87981CCF,
+                                              0x4AC21251,
+                                              0x53D92310,
+                                              0x78F470D3,
+                                              0x61EF4192,
+                                              0x2EAED755,
+                                              0x37B5E614,
+                                              0x1C98B5D7,
+                                              0x05838496,
+                                              0x821B9859,
+                                              0x9B00A918,
+                                              0xB02DFADB,
+                                              0xA936CB9A,
+                                              0xE6775D5D,
+                                              0xFF6C6C1C,
+                                              0xD4413FDF,
+                                              0xCD5A0E9E,
+                                              0x958424A2,
+                                              0x8C9F15E3,
+                                              0xA7B24620,
+                                              0xBEA97761,
+                                              0xF1E8E1A6,
+                                              0xE8F3D0E7,
+                                              0xC3DE8324,
+                                              0xDAC5B265,
+                                              0x5D5DAEAA,
+                                              0x44469FEB,
+                                              0x6F6BCC28,
+                                              0x7670FD69,
+                                              0x39316BAE,
+                                              0x202A5AEF,
+                                              0x0B07092C,
+                                              0x121C386D,
+                                              0xDF4636F3,
+                                              0xC65D07B2,
+                                              0xED705471,
+                                              0xF46B6530,
+                                              0xBB2AF3F7,
+                                              0xA231C2B6,
+                                              0x891C9175,
+                                              0x9007A034,
+                                              0x179FBCFB,
+                                              0x0E848DBA,
+                                              0x25A9DE79,
+                                              0x3CB2EF38,
+                                              0x73F379FF,
+                                              0x6AE848BE,
+                                              0x41C51B7D,
+                                              0x58DE2A3C,
+                                              0xF0794F05,
+                                              0xE9627E44,
+                                              0xC24F2D87,
+                                              0xDB541CC6,
+                                              0x94158A01,
+                                              0x8D0EBB40,
+                                              0xA623E883,
+                                              0xBF38D9C2,
+                                              0x38A0C50D,
+                                              0x21BBF44C,
+                                              0x0A96A78F,
+                                              0x138D96CE,
+                                              0x5CCC0009,
+                                              0x45D73148,
+                                              0x6EFA628B,
+                                              0x77E153CA,
+                                              0xBABB5D54,
+                                              0xA3A06C15,
+                                              0x888D3FD6,
+                                              0x91960E97,
+                                              0xDED79850,
+                                              0xC7CCA911,
+                                              0xECE1FAD2,
+                                              0xF5FACB93,
+                                              0x7262D75C,
+                                              0x6B79E61D,
+                                              0x4054B5DE,
+                                              0x594F849F,
+                                              0x160E1258,
+                                              0x0F152319,
+                                              0x243870DA,
+                                              0x3D23419B,
+                                              0x65FD6BA7,
+                                              0x7CE65AE6,
+                                              0x57CB0925,
+                                              0x4ED03864,
+                                              0x0191AEA3,
+                                              0x188A9FE2,
+                                              0x33A7CC21,
+                                              0x2ABCFD60,
+                                              0xAD24E1AF,
+                                              0xB43FD0EE,
+                                              0x9F12832D,
+                                              0x8609B26C,
+                                              0xC94824AB,
+                                              0xD05315EA,
+                                              0xFB7E4629,
+                                              0xE2657768,
+                                              0x2F3F79F6,
+                                              0x362448B7,
+                                              0x1D091B74,
+                                              0x04122A35,
+                                              0x4B53BCF2,
+                                              0x52488DB3,
+                                              0x7965DE70,
+                                              0x607EEF31,
+                                              0xE7E6F3FE,
+                                              0xFEFDC2BF,
+                                              0xD5D0917C,
+                                              0xCCCBA03D,
+                                              0x838A36FA,
+                                              0x9A9107BB,
+                                              0xB1BC5478,
+                                              0xA8A76539,
+                                              0x3B83984B,
+                                              0x2298A90A,
+                                              0x09B5FAC9,
+                                              0x10AECB88,
+                                              0x5FEF5D4F,
+                                              0x46F46C0E,
+                                              0x6DD93FCD,
+                                              0x74C20E8C,
+                                              0xF35A1243,
+                                              0xEA412302,
+                                              0xC16C70C1,
+                                              0xD8774180,
+                                              0x9736D747,
+                                              0x8E2DE606,
+                                              0xA500B5C5,
+                                              0xBC1B8484,
+                                              0x71418A1A,
+                                              0x685ABB5B,
+                                              0x4377E898,
+                                              0x5A6CD9D9,
+                                              0x152D4F1E,
+                                              0x0C367E5F,
+                                              0x271B2D9C,
+                                              0x3E001CDD,
+                                              0xB9980012,
+                                              0xA0833153,
+                                              0x8BAE6290,
+                                              0x92B553D1,
+                                              0xDDF4C516,
+                                              0xC4EFF457,
+                                              0xEFC2A794,
+                                              0xF6D996D5,
+                                              0xAE07BCE9,
+                                              0xB71C8DA8,
+                                              0x9C31DE6B,
+                                              0x852AEF2A,
+                                              0xCA6B79ED,
+                                              0xD37048AC,
+                                              0xF85D1B6F,
+                                              0xE1462A2E,
+                                              0x66DE36E1,
+                                              0x7FC507A0,
+                                              0x54E85463,
+                                              0x4DF36522,
+                                              0x02B2F3E5,
+                                              0x1BA9C2A4,
+                                              0x30849167,
+                                              0x299FA026,
+                                              0xE4C5AEB8,
+                                              0xFDDE9FF9,
+                                              0xD6F3CC3A,
+                                              0xCFE8FD7B,
+                                              0x80A96BBC,
+                                              0x99B25AFD,
+                                              0xB29F093E,
+                                              0xAB84387F,
+                                              0x2C1C24B0,
+                                              0x350715F1,
+                                              0x1E2A4632,
+                                              0x07317773,
+                                              0x4870E1B4,
+                                              0x516BD0F5,
+                                              0x7A468336,
+                                              0x635DB277,
+                                              0xCBFAD74E,
+                                              0xD2E1E60F,
+                                              0xF9CCB5CC,
+                                              0xE0D7848D,
+                                              0xAF96124A,
+                                              0xB68D230B,
+                                              0x9DA070C8,
+                                              0x84BB4189,
+                                              0x03235D46,
+                                              0x1A386C07,
+                                              0x31153FC4,
+                                              0x280E0E85,
+                                              0x674F9842,
+                                              0x7E54A903,
+                                              0x5579FAC0,
+                                              0x4C62CB81,
+                                              0x8138C51F,
+                                              0x9823F45E,
+                                              0xB30EA79D,
+                                              0xAA1596DC,
+                                              0xE554001B,
+                                              0xFC4F315A,
+                                              0xD7626299,
+                                              0xCE7953D8,
+                                              0x49E14F17,
+                                              0x50FA7E56,
+                                              0x7BD72D95,
+                                              0x62CC1CD4,
+                                              0x2D8D8A13,
+                                              0x3496BB52,
+                                              0x1FBBE891,
+                                              0x06A0D9D0,
+                                              0x5E7EF3EC,
+                                              0x4765C2AD,
+                                              0x6C48916E,
+                                              0x7553A02F,
+                                              0x3A1236E8,
+                                              0x230907A9,
+                                              0x0824546A,
+                                              0x113F652B,
+                                              0x96A779E4,
+                                              0x8FBC48A5,
+                                              0xA4911B66,
+                                              0xBD8A2A27,
+                                              0xF2CBBCE0,
+                                              0xEBD08DA1,
+                                              0xC0FDDE62,
+                                              0xD9E6EF23,
+                                              0x14BCE1BD,
+                                              0x0DA7D0FC,
+                                              0x268A833F,
+                                              0x3F91B27E,
+                                              0x70D024B9,
+                                              0x69CB15F8,
+                                              0x42E6463B,
+                                              0x5BFD777A,
+                                              0xDC656BB5,
+                                              0xC57E5AF4,
+                                              0xEE530937,
+                                              0xF7483876,
+                                              0xB809AEB1,
+                                              0xA1129FF0,
+                                              0x8A3FCC33,
+                                              0x9324FD72,
+                                              /* T8_2 */
+                                              0x00000000,
+                                              0x01C26A37,
+                                              0x0384D46E,
+                                              0x0246BE59,
+                                              0x0709A8DC,
+                                              0x06CBC2EB,
+                                              0x048D7CB2,
+                                              0x054F1685,
+                                              0x0E1351B8,
+                                              0x0FD13B8F,
+                                              0x0D9785D6,
+                                              0x0C55EFE1,
+                                              0x091AF964,
+                                              0x08D89353,
+                                              0x0A9E2D0A,
+                                              0x0B5C473D,
+                                              0x1C26A370,
+                                              0x1DE4C947,
+                                              0x1FA2771E,
+                                              0x1E601D29,
+                                              0x1B2F0BAC,
+                                              0x1AED619B,
+                                              0x18ABDFC2,
+                                              0x1969B5F5,
+                                              0x1235F2C8,
+                                              0x13F798FF,
+                                              0x11B126A6,
+                                              0x10734C91,
+                                              0x153C5A14,
+                                              0x14FE3023,
+                                              0x16B88E7A,
+                                              0x177AE44D,
+                                              0x384D46E0,
+                                              0x398F2CD7,
+                                              0x3BC9928E,
+                                              0x3A0BF8B9,
+                                              0x3F44EE3C,
+                                              0x3E86840B,
+                                              0x3CC03A52,
+                                              0x3D025065,
+                                              0x365E1758,
+                                              0x379C7D6F,
+                                              0x35DAC336,
+                                              0x3418A901,
+                                              0x3157BF84,
+                                              0x3095D5B3,
+                                              0x32D36BEA,
+                                              0x331101DD,
+                                              0x246BE590,
+                                              0x25A98FA7,
+                                              0x27EF31FE,
+                                              0x262D5BC9,
+                                              0x23624D4C,
+                                              0x22A0277B,
+                                              0x20E69922,
+                                              0x2124F315,
+                                              0x2A78B428,
+                                              0x2BBADE1F,
+                                              0x29FC6046,
+                                              0x283E0A71,
+                                              0x2D711CF4,
+                                              0x2CB376C3,
+                                              0x2EF5C89A,
+                                              0x2F37A2AD,
+                                              0x709A8DC0,
+                                              0x7158E7F7,
+                                              0x731E59AE,
+                                              0x72DC3399,
+                                              0x7793251C,
+                                              0x76514F2B,
+                                              0x7417F172,
+                                              0x75D59B45,
+                                              0x7E89DC78,
+                                              0x7F4BB64F,
+                                              0x7D0D0816,
+                                              0x7CCF6221,
+                                              0x798074A4,
+                                              0x78421E93,
+                                              0x7A04A0CA,
+                                              0x7BC6CAFD,
+                                              0x6CBC2EB0,
+                                              0x6D7E4487,
+                                              0x6F38FADE,
+                                              0x6EFA90E9,
+                                              0x6BB5866C,
+                                              0x6A77EC5B,
+                                              0x68315202,
+                                              0x69F33835,
+                                              0x62AF7F08,
+                                              0x636D153F,
+                                              0x612BAB66,
+                                              0x60E9C151,
+                                              0x65A6D7D4,
+                                              0x6464BDE3,
+                                              0x662203BA,
+                                              0x67E0698D,
+                                              0x48D7CB20,
+                                              0x4915A117,
+                                              0x4B531F4E,
+                                              0x4A917579,
+                                              0x4FDE63FC,
+                                              0x4E1C09CB,
+                                              0x4C5AB792,
+                                              0x4D98DDA5,
+                                              0x46C49A98,
+                                              0x4706F0AF,
+                                              0x45404EF6,
+                                              0x448224C1,
+                                              0x41CD3244,
+                                              0x400F5873,
+                                              0x4249E62A,
+                                              0x438B8C1D,
+                                              0x54F16850,
+                                              0x55330267,
+                                              0x5775BC3E,
+                                              0x56B7D609,
+                                              0x53F8C08C,
+                                              0x523AAABB,
+                                              0x507C14E2,
+                                              0x51BE7ED5,
+                                              0x5AE239E8,
+                                              0x5B2053DF,
+                                              0x5966ED86,
+                                              0x58A487B1,
+                                              0x5DEB9134,
+                                              0x5C29FB03,
+                                              0x5E6F455A,
+                                              0x5FAD2F6D,
+                                              0xE1351B80,
+                                              0xE0F771B7,
+                                              0xE2B1CFEE,
+                                              0xE373A5D9,
+                                              0xE63CB35C,
+                                              0xE7FED96B,
+                                              0xE5B86732,
+                                              0xE47A0D05,
+                                              0xEF264A38,
+                                              0xEEE4200F,
+                                              0xECA29E56,
+                                              0xED60F461,
+                                              0xE82FE2E4,
+                                              0xE9ED88D3,
+                                              0xEBAB368A,
+                                              0xEA695CBD,
+                                              0xFD13B8F0,
+                                              0xFCD1D2C7,
+                                              0xFE976C9E,
+                                              0xFF5506A9,
+                                              0xFA1A102C,
+                                              0xFBD87A1B,
+                                              0xF99EC442,
+                                              0xF85CAE75,
+                                              0xF300E948,
+                                              0xF2C2837F,
+                                              0xF0843D26,
+                                              0xF1465711,
+                                              0xF4094194,
+                                              0xF5CB2BA3,
+                                              0xF78D95FA,
+                                              0xF64FFFCD,
+                                              0xD9785D60,
+                                              0xD8BA3757,
+                                              0xDAFC890E,
+                                              0xDB3EE339,
+                                              0xDE71F5BC,
+                                              0xDFB39F8B,
+                                              0xDDF521D2,
+                                              0xDC374BE5,
+                                              0xD76B0CD8,
+                                              0xD6A966EF,
+                                              0xD4EFD8B6,
+                                              0xD52DB281,
+                                              0xD062A404,
+                                              0xD1A0CE33,
+                                              0xD3E6706A,
+                                              0xD2241A5D,
+                                              0xC55EFE10,
+                                              0xC49C9427,
+                                              0xC6DA2A7E,
+                                              0xC7184049,
+                                              0xC25756CC,
+                                              0xC3953CFB,
+                                              0xC1D382A2,
+                                              0xC011E895,
+                                              0xCB4DAFA8,
+                                              0xCA8FC59F,
+                                              0xC8C97BC6,
+                                              0xC90B11F1,
+                                              0xCC440774,
+                                              0xCD866D43,
+                                              0xCFC0D31A,
+                                              0xCE02B92D,
+                                              0x91AF9640,
+                                              0x906DFC77,
+                                              0x922B422E,
+                                              0x93E92819,
+                                              0x96A63E9C,
+                                              0x976454AB,
+                                              0x9522EAF2,
+                                              0x94E080C5,
+                                              0x9FBCC7F8,
+                                              0x9E7EADCF,
+                                              0x9C381396,
+                                              0x9DFA79A1,
+                                              0x98B56F24,
+                                              0x99770513,
+                                              0x9B31BB4A,
+                                              0x9AF3D17D,
+                                              0x8D893530,
+                                              0x8C4B5F07,
+                                              0x8E0DE15E,
+                                              0x8FCF8B69,
+                                              0x8A809DEC,
+                                              0x8B42F7DB,
+                                              0x89044982,
+                                              0x88C623B5,
+                                              0x839A6488,
+                                              0x82580EBF,
+                                              0x801EB0E6,
+                                              0x81DCDAD1,
+                                              0x8493CC54,
+                                              0x8551A663,
+                                              0x8717183A,
+                                              0x86D5720D,
+                                              0xA9E2D0A0,
+                                              0xA820BA97,
+                                              0xAA6604CE,
+                                              0xABA46EF9,
+                                              0xAEEB787C,
+                                              0xAF29124B,
+                                              0xAD6FAC12,
+                                              0xACADC625,
+                                              0xA7F18118,
+                                              0xA633EB2F,
+                                              0xA4755576,
+                                              0xA5B73F41,
+                                              0xA0F829C4,
+                                              0xA13A43F3,
+                                              0xA37CFDAA,
+                                              0xA2BE979D,
+                                              0xB5C473D0,
+                                              0xB40619E7,
+                                              0xB640A7BE,
+                                              0xB782CD89,
+                                              0xB2CDDB0C,
+                                              0xB30FB13B,
+                                              0xB1490F62,
+                                              0xB08B6555,
+                                              0xBBD72268,
+                                              0xBA15485F,
+                                              0xB853F606,
+                                              0xB9919C31,
+                                              0xBCDE8AB4,
+                                              0xBD1CE083,
+                                              0xBF5A5EDA,
+                                              0xBE9834ED,
+                                              /* T8_3 */
+                                              0x00000000,
+                                              0xB8BC6765,
+                                              0xAA09C88B,
+                                              0x12B5AFEE,
+                                              0x8F629757,
+                                              0x37DEF032,
+                                              0x256B5FDC,
+                                              0x9DD738B9,
+                                              0xC5B428EF,
+                                              0x7D084F8A,
+                                              0x6FBDE064,
+                                              0xD7018701,
+                                              0x4AD6BFB8,
+                                              0xF26AD8DD,
+                                              0xE0DF7733,
+                                              0x58631056,
+                                              0x5019579F,
+                                              0xE8A530FA,
+                                              0xFA109F14,
+                                              0x42ACF871,
+                                              0xDF7BC0C8,
+                                              0x67C7A7AD,
+                                              0x75720843,
+                                              0xCDCE6F26,
+                                              0x95AD7F70,
+                                              0x2D111815,
+                                              0x3FA4B7FB,
+                                              0x8718D09E,
+                                              0x1ACFE827,
+                                              0xA2738F42,
+                                              0xB0C620AC,
+                                              0x087A47C9,
+                                              0xA032AF3E,
+                                              0x188EC85B,
+                                              0x0A3B67B5,
+                                              0xB28700D0,
+                                              0x2F503869,
+                                              0x97EC5F0C,
+                                              0x8559F0E2,
+                                              0x3DE59787,
+                                              0x658687D1,
+                                              0xDD3AE0B4,
+                                              0xCF8F4F5A,
+                                              0x7733283F,
+                                              0xEAE41086,
+                                              0x525877E3,
+                                              0x40EDD80D,
+                                              0xF851BF68,
+                                              0xF02BF8A1,
+                                              0x48979FC4,
+                                              0x5A22302A,
+                                              0xE29E574F,
+                                              0x7F496FF6,
+                                              0xC7F50893,
+                                              0xD540A77D,
+                                              0x6DFCC018,
+                                              0x359FD04E,
+                                              0x8D23B72B,
+                                              0x9F9618C5,
+                                              0x272A7FA0,
+                                              0xBAFD4719,
+                                              0x0241207C,
+                                              0x10F48F92,
+                                              0xA848E8F7,
+                                              0x9B14583D,
+                                              0x23A83F58,
+                                              0x311D90B6,
+                                              0x89A1F7D3,
+                                              0x1476CF6A,
+                                              0xACCAA80F,
+                                              0xBE7F07E1,
+                                              0x06C36084,
+                                              0x5EA070D2,
+                                              0xE61C17B7,
+                                              0xF4A9B859,
+                                              0x4C15DF3C,
+                                              0xD1C2E785,
+                                              0x697E80E0,
+                                              0x7BCB2F0E,
+                                              0xC377486B,
+                                              0xCB0D0FA2,
+                                              0x73B168C7,
+                                              0x6104C729,
+                                              0xD9B8A04C,
+                                              0x446F98F5,
+                                              0xFCD3FF90,
+                                              0xEE66507E,
+                                              0x56DA371B,
+                                              0x0EB9274D,
+                                              0xB6054028,
+                                              0xA4B0EFC6,
+                                              0x1C0C88A3,
+                                              0x81DBB01A,
+                                              0x3967D77F,
+                                              0x2BD27891,
+                                              0x936E1FF4,
+                                              0x3B26F703,
+                                              0x839A9066,
+                                              0x912F3F88,
+                                              0x299358ED,
+                                              0xB4446054,
+                                              0x0CF80731,
+                                              0x1E4DA8DF,
+                                              0xA6F1CFBA,
+                                              0xFE92DFEC,
+                                              0x462EB889,
+                                              0x549B1767,
+                                              0xEC277002,
+                                              0x71F048BB,
+                                              0xC94C2FDE,
+                                              0xDBF98030,
+                                              0x6345E755,
+                                              0x6B3FA09C,
+                                              0xD383C7F9,
+                                              0xC1366817,
+                                              0x798A0F72,
+                                              0xE45D37CB,
+                                              0x5CE150AE,
+                                              0x4E54FF40,
+                                              0xF6E89825,
+                                              0xAE8B8873,
+                                              0x1637EF16,
+                                              0x048240F8,
+                                              0xBC3E279D,
+                                              0x21E91F24,
+                                              0x99557841,
+                                              0x8BE0D7AF,
+                                              0x335CB0CA,
+                                              0xED59B63B,
+                                              0x55E5D15E,
+                                              0x47507EB0,
+                                              0xFFEC19D5,
+                                              0x623B216C,
+                                              0xDA874609,
+                                              0xC832E9E7,
+                                              0x708E8E82,
+                                              0x28ED9ED4,
+                                              0x9051F9B1,
+                                              0x82E4565F,
+                                              0x3A58313A,
+                                              0xA78F0983,
+                                              0x1F336EE6,
+                                              0x0D86C108,
+                                              0xB53AA66D,
+                                              0xBD40E1A4,
+                                              0x05FC86C1,
+                                              0x1749292F,
+                                              0xAFF54E4A,
+                                              0x322276F3,
+                                              0x8A9E1196,
+                                              0x982BBE78,
+                                              0x2097D91D,
+                                              0x78F4C94B,
+                                              0xC048AE2E,
+                                              0xD2FD01C0,
+                                              0x6A4166A5,
+                                              0xF7965E1C,
+                                              0x4F2A3979,
+                                              0x5D9F9697,
+                                              0xE523F1F2,
+                                              0x4D6B1905,
+                                              0xF5D77E60,
+                                              0xE762D18E,
+                                              0x5FDEB6EB,
+                                              0xC2098E52,
+                                              0x7AB5E937,
+                                              0x680046D9,
+                                              0xD0BC21BC,
+                                              0x88DF31EA,
+                                              0x3063568F,
+                                              0x22D6F961,
+                                              0x9A6A9E04,
+                                              0x07BDA6BD,
+                                              0xBF01C1D8,
+                                              0xADB46E36,
+                                              0x15080953,
+                                              0x1D724E9A,
+                                              0xA5CE29FF,
+                                              0xB77B8611,
+                                              0x0FC7E174,
+                                              0x9210D9CD,
+                                              0x2AACBEA8,
+                                              0x38191146,
+                                              0x80A57623,
+                                              0xD8C66675,
+                                              0x607A0110,
+                                              0x72CFAEFE,
+                                              0xCA73C99B,
+                                              0x57A4F122,
+                                              0xEF189647,
+                                              0xFDAD39A9,
+                                              0x45115ECC,
+                                              0x764DEE06,
+                                              0xCEF18963,
+                                              0xDC44268D,
+                                              0x64F841E8,
+                                              0xF92F7951,
+                                              0x41931E34,
+                                              0x5326B1DA,
+                                              0xEB9AD6BF,
+                                              0xB3F9C6E9,
+                                              0x0B45A18C,
+                                              0x19F00E62,
+                                              0xA14C6907,
+                                              0x3C9B51BE,
+                                              0x842736DB,
+                                              0x96929935,
+                                              0x2E2EFE50,
+                                              0x2654B999,
+                                              0x9EE8DEFC,
+                                              0x8C5D7112,
+                                              0x34E11677,
+                                              0xA9362ECE,
+                                              0x118A49AB,
+                                              0x033FE645,
+                                              0xBB838120,
+                                              0xE3E09176,
+                                              0x5B5CF613,
+                                              0x49E959FD,
+                                              0xF1553E98,
+                                              0x6C820621,
+                                              0xD43E6144,
+                                              0xC68BCEAA,
+                                              0x7E37A9CF,
+                                              0xD67F4138,
+                                              0x6EC3265D,
+                                              0x7C7689B3,
+                                              0xC4CAEED6,
+                                              0x591DD66F,
+                                              0xE1A1B10A,
+                                              0xF3141EE4,
+                                              0x4BA87981,
+                                              0x13CB69D7,
+                                              0xAB770EB2,
+                                              0xB9C2A15C,
+                                              0x017EC639,
+                                              0x9CA9FE80,
+                                              0x241599E5,
+                                              0x36A0360B,
+                                              0x8E1C516E,
+                                              0x866616A7,
+                                              0x3EDA71C2,
+                                              0x2C6FDE2C,
+                                              0x94D3B949,
+                                              0x090481F0,
+                                              0xB1B8E695,
+                                              0xA30D497B,
+                                              0x1BB12E1E,
+                                              0x43D23E48,
+                                              0xFB6E592D,
+                                              0xE9DBF6C3,
+                                              0x516791A6,
+                                              0xCCB0A91F,
+                                              0x740CCE7A,
+                                              0x66B96194,
+                                              0xDE0506F1,
+                                              /* T8_4 */
+                                              0x00000000,
+                                              0x3D6029B0,
+                                              0x7AC05360,
+                                              0x47A07AD0,
+                                              0xF580A6C0,
+                                              0xC8E08F70,
+                                              0x8F40F5A0,
+                                              0xB220DC10,
+                                              0x30704BC1,
+                                              0x0D106271,
+                                              0x4AB018A1,
+                                              0x77D03111,
+                                              0xC5F0ED01,
+                                              0xF890C4B1,
+                                              0xBF30BE61,
+                                              0x825097D1,
+                                              0x60E09782,
+                                              0x5D80BE32,
+                                              0x1A20C4E2,
+                                              0x2740ED52,
+                                              0x95603142,
+                                              0xA80018F2,
+                                              0xEFA06222,
+                                              0xD2C04B92,
+                                              0x5090DC43,
+                                              0x6DF0F5F3,
+                                              0x2A508F23,
+                                              0x1730A693,
+                                              0xA5107A83,
+                                              0x98705333,
+                                              0xDFD029E3,
+                                              0xE2B00053,
+                                              0xC1C12F04,
+                                              0xFCA106B4,
+                                              0xBB017C64,
+                                              0x866155D4,
+                                              0x344189C4,
+                                              0x0921A074,
+                                              0x4E81DAA4,
+                                              0x73E1F314,
+                                              0xF1B164C5,
+                                              0xCCD14D75,
+                                              0x8B7137A5,
+                                              0xB6111E15,
+                                              0x0431C205,
+                                              0x3951EBB5,
+                                              0x7EF19165,
+                                              0x4391B8D5,
+                                              0xA121B886,
+                                              0x9C419136,
+                                              0xDBE1EBE6,
+                                              0xE681C256,
+                                              0x54A11E46,
+                                              0x69C137F6,
+                                              0x2E614D26,
+                                              0x13016496,
+                                              0x9151F347,
+                                              0xAC31DAF7,
+                                              0xEB91A027,
+                                              0xD6F18997,
+                                              0x64D15587,
+                                              0x59B17C37,
+                                              0x1E1106E7,
+                                              0x23712F57,
+                                              0x58F35849,
+                                              0x659371F9,
+                                              0x22330B29,
+                                              0x1F532299,
+                                              0xAD73FE89,
+                                              0x9013D739,
+                                              0xD7B3ADE9,
+                                              0xEAD38459,
+                                              0x68831388,
+                                              0x55E33A38,
+                                              0x124340E8,
+                                              0x2F236958,
+                                              0x9D03B548,
+                                              0xA0639CF8,
+                                              0xE7C3E628,
+                                              0xDAA3CF98,
+                                              0x3813CFCB,
+                                              0x0573E67B,
+                                              0x42D39CAB,
+                                              0x7FB3B51B,
+                                              0xCD93690B,
+                                              0xF0F340BB,
+                                              0xB7533A6B,
+                                              0x8A3313DB,
+                                              0x0863840A,
+                                              0x3503ADBA,
+                                              0x72A3D76A,
+                                              0x4FC3FEDA,
+                                              0xFDE322CA,
+                                              0xC0830B7A,
+                                              0x872371AA,
+                                              0xBA43581A,
+                                              0x9932774D,
+                                              0xA4525EFD,
+                                              0xE3F2242D,
+                                              0xDE920D9D,
+                                              0x6CB2D18D,
+                                              0x51D2F83D,
+                                              0x167282ED,
+                                              0x2B12AB5D,
+                                              0xA9423C8C,
+                                              0x9422153C,
+                                              0xD3826FEC,
+                                              0xEEE2465C,
+                                              0x5CC29A4C,
+                                              0x61A2B3FC,
+                                              0x2602C92C,
+                                              0x1B62E09C,
+                                              0xF9D2E0CF,
+                                              0xC4B2C97F,
+                                              0x8312B3AF,
+                                              0xBE729A1F,
+                                              0x0C52460F,
+                                              0x31326FBF,
+                                              0x7692156F,
+                                              0x4BF23CDF,
+                                              0xC9A2AB0E,
+                                              0xF4C282BE,
+                                              0xB362F86E,
+                                              0x8E02D1DE,
+                                              0x3C220DCE,
+                                              0x0142247E,
+                                              0x46E25EAE,
+                                              0x7B82771E,
+                                              0xB1E6B092,
+                                              0x8C869922,
+                                              0xCB26E3F2,
+                                              0xF646CA42,
+                                              0x44661652,
+                                              0x79063FE2,
+                                              0x3EA64532,
+                                              0x03C66C82,
+                                              0x8196FB53,
+                                              0xBCF6D2E3,
+                                              0xFB56A833,
+                                              0xC6368183,
+                                              0x74165D93,
+                                              0x49767423,
+                                              0x0ED60EF3,
+                                              0x33B62743,
+                                              0xD1062710,
+                                              0xEC660EA0,
+                                              0xABC67470,
+                                              0x96A65DC0,
+                                              0x248681D0,
+                                              0x19E6A860,
+                                              0x5E46D2B0,
+                                              0x6326FB00,
+                                              0xE1766CD1,
+                                              0xDC164561,
+                                              0x9BB63FB1,
+                                              0xA6D61601,
+                                              0x14F6CA11,
+                                              0x2996E3A1,
+                                              0x6E369971,
+                                              0x5356B0C1,
+                                              0x70279F96,
+                                              0x4D47B626,
+                                              0x0AE7CCF6,
+                                              0x3787E546,
+                                              0x85A73956,
+                                              0xB8C710E6,
+                                              0xFF676A36,
+                                              0xC2074386,
+                                              0x4057D457,
+                                              0x7D37FDE7,
+                                              0x3A978737,
+                                              0x07F7AE87,
+                                              0xB5D77297,
+                                              0x88B75B27,
+                                              0xCF1721F7,
+                                              0xF2770847,
+                                              0x10C70814,
+                                              0x2DA721A4,
+                                              0x6A075B74,
+                                              0x576772C4,
+                                              0xE547AED4,
+                                              0xD8278764,
+                                              0x9F87FDB4,
+                                              0xA2E7D404,
+                                              0x20B743D5,
+                                              0x1DD76A65,
+                                              0x5A7710B5,
+                                              0x67173905,
+                                              0xD537E515,
+                                              0xE857CCA5,
+                                              0xAFF7B675,
+                                              0x92979FC5,
+                                              0xE915E8DB,
+                                              0xD475C16B,
+                                              0x93D5BBBB,
+                                              0xAEB5920B,
+                                              0x1C954E1B,
+                                              0x21F567AB,
+                                              0x66551D7B,
+                                              0x5B3534CB,
+                                              0xD965A31A,
+                                              0xE4058AAA,
+                                              0xA3A5F07A,
+                                              0x9EC5D9CA,
+                                              0x2CE505DA,
+                                              0x11852C6A,
+                                              0x562556BA,
+                                              0x6B457F0A,
+                                              0x89F57F59,
+                                              0xB49556E9,
+                                              0xF3352C39,
+                                              0xCE550589,
+                                              0x7C75D999,
+                                              0x4115F029,
+                                              0x06B58AF9,
+                                              0x3BD5A349,
+                                              0xB9853498,
+                                              0x84E51D28,
+                                              0xC34567F8,
+                                              0xFE254E48,
+                                              0x4C059258,
+                                              0x7165BBE8,
+                                              0x36C5C138,
+                                              0x0BA5E888,
+                                              0x28D4C7DF,
+                                              0x15B4EE6F,
+                                              0x521494BF,
+                                              0x6F74BD0F,
+                                              0xDD54611F,
+                                              0xE03448AF,
+                                              0xA794327F,
+                                              0x9AF41BCF,
+                                              0x18A48C1E,
+                                              0x25C4A5AE,
+                                              0x6264DF7E,
+                                              0x5F04F6CE,
+                                              0xED242ADE,
+                                              0xD044036E,
+                                              0x97E479BE,
+                                              0xAA84500E,
+                                              0x4834505D,
+                                              0x755479ED,
+                                              0x32F4033D,
+                                              0x0F942A8D,
+                                              0xBDB4F69D,
+                                              0x80D4DF2D,
+                                              0xC774A5FD,
+                                              0xFA148C4D,
+                                              0x78441B9C,
+                                              0x4524322C,
+                                              0x028448FC,
+                                              0x3FE4614C,
+                                              0x8DC4BD5C,
+                                              0xB0A494EC,
+                                              0xF704EE3C,
+                                              0xCA64C78C,
+                                              /* T8_5 */
+                                              0x00000000,
+                                              0xCB5CD3A5,
+                                              0x4DC8A10B,
+                                              0x869472AE,
+                                              0x9B914216,
+                                              0x50CD91B3,
+                                              0xD659E31D,
+                                              0x1D0530B8,
+                                              0xEC53826D,
+                                              0x270F51C8,
+                                              0xA19B2366,
+                                              0x6AC7F0C3,
+                                              0x77C2C07B,
+                                              0xBC9E13DE,
+                                              0x3A0A6170,
+                                              0xF156B2D5,
+                                              0x03D6029B,
+                                              0xC88AD13E,
+                                              0x4E1EA390,
+                                              0x85427035,
+                                              0x9847408D,
+                                              0x531B9328,
+                                              0xD58FE186,
+                                              0x1ED33223,
+                                              0xEF8580F6,
+                                              0x24D95353,
+                                              0xA24D21FD,
+                                              0x6911F258,
+                                              0x7414C2E0,
+                                              0xBF481145,
+                                              0x39DC63EB,
+                                              0xF280B04E,
+                                              0x07AC0536,
+                                              0xCCF0D693,
+                                              0x4A64A43D,
+                                              0x81387798,
+                                              0x9C3D4720,
+                                              0x57619485,
+                                              0xD1F5E62B,
+                                              0x1AA9358E,
+                                              0xEBFF875B,
+                                              0x20A354FE,
+                                              0xA6372650,
+                                              0x6D6BF5F5,
+                                              0x706EC54D,
+                                              0xBB3216E8,
+                                              0x3DA66446,
+                                              0xF6FAB7E3,
+                                              0x047A07AD,
+                                              0xCF26D408,
+                                              0x49B2A6A6,
+                                              0x82EE7503,
+                                              0x9FEB45BB,
+                                              0x54B7961E,
+                                              0xD223E4B0,
+                                              0x197F3715,
+                                              0xE82985C0,
+                                              0x23755665,
+                                              0xA5E124CB,
+                                              0x6EBDF76E,
+                                              0x73B8C7D6,
+                                              0xB8E41473,
+                                              0x3E7066DD,
+                                              0xF52CB578,
+                                              0x0F580A6C,
+                                              0xC404D9C9,
+                                              0x4290AB67,
+                                              0x89CC78C2,
+                                              0x94C9487A,
+                                              0x5F959BDF,
+                                              0xD901E971,
+                                              0x125D3AD4,
+                                              0xE30B8801,
+                                              0x28575BA4,
+                                              0xAEC3290A,
+                                              0x659FFAAF,
+                                              0x789ACA17,
+                                              0xB3C619B2,
+                                              0x35526B1C,
+                                              0xFE0EB8B9,
+                                              0x0C8E08F7,
+                                              0xC7D2DB52,
+                                              0x4146A9FC,
+                                              0x8A1A7A59,
+                                              0x971F4AE1,
+                                              0x5C439944,
+                                              0xDAD7EBEA,
+                                              0x118B384F,
+                                              0xE0DD8A9A,
+                                              0x2B81593F,
+                                              0xAD152B91,
+                                              0x6649F834,
+                                              0x7B4CC88C,
+                                              0xB0101B29,
+                                              0x36846987,
+                                              0xFDD8BA22,
+                                              0x08F40F5A,
+                                              0xC3A8DCFF,
+                                              0x453CAE51,
+                                              0x8E607DF4,
+                                              0x93654D4C,
+                                              0x58399EE9,
+                                              0xDEADEC47,
+                                              0x15F13FE2,
+                                              0xE4A78D37,
+                                              0x2FFB5E92,
+                                              0xA96F2C3C,
+                                              0x6233FF99,
+                                              0x7F36CF21,
+                                              0xB46A1C84,
+                                              0x32FE6E2A,
+                                              0xF9A2BD8F,
+                                              0x0B220DC1,
+                                              0xC07EDE64,
+                                              0x46EAACCA,
+                                              0x8DB67F6F,
+                                              0x90B34FD7,
+                                              0x5BEF9C72,
+                                              0xDD7BEEDC,
+                                              0x16273D79,
+                                              0xE7718FAC,
+                                              0x2C2D5C09,
+                                              0xAAB92EA7,
+                                              0x61E5FD02,
+                                              0x7CE0CDBA,
+                                              0xB7BC1E1F,
+                                              0x31286CB1,
+                                              0xFA74BF14,
+                                              0x1EB014D8,
+                                              0xD5ECC77D,
+                                              0x5378B5D3,
+                                              0x98246676,
+                                              0x852156CE,
+                                              0x4E7D856B,
+                                              0xC8E9F7C5,
+                                              0x03B52460,
+                                              0xF2E396B5,
+                                              0x39BF4510,
+                                              0xBF2B37BE,
+                                              0x7477E41B,
+                                              0x6972D4A3,
+                                              0xA22E0706,
+                                              0x24BA75A8,
+                                              0xEFE6A60D,
+                                              0x1D661643,
+                                              0xD63AC5E6,
+                                              0x50AEB748,
+                                              0x9BF264ED,
+                                              0x86F75455,
+                                              0x4DAB87F0,
+ 

<TRUNCATED>

[5/7] KAFKA-1227 New producer!

Posted by jk...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/Metrics.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/Metrics.java b/clients/src/main/java/kafka/common/metrics/Metrics.java
new file mode 100644
index 0000000..f2cb782
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/Metrics.java
@@ -0,0 +1,190 @@
+package kafka.common.metrics;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.ConcurrentMap;
+
+import kafka.common.utils.SystemTime;
+import kafka.common.utils.Time;
+import kafka.common.utils.Utils;
+
+/**
+ * A registry of sensors and metrics.
+ * <p>
+ * A metric is a named, numerical measurement. A sensor is a handle to record numerical measurements as they occur. Each
+ * Sensor has zero or more associated metrics. For example a Sensor might represent message sizes and we might associate
+ * with this sensor a metric for the average, maximum, or other statistics computed off the sequence of message sizes
+ * that are recorded by the sensor.
+ * <p>
+ * Usage looks something like this:
+ * 
+ * <pre>
+ * // set up metrics:
+ * Metrics metrics = new Metrics(); // this is the global repository of metrics and sensors
+ * Sensor sensor = metrics.sensor(&quot;message-sizes&quot;);
+ * sensor.add(&quot;kafka.producer.message-sizes.avg&quot;, new Avg());
+ * sensor.add(&quot;kafka.producer.message-sizes.max&quot;, new Max());
+ * 
+ * // as messages are sent we record the sizes
+ * sensor.record(messageSize);
+ * </pre>
+ */
+public class Metrics {
+
+    private final MetricConfig config;
+    private final ConcurrentMap<String, KafkaMetric> metrics;
+    private final ConcurrentMap<String, Sensor> sensors;
+    private final List<MetricsReporter> reporters;
+    private final Time time;
+
+    /**
+     * Create a metrics repository with no metric reporters and default configuration.
+     */
+    public Metrics() {
+        this(new MetricConfig());
+    }
+
+    /**
+     * Create a metrics repository with no metric reporters and default configuration.
+     */
+    public Metrics(Time time) {
+        this(new MetricConfig(), new ArrayList<MetricsReporter>(), time);
+    }
+
+    /**
+     * Create a metrics repository with no reporters and the given default config. This config will be used for any
+     * metric that doesn't override its own config.
+     * @param defaultConfig The default config to use for all metrics that don't override their config
+     */
+    public Metrics(MetricConfig defaultConfig) {
+        this(defaultConfig, new ArrayList<MetricsReporter>(0), new SystemTime());
+    }
+
+    /**
+     * Create a metrics repository with a default config and the given metric reporters
+     * @param defaultConfig The default config
+     * @param reporters The metrics reporters
+     * @param time The time instance to use with the metrics
+     */
+    public Metrics(MetricConfig defaultConfig, List<MetricsReporter> reporters, Time time) {
+        this.config = defaultConfig;
+        this.sensors = new ConcurrentHashMap<String, Sensor>();
+        this.metrics = new ConcurrentHashMap<String, KafkaMetric>();
+        this.reporters = Utils.notNull(reporters);
+        this.time = time;
+        for (MetricsReporter reporter : reporters)
+            reporter.init(new ArrayList<KafkaMetric>());
+    }
+
+    /**
+     * Create a sensor with the given unique name and zero or more parent sensors. All parent sensors will receive every
+     * value recorded with this sensor.
+     * @param name The name of the sensor
+     * @param parents The parent sensors
+     * @return The sensor that is created
+     */
+    public Sensor sensor(String name, Sensor... parents) {
+        return sensor(name, null, parents);
+    }
+
+    /**
+     * Create a sensor with the given unique name and zero or more parent sensors. All parent sensors will receive every
+     * value recorded with this sensor.
+     * @param name The name of the sensor
+     * @param config A default configuration to use for this sensor for metrics that don't have their own config
+     * @param parents The parent sensors
+     * @return The sensor that is created
+     */
+    public synchronized Sensor sensor(String name, MetricConfig config, Sensor... parents) {
+        Sensor s = this.sensors.get(Utils.notNull(name));
+        if (s == null) {
+            s = new Sensor(this, name, parents, config == null ? this.config : config, time);
+            this.sensors.put(name, s);
+        }
+        return s;
+    }
+
+    /**
+     * Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor.
+     * This is a way to expose existing values as metrics.
+     * @param name The name of the metric
+     * @param measurable The measurable that will be measured by this metric
+     */
+    public void addMetric(String name, Measurable measurable) {
+        addMetric(name, "", measurable);
+    }
+
+    /**
+     * Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor.
+     * This is a way to expose existing values as metrics.
+     * @param name The name of the metric
+     * @param description A human-readable description to include in the metric
+     * @param measurable The measurable that will be measured by this metric
+     */
+    public void addMetric(String name, String description, Measurable measurable) {
+        addMetric(name, description, null, measurable);
+    }
+
+    /**
+     * Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor.
+     * This is a way to expose existing values as metrics.
+     * @param name The name of the metric
+     * @param config The configuration to use when measuring this measurable
+     * @param measurable The measurable that will be measured by this metric
+     */
+    public void addMetric(String name, MetricConfig config, Measurable measurable) {
+        addMetric(name, "", config, measurable);
+    }
+
+    /**
+     * Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor.
+     * This is a way to expose existing values as metrics.
+     * @param name The name of the metric
+     * @param description A human-readable description to include in the metric
+     * @param config The configuration to use when measuring this measurable
+     * @param measurable The measurable that will be measured by this metric
+     */
+    public synchronized void addMetric(String name, String description, MetricConfig config, Measurable measurable) {
+        KafkaMetric m = new KafkaMetric(new Object(),
+                                        Utils.notNull(name),
+                                        Utils.notNull(description),
+                                        Utils.notNull(measurable),
+                                        config == null ? this.config : config,
+                                        time);
+        registerMetric(m);
+    }
+
+    /**
+     * Add a MetricReporter
+     */
+    public synchronized void addReporter(MetricsReporter reporter) {
+        Utils.notNull(reporter).init(new ArrayList<KafkaMetric>(metrics.values()));
+        this.reporters.add(reporter);
+    }
+
+    synchronized void registerMetric(KafkaMetric metric) {
+        if (this.metrics.containsKey(metric.name()))
+            throw new IllegalArgumentException("A metric named '" + metric.name() + "' already exists, can't register another one.");
+        this.metrics.put(metric.name(), metric);
+        for (MetricsReporter reporter : reporters)
+            reporter.metricChange(metric);
+    }
+
+    /**
+     * Get all the metrics currently maintained indexed by metric name
+     */
+    public Map<String, KafkaMetric> metrics() {
+        return this.metrics;
+    }
+
+    /**
+     * Close this metrics repository.
+     */
+    public void close() {
+        for (MetricsReporter reporter : this.reporters)
+            reporter.close();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/MetricsReporter.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/MetricsReporter.java b/clients/src/main/java/kafka/common/metrics/MetricsReporter.java
new file mode 100644
index 0000000..bf0b39e
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/MetricsReporter.java
@@ -0,0 +1,27 @@
+package kafka.common.metrics;
+
+import java.util.List;
+
+/**
+ * A plugin interface to allow things to listen as new metrics are created so they can be reported
+ */
+public interface MetricsReporter {
+
+    /**
+     * This is called when the reporter is first registered to initially register all existing metrics
+     * @param metrics All currently existing metrics
+     */
+    public void init(List<KafkaMetric> metrics);
+
+    /**
+     * This is called whenever a metric is updated or added
+     * @param metric
+     */
+    public void metricChange(KafkaMetric metric);
+
+    /**
+     * Called when the metrics repository is closed.
+     */
+    public void close();
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/Quota.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/Quota.java b/clients/src/main/java/kafka/common/metrics/Quota.java
new file mode 100644
index 0000000..6278246
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/Quota.java
@@ -0,0 +1,36 @@
+package kafka.common.metrics;
+
+/**
+ * An upper or lower bound for metrics
+ */
+public final class Quota {
+
+    private final boolean upper;
+    private final double bound;
+
+    public Quota(double bound, boolean upper) {
+        this.bound = bound;
+        this.upper = upper;
+    }
+
+    public static Quota lessThan(double upperBound) {
+        return new Quota(upperBound, true);
+    }
+
+    public static Quota moreThan(double lowerBound) {
+        return new Quota(lowerBound, false);
+    }
+
+    public boolean isUpperBound() {
+        return this.upper;
+    }
+
+    public double bound() {
+        return this.bound;
+    }
+
+    public boolean acceptable(double value) {
+        return (upper && value <= bound) || (!upper && value >= bound);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/QuotaViolationException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/QuotaViolationException.java b/clients/src/main/java/kafka/common/metrics/QuotaViolationException.java
new file mode 100644
index 0000000..b9005cd
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/QuotaViolationException.java
@@ -0,0 +1,16 @@
+package kafka.common.metrics;
+
+import kafka.common.KafkaException;
+
+/**
+ * Thrown when a sensor records a value that causes a metric to go outside the bounds configured as its quota
+ */
+public class QuotaViolationException extends KafkaException {
+
+    private static final long serialVersionUID = 1L;
+
+    public QuotaViolationException(String m) {
+        super(m);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/Sensor.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/Sensor.java b/clients/src/main/java/kafka/common/metrics/Sensor.java
new file mode 100644
index 0000000..9c11835
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/Sensor.java
@@ -0,0 +1,171 @@
+package kafka.common.metrics;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import kafka.common.metrics.CompoundStat.NamedMeasurable;
+import kafka.common.utils.Time;
+import kafka.common.utils.Utils;
+
+/**
+ * A sensor applies a continuous sequence of numerical values to a set of associated metrics. For example a sensor on
+ * message size would record a sequence of message sizes using the {@link #record(double)} api and would maintain a set
+ * of metrics about request sizes such as the average or max.
+ */
+public final class Sensor {
+
+    private final Metrics registry;
+    private final String name;
+    private final Sensor[] parents;
+    private final List<Stat> stats;
+    private final List<KafkaMetric> metrics;
+    private final MetricConfig config;
+    private final Time time;
+
+    Sensor(Metrics registry, String name, Sensor[] parents, MetricConfig config, Time time) {
+        super();
+        this.registry = registry;
+        this.name = Utils.notNull(name);
+        this.parents = parents;
+        this.metrics = new ArrayList<KafkaMetric>();
+        this.stats = new ArrayList<Stat>();
+        this.config = config;
+        this.time = time;
+        checkForest(new HashSet<Sensor>());
+    }
+
+    /* Validate that this sensor doesn't end up referencing itself */
+    private void checkForest(Set<Sensor> sensors) {
+        if (!sensors.add(this))
+            throw new IllegalArgumentException("Circular dependency in sensors: " + name() + " is its own parent.");
+        for (int i = 0; i < parents.length; i++)
+            parents[i].checkForest(sensors);
+    }
+
+    /**
+     * The name this sensor is registered with. This name will be unique among all registered sensors.
+     */
+    public String name() {
+        return this.name;
+    }
+
+    /**
+     * Record an occurrence, this is just short-hand for {@link #record(double) record(1.0)}
+     */
+    public void record() {
+        record(1.0);
+    }
+
+    /**
+     * Record a value with this sensor
+     * @param value The value to record
+     * @throws QuotaViolationException if recording this value moves a metric beyond its configured maximum or minimum
+     *         bound
+     */
+    public void record(double value) {
+        record(value, time.nanoseconds());
+    }
+
+    private void record(double value, long time) {
+        synchronized (this) {
+            // increment all the stats
+            for (int i = 0; i < this.stats.size(); i++)
+                this.stats.get(i).record(config, value, time);
+            checkQuotas(time);
+
+        }
+        for (int i = 0; i < parents.length; i++)
+            parents[i].record(value, time);
+    }
+
+    private void checkQuotas(long time) {
+        for (int i = 0; i < this.metrics.size(); i++) {
+            KafkaMetric metric = this.metrics.get(i);
+            MetricConfig config = metric.config();
+            if (config != null) {
+                Quota quota = config.quota();
+                if (quota != null)
+                    if (!quota.acceptable(metric.value(time)))
+                        throw new QuotaViolationException("Metric " + metric.name() + " is in violation of its quota of " + quota.bound());
+            }
+        }
+    }
+
+    /**
+     * Register a compound statistic with this sensor with no config override
+     */
+    public void add(CompoundStat stat) {
+        add(stat, null);
+    }
+
+    /**
+     * Register a compound statistic with this sensor which yields multiple measurable quantities (like a histogram)
+     * @param stat The stat to register
+     * @param config The configuration for this stat. If null then the stat will use the default configuration for this
+     *        sensor.
+     */
+    public synchronized void add(CompoundStat stat, MetricConfig config) {
+        this.stats.add(Utils.notNull(stat));
+        for (NamedMeasurable m : stat.stats()) {
+            KafkaMetric metric = new KafkaMetric(this, m.name(), m.description(), m.stat(), config == null ? this.config : config, time);
+            this.registry.registerMetric(metric);
+            this.metrics.add(metric);
+        }
+    }
+
+    /**
+     * Add a metric with default configuration and no description. Equivalent to
+     * {@link Sensor#add(String, String, MeasurableStat, MetricConfig) add(name, "", stat, null)}
+     * 
+     */
+    public void add(String name, MeasurableStat stat) {
+        add(name, stat, null);
+    }
+
+    /**
+     * Add a metric with default configuration. Equivalent to
+     * {@link Sensor#add(String, String, MeasurableStat, MetricConfig) add(name, description, stat, null)}
+     * 
+     */
+    public void add(String name, String description, MeasurableStat stat) {
+        add(name, description, stat, null);
+    }
+
+    /**
+     * Add a metric to this sensor with no description. Equivalent to
+     * {@link Sensor#add(String, String, MeasurableStat, MetricConfig) add(name, "", stat, config)}
+     * @param name
+     * @param stat
+     * @param config
+     */
+    public void add(String name, MeasurableStat stat, MetricConfig config) {
+        add(name, "", stat, config);
+    }
+
+    /**
+     * Register a metric with this sensor
+     * @param name The name of the metric
+     * @param description A description used when reporting the value
+     * @param stat The statistic to keep
+     * @param config A special configuration for this metric. If null use the sensor default configuration.
+     */
+    public synchronized void add(String name, String description, MeasurableStat stat, MetricConfig config) {
+        KafkaMetric metric = new KafkaMetric(this,
+                                             Utils.notNull(name),
+                                             Utils.notNull(description),
+                                             Utils.notNull(stat),
+                                             config == null ? this.config : config,
+                                             time);
+        this.registry.registerMetric(metric);
+        this.metrics.add(metric);
+        this.stats.add(stat);
+    }
+
+    synchronized List<KafkaMetric> metrics() {
+        return Collections.unmodifiableList(this.metrics);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/Stat.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/Stat.java b/clients/src/main/java/kafka/common/metrics/Stat.java
new file mode 100644
index 0000000..8844545
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/Stat.java
@@ -0,0 +1,16 @@
+package kafka.common.metrics;
+
+/**
+ * A Stat is a quanity such as average, max, etc that is computed off the stream of updates to a sensor
+ */
+public interface Stat {
+
+    /**
+     * Record the given value
+     * @param config The configuration to use for this metric
+     * @param value The value to record
+     * @param time The time this value occurred
+     */
+    public void record(MetricConfig config, double value, long time);
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/stats/Avg.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/stats/Avg.java b/clients/src/main/java/kafka/common/metrics/stats/Avg.java
new file mode 100644
index 0000000..b9d3d5d
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/stats/Avg.java
@@ -0,0 +1,33 @@
+package kafka.common.metrics.stats;
+
+import java.util.List;
+
+import kafka.common.metrics.MetricConfig;
+
+/**
+ * A {@link SampledStat} that maintains a simple average over its samples.
+ */
+public class Avg extends SampledStat {
+
+    public Avg() {
+        super(0.0);
+    }
+
+    @Override
+    protected void update(Sample sample, MetricConfig config, double value, long now) {
+        sample.value += value;
+    }
+
+    @Override
+    public double combine(List<Sample> samples, MetricConfig config, long now) {
+        double total = 0.0;
+        long count = 0;
+        for (int i = 0; i < samples.size(); i++) {
+            Sample s = samples.get(i);
+            total += s.value;
+            count += s.eventCount;
+        }
+        return total / count;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/stats/Count.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/stats/Count.java b/clients/src/main/java/kafka/common/metrics/stats/Count.java
new file mode 100644
index 0000000..3712e78
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/stats/Count.java
@@ -0,0 +1,29 @@
+package kafka.common.metrics.stats;
+
+import java.util.List;
+
+import kafka.common.metrics.MetricConfig;
+
+/**
+ * A {@link SampledStat} that maintains a simple count of what it has seen.
+ */
+public class Count extends SampledStat {
+
+    public Count() {
+        super(0);
+    }
+
+    @Override
+    protected void update(Sample sample, MetricConfig config, double value, long now) {
+        sample.value += 1.0;
+    }
+
+    @Override
+    public double combine(List<Sample> samples, MetricConfig config, long now) {
+        double total = 0.0;
+        for (int i = 0; i < samples.size(); i++)
+            total += samples.get(i).value;
+        return total;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/stats/Histogram.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/stats/Histogram.java b/clients/src/main/java/kafka/common/metrics/stats/Histogram.java
new file mode 100644
index 0000000..c59b585
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/stats/Histogram.java
@@ -0,0 +1,137 @@
+package kafka.common.metrics.stats;
+
+public class Histogram {
+
+    private final BinScheme binScheme;
+    private final float[] hist;
+    private double count;
+
+    public Histogram(BinScheme binScheme) {
+        this.hist = new float[binScheme.bins()];
+        this.count = 0.0f;
+        this.binScheme = binScheme;
+    }
+
+    public void record(double value) {
+        this.hist[binScheme.toBin(value)] += 1.0f;
+        this.count += 1.0f;
+    }
+
+    public double value(double quantile) {
+        if (count == 0L)
+            return Double.NaN;
+        float sum = 0.0f;
+        float quant = (float) quantile;
+        for (int i = 0; i < this.hist.length - 1; i++) {
+            sum += this.hist[i];
+            if (sum / count > quant)
+                return binScheme.fromBin(i);
+        }
+        return Float.POSITIVE_INFINITY;
+    }
+
+    public void clear() {
+        for (int i = 0; i < this.hist.length; i++)
+            this.hist[i] = 0.0f;
+        this.count = 0;
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder b = new StringBuilder('{');
+        for (int i = 0; i < this.hist.length - 1; i++) {
+            b.append(String.format("%.10f", binScheme.fromBin(i)));
+            b.append(':');
+            b.append(String.format("%.0f", this.hist[i]));
+            b.append(',');
+        }
+        b.append(Float.POSITIVE_INFINITY);
+        b.append(':');
+        b.append(this.hist[this.hist.length - 1]);
+        b.append('}');
+        return b.toString();
+    }
+
+    public interface BinScheme {
+        public int bins();
+
+        public int toBin(double value);
+
+        public double fromBin(int bin);
+    }
+
+    public static class ConstantBinScheme implements BinScheme {
+        private final double min;
+        private final double max;
+        private final int bins;
+        private final double bucketWidth;
+
+        public ConstantBinScheme(int bins, double min, double max) {
+            if (bins < 2)
+                throw new IllegalArgumentException("Must have at least 2 bins.");
+            this.min = min;
+            this.max = max;
+            this.bins = bins;
+            this.bucketWidth = (max - min) / (bins - 2);
+        }
+
+        public int bins() {
+            return this.bins;
+        }
+
+        public double fromBin(int b) {
+            if (b == 0)
+                return Double.NEGATIVE_INFINITY;
+            else if (b == bins - 1)
+                return Double.POSITIVE_INFINITY;
+            else
+                return min + (b - 1) * bucketWidth;
+        }
+
+        public int toBin(double x) {
+            if (x < min)
+                return 0;
+            else if (x > max)
+                return bins - 1;
+            else
+                return (int) ((x - min) / bucketWidth) + 1;
+        }
+    }
+
+    public static class LinearBinScheme implements BinScheme {
+        private final int bins;
+        private final double max;
+        private final double scale;
+
+        public LinearBinScheme(int numBins, double max) {
+            this.bins = numBins;
+            this.max = max;
+            this.scale = max / (numBins * (numBins - 1) / 2);
+        }
+
+        public int bins() {
+            return this.bins;
+        }
+
+        public double fromBin(int b) {
+            if (b == this.bins - 1) {
+                return Float.POSITIVE_INFINITY;
+            } else {
+                double unscaled = (b * (b - 1.0)) / 2.0;
+                return unscaled * this.scale;
+            }
+        }
+
+        public int toBin(double x) {
+            if (x < 0.0d) {
+                throw new IllegalArgumentException("Values less than 0.0 not accepted.");
+            } else if (x > this.max) {
+                return this.bins - 1;
+            } else {
+                double scaled = x / this.scale;
+                return (int) (-0.5 + Math.sqrt(2.0 * scaled + 0.25));
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/stats/Max.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/stats/Max.java b/clients/src/main/java/kafka/common/metrics/stats/Max.java
new file mode 100644
index 0000000..e7bd1d2
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/stats/Max.java
@@ -0,0 +1,29 @@
+package kafka.common.metrics.stats;
+
+import java.util.List;
+
+import kafka.common.metrics.MetricConfig;
+
+/**
+ * A {@link SampledStat} that gives the max over its samples.
+ */
+public final class Max extends SampledStat {
+
+    public Max() {
+        super(Double.NEGATIVE_INFINITY);
+    }
+
+    @Override
+    protected void update(Sample sample, MetricConfig config, double value, long now) {
+        sample.value = Math.max(sample.value, value);
+    }
+
+    @Override
+    public double combine(List<Sample> samples, MetricConfig config, long now) {
+        double max = Double.NEGATIVE_INFINITY;
+        for (int i = 0; i < samples.size(); i++)
+            max = Math.max(max, samples.get(i).value);
+        return max;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/stats/Min.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/stats/Min.java b/clients/src/main/java/kafka/common/metrics/stats/Min.java
new file mode 100644
index 0000000..db0ab92
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/stats/Min.java
@@ -0,0 +1,29 @@
+package kafka.common.metrics.stats;
+
+import java.util.List;
+
+import kafka.common.metrics.MetricConfig;
+
+/**
+ * A {@link SampledStat} that gives the min over its samples.
+ */
+public class Min extends SampledStat {
+
+    public Min() {
+        super(Double.MIN_VALUE);
+    }
+
+    @Override
+    protected void update(Sample sample, MetricConfig config, double value, long now) {
+        sample.value = Math.min(sample.value, value);
+    }
+
+    @Override
+    public double combine(List<Sample> samples, MetricConfig config, long now) {
+        double max = Double.MAX_VALUE;
+        for (int i = 0; i < samples.size(); i++)
+            max = Math.min(max, samples.get(i).value);
+        return max;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/stats/Percentile.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/stats/Percentile.java b/clients/src/main/java/kafka/common/metrics/stats/Percentile.java
new file mode 100644
index 0000000..84320bb
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/stats/Percentile.java
@@ -0,0 +1,32 @@
+package kafka.common.metrics.stats;
+
+public class Percentile {
+
+    private final String name;
+    private final String description;
+    private final double percentile;
+
+    public Percentile(String name, double percentile) {
+        this(name, "", percentile);
+    }
+
+    public Percentile(String name, String description, double percentile) {
+        super();
+        this.name = name;
+        this.description = description;
+        this.percentile = percentile;
+    }
+
+    public String name() {
+        return this.name;
+    }
+
+    public String description() {
+        return this.description;
+    }
+
+    public double percentile() {
+        return this.percentile;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/stats/Percentiles.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/stats/Percentiles.java b/clients/src/main/java/kafka/common/metrics/stats/Percentiles.java
new file mode 100644
index 0000000..686c726
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/stats/Percentiles.java
@@ -0,0 +1,76 @@
+package kafka.common.metrics.stats;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import kafka.common.metrics.CompoundStat;
+import kafka.common.metrics.Measurable;
+import kafka.common.metrics.MetricConfig;
+import kafka.common.metrics.stats.Histogram.BinScheme;
+import kafka.common.metrics.stats.Histogram.ConstantBinScheme;
+import kafka.common.metrics.stats.Histogram.LinearBinScheme;
+
+/**
+ * A compound stat that reports one or more percentiles
+ */
+public class Percentiles implements CompoundStat {
+
+    public static enum BucketSizing {
+        CONSTANT, LINEAR
+    }
+
+    private final Percentile[] percentiles;
+    private Histogram current;
+    private Histogram shadow;
+    private long lastWindow;
+    private long eventCount;
+
+    public Percentiles(int sizeInBytes, double max, BucketSizing bucketing, Percentile... percentiles) {
+        this(sizeInBytes, 0.0, max, bucketing, percentiles);
+    }
+
+    public Percentiles(int sizeInBytes, double min, double max, BucketSizing bucketing, Percentile... percentiles) {
+        this.percentiles = percentiles;
+        BinScheme scheme = null;
+        if (bucketing == BucketSizing.CONSTANT) {
+            scheme = new ConstantBinScheme(sizeInBytes / 4, min, max);
+        } else if (bucketing == BucketSizing.LINEAR) {
+            if (min != 0.0d)
+                throw new IllegalArgumentException("Linear bucket sizing requires min to be 0.0.");
+            scheme = new LinearBinScheme(sizeInBytes / 4, max);
+        }
+        this.current = new Histogram(scheme);
+        this.shadow = new Histogram(scheme);
+        this.eventCount = 0L;
+    }
+
+    @Override
+    public List<NamedMeasurable> stats() {
+        List<NamedMeasurable> ms = new ArrayList<NamedMeasurable>(this.percentiles.length);
+        for (Percentile percentile : this.percentiles) {
+            final double pct = percentile.percentile();
+            ms.add(new NamedMeasurable(percentile.name(), percentile.description(), new Measurable() {
+                public double measure(MetricConfig config, long now) {
+                    return current.value(pct / 100.0);
+                }
+            }));
+        }
+        return ms;
+    }
+
+    @Override
+    public void record(MetricConfig config, double value, long time) {
+        long ellapsed = time - this.lastWindow;
+        if (ellapsed > config.timeWindowNs() / 2 || this.eventCount > config.eventWindow() / 2)
+            this.shadow.clear();
+        if (ellapsed > config.timeWindowNs() || this.eventCount > config.eventWindow()) {
+            Histogram tmp = this.current;
+            this.current = this.shadow;
+            this.shadow = tmp;
+            this.shadow.clear();
+        }
+        this.current.record(value);
+        this.shadow.record(value);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/stats/Rate.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/stats/Rate.java b/clients/src/main/java/kafka/common/metrics/stats/Rate.java
new file mode 100644
index 0000000..3f24a92
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/stats/Rate.java
@@ -0,0 +1,85 @@
+package kafka.common.metrics.stats;
+
+import java.util.List;
+import java.util.concurrent.TimeUnit;
+
+import kafka.common.metrics.MeasurableStat;
+import kafka.common.metrics.MetricConfig;
+
+/**
+ * The rate of the given quanitity. By default this is the total observed over a set of samples from a sampled statistic
+ * divided by the ellapsed time over the sample windows. Alternative {@link SampledStat} implementations can be
+ * provided, however, to record the rate of occurences (e.g. the count of values measured over the time interval) or
+ * other such values.
+ */
+public class Rate implements MeasurableStat {
+
+    private final TimeUnit unit;
+    private final SampledStat stat;
+
+    public Rate(TimeUnit unit) {
+        this(unit, new SampledTotal());
+    }
+
+    public Rate(TimeUnit unit, SampledStat stat) {
+        this.stat = stat;
+        this.unit = unit;
+    }
+
+    public String unitName() {
+        return unit.name().substring(0, unit.name().length() - 2).toLowerCase();
+    }
+
+    @Override
+    public void record(MetricConfig config, double value, long time) {
+        this.stat.record(config, value, time);
+    }
+
+    @Override
+    public double measure(MetricConfig config, long now) {
+        double ellapsed = convert(now - stat.oldest().lastWindow);
+        return stat.measure(config, now) / ellapsed;
+    }
+
+    private double convert(long time) {
+        switch (unit) {
+            case NANOSECONDS:
+                return time;
+            case MICROSECONDS:
+                return time / 1000.0;
+            case MILLISECONDS:
+                return time / (1000.0 * 1000.0);
+            case SECONDS:
+                return time / (1000.0 * 1000.0 * 1000.0);
+            case MINUTES:
+                return time / (60.0 * 1000.0 * 1000.0 * 1000.0);
+            case HOURS:
+                return time / (60.0 * 60.0 * 1000.0 * 1000.0 * 1000.0);
+            case DAYS:
+                return time / (24.0 * 60.0 * 60.0 * 1000.0 * 1000.0 * 1000.0);
+            default:
+                throw new IllegalStateException("Unknown unit: " + unit);
+        }
+    }
+
+    public static class SampledTotal extends SampledStat {
+
+        public SampledTotal() {
+            super(0.0d);
+        }
+
+        @Override
+        protected void update(Sample sample, MetricConfig config, double value, long now) {
+            sample.value += value;
+        }
+
+        @Override
+        public double combine(List<Sample> samples, MetricConfig config, long now) {
+            double total = 0.0;
+            for (int i = 0; i < samples.size(); i++)
+                total += samples.get(i).value;
+            return total;
+        }
+
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/stats/SampledStat.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/stats/SampledStat.java b/clients/src/main/java/kafka/common/metrics/stats/SampledStat.java
new file mode 100644
index 0000000..6f820fa
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/stats/SampledStat.java
@@ -0,0 +1,106 @@
+package kafka.common.metrics.stats;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import kafka.common.metrics.MeasurableStat;
+import kafka.common.metrics.MetricConfig;
+
+/**
+ * A SampledStat records a single scalar value measured over one or more samples. Each sample is recorded over a
+ * configurable window. The window can be defined by number of events or ellapsed time (or both, if both are given the
+ * window is complete when <i>either</i> the event count or ellapsed time criterion is met).
+ * <p>
+ * All the samples are combined to produce the measurement. When a window is complete the oldest sample is cleared and
+ * recycled to begin recording the next sample.
+ * 
+ * Subclasses of this class define different statistics measured using this basic pattern.
+ */
+public abstract class SampledStat implements MeasurableStat {
+
+    private double initialValue;
+    private int current = 0;
+    private List<Sample> samples;
+
+    public SampledStat(double initialValue) {
+        this.initialValue = initialValue;
+        this.samples = new ArrayList<Sample>(2);
+    }
+
+    @Override
+    public void record(MetricConfig config, double value, long now) {
+        Sample sample = current(now);
+        if (sample.isComplete(now, config))
+            sample = advance(config, now);
+        update(sample, config, value, now);
+        sample.eventCount += 1;
+    }
+
+    private Sample advance(MetricConfig config, long now) {
+        this.current = (this.current + 1) % config.samples();
+        if (this.current >= samples.size()) {
+            Sample sample = new Sample(this.initialValue, now);
+            this.samples.add(sample);
+            return sample;
+        } else {
+            Sample sample = current(now);
+            sample.reset(now);
+            return sample;
+        }
+    }
+
+    @Override
+    public double measure(MetricConfig config, long now) {
+        timeoutObsoleteSamples(config, now);
+        return combine(this.samples, config, now);
+    }
+
+    public Sample current(long now) {
+        if (samples.size() == 0)
+            this.samples.add(new Sample(initialValue, now));
+        return this.samples.get(this.current);
+    }
+
+    public Sample oldest() {
+        return this.samples.get((this.current + 1) % this.samples.size());
+    }
+
+    protected abstract void update(Sample sample, MetricConfig config, double value, long now);
+
+    public abstract double combine(List<Sample> samples, MetricConfig config, long now);
+
+    /* Timeout any windows that have expired in the absense of any events */
+    private void timeoutObsoleteSamples(MetricConfig config, long now) {
+        for (int i = 0; i < samples.size(); i++) {
+            int idx = (this.current + i) % samples.size();
+            Sample sample = this.samples.get(idx);
+            if (now - sample.lastWindow >= (i + 1) * config.timeWindowNs())
+                sample.reset(now);
+        }
+    }
+
+    protected static class Sample {
+        public double initialValue;
+        public long eventCount;
+        public long lastWindow;
+        public double value;
+
+        public Sample(double initialValue, long now) {
+            this.initialValue = initialValue;
+            this.eventCount = 0;
+            this.lastWindow = now;
+            this.value = initialValue;
+        }
+
+        public void reset(long now) {
+            this.eventCount = 0;
+            this.lastWindow = now;
+            this.value = initialValue;
+        }
+
+        public boolean isComplete(long now, MetricConfig config) {
+            return now - lastWindow >= config.timeWindowNs() || eventCount >= config.eventWindow();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/metrics/stats/Total.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/metrics/stats/Total.java b/clients/src/main/java/kafka/common/metrics/stats/Total.java
new file mode 100644
index 0000000..c87b1ba
--- /dev/null
+++ b/clients/src/main/java/kafka/common/metrics/stats/Total.java
@@ -0,0 +1,31 @@
+package kafka.common.metrics.stats;
+
+import kafka.common.metrics.MeasurableStat;
+import kafka.common.metrics.MetricConfig;
+
+/**
+ * An un-windowed cumulative total maintained over all time.
+ */
+public class Total implements MeasurableStat {
+
+    private double total;
+
+    public Total() {
+        this.total = 0.0;
+    }
+
+    public Total(double value) {
+        this.total = value;
+    }
+
+    @Override
+    public void record(MetricConfig config, double value, long time) {
+        this.total += value;
+    }
+
+    @Override
+    public double measure(MetricConfig config, long now) {
+        return this.total;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/network/ByteBufferReceive.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/network/ByteBufferReceive.java b/clients/src/main/java/kafka/common/network/ByteBufferReceive.java
new file mode 100644
index 0000000..cb1aaae
--- /dev/null
+++ b/clients/src/main/java/kafka/common/network/ByteBufferReceive.java
@@ -0,0 +1,43 @@
+package kafka.common.network;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ScatteringByteChannel;
+
+/**
+ * A receive backed by an array of ByteBuffers
+ */
+public class ByteBufferReceive implements Receive {
+
+    private final int source;
+    private final ByteBuffer[] buffers;
+    private int remaining;
+
+    public ByteBufferReceive(int source, ByteBuffer... buffers) {
+        super();
+        this.source = source;
+        this.buffers = buffers;
+        for (int i = 0; i < buffers.length; i++)
+            remaining += buffers[i].remaining();
+    }
+
+    @Override
+    public int source() {
+        return source;
+    }
+
+    @Override
+    public boolean complete() {
+        return remaining > 0;
+    }
+
+    @Override
+    public long readFrom(ScatteringByteChannel channel) throws IOException {
+        return channel.read(buffers);
+    }
+
+    public ByteBuffer[] reify() {
+        return buffers;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/network/ByteBufferSend.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/network/ByteBufferSend.java b/clients/src/main/java/kafka/common/network/ByteBufferSend.java
new file mode 100644
index 0000000..43bf963
--- /dev/null
+++ b/clients/src/main/java/kafka/common/network/ByteBufferSend.java
@@ -0,0 +1,54 @@
+package kafka.common.network;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.GatheringByteChannel;
+
+/**
+ * A send backed by an array of byte buffers
+ */
+public class ByteBufferSend implements Send {
+
+    private final int destination;
+    protected final ByteBuffer[] buffers;
+    private int remaining;
+
+    public ByteBufferSend(int destination, ByteBuffer... buffers) {
+        super();
+        this.destination = destination;
+        this.buffers = buffers;
+        for (int i = 0; i < buffers.length; i++)
+            remaining += buffers[i].remaining();
+    }
+
+    @Override
+    public int destination() {
+        return destination;
+    }
+
+    @Override
+    public boolean complete() {
+        return remaining > 0;
+    }
+
+    @Override
+    public ByteBuffer[] reify() {
+        return this.buffers;
+    }
+
+    @Override
+    public int remaining() {
+        return this.remaining;
+    }
+
+    @Override
+    public long writeTo(GatheringByteChannel channel) throws IOException {
+        long written = channel.write(buffers);
+        if (written < 0)
+            throw new EOFException("This shouldn't happen.");
+        remaining -= written;
+        return written;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/network/NetworkReceive.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/network/NetworkReceive.java b/clients/src/main/java/kafka/common/network/NetworkReceive.java
new file mode 100644
index 0000000..68ae48e
--- /dev/null
+++ b/clients/src/main/java/kafka/common/network/NetworkReceive.java
@@ -0,0 +1,74 @@
+package kafka.common.network;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ScatteringByteChannel;
+
+/**
+ * A size delimited Receive that consists of a 4 byte network-ordered size N followed by N bytes of content
+ */
+public class NetworkReceive implements Receive {
+
+    private final int source;
+    private final ByteBuffer size;
+    private ByteBuffer buffer;
+
+    public NetworkReceive(int source, ByteBuffer buffer) {
+        this.source = source;
+        this.buffer = buffer;
+        this.size = null;
+    }
+
+    public NetworkReceive(int source) {
+        this.source = source;
+        this.size = ByteBuffer.allocate(4);
+        this.buffer = null;
+    }
+
+    @Override
+    public int source() {
+        return source;
+    }
+
+    @Override
+    public boolean complete() {
+        return !size.hasRemaining() && !buffer.hasRemaining();
+    }
+
+    @Override
+    public ByteBuffer[] reify() {
+        return new ByteBuffer[] { this.buffer };
+    }
+
+    @Override
+    public long readFrom(ScatteringByteChannel channel) throws IOException {
+        int read = 0;
+        if (size.hasRemaining()) {
+            int bytesRead = channel.read(size);
+            if (bytesRead < 0)
+                throw new EOFException();
+            read += bytesRead;
+            if (!size.hasRemaining()) {
+                size.rewind();
+                int requestSize = size.getInt();
+                if (requestSize < 0)
+                    throw new IllegalStateException("Invalid request (size = " + requestSize + ")");
+                this.buffer = ByteBuffer.allocate(requestSize);
+            }
+        }
+        if (buffer != null) {
+            int bytesRead = channel.read(buffer);
+            if (bytesRead < 0)
+                throw new EOFException();
+            read += bytesRead;
+        }
+
+        return read;
+    }
+
+    public ByteBuffer payload() {
+        return this.buffer;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/network/NetworkSend.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/network/NetworkSend.java b/clients/src/main/java/kafka/common/network/NetworkSend.java
new file mode 100644
index 0000000..4e4ac98
--- /dev/null
+++ b/clients/src/main/java/kafka/common/network/NetworkSend.java
@@ -0,0 +1,26 @@
+package kafka.common.network;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A size delimited Send that consists of a 4 byte network-ordered size N followed by N bytes of content
+ */
+public class NetworkSend extends ByteBufferSend {
+
+    public NetworkSend(int destination, ByteBuffer... buffers) {
+        super(destination, sizeDelimit(buffers));
+    }
+
+    private static ByteBuffer[] sizeDelimit(ByteBuffer[] buffers) {
+        int size = 0;
+        for (int i = 0; i < buffers.length; i++)
+            size += buffers[i].remaining();
+        ByteBuffer[] delimited = new ByteBuffer[buffers.length + 1];
+        delimited[0] = ByteBuffer.allocate(4);
+        delimited[0].putInt(size);
+        delimited[0].rewind();
+        System.arraycopy(buffers, 0, delimited, 1, buffers.length);
+        return delimited;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/network/Receive.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/network/Receive.java b/clients/src/main/java/kafka/common/network/Receive.java
new file mode 100644
index 0000000..40ee942
--- /dev/null
+++ b/clients/src/main/java/kafka/common/network/Receive.java
@@ -0,0 +1,35 @@
+package kafka.common.network;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ScatteringByteChannel;
+
+/**
+ * This interface models the in-progress reading of data from a channel to a source identified by an integer id
+ */
+public interface Receive {
+
+    /**
+     * The numeric id of the source from which we are receiving data.
+     */
+    public int source();
+
+    /**
+     * Are we done receiving data?
+     */
+    public boolean complete();
+
+    /**
+     * Turn this receive into ByteBuffer instances, if possible (otherwise returns null).
+     */
+    public ByteBuffer[] reify();
+
+    /**
+     * Read bytes into this receive from the given channel
+     * @param channel The channel to read from
+     * @return The number of bytes read
+     * @throws IOException If the reading fails
+     */
+    public long readFrom(ScatteringByteChannel channel) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/network/Selectable.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/network/Selectable.java b/clients/src/main/java/kafka/common/network/Selectable.java
new file mode 100644
index 0000000..794fc60
--- /dev/null
+++ b/clients/src/main/java/kafka/common/network/Selectable.java
@@ -0,0 +1,68 @@
+package kafka.common.network;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.List;
+
+/**
+ * An interface for asynchronous, multi-channel network I/O
+ */
+public interface Selectable {
+
+    /**
+     * Begin establishing a socket connection to the given address identified by the given address
+     * @param id The id for this connection
+     * @param address The address to connect to
+     * @param sendBufferSize The send buffer for the socket
+     * @param receiveBufferSize The receive buffer for the socket
+     * @throws IOException If we cannot begin connecting
+     */
+    public void connect(int id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException;
+
+    /**
+     * Begin disconnecting the connection identified by the given id
+     */
+    public void disconnect(int id);
+
+    /**
+     * Wakeup this selector if it is blocked on I/O
+     */
+    public void wakeup();
+
+    /**
+     * Close this selector
+     */
+    public void close();
+
+    /**
+     * Initiate any sends provided, and make progress on any other I/O operations in-flight (connections,
+     * disconnections, existing sends, and receives)
+     * @param timeout The amount of time to block if there is nothing to do
+     * @param sends The new sends to initiate
+     * @throws IOException
+     */
+    public void poll(long timeout, List<NetworkSend> sends) throws IOException;
+
+    /**
+     * The list of sends that completed on the last {@link #poll(long, List<NetworkSend>) poll()} call.
+     */
+    public List<NetworkSend> completedSends();
+
+    /**
+     * The list of receives that completed on the last {@link #poll(long, List<NetworkSend>) poll()} call.
+     */
+    public List<NetworkReceive> completedReceives();
+
+    /**
+     * The list of connections that finished disconnecting on the last {@link #poll(long, List<NetworkSend>) poll()}
+     * call.
+     */
+    public List<Integer> disconnected();
+
+    /**
+     * The list of connections that completed their connection on the last {@link #poll(long, List<NetworkSend>) poll()}
+     * call.
+     */
+    public List<Integer> connected();
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/network/Selector.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/network/Selector.java b/clients/src/main/java/kafka/common/network/Selector.java
new file mode 100644
index 0000000..f53060c
--- /dev/null
+++ b/clients/src/main/java/kafka/common/network/Selector.java
@@ -0,0 +1,349 @@
+package kafka.common.network;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.nio.channels.CancelledKeyException;
+import java.nio.channels.SelectionKey;
+import java.nio.channels.SocketChannel;
+import java.nio.channels.UnresolvedAddressException;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Set;
+
+import kafka.common.KafkaException;
+
+/**
+ * A selector interface for doing non-blocking multi-connection network I/O.
+ * <p>
+ * This class works with {@link NetworkSend} and {@link NetworkReceive} to transmit size-delimited network requests and
+ * responses.
+ * <p>
+ * A connection can be added to the selector associated with an integer id by doing
+ * 
+ * <pre>
+ * selector.connect(42, new InetSocketAddress(&quot;google.com&quot;, server.port), 64000, 64000);
+ * </pre>
+ * 
+ * The connect call does not block on the creation of the TCP connection, so the connect method only begins initiating
+ * the connection. The successful invocation of this method does not mean a valid connection has been established.
+ * 
+ * Sending requests, receiving responses, processing connection completions, and disconnections on the existing
+ * connections are all done using the <code>poll()</code> call.
+ * 
+ * <pre>
+ * List&lt;NetworkRequest&gt; requestsToSend = Arrays.asList(new NetworkRequest(0, myBytes), new NetworkRequest(1, myOtherBytes));
+ * selector.poll(TIMEOUT_MS, requestsToSend);
+ * </pre>
+ * 
+ * The selector maintains several lists that are reset by each call to <code>poll()</code> which are available via
+ * various getters. These are reset by each call to <code>poll()</code>.
+ * 
+ * This class is not thread safe!
+ */
+public class Selector implements Selectable {
+
+    private final java.nio.channels.Selector selector;
+    private final Map<Integer, SelectionKey> keys;
+    private final List<NetworkSend> completedSends;
+    private final List<NetworkReceive> completedReceives;
+    private final List<Integer> disconnected;
+    private final List<Integer> connected;
+
+    /**
+     * Create a new selector
+     */
+    public Selector() {
+        try {
+            this.selector = java.nio.channels.Selector.open();
+        } catch (IOException e) {
+            throw new KafkaException(e);
+        }
+        this.keys = new HashMap<Integer, SelectionKey>();
+        this.completedSends = new ArrayList<NetworkSend>();
+        this.completedReceives = new ArrayList<NetworkReceive>();
+        this.connected = new ArrayList<Integer>();
+        this.disconnected = new ArrayList<Integer>();
+    }
+
+    /**
+     * Begin connecting to the given address and add the connection to this selector associated with the given id
+     * number.
+     * <p>
+     * Note that this call only initiates the connection, which will be completed on a future {@link #poll(long, List)}
+     * call. Check {@link #connected()} to see which (if any) connections have completed after a given poll call.
+     * @param id The id for the new connection
+     * @param address The address to connect to
+     * @param sendBufferSize The send buffer for the new connection
+     * @param receiveBufferSize The receive buffer for the new connection
+     * @throws IllegalStateException if there is already a connection for that id
+     * @throws UnresolvedAddressException if DNS resolution fails on the hostname
+     */
+    @Override
+    public void connect(int id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException {
+        SocketChannel channel = SocketChannel.open();
+        channel.configureBlocking(false);
+        Socket socket = channel.socket();
+        socket.setKeepAlive(true);
+        socket.setSendBufferSize(sendBufferSize);
+        socket.setReceiveBufferSize(receiveBufferSize);
+        socket.setTcpNoDelay(true);
+        try {
+            channel.connect(address);
+        } catch (UnresolvedAddressException e) {
+            channel.close();
+            throw e;
+        }
+        SelectionKey key = channel.register(this.selector, SelectionKey.OP_CONNECT);
+        key.attach(new Transmissions(id));
+        if (this.keys.containsKey(key))
+            throw new IllegalStateException("There is already a connection for id " + id);
+        this.keys.put(id, key);
+    }
+
+    /**
+     * Disconnect any connections for the given id (if there are any). The disconnection is asynchronous and will not be
+     * processed until the next {@link #poll(long, List) poll()} call.
+     */
+    @Override
+    public void disconnect(int id) {
+        SelectionKey key = this.keys.get(id);
+        if (key != null)
+            key.cancel();
+    }
+
+    /**
+     * Interrupt the selector if it is blocked waiting to do I/O.
+     */
+    @Override
+    public void wakeup() {
+        this.selector.wakeup();
+    }
+
+    /**
+     * Close this selector and all associated connections
+     */
+    @Override
+    public void close() {
+        for (SelectionKey key : this.selector.keys()) {
+            try {
+                close(key);
+            } catch (IOException e) {
+                e.printStackTrace();
+            }
+        }
+        try {
+            this.selector.close();
+        } catch (IOException e) {
+            e.printStackTrace();
+        }
+    }
+
+    /**
+     * Do whatever I/O can be done on each connection without blocking. This includes completing connections, completing
+     * disconnections, initiating new sends, or making progress on in-progress sends or receives.
+     * <p>
+     * The provided network sends will be started.
+     * 
+     * When this call is completed the user can check for completed sends, receives, connections or disconnects using
+     * {@link #completedSends()}, {@link #completedReceives()}, {@link #connected()}, {@link #disconnected()}. These
+     * lists will be cleared at the beginning of each {@link #poll(long, List)} call and repopulated by the call if any
+     * completed I/O.
+     * 
+     * @param timeout The amount of time to wait, in milliseconds. If negative, wait indefinitely.
+     * @param sends The list of new sends to begin
+     * 
+     * @throws IllegalStateException If a send is given for which we have no existing connection or for which there is
+     *         already an in-progress send
+     */
+    @Override
+    public void poll(long timeout, List<NetworkSend> sends) throws IOException {
+        clear();
+
+        /* register for write interest on any new sends */
+        for (NetworkSend send : sends) {
+            SelectionKey key = keyForId(send.destination());
+            Transmissions transmissions = transmissions(key);
+            if (transmissions.hasSend())
+                throw new IllegalStateException("Attempt to begin a send operation with prior send operation still in progress.");
+            transmissions.send = send;
+            try {
+                key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
+            } catch (CancelledKeyException e) {
+                close(key);
+            }
+        }
+
+        /* check ready keys */
+        int readyKeys = select(timeout);
+        if (readyKeys > 0) {
+            Set<SelectionKey> keys = this.selector.selectedKeys();
+            Iterator<SelectionKey> iter = keys.iterator();
+            while (iter.hasNext()) {
+                SelectionKey key = iter.next();
+                iter.remove();
+
+                Transmissions transmissions = transmissions(key);
+                SocketChannel channel = channel(key);
+                try {
+                    /*
+                     * complete any connections that have finished their handshake
+                     */
+                    if (key.isConnectable()) {
+                        channel.finishConnect();
+                        key.interestOps(key.interestOps() & ~SelectionKey.OP_CONNECT | SelectionKey.OP_READ);
+                        this.connected.add(transmissions.id);
+                    }
+
+                    /* read from any connections that have readable data */
+                    if (key.isReadable()) {
+                        if (!transmissions.hasReceive())
+                            transmissions.receive = new NetworkReceive(transmissions.id);
+                        transmissions.receive.readFrom(channel);
+                        if (transmissions.receive.complete()) {
+                            transmissions.receive.payload().rewind();
+                            this.completedReceives.add(transmissions.receive);
+                            transmissions.clearReceive();
+                        }
+                    }
+
+                    /*
+                     * write to any sockets that have space in their buffer and for which we have data
+                     */
+                    if (key.isWritable()) {
+                        transmissions.send.writeTo(channel);
+                        if (transmissions.send.remaining() <= 0) {
+                            this.completedSends.add(transmissions.send);
+                            transmissions.clearSend();
+                            key.interestOps(key.interestOps() & ~SelectionKey.OP_WRITE);
+                        }
+                    }
+
+                    /* cancel any defunct sockets */
+                    if (!key.isValid())
+                        close(key);
+                } catch (IOException e) {
+                    e.printStackTrace();
+                    close(key);
+                }
+            }
+        }
+    }
+
+    @Override
+    public List<NetworkSend> completedSends() {
+        return this.completedSends;
+    }
+
+    @Override
+    public List<NetworkReceive> completedReceives() {
+        return this.completedReceives;
+    }
+
+    @Override
+    public List<Integer> disconnected() {
+        return this.disconnected;
+    }
+
+    @Override
+    public List<Integer> connected() {
+        return this.connected;
+    }
+
+    /**
+     * Clear the results from the prior poll
+     */
+    private void clear() {
+        this.completedSends.clear();
+        this.completedReceives.clear();
+        this.connected.clear();
+        this.disconnected.clear();
+    }
+
+    /**
+     * Check for data, waiting up to the given timeout.
+     * 
+     * @param ms Length of time to wait, in milliseconds. If negative, wait indefinitely.
+     * @return The number of keys ready
+     * @throws IOException
+     */
+    private int select(long ms) throws IOException {
+        if (ms == 0L)
+            return this.selector.selectNow();
+        else if (ms < 0L)
+            return this.selector.select();
+        else
+            return this.selector.select(ms);
+    }
+
+    /**
+     * Begin closing this connection
+     */
+    private void close(SelectionKey key) throws IOException {
+        SocketChannel channel = channel(key);
+        Transmissions trans = transmissions(key);
+        if (trans != null)
+            this.disconnected.add(trans.id);
+        key.attach(null);
+        key.cancel();
+        channel.socket().close();
+        channel.close();
+    }
+
+    /**
+     * Get the selection key associated with this numeric id
+     */
+    private SelectionKey keyForId(int id) {
+        SelectionKey key = this.keys.get(id);
+        if (key == null)
+            throw new IllegalStateException("Attempt to write to socket for which there is no open connection.");
+        return key;
+    }
+
+    /**
+     * Get the transmissions for the given connection
+     */
+    private Transmissions transmissions(SelectionKey key) {
+        return (Transmissions) key.attachment();
+    }
+
+    /**
+     * Get the socket channel associated with this selection key
+     */
+    private SocketChannel channel(SelectionKey key) {
+        return (SocketChannel) key.channel();
+    }
+
+    /**
+     * The id and in-progress send and receive associated with a connection
+     */
+    private static class Transmissions {
+        public int id;
+        public NetworkSend send;
+        public NetworkReceive receive;
+
+        public Transmissions(int id) {
+            this.id = id;
+        }
+
+        public boolean hasSend() {
+            return this.send != null;
+        }
+
+        public void clearSend() {
+            this.send = null;
+        }
+
+        public boolean hasReceive() {
+            return this.receive != null;
+        }
+
+        public void clearReceive() {
+            this.receive = null;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/network/Send.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/network/Send.java b/clients/src/main/java/kafka/common/network/Send.java
new file mode 100644
index 0000000..e7ef68a
--- /dev/null
+++ b/clients/src/main/java/kafka/common/network/Send.java
@@ -0,0 +1,41 @@
+package kafka.common.network;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.GatheringByteChannel;
+
+/**
+ * This interface models the in-progress sending of data to a destination identified by an integer id.
+ */
+public interface Send {
+
+    /**
+     * The numeric id for the destination of this send
+     */
+    public int destination();
+
+    /**
+     * The number of bytes remaining to send
+     */
+    public int remaining();
+
+    /**
+     * Is this send complete?
+     */
+    public boolean complete();
+
+    /**
+     * An optional method to turn this send into an array of ByteBuffers if possible (otherwise returns null)
+     */
+    public ByteBuffer[] reify();
+
+    /**
+     * Write some as-yet unwritten bytes from this send to the provided channel. It may take multiple calls for the send
+     * to be completely written
+     * @param channel The channel to write to
+     * @return The number of bytes written
+     * @throws IOException If the write fails
+     */
+    public long writeTo(GatheringByteChannel channel) throws IOException;
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/protocol/ApiKeys.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/kafka/common/protocol/ApiKeys.java
new file mode 100644
index 0000000..1e2f8bb
--- /dev/null
+++ b/clients/src/main/java/kafka/common/protocol/ApiKeys.java
@@ -0,0 +1,35 @@
+package kafka.common.protocol;
+
+/**
+ * Identifiers for all the Kafka APIs
+ */
+public enum ApiKeys {
+    PRODUCE(0, "produce"),
+    FETCH(1, "fetch"),
+    LIST_OFFSETS(2, "list_offsets"),
+    METADATA(3, "metadata"),
+    LEADER_AND_ISR(4, "leader_and_isr"),
+    STOP_REPLICA(5, "stop_replica"),
+    OFFSET_COMMIT(6, "offset_commit"),
+    OFFSET_FETCH(7, "offset_fetch");
+
+    public static int MAX_API_KEY = 0;
+
+    static {
+        for (ApiKeys key : ApiKeys.values()) {
+            MAX_API_KEY = Math.max(MAX_API_KEY, key.id);
+        }
+    }
+
+    /** the perminant and immutable id of an API--this can't change ever */
+    public final short id;
+
+    /** an english description of the api--this is for debugging and can change */
+    public final String name;
+
+    private ApiKeys(int id, String name) {
+        this.id = (short) id;
+        this.name = name;
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/protocol/Errors.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/protocol/Errors.java b/clients/src/main/java/kafka/common/protocol/Errors.java
new file mode 100644
index 0000000..fb1a3e5
--- /dev/null
+++ b/clients/src/main/java/kafka/common/protocol/Errors.java
@@ -0,0 +1,97 @@
+package kafka.common.protocol;
+
+import java.util.HashMap;
+import java.util.Map;
+
+import kafka.common.errors.ApiException;
+import kafka.common.errors.CorruptMessageException;
+import kafka.common.errors.LeaderNotAvailableException;
+import kafka.common.errors.MessageTooLargeException;
+import kafka.common.errors.NetworkException;
+import kafka.common.errors.NotLeaderForPartitionException;
+import kafka.common.errors.OffsetMetadataTooLarge;
+import kafka.common.errors.OffsetOutOfRangeException;
+import kafka.common.errors.TimeoutException;
+import kafka.common.errors.UnknownServerException;
+import kafka.common.errors.UnknownTopicOrPartitionException;
+
+/**
+ * This class contains all the client-server errors--those errors that must be sent from the server to the client. These
+ * are thus part of the protocol. The names can be changed but the error code cannot.
+ * 
+ * Do not add exceptions that occur only on the client or only on the server here.
+ */
+public enum Errors {
+    UNKNOWN(-1, new UnknownServerException("The server experienced an unexpected error when processing the request")),
+    NONE(0, null),
+    OFFSET_OUT_OF_RANGE(1,
+                        new OffsetOutOfRangeException("The requested offset is not within the range of offsets maintained by the server.")),
+    CORRUPT_MESSAGE(2,
+                    new CorruptMessageException("The message contents does not match the message CRC or the message is otherwise corrupt.")),
+    UNKNOWN_TOPIC_OR_PARTITION(3, new UnknownTopicOrPartitionException("This server does not host this topic-partition.")),
+    LEADER_NOT_AVAILABLE(5,
+                         new LeaderNotAvailableException("There is no leader for this topic-partition as we are in the middle of a leadership election.")),
+    NOT_LEADER_FOR_PARTITION(6, new NotLeaderForPartitionException("This server is not the leader for that topic-partition.")),
+    REQUEST_TIMED_OUT(7, new TimeoutException("The request timed out.")),
+    MESSAGE_TOO_LARGE(10,
+                      new MessageTooLargeException("The request included a message larger than the max message size the server will accept.")),
+    OFFSET_METADATA_TOO_LARGE(12, new OffsetMetadataTooLarge("The metadata field of the offset request was too large.")),
+    NETWORK_EXCEPTION(13, new NetworkException("The server disconnected before a response was received."));
+
+    private static Map<Class<?>, Errors> classToError = new HashMap<Class<?>, Errors>();
+    private static Map<Short, Errors> codeToError = new HashMap<Short, Errors>();
+    static {
+        for (Errors error : Errors.values()) {
+            codeToError.put(error.code(), error);
+            if (error.exception != null)
+                classToError.put(error.exception.getClass(), error);
+        }
+
+    }
+
+    private final short code;
+    private final ApiException exception;
+
+    private Errors(int code, ApiException exception) {
+        this.code = (short) code;
+        this.exception = exception;
+    }
+
+    /**
+     * An instance of the exception
+     */
+    public ApiException exception() {
+        return this.exception;
+    }
+
+    /**
+     * The error code for the exception
+     */
+    public short code() {
+        return this.code;
+    }
+
+    /**
+     * Throw the exception corresponding to this error if there is one
+     */
+    public void maybeThrow() {
+        if (exception != null)
+            throw this.exception;
+    }
+
+    /**
+     * Throw the exception if there is one
+     */
+    public static Errors forCode(short code) {
+        Errors error = codeToError.get(code);
+        return error == null ? UNKNOWN : error;
+    }
+
+    /**
+     * Return the error instance associated with this exception (or UKNOWN if there is none)
+     */
+    public static Errors forException(Throwable t) {
+        Errors error = classToError.get(t.getClass());
+        return error == null ? UNKNOWN : error;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/protocol/ProtoUtils.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/protocol/ProtoUtils.java b/clients/src/main/java/kafka/common/protocol/ProtoUtils.java
new file mode 100644
index 0000000..83dad53
--- /dev/null
+++ b/clients/src/main/java/kafka/common/protocol/ProtoUtils.java
@@ -0,0 +1,95 @@
+package kafka.common.protocol;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+
+import kafka.common.Cluster;
+import kafka.common.Node;
+import kafka.common.PartitionInfo;
+import kafka.common.protocol.types.Schema;
+import kafka.common.protocol.types.Struct;
+
+public class ProtoUtils {
+
+    private static Schema schemaFor(Schema[][] schemas, int apiKey, int version) {
+        if (apiKey < 0 || apiKey > schemas.length)
+            throw new IllegalArgumentException("Invalid api key: " + apiKey);
+        Schema[] versions = schemas[apiKey];
+        if (version < 0 || version > versions.length)
+            throw new IllegalArgumentException("Invalid version for API key " + apiKey + ": " + version);
+        return versions[version];
+    }
+
+    public static short latestVersion(int apiKey) {
+        if (apiKey < 0 || apiKey >= Protocol.CURR_VERSION.length)
+            throw new IllegalArgumentException("Invalid api key: " + apiKey);
+        return Protocol.CURR_VERSION[apiKey];
+    }
+
+    public static Schema requestSchema(int apiKey, int version) {
+        return schemaFor(Protocol.REQUESTS, apiKey, version);
+    }
+
+    public static Schema currentRequestSchema(int apiKey) {
+        return requestSchema(apiKey, latestVersion(apiKey));
+    }
+
+    public static Schema responseSchema(int apiKey, int version) {
+        return schemaFor(Protocol.RESPONSES, apiKey, version);
+    }
+
+    public static Schema currentResponseSchema(int apiKey) {
+        return schemaFor(Protocol.RESPONSES, apiKey, latestVersion(apiKey));
+    }
+
+    public static Struct parseRequest(int apiKey, int version, ByteBuffer buffer) {
+        return (Struct) requestSchema(apiKey, version).read(buffer);
+    }
+
+    public static Struct parseResponse(int apiKey, ByteBuffer buffer) {
+        return (Struct) currentResponseSchema(apiKey).read(buffer);
+    }
+
+    public static Cluster parseMetadataResponse(Struct response) {
+        List<Node> brokers = new ArrayList<Node>();
+        Object[] brokerStructs = (Object[]) response.get("brokers");
+        for (int i = 0; i < brokerStructs.length; i++) {
+            Struct broker = (Struct) brokerStructs[i];
+            int nodeId = (Integer) broker.get("node_id");
+            String host = (String) broker.get("host");
+            int port = (Integer) broker.get("port");
+            brokers.add(new Node(nodeId, host, port));
+        }
+        List<PartitionInfo> partitions = new ArrayList<PartitionInfo>();
+        Object[] topicInfos = (Object[]) response.get("topic_metadata");
+        for (int i = 0; i < topicInfos.length; i++) {
+            Struct topicInfo = (Struct) topicInfos[i];
+            short topicError = topicInfo.getShort("topic_error_code");
+            if (topicError == Errors.NONE.code()) {
+                String topic = topicInfo.getString("topic");
+                Object[] partitionInfos = (Object[]) topicInfo.get("partition_metadata");
+                for (int j = 0; j < partitionInfos.length; j++) {
+                    Struct partitionInfo = (Struct) partitionInfos[j];
+                    short partError = partitionInfo.getShort("partition_error_code");
+                    if (partError == Errors.NONE.code()) {
+                        int partition = partitionInfo.getInt("partition_id");
+                        int leader = partitionInfo.getInt("leader");
+                        int[] replicas = intArray((Object[]) partitionInfo.get("replicas"));
+                        int[] isr = intArray((Object[]) partitionInfo.get("isr"));
+                        partitions.add(new PartitionInfo(topic, partition, leader, replicas, isr));
+                    }
+                }
+            }
+        }
+        return new Cluster(brokers, partitions);
+    }
+
+    private static int[] intArray(Object[] ints) {
+        int[] copy = new int[ints.length];
+        for (int i = 0; i < ints.length; i++)
+            copy[i] = (Integer) ints[i];
+        return copy;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/protocol/Protocol.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/protocol/Protocol.java b/clients/src/main/java/kafka/common/protocol/Protocol.java
new file mode 100644
index 0000000..e191d6a
--- /dev/null
+++ b/clients/src/main/java/kafka/common/protocol/Protocol.java
@@ -0,0 +1,130 @@
+package kafka.common.protocol;
+
+import static kafka.common.protocol.types.Type.BYTES;
+import static kafka.common.protocol.types.Type.INT16;
+import static kafka.common.protocol.types.Type.INT32;
+import static kafka.common.protocol.types.Type.INT64;
+import static kafka.common.protocol.types.Type.STRING;
+import kafka.common.protocol.types.ArrayOf;
+import kafka.common.protocol.types.Field;
+import kafka.common.protocol.types.Schema;
+
+public class Protocol {
+
+    public static Schema REQUEST_HEADER = new Schema(new Field("api_key", INT16, "The id of the request type."),
+                                                     new Field("api_version", INT16, "The version of the API."),
+                                                     new Field("correlation_id",
+                                                               INT32,
+                                                               "A user-supplied integer value that will be passed back with the response"),
+                                                     new Field("client_id",
+                                                               STRING,
+                                                               "A user specified identifier for the client making the request."));
+
+    public static Schema RESPONSE_HEADER = new Schema(new Field("correlation_id",
+                                                                INT32,
+                                                                "The user-supplied value passed in with the request"));
+
+    /* Metadata api */
+
+    public static Schema METADATA_REQUEST_V0 = new Schema(new Field("topics",
+                                                                    new ArrayOf(STRING),
+                                                                    "An array of topics to fetch metadata for. If no topics are specified fetch metadtata for all topics."));
+
+    public static Schema BROKER = new Schema(new Field("node_id", INT32, "The broker id."),
+                                             new Field("host", STRING, "The hostname of the broker."),
+                                             new Field("port", INT32, "The port on which the broker accepts requests."));
+
+    public static Schema PARTITION_METADATA_V0 = new Schema(new Field("partition_error_code",
+                                                                      INT16,
+                                                                      "The error code for the partition, if any."),
+                                                            new Field("partition_id", INT32, "The id of the partition."),
+                                                            new Field("leader",
+                                                                      INT32,
+                                                                      "The id of the broker acting as leader for this partition."),
+                                                            new Field("replicas",
+                                                                      new ArrayOf(INT32),
+                                                                      "The set of all nodes that host this partition."),
+                                                            new Field("isr",
+                                                                      new ArrayOf(INT32),
+                                                                      "The set of nodes that are in sync with the leader for this partition."));
+
+    public static Schema TOPIC_METADATA_V0 = new Schema(new Field("topic_error_code", INT16, "The error code for the given topic."),
+                                                        new Field("topic", STRING, "The name of the topic"),
+                                                        new Field("partition_metadata",
+                                                                  new ArrayOf(PARTITION_METADATA_V0),
+                                                                  "Metadata for each partition of the topic."));
+
+    public static Schema METADATA_RESPONSE_V0 = new Schema(new Field("brokers",
+                                                                     new ArrayOf(BROKER),
+                                                                     "Host and port information for all brokers."),
+                                                           new Field("topic_metadata", new ArrayOf(TOPIC_METADATA_V0)));
+
+    public static Schema[] METADATA_REQUEST = new Schema[] { METADATA_REQUEST_V0 };
+    public static Schema[] METADATA_RESPONSE = new Schema[] { METADATA_RESPONSE_V0 };
+
+    /* Produce api */
+
+    public static Schema TOPIC_PRODUCE_DATA_V0 = new Schema(new Field("topic", STRING),
+                                                            new Field("data", new ArrayOf(new Schema(new Field("partition", INT32),
+                                                                                                     new Field("message_set", BYTES)))));
+
+    public static Schema PRODUCE_REQUEST_V0 = new Schema(new Field("acks",
+                                                                   INT16,
+                                                                   "The number of nodes that should replicate the produce before returning. -1 indicates the full ISR."),
+                                                         new Field("timeout", INT32, "The time to await a response in ms."),
+                                                         new Field("topic_data", new ArrayOf(TOPIC_PRODUCE_DATA_V0)));
+
+    public static Schema PRODUCE_RESPONSE_V0 = new Schema(new Field("responses",
+                                                                    new ArrayOf(new Schema(new Field("topic", STRING),
+                                                                                           new Field("partition_responses",
+                                                                                                     new ArrayOf(new Schema(new Field("partition",
+                                                                                                                                      INT32),
+                                                                                                                            new Field("error_code",
+                                                                                                                                      INT16),
+                                                                                                                            new Field("base_offset",
+                                                                                                                                      INT64))))))));
+
+    public static Schema[] PRODUCE_REQUEST = new Schema[] { PRODUCE_REQUEST_V0 };
+    public static Schema[] PRODUCE_RESPONSE = new Schema[] { PRODUCE_RESPONSE_V0 };
+
+    /* an array of all requests and responses with all schema versions */
+    public static Schema[][] REQUESTS = new Schema[ApiKeys.MAX_API_KEY + 1][];
+    public static Schema[][] RESPONSES = new Schema[ApiKeys.MAX_API_KEY + 1][];
+
+    /* the latest version of each api */
+    public static short[] CURR_VERSION = new short[ApiKeys.MAX_API_KEY + 1];
+
+    static {
+        REQUESTS[ApiKeys.PRODUCE.id] = PRODUCE_REQUEST;
+        REQUESTS[ApiKeys.FETCH.id] = new Schema[] {};
+        REQUESTS[ApiKeys.LIST_OFFSETS.id] = new Schema[] {};
+        REQUESTS[ApiKeys.METADATA.id] = METADATA_REQUEST;
+        REQUESTS[ApiKeys.LEADER_AND_ISR.id] = new Schema[] {};
+        REQUESTS[ApiKeys.STOP_REPLICA.id] = new Schema[] {};
+        REQUESTS[ApiKeys.OFFSET_COMMIT.id] = new Schema[] {};
+        REQUESTS[ApiKeys.OFFSET_FETCH.id] = new Schema[] {};
+
+        RESPONSES[ApiKeys.PRODUCE.id] = PRODUCE_RESPONSE;
+        RESPONSES[ApiKeys.FETCH.id] = new Schema[] {};
+        RESPONSES[ApiKeys.LIST_OFFSETS.id] = new Schema[] {};
+        RESPONSES[ApiKeys.METADATA.id] = METADATA_RESPONSE;
+        RESPONSES[ApiKeys.LEADER_AND_ISR.id] = new Schema[] {};
+        RESPONSES[ApiKeys.STOP_REPLICA.id] = new Schema[] {};
+        RESPONSES[ApiKeys.OFFSET_COMMIT.id] = new Schema[] {};
+        RESPONSES[ApiKeys.OFFSET_FETCH.id] = new Schema[] {};
+
+        /* set the maximum version of each api */
+        for (ApiKeys api : ApiKeys.values())
+            CURR_VERSION[api.id] = (short) (REQUESTS[api.id].length - 1);
+
+        /* sanity check that we have the same number of request and response versions for each api */
+        for (ApiKeys api : ApiKeys.values())
+            if (REQUESTS[api.id].length != RESPONSES[api.id].length)
+                throw new IllegalStateException(REQUESTS[api.id].length + " request versions for api "
+                                                + api.name
+                                                + " but "
+                                                + RESPONSES[api.id].length
+                                                + " response versions.");
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/protocol/types/ArrayOf.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/protocol/types/ArrayOf.java b/clients/src/main/java/kafka/common/protocol/types/ArrayOf.java
new file mode 100644
index 0000000..5daf95b
--- /dev/null
+++ b/clients/src/main/java/kafka/common/protocol/types/ArrayOf.java
@@ -0,0 +1,63 @@
+package kafka.common.protocol.types;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Represents a type for an array of a particular type
+ */
+public class ArrayOf extends Type {
+
+    private final Type type;
+
+    public ArrayOf(Type type) {
+        this.type = type;
+    }
+
+    @Override
+    public void write(ByteBuffer buffer, Object o) {
+        Object[] objs = (Object[]) o;
+        int size = objs.length;
+        buffer.putInt(size);
+        for (int i = 0; i < size; i++)
+            type.write(buffer, objs[i]);
+    }
+
+    @Override
+    public Object read(ByteBuffer buffer) {
+        int size = buffer.getInt();
+        Object[] objs = new Object[size];
+        for (int i = 0; i < size; i++)
+            objs[i] = type.read(buffer);
+        return objs;
+    }
+
+    @Override
+    public int sizeOf(Object o) {
+        Object[] objs = (Object[]) o;
+        int size = 4;
+        for (int i = 0; i < objs.length; i++)
+            size += type.sizeOf(objs[i]);
+        return size;
+    }
+
+    public Type type() {
+        return type;
+    }
+
+    @Override
+    public String toString() {
+        return "ARRAY(" + type + ")";
+    }
+
+    @Override
+    public Object[] validate(Object item) {
+        try {
+            Object[] array = (Object[]) item;
+            for (int i = 0; i < array.length; i++)
+                type.validate(array[i]);
+            return array;
+        } catch (ClassCastException e) {
+            throw new SchemaException("Not an Object[].");
+        }
+    }
+}


[7/7] git commit: KAFKA-1227 New producer!

Posted by jk...@apache.org.
KAFKA-1227 New producer!


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

Branch: refs/heads/trunk
Commit: 269d16d3c915d09f650ae32aa81542bd8522ca68
Parents: 26a02c3
Author: Jay Kreps <ja...@gmail.com>
Authored: Thu Jan 23 12:41:11 2014 -0800
Committer: Jay Kreps <ja...@gmail.com>
Committed: Tue Jan 28 11:15:05 2014 -0800

----------------------------------------------------------------------
 clients/build.sbt                               |   11 +
 .../producer/BufferExhaustedException.java      |   17 +
 .../java/kafka/clients/producer/Callback.java   |   15 +
 .../clients/producer/DefaultPartitioner.java    |   35 +
 .../kafka/clients/producer/KafkaProducer.java   |  240 ++
 .../kafka/clients/producer/MockProducer.java    |  201 ++
 .../kafka/clients/producer/Partitioner.java     |   30 +
 .../java/kafka/clients/producer/Producer.java   |   38 +
 .../kafka/clients/producer/ProducerConfig.java  |  148 ++
 .../kafka/clients/producer/ProducerRecord.java  |   84 +
 .../java/kafka/clients/producer/RecordSend.java |   88 +
 .../clients/producer/internals/BufferPool.java  |  223 ++
 .../clients/producer/internals/Metadata.java    |  120 +
 .../internals/ProduceRequestResult.java         |   83 +
 .../producer/internals/RecordAccumulator.java   |  236 ++
 .../clients/producer/internals/RecordBatch.java |   84 +
 .../clients/producer/internals/Sender.java      |  503 ++++
 .../clients/tools/ProducerPerformance.java      |   70 +
 .../java/kafka/common/ByteSerialization.java    |   18 +
 clients/src/main/java/kafka/common/Cluster.java |  102 +
 .../main/java/kafka/common/Configurable.java    |   15 +
 .../main/java/kafka/common/Deserializer.java    |   18 +
 .../main/java/kafka/common/KafkaException.java  |   26 +
 clients/src/main/java/kafka/common/Metric.java  |   23 +
 clients/src/main/java/kafka/common/Node.java    |   76 +
 .../main/java/kafka/common/PartitionInfo.java   |   58 +
 .../src/main/java/kafka/common/Serializer.java  |   21 +
 .../java/kafka/common/StringSerialization.java  |   58 +
 .../main/java/kafka/common/TopicPartition.java  |   61 +
 .../kafka/common/config/AbstractConfig.java     |   93 +
 .../java/kafka/common/config/ConfigDef.java     |  253 ++
 .../kafka/common/config/ConfigException.java    |   24 +
 .../java/kafka/common/errors/ApiException.java  |   35 +
 .../common/errors/CorruptMessageException.java  |   23 +
 .../errors/LeaderNotAvailableException.java     |   19 +
 .../common/errors/MessageTooLargeException.java |   23 +
 .../kafka/common/errors/NetworkException.java   |   23 +
 .../errors/NotLeaderForPartitionException.java  |   23 +
 .../common/errors/OffsetMetadataTooLarge.java   |   22 +
 .../errors/OffsetOutOfRangeException.java       |   22 +
 .../kafka/common/errors/RetryableException.java |   31 +
 .../kafka/common/errors/TimeoutException.java   |   23 +
 .../common/errors/UnknownServerException.java   |   22 +
 .../UnknownTopicOrPartitionException.java       |   22 +
 .../java/kafka/common/metrics/CompoundStat.java |   40 +
 .../java/kafka/common/metrics/JmxReporter.java  |  184 ++
 .../java/kafka/common/metrics/KafkaMetric.java  |   55 +
 .../java/kafka/common/metrics/Measurable.java   |   16 +
 .../kafka/common/metrics/MeasurableStat.java    |   10 +
 .../java/kafka/common/metrics/MetricConfig.java |   71 +
 .../main/java/kafka/common/metrics/Metrics.java |  190 ++
 .../kafka/common/metrics/MetricsReporter.java   |   27 +
 .../main/java/kafka/common/metrics/Quota.java   |   36 +
 .../common/metrics/QuotaViolationException.java |   16 +
 .../main/java/kafka/common/metrics/Sensor.java  |  171 ++
 .../main/java/kafka/common/metrics/Stat.java    |   16 +
 .../java/kafka/common/metrics/stats/Avg.java    |   33 +
 .../java/kafka/common/metrics/stats/Count.java  |   29 +
 .../kafka/common/metrics/stats/Histogram.java   |  137 ++
 .../java/kafka/common/metrics/stats/Max.java    |   29 +
 .../java/kafka/common/metrics/stats/Min.java    |   29 +
 .../kafka/common/metrics/stats/Percentile.java  |   32 +
 .../kafka/common/metrics/stats/Percentiles.java |   76 +
 .../java/kafka/common/metrics/stats/Rate.java   |   85 +
 .../kafka/common/metrics/stats/SampledStat.java |  106 +
 .../java/kafka/common/metrics/stats/Total.java  |   31 +
 .../kafka/common/network/ByteBufferReceive.java |   43 +
 .../kafka/common/network/ByteBufferSend.java    |   54 +
 .../kafka/common/network/NetworkReceive.java    |   74 +
 .../java/kafka/common/network/NetworkSend.java  |   26 +
 .../main/java/kafka/common/network/Receive.java |   35 +
 .../java/kafka/common/network/Selectable.java   |   68 +
 .../java/kafka/common/network/Selector.java     |  349 +++
 .../main/java/kafka/common/network/Send.java    |   41 +
 .../java/kafka/common/protocol/ApiKeys.java     |   35 +
 .../main/java/kafka/common/protocol/Errors.java |   97 +
 .../java/kafka/common/protocol/ProtoUtils.java  |   95 +
 .../java/kafka/common/protocol/Protocol.java    |  130 ++
 .../kafka/common/protocol/types/ArrayOf.java    |   63 +
 .../java/kafka/common/protocol/types/Field.java |   48 +
 .../kafka/common/protocol/types/Schema.java     |  134 ++
 .../common/protocol/types/SchemaException.java  |   13 +
 .../kafka/common/protocol/types/Struct.java     |  227 ++
 .../java/kafka/common/protocol/types/Type.java  |  216 ++
 .../kafka/common/record/CompressionType.java    |   40 +
 .../common/record/InvalidRecordException.java   |   11 +
 .../main/java/kafka/common/record/LogEntry.java |   28 +
 .../java/kafka/common/record/MemoryRecords.java |  102 +
 .../main/java/kafka/common/record/Record.java   |  286 +++
 .../main/java/kafka/common/record/Records.java  |   29 +
 .../kafka/common/requests/RequestHeader.java    |   68 +
 .../java/kafka/common/requests/RequestSend.java |   38 +
 .../kafka/common/requests/ResponseHeader.java   |   45 +
 .../kafka/common/utils/AbstractIterator.java    |   72 +
 .../java/kafka/common/utils/CopyOnWriteMap.java |  130 ++
 .../src/main/java/kafka/common/utils/Crc32.java | 2169 ++++++++++++++++++
 .../java/kafka/common/utils/KafkaThread.java    |   18 +
 .../java/kafka/common/utils/SystemTime.java     |   26 +
 .../src/main/java/kafka/common/utils/Time.java  |   23 +
 .../src/main/java/kafka/common/utils/Utils.java |  230 ++
 .../clients/common/network/SelectorTest.java    |  292 +++
 .../kafka/clients/producer/BufferPoolTest.java  |  170 ++
 .../kafka/clients/producer/MetadataTest.java    |   55 +
 .../clients/producer/MockProducerTest.java      |   66 +
 .../clients/producer/RecordAccumulatorTest.java |  135 ++
 .../kafka/clients/producer/RecordSendTest.java  |   76 +
 .../java/kafka/clients/producer/SenderTest.java |   92 +
 .../java/kafka/common/config/ConfigDefTest.java |   88 +
 .../kafka/common/metrics/JmxReporterTest.java   |   21 +
 .../java/kafka/common/metrics/MetricsTest.java  |  176 ++
 .../common/metrics/stats/HistogramTest.java     |   56 +
 .../types/ProtocolSerializationTest.java        |   96 +
 .../kafka/common/record/MemoryRecordsTest.java  |   44 +
 .../java/kafka/common/record/RecordTest.java    |   87 +
 .../common/utils/AbstractIteratorTest.java      |   54 +
 .../test/java/kafka/common/utils/MockTime.java  |   28 +
 .../src/test/java/kafka/test/MetricsBench.java  |   38 +
 .../test/java/kafka/test/Microbenchmarks.java   |  143 ++
 .../src/test/java/kafka/test/MockSelector.java  |   87 +
 clients/src/test/java/kafka/test/TestUtils.java |   73 +
 project/Build.scala                             |    1 +
 121 files changed, 11720 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/build.sbt
----------------------------------------------------------------------
diff --git a/clients/build.sbt b/clients/build.sbt
new file mode 100644
index 0000000..ca3c8ee
--- /dev/null
+++ b/clients/build.sbt
@@ -0,0 +1,11 @@
+import sbt._
+import Keys._
+import AssemblyKeys._
+
+name := "clients"
+
+libraryDependencies ++= Seq(
+  "com.novocode"          % "junit-interface" % "0.9" % "test"
+)
+
+assemblySettings

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/clients/producer/BufferExhaustedException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/clients/producer/BufferExhaustedException.java b/clients/src/main/java/kafka/clients/producer/BufferExhaustedException.java
new file mode 100644
index 0000000..d1aa4b6
--- /dev/null
+++ b/clients/src/main/java/kafka/clients/producer/BufferExhaustedException.java
@@ -0,0 +1,17 @@
+package kafka.clients.producer;
+
+import kafka.common.KafkaException;
+
+/**
+ * This exception is thrown if the producer is in non-blocking mode and the rate of data production exceeds the rate at
+ * which data can be sent for long enough for the alloted buffer to be exhausted.
+ */
+public class BufferExhaustedException extends KafkaException {
+
+    private static final long serialVersionUID = 1L;
+
+    public BufferExhaustedException(String message) {
+        super(message);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/clients/producer/Callback.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/clients/producer/Callback.java b/clients/src/main/java/kafka/clients/producer/Callback.java
new file mode 100644
index 0000000..47e5af3
--- /dev/null
+++ b/clients/src/main/java/kafka/clients/producer/Callback.java
@@ -0,0 +1,15 @@
+package kafka.clients.producer;
+
+/**
+ * A callback interface that the user can implement to allow code to execute when the request is complete. This callback
+ * will execute in the background I/O thread so it should be fast.
+ */
+public interface Callback {
+
+    /**
+     * A callback method the user should implement. This method will be called when the send to the server has
+     * completed.
+     * @param send The results of the call. This send is guaranteed to be completed so none of its methods will block.
+     */
+    public void onCompletion(RecordSend send);
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/clients/producer/DefaultPartitioner.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/clients/producer/DefaultPartitioner.java b/clients/src/main/java/kafka/clients/producer/DefaultPartitioner.java
new file mode 100644
index 0000000..b82fcfb
--- /dev/null
+++ b/clients/src/main/java/kafka/clients/producer/DefaultPartitioner.java
@@ -0,0 +1,35 @@
+package kafka.clients.producer;
+
+import java.util.Random;
+import java.util.concurrent.atomic.AtomicInteger;
+
+import kafka.common.Cluster;
+import kafka.common.utils.Utils;
+
+/**
+ * A simple partitioning strategy that will work for messages with or without keys.
+ * <p>
+ * If there is a partition key specified in the record the partitioner will use that for partitioning. Otherwise, if
+ * there there is no partitionKey but there is a normal key that will be used. If neither key is specified the
+ * partitioner will round-robin over partitions in the topic.
+ * <p>
+ * For the cases where there is some key present the partition is computed based on the murmur2 hash of the serialized
+ * key.
+ */
+public class DefaultPartitioner implements Partitioner {
+
+    private final AtomicInteger counter = new AtomicInteger(new Random().nextInt());
+
+    /**
+     * Compute the partition
+     */
+    @Override
+    public int partition(ProducerRecord record, byte[] key, byte[] partitionKey, byte[] value, Cluster cluster, int numPartitions) {
+        byte[] keyToUse = partitionKey != null ? partitionKey : key;
+        if (keyToUse == null)
+            return Utils.abs(counter.getAndIncrement()) % numPartitions;
+        else
+            return Utils.abs(Utils.murmur2(keyToUse)) % numPartitions;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/clients/producer/KafkaProducer.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/kafka/clients/producer/KafkaProducer.java
new file mode 100644
index 0000000..58eee0c
--- /dev/null
+++ b/clients/src/main/java/kafka/clients/producer/KafkaProducer.java
@@ -0,0 +1,240 @@
+package kafka.clients.producer;
+
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+
+import kafka.clients.producer.internals.Metadata;
+import kafka.clients.producer.internals.RecordAccumulator;
+import kafka.clients.producer.internals.Sender;
+import kafka.common.Cluster;
+import kafka.common.KafkaException;
+import kafka.common.Metric;
+import kafka.common.Serializer;
+import kafka.common.TopicPartition;
+import kafka.common.config.ConfigException;
+import kafka.common.errors.MessageTooLargeException;
+import kafka.common.metrics.JmxReporter;
+import kafka.common.metrics.MetricConfig;
+import kafka.common.metrics.Metrics;
+import kafka.common.metrics.MetricsReporter;
+import kafka.common.network.Selector;
+import kafka.common.record.CompressionType;
+import kafka.common.record.Record;
+import kafka.common.record.Records;
+import kafka.common.utils.KafkaThread;
+import kafka.common.utils.SystemTime;
+
+/**
+ * A Kafka producer that can be used to send data to the Kafka cluster.
+ * <P>
+ * The producer is <i>thread safe</i> and should generally be shared among all threads for best performance.
+ * <p>
+ * The producer manages a single background thread that does I/O as well as a TCP connection to each of the brokers it
+ * needs to communicate with. Failure to close the producer after use will leak these.
+ */
+public class KafkaProducer implements Producer {
+
+    private final int maxRequestSize;
+    private final long metadataFetchTimeoutMs;
+    private final long totalMemorySize;
+    private final Partitioner partitioner;
+    private final Metadata metadata;
+    private final RecordAccumulator accumulator;
+    private final Sender sender;
+    private final Serializer keySerializer;
+    private final Serializer valueSerializer;
+    private final Metrics metrics;
+    private final Thread ioThread;
+
+    /**
+     * A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings
+     * are documented <a href="http://kafka.apache.org/documentation.html#producerconfigs">here</a>. Values can be
+     * either strings or Objects of the appropriate type (for example a numeric configuration would accept either the
+     * string "42" or the integer 42).
+     */
+    public KafkaProducer(Map<String, Object> configs) {
+        this(new ProducerConfig(configs));
+    }
+
+    /**
+     * A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings
+     * are documented <a href="http://kafka.apache.org/documentation.html#producerconfigs">here</a>.
+     */
+    public KafkaProducer(Properties properties) {
+        this(new ProducerConfig(properties));
+    }
+
+    private KafkaProducer(ProducerConfig config) {
+        this.metrics = new Metrics(new MetricConfig(),
+                                   Collections.singletonList((MetricsReporter) new JmxReporter("kafka.producer.")),
+                                   new SystemTime());
+        this.keySerializer = config.getConfiguredInstance(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, Serializer.class);
+        this.valueSerializer = config.getConfiguredInstance(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, Serializer.class);
+        this.partitioner = config.getConfiguredInstance(ProducerConfig.PARTITIONER_CLASS_CONFIG, Partitioner.class);
+        this.metadataFetchTimeoutMs = config.getLong(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG);
+        this.metadata = new Metadata();
+        this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG);
+        this.totalMemorySize = config.getLong(ProducerConfig.TOTAL_BUFFER_MEMORY_CONFIG);
+        this.accumulator = new RecordAccumulator(config.getInt(ProducerConfig.MAX_PARTITION_SIZE_CONFIG),
+                                                 this.totalMemorySize,
+                                                 config.getLong(ProducerConfig.LINGER_MS_CONFIG),
+                                                 config.getBoolean(ProducerConfig.BLOCK_ON_BUFFER_FULL),
+                                                 metrics,
+                                                 new SystemTime());
+        List<InetSocketAddress> addresses = parseAndValidateAddresses(config.getList(ProducerConfig.BROKER_LIST_CONFIG));
+        this.metadata.update(Cluster.bootstrap(addresses), System.currentTimeMillis());
+        this.sender = new Sender(new Selector(),
+                                 this.metadata,
+                                 this.accumulator,
+                                 config.getString(ProducerConfig.CLIENT_ID_CONFIG),
+                                 config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG),
+                                 config.getLong(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG),
+                                 (short) config.getInt(ProducerConfig.REQUIRED_ACKS_CONFIG),
+                                 config.getInt(ProducerConfig.REQUEST_TIMEOUT_CONFIG),
+                                 new SystemTime());
+        this.ioThread = new KafkaThread("kafka-network-thread", this.sender, true);
+        this.ioThread.start();
+    }
+
+    private static List<InetSocketAddress> parseAndValidateAddresses(List<String> urls) {
+        List<InetSocketAddress> addresses = new ArrayList<InetSocketAddress>();
+        for (String url : urls) {
+            if (url != null && url.length() > 0) {
+                String[] pieces = url.split(":");
+                if (pieces.length != 2)
+                    throw new ConfigException("Invalid url in metadata.broker.list: " + url);
+                try {
+                    InetSocketAddress address = new InetSocketAddress(pieces[0], Integer.parseInt(pieces[1]));
+                    if (address.isUnresolved())
+                        throw new ConfigException("DNS resolution failed for metadata bootstrap url: " + url);
+                    addresses.add(address);
+                } catch (NumberFormatException e) {
+                    throw new ConfigException("Invalid port in metadata.broker.list: " + url);
+                }
+            }
+        }
+        if (addresses.size() < 1)
+            throw new ConfigException("No bootstrap urls given in metadata.broker.list.");
+        return addresses;
+    }
+
+    /**
+     * Asynchronously send a record to a topic. Equivalent to {@link #send(ProducerRecord, Callback) send(record, null)}
+     */
+    @Override
+    public RecordSend send(ProducerRecord record) {
+        return send(record, null);
+    }
+
+    /**
+     * Asynchronously send a record to a topic and invoke the provided callback when the send has been acknowledged.
+     * <p>
+     * The send is asynchronous and this method will return immediately once the record has been serialized and stored
+     * in the buffer of messages waiting to be sent. This allows sending many records in parallel without necessitating
+     * blocking to wait for the response after each one.
+     * <p>
+     * The {@link RecordSend} returned by this call will hold the future response data including the offset assigned to
+     * the message and the error (if any) when the request has completed (or returned an error), and this object can be
+     * used to block awaiting the response. If you want the equivalent of a simple blocking send you can easily achieve
+     * that using the {@link kafka.clients.producer.RecordSend#await() await()} method on the {@link RecordSend} this
+     * call returns:
+     * 
+     * <pre>
+     *   ProducerRecord record = new ProducerRecord("the-topic", "key, "value");
+     *   producer.send(myRecord, null).await();
+     * </pre>
+     * 
+     * Note that the send method will not throw an exception if the request fails while communicating with the cluster,
+     * rather that exception will be thrown when accessing the {@link RecordSend} that is returned.
+     * <p>
+     * Those desiring fully non-blocking usage can make use of the {@link Callback} parameter to provide a callback that
+     * will be invoked when the request is complete. Note that the callback will execute in the I/O thread of the
+     * producer and so should be reasonably fast. An example usage of an inline callback would be the following:
+     * 
+     * <pre>
+     *   ProducerRecord record = new ProducerRecord("the-topic", "key, "value");
+     *   producer.send(myRecord,
+     *                 new Callback() {
+     *                     public void onCompletion(RecordSend send) {
+     *                         try {
+     *                             System.out.println("The offset of the message we just sent is: " + send.offset());
+     *                         } catch(KafkaException e) {
+     *                             e.printStackTrace();
+     *                         }
+     *                     }
+     *                 });
+     * </pre>
+     * <p>
+     * This call enqueues the message in the buffer of outgoing messages to be sent. This buffer has a hard limit on
+     * it's size controlled by the configuration <code>total.memory.bytes</code>. If <code>send()</code> is called
+     * faster than the I/O thread can send data to the brokers we will eventually run out of buffer space. The default
+     * behavior in this case is to block the send call until the I/O thread catches up and more buffer space is
+     * available. However if non-blocking usage is desired the setting <code>block.on.buffer.full=false</code> will
+     * cause the producer to instead throw an exception when this occurs.
+     * 
+     * @param record The record to send
+     * @param callback A user-supplied callback to execute when the record has been acknowledged by the server (null
+     *        indicates no callback)
+     * @throws BufferExhausedException This exception is thrown if the buffer is full and blocking has been disabled.
+     * @throws MessageTooLargeException This exception is thrown if the serialized size of the message is larger than
+     *         the maximum buffer memory or maximum request size that has been configured (whichever is smaller).
+     */
+    @Override
+    public RecordSend send(ProducerRecord record, Callback callback) {
+        Cluster cluster = metadata.fetch(record.topic(), this.metadataFetchTimeoutMs);
+        byte[] key = keySerializer.toBytes(record.key());
+        byte[] value = valueSerializer.toBytes(record.value());
+        byte[] partitionKey = keySerializer.toBytes(record.partitionKey());
+        int partition = partitioner.partition(record, key, partitionKey, value, cluster, cluster.partitionsFor(record.topic()).size());
+        ensureValidSize(key, value);
+        try {
+            TopicPartition tp = new TopicPartition(record.topic(), partition);
+            RecordSend send = accumulator.append(tp, key, value, CompressionType.NONE, callback);
+            this.sender.wakeup();
+            return send;
+        } catch (InterruptedException e) {
+            throw new KafkaException(e);
+        }
+    }
+
+    /**
+     * Check that this key-value pair will have a serialized size small enough
+     */
+    private void ensureValidSize(byte[] key, byte[] value) {
+        int serializedSize = Records.LOG_OVERHEAD + Record.recordSize(key, value);
+        if (serializedSize > this.maxRequestSize)
+            throw new MessageTooLargeException("The message is " + serializedSize
+                                               + " bytes when serialized which is larger than the maximum request size you have configured with the "
+                                               + ProducerConfig.MAX_REQUEST_SIZE_CONFIG
+                                               + " configuration.");
+        if (serializedSize > this.totalMemorySize)
+            throw new MessageTooLargeException("The message is " + serializedSize
+                                               + " bytes when serialized which is larger than the total memory buffer you have configured with the "
+                                               + ProducerConfig.TOTAL_BUFFER_MEMORY_CONFIG
+                                               + " configuration.");
+    }
+
+    @Override
+    public Map<String, ? extends Metric> metrics() {
+        return Collections.unmodifiableMap(this.metrics.metrics());
+    }
+
+    /**
+     * Close this producer. This method blocks until all in-flight requests complete.
+     */
+    @Override
+    public void close() {
+        this.sender.initiateClose();
+        try {
+            this.ioThread.join();
+        } catch (InterruptedException e) {
+            throw new KafkaException(e);
+        }
+        this.metrics.close();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/clients/producer/MockProducer.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/clients/producer/MockProducer.java b/clients/src/main/java/kafka/clients/producer/MockProducer.java
new file mode 100644
index 0000000..2ea2030
--- /dev/null
+++ b/clients/src/main/java/kafka/clients/producer/MockProducer.java
@@ -0,0 +1,201 @@
+package kafka.clients.producer;
+
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import kafka.clients.producer.internals.ProduceRequestResult;
+import kafka.common.Cluster;
+import kafka.common.Metric;
+import kafka.common.Serializer;
+import kafka.common.TopicPartition;
+
+/**
+ * A mock of the producer interface you can use for testing code that uses Kafka.
+ * <p>
+ * By default this mock will synchronously complete each send call successfully. However it can be configured to allow
+ * the user to control the completion of the call and supply an optional error for the producer to throw.
+ */
+public class MockProducer implements Producer {
+
+    private final Serializer keySerializer;
+    private final Serializer valueSerializer;
+    private final Partitioner partitioner;
+    private final Cluster cluster;
+    private final List<ProducerRecord> sent;
+    private final Deque<Completion> completions;
+    private boolean autoComplete;
+    private Map<TopicPartition, Long> offsets;
+
+    /**
+     * Create a mock producer
+     * 
+     * @param keySerializer A serializer to use on keys (useful to test your serializer on the values)
+     * @param valueSerializer A serializer to use on values (useful to test your serializer on the values)
+     * @param partitioner A partitioner to choose partitions (if null the partition will always be 0)
+     * @param cluster The cluster to pass to the partitioner (can be null if partitioner is null)
+     * @param autoComplete If true automatically complete all requests successfully and execute the callback. Otherwise
+     *        the user must call {@link #completeNext()} or {@link #errorNext(RuntimeException)} after
+     *        {@link #send(ProducerRecord) send()} to complete the call and unblock the @{link RecordSend} that is
+     *        returned.
+     */
+    public MockProducer(Serializer keySerializer, Serializer valueSerializer, Partitioner partitioner, Cluster cluster, boolean autoComplete) {
+        if (partitioner != null && (cluster == null | keySerializer == null | valueSerializer == null))
+            throw new IllegalArgumentException("If a partitioner is provided a cluster instance and key and value serializer for partitioning must also be given.");
+        this.keySerializer = keySerializer;
+        this.valueSerializer = valueSerializer;
+        this.partitioner = partitioner;
+        this.cluster = cluster;
+        this.autoComplete = autoComplete;
+        this.offsets = new HashMap<TopicPartition, Long>();
+        this.sent = new ArrayList<ProducerRecord>();
+        this.completions = new ArrayDeque<Completion>();
+    }
+
+    /**
+     * Create a new mock producer with no serializers or partitioner and the given autoComplete setting.
+     * 
+     * Equivalent to {@link #MockProducer(Serializer, Serializer, Partitioner, Cluster, boolean) new MockProducer(null,
+     * null, null, null, autoComplete)}
+     */
+    public MockProducer(boolean autoComplete) {
+        this(null, null, null, null, autoComplete);
+    }
+
+    /**
+     * Create a new auto completing mock producer with no serializers or partitioner.
+     * 
+     * Equivalent to {@link #MockProducer(boolean) new MockProducer(true)}
+     */
+    public MockProducer() {
+        this(true);
+    }
+
+    /**
+     * Adds the record to the list of sent records. The {@link RecordSend} returned will be immediately satisfied.
+     * 
+     * @see #history()
+     */
+    @Override
+    public synchronized RecordSend send(ProducerRecord record) {
+        return send(record, null);
+    }
+
+    /**
+     * Adds the record to the list of sent records. The {@link RecordSend} returned will be immediately satisfied and
+     * the callback will be synchronously executed.
+     * 
+     * @see #history()
+     */
+    @Override
+    public synchronized RecordSend send(ProducerRecord record, Callback callback) {
+        byte[] key = keySerializer == null ? null : keySerializer.toBytes(record.key());
+        byte[] partitionKey = keySerializer == null ? null : keySerializer.toBytes(record.partitionKey());
+        byte[] value = valueSerializer == null ? null : valueSerializer.toBytes(record.value());
+        int numPartitions = partitioner == null ? 0 : this.cluster.partitionsFor(record.topic()).size();
+        int partition = partitioner == null ? 0 : partitioner.partition(record, key, partitionKey, value, this.cluster, numPartitions);
+        ProduceRequestResult result = new ProduceRequestResult();
+        RecordSend send = new RecordSend(0, result);
+        TopicPartition topicPartition = new TopicPartition(record.topic(), partition);
+        long offset = nextOffset(topicPartition);
+        Completion completion = new Completion(topicPartition, offset, send, result, callback);
+        this.sent.add(record);
+        if (autoComplete)
+            completion.complete(null);
+        else
+            this.completions.addLast(completion);
+        return send;
+    }
+
+    /**
+     * Get the next offset for this topic/partition
+     */
+    private long nextOffset(TopicPartition tp) {
+        Long offset = this.offsets.get(tp);
+        if (offset == null) {
+            this.offsets.put(tp, 1L);
+            return 0L;
+        } else {
+            Long next = offset + 1;
+            this.offsets.put(tp, next);
+            return offset;
+        }
+    }
+
+    public Map<String, Metric> metrics() {
+        return Collections.emptyMap();
+    }
+
+    /**
+     * "Closes" the producer
+     */
+    @Override
+    public void close() {
+    }
+
+    /**
+     * Get the list of sent records since the last call to {@link #clear()}
+     */
+    public synchronized List<ProducerRecord> history() {
+        return new ArrayList<ProducerRecord>(this.sent);
+    }
+
+    /**
+     * Clear the stored history of sent records
+     */
+    public synchronized void clear() {
+        this.sent.clear();
+        this.completions.clear();
+    }
+
+    /**
+     * Complete the earliest uncompleted call successfully.
+     * 
+     * @return true if there was an uncompleted call to complete
+     */
+    public synchronized boolean completeNext() {
+        return errorNext(null);
+    }
+
+    /**
+     * Complete the earliest uncompleted call with the given error.
+     * 
+     * @return true if there was an uncompleted call to complete
+     */
+    public synchronized boolean errorNext(RuntimeException e) {
+        Completion completion = this.completions.pollFirst();
+        if (completion != null) {
+            completion.complete(e);
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    private static class Completion {
+        private final long offset;
+        private final RecordSend send;
+        private final ProduceRequestResult result;
+        private final Callback callback;
+        private final TopicPartition topicPartition;
+
+        public Completion(TopicPartition topicPartition, long offset, RecordSend send, ProduceRequestResult result, Callback callback) {
+            this.send = send;
+            this.offset = offset;
+            this.result = result;
+            this.callback = callback;
+            this.topicPartition = topicPartition;
+        }
+
+        public void complete(RuntimeException e) {
+            result.done(topicPartition, e == null ? offset : -1L, e);
+            if (callback != null)
+                callback.onCompletion(send);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/clients/producer/Partitioner.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/clients/producer/Partitioner.java b/clients/src/main/java/kafka/clients/producer/Partitioner.java
new file mode 100644
index 0000000..1b8e51f
--- /dev/null
+++ b/clients/src/main/java/kafka/clients/producer/Partitioner.java
@@ -0,0 +1,30 @@
+package kafka.clients.producer;
+
+import kafka.common.Cluster;
+
+/**
+ * An interface by which clients can override the default partitioning behavior that maps records to topic partitions.
+ * <p>
+ * A partitioner can use either the original java object the user provided or the serialized bytes.
+ * <p>
+ * It is expected that the partitioner will make use the key for partitioning, but there is no requirement that an
+ * implementation do so. An implementation can use the key, the value, the state of the cluster, or any other side data.
+ */
+public interface Partitioner {
+
+    /**
+     * Compute the partition for the given record. This partition number must be in the range [0...numPartitions). The
+     * cluster state provided is the most up-to-date view that the client has but leadership can change at any time so
+     * there is no guarantee that the node that is the leader for a particular partition at the time the partition
+     * function is called will still be the leader by the time the request is sent.
+     * 
+     * @param record The record being sent
+     * @param key The serialized bytes of the key (null if no key is given or the serialized form is null)
+     * @param value The serialized bytes of the value (null if no value is given or the serialized form is null)
+     * @param cluster The current state of the cluster
+     * @param numPartitions The total number of partitions for the given topic
+     * @return The partition to send this record to
+     */
+    public int partition(ProducerRecord record, byte[] key, byte[] partitionKey, byte[] value, Cluster cluster, int numPartitions);
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/clients/producer/Producer.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/clients/producer/Producer.java b/clients/src/main/java/kafka/clients/producer/Producer.java
new file mode 100644
index 0000000..6ba6633
--- /dev/null
+++ b/clients/src/main/java/kafka/clients/producer/Producer.java
@@ -0,0 +1,38 @@
+package kafka.clients.producer;
+
+import java.util.Map;
+
+import kafka.common.Metric;
+
+/**
+ * The interface for the {@link KafkaProducer}
+ * 
+ * @see KafkaProducer
+ * @see MockProducer
+ */
+public interface Producer {
+
+    /**
+     * Send the given record asynchronously and return a future which will eventually contain the response information.
+     * 
+     * @param record The record to send
+     * @return A future which will eventually contain the response information
+     */
+    public RecordSend send(ProducerRecord record);
+
+    /**
+     * Send a message and invoke the given callback when the send is complete
+     */
+    public RecordSend send(ProducerRecord record, Callback callback);
+
+    /**
+     * Return a map of metrics maintained by the producer
+     */
+    public Map<String, ? extends Metric> metrics();
+
+    /**
+     * Close this producer
+     */
+    public void close();
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/clients/producer/ProducerConfig.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/kafka/clients/producer/ProducerConfig.java
new file mode 100644
index 0000000..9758293
--- /dev/null
+++ b/clients/src/main/java/kafka/clients/producer/ProducerConfig.java
@@ -0,0 +1,148 @@
+package kafka.clients.producer;
+
+import static kafka.common.config.ConfigDef.Range.atLeast;
+import static kafka.common.config.ConfigDef.Range.between;
+
+import java.util.Map;
+
+import kafka.common.config.AbstractConfig;
+import kafka.common.config.ConfigDef;
+import kafka.common.config.ConfigDef.Type;
+
+/**
+ * The producer configuration keys
+ */
+public class ProducerConfig extends AbstractConfig {
+
+    private static final ConfigDef config;
+
+    /**
+     * A list of URLs to use for establishing the initial connection to the cluster. This list should be in the form
+     * <code>host1:port1,host2:port2,...</code>. These urls are just used for the initial connection to discover the
+     * full cluster membership (which may change dynamically) so this list need not contain the full set of servers (you
+     * may want more than one, though, in case a server is down).
+     */
+    public static final String BROKER_LIST_CONFIG = "metadata.broker.list";
+
+    /**
+     * The amount of time to block waiting to fetch metadata about a topic the first time a message is sent to that
+     * topic.
+     */
+    public static final String METADATA_FETCH_TIMEOUT_CONFIG = "metadata.fetch.timeout.ms";
+
+    /**
+     * The buffer size allocated for a partition. When messages are received which are smaller than this size the
+     * producer will attempt to optimistically group them together until this size is reached.
+     */
+    public static final String MAX_PARTITION_SIZE_CONFIG = "max.partition.bytes";
+
+    /**
+     * The total memory used by the producer to buffer messages waiting to be sent to the server. If messages are sent
+     * faster than they can be delivered to the server the producer will either block or throw an exception based on the
+     * preference specified by {@link #BLOCK_ON_BUFFER_FULL}.
+     */
+    public static final String TOTAL_BUFFER_MEMORY_CONFIG = "total.memory.bytes";
+
+    /**
+     * The number of acknowledgments the producer requires from the server before considering a request complete.
+     */
+    public static final String REQUIRED_ACKS_CONFIG = "request.required.acks";
+
+    /**
+     * The maximum amount of time the server will wait for acknowledgments from followers to meet the acknowledgment
+     * requirements the producer has specified. If the requested number of acknowledgments are not met an error will be
+     * returned.
+     */
+    public static final String REQUEST_TIMEOUT_CONFIG = "request.timeout.ms";
+
+    /**
+     * The producer groups together any messages that arrive in between request sends. Normally this occurs only under
+     * load when messages arrive faster than they can be sent out. However the client can reduce the number of requests
+     * and increase throughput by adding a small amount of artificial delay to force more messages to batch together.
+     * This setting gives an upper bound on this delay. If we get {@link #MAX_PARTITION_SIZE_CONFIG} worth of messages
+     * for a partition it will be sent immediately regardless of this setting, however if we have fewer than this many
+     * bytes accumulated for this partition we will "linger" for the specified time waiting for more messages to show
+     * up. This setting defaults to 0.
+     */
+    public static final String LINGER_MS_CONFIG = "linger.ms";
+
+    /**
+     * The fully qualified name of the {@link kafka.common.Serializer} class to use for serializing record values.
+     */
+    public static final String VALUE_SERIALIZER_CLASS_CONFIG = "value.serializer.class";
+
+    /**
+     * The fully qualified name of the {@link kafka.common.Serializer} class to use for serializing record keys.
+     */
+    public static final String KEY_SERIALIZER_CLASS_CONFIG = "key.serializer.class";
+
+    /**
+     * The class to use for choosing a partition to send the message to
+     */
+    public static final String PARTITIONER_CLASS_CONFIG = "partitioner.class";
+
+    /**
+     * Force a refresh of the cluster metadata after this period of time. This ensures that changes to the number of
+     * partitions or other settings will by taken up by producers without restart.
+     */
+    public static final String METADATA_REFRESH_MS_CONFIG = "topic.metadata.refresh.interval.ms";
+
+    /**
+     * The id string to pass to the server when making requests. The purpose of this is to be able to track the source
+     * of requests beyond just ip/port by allowing a logical application name to be included.
+     */
+    public static final String CLIENT_ID_CONFIG = "client.id";
+
+    /**
+     * The size of the TCP send buffer to use when sending data
+     */
+    public static final String SEND_BUFFER_CONFIG = "send.buffer.bytes";
+
+    /**
+     * The maximum size of a request. This is also effectively a cap on the maximum message size. Note that the server
+     * has its own cap on message size which may be different from this.
+     */
+    public static final String MAX_REQUEST_SIZE_CONFIG = "max.request.size";
+
+    /**
+     * The amount of time to wait before attempting to reconnect to a given host. This avoids repeated connecting to a
+     * host in a tight loop.
+     */
+    public static final String RECONNECT_BACKOFF_MS_CONFIG = "reconnect.backoff.ms";
+
+    /**
+     * When our memory buffer is exhausted we must either stop accepting new messages (block) or throw errors. By
+     * default this setting is true and we block, however users who want to guarantee we never block can turn this into
+     * an error.
+     */
+    public static final String BLOCK_ON_BUFFER_FULL = "block.on.buffer.full";
+
+    public static final String ENABLE_JMX = "enable.jmx";
+
+    static {
+        /* TODO: add docs */
+        config = new ConfigDef().define(BROKER_LIST_CONFIG, Type.LIST, "blah blah")
+                                .define(METADATA_FETCH_TIMEOUT_CONFIG, Type.LONG, 60 * 1000, atLeast(0), "blah blah")
+                                .define(MAX_PARTITION_SIZE_CONFIG, Type.INT, 16384, atLeast(0), "blah blah")
+                                .define(TOTAL_BUFFER_MEMORY_CONFIG, Type.LONG, 32 * 1024 * 1024L, atLeast(0L), "blah blah")
+                                /* TODO: should be a string to handle acks=in-sync */
+                                .define(REQUIRED_ACKS_CONFIG, Type.INT, 1, between(-1, Short.MAX_VALUE), "blah blah")
+                                .define(REQUEST_TIMEOUT_CONFIG, Type.INT, 30 * 1000, atLeast(0), "blah blah")
+                                .define(LINGER_MS_CONFIG, Type.LONG, 0, atLeast(0L), "blah blah")
+                                .define(VALUE_SERIALIZER_CLASS_CONFIG, Type.CLASS, "blah blah")
+                                .define(KEY_SERIALIZER_CLASS_CONFIG, Type.CLASS, "blah blah")
+                                .define(PARTITIONER_CLASS_CONFIG, Type.CLASS, DefaultPartitioner.class.getName(), "blah blah")
+                                .define(METADATA_REFRESH_MS_CONFIG, Type.LONG, 10 * 60 * 1000, atLeast(-1L), "blah blah")
+                                .define(CLIENT_ID_CONFIG, Type.STRING, "", "blah blah")
+                                .define(SEND_BUFFER_CONFIG, Type.INT, 128 * 1024, atLeast(0), "blah blah")
+                                .define(MAX_REQUEST_SIZE_CONFIG, Type.INT, 1 * 1024 * 1024, atLeast(0), "blah blah")
+                                .define(RECONNECT_BACKOFF_MS_CONFIG, Type.LONG, 10L, atLeast(0L), "blah blah")
+                                .define(BLOCK_ON_BUFFER_FULL, Type.BOOLEAN, true, "blah blah")
+                                .define(ENABLE_JMX, Type.BOOLEAN, true, "");
+    }
+
+    ProducerConfig(Map<? extends Object, ? extends Object> props) {
+        super(config, props);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/clients/producer/ProducerRecord.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/clients/producer/ProducerRecord.java b/clients/src/main/java/kafka/clients/producer/ProducerRecord.java
new file mode 100644
index 0000000..5fddbef
--- /dev/null
+++ b/clients/src/main/java/kafka/clients/producer/ProducerRecord.java
@@ -0,0 +1,84 @@
+package kafka.clients.producer;
+
+/**
+ * An unserialized key/value pair to be sent to Kafka. This consists of a topic name to which the record is being sent,
+ * a value (which can be null) which is the contents of the record and an optional key (which can also be null). In
+ * cases the key used for choosing a partition is going to be different the user can specify a partition key which will
+ * be used only for computing the partition to which this record will be sent and will not be retained with the record.
+ */
+public final class ProducerRecord {
+
+    private final String topic;
+    private final Object key;
+    private final Object partitionKey;
+    private final Object value;
+
+    /**
+     * Creates a record to be sent to Kafka using a special override key for partitioning that is different form the key
+     * retained in the record
+     * 
+     * @param topic The topic the record will be appended to
+     * @param key The key that will be included in the record
+     * @param partitionKey An override for the key to be used only for partitioning purposes in the client. This key
+     *        will not be retained or available to downstream consumers.
+     * @param value The record contents
+     */
+    public ProducerRecord(String topic, Object key, Object partitionKey, Object value) {
+        if (topic == null)
+            throw new IllegalArgumentException("Topic cannot be null");
+        this.topic = topic;
+        this.key = key;
+        this.partitionKey = partitionKey;
+        this.value = value;
+    }
+
+    /**
+     * Create a record to be sent to Kafka
+     * 
+     * @param topic The topic the record will be appended to
+     * @param key The key that will be included in the record
+     * @param value The record contents
+     */
+    public ProducerRecord(String topic, Object key, Object value) {
+        this(topic, key, key, value);
+    }
+
+    /**
+     * Create a record with no key
+     * 
+     * @param topic The topic this record should be sent to
+     * @param value The record contents
+     */
+    public ProducerRecord(String topic, Object value) {
+        this(topic, null, value);
+    }
+
+    /**
+     * The topic this record is being sent to
+     */
+    public String topic() {
+        return topic;
+    }
+
+    /**
+     * The key (or null if no key is specified)
+     */
+    public Object key() {
+        return key;
+    }
+
+    /**
+     * An override key to use instead of the main record key
+     */
+    public Object partitionKey() {
+        return partitionKey;
+    }
+
+    /**
+     * @return The value
+     */
+    public Object value() {
+        return value;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/clients/producer/RecordSend.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/clients/producer/RecordSend.java b/clients/src/main/java/kafka/clients/producer/RecordSend.java
new file mode 100644
index 0000000..1883dab
--- /dev/null
+++ b/clients/src/main/java/kafka/clients/producer/RecordSend.java
@@ -0,0 +1,88 @@
+package kafka.clients.producer;
+
+import java.util.concurrent.TimeUnit;
+
+import kafka.clients.producer.internals.ProduceRequestResult;
+import kafka.common.errors.ApiException;
+import kafka.common.errors.TimeoutException;
+
+/**
+ * An asynchronously computed response from sending a record. Calling <code>await()</code> or most of the other accessor
+ * methods will block until the response for this record is available. If you wish to avoid blocking provide a
+ * {@link kafka.clients.producer.Callback Callback} with the record send.
+ */
+public final class RecordSend {
+
+    private final long relativeOffset;
+    private final ProduceRequestResult result;
+
+    public RecordSend(long relativeOffset, ProduceRequestResult result) {
+        this.relativeOffset = relativeOffset;
+        this.result = result;
+    }
+
+    /**
+     * Block until this send has completed successfully. If the request fails, throw the error that occurred in sending
+     * the request.
+     * @return the same object for chaining of calls
+     * @throws TimeoutException if the thread is interrupted while waiting
+     * @throws ApiException if the request failed.
+     */
+    public RecordSend await() {
+        result.await();
+        if (result.error() != null)
+            throw result.error();
+        return this;
+    }
+
+    /**
+     * Block until this send is complete or the given timeout elapses
+     * @param timeout the time to wait
+     * @param unit the units of the time given
+     * @return the same object for chaining
+     * @throws TimeoutException if the request isn't satisfied in the time period given or the thread is interrupted
+     *         while waiting
+     * @throws ApiException if the request failed.
+     */
+    public RecordSend await(long timeout, TimeUnit unit) {
+        boolean success = result.await(timeout, unit);
+        if (!success)
+            throw new TimeoutException("Request did not complete after " + timeout + " " + unit);
+        if (result.error() != null)
+            throw result.error();
+        return this;
+    }
+
+    /**
+     * Get the offset for the given message. This method will block until the request is complete and will throw an
+     * exception if the request fails.
+     * @return The offset
+     */
+    public long offset() {
+        await();
+        return this.result.baseOffset() + this.relativeOffset;
+    }
+
+    /**
+     * Check if the request is complete without blocking
+     */
+    public boolean completed() {
+        return this.result.completed();
+    }
+
+    /**
+     * Block on request completion and return true if there was an error.
+     */
+    public boolean hasError() {
+        result.await();
+        return this.result.error() != null;
+    }
+
+    /**
+     * Return the error thrown
+     */
+    public Exception error() {
+        result.await();
+        return this.result.error();
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/clients/producer/internals/BufferPool.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/clients/producer/internals/BufferPool.java b/clients/src/main/java/kafka/clients/producer/internals/BufferPool.java
new file mode 100644
index 0000000..c222ca0
--- /dev/null
+++ b/clients/src/main/java/kafka/clients/producer/internals/BufferPool.java
@@ -0,0 +1,223 @@
+package kafka.clients.producer.internals;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.Deque;
+import java.util.concurrent.locks.Condition;
+import java.util.concurrent.locks.ReentrantLock;
+
+import kafka.clients.producer.BufferExhaustedException;
+
+/**
+ * A pool of ByteBuffers kept under a given memory limit. This class is fairly specific to the needs of the producer. In
+ * particular it has the following properties:
+ * <ol>
+ * <li>There is a special "poolable size" and buffers of this size are kept in a free list and recycled
+ * <li>It is fair. That is all memory is given to the longest waiting thread until it has sufficient memory. This
+ * prevents starvation or deadlock when a thread asks for a large chunk of memory and needs to block until multiple
+ * buffers are deallocated.
+ * </ol>
+ */
+public final class BufferPool {
+
+    private final long totalMemory;
+    private final int poolableSize;
+    private final boolean blockOnExhaustion;
+    private final ReentrantLock lock;
+    private final Deque<ByteBuffer> free;
+    private final Deque<Condition> waiters;
+    private long availableMemory;
+
+    /**
+     * Create a new buffer pool
+     * 
+     * @param memory The maximum amount of memory that this buffer pool can allocate
+     * @param poolableSize The buffer size to cache in the free list rather than deallocating
+     * @param blockOnExhaustion This controls the behavior when the buffer pool is out of memory. If true the
+     *        {@link #allocate(int)} call will block and wait for memory to be returned to the pool. If false
+     *        {@link #allocate(int)} will throw an exception if the buffer is out of memory.
+     */
+    public BufferPool(long memory, int poolableSize, boolean blockOnExhaustion) {
+        this.poolableSize = poolableSize;
+        this.blockOnExhaustion = blockOnExhaustion;
+        this.lock = new ReentrantLock();
+        this.free = new ArrayDeque<ByteBuffer>();
+        this.waiters = new ArrayDeque<Condition>();
+        this.totalMemory = memory;
+        this.availableMemory = memory;
+    }
+
+    /**
+     * Allocate a buffer of the given size
+     * 
+     * @param size The buffer size to allocate in bytes
+     * @return The buffer
+     * @throws InterruptedException If the thread is interrupted while blocked
+     * @throws IllegalArgument if size is larger than the total memory controlled by the pool (and hence we would block
+     *         forever)
+     * @throws BufferExhaustedException if the pool is in non-blocking mode and size exceeds the free memory in the pool
+     */
+    public ByteBuffer allocate(int size) throws InterruptedException {
+        if (size > this.totalMemory)
+            throw new IllegalArgumentException("Attempt to allocate " + size
+                                               + " bytes, but there is a hard limit of "
+                                               + this.totalMemory
+                                               + " on memory allocations.");
+
+        this.lock.lock();
+        try {
+            // check if we have a free buffer of the right size pooled
+            if (size == poolableSize && !this.free.isEmpty())
+                return this.free.pollFirst();
+
+            // now check if the request is immediately satisfiable with the
+            // memory on hand or if we need to block
+            int freeListSize = this.free.size() * this.poolableSize;
+            if (this.availableMemory + freeListSize >= size) {
+                // we have enough unallocated or pooled memory to immediately
+                // satisfy the request
+                freeUp(size);
+                this.availableMemory -= size;
+                lock.unlock();
+                return ByteBuffer.allocate(size);
+            } else if (!blockOnExhaustion) {
+                throw new BufferExhaustedException("You have exhausted the " + this.totalMemory
+                                                   + " bytes of memory you configured for the client and the client is configured to error"
+                                                   + " rather than block when memory is exhausted.");
+            } else {
+                // we are out of memory and will have to block
+                int accumulated = 0;
+                ByteBuffer buffer = null;
+                Condition moreMemory = this.lock.newCondition();
+                this.waiters.addLast(moreMemory);
+                // loop over and over until we have a buffer or have reserved
+                // enough memory to allocate one
+                while (accumulated < size) {
+                    moreMemory.await();
+                    // check if we can satisfy this request from the free list,
+                    // otherwise allocate memory
+                    if (accumulated == 0 && size == this.poolableSize && !this.free.isEmpty()) {
+                        // just grab a buffer from the free list
+                        buffer = this.free.pollFirst();
+                        accumulated = size;
+                    } else {
+                        // we'll need to allocate memory, but we may only get
+                        // part of what we need on this iteration
+                        freeUp(size - accumulated);
+                        int got = (int) Math.min(size - accumulated, this.availableMemory);
+                        this.availableMemory -= got;
+                        accumulated += got;
+                    }
+                }
+
+                // remove the condition for this thread to let the next thread
+                // in line start getting memory
+                Condition removed = this.waiters.removeFirst();
+                if (removed != moreMemory)
+                    throw new IllegalStateException("Wrong condition: this shouldn't happen.");
+
+                // signal any additional waiters if there is more memory left
+                // over for them
+                if (this.availableMemory > 0 || !this.free.isEmpty()) {
+                    if (!this.waiters.isEmpty())
+                        this.waiters.peekFirst().signal();
+                }
+
+                // unlock and return the buffer
+                lock.unlock();
+                if (buffer == null)
+                    return ByteBuffer.allocate(size);
+                else
+                    return buffer;
+            }
+        } finally {
+            if (lock.isHeldByCurrentThread())
+                lock.unlock();
+        }
+    }
+
+    /**
+     * Attempt to ensure we have at least the requested number of bytes of memory for allocation by deallocating pooled
+     * buffers (if needed)
+     */
+    private void freeUp(int size) {
+        while (!this.free.isEmpty() && this.availableMemory < size)
+            this.availableMemory += this.free.pollLast().capacity();
+    }
+
+    /**
+     * Return buffers to the pool. If they are of the poolable size add them to the free list, otherwise just mark the
+     * memory as free.
+     * 
+     * @param buffers The buffers to return
+     */
+    public void deallocate(ByteBuffer... buffers) {
+        lock.lock();
+        try {
+            for (int i = 0; i < buffers.length; i++) {
+                int size = buffers[i].capacity();
+                if (size == this.poolableSize) {
+                    buffers[i].clear();
+                    this.free.add(buffers[i]);
+                } else {
+                    this.availableMemory += size;
+                }
+                Condition moreMem = this.waiters.peekFirst();
+                if (moreMem != null)
+                    moreMem.signal();
+            }
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * the total free memory both unallocated and in the free list
+     */
+    public long availableMemory() {
+        lock.lock();
+        try {
+            return this.availableMemory + this.free.size() * this.poolableSize;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * Get the unallocated memory (not in the free list or in use)
+     */
+    public long unallocatedMemory() {
+        lock.lock();
+        try {
+            return this.availableMemory;
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * The number of threads blocked waiting on memory
+     */
+    public int queued() {
+        lock.lock();
+        try {
+            return this.waiters.size();
+        } finally {
+            lock.unlock();
+        }
+    }
+
+    /**
+     * The buffer size that will be retained in the free list after use
+     */
+    public int poolableSize() {
+        return this.poolableSize;
+    }
+
+    /**
+     * The total memory managed by this pool
+     */
+    public long totalMemory() {
+        return this.totalMemory;
+    }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/clients/producer/internals/Metadata.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/clients/producer/internals/Metadata.java b/clients/src/main/java/kafka/clients/producer/internals/Metadata.java
new file mode 100644
index 0000000..f5f8b9b
--- /dev/null
+++ b/clients/src/main/java/kafka/clients/producer/internals/Metadata.java
@@ -0,0 +1,120 @@
+package kafka.clients.producer.internals;
+
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import kafka.common.Cluster;
+import kafka.common.PartitionInfo;
+import kafka.common.errors.TimeoutException;
+
+/**
+ * A class encapsulating some of the logic around metadata.
+ * <p>
+ * This class is shared by the client thread (for partitioning) and the background sender thread.
+ * 
+ * Metadata is maintained for only a subset of topics, which can be added to over time. When we request metdata for a
+ * topic we don't have any metadata for it will trigger a metadata update.
+ */
+public final class Metadata {
+
+    private final long refreshBackoffMs;
+    private final long metadataExpireMs;
+    private long lastRefresh;
+    private Cluster cluster;
+    private boolean forceUpdate;
+    private final Set<String> topics;
+
+    /**
+     * Create a metadata instance with reasonable defaults
+     */
+    public Metadata() {
+        this(100L, 60 * 60 * 1000L);
+    }
+
+    /**
+     * Create a new Metadata instance
+     * @param refreshBackoffMs The minimum amount of time that must expire between metadata refreshes to avoid busy
+     *        polling
+     * @param metadataExpireMs The maximum amount of time that metadata can be retained without refresh
+     */
+    public Metadata(long refreshBackoffMs, long metadataExpireMs) {
+        this.refreshBackoffMs = refreshBackoffMs;
+        this.metadataExpireMs = metadataExpireMs;
+        this.lastRefresh = 0L;
+        this.cluster = Cluster.empty();
+        this.forceUpdate = false;
+        this.topics = new HashSet<String>();
+    }
+
+    /**
+     * Get the current cluster info without blocking
+     */
+    public synchronized Cluster fetch() {
+        return this.cluster;
+    }
+
+    /**
+     * Fetch cluster metadata including partitions for the given topic. If there is no metadata for the given topic,
+     * block waiting for an update.
+     * @param topic The topic we want metadata for
+     * @param maxWaitMs The maximum amount of time to block waiting for metadata
+     */
+    public synchronized Cluster fetch(String topic, long maxWaitMs) {
+        List<PartitionInfo> partitions = null;
+        do {
+            partitions = cluster.partitionsFor(topic);
+            if (partitions == null) {
+                long begin = System.currentTimeMillis();
+                topics.add(topic);
+                forceUpdate = true;
+                try {
+                    wait(maxWaitMs);
+                } catch (InterruptedException e) { /* this is fine, just try again */
+                }
+                long ellapsed = System.currentTimeMillis() - begin;
+                if (ellapsed > maxWaitMs)
+                    throw new TimeoutException("Failed to update metadata after " + maxWaitMs + " ms.");
+            } else {
+                return cluster;
+            }
+        } while (true);
+    }
+
+    /**
+     * Does the current cluster info need to be updated? An update is needed if it has been at least refreshBackoffMs
+     * since our last update and either (1) an update has been requested or (2) the current metadata has expired (more
+     * than metadataExpireMs has passed since the last refresh)
+     */
+    public synchronized boolean needsUpdate(long now) {
+        long msSinceLastUpdate = now - this.lastRefresh;
+        boolean updateAllowed = msSinceLastUpdate >= this.refreshBackoffMs;
+        boolean updateNeeded = this.forceUpdate || msSinceLastUpdate >= this.metadataExpireMs;
+        return updateAllowed && updateNeeded;
+    }
+
+    /**
+     * Force an update of the current cluster info
+     */
+    public synchronized void forceUpdate() {
+        this.forceUpdate = true;
+    }
+
+    /**
+     * Get the list of topics we are currently maintaining metadata for
+     */
+    public synchronized Set<String> topics() {
+        return new HashSet<String>(this.topics);
+    }
+
+    /**
+     * Update the cluster metadata
+     */
+    public synchronized void update(Cluster cluster, long now) {
+        this.forceUpdate = false;
+        this.lastRefresh = now;
+        this.cluster = cluster;
+        notifyAll();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/clients/producer/internals/ProduceRequestResult.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/clients/producer/internals/ProduceRequestResult.java b/clients/src/main/java/kafka/clients/producer/internals/ProduceRequestResult.java
new file mode 100644
index 0000000..1049b61
--- /dev/null
+++ b/clients/src/main/java/kafka/clients/producer/internals/ProduceRequestResult.java
@@ -0,0 +1,83 @@
+package kafka.clients.producer.internals;
+
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.TimeUnit;
+
+import kafka.clients.producer.RecordSend;
+import kafka.common.TopicPartition;
+import kafka.common.errors.TimeoutException;
+
+/**
+ * A class that models the future completion of a produce request for a single partition. There is one of these per
+ * partition in a produce request and it is shared by all the {@link RecordSend} instances that are batched together for
+ * the same partition in the request.
+ */
+public final class ProduceRequestResult {
+
+    private final CountDownLatch latch = new CountDownLatch(1);
+    private volatile TopicPartition topicPartition;
+    private volatile long baseOffset = -1L;
+    private volatile RuntimeException error;
+
+    public ProduceRequestResult() {
+    }
+
+    /**
+     * Mark this request as complete and unblock any threads waiting on its completion.
+     * @param topicPartition The topic and partition to which this record set was sent was sent
+     * @param baseOffset The base offset assigned to the message
+     * @param error The error that occurred if there was one, or null.
+     */
+    public void done(TopicPartition topicPartition, long baseOffset, RuntimeException error) {
+        this.topicPartition = topicPartition;
+        this.baseOffset = baseOffset;
+        this.error = error;
+        this.latch.countDown();
+    }
+
+    /**
+     * Await the completion of this request
+     */
+    public void await() {
+        try {
+            latch.await();
+        } catch (InterruptedException e) {
+            throw new TimeoutException("Interrupted while waiting for request to complete.");
+        }
+    }
+
+    /**
+     * Await the completion of this request (up to the given time interval)
+     * @param timeout The maximum time to wait
+     * @param unit The unit for the max time
+     * @return true if the request completed, false if we timed out
+     */
+    public boolean await(long timeout, TimeUnit unit) {
+        try {
+            return latch.await(timeout, unit);
+        } catch (InterruptedException e) {
+            throw new TimeoutException("Interrupted while waiting for request to complete.");
+        }
+    }
+
+    /**
+     * The base offset for the request (the first offset in the message set)
+     */
+    public long baseOffset() {
+        return baseOffset;
+    }
+
+    /**
+     * The error thrown (generally on the server) while processing this request
+     */
+    public RuntimeException error() {
+        return error;
+    }
+
+    /**
+     * Has the request completed?
+     */
+    public boolean completed() {
+        return this.latch.getCount() == 0L;
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/clients/producer/internals/RecordAccumulator.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/kafka/clients/producer/internals/RecordAccumulator.java
new file mode 100644
index 0000000..a2b536c
--- /dev/null
+++ b/clients/src/main/java/kafka/clients/producer/internals/RecordAccumulator.java
@@ -0,0 +1,236 @@
+package kafka.clients.producer.internals;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayDeque;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.Deque;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.ConcurrentMap;
+
+import kafka.clients.producer.Callback;
+import kafka.clients.producer.RecordSend;
+import kafka.common.TopicPartition;
+import kafka.common.metrics.Measurable;
+import kafka.common.metrics.MetricConfig;
+import kafka.common.metrics.Metrics;
+import kafka.common.record.CompressionType;
+import kafka.common.record.MemoryRecords;
+import kafka.common.record.Record;
+import kafka.common.record.Records;
+import kafka.common.utils.CopyOnWriteMap;
+import kafka.common.utils.Time;
+import kafka.common.utils.Utils;
+
+/**
+ * This class acts as a queue that accumulates records into {@link kafka.common.record.MemoryRecords} instances to be
+ * sent to the server.
+ * <p>
+ * The accumulator uses a bounded amount of memory and append calls will block when that memory is exhausted, unless
+ * this behavior is explicitly disabled.
+ */
+public final class RecordAccumulator {
+
+    private volatile boolean closed;
+    private int drainIndex;
+    private final int batchSize;
+    private final long lingerMs;
+    private final ConcurrentMap<TopicPartition, Deque<RecordBatch>> batches;
+    private final BufferPool free;
+    private final Time time;
+
+    /**
+     * Create a new record accumulator
+     * 
+     * @param batchSize The size to use when allocating {@link kafka.common.record.MemoryRecords} instances
+     * @param totalSize The maximum memory the record accumulator can use.
+     * @param lingerMs An artificial delay time to add before declaring a records instance that isn't full ready for
+     *        sending. This allows time for more records to arrive. Setting a non-zero lingerMs will trade off some
+     *        latency for potentially better throughput due to more batching (and hence fewer, larger requests).
+     * @param blockOnBufferFull If true block when we are out of memory; if false throw an exception when we are out of
+     *        memory
+     * @param metrics The metrics
+     * @param time The time instance to use
+     */
+    public RecordAccumulator(int batchSize, long totalSize, long lingerMs, boolean blockOnBufferFull, Metrics metrics, Time time) {
+        this.drainIndex = 0;
+        this.closed = false;
+        this.batchSize = batchSize;
+        this.lingerMs = lingerMs;
+        this.batches = new CopyOnWriteMap<TopicPartition, Deque<RecordBatch>>();
+        this.free = new BufferPool(totalSize, batchSize, blockOnBufferFull);
+        this.time = time;
+        registerMetrics(metrics);
+    }
+
+    private void registerMetrics(Metrics metrics) {
+        metrics.addMetric("blocked_threads",
+                          "The number of user threads blocked waiting for buffer memory to enqueue their messages",
+                          new Measurable() {
+                              public double measure(MetricConfig config, long now) {
+                                  return free.queued();
+                              }
+                          });
+        metrics.addMetric("buffer_total_bytes",
+                          "The total amount of buffer memory that is available (not currently used for buffering messages).",
+                          new Measurable() {
+                              public double measure(MetricConfig config, long now) {
+                                  return free.totalMemory();
+                              }
+                          });
+        metrics.addMetric("buffer_available_bytes",
+                          "The total amount of buffer memory that is available (not currently used for buffering messages).",
+                          new Measurable() {
+                              public double measure(MetricConfig config, long now) {
+                                  return free.availableMemory();
+                              }
+                          });
+    }
+
+    /**
+     * Add a record to the accumulator.
+     * <p>
+     * This method will block if sufficient memory isn't available for the record unless blocking has been disabled.
+     * 
+     * @param tp The topic/partition to which this record is being sent
+     * @param key The key for the record
+     * @param value The value for the record
+     * @param compression The compression codec for the record
+     * @param callback The user-supplied callback to execute when the request is complete
+     */
+    public RecordSend append(TopicPartition tp, byte[] key, byte[] value, CompressionType compression, Callback callback) throws InterruptedException {
+        if (closed)
+            throw new IllegalStateException("Cannot send after the producer is closed.");
+        // check if we have an in-progress batch
+        Deque<RecordBatch> dq = dequeFor(tp);
+        synchronized (dq) {
+            RecordBatch batch = dq.peekLast();
+            if (batch != null) {
+                RecordSend send = batch.tryAppend(key, value, compression, callback);
+                if (send != null)
+                    return send;
+            }
+        }
+
+        // we don't have an in-progress record batch try to allocate a new batch
+        int size = Math.max(this.batchSize, Records.LOG_OVERHEAD + Record.recordSize(key, value));
+        ByteBuffer buffer = free.allocate(size);
+        synchronized (dq) {
+            RecordBatch first = dq.peekLast();
+            if (first != null) {
+                RecordSend send = first.tryAppend(key, value, compression, callback);
+                if (send != null) {
+                    // somebody else found us a batch, return the one we waited for!
+                    // Hopefully this doesn't happen
+                    // often...
+                    free.deallocate(buffer);
+                    return send;
+                }
+            }
+            RecordBatch batch = new RecordBatch(tp, new MemoryRecords(buffer), time.milliseconds());
+            RecordSend send = Utils.notNull(batch.tryAppend(key, value, compression, callback));
+            dq.addLast(batch);
+            return send;
+        }
+    }
+
+    /**
+     * Get a list of topic-partitions which are ready to be sent.
+     * <p>
+     * A partition is ready if ANY of the following are true:
+     * <ol>
+     * <li>The record set is full
+     * <li>The record set has sat in the accumulator for at least lingerMs milliseconds
+     * <li>The accumulator is out of memory and threads are blocking waiting for data (in this case all partitions are
+     * immediately considered ready).
+     * <li>The accumulator has been closed
+     * </ol>
+     */
+    public List<TopicPartition> ready(long now) {
+        List<TopicPartition> ready = new ArrayList<TopicPartition>();
+        boolean exhausted = this.free.queued() > 0;
+        for (Map.Entry<TopicPartition, Deque<RecordBatch>> entry : this.batches.entrySet()) {
+            Deque<RecordBatch> deque = entry.getValue();
+            synchronized (deque) {
+                RecordBatch batch = deque.peekFirst();
+                if (batch != null) {
+                    boolean full = deque.size() > 1 || !batch.records.buffer().hasRemaining();
+                    boolean expired = now - batch.created >= lingerMs;
+                    if (full | expired | exhausted | closed)
+                        ready.add(batch.topicPartition);
+                }
+            }
+        }
+        return ready;
+    }
+
+    /**
+     * Drain all the data for the given topic-partitions that will fit within the specified size. This method attempts
+     * to avoid choosing the same topic-partitions over and over.
+     * 
+     * @param partitions The list of partitions to drain
+     * @param maxSize The maximum number of bytes to drain
+     * @return A list of {@link RecordBatch} for partitions specified with total size less than the requested maxSize.
+     *         TODO: There may be a starvation issue due to iteration order
+     */
+    public List<RecordBatch> drain(List<TopicPartition> partitions, int maxSize) {
+        if (partitions.isEmpty())
+            return Collections.emptyList();
+        int size = 0;
+        List<RecordBatch> ready = new ArrayList<RecordBatch>();
+        /* to make starvation less likely this loop doesn't start at 0 */
+        int start = drainIndex = drainIndex % partitions.size();
+        do {
+            TopicPartition tp = partitions.get(drainIndex);
+            Deque<RecordBatch> deque = dequeFor(tp);
+            if (deque != null) {
+                synchronized (deque) {
+                    if (size + deque.peekFirst().records.sizeInBytes() > maxSize) {
+                        return ready;
+                    } else {
+                        RecordBatch batch = deque.pollFirst();
+                        size += batch.records.sizeInBytes();
+                        ready.add(batch);
+                    }
+                }
+            }
+            this.drainIndex = (this.drainIndex + 1) % partitions.size();
+        } while (start != drainIndex);
+        return ready;
+    }
+
+    /**
+     * Get the deque for the given topic-partition, creating it if necessary. Since new topics will only be added rarely
+     * we copy-on-write the hashmap
+     */
+    private Deque<RecordBatch> dequeFor(TopicPartition tp) {
+        Deque<RecordBatch> d = this.batches.get(tp);
+        if (d != null)
+            return d;
+        this.batches.putIfAbsent(tp, new ArrayDeque<RecordBatch>());
+        return this.batches.get(tp);
+    }
+
+    /**
+     * Deallocate the list of record batches
+     */
+    public void deallocate(Collection<RecordBatch> batches) {
+        ByteBuffer[] buffers = new ByteBuffer[batches.size()];
+        int i = 0;
+        for (RecordBatch batch : batches) {
+            buffers[i] = batch.records.buffer();
+            i++;
+        }
+        free.deallocate(buffers);
+    }
+
+    /**
+     * Close this accumulator and force all the record buffers to be drained
+     */
+    public void close() {
+        this.closed = true;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/clients/producer/internals/RecordBatch.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/clients/producer/internals/RecordBatch.java b/clients/src/main/java/kafka/clients/producer/internals/RecordBatch.java
new file mode 100644
index 0000000..4a536a2
--- /dev/null
+++ b/clients/src/main/java/kafka/clients/producer/internals/RecordBatch.java
@@ -0,0 +1,84 @@
+package kafka.clients.producer.internals;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import kafka.clients.producer.Callback;
+import kafka.clients.producer.RecordSend;
+import kafka.common.TopicPartition;
+import kafka.common.record.CompressionType;
+import kafka.common.record.MemoryRecords;
+
+/**
+ * A batch of records that is or will be sent.
+ * 
+ * This class is not thread safe and external synchronization must be used when modifying it
+ */
+public final class RecordBatch {
+    public int recordCount = 0;
+    public final long created;
+    public final MemoryRecords records;
+    public final TopicPartition topicPartition;
+    private final ProduceRequestResult produceFuture;
+    private final List<Thunk> thunks;
+
+    public RecordBatch(TopicPartition tp, MemoryRecords records, long now) {
+        this.created = now;
+        this.records = records;
+        this.topicPartition = tp;
+        this.produceFuture = new ProduceRequestResult();
+        this.thunks = new ArrayList<Thunk>();
+    }
+
+    /**
+     * Append the message to the current message set and return the relative offset within that message set
+     * 
+     * @return The RecordSend corresponding to this message or null if there isn't sufficient room.
+     */
+    public RecordSend tryAppend(byte[] key, byte[] value, CompressionType compression, Callback callback) {
+        if (!this.records.hasRoomFor(key, value)) {
+            return null;
+        } else {
+            this.records.append(0L, key, value, compression);
+            RecordSend send = new RecordSend(this.recordCount++, this.produceFuture);
+            if (callback != null)
+                thunks.add(new Thunk(callback, send));
+            return send;
+        }
+    }
+
+    /**
+     * Complete the request
+     * 
+     * @param offset The offset
+     * @param errorCode The error code or 0 if no error
+     */
+    public void done(long offset, RuntimeException exception) {
+        this.produceFuture.done(topicPartition, offset, exception);
+        // execute callbacks
+        for (int i = 0; i < this.thunks.size(); i++) {
+            try {
+                this.thunks.get(i).execute();
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
+    }
+
+    /**
+     * A callback and the associated RecordSend argument to pass to it.
+     */
+    final private static class Thunk {
+        final Callback callback;
+        final RecordSend send;
+
+        public Thunk(Callback callback, RecordSend send) {
+            this.callback = callback;
+            this.send = send;
+        }
+
+        public void execute() {
+            this.callback.onCompletion(this.send);
+        }
+    }
+}
\ No newline at end of file


[4/7] KAFKA-1227 New producer!

Posted by jk...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/protocol/types/Field.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/protocol/types/Field.java b/clients/src/main/java/kafka/common/protocol/types/Field.java
new file mode 100644
index 0000000..d018a12
--- /dev/null
+++ b/clients/src/main/java/kafka/common/protocol/types/Field.java
@@ -0,0 +1,48 @@
+package kafka.common.protocol.types;
+
+/**
+ * A field in a schema
+ */
+public class Field {
+
+    public static final Object NO_DEFAULT = new Object();
+
+    final int index;
+    public final String name;
+    public final Type type;
+    public final Object defaultValue;
+    public final String doc;
+    final Schema schema;
+
+    public Field(int index, String name, Type type, String doc, Object defaultValue, Schema schema) {
+        this.index = index;
+        this.name = name;
+        this.type = type;
+        this.doc = doc;
+        this.defaultValue = defaultValue;
+        this.schema = schema;
+        if (defaultValue != NO_DEFAULT)
+            type.validate(defaultValue);
+    }
+
+    public Field(int index, String name, Type type, String doc, Object defaultValue) {
+        this(index, name, type, doc, defaultValue, null);
+    }
+
+    public Field(String name, Type type, String doc, Object defaultValue) {
+        this(-1, name, type, doc, defaultValue);
+    }
+
+    public Field(String name, Type type, String doc) {
+        this(name, type, doc, NO_DEFAULT);
+    }
+
+    public Field(String name, Type type) {
+        this(name, type, "");
+    }
+
+    public Type type() {
+        return type;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/protocol/types/Schema.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/protocol/types/Schema.java b/clients/src/main/java/kafka/common/protocol/types/Schema.java
new file mode 100644
index 0000000..b7b1c75
--- /dev/null
+++ b/clients/src/main/java/kafka/common/protocol/types/Schema.java
@@ -0,0 +1,134 @@
+package kafka.common.protocol.types;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * The schema for a compound record definition
+ */
+public class Schema extends Type {
+
+    private final Field[] fields;
+    private final Map<String, Field> fieldsByName;
+
+    public Schema(Field... fs) {
+        this.fields = new Field[fs.length];
+        this.fieldsByName = new HashMap<String, Field>();
+        for (int i = 0; i < this.fields.length; i++) {
+            Field field = fs[i];
+            if (fieldsByName.containsKey(field.name))
+                throw new SchemaException("Schema contains a duplicate field: " + field.name);
+            this.fields[i] = new Field(i, field.name, field.type, field.doc, field.defaultValue, this);
+            this.fieldsByName.put(fs[i].name, this.fields[i]);
+        }
+    }
+
+    /**
+     * Write a struct to the buffer
+     */
+    public void write(ByteBuffer buffer, Object o) {
+        Struct r = (Struct) o;
+        for (int i = 0; i < fields.length; i++) {
+            Field f = fields[i];
+            try {
+                Object value = f.type().validate(r.get(f));
+                f.type.write(buffer, value);
+            } catch (Exception e) {
+                throw new SchemaException("Error writing field '" + f.name + "': " + e.getMessage() == null ? e.getMessage() : e.getClass()
+                                                                                                                                .getName());
+            }
+        }
+    }
+
+    /**
+     * Read a struct from the buffer
+     */
+    public Object read(ByteBuffer buffer) {
+        Object[] objects = new Object[fields.length];
+        for (int i = 0; i < fields.length; i++)
+            objects[i] = fields[i].type.read(buffer);
+        return new Struct(this, objects);
+    }
+
+    /**
+     * The size of the given record
+     */
+    public int sizeOf(Object o) {
+        int size = 0;
+        Struct r = (Struct) o;
+        for (int i = 0; i < fields.length; i++)
+            size += fields[i].type.sizeOf(r.get(fields[i]));
+        return size;
+    }
+
+    /**
+     * The number of fields in this schema
+     */
+    public int numFields() {
+        return this.fields.length;
+    }
+
+    /**
+     * Get a field by its slot in the record array
+     * 
+     * @param slot The slot at which this field sits
+     * @return The field
+     */
+    public Field get(int slot) {
+        return this.fields[slot];
+    }
+
+    /**
+     * Get a field by its name
+     * 
+     * @param name The name of the field
+     * @return The field
+     */
+    public Field get(String name) {
+        return this.fieldsByName.get(name);
+    }
+
+    /**
+     * Get all the fields in this schema
+     */
+    public Field[] fields() {
+        return this.fields;
+    }
+
+    /**
+     * Display a string representation of the schema
+     */
+    public String toString() {
+        StringBuilder b = new StringBuilder();
+        b.append('{');
+        for (int i = 0; i < this.fields.length; i++) {
+            b.append(this.fields[i].name);
+            b.append(':');
+            b.append(this.fields[i].type());
+            if (i < this.fields.length - 1)
+                b.append(',');
+        }
+        b.append("}");
+        return b.toString();
+    }
+
+    @Override
+    public Struct validate(Object item) {
+        try {
+            Struct struct = (Struct) item;
+            for (int i = 0; i < this.fields.length; i++) {
+                Field field = this.fields[i];
+                try {
+                    field.type.validate(struct.get(field));
+                } catch (SchemaException e) {
+                    throw new SchemaException("Invalid value for field '" + field.name + "': " + e.getMessage());
+                }
+            }
+            return struct;
+        } catch (ClassCastException e) {
+            throw new SchemaException("Not a Struct.");
+        }
+    }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/protocol/types/SchemaException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/protocol/types/SchemaException.java b/clients/src/main/java/kafka/common/protocol/types/SchemaException.java
new file mode 100644
index 0000000..a2a2d50
--- /dev/null
+++ b/clients/src/main/java/kafka/common/protocol/types/SchemaException.java
@@ -0,0 +1,13 @@
+package kafka.common.protocol.types;
+
+import kafka.common.KafkaException;
+
+public class SchemaException extends KafkaException {
+
+    private static final long serialVersionUID = 1L;
+
+    public SchemaException(String message) {
+        super(message);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/protocol/types/Struct.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/protocol/types/Struct.java b/clients/src/main/java/kafka/common/protocol/types/Struct.java
new file mode 100644
index 0000000..c83aefa
--- /dev/null
+++ b/clients/src/main/java/kafka/common/protocol/types/Struct.java
@@ -0,0 +1,227 @@
+package kafka.common.protocol.types;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+/**
+ * A record that can be serialized and deserialized according to a pre-defined schema
+ */
+public class Struct {
+    private final Schema schema;
+    private final Object[] values;
+
+    Struct(Schema schema, Object[] values) {
+        this.schema = schema;
+        this.values = values;
+    }
+
+    public Struct(Schema schema) {
+        this.schema = schema;
+        this.values = new Object[this.schema.numFields()];
+    }
+
+    /**
+     * The schema for this struct.
+     */
+    public Schema schema() {
+        return this.schema;
+    }
+
+    /**
+     * Return the value of the given pre-validated field, or if the value is missing return the default value.
+     * 
+     * @param field The field for which to get the default value
+     * @throws SchemaException if the field has no value and has no default.
+     */
+    private Object getFieldOrDefault(Field field) {
+        Object value = this.values[field.index];
+        if (value != null)
+            return value;
+        else if (field.defaultValue != Field.NO_DEFAULT)
+            return field.defaultValue;
+        else
+            throw new SchemaException("Missing value for field '" + field.name + " which has no default value.");
+    }
+
+    /**
+     * Get the value for the field directly by the field index with no lookup needed (faster!)
+     * 
+     * @param field The field to look up
+     * @return The value for that field.
+     */
+    public Object get(Field field) {
+        validateField(field);
+        return getFieldOrDefault(field);
+    }
+
+    /**
+     * Get the record value for the field with the given name by doing a hash table lookup (slower!)
+     * 
+     * @param name The name of the field
+     * @return The value in the record
+     */
+    public Object get(String name) {
+        Field field = schema.get(name);
+        if (field == null)
+            throw new SchemaException("No such field: " + name);
+        return getFieldOrDefault(field);
+    }
+
+    public Struct getStruct(Field field) {
+        return (Struct) get(field);
+    }
+
+    public Struct getStruct(String name) {
+        return (Struct) get(name);
+    }
+
+    public Short getShort(Field field) {
+        return (Short) get(field);
+    }
+
+    public Short getShort(String name) {
+        return (Short) get(name);
+    }
+
+    public Integer getInt(Field field) {
+        return (Integer) get(field);
+    }
+
+    public Integer getInt(String name) {
+        return (Integer) get(name);
+    }
+
+    public Object[] getArray(Field field) {
+        return (Object[]) get(field);
+    }
+
+    public Object[] getArray(String name) {
+        return (Object[]) get(name);
+    }
+
+    public String getString(Field field) {
+        return (String) get(field);
+    }
+
+    public String getString(String name) {
+        return (String) get(name);
+    }
+
+    /**
+     * Set the given field to the specified value
+     * 
+     * @param field The field
+     * @param value The value
+     */
+    public Struct set(Field field, Object value) {
+        validateField(field);
+        this.values[field.index] = value;
+        return this;
+    }
+
+    /**
+     * Set the field specified by the given name to the value
+     * 
+     * @param name The name of the field
+     * @param value The value to set
+     */
+    public Struct set(String name, Object value) {
+        Field field = this.schema.get(name);
+        if (field == null)
+            throw new SchemaException("Unknown field: " + name);
+        this.values[field.index] = value;
+        return this;
+    }
+
+    /**
+     * Create a struct for the schema of a container type (struct or array)
+     * 
+     * @param field The field to create an instance of
+     * @return The struct
+     */
+    public Struct instance(Field field) {
+        validateField(field);
+        if (field.type() instanceof Schema) {
+            return new Struct((Schema) field.type());
+        } else if (field.type() instanceof ArrayOf) {
+            ArrayOf array = (ArrayOf) field.type();
+            return new Struct((Schema) array.type());
+        } else {
+            throw new SchemaException("Field " + field.name + " is not a container type, it is of type " + field.type());
+        }
+    }
+
+    /**
+     * Create a struct instance for the given field which must be a container type (struct or array)
+     * 
+     * @param field The name of the field to create (field must be a schema type)
+     * @return The struct
+     */
+    public Struct instance(String field) {
+        return instance(schema.get(field));
+    }
+
+    /**
+     * Empty all the values from this record
+     */
+    public void clear() {
+        Arrays.fill(this.values, null);
+    }
+
+    /**
+     * Get the serialized size of this object
+     */
+    public int sizeOf() {
+        return this.schema.sizeOf(this);
+    }
+
+    /**
+     * Write this struct to a buffer
+     */
+    public void writeTo(ByteBuffer buffer) {
+        this.schema.write(buffer, this);
+    }
+
+    /**
+     * Ensure the user doesn't try to access fields from the wrong schema
+     */
+    private void validateField(Field field) {
+        if (this.schema != field.schema)
+            throw new SchemaException("Attempt to access field '" + field.name + " from a different schema instance.");
+        if (field.index > values.length)
+            throw new SchemaException("Invalid field index: " + field.index);
+    }
+
+    /**
+     * Validate the contents of this struct against its schema
+     */
+    public void validate() {
+        this.schema.validate(this);
+    }
+
+    /**
+     * Create a byte buffer containing the serialized form of the values in this struct. This method can choose to break
+     * the struct into multiple ByteBuffers if need be.
+     */
+    public ByteBuffer[] toBytes() {
+        ByteBuffer buffer = ByteBuffer.allocate(sizeOf());
+        writeTo(buffer);
+        return new ByteBuffer[] { buffer };
+    }
+
+    @Override
+    public String toString() {
+        StringBuilder b = new StringBuilder();
+        b.append('{');
+        for (int i = 0; i < this.values.length; i++) {
+            b.append(this.schema.get(i).name);
+            b.append('=');
+            b.append(this.values[i]);
+            if (i < this.values.length - 1)
+                b.append(',');
+        }
+        b.append('}');
+        return b.toString();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/protocol/types/Type.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/protocol/types/Type.java b/clients/src/main/java/kafka/common/protocol/types/Type.java
new file mode 100644
index 0000000..f4c93e3
--- /dev/null
+++ b/clients/src/main/java/kafka/common/protocol/types/Type.java
@@ -0,0 +1,216 @@
+package kafka.common.protocol.types;
+
+import java.nio.ByteBuffer;
+
+import kafka.common.utils.Utils;
+
+/**
+ * A serializable type
+ */
+public abstract class Type {
+
+    public abstract void write(ByteBuffer buffer, Object o);
+
+    public abstract Object read(ByteBuffer buffer);
+
+    public abstract int sizeOf(Object o);
+
+    public abstract Object validate(Object o);
+
+    public static final Type INT8 = new Type() {
+        @Override
+        public void write(ByteBuffer buffer, Object o) {
+            buffer.put((Byte) o);
+        }
+
+        @Override
+        public Object read(ByteBuffer buffer) {
+            return buffer.get();
+        }
+
+        @Override
+        public int sizeOf(Object o) {
+            return 1;
+        }
+
+        @Override
+        public String toString() {
+            return "INT8";
+        }
+
+        @Override
+        public Byte validate(Object item) {
+            if (item instanceof Byte)
+                return (Byte) item;
+            else
+                throw new SchemaException(item + " is not a Byte.");
+        }
+    };
+
+    public static final Type INT16 = new Type() {
+        @Override
+        public void write(ByteBuffer buffer, Object o) {
+            buffer.putShort((Short) o);
+        }
+
+        @Override
+        public Object read(ByteBuffer buffer) {
+            return buffer.getShort();
+        }
+
+        @Override
+        public int sizeOf(Object o) {
+            return 2;
+        }
+
+        @Override
+        public String toString() {
+            return "INT16";
+        }
+
+        @Override
+        public Short validate(Object item) {
+            if (item instanceof Short)
+                return (Short) item;
+            else
+                throw new SchemaException(item + " is not a Short.");
+        }
+    };
+
+    public static final Type INT32 = new Type() {
+        @Override
+        public void write(ByteBuffer buffer, Object o) {
+            buffer.putInt((Integer) o);
+        }
+
+        @Override
+        public Object read(ByteBuffer buffer) {
+            return buffer.getInt();
+        }
+
+        @Override
+        public int sizeOf(Object o) {
+            return 4;
+        }
+
+        @Override
+        public String toString() {
+            return "INT32";
+        }
+
+        @Override
+        public Integer validate(Object item) {
+            if (item instanceof Integer)
+                return (Integer) item;
+            else
+                throw new SchemaException(item + " is not an Integer.");
+        }
+    };
+
+    public static final Type INT64 = new Type() {
+        @Override
+        public void write(ByteBuffer buffer, Object o) {
+            buffer.putLong((Long) o);
+        }
+
+        @Override
+        public Object read(ByteBuffer buffer) {
+            return buffer.getLong();
+        }
+
+        @Override
+        public int sizeOf(Object o) {
+            return 8;
+        }
+
+        @Override
+        public String toString() {
+            return "INT64";
+        }
+
+        @Override
+        public Long validate(Object item) {
+            if (item instanceof Long)
+                return (Long) item;
+            else
+                throw new SchemaException(item + " is not a Long.");
+        }
+    };
+
+    public static final Type STRING = new Type() {
+        @Override
+        public void write(ByteBuffer buffer, Object o) {
+            byte[] bytes = Utils.utf8((String) o);
+            if (bytes.length > Short.MAX_VALUE)
+                throw new SchemaException("String is longer than the maximum string length.");
+            buffer.putShort((short) bytes.length);
+            buffer.put(bytes);
+        }
+
+        @Override
+        public Object read(ByteBuffer buffer) {
+            int length = buffer.getShort();
+            byte[] bytes = new byte[length];
+            buffer.get(bytes);
+            return Utils.utf8(bytes);
+        }
+
+        @Override
+        public int sizeOf(Object o) {
+            return 2 + Utils.utf8Length((String) o);
+        }
+
+        @Override
+        public String toString() {
+            return "STRING";
+        }
+
+        @Override
+        public String validate(Object item) {
+            if (item instanceof String)
+                return (String) item;
+            else
+                throw new SchemaException(item + " is not a String.");
+        }
+    };
+
+    public static final Type BYTES = new Type() {
+        @Override
+        public void write(ByteBuffer buffer, Object o) {
+            ByteBuffer arg = (ByteBuffer) o;
+            int pos = arg.position();
+            buffer.putInt(arg.remaining());
+            buffer.put(arg);
+            arg.position(pos);
+        }
+
+        @Override
+        public Object read(ByteBuffer buffer) {
+            int size = buffer.getInt();
+            ByteBuffer val = buffer.slice();
+            val.limit(size);
+            buffer.position(buffer.position() + size);
+            return val;
+        }
+
+        @Override
+        public int sizeOf(Object o) {
+            ByteBuffer buffer = (ByteBuffer) o;
+            return 4 + buffer.remaining();
+        }
+
+        @Override
+        public String toString() {
+            return "BYTES";
+        }
+
+        @Override
+        public ByteBuffer validate(Object item) {
+            if (item instanceof ByteBuffer)
+                return (ByteBuffer) item;
+            else
+                throw new SchemaException(item + " is not a java.nio.ByteBuffer.");
+        }
+    };
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/record/CompressionType.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/record/CompressionType.java b/clients/src/main/java/kafka/common/record/CompressionType.java
new file mode 100644
index 0000000..f6d9026
--- /dev/null
+++ b/clients/src/main/java/kafka/common/record/CompressionType.java
@@ -0,0 +1,40 @@
+package kafka.common.record;
+
+/**
+ * The compression type to use
+ */
+public enum CompressionType {
+    NONE(0, "none"), GZIP(1, "gzip"), SNAPPY(2, "snappy");
+
+    public final int id;
+    public final String name;
+
+    private CompressionType(int id, String name) {
+        this.id = id;
+        this.name = name;
+    }
+
+    public static CompressionType forId(int id) {
+        switch (id) {
+            case 0:
+                return NONE;
+            case 1:
+                return GZIP;
+            case 2:
+                return SNAPPY;
+            default:
+                throw new IllegalArgumentException("Unknown compression type id: " + id);
+        }
+    }
+
+    public static CompressionType forName(String name) {
+        if (NONE.name.equals(name))
+            return NONE;
+        else if (GZIP.name.equals(name))
+            return GZIP;
+        else if (SNAPPY.name.equals(name))
+            return SNAPPY;
+        else
+            throw new IllegalArgumentException("Unknown compression name: " + name);
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/record/InvalidRecordException.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/record/InvalidRecordException.java b/clients/src/main/java/kafka/common/record/InvalidRecordException.java
new file mode 100644
index 0000000..97fbe50
--- /dev/null
+++ b/clients/src/main/java/kafka/common/record/InvalidRecordException.java
@@ -0,0 +1,11 @@
+package kafka.common.record;
+
+public class InvalidRecordException extends RuntimeException {
+
+    private static final long serialVersionUID = 1;
+
+    public InvalidRecordException(String s) {
+        super(s);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/record/LogEntry.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/record/LogEntry.java b/clients/src/main/java/kafka/common/record/LogEntry.java
new file mode 100644
index 0000000..f5e99c9
--- /dev/null
+++ b/clients/src/main/java/kafka/common/record/LogEntry.java
@@ -0,0 +1,28 @@
+package kafka.common.record;
+
+/**
+ * An offset and record pair
+ */
+public final class LogEntry {
+
+    private final long offset;
+    private final Record record;
+
+    public LogEntry(long offset, Record record) {
+        this.offset = offset;
+        this.record = record;
+    }
+
+    public long offset() {
+        return this.offset;
+    }
+
+    public Record record() {
+        return this.record;
+    }
+
+    @Override
+    public String toString() {
+        return "LogEntry(" + offset + ", " + record + ")";
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/record/MemoryRecords.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/record/MemoryRecords.java b/clients/src/main/java/kafka/common/record/MemoryRecords.java
new file mode 100644
index 0000000..ec98226
--- /dev/null
+++ b/clients/src/main/java/kafka/common/record/MemoryRecords.java
@@ -0,0 +1,102 @@
+package kafka.common.record;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.GatheringByteChannel;
+import java.util.Iterator;
+
+import kafka.common.utils.AbstractIterator;
+
+/**
+ * A {@link Records} implementation backed by a ByteBuffer.
+ */
+public class MemoryRecords implements Records {
+
+    private final ByteBuffer buffer;
+
+    public MemoryRecords(int size) {
+        this(ByteBuffer.allocate(size));
+    }
+
+    public MemoryRecords(ByteBuffer buffer) {
+        this.buffer = buffer;
+    }
+
+    /**
+     * Append the given record and offset to the buffer
+     */
+    public void append(long offset, Record record) {
+        buffer.putLong(offset);
+        buffer.putInt(record.size());
+        buffer.put(record.buffer());
+        record.buffer().rewind();
+    }
+
+    /**
+     * Append a new record and offset to the buffer
+     */
+    public void append(long offset, byte[] key, byte[] value, CompressionType type) {
+        buffer.putLong(offset);
+        buffer.putInt(Record.recordSize(key, value));
+        Record.write(this.buffer, key, value, type);
+    }
+
+    /**
+     * Check if we have room for a new record containing the given key/value pair
+     */
+    public boolean hasRoomFor(byte[] key, byte[] value) {
+        return this.buffer.remaining() >= Records.LOG_OVERHEAD + Record.recordSize(key, value);
+    }
+
+    /** Write the messages in this set to the given channel */
+    public int writeTo(GatheringByteChannel channel) throws IOException {
+        return channel.write(buffer);
+    }
+
+    /**
+     * The size of this record set
+     */
+    public int sizeInBytes() {
+        return this.buffer.position();
+    }
+
+    /**
+     * Get the byte buffer that backs this records instance
+     */
+    public ByteBuffer buffer() {
+        return buffer.duplicate();
+    }
+
+    @Override
+    public Iterator<LogEntry> iterator() {
+        return new RecordsIterator(this.buffer);
+    }
+
+    /* TODO: allow reuse of the buffer used for iteration */
+    public static class RecordsIterator extends AbstractIterator<LogEntry> {
+        private final ByteBuffer buffer;
+
+        public RecordsIterator(ByteBuffer buffer) {
+            ByteBuffer copy = buffer.duplicate();
+            copy.flip();
+            this.buffer = copy;
+        }
+
+        @Override
+        protected LogEntry makeNext() {
+            if (buffer.remaining() < Records.LOG_OVERHEAD)
+                return allDone();
+            long offset = buffer.getLong();
+            int size = buffer.getInt();
+            if (size < 0)
+                throw new IllegalStateException("Message with size " + size);
+            if (buffer.remaining() < size)
+                return allDone();
+            ByteBuffer rec = buffer.slice();
+            rec.limit(size);
+            this.buffer.position(this.buffer.position() + size);
+            return new LogEntry(offset, new Record(rec));
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/record/Record.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/record/Record.java b/clients/src/main/java/kafka/common/record/Record.java
new file mode 100644
index 0000000..835a0a4
--- /dev/null
+++ b/clients/src/main/java/kafka/common/record/Record.java
@@ -0,0 +1,286 @@
+package kafka.common.record;
+
+import java.nio.ByteBuffer;
+
+import kafka.common.utils.Utils;
+
+/**
+ * A record: a serialized key and value along with the associated CRC and other fields
+ */
+public final class Record {
+
+    /**
+     * The current offset and size for all the fixed-length fields
+     */
+    public static final int CRC_OFFSET = 0;
+    public static final int CRC_LENGTH = 4;
+    public static final int MAGIC_OFFSET = CRC_OFFSET + CRC_LENGTH;
+    public static final int MAGIC_LENGTH = 1;
+    public static final int ATTRIBUTES_OFFSET = MAGIC_OFFSET + MAGIC_LENGTH;
+    public static final int ATTRIBUTE_LENGTH = 1;
+    public static final int KEY_SIZE_OFFSET = ATTRIBUTES_OFFSET + ATTRIBUTE_LENGTH;
+    public static final int KEY_SIZE_LENGTH = 4;
+    public static final int KEY_OFFSET = KEY_SIZE_OFFSET + KEY_SIZE_LENGTH;
+    public static final int VALUE_SIZE_LENGTH = 4;
+
+    /** The amount of overhead bytes in a record */
+    public static final int RECORD_OVERHEAD = KEY_OFFSET + VALUE_SIZE_LENGTH;
+
+    /**
+     * The minimum valid size for the record header
+     */
+    public static final int MIN_HEADER_SIZE = CRC_LENGTH + MAGIC_LENGTH + ATTRIBUTE_LENGTH + KEY_SIZE_LENGTH + VALUE_SIZE_LENGTH;
+
+    /**
+     * The current "magic" value
+     */
+    public static final byte CURRENT_MAGIC_VALUE = 0;
+
+    /**
+     * Specifies the mask for the compression code. 2 bits to hold the compression codec. 0 is reserved to indicate no
+     * compression
+     */
+    public static final int COMPRESSION_CODEC_MASK = 0x03;
+
+    /**
+     * Compression code for uncompressed records
+     */
+    public static final int NO_COMPRESSION = 0;
+
+    private final ByteBuffer buffer;
+
+    public Record(ByteBuffer buffer) {
+        this.buffer = buffer;
+    }
+
+    /**
+     * A constructor to create a LogRecord
+     * 
+     * @param key The key of the record (null, if none)
+     * @param value The record value
+     * @param codec The compression codec used on the contents of the record (if any)
+     * @param valueOffset The offset into the payload array used to extract payload
+     * @param valueSize The size of the payload to use
+     */
+    public Record(byte[] key, byte[] value, CompressionType codec, int valueOffset, int valueSize) {
+        this(ByteBuffer.allocate(recordSize(key == null ? 0 : key.length, value == null ? 0 : valueSize >= 0 ? valueSize
+                                                                                                            : value.length - valueOffset)));
+        write(this.buffer, key, value, codec, valueOffset, valueSize);
+        this.buffer.rewind();
+    }
+
+    public Record(byte[] key, byte[] value, CompressionType codec) {
+        this(key, value, codec, 0, -1);
+    }
+
+    public Record(byte[] value, CompressionType codec) {
+        this(null, value, codec);
+    }
+
+    public Record(byte[] key, byte[] value) {
+        this(key, value, CompressionType.NONE);
+    }
+
+    public Record(byte[] value) {
+        this(null, value, CompressionType.NONE);
+    }
+
+    public static void write(ByteBuffer buffer, byte[] key, byte[] value, CompressionType codec, int valueOffset, int valueSize) {
+        // skip crc, we will fill that in at the end
+        int pos = buffer.position();
+        buffer.position(pos + MAGIC_OFFSET);
+        buffer.put(CURRENT_MAGIC_VALUE);
+        byte attributes = 0;
+        if (codec.id > 0)
+            attributes = (byte) (attributes | (COMPRESSION_CODEC_MASK & codec.id));
+        buffer.put(attributes);
+        // write the key
+        if (key == null) {
+            buffer.putInt(-1);
+        } else {
+            buffer.putInt(key.length);
+            buffer.put(key, 0, key.length);
+        }
+        // write the value
+        if (value == null) {
+            buffer.putInt(-1);
+        } else {
+            int size = valueSize >= 0 ? valueSize : (value.length - valueOffset);
+            buffer.putInt(size);
+            buffer.put(value, valueOffset, size);
+        }
+
+        // now compute the checksum and fill it in
+        long crc = computeChecksum(buffer,
+                                   buffer.arrayOffset() + pos + MAGIC_OFFSET,
+                                   buffer.position() - pos - MAGIC_OFFSET - buffer.arrayOffset());
+        Utils.writeUnsignedInt(buffer, pos + CRC_OFFSET, crc);
+    }
+
+    public static void write(ByteBuffer buffer, byte[] key, byte[] value, CompressionType codec) {
+        write(buffer, key, value, codec, 0, -1);
+    }
+
+    public static int recordSize(byte[] key, byte[] value) {
+        return recordSize(key == null ? 0 : key.length, value == null ? 0 : value.length);
+    }
+
+    public static int recordSize(int keySize, int valueSize) {
+        return CRC_LENGTH + MAGIC_LENGTH + ATTRIBUTE_LENGTH + KEY_SIZE_LENGTH + keySize + VALUE_SIZE_LENGTH + valueSize;
+    }
+
+    public ByteBuffer buffer() {
+        return this.buffer;
+    }
+
+    /**
+     * Compute the checksum of the record from the record contents
+     */
+    public static long computeChecksum(ByteBuffer buffer, int position, int size) {
+        return Utils.crc32(buffer.array(), buffer.arrayOffset() + position, size - buffer.arrayOffset());
+    }
+
+    /**
+     * Compute the checksum of the record from the record contents
+     */
+    public long computeChecksum() {
+        return computeChecksum(buffer, MAGIC_OFFSET, buffer.limit() - MAGIC_OFFSET);
+    }
+
+    /**
+     * Retrieve the previously computed CRC for this record
+     */
+    public long checksum() {
+        return Utils.readUnsignedInt(buffer, CRC_OFFSET);
+    }
+
+    /**
+     * Returns true if the crc stored with the record matches the crc computed off the record contents
+     */
+    public boolean isValid() {
+        return checksum() == computeChecksum();
+    }
+
+    /**
+     * Throw an InvalidMessageException if isValid is false for this record
+     */
+    public void ensureValid() {
+        if (!isValid())
+            throw new InvalidRecordException("Record is corrupt (stored crc = " + checksum()
+                                             + ", computed crc = "
+                                             + computeChecksum()
+                                             + ")");
+    }
+
+    /**
+     * The complete serialized size of this record in bytes (including crc, header attributes, etc)
+     */
+    public int size() {
+        return buffer.limit();
+    }
+
+    /**
+     * The length of the key in bytes
+     */
+    public int keySize() {
+        return buffer.getInt(KEY_SIZE_OFFSET);
+    }
+
+    /**
+     * Does the record have a key?
+     */
+    public boolean hasKey() {
+        return keySize() >= 0;
+    }
+
+    /**
+     * The position where the value size is stored
+     */
+    private int valueSizeOffset() {
+        return KEY_OFFSET + Math.max(0, keySize());
+    }
+
+    /**
+     * The length of the value in bytes
+     */
+    public int valueSize() {
+        return buffer.getInt(valueSizeOffset());
+    }
+
+    /**
+     * The magic version of this record
+     */
+    public byte magic() {
+        return buffer.get(MAGIC_OFFSET);
+    }
+
+    /**
+     * The attributes stored with this record
+     */
+    public byte attributes() {
+        return buffer.get(ATTRIBUTES_OFFSET);
+    }
+
+    /**
+     * The compression codec used with this record
+     */
+    public CompressionType compressionType() {
+        return CompressionType.forId(buffer.get(ATTRIBUTES_OFFSET) & COMPRESSION_CODEC_MASK);
+    }
+
+    /**
+     * A ByteBuffer containing the value of this record
+     */
+    public ByteBuffer value() {
+        return sliceDelimited(valueSizeOffset());
+    }
+
+    /**
+     * A ByteBuffer containing the message key
+     */
+    public ByteBuffer key() {
+        return sliceDelimited(KEY_SIZE_OFFSET);
+    }
+
+    /**
+     * Read a size-delimited byte buffer starting at the given offset
+     */
+    private ByteBuffer sliceDelimited(int start) {
+        int size = buffer.getInt(start);
+        if (size < 0) {
+            return null;
+        } else {
+            ByteBuffer b = buffer.duplicate();
+            b.position(start + 4);
+            b = b.slice();
+            b.limit(size);
+            b.rewind();
+            return b;
+        }
+    }
+
+    public String toString() {
+        return String.format("Message(magic = %d, attributes = %d, crc = %d, key = %d bytes, value = %d bytes)",
+                             magic(),
+                             attributes(),
+                             checksum(),
+                             key().limit(),
+                             value().limit());
+    }
+
+    public boolean equals(Object other) {
+        if (this == other)
+            return true;
+        if (other == null)
+            return false;
+        if (!other.getClass().equals(Record.class))
+            return false;
+        Record record = (Record) other;
+        return this.buffer.equals(record.buffer);
+    }
+
+    public int hashCode() {
+        return buffer.hashCode();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/record/Records.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/record/Records.java b/clients/src/main/java/kafka/common/record/Records.java
new file mode 100644
index 0000000..6531ca0
--- /dev/null
+++ b/clients/src/main/java/kafka/common/record/Records.java
@@ -0,0 +1,29 @@
+package kafka.common.record;
+
+import java.io.IOException;
+import java.nio.channels.GatheringByteChannel;
+
+/**
+ * A binary format which consists of a 4 byte size, an 8 byte offset, and the record bytes. See {@link MemoryRecords}
+ * for the in-memory representation.
+ */
+public interface Records extends Iterable<LogEntry> {
+
+    int SIZE_LENGTH = 4;
+    int OFFSET_LENGTH = 8;
+    int LOG_OVERHEAD = SIZE_LENGTH + OFFSET_LENGTH;
+
+    /**
+     * Write these records to the given channel
+     * @param channel The channel to write to
+     * @return The number of bytes written
+     * @throws IOException If the write fails.
+     */
+    public int writeTo(GatheringByteChannel channel) throws IOException;
+
+    /**
+     * The size of these records in bytes
+     */
+    public int sizeInBytes();
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/requests/RequestHeader.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/requests/RequestHeader.java b/clients/src/main/java/kafka/common/requests/RequestHeader.java
new file mode 100644
index 0000000..4ce67f8
--- /dev/null
+++ b/clients/src/main/java/kafka/common/requests/RequestHeader.java
@@ -0,0 +1,68 @@
+package kafka.common.requests;
+
+import static kafka.common.protocol.Protocol.REQUEST_HEADER;
+
+import java.nio.ByteBuffer;
+
+import kafka.common.protocol.ProtoUtils;
+import kafka.common.protocol.Protocol;
+import kafka.common.protocol.types.Field;
+import kafka.common.protocol.types.Struct;
+
+/**
+ * The header for a request in the Kafka protocol
+ */
+public class RequestHeader {
+
+    private static Field API_KEY_FIELD = REQUEST_HEADER.get("api_key");
+    private static Field API_VERSION_FIELD = REQUEST_HEADER.get("api_version");
+    private static Field CLIENT_ID_FIELD = REQUEST_HEADER.get("client_id");
+    private static Field CORRELATION_ID_FIELD = REQUEST_HEADER.get("correlation_id");
+
+    private final Struct header;
+
+    public RequestHeader(Struct header) {
+        super();
+        this.header = header;
+    }
+
+    public RequestHeader(short apiKey, String client, int correlation) {
+        this(apiKey, ProtoUtils.latestVersion(apiKey), client, correlation);
+    }
+
+    public RequestHeader(short apiKey, short version, String client, int correlation) {
+        this(new Struct(Protocol.REQUEST_HEADER));
+        this.header.set(API_KEY_FIELD, apiKey);
+        this.header.set(API_VERSION_FIELD, version);
+        this.header.set(CLIENT_ID_FIELD, client);
+        this.header.set(CORRELATION_ID_FIELD, correlation);
+    }
+
+    public short apiKey() {
+        return (Short) this.header.get(API_KEY_FIELD);
+    }
+
+    public short apiVersion() {
+        return (Short) this.header.get(API_VERSION_FIELD);
+    }
+
+    public String clientId() {
+        return (String) this.header.get(CLIENT_ID_FIELD);
+    }
+
+    public int correlationId() {
+        return (Integer) this.header.get(CORRELATION_ID_FIELD);
+    }
+
+    public static RequestHeader parse(ByteBuffer buffer) {
+        return new RequestHeader((Struct) Protocol.REQUEST_HEADER.read(buffer));
+    }
+
+    public void writeTo(ByteBuffer buffer) {
+        header.writeTo(buffer);
+    }
+
+    public int sizeOf() {
+        return header.sizeOf();
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/requests/RequestSend.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/requests/RequestSend.java b/clients/src/main/java/kafka/common/requests/RequestSend.java
new file mode 100644
index 0000000..f6a9a86
--- /dev/null
+++ b/clients/src/main/java/kafka/common/requests/RequestSend.java
@@ -0,0 +1,38 @@
+package kafka.common.requests;
+
+import java.nio.ByteBuffer;
+
+import kafka.common.network.NetworkSend;
+import kafka.common.protocol.types.Struct;
+
+/**
+ * A send object for a kafka request
+ */
+public class RequestSend extends NetworkSend {
+
+    private final RequestHeader header;
+    private final Struct body;
+
+    public RequestSend(int destination, RequestHeader header, Struct body) {
+        super(destination, serialize(header, body));
+        this.header = header;
+        this.body = body;
+    }
+
+    private static ByteBuffer serialize(RequestHeader header, Struct body) {
+        ByteBuffer buffer = ByteBuffer.allocate(header.sizeOf() + body.sizeOf());
+        header.writeTo(buffer);
+        body.writeTo(buffer);
+        buffer.rewind();
+        return buffer;
+    }
+
+    public RequestHeader header() {
+        return this.header;
+    }
+
+    public Struct body() {
+        return body;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/requests/ResponseHeader.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/requests/ResponseHeader.java b/clients/src/main/java/kafka/common/requests/ResponseHeader.java
new file mode 100644
index 0000000..1ef8e15
--- /dev/null
+++ b/clients/src/main/java/kafka/common/requests/ResponseHeader.java
@@ -0,0 +1,45 @@
+package kafka.common.requests;
+
+import static kafka.common.protocol.Protocol.RESPONSE_HEADER;
+
+import java.nio.ByteBuffer;
+
+import kafka.common.protocol.Protocol;
+import kafka.common.protocol.types.Field;
+import kafka.common.protocol.types.Struct;
+
+/**
+ * A response header in the kafka protocol.
+ */
+public class ResponseHeader {
+
+    private static Field CORRELATION_KEY_FIELD = RESPONSE_HEADER.get("correlation_id");
+
+    private final Struct header;
+
+    public ResponseHeader(Struct header) {
+        this.header = header;
+    }
+
+    public ResponseHeader(int correlationId) {
+        this(new Struct(Protocol.RESPONSE_HEADER));
+        this.header.set(CORRELATION_KEY_FIELD, correlationId);
+    }
+
+    public int correlationId() {
+        return (Integer) header.get(CORRELATION_KEY_FIELD);
+    }
+
+    public void writeTo(ByteBuffer buffer) {
+        header.writeTo(buffer);
+    }
+
+    public int sizeOf() {
+        return header.sizeOf();
+    }
+
+    public static ResponseHeader parse(ByteBuffer buffer) {
+        return new ResponseHeader(((Struct) Protocol.RESPONSE_HEADER.read(buffer)));
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/utils/AbstractIterator.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/utils/AbstractIterator.java b/clients/src/main/java/kafka/common/utils/AbstractIterator.java
new file mode 100644
index 0000000..f3190d7
--- /dev/null
+++ b/clients/src/main/java/kafka/common/utils/AbstractIterator.java
@@ -0,0 +1,72 @@
+package kafka.common.utils;
+
+import java.util.Iterator;
+import java.util.NoSuchElementException;
+
+/**
+ * A base class that simplifies implementing an iterator
+ * @param <T> The type of thing we are iterating over
+ */
+public abstract class AbstractIterator<T> implements Iterator<T> {
+
+    private static enum State {
+        READY, NOT_READY, DONE, FAILED
+    };
+
+    private State state = State.NOT_READY;
+    private T next;
+
+    @Override
+    public boolean hasNext() {
+        switch (state) {
+            case FAILED:
+                throw new IllegalStateException("Iterator is in failed state");
+            case DONE:
+                return false;
+            case READY:
+                return true;
+            default:
+                return maybeComputeNext();
+        }
+    }
+
+    @Override
+    public T next() {
+        if (!hasNext())
+            throw new NoSuchElementException();
+        state = State.NOT_READY;
+        if (next == null)
+            throw new IllegalStateException("Expected item but none found.");
+        return next;
+    }
+
+    @Override
+    public void remove() {
+        throw new UnsupportedOperationException("Removal not supported");
+    }
+
+    public T peek() {
+        if (!hasNext())
+            throw new NoSuchElementException();
+        return next;
+    }
+
+    protected T allDone() {
+        state = State.DONE;
+        return null;
+    }
+
+    protected abstract T makeNext();
+
+    private Boolean maybeComputeNext() {
+        state = State.FAILED;
+        next = makeNext();
+        if (state == State.DONE) {
+            return false;
+        } else {
+            state = State.READY;
+            return true;
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/utils/CopyOnWriteMap.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/utils/CopyOnWriteMap.java b/clients/src/main/java/kafka/common/utils/CopyOnWriteMap.java
new file mode 100644
index 0000000..e45df98
--- /dev/null
+++ b/clients/src/main/java/kafka/common/utils/CopyOnWriteMap.java
@@ -0,0 +1,130 @@
+package kafka.common.utils;
+
+import java.util.Collection;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Set;
+import java.util.concurrent.ConcurrentMap;
+
+/**
+ * A simple read-optimized map implementation that synchronizes only writes and does a fully copy on each modification
+ */
+public class CopyOnWriteMap<K, V> implements ConcurrentMap<K, V> {
+
+    private volatile Map<K, V> map;
+
+    public CopyOnWriteMap() {
+        this.map = Collections.emptyMap();
+    }
+
+    public CopyOnWriteMap(Map<K, V> map) {
+        this.map = Collections.unmodifiableMap(map);
+    }
+
+    @Override
+    public boolean containsKey(Object k) {
+        return map.containsKey(k);
+    }
+
+    @Override
+    public boolean containsValue(Object v) {
+        return map.containsValue(v);
+    }
+
+    @Override
+    public Set<java.util.Map.Entry<K, V>> entrySet() {
+        return map.entrySet();
+    }
+
+    @Override
+    public V get(Object k) {
+        return map.get(k);
+    }
+
+    @Override
+    public boolean isEmpty() {
+        return map.isEmpty();
+    }
+
+    @Override
+    public Set<K> keySet() {
+        return map.keySet();
+    }
+
+    @Override
+    public int size() {
+        return map.size();
+    }
+
+    @Override
+    public Collection<V> values() {
+        return map.values();
+    }
+
+    @Override
+    public synchronized void clear() {
+        this.map = Collections.emptyMap();
+    }
+
+    @Override
+    public synchronized V put(K k, V v) {
+        Map<K, V> copy = new HashMap<K, V>(this.map);
+        V prev = copy.put(k, v);
+        this.map = Collections.unmodifiableMap(copy);
+        return prev;
+    }
+
+    @Override
+    public synchronized void putAll(Map<? extends K, ? extends V> entries) {
+        Map<K, V> copy = new HashMap<K, V>(this.map);
+        copy.putAll(entries);
+        this.map = Collections.unmodifiableMap(copy);
+    }
+
+    @Override
+    public synchronized V remove(Object key) {
+        Map<K, V> copy = new HashMap<K, V>(this.map);
+        V prev = copy.remove(key);
+        this.map = Collections.unmodifiableMap(copy);
+        return prev;
+    }
+
+    @Override
+    public synchronized V putIfAbsent(K k, V v) {
+        if (!containsKey(k))
+            return put(k, v);
+        else
+            return get(k);
+    }
+
+    @Override
+    public synchronized boolean remove(Object k, Object v) {
+        if (containsKey(k) && get(k).equals(v)) {
+            remove(k);
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    @Override
+    public synchronized boolean replace(K k, V original, V replacement) {
+        if (containsKey(k) && get(k).equals(original)) {
+            put(k, replacement);
+            return true;
+        } else {
+            return false;
+        }
+    }
+
+    @Override
+    public synchronized V replace(K k, V v) {
+        if (containsKey(k)) {
+            return put(k, v);
+        } else {
+            return null;
+        }
+    }
+
+}


[2/7] KAFKA-1227 New producer!

Posted by jk...@apache.org.
http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/utils/KafkaThread.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/utils/KafkaThread.java b/clients/src/main/java/kafka/common/utils/KafkaThread.java
new file mode 100644
index 0000000..f830aba
--- /dev/null
+++ b/clients/src/main/java/kafka/common/utils/KafkaThread.java
@@ -0,0 +1,18 @@
+package kafka.common.utils;
+
+/**
+ * A wrapper for Thread that sets things up nicely
+ */
+public class KafkaThread extends Thread {
+
+    public KafkaThread(String name, Runnable runnable, boolean daemon) {
+        super(runnable, name);
+        setDaemon(daemon);
+        setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() {
+            public void uncaughtException(Thread t, Throwable e) {
+                e.printStackTrace();
+            }
+        });
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/utils/SystemTime.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/utils/SystemTime.java b/clients/src/main/java/kafka/common/utils/SystemTime.java
new file mode 100644
index 0000000..c8ca09c
--- /dev/null
+++ b/clients/src/main/java/kafka/common/utils/SystemTime.java
@@ -0,0 +1,26 @@
+package kafka.common.utils;
+
+/**
+ * A time implementation that uses the system clock and sleep call
+ */
+public class SystemTime implements Time {
+
+    @Override
+    public long milliseconds() {
+        return System.currentTimeMillis();
+    }
+
+    public long nanoseconds() {
+        return System.nanoTime();
+    }
+
+    @Override
+    public void sleep(long ms) {
+        try {
+            Thread.sleep(ms);
+        } catch (InterruptedException e) {
+            // no stress
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/utils/Time.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/utils/Time.java b/clients/src/main/java/kafka/common/utils/Time.java
new file mode 100644
index 0000000..390d16f
--- /dev/null
+++ b/clients/src/main/java/kafka/common/utils/Time.java
@@ -0,0 +1,23 @@
+package kafka.common.utils;
+
+/**
+ * An interface abstracting the clock to use in unit testing classes that make use of clock time
+ */
+public interface Time {
+
+    /**
+     * The current time in milliseconds
+     */
+    public long milliseconds();
+
+    /**
+     * The current time in nanoseconds
+     */
+    public long nanoseconds();
+
+    /**
+     * Sleep for the given number of milliseconds
+     */
+    public void sleep(long ms);
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/main/java/kafka/common/utils/Utils.java
----------------------------------------------------------------------
diff --git a/clients/src/main/java/kafka/common/utils/Utils.java b/clients/src/main/java/kafka/common/utils/Utils.java
new file mode 100644
index 0000000..f132771
--- /dev/null
+++ b/clients/src/main/java/kafka/common/utils/Utils.java
@@ -0,0 +1,230 @@
+package kafka.common.utils;
+
+import java.io.UnsupportedEncodingException;
+import java.nio.ByteBuffer;
+
+import kafka.common.KafkaException;
+
+public class Utils {
+
+    /**
+     * Turn the given UTF8 byte array into a string
+     * 
+     * @param bytes The byte array
+     * @return The string
+     */
+    public static String utf8(byte[] bytes) {
+        try {
+            return new String(bytes, "UTF8");
+        } catch (UnsupportedEncodingException e) {
+            throw new RuntimeException("This shouldn't happen.", e);
+        }
+    }
+
+    /**
+     * Turn a string into a utf8 byte[]
+     * 
+     * @param string The string
+     * @return The byte[]
+     */
+    public static byte[] utf8(String string) {
+        try {
+            return string.getBytes("UTF8");
+        } catch (UnsupportedEncodingException e) {
+            throw new RuntimeException("This shouldn't happen.", e);
+        }
+    }
+
+    /**
+     * Read an unsigned integer from the current position in the buffer, incrementing the position by 4 bytes
+     * 
+     * @param buffer The buffer to read from
+     * @return The integer read, as a long to avoid signedness
+     */
+    public static long readUnsignedInt(ByteBuffer buffer) {
+        return buffer.getInt() & 0xffffffffL;
+    }
+
+    /**
+     * Read an unsigned integer from the given position without modifying the buffers position
+     * 
+     * @param buffer the buffer to read from
+     * @param index the index from which to read the integer
+     * @return The integer read, as a long to avoid signedness
+     */
+    public static long readUnsignedInt(ByteBuffer buffer, int index) {
+        return buffer.getInt(index) & 0xffffffffL;
+    }
+
+    /**
+     * Write the given long value as a 4 byte unsigned integer. Overflow is ignored.
+     * 
+     * @param buffer The buffer to write to
+     * @param value The value to write
+     */
+    public static void writetUnsignedInt(ByteBuffer buffer, long value) {
+        buffer.putInt((int) (value & 0xffffffffL));
+    }
+
+    /**
+     * Write the given long value as a 4 byte unsigned integer. Overflow is ignored.
+     * 
+     * @param buffer The buffer to write to
+     * @param index The position in the buffer at which to begin writing
+     * @param value The value to write
+     */
+    public static void writeUnsignedInt(ByteBuffer buffer, int index, long value) {
+        buffer.putInt(index, (int) (value & 0xffffffffL));
+    }
+
+    /**
+     * Compute the CRC32 of the byte array
+     * 
+     * @param bytes The array to compute the checksum for
+     * @return The CRC32
+     */
+    public static long crc32(byte[] bytes) {
+        return crc32(bytes, 0, bytes.length);
+    }
+
+    /**
+     * Compute the CRC32 of the segment of the byte array given by the specificed size and offset
+     * 
+     * @param bytes The bytes to checksum
+     * @param offset the offset at which to begin checksumming
+     * @param size the number of bytes to checksum
+     * @return The CRC32
+     */
+    public static long crc32(byte[] bytes, int offset, int size) {
+        Crc32 crc = new Crc32();
+        crc.update(bytes, offset, size);
+        return crc.getValue();
+    }
+
+    /**
+     * Get the absolute value of the given number. If the number is Int.MinValue return 0. This is different from
+     * java.lang.Math.abs or scala.math.abs in that they return Int.MinValue (!).
+     */
+    public static int abs(int n) {
+        return n & 0x7fffffff;
+    }
+
+    /**
+     * Get the length for UTF8-encoding a string without encoding it first
+     * 
+     * @param s The string to calculate the length for
+     * @return The length when serialized
+     */
+    public static int utf8Length(CharSequence s) {
+        int count = 0;
+        for (int i = 0, len = s.length(); i < len; i++) {
+            char ch = s.charAt(i);
+            if (ch <= 0x7F) {
+                count++;
+            } else if (ch <= 0x7FF) {
+                count += 2;
+            } else if (Character.isHighSurrogate(ch)) {
+                count += 4;
+                ++i;
+            } else {
+                count += 3;
+            }
+        }
+        return count;
+    }
+
+    /**
+     * Read the given byte buffer into a byte array
+     */
+    public static byte[] toArray(ByteBuffer buffer) {
+        return toArray(buffer, 0, buffer.limit());
+    }
+
+    /**
+     * Read a byte array from the given offset and size in the buffer
+     */
+    public static byte[] toArray(ByteBuffer buffer, int offset, int size) {
+        byte[] dest = new byte[size];
+        if (buffer.hasArray()) {
+            System.arraycopy(buffer.array(), buffer.arrayOffset() + offset, dest, 0, size);
+        } else {
+            int pos = buffer.position();
+            buffer.get(dest);
+            buffer.position(pos);
+        }
+        return dest;
+    }
+
+    /**
+     * Check that the parameter t is not null
+     * 
+     * @param t The object to check
+     * @return t if it isn't null
+     * @throws NullPointerException if t is null.
+     */
+    public static <T> T notNull(T t) {
+        if (t == null)
+            throw new NullPointerException();
+        else
+            return t;
+    }
+
+    /**
+     * Instantiate the class
+     */
+    public static Object newInstance(Class<?> c) {
+        try {
+            return c.newInstance();
+        } catch (IllegalAccessException e) {
+            throw new KafkaException("Could not instantiate class " + c.getName(), e);
+        } catch (InstantiationException e) {
+            throw new KafkaException("Could not instantiate class " + c.getName() + " Does it have a public no-argument constructor?", e);
+        }
+    }
+
+    /**
+     * Generates 32 bit murmur2 hash from byte array
+     * @param data byte array to hash
+     * @return 32 bit hash of the given array
+     */
+    public static int murmur2(final byte[] data) {
+        int length = data.length;
+        int seed = 0x9747b28c;
+        // 'm' and 'r' are mixing constants generated offline.
+        // They're not really 'magic', they just happen to work well.
+        final int m = 0x5bd1e995;
+        final int r = 24;
+
+        // Initialize the hash to a random value
+        int h = seed ^ length;
+        int length4 = length / 4;
+
+        for (int i = 0; i < length4; i++) {
+            final int i4 = i * 4;
+            int k = (data[i4 + 0] & 0xff) + ((data[i4 + 1] & 0xff) << 8) + ((data[i4 + 2] & 0xff) << 16) + ((data[i4 + 3] & 0xff) << 24);
+            k *= m;
+            k ^= k >>> r;
+            k *= m;
+            h *= m;
+            h ^= k;
+        }
+
+        // Handle the last few bytes of the input array
+        switch (length % 4) {
+            case 3:
+                h ^= (data[(length & ~3) + 2] & 0xff) << 16;
+            case 2:
+                h ^= (data[(length & ~3) + 1] & 0xff) << 8;
+            case 1:
+                h ^= (data[length & ~3] & 0xff);
+                h *= m;
+        }
+
+        h ^= h >>> 13;
+        h *= m;
+        h ^= h >>> 15;
+
+        return h;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/test/java/kafka/clients/common/network/SelectorTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/kafka/clients/common/network/SelectorTest.java b/clients/src/test/java/kafka/clients/common/network/SelectorTest.java
new file mode 100644
index 0000000..68bc9ee
--- /dev/null
+++ b/clients/src/test/java/kafka/clients/common/network/SelectorTest.java
@@ -0,0 +1,292 @@
+package kafka.clients.common.network;
+
+import static java.util.Arrays.asList;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.ServerSocket;
+import java.net.Socket;
+import java.nio.ByteBuffer;
+import java.nio.channels.UnresolvedAddressException;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import kafka.common.network.NetworkReceive;
+import kafka.common.network.NetworkSend;
+import kafka.common.network.Selectable;
+import kafka.common.network.Selector;
+import kafka.common.utils.Utils;
+import kafka.test.TestUtils;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * A set of tests for the selector. These use a test harness that runs a simple socket server that echos back responses.
+ */
+public class SelectorTest {
+
+    private static final List<NetworkSend> EMPTY = new ArrayList<NetworkSend>();
+    private static final int BUFFER_SIZE = 4 * 1024;
+
+    private EchoServer server;
+    private Selectable selector;
+
+    @Before
+    public void setup() throws Exception {
+        this.server = new EchoServer();
+        this.server.start();
+        this.selector = new Selector();
+    }
+
+    @After
+    public void teardown() throws Exception {
+        this.selector.close();
+        this.server.close();
+    }
+
+    /**
+     * Validate that when the server disconnects, a client send ends up with that node in the disconnected list.
+     */
+    @Test
+    public void testServerDisconnect() throws Exception {
+        int node = 0;
+
+        // connect and do a simple request
+        blockingConnect(node);
+        assertEquals("hello", blockingRequest(node, "hello"));
+
+        // disconnect
+        this.server.closeConnections();
+        while (!selector.disconnected().contains(node))
+            selector.poll(1000L, EMPTY);
+
+        // reconnect and do another request
+        blockingConnect(node);
+        assertEquals("hello", blockingRequest(node, "hello"));
+    }
+
+    /**
+     * Validate that the client can intentionally disconnect and reconnect
+     */
+    @Test
+    public void testClientDisconnect() throws Exception {
+        int node = 0;
+        blockingConnect(node);
+        selector.disconnect(node);
+        selector.poll(10, asList(createSend(node, "hello1")));
+        assertEquals("Request should not have succeeded", 0, selector.completedSends().size());
+        assertEquals("There should be a disconnect", 1, selector.disconnected().size());
+        assertTrue("The disconnect should be from our node", selector.disconnected().contains(node));
+        blockingConnect(node);
+        assertEquals("hello2", blockingRequest(node, "hello2"));
+    }
+
+    /**
+     * Sending a request with one already in flight should result in an exception
+     */
+    @Test(expected = IllegalStateException.class)
+    public void testCantSendWithInProgress() throws Exception {
+        int node = 0;
+        blockingConnect(node);
+        selector.poll(1000L, asList(createSend(node, "test1"), createSend(node, "test2")));
+    }
+
+    /**
+     * Sending a request to a node without an existing connection should result in an exception
+     */
+    @Test(expected = IllegalStateException.class)
+    public void testCantSendWithoutConnecting() throws Exception {
+        selector.poll(1000L, asList(createSend(0, "test")));
+    }
+
+    /**
+     * Sending a request to a node with a bad hostname should result in an exception during connect
+     */
+    @Test(expected = UnresolvedAddressException.class)
+    public void testNoRouteToHost() throws Exception {
+        selector.connect(0, new InetSocketAddress("asdf.asdf.dsc", server.port), BUFFER_SIZE, BUFFER_SIZE);
+    }
+
+    /**
+     * Sending a request to a node not listening on that port should result in disconnection
+     */
+    @Test
+    public void testConnectionRefused() throws Exception {
+        int node = 0;
+        selector.connect(node, new InetSocketAddress("localhost", TestUtils.choosePort()), BUFFER_SIZE, BUFFER_SIZE);
+        while (selector.disconnected().contains(node))
+            selector.poll(1000L, EMPTY);
+    }
+
+    /**
+     * Send multiple requests to several connections in parallel. Validate that responses are received in the order that
+     * requests were sent.
+     */
+    @Test
+    public void testNormalOperation() throws Exception {
+        int conns = 5;
+        int reqs = 500;
+
+        // create connections
+        InetSocketAddress addr = new InetSocketAddress("localhost", server.port);
+        for (int i = 0; i < conns; i++)
+            selector.connect(i, addr, BUFFER_SIZE, BUFFER_SIZE);
+
+        // send echo requests and receive responses
+        int[] requests = new int[conns];
+        int[] responses = new int[conns];
+        int responseCount = 0;
+        List<NetworkSend> sends = new ArrayList<NetworkSend>();
+        for (int i = 0; i < conns; i++)
+            sends.add(createSend(i, i + "-" + 0));
+
+        // loop until we complete all requests
+        while (responseCount < conns * reqs) {
+            // do the i/o
+            selector.poll(0L, sends);
+
+            assertEquals("No disconnects should have occurred.", 0, selector.disconnected().size());
+
+            // handle any responses we may have gotten
+            for (NetworkReceive receive : selector.completedReceives()) {
+                String[] pieces = asString(receive).split("-");
+                assertEquals("Should be in the form 'conn-counter'", 2, pieces.length);
+                assertEquals("Check the source", receive.source(), Integer.parseInt(pieces[0]));
+                assertEquals("Check that the receive has kindly been rewound", 0, receive.payload().position());
+                assertEquals("Check the request counter", responses[receive.source()], Integer.parseInt(pieces[1]));
+                responses[receive.source()]++; // increment the expected counter
+                responseCount++;
+            }
+
+            // prepare new sends for the next round
+            sends.clear();
+            for (NetworkSend send : selector.completedSends()) {
+                int dest = send.destination();
+                requests[dest]++;
+                if (requests[dest] < reqs)
+                    sends.add(createSend(dest, dest + "-" + requests[dest]));
+            }
+        }
+    }
+
+    /**
+     * Validate that we can send and receive a message larger than the receive and send buffer size
+     */
+    @Test
+    public void testSendLargeRequest() throws Exception {
+        int node = 0;
+        blockingConnect(node);
+        String big = TestUtils.randomString(10 * BUFFER_SIZE);
+        assertEquals(big, blockingRequest(node, big));
+    }
+
+    /**
+     * Test sending an empty string
+     */
+    @Test
+    public void testEmptyRequest() throws Exception {
+        int node = 0;
+        blockingConnect(node);
+        assertEquals("", blockingRequest(node, ""));
+    }
+
+    private String blockingRequest(int node, String s) throws IOException {
+        selector.poll(1000L, asList(createSend(node, s)));
+        while (true) {
+            selector.poll(1000L, EMPTY);
+            for (NetworkReceive receive : selector.completedReceives())
+                if (receive.source() == node)
+                    return asString(receive);
+        }
+    }
+
+    /* connect and wait for the connection to complete */
+    private void blockingConnect(int node) throws IOException {
+        selector.connect(node, new InetSocketAddress("localhost", server.port), BUFFER_SIZE, BUFFER_SIZE);
+        while (!selector.connected().contains(node))
+            selector.poll(10000L, EMPTY);
+    }
+
+    private NetworkSend createSend(int node, String s) {
+        return new NetworkSend(node, ByteBuffer.wrap(s.getBytes()));
+    }
+
+    private String asString(NetworkReceive receive) {
+        return new String(Utils.toArray(receive.payload()));
+    }
+
+    /**
+     * A simple server that takes size delimited byte arrays and just echos them back to the sender.
+     */
+    static class EchoServer extends Thread {
+        public final int port;
+        private final ServerSocket serverSocket;
+        private final List<Thread> threads;
+        private final List<Socket> sockets;
+
+        public EchoServer() throws Exception {
+            this.port = TestUtils.choosePort();
+            this.serverSocket = new ServerSocket(port);
+            this.threads = Collections.synchronizedList(new ArrayList<Thread>());
+            this.sockets = Collections.synchronizedList(new ArrayList<Socket>());
+        }
+
+        public void run() {
+            try {
+                while (true) {
+                    final Socket socket = serverSocket.accept();
+                    sockets.add(socket);
+                    Thread thread = new Thread() {
+                        public void run() {
+                            try {
+                                DataInputStream input = new DataInputStream(socket.getInputStream());
+                                DataOutputStream output = new DataOutputStream(socket.getOutputStream());
+                                while (socket.isConnected() && !socket.isClosed()) {
+                                    int size = input.readInt();
+                                    byte[] bytes = new byte[size];
+                                    input.readFully(bytes);
+                                    output.writeInt(size);
+                                    output.write(bytes);
+                                    output.flush();
+                                }
+                            } catch (IOException e) {
+                                // ignore
+                            } finally {
+                                try {
+                                    socket.close();
+                                } catch (IOException e) {
+                                    // ignore
+                                }
+                            }
+                        }
+                    };
+                    thread.start();
+                    threads.add(thread);
+                }
+            } catch (IOException e) {
+                // ignore
+            }
+        }
+
+        public void closeConnections() throws IOException {
+            for (Socket socket : sockets)
+                socket.close();
+        }
+
+        public void close() throws IOException, InterruptedException {
+            this.serverSocket.close();
+            closeConnections();
+            for (Thread t : threads)
+                t.join();
+            join();
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/test/java/kafka/clients/producer/BufferPoolTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/kafka/clients/producer/BufferPoolTest.java b/clients/src/test/java/kafka/clients/producer/BufferPoolTest.java
new file mode 100644
index 0000000..70603c4
--- /dev/null
+++ b/clients/src/test/java/kafka/clients/producer/BufferPoolTest.java
@@ -0,0 +1,170 @@
+package kafka.clients.producer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import kafka.clients.producer.internals.BufferPool;
+import kafka.test.TestUtils;
+
+import org.junit.Test;
+
+public class BufferPoolTest {
+
+    /**
+     * Test the simple non-blocking allocation paths
+     */
+    @Test
+    public void testSimple() throws Exception {
+        int totalMemory = 64 * 1024;
+        int size = 1024;
+        BufferPool pool = new BufferPool(totalMemory, size, false);
+        ByteBuffer buffer = pool.allocate(size);
+        assertEquals("Buffer size should equal requested size.", size, buffer.limit());
+        assertEquals("Unallocated memory should have shrunk", totalMemory - size, pool.unallocatedMemory());
+        assertEquals("Available memory should have shrunk", totalMemory - size, pool.availableMemory());
+        buffer.putInt(1);
+        buffer.flip();
+        pool.deallocate(buffer);
+        assertEquals("All memory should be available", totalMemory, pool.availableMemory());
+        assertEquals("But now some is on the free list", totalMemory - size, pool.unallocatedMemory());
+        buffer = pool.allocate(size);
+        assertEquals("Recycled buffer should be cleared.", 0, buffer.position());
+        assertEquals("Recycled buffer should be cleared.", buffer.capacity(), buffer.limit());
+        pool.deallocate(buffer);
+        assertEquals("All memory should be available", totalMemory, pool.availableMemory());
+        assertEquals("Still a single buffer on the free list", totalMemory - size, pool.unallocatedMemory());
+        buffer = pool.allocate(2 * size);
+        pool.deallocate(buffer);
+        assertEquals("All memory should be available", totalMemory, pool.availableMemory());
+        assertEquals("Non-standard size didn't go to the free list.", totalMemory - size, pool.unallocatedMemory());
+    }
+
+    /**
+     * Test that we cannot try to allocate more memory then we have in the whole pool
+     */
+    @Test(expected = IllegalArgumentException.class)
+    public void testCantAllocateMoreMemoryThanWeHave() throws Exception {
+        BufferPool pool = new BufferPool(1024, 512, true);
+        ByteBuffer buffer = pool.allocate(1024);
+        assertEquals(1024, buffer.limit());
+        pool.deallocate(buffer);
+        buffer = pool.allocate(1025);
+    }
+
+    @Test
+    public void testNonblockingMode() throws Exception {
+        BufferPool pool = new BufferPool(2, 1, false);
+        pool.allocate(1);
+        try {
+            pool.allocate(2);
+            fail("The buffer allocated more than it has!");
+        } catch (BufferExhaustedException e) {
+            // this is good
+        }
+    }
+
+    /**
+     * Test that delayed allocation blocks
+     */
+    @Test
+    public void testDelayedAllocation() throws Exception {
+        BufferPool pool = new BufferPool(5 * 1024, 1024, true);
+        ByteBuffer buffer = pool.allocate(1024);
+        CountDownLatch doDealloc = asyncDeallocate(pool, buffer);
+        CountDownLatch allocation = asyncAllocate(pool, 5 * 1024);
+        assertEquals("Allocation shouldn't have happened yet, waiting on memory.", 1, allocation.getCount());
+        doDealloc.countDown(); // return the memory
+        allocation.await();
+    }
+
+    private CountDownLatch asyncDeallocate(final BufferPool pool, final ByteBuffer buffer) {
+        final CountDownLatch latch = new CountDownLatch(1);
+        new Thread() {
+            public void run() {
+                try {
+                    latch.await();
+                } catch (InterruptedException e) {
+                    e.printStackTrace();
+                }
+                pool.deallocate(buffer);
+            }
+        }.start();
+        return latch;
+    }
+
+    private CountDownLatch asyncAllocate(final BufferPool pool, final int size) {
+        final CountDownLatch completed = new CountDownLatch(1);
+        new Thread() {
+            public void run() {
+                try {
+                    pool.allocate(size);
+                } catch (InterruptedException e) {
+                    e.printStackTrace();
+                } finally {
+                    completed.countDown();
+                }
+            }
+        }.start();
+        return completed;
+    }
+
+    /**
+     * This test creates lots of threads that hammer on the pool
+     */
+    @Test
+    public void testStressfulSituation() throws Exception {
+        int numThreads = 10;
+        final int iterations = 50000;
+        final int poolableSize = 1024;
+        final int totalMemory = numThreads / 2 * poolableSize;
+        final BufferPool pool = new BufferPool(totalMemory, poolableSize, true);
+        List<StressTestThread> threads = new ArrayList<StressTestThread>();
+        for (int i = 0; i < numThreads; i++)
+            threads.add(new StressTestThread(pool, iterations));
+        for (StressTestThread thread : threads)
+            thread.start();
+        for (StressTestThread thread : threads)
+            thread.join();
+        for (StressTestThread thread : threads)
+            assertTrue("Thread should have completed all iterations successfully.", thread.success.get());
+        assertEquals(totalMemory, pool.availableMemory());
+    }
+
+    public static class StressTestThread extends Thread {
+        private final int iterations;
+        private final BufferPool pool;
+        public final AtomicBoolean success = new AtomicBoolean(false);
+
+        public StressTestThread(BufferPool pool, int iterations) {
+            this.iterations = iterations;
+            this.pool = pool;
+        }
+
+        public void run() {
+            try {
+                for (int i = 0; i < iterations; i++) {
+                    int size;
+                    if (TestUtils.random.nextBoolean())
+                        // allocate poolable size
+                        size = pool.poolableSize();
+                    else
+                        // allocate a random size
+                        size = TestUtils.random.nextInt((int) pool.totalMemory());
+                    ByteBuffer buffer = pool.allocate(size);
+                    pool.deallocate(buffer);
+                }
+                success.set(true);
+            } catch (Exception e) {
+                e.printStackTrace();
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/test/java/kafka/clients/producer/MetadataTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/kafka/clients/producer/MetadataTest.java b/clients/src/test/java/kafka/clients/producer/MetadataTest.java
new file mode 100644
index 0000000..68e4bd7
--- /dev/null
+++ b/clients/src/test/java/kafka/clients/producer/MetadataTest.java
@@ -0,0 +1,55 @@
+package kafka.clients.producer;
+
+import static java.util.Arrays.asList;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import kafka.clients.producer.internals.Metadata;
+import kafka.common.Cluster;
+import kafka.common.Node;
+import kafka.common.PartitionInfo;
+
+import org.junit.Test;
+
+public class MetadataTest {
+
+    private long refreshBackoffMs = 100;
+    private long metadataExpireMs = 1000;
+    private Metadata metadata = new Metadata(refreshBackoffMs, metadataExpireMs);
+
+    @Test
+    public void testMetadata() throws Exception {
+        long time = 0;
+        metadata.update(Cluster.empty(), time);
+        assertFalse("No update needed.", metadata.needsUpdate(time));
+        metadata.forceUpdate();
+        assertFalse("Still no updated needed due to backoff", metadata.needsUpdate(time));
+        time += refreshBackoffMs;
+        assertTrue("Update needed now that backoff time expired", metadata.needsUpdate(time));
+        String topic = "my-topic";
+        Thread t1 = asyncFetch(topic);
+        Thread t2 = asyncFetch(topic);
+        assertTrue("Awaiting update", t1.isAlive());
+        assertTrue("Awaiting update", t2.isAlive());
+        metadata.update(clusterWith(topic), time);
+        t1.join();
+        t2.join();
+        assertFalse("No update needed.", metadata.needsUpdate(time));
+        time += metadataExpireMs;
+        assertTrue("Update needed due to stale metadata.", metadata.needsUpdate(time));
+    }
+
+    private Cluster clusterWith(String topic) {
+        return new Cluster(asList(new Node(0, "localhost", 1969)), asList(new PartitionInfo(topic, 0, 0, new int[0], new int[0])));
+    }
+
+    private Thread asyncFetch(final String topic) {
+        Thread thread = new Thread() {
+            public void run() {
+                metadata.fetch(topic, Integer.MAX_VALUE);
+            }
+        };
+        thread.start();
+        return thread;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/test/java/kafka/clients/producer/MockProducerTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/kafka/clients/producer/MockProducerTest.java b/clients/src/test/java/kafka/clients/producer/MockProducerTest.java
new file mode 100644
index 0000000..61929a4
--- /dev/null
+++ b/clients/src/test/java/kafka/clients/producer/MockProducerTest.java
@@ -0,0 +1,66 @@
+package kafka.clients.producer;
+
+import static java.util.Arrays.asList;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+import kafka.common.Cluster;
+import kafka.common.Node;
+import kafka.common.PartitionInfo;
+import kafka.common.Serializer;
+import kafka.common.StringSerialization;
+
+import org.junit.Test;
+
+public class MockProducerTest {
+
+    @Test
+    public void testAutoCompleteMock() {
+        MockProducer producer = new MockProducer(true);
+        ProducerRecord record = new ProducerRecord("topic", "key", "value");
+        RecordSend send = producer.send(record);
+        assertTrue("Send should be immediately complete", send.completed());
+        assertFalse("Send should be successful", send.hasError());
+        assertEquals("Offset should be 0", 0, send.offset());
+        assertEquals("We should have the record in our history", asList(record), producer.history());
+        producer.clear();
+        assertEquals("Clear should erase our history", 0, producer.history().size());
+    }
+
+    public void testManualCompletion() {
+        MockProducer producer = new MockProducer(false);
+        ProducerRecord record1 = new ProducerRecord("topic", "key1", "value1");
+        ProducerRecord record2 = new ProducerRecord("topic", "key2", "value2");
+        RecordSend send1 = producer.send(record1);
+        assertFalse("Send shouldn't have completed", send1.completed());
+        RecordSend send2 = producer.send(record2);
+        assertFalse("Send shouldn't have completed", send2.completed());
+        assertTrue("Complete the first request", producer.completeNext());
+        assertFalse("Requst should be successful", send1.hasError());
+        assertFalse("Second request still incomplete", send2.completed());
+        IllegalArgumentException e = new IllegalArgumentException("blah");
+        assertTrue("Complete the second request with an error", producer.errorNext(e));
+        try {
+            send2.await();
+            fail("Expected error to be thrown");
+        } catch (IllegalArgumentException err) {
+            // this is good
+        }
+        assertFalse("No more requests to complete", producer.completeNext());
+    }
+
+    public void testSerializationAndPartitioning() {
+        Cluster cluster = new Cluster(asList(new Node(0, "host", -1)), asList(new PartitionInfo("topic",
+                                                                                                0,
+                                                                                                0,
+                                                                                                new int[] { 0 },
+                                                                                                new int[] { 0 })));
+        Serializer serializer = new StringSerialization();
+        Partitioner partitioner = new DefaultPartitioner();
+        MockProducer producer = new MockProducer(serializer, serializer, partitioner, cluster, true);
+        ProducerRecord record = new ProducerRecord("topic", "key", "value");
+        RecordSend send = producer.send(record);
+        assertTrue("Send should be immediately complete", send.completed());
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/test/java/kafka/clients/producer/RecordAccumulatorTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/kafka/clients/producer/RecordAccumulatorTest.java b/clients/src/test/java/kafka/clients/producer/RecordAccumulatorTest.java
new file mode 100644
index 0000000..b1ab361
--- /dev/null
+++ b/clients/src/test/java/kafka/clients/producer/RecordAccumulatorTest.java
@@ -0,0 +1,135 @@
+package kafka.clients.producer;
+
+import static java.util.Arrays.asList;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import kafka.clients.producer.internals.RecordAccumulator;
+import kafka.clients.producer.internals.RecordBatch;
+import kafka.common.TopicPartition;
+import kafka.common.metrics.Metrics;
+import kafka.common.record.CompressionType;
+import kafka.common.record.LogEntry;
+import kafka.common.record.Record;
+import kafka.common.record.Records;
+import kafka.common.utils.MockTime;
+
+import org.junit.Test;
+
+public class RecordAccumulatorTest {
+
+    private TopicPartition tp = new TopicPartition("test", 0);
+    private MockTime time = new MockTime();
+    private byte[] key = "key".getBytes();
+    private byte[] value = "value".getBytes();
+    private int msgSize = Records.LOG_OVERHEAD + Record.recordSize(key, value);
+    private Metrics metrics = new Metrics(time);
+
+    @Test
+    public void testFull() throws Exception {
+        long now = time.milliseconds();
+        RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, 10L, false, metrics, time);
+        int appends = 1024 / msgSize;
+        for (int i = 0; i < appends; i++) {
+            accum.append(tp, key, value, CompressionType.NONE, null);
+            assertEquals("No partitions should be ready.", 0, accum.ready(now).size());
+        }
+        accum.append(tp, key, value, CompressionType.NONE, null);
+        assertEquals("Our partition should be ready", asList(tp), accum.ready(time.milliseconds()));
+        List<RecordBatch> batches = accum.drain(asList(tp), Integer.MAX_VALUE);
+        assertEquals(1, batches.size());
+        RecordBatch batch = batches.get(0);
+        Iterator<LogEntry> iter = batch.records.iterator();
+        for (int i = 0; i < appends; i++) {
+            LogEntry entry = iter.next();
+            assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key());
+            assertEquals("Values should match", ByteBuffer.wrap(value), entry.record().value());
+        }
+        assertFalse("No more records", iter.hasNext());
+    }
+
+    @Test
+    public void testAppendLarge() throws Exception {
+        int batchSize = 512;
+        RecordAccumulator accum = new RecordAccumulator(batchSize, 10 * 1024, 0L, false, metrics, time);
+        accum.append(tp, key, new byte[2 * batchSize], CompressionType.NONE, null);
+        assertEquals("Our partition should be ready", asList(tp), accum.ready(time.milliseconds()));
+    }
+
+    @Test
+    public void testLinger() throws Exception {
+        long lingerMs = 10L;
+        RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, lingerMs, false, metrics, time);
+        accum.append(tp, key, value, CompressionType.NONE, null);
+        assertEquals("No partitions should be ready", 0, accum.ready(time.milliseconds()).size());
+        time.sleep(10);
+        assertEquals("Our partition should be ready", asList(tp), accum.ready(time.milliseconds()));
+        List<RecordBatch> batches = accum.drain(asList(tp), Integer.MAX_VALUE);
+        assertEquals(1, batches.size());
+        RecordBatch batch = batches.get(0);
+        Iterator<LogEntry> iter = batch.records.iterator();
+        LogEntry entry = iter.next();
+        assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key());
+        assertEquals("Values should match", ByteBuffer.wrap(value), entry.record().value());
+        assertFalse("No more records", iter.hasNext());
+    }
+
+    @Test
+    public void testPartialDrain() throws Exception {
+        RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, 10L, false, metrics, time);
+        int appends = 1024 / msgSize + 1;
+        List<TopicPartition> partitions = asList(new TopicPartition("test", 0), new TopicPartition("test", 1));
+        for (TopicPartition tp : partitions) {
+            for (int i = 0; i < appends; i++)
+                accum.append(tp, key, value, CompressionType.NONE, null);
+        }
+        assertEquals("Both partitions should be ready", 2, accum.ready(time.milliseconds()).size());
+
+        List<RecordBatch> batches = accum.drain(partitions, 1024);
+        assertEquals("But due to size bound only one partition should have been retrieved", 1, batches.size());
+    }
+
+    @Test
+    public void testStressfulSituation() throws Exception {
+        final int numThreads = 5;
+        final int msgs = 10000;
+        final int numParts = 10;
+        final RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, 0L, true, metrics, time);
+        List<Thread> threads = new ArrayList<Thread>();
+        for (int i = 0; i < numThreads; i++) {
+            threads.add(new Thread() {
+                public void run() {
+                    for (int i = 0; i < msgs; i++) {
+                        try {
+                            accum.append(new TopicPartition("test", i % numParts), key, value, CompressionType.NONE, null);
+                        } catch (Exception e) {
+                            e.printStackTrace();
+                        }
+                    }
+                }
+            });
+        }
+        for (Thread t : threads)
+            t.start();
+        int read = 0;
+        long now = time.milliseconds();
+        while (read < numThreads * msgs) {
+            List<TopicPartition> tps = accum.ready(now);
+            List<RecordBatch> batches = accum.drain(tps, 5 * 1024);
+            for (RecordBatch batch : batches) {
+                for (LogEntry entry : batch.records)
+                    read++;
+            }
+            accum.deallocate(batches);
+        }
+
+        for (Thread t : threads)
+            t.join();
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/test/java/kafka/clients/producer/RecordSendTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/kafka/clients/producer/RecordSendTest.java b/clients/src/test/java/kafka/clients/producer/RecordSendTest.java
new file mode 100644
index 0000000..f8fd14b
--- /dev/null
+++ b/clients/src/test/java/kafka/clients/producer/RecordSendTest.java
@@ -0,0 +1,76 @@
+package kafka.clients.producer;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.util.concurrent.TimeUnit;
+
+import kafka.clients.producer.internals.ProduceRequestResult;
+import kafka.common.TopicPartition;
+import kafka.common.errors.CorruptMessageException;
+import kafka.common.errors.TimeoutException;
+
+import org.junit.Test;
+
+public class RecordSendTest {
+
+    private TopicPartition topicPartition = new TopicPartition("test", 0);
+    private long baseOffset = 45;
+    private long relOffset = 5;
+
+    /**
+     * Test that waiting on a request that never completes times out
+     */
+    @Test
+    public void testTimeout() {
+        ProduceRequestResult request = new ProduceRequestResult();
+        RecordSend send = new RecordSend(relOffset, request);
+        assertFalse("Request is not completed", send.completed());
+        try {
+            send.await(5, TimeUnit.MILLISECONDS);
+            fail("Should have thrown exception.");
+        } catch (TimeoutException e) { /* this is good */
+        }
+
+        request.done(topicPartition, baseOffset, null);
+        assertTrue(send.completed());
+        assertEquals(baseOffset + relOffset, send.offset());
+    }
+
+    /**
+     * Test that an asynchronous request will eventually throw the right exception
+     */
+    @Test(expected = CorruptMessageException.class)
+    public void testError() {
+        RecordSend send = new RecordSend(relOffset, asyncRequest(baseOffset, new CorruptMessageException(), 50L));
+        send.await();
+    }
+
+    /**
+     * Test that an asynchronous request will eventually return the right offset
+     */
+    @Test
+    public void testBlocking() {
+        RecordSend send = new RecordSend(relOffset, asyncRequest(baseOffset, null, 50L));
+        assertEquals(baseOffset + relOffset, send.offset());
+    }
+
+    /* create a new request result that will be completed after the given timeout */
+    public ProduceRequestResult asyncRequest(final long baseOffset, final RuntimeException error, final long timeout) {
+        final ProduceRequestResult request = new ProduceRequestResult();
+        new Thread() {
+            public void run() {
+                try {
+                    sleep(timeout);
+                    request.done(topicPartition, baseOffset, error);
+                } catch (InterruptedException e) {
+                    e.printStackTrace();
+                }
+            }
+        }.start();
+        return request;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/test/java/kafka/clients/producer/SenderTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/kafka/clients/producer/SenderTest.java b/clients/src/test/java/kafka/clients/producer/SenderTest.java
new file mode 100644
index 0000000..73f1aba
--- /dev/null
+++ b/clients/src/test/java/kafka/clients/producer/SenderTest.java
@@ -0,0 +1,92 @@
+package kafka.clients.producer;
+
+import static java.util.Arrays.asList;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+
+import kafka.clients.producer.internals.Metadata;
+import kafka.clients.producer.internals.RecordAccumulator;
+import kafka.clients.producer.internals.Sender;
+import kafka.common.Cluster;
+import kafka.common.Node;
+import kafka.common.PartitionInfo;
+import kafka.common.TopicPartition;
+import kafka.common.metrics.Metrics;
+import kafka.common.network.NetworkReceive;
+import kafka.common.protocol.ApiKeys;
+import kafka.common.protocol.Errors;
+import kafka.common.protocol.ProtoUtils;
+import kafka.common.protocol.types.Struct;
+import kafka.common.record.CompressionType;
+import kafka.common.requests.RequestSend;
+import kafka.common.requests.ResponseHeader;
+import kafka.common.utils.MockTime;
+import kafka.test.MockSelector;
+
+import org.junit.Before;
+import org.junit.Test;
+
+public class SenderTest {
+
+    private MockTime time = new MockTime();
+    private MockSelector selector = new MockSelector(time);
+    private int batchSize = 16 * 1024;
+    private Metadata metadata = new Metadata(0, Long.MAX_VALUE);
+    private Cluster cluster = new Cluster(asList(new Node(0, "localhost", 1969)), asList(new PartitionInfo("test",
+                                                                                                           0,
+                                                                                                           0,
+                                                                                                           new int[0],
+                                                                                                           new int[0])));
+    private Metrics metrics = new Metrics(time);
+    private RecordAccumulator accumulator = new RecordAccumulator(batchSize, 1024 * 1024, 0L, false, metrics, time);
+    private Sender sender = new Sender(selector, metadata, this.accumulator, "", 1024 * 1024, 0L, (short) -1, 10000, time);
+
+    @Before
+    public void setup() {
+        metadata.update(cluster, time.milliseconds());
+    }
+
+    @Test
+    public void testSimple() throws Exception {
+        TopicPartition tp = new TopicPartition("test", 0);
+        RecordSend send = accumulator.append(tp, "key".getBytes(), "value".getBytes(), CompressionType.NONE, null);
+        sender.run(time.milliseconds());
+        assertEquals("We should have connected", 1, selector.connected().size());
+        selector.clear();
+        sender.run(time.milliseconds());
+        assertEquals("Single request should be sent", 1, selector.completedSends().size());
+        RequestSend request = (RequestSend) selector.completedSends().get(0);
+        selector.clear();
+        long offset = 42;
+        selector.completeReceive(produceResponse(request.header().correlationId(),
+                                                 cluster.leaderFor(tp).id(),
+                                                 tp.topic(),
+                                                 tp.partition(),
+                                                 offset,
+                                                 Errors.NONE.code()));
+        sender.run(time.milliseconds());
+        assertTrue("Request should be completed", send.completed());
+        assertEquals(offset, send.offset());
+    }
+
+    private NetworkReceive produceResponse(int correlation, int source, String topic, int part, long offset, int error) {
+        Struct struct = new Struct(ProtoUtils.currentResponseSchema(ApiKeys.PRODUCE.id));
+        Struct response = struct.instance("responses");
+        response.set("topic", topic);
+        Struct partResp = response.instance("partition_responses");
+        partResp.set("partition", part);
+        partResp.set("error_code", (short) error);
+        partResp.set("base_offset", offset);
+        response.set("partition_responses", new Object[] { partResp });
+        struct.set("responses", new Object[] { response });
+        ResponseHeader header = new ResponseHeader(correlation);
+        ByteBuffer buffer = ByteBuffer.allocate(header.sizeOf() + struct.sizeOf());
+        header.writeTo(buffer);
+        struct.writeTo(buffer);
+        buffer.rewind();
+        return new NetworkReceive(source, buffer);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/test/java/kafka/common/config/ConfigDefTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/kafka/common/config/ConfigDefTest.java b/clients/src/test/java/kafka/common/config/ConfigDefTest.java
new file mode 100644
index 0000000..a6a91ac
--- /dev/null
+++ b/clients/src/test/java/kafka/common/config/ConfigDefTest.java
@@ -0,0 +1,88 @@
+package kafka.common.config;
+
+import static java.util.Arrays.asList;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Properties;
+
+import kafka.common.config.ConfigDef.Range;
+import kafka.common.config.ConfigDef.Type;
+
+import org.junit.Test;
+
+public class ConfigDefTest {
+
+    @Test
+    public void testBasicTypes() {
+        ConfigDef def = new ConfigDef().define("a", Type.INT, 5, Range.between(0, 14), "docs")
+                                       .define("b", Type.LONG, "docs")
+                                       .define("c", Type.STRING, "hello", "docs")
+                                       .define("d", Type.LIST, "docs")
+                                       .define("e", Type.DOUBLE, "docs")
+                                       .define("f", Type.CLASS, "docs")
+                                       .define("g", Type.BOOLEAN, "docs");
+
+        Properties props = new Properties();
+        props.put("a", "1   ");
+        props.put("b", 2);
+        props.put("d", " a , b, c");
+        props.put("e", 42.5d);
+        props.put("f", String.class.getName());
+        props.put("g", "true");
+
+        Map<String, Object> vals = def.parse(props);
+        assertEquals(1, vals.get("a"));
+        assertEquals(2L, vals.get("b"));
+        assertEquals("hello", vals.get("c"));
+        assertEquals(asList("a", "b", "c"), vals.get("d"));
+        assertEquals(42.5d, vals.get("e"));
+        assertEquals(String.class, vals.get("f"));
+        assertEquals(true, vals.get("g"));
+    }
+
+    @Test(expected = ConfigException.class)
+    public void testInvalidDefault() {
+        new ConfigDef().define("a", Type.INT, "hello", "docs");
+    }
+
+    @Test(expected = ConfigException.class)
+    public void testNullDefault() {
+        new ConfigDef().define("a", Type.INT, null, null, "docs");
+    }
+
+    @Test(expected = ConfigException.class)
+    public void testMissingRequired() {
+        new ConfigDef().define("a", Type.INT, "docs").parse(new HashMap<String, Object>());
+    }
+
+    @Test(expected = ConfigException.class)
+    public void testDefinedTwice() {
+        new ConfigDef().define("a", Type.STRING, "docs").define("a", Type.INT, "docs");
+    }
+
+    @Test
+    public void testBadInputs() {
+        testBadInputs(Type.INT, "hello", null, "42.5", 42.5, Long.MAX_VALUE, Long.toString(Long.MAX_VALUE), new Object());
+        testBadInputs(Type.LONG, "hello", null, "42.5", Long.toString(Long.MAX_VALUE) + "00", new Object());
+        testBadInputs(Type.DOUBLE, "hello", null, new Object());
+        testBadInputs(Type.STRING, new Object());
+        testBadInputs(Type.LIST, 53, new Object());
+    }
+
+    private void testBadInputs(Type type, Object... values) {
+        for (Object value : values) {
+            Map<String, Object> m = new HashMap<String, Object>();
+            m.put("name", value);
+            ConfigDef def = new ConfigDef().define("name", type, "docs");
+            try {
+                def.parse(m);
+                fail("Expected a config exception on bad input for value " + value);
+            } catch (ConfigException e) {
+                // this is good
+            }
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/test/java/kafka/common/metrics/JmxReporterTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/kafka/common/metrics/JmxReporterTest.java b/clients/src/test/java/kafka/common/metrics/JmxReporterTest.java
new file mode 100644
index 0000000..e286261
--- /dev/null
+++ b/clients/src/test/java/kafka/common/metrics/JmxReporterTest.java
@@ -0,0 +1,21 @@
+package kafka.common.metrics;
+
+import kafka.common.metrics.stats.Avg;
+import kafka.common.metrics.stats.Total;
+
+import org.junit.Test;
+
+public class JmxReporterTest {
+
+    @Test
+    public void testJmxRegistration() throws Exception {
+        Metrics metrics = new Metrics();
+        metrics.addReporter(new JmxReporter());
+        Sensor sensor = metrics.sensor("kafka.requests");
+        sensor.add("pack.bean1.avg", new Avg());
+        sensor.add("pack.bean2.total", new Total());
+        Sensor sensor2 = metrics.sensor("kafka.blah");
+        sensor2.add("pack.bean1.some", new Total());
+        sensor2.add("pack.bean2.some", new Total());
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/test/java/kafka/common/metrics/MetricsTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/kafka/common/metrics/MetricsTest.java b/clients/src/test/java/kafka/common/metrics/MetricsTest.java
new file mode 100644
index 0000000..7d06864
--- /dev/null
+++ b/clients/src/test/java/kafka/common/metrics/MetricsTest.java
@@ -0,0 +1,176 @@
+package kafka.common.metrics;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
+
+import java.util.Arrays;
+import java.util.concurrent.TimeUnit;
+
+import kafka.common.metrics.stats.Avg;
+import kafka.common.metrics.stats.Count;
+import kafka.common.metrics.stats.Max;
+import kafka.common.metrics.stats.Min;
+import kafka.common.metrics.stats.Percentile;
+import kafka.common.metrics.stats.Percentiles;
+import kafka.common.metrics.stats.Percentiles.BucketSizing;
+import kafka.common.metrics.stats.Rate;
+import kafka.common.metrics.stats.Total;
+import kafka.common.utils.MockTime;
+
+import org.junit.Test;
+
+public class MetricsTest {
+
+    private static double EPS = 0.000001;
+
+    MockTime time = new MockTime();
+    Metrics metrics = new Metrics(new MetricConfig(), Arrays.asList((MetricsReporter) new JmxReporter()), time);
+
+    @Test
+    public void testSimpleStats() throws Exception {
+        ConstantMeasurable measurable = new ConstantMeasurable();
+        metrics.addMetric("direct.measurable", measurable);
+        Sensor s = metrics.sensor("test.sensor");
+        s.add("test.avg", new Avg());
+        s.add("test.max", new Max());
+        s.add("test.min", new Min());
+        s.add("test.rate", new Rate(TimeUnit.SECONDS));
+        s.add("test.occurences", new Rate(TimeUnit.SECONDS, new Count()));
+        s.add("test.count", new Count());
+        s.add(new Percentiles(100, -100, 100, BucketSizing.CONSTANT, new Percentile("test.median", 50.0), new Percentile("test.perc99_9",
+                                                                                                                         99.9)));
+
+        Sensor s2 = metrics.sensor("test.sensor2");
+        s2.add("s2.total", new Total());
+        s2.record(5.0);
+
+        for (int i = 0; i < 10; i++)
+            s.record(i);
+
+        // pretend 2 seconds passed...
+        time.sleep(2000);
+
+        assertEquals("s2 reflects the constant value", 5.0, metrics.metrics().get("s2.total").value(), EPS);
+        assertEquals("Avg(0...9) = 4.5", 4.5, metrics.metrics().get("test.avg").value(), EPS);
+        assertEquals("Max(0...9) = 9", 9.0, metrics.metrics().get("test.max").value(), EPS);
+        assertEquals("Min(0...9) = 0", 0.0, metrics.metrics().get("test.min").value(), EPS);
+        assertEquals("Rate(0...9) = 22.5", 22.5, metrics.metrics().get("test.rate").value(), EPS);
+        assertEquals("Occurences(0...9) = 5", 5.0, metrics.metrics().get("test.occurences").value(), EPS);
+        assertEquals("Count(0...9) = 10", 10.0, metrics.metrics().get("test.count").value(), EPS);
+    }
+
+    @Test
+    public void testHierarchicalSensors() {
+        Sensor parent1 = metrics.sensor("test.parent1");
+        parent1.add("test.parent1.count", new Count());
+        Sensor parent2 = metrics.sensor("test.parent2");
+        parent2.add("test.parent2.count", new Count());
+        Sensor child1 = metrics.sensor("test.child1", parent1, parent2);
+        child1.add("test.child1.count", new Count());
+        Sensor child2 = metrics.sensor("test.child2", parent1);
+        child2.add("test.child2.count", new Count());
+        Sensor grandchild = metrics.sensor("test.grandchild", child1);
+        grandchild.add("test.grandchild.count", new Count());
+
+        /* increment each sensor one time */
+        parent1.record();
+        parent2.record();
+        child1.record();
+        child2.record();
+        grandchild.record();
+
+        double p1 = parent1.metrics().get(0).value();
+        double p2 = parent2.metrics().get(0).value();
+        double c1 = child1.metrics().get(0).value();
+        double c2 = child2.metrics().get(0).value();
+        double gc = grandchild.metrics().get(0).value();
+
+        /* each metric should have a count equal to one + its children's count */
+        assertEquals(1.0, gc, EPS);
+        assertEquals(1.0 + gc, child1.metrics().get(0).value(), EPS);
+        assertEquals(1.0, c2, EPS);
+        assertEquals(1.0 + c1, p2, EPS);
+        assertEquals(1.0 + c1 + c2, p1, EPS);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testBadSensorHiearchy() {
+        Sensor p = metrics.sensor("parent");
+        Sensor c1 = metrics.sensor("child1", p);
+        Sensor c2 = metrics.sensor("child2", p);
+        metrics.sensor("gc", c1, c2); // should fail
+    }
+
+    @Test
+    public void testEventWindowing() {
+        Count count = new Count();
+        MetricConfig config = new MetricConfig().eventWindow(1).samples(2);
+        count.record(config, 1.0, time.nanoseconds());
+        count.record(config, 1.0, time.nanoseconds());
+        assertEquals(2.0, count.measure(config, time.nanoseconds()), EPS);
+        count.record(config, 1.0, time.nanoseconds()); // first event times out
+        assertEquals(2.0, count.measure(config, time.nanoseconds()), EPS);
+    }
+
+    @Test
+    public void testTimeWindowing() {
+        Count count = new Count();
+        MetricConfig config = new MetricConfig().timeWindow(1, TimeUnit.MILLISECONDS).samples(2);
+        count.record(config, 1.0, time.nanoseconds());
+        time.sleep(1);
+        count.record(config, 1.0, time.nanoseconds());
+        assertEquals(2.0, count.measure(config, time.nanoseconds()), EPS);
+        time.sleep(1);
+        count.record(config, 1.0, time.nanoseconds()); // oldest event times out
+        assertEquals(2.0, count.measure(config, time.nanoseconds()), EPS);
+    }
+
+    @Test
+    public void testOldDataHasNoEffect() {
+        Max max = new Max();
+        long windowMs = 100;
+        MetricConfig config = new MetricConfig().timeWindow(windowMs, TimeUnit.MILLISECONDS);
+        max.record(config, 50, time.nanoseconds());
+        time.sleep(windowMs);
+        assertEquals(Double.NEGATIVE_INFINITY, max.measure(config, time.nanoseconds()), EPS);
+    }
+
+    @Test(expected = IllegalArgumentException.class)
+    public void testDuplicateMetricName() {
+        metrics.sensor("test").add("test", new Avg());
+        metrics.sensor("test2").add("test", new Total());
+    }
+
+    @Test
+    public void testQuotas() {
+        Sensor sensor = metrics.sensor("test");
+        sensor.add("test1.total", new Total(), new MetricConfig().quota(Quota.lessThan(5.0)));
+        sensor.add("test2.total", new Total(), new MetricConfig().quota(Quota.moreThan(0.0)));
+        sensor.record(5.0);
+        try {
+            sensor.record(1.0);
+            fail("Should have gotten a quota violation.");
+        } catch (QuotaViolationException e) {
+            // this is good
+        }
+        assertEquals(6.0, metrics.metrics().get("test1.total").value(), EPS);
+        sensor.record(-6.0);
+        try {
+            sensor.record(-1.0);
+            fail("Should have gotten a quota violation.");
+        } catch (QuotaViolationException e) {
+            // this is good
+        }
+    }
+
+    public static class ConstantMeasurable implements Measurable {
+        public double value = 0.0;
+
+        @Override
+        public double measure(MetricConfig config, long now) {
+            return value;
+        }
+
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/test/java/kafka/common/metrics/stats/HistogramTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/kafka/common/metrics/stats/HistogramTest.java b/clients/src/test/java/kafka/common/metrics/stats/HistogramTest.java
new file mode 100644
index 0000000..03bdd2b
--- /dev/null
+++ b/clients/src/test/java/kafka/common/metrics/stats/HistogramTest.java
@@ -0,0 +1,56 @@
+package kafka.common.metrics.stats;
+
+import static org.junit.Assert.assertEquals;
+import kafka.common.metrics.stats.Histogram.BinScheme;
+import kafka.common.metrics.stats.Histogram.ConstantBinScheme;
+import kafka.common.metrics.stats.Histogram.LinearBinScheme;
+
+import org.junit.Test;
+
+public class HistogramTest {
+
+    private static final double EPS = 0.0000001d;
+
+    // @Test
+    public void testHistogram() {
+        BinScheme scheme = new ConstantBinScheme(12, -5, 5);
+        Histogram hist = new Histogram(scheme);
+        for (int i = -5; i < 5; i++)
+            hist.record(i);
+        for (int i = 0; i < 10; i++)
+            assertEquals(scheme.fromBin(i + 1), hist.value(i / 10.0), EPS);
+    }
+
+    @Test
+    public void testConstantBinScheme() {
+        ConstantBinScheme scheme = new ConstantBinScheme(5, -5, 5);
+        assertEquals("A value below the lower bound should map to the first bin", 0, scheme.toBin(-5.01));
+        assertEquals("A value above the upper bound should map to the last bin", 4, scheme.toBin(5.01));
+        assertEquals("Check boundary of bucket 1", 1, scheme.toBin(-5));
+        assertEquals("Check boundary of bucket 4", 4, scheme.toBin(5));
+        assertEquals("Check boundary of bucket 3", 3, scheme.toBin(4.9999));
+        checkBinningConsistency(new ConstantBinScheme(4, 0, 5));
+        checkBinningConsistency(scheme);
+    }
+
+    public void testLinearBinScheme() {
+        LinearBinScheme scheme = new LinearBinScheme(5, 5);
+        for (int i = 0; i < scheme.bins(); i++)
+            System.out.println(i + " " + scheme.fromBin(i));
+        checkBinningConsistency(scheme);
+    }
+
+    private void checkBinningConsistency(BinScheme scheme) {
+        for (int bin = 0; bin < scheme.bins(); bin++) {
+            double fromBin = scheme.fromBin(bin);
+            int binAgain = scheme.toBin(fromBin);
+            assertEquals("unbinning and rebinning " + bin
+                         + " gave a different result ("
+                         + fromBin
+                         + " was placed in bin "
+                         + binAgain
+                         + " )", bin, binAgain);
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/test/java/kafka/common/protocol/types/ProtocolSerializationTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/kafka/common/protocol/types/ProtocolSerializationTest.java b/clients/src/test/java/kafka/common/protocol/types/ProtocolSerializationTest.java
new file mode 100644
index 0000000..5204f3a
--- /dev/null
+++ b/clients/src/test/java/kafka/common/protocol/types/ProtocolSerializationTest.java
@@ -0,0 +1,96 @@
+package kafka.common.protocol.types;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.fail;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+
+import org.junit.Before;
+import org.junit.Test;
+
+public class ProtocolSerializationTest {
+
+    private Schema schema;
+    private Struct struct;
+
+    @Before
+    public void setup() {
+        this.schema = new Schema(new Field("int8", Type.INT8),
+                                 new Field("int16", Type.INT16),
+                                 new Field("int32", Type.INT32),
+                                 new Field("int64", Type.INT64),
+                                 new Field("string", Type.STRING),
+                                 new Field("bytes", Type.BYTES),
+                                 new Field("array", new ArrayOf(Type.INT32)),
+                                 new Field("struct", new Schema(new Field("field", Type.INT32))));
+        this.struct = new Struct(this.schema).set("int8", (byte) 1)
+                                             .set("int16", (short) 1)
+                                             .set("int32", (int) 1)
+                                             .set("int64", (long) 1)
+                                             .set("string", "1")
+                                             .set("bytes", "1".getBytes())
+                                             .set("array", new Object[] { 1 });
+        this.struct.set("struct", this.struct.instance("struct").set("field", new Object[] { 1, 2, 3 }));
+    }
+
+    @Test
+    public void testSimple() {
+        check(Type.INT8, (byte) -111);
+        check(Type.INT16, (short) -11111);
+        check(Type.INT32, -11111111);
+        check(Type.INT64, -11111111111L);
+        check(Type.STRING, "");
+        check(Type.STRING, "hello");
+        check(Type.STRING, "A\u00ea\u00f1\u00fcC");
+        check(Type.BYTES, ByteBuffer.allocate(0));
+        check(Type.BYTES, ByteBuffer.wrap("abcd".getBytes()));
+        check(new ArrayOf(Type.INT32), new Object[] { 1, 2, 3, 4 });
+        check(new ArrayOf(Type.STRING), new Object[] {});
+        check(new ArrayOf(Type.STRING), new Object[] { "hello", "there", "beautiful" });
+    }
+
+    @Test
+    public void testNulls() {
+        for (Field f : this.schema.fields()) {
+            Object o = this.struct.get(f);
+            try {
+                this.struct.set(f, null);
+                this.struct.validate();
+                fail("Should not allow serialization of null value.");
+            } catch (SchemaException e) {
+                // this is good
+                this.struct.set(f, o);
+            }
+        }
+    }
+
+    @Test
+    public void testDefault() {
+        Schema schema = new Schema(new Field("field", Type.INT32, "doc", 42));
+        Struct struct = new Struct(schema);
+        assertEquals("Should get the default value", 42, struct.get("field"));
+        struct.validate(); // should be valid even with missing value
+    }
+
+    private Object roundtrip(Type type, Object obj) {
+        ByteBuffer buffer = ByteBuffer.allocate(type.sizeOf(obj));
+        type.write(buffer, obj);
+        assertFalse("The buffer should now be full.", buffer.hasRemaining());
+        buffer.rewind();
+        Object read = type.read(buffer);
+        assertFalse("All bytes should have been read.", buffer.hasRemaining());
+        return read;
+    }
+
+    private void check(Type type, Object obj) {
+        Object result = roundtrip(type, obj);
+        if (obj instanceof Object[]) {
+            obj = Arrays.asList((Object[]) obj);
+            result = Arrays.asList((Object[]) result);
+        }
+        assertEquals("The object read back should be the same as what was written.", obj, result);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/test/java/kafka/common/record/MemoryRecordsTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/kafka/common/record/MemoryRecordsTest.java b/clients/src/test/java/kafka/common/record/MemoryRecordsTest.java
new file mode 100644
index 0000000..6906309
--- /dev/null
+++ b/clients/src/test/java/kafka/common/record/MemoryRecordsTest.java
@@ -0,0 +1,44 @@
+package kafka.common.record;
+
+import static kafka.common.utils.Utils.toArray;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+
+import org.junit.Test;
+
+public class MemoryRecordsTest {
+
+    @Test
+    public void testIterator() {
+        MemoryRecords recs1 = new MemoryRecords(ByteBuffer.allocate(1024));
+        MemoryRecords recs2 = new MemoryRecords(ByteBuffer.allocate(1024));
+        List<Record> list = Arrays.asList(new Record("a".getBytes(), "1".getBytes()),
+                                          new Record("b".getBytes(), "2".getBytes()),
+                                          new Record("c".getBytes(), "3".getBytes()));
+        for (int i = 0; i < list.size(); i++) {
+            Record r = list.get(i);
+            recs1.append(i, r);
+            recs2.append(i, toArray(r.key()), toArray(r.value()), r.compressionType());
+        }
+
+        for (int iteration = 0; iteration < 2; iteration++) {
+            for (MemoryRecords recs : Arrays.asList(recs1, recs2)) {
+                Iterator<LogEntry> iter = recs.iterator();
+                for (int i = 0; i < list.size(); i++) {
+                    assertTrue(iter.hasNext());
+                    LogEntry entry = iter.next();
+                    assertEquals((long) i, entry.offset());
+                    assertEquals(list.get(i), entry.record());
+                }
+                assertFalse(iter.hasNext());
+            }
+        }
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/test/java/kafka/common/record/RecordTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/kafka/common/record/RecordTest.java b/clients/src/test/java/kafka/common/record/RecordTest.java
new file mode 100644
index 0000000..9c59c9b
--- /dev/null
+++ b/clients/src/test/java/kafka/common/record/RecordTest.java
@@ -0,0 +1,87 @@
+package kafka.common.record;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collection;
+import java.util.List;
+
+import org.junit.Test;
+import org.junit.runner.RunWith;
+import org.junit.runners.Parameterized;
+import org.junit.runners.Parameterized.Parameters;
+
+@RunWith(value = Parameterized.class)
+public class RecordTest {
+
+    private ByteBuffer key;
+    private ByteBuffer value;
+    private CompressionType compression;
+    private Record record;
+
+    public RecordTest(byte[] key, byte[] value, CompressionType compression) {
+        this.key = key == null ? null : ByteBuffer.wrap(key);
+        this.value = value == null ? null : ByteBuffer.wrap(value);
+        this.compression = compression;
+        this.record = new Record(key, value, compression);
+    }
+
+    @Test
+    public void testFields() {
+        assertEquals(compression, record.compressionType());
+        assertEquals(key != null, record.hasKey());
+        assertEquals(key, record.key());
+        if (key != null)
+            assertEquals(key.limit(), record.keySize());
+        assertEquals(Record.CURRENT_MAGIC_VALUE, record.magic());
+        assertEquals(value, record.value());
+        if (value != null)
+            assertEquals(value.limit(), record.valueSize());
+    }
+
+    @Test
+    public void testChecksum() {
+        assertEquals(record.checksum(), record.computeChecksum());
+        assertTrue(record.isValid());
+        for (int i = Record.CRC_OFFSET + Record.CRC_LENGTH; i < record.size(); i++) {
+            Record copy = copyOf(record);
+            copy.buffer().put(i, (byte) 69);
+            assertFalse(copy.isValid());
+            try {
+                copy.ensureValid();
+                fail("Should fail the above test.");
+            } catch (InvalidRecordException e) {
+                // this is good
+            }
+        }
+    }
+
+    private Record copyOf(Record record) {
+        ByteBuffer buffer = ByteBuffer.allocate(record.size());
+        record.buffer().put(buffer);
+        buffer.rewind();
+        record.buffer().rewind();
+        return new Record(buffer);
+    }
+
+    @Test
+    public void testEquality() {
+        assertEquals(record, copyOf(record));
+    }
+
+    @Parameters
+    public static Collection<Object[]> data() {
+        List<Object[]> values = new ArrayList<Object[]>();
+        for (byte[] key : Arrays.asList(null, "".getBytes(), "key".getBytes()))
+            for (byte[] value : Arrays.asList(null, "".getBytes(), "value".getBytes()))
+                for (CompressionType compression : CompressionType.values())
+                    values.add(new Object[] { key, value, compression });
+        return values;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/test/java/kafka/common/utils/AbstractIteratorTest.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/kafka/common/utils/AbstractIteratorTest.java b/clients/src/test/java/kafka/common/utils/AbstractIteratorTest.java
new file mode 100644
index 0000000..7662d38
--- /dev/null
+++ b/clients/src/test/java/kafka/common/utils/AbstractIteratorTest.java
@@ -0,0 +1,54 @@
+package kafka.common.utils;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Iterator;
+import java.util.List;
+import java.util.NoSuchElementException;
+
+import org.junit.Test;
+
+public class AbstractIteratorTest {
+
+    @Test
+    public void testIterator() {
+        int max = 10;
+        List<Integer> l = new ArrayList<Integer>();
+        for (int i = 0; i < max; i++)
+            l.add(i);
+        ListIterator<Integer> iter = new ListIterator<Integer>(l);
+        for (int i = 0; i < max; i++) {
+            Integer value = i;
+            assertEquals(value, iter.peek());
+            assertTrue(iter.hasNext());
+            assertEquals(value, iter.next());
+        }
+        assertFalse(iter.hasNext());
+    }
+
+    @Test(expected = NoSuchElementException.class)
+    public void testEmptyIterator() {
+        Iterator<Object> iter = new ListIterator<Object>(Arrays.asList());
+        iter.next();
+    }
+
+    class ListIterator<T> extends AbstractIterator<T> {
+        private List<T> list;
+        private int position = 0;
+
+        public ListIterator(List<T> l) {
+            this.list = l;
+        }
+
+        public T makeNext() {
+            if (position < list.size())
+                return list.get(position++);
+            else
+                return allDone();
+        }
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/test/java/kafka/common/utils/MockTime.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/kafka/common/utils/MockTime.java b/clients/src/test/java/kafka/common/utils/MockTime.java
new file mode 100644
index 0000000..095d4f6
--- /dev/null
+++ b/clients/src/test/java/kafka/common/utils/MockTime.java
@@ -0,0 +1,28 @@
+package kafka.common.utils;
+
+import java.util.concurrent.TimeUnit;
+
+public class MockTime implements Time {
+
+    private long nanos = 0;
+
+    public MockTime() {
+        this.nanos = System.nanoTime();
+    }
+
+    @Override
+    public long milliseconds() {
+        return TimeUnit.MILLISECONDS.convert(this.nanos, TimeUnit.NANOSECONDS);
+    }
+
+    @Override
+    public long nanoseconds() {
+        return nanos;
+    }
+
+    @Override
+    public void sleep(long ms) {
+        this.nanos += TimeUnit.NANOSECONDS.convert(ms, TimeUnit.MILLISECONDS);
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/test/java/kafka/test/MetricsBench.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/kafka/test/MetricsBench.java b/clients/src/test/java/kafka/test/MetricsBench.java
new file mode 100644
index 0000000..2b164bd
--- /dev/null
+++ b/clients/src/test/java/kafka/test/MetricsBench.java
@@ -0,0 +1,38 @@
+package kafka.test;
+
+import java.util.Arrays;
+
+import kafka.common.metrics.Metrics;
+import kafka.common.metrics.Sensor;
+import kafka.common.metrics.stats.Avg;
+import kafka.common.metrics.stats.Count;
+import kafka.common.metrics.stats.Max;
+import kafka.common.metrics.stats.Percentile;
+import kafka.common.metrics.stats.Percentiles;
+import kafka.common.metrics.stats.Percentiles.BucketSizing;
+
+public class MetricsBench {
+
+    public static void main(String[] args) {
+        long iters = Long.parseLong(args[0]);
+        Metrics metrics = new Metrics();
+        Sensor parent = metrics.sensor("parent");
+        Sensor child = metrics.sensor("child", parent);
+        for (Sensor sensor : Arrays.asList(parent, child)) {
+            sensor.add(sensor.name() + ".avg", new Avg());
+            sensor.add(sensor.name() + ".count", new Count());
+            sensor.add(sensor.name() + ".max", new Max());
+            sensor.add(new Percentiles(1024,
+                                       0.0,
+                                       iters,
+                                       BucketSizing.CONSTANT,
+                                       new Percentile(sensor.name() + ".median", 50.0),
+                                       new Percentile(sensor.name() + ".p_99", 99.0)));
+        }
+        long start = System.nanoTime();
+        for (int i = 0; i < iters; i++)
+            child.record(i);
+        double ellapsed = (System.nanoTime() - start) / (double) iters;
+        System.out.println(String.format("%.2f ns per metric recording.", ellapsed));
+    }
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/test/java/kafka/test/Microbenchmarks.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/kafka/test/Microbenchmarks.java b/clients/src/test/java/kafka/test/Microbenchmarks.java
new file mode 100644
index 0000000..a0ddecb
--- /dev/null
+++ b/clients/src/test/java/kafka/test/Microbenchmarks.java
@@ -0,0 +1,143 @@
+package kafka.test;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+import java.util.Random;
+import java.util.concurrent.ConcurrentHashMap;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import kafka.common.utils.CopyOnWriteMap;
+import kafka.common.utils.SystemTime;
+
+public class Microbenchmarks {
+
+    public static void main(String[] args) throws Exception {
+
+        final int iters = Integer.parseInt(args[0]);
+        double x = 0.0;
+        long start = System.nanoTime();
+        for (int i = 0; i < iters; i++)
+            x += Math.sqrt(x);
+        System.out.println(x);
+        System.out.println("sqrt: " + (System.nanoTime() - start) / (double) iters);
+
+        // test clocks
+        systemMillis(iters);
+        systemNanos(iters);
+        long total = 0;
+        start = System.nanoTime();
+        total += systemMillis(iters);
+        System.out.println("System.currentTimeMillis(): " + (System.nanoTime() - start) / iters);
+        start = System.nanoTime();
+        total += systemNanos(iters);
+        System.out.println("System.nanoTime(): " + (System.nanoTime() - start) / iters);
+        System.out.println(total);
+
+        // test random
+        int n = 0;
+        Random random = new Random();
+        start = System.nanoTime();
+        for (int i = 0; i < iters; i++) {
+            n += random.nextInt();
+        }
+        System.out.println(n);
+        System.out.println("random: " + (System.nanoTime() - start) / iters);
+
+        float[] floats = new float[1024];
+        for (int i = 0; i < floats.length; i++)
+            floats[i] = random.nextFloat();
+        Arrays.sort(floats);
+
+        int loc = 0;
+        start = System.nanoTime();
+        for (int i = 0; i < iters; i++)
+            loc += Arrays.binarySearch(floats, floats[i % floats.length]);
+        System.out.println(loc);
+        System.out.println("binary search: " + (System.nanoTime() - start) / iters);
+
+        final SystemTime time = new SystemTime();
+        final AtomicBoolean done = new AtomicBoolean(false);
+        final Object lock = new Object();
+        Thread t1 = new Thread() {
+            public void run() {
+                time.sleep(1);
+                int counter = 0;
+                long start = time.nanoseconds();
+                for (int i = 0; i < iters; i++) {
+                    synchronized (lock) {
+                        counter++;
+                    }
+                }
+                System.out.println("synchronized: " + ((System.nanoTime() - start) / iters));
+                System.out.println(counter);
+                done.set(true);
+            }
+        };
+
+        Thread t2 = new Thread() {
+            public void run() {
+                int counter = 0;
+                while (!done.get()) {
+                    time.sleep(1);
+                    synchronized (lock) {
+                        counter += 1;
+                    }
+                }
+                System.out.println("Counter: " + counter);
+            }
+        };
+
+        t1.start();
+        t2.start();
+        t1.join();
+        t2.join();
+
+        Map<String, Integer> values = new HashMap<String, Integer>();
+        for (int i = 0; i < 100; i++)
+            values.put(Integer.toString(i), i);
+        System.out.println("HashMap:");
+        benchMap(2, 1000000, values);
+        System.out.println("ConcurentHashMap:");
+        benchMap(2, 1000000, new ConcurrentHashMap<String, Integer>(values));
+        System.out.println("CopyOnWriteMap:");
+        benchMap(2, 1000000, new CopyOnWriteMap<String, Integer>(values));
+    }
+
+    private static void benchMap(int numThreads, final int iters, final Map<String, Integer> map) throws Exception {
+        final List<String> keys = new ArrayList<String>(map.keySet());
+        final List<Thread> threads = new ArrayList<Thread>();
+        for (int i = 0; i < numThreads; i++) {
+            threads.add(new Thread() {
+                public void run() {
+                    int sum = 0;
+                    long start = System.nanoTime();
+                    for (int j = 0; j < iters; j++)
+                        map.get(keys.get(j % threads.size()));
+                    System.out.println("Map access time: " + ((System.nanoTime() - start) / (double) iters));
+                }
+            });
+        }
+        for (Thread thread : threads)
+            thread.start();
+        for (Thread thread : threads)
+            thread.join();
+    }
+
+    private static long systemMillis(int iters) {
+        long total = 0;
+        for (int i = 0; i < iters; i++)
+            total += System.currentTimeMillis();
+        return total;
+    }
+
+    private static long systemNanos(int iters) {
+        long total = 0;
+        for (int i = 0; i < iters; i++)
+            total += System.currentTimeMillis();
+        return total;
+    }
+
+}

http://git-wip-us.apache.org/repos/asf/kafka/blob/269d16d3/clients/src/test/java/kafka/test/MockSelector.java
----------------------------------------------------------------------
diff --git a/clients/src/test/java/kafka/test/MockSelector.java b/clients/src/test/java/kafka/test/MockSelector.java
new file mode 100644
index 0000000..15508f4
--- /dev/null
+++ b/clients/src/test/java/kafka/test/MockSelector.java
@@ -0,0 +1,87 @@
+package kafka.test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.util.ArrayList;
+import java.util.List;
+
+import kafka.common.network.NetworkReceive;
+import kafka.common.network.NetworkSend;
+import kafka.common.network.Selectable;
+import kafka.common.utils.Time;
+
+/**
+ * A fake selector to use for testing
+ */
+public class MockSelector implements Selectable {
+
+    private final Time time;
+    private final List<NetworkSend> completedSends = new ArrayList<NetworkSend>();
+    private final List<NetworkReceive> completedReceives = new ArrayList<NetworkReceive>();
+    private final List<Integer> disconnected = new ArrayList<Integer>();
+    private final List<Integer> connected = new ArrayList<Integer>();
+
+    public MockSelector(Time time) {
+        this.time = time;
+    }
+
+    @Override
+    public void connect(int id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException {
+        this.connected.add(id);
+    }
+
+    @Override
+    public void disconnect(int id) {
+        this.disconnected.add(id);
+    }
+
+    @Override
+    public void wakeup() {
+    }
+
+    @Override
+    public void close() {
+    }
+
+    public void clear() {
+        this.completedSends.clear();
+        this.completedReceives.clear();
+        this.disconnected.clear();
+        this.connected.clear();
+    }
+
+    @Override
+    public void poll(long timeout, List<NetworkSend> sends) throws IOException {
+        this.completedSends.addAll(sends);
+        time.sleep(timeout);
+    }
+
+    @Override
+    public List<NetworkSend> completedSends() {
+        return completedSends;
+    }
+
+    public void completeSend(NetworkSend send) {
+        this.completedSends.add(send);
+    }
+
+    @Override
+    public List<NetworkReceive> completedReceives() {
+        return completedReceives;
+    }
+
+    public void completeReceive(NetworkReceive receive) {
+        this.completedReceives.add(receive);
+    }
+
+    @Override
+    public List<Integer> disconnected() {
+        return disconnected;
+    }
+
+    @Override
+    public List<Integer> connected() {
+        return connected;
+    }
+
+}