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 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..696dc97 --- /dev/null +++ b/clients/src/main/java/kafka/clients/producer/BufferExhaustedException.java @@ -0,0 +1,13 @@ +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; + +} 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); +} 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..2e5271a --- /dev/null +++ b/clients/src/main/java/kafka/clients/producer/DefaultPartitioner.java @@ -0,0 +1,29 @@ +package kafka.clients.producer; + +import java.util.concurrent.atomic.AtomicInteger; + +import kafka.common.Cluster; +import kafka.common.utils.Utils; + +/** + * A simple partitioner that computes the partition using the murmur2 hash of the serialized key. This partitioner will + * use the partitionKey if it is supplied, otherwise it will partition based on the record key. When neither key is + * provided this partitioner will round-robin over all available partitions. + */ +public class DefaultPartitioner implements Partitioner { + + private final AtomicInteger counter = new AtomicInteger(0); + + /** + * 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; + } + +} 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..0da9185 --- /dev/null +++ b/clients/src/main/java/kafka/clients/producer/KafkaProducer.java @@ -0,0 +1,233 @@ +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.Metrics; +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. + *

+ * The producer is thread safe and should generally be shared among all threads for best performance. + *

+ * 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 here. 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 configs) { + this(new ProducerConfig(configs)); + } + + /** + * A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings + * are documented here. + */ + public KafkaProducer(Properties properties) { + this(new ProducerConfig(properties)); + } + + private KafkaProducer(ProducerConfig config) { + 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.metadata = new Metadata(); + this.metadataFetchTimeoutMs = config.getLong(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG); + 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), + new SystemTime()); + List 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.metrics = new Metrics(); + this.ioThread = new KafkaThread("kafka-network-thread", this.sender, true); + this.ioThread.start(); + } + + private static List parseAndValidateAddresses(List urls) { + List addresses = new ArrayList(); + 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. + *

+ * The send is asynchronous and this method will return immediately once the record has been serialized and stored + * in the buffer of messaging waiting to be sent. This allows sending many records in parallel without necessitating + * blocking to wait for the response after each one. + *

+ * 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 archive + * that using the {@link kafka.clients.producer.RecordSend.await() await()} method on the {@link RecordSend} this + * call returns: + * + *

+     *   ProducerRecord record = new ProducerRecord("the-topic", "key, "value");
+     *   producer.send(myRecord, null).await();
+     * 
+ * + * 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. + *

+ * 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: + * + *

+     *   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();
+     *                         }
+     *                     }
+     *                 });
+     * 
+ *

+ * 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 total.memory.bytes. If send() 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 block.on.buffer.full=false 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 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); + } + } + +} 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..75721d1 --- /dev/null +++ b/clients/src/main/java/kafka/clients/producer/MockProducer.java @@ -0,0 +1,193 @@ +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. + *

+ * 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 sent; + private final Deque completions; + private boolean autoComplete; + private Map 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 #completeLast()} or {@link #errorLast(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(); + this.sent = new ArrayList(); + this.completions = new ArrayDeque(); + } + + /** + * Create a new mock producer with no serializers or partitioner and the given autoComplete setting. + * + * Equivalent to {@link #MockProducer(Serializer, Serializer, Partition, boolean) new MockProducer(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(Serializer, Serializer, Partition, boolean) new MockProducer(null, null, null, + * 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); + long offset = nextOffset(new TopicPartition(record.topic(), partition)); + Completion completion = new Completion(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 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 history() { + return new ArrayList(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; + + public Completion(long offset, RecordSend send, ProduceRequestResult result, Callback callback) { + this.send = send; + this.offset = offset; + this.result = result; + this.callback = callback; + } + + public void complete(RuntimeException e) { + result.done(e == null ? offset : -1L, e); + if (callback != null) + callback.onCompletion(send); + } + } + +} 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. + *

+ * A partitioner can use either the original java object the user provided or the serialized bytes. + *

+ * 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); + +} 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 metrics(); + + /** + * Close this producer + */ + public void close(); + +} 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..d390887 --- /dev/null +++ b/clients/src/main/java/kafka/clients/producer/ProducerConfig.java @@ -0,0 +1,61 @@ +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; + + public static final String BROKER_LIST_CONFIG = "metadata.broker.list"; + public static final String METADATA_FETCH_TIMEOUT_CONFIG = "metadata.fetch.timeout.ms"; + public static final String MAX_PARTITION_SIZE_CONFIG = "max.partition.bytes"; + public static final String TOTAL_BUFFER_MEMORY_CONFIG = "total.memory.bytes"; + public static final String REQUIRED_ACKS_CONFIG = "request.required.acks"; + public static final String REQUEST_TIMEOUT_CONFIG = "request.timeout.ms"; + public static final String LINGER_MS_CONFIG = "linger.ms"; + public static final String VALUE_SERIALIZER_CLASS_CONFIG = "value.serializer.class"; + public static final String KEY_SERIALIZER_CLASS_CONFIG = "key.serializer.class"; + public static final String PARTITIONER_CLASS_CONFIG = "partitioner.class"; + public static final String METADATA_REFRESH_MS_CONFIG = "topic.metadata.refresh.interval.ms"; + public static final String CLIENT_ID_CONFIG = "client.id"; + public static final String SEND_BUFFER_CONFIG = "send.buffer.bytes"; + public static final String MAX_REQUEST_SIZE_CONFIG = "max.request.size"; + public static final String RECONNECT_BACKOFF_MS_CONFIG = "reconnect.backoff.ms"; + public static final String BLOCK_ON_BUFFER_FULL = "block.on.buffer.full"; + + 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"); + } + + ProducerConfig(Map props) { + super(config, props); + } + +} 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..3f74f4b --- /dev/null +++ b/clients/src/main/java/kafka/clients/producer/ProducerRecord.java @@ -0,0 +1,81 @@ +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 chosing 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; + } + +} 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 await() 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(); + } +} 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..be60821 --- /dev/null +++ b/clients/src/main/java/kafka/clients/producer/internals/BufferPool.java @@ -0,0 +1,220 @@ +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: + *

    + *
  1. There is a special "poolable size" and buffers of this size are kept in a free list and recycled + *
  2. 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. + *
+ */ +public final class BufferPool { + + private final long totalMemory; + private final int poolableSize; + private final boolean blockOnExhaustion; + private final ReentrantLock lock; + private final Deque free; + private final Deque 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(); + this.waiters = new ArrayDeque(); + 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) { + // we are out of memory and this buffer pool doesn't allow us to + // block + throw new BufferExhaustedException(); + } 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().limit(); + } + + /** + * 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 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. + *

+ * 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 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(); + } + + /** + * 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 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 topics() { + return new HashSet(this.topics); + } + + /** + * Update the cluster metadata + */ + public synchronized void update(Cluster cluster, long now) { + this.forceUpdate = false; + this.lastRefresh = now; + this.cluster = cluster; + notifyAll(); + } + +} 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..34f881e --- /dev/null +++ b/clients/src/main/java/kafka/clients/producer/internals/ProduceRequestResult.java @@ -0,0 +1,79 @@ +package kafka.clients.producer.internals; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; + +import kafka.clients.producer.RecordSend; +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 long baseOffset = -1L; + private volatile RuntimeException error; + + public ProduceRequestResult() { + } + + /** + * Mark this request as complete and unblock any threads waiting on its completion. + * @param baseOffset + * @param error + */ + public void done(long baseOffset, RuntimeException error) { + 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; + } +} 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..220b6da --- /dev/null +++ b/clients/src/main/java/kafka/clients/producer/internals/RecordAccumulator.java @@ -0,0 +1,203 @@ +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.ConcurrentHashMap; +import java.util.concurrent.ConcurrentMap; + +import kafka.clients.producer.Callback; +import kafka.clients.producer.RecordSend; +import kafka.common.TopicPartition; +import kafka.common.record.CompressionType; +import kafka.common.record.MemoryRecords; +import kafka.common.record.Record; +import kafka.common.record.Records; +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. + *

+ * 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> 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 time The time instance to use + */ + public RecordAccumulator(int batchSize, long totalSize, long lingerMs, boolean blockOnBufferFull, Time time) { + this.drainIndex = 0; + this.closed = false; + this.batchSize = batchSize; + this.lingerMs = lingerMs; + this.batches = new ConcurrentHashMap>(); + this.free = new BufferPool(totalSize, batchSize, blockOnBufferFull); + this.time = time; + } + + /** + * Add a record to the accumulator. + *

+ * 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 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 send. + *

+ * A partition is ready if ANY of the following are true: + *

    + *
  1. The record set is full + *
  2. The record set has sat in the accumulator for at least lingerMs milliseconds + *
  3. The accumulator is out of memory and threads are blocking waiting for data (in this case all partitions are + * immediately considered ready). + *
  4. The accumulator has been closed + *
+ */ + public List ready(long now) { + List ready = new ArrayList(); + boolean exhausted = this.free.queued() > 0; + for (Map.Entry> entry : this.batches.entrySet()) { + Deque deque = entry.getValue(); + synchronized (deque) { + RecordBatch batch = deque.peekFirst(); + if (batch != null) { + boolean full = deque.size() > 1; + 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 drain(List partitions, int maxSize) { + if (partitions.isEmpty()) + return Collections.emptyList(); + int size = 0; + List ready = new ArrayList(); + /* 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 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 dequeFor(TopicPartition tp) { + Deque d = this.batches.get(tp); + if (d != null) + return d; + this.batches.putIfAbsent(tp, new ArrayDeque()); + return this.batches.get(tp); + } + + /** + * Deallocate the list of record batches + */ + public void deallocate(Collection 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; + } + +} 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..7f41ad7 --- /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 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(5); + } + + /** + * 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 sufficent 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(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 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 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(); + 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 ready = this.accumulator.ready(now); + + // prune the list of ready topics to eliminate any that we aren't ready to send yet + List sendable = processReadyPartitions(cluster, ready, now); + + // should we update our metadata? + List sends = new ArrayList(sendable.size()); + InFlightRequest metadataReq = maybeMetadataRequest(cluster, now); + if (metadataReq != null) { + sends.add(metadataReq.request); + this.inFlightRequests.add(metadataReq); + } + + // create produce requests + List batches = this.accumulator.drain(sendable, this.maxRequestSize); + List 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 processReadyPartitions(Cluster cluster, List ready, long now) { + List sendable = new ArrayList(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 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 connects) { + for (Integer id : connects) + this.nodeState.get(id).state = ConnectionState.CONNECTED; + } + + /** + * Process completed sends + */ + public void handleSends(List sends) { + /* if acks = 0 then the request is satisfied once sent */ + for (NetworkSend send : sends) { + Deque 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 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 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 collate(Cluster cluster, List batches) { + Map> collated = new HashMap>(); + for (RecordBatch batch : batches) { + Node node = cluster.leaderFor(batch.topicPartition); + List found = collated.get(node.id()); + if (found == null) { + found = new ArrayList(); + collated.put(node.id(), found); + } + found.add(batch); + } + List requests = new ArrayList(collated.size()); + for (Map.Entry> 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 batches) { + Map batchesByPartition = new HashMap(); + Map> batchesByTopic = new HashMap>(); + for (RecordBatch batch : batches) { + batchesByPartition.put(batch.topicPartition, batch); + List found = batchesByTopic.get(batch.topicPartition.topic()); + if (found == null) { + found = new ArrayList(); + 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 topicDatas = new ArrayList(batchesByTopic.size()); + for (Map.Entry> entry : batchesByTopic.entrySet()) { + Struct topicData = produce.instance("topic_data"); + topicData.set("topic", entry.getKey()); + List 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 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 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> requests = new HashMap>(); + + /** + * Add the given request to the queue for the node it was directed to + */ + public void add(InFlightRequest request) { + Deque reqs = this.requests.get(request.request.destination()); + if (reqs == null) { + reqs = new ArrayDeque(); + this.requests.put(request.request.destination(), reqs); + } + reqs.addFirst(request); + } + + public Deque requestQueue(int node) { + Deque 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 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 clearAll(int node) { + Deque reqs = requests.get(node); + if (reqs == null) { + return Collections.emptyList(); + } else { + return requests.remove(node); + } + } + } + +} 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(); + } + +} 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; + } + +} 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..c3c1aa6 --- /dev/null +++ b/clients/src/main/java/kafka/common/Cluster.java @@ -0,0 +1,101 @@ +package kafka.common; + +import java.net.InetSocketAddress; +import java.util.ArrayList; +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 nodes; + private final Map nodesById; + private final Map partitionsByTopicPartition; + private final Map> 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(List nodes, List partitions) { + this.nodes = new ArrayList(nodes); + this.nodesById = new HashMap(nodes.size()); + this.partitionsByTopicPartition = new HashMap(partitions.size()); + this.partitionsByTopic = new HashMap>(partitions.size()); + + Collections.shuffle(nodes); + for (Node n : 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()); + List 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(0), new ArrayList(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 addresses) { + List nodes = new ArrayList(); + int nodeId = Integer.MIN_VALUE; + for (InetSocketAddress address : addresses) + nodes.add(new Node(nodeId++, address.getHostName(), address.getPort())); + return new Cluster(nodes, new ArrayList(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 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); + } + +} 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 configs); + +} 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. + *

+ * 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); + +} 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(); + } + +} 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(); + +} 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 + ")"; + } + +} 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; + } + +} 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. + *

+ * An implementation should handle null inputs. + *

+ * 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); + +} 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. + *

+ * 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 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); + } + } + } + +} 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; + } + +} 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. + *

+ * This class holds both the original configuration that was provided as well as the parsed + */ +public class AbstractConfig { + + private final Set used; + private final Map values; + private final Map 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) originals; + this.values = definition.parse(this.originals); + this.used = Collections.synchronizedSet(new HashSet()); + } + + 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 getList(String key) { + return (List) 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 unused() { + Set keys = new HashSet(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 getConfiguredInstance(String key, Class 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); + } + +} 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. + *

+ * Usage of this class looks something like this: + * + *

+ * ConfigDef defs = new ConfigDef();
+ * defs.define("config_name", Type.STRING, "default string value", "This configuration is used for blah blah blah.");
+ * defs.define("another_config_name", Type.INT, 42, Range.atLeast(0), "More documentation on this config");
+ * 
+ * Properties props = new Properties();
+ * props.setProperty("config_name", "some value");
+ * Map<String, Object> configs = defs.parse(props);
+ * 
+ * String someConfig = (String) configs.get("config_name"); // will return "some value"
+ * int anotherConfig = (Integer) configs.get("another_config_name"); // will return default value of 42
+ * 
+ * + * 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 configKeys = new HashMap(); + + /** + * 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 parse(Map props) { + /* parse all known keys */ + Map values = new HashMap(); + 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; + } + + } +} 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)); + } + +} 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; + } + +} 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); + } + +} 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); + } + +} 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); + } + +} 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); + } + +} 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); + } + +} 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); + } + +} 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); + } + +} 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 + *
    + *
  1. Transient, there is no point retrying a error due to a non-existant topic or message too large + *
  2. Idempotent, the exception is known to not change any state on the server + *
+ * 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() { + } + +} 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); + } + +} 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); + } + +} 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); + } + +} 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..42e787b --- /dev/null +++ b/clients/src/main/java/kafka/common/metrics/CompoundStat.java @@ -0,0 +1,36 @@ +package kafka.common.metrics; + +import java.util.List; + +public interface CompoundStat extends Stat { + + public List 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; + } + + } + +} 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..f354b67 --- /dev/null +++ b/clients/src/main/java/kafka/common/metrics/JmxReporter.java @@ -0,0 +1,169 @@ +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; + +public class JmxReporter implements MetricsReporter { + + private final Map mbeans = new HashMap(); + + @Override + public synchronized void init(List 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(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 metrics; + + public KafkaMbean(String packageName, String beanName) throws MalformedObjectNameException { + this.beanName = beanName; + this.metrics = new HashMap(); + 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 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."); + } + + } + +} 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; + } + } +} 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..b815b59 --- /dev/null +++ b/clients/src/main/java/kafka/common/metrics/Measurable.java @@ -0,0 +1,7 @@ +package kafka.common.metrics; + +public interface Measurable { + + public double measure(MetricConfig config, long now); + +} 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..912ef2f --- /dev/null +++ b/clients/src/main/java/kafka/common/metrics/MeasurableStat.java @@ -0,0 +1,5 @@ +package kafka.common.metrics; + +public interface MeasurableStat extends Stat, Measurable { + +} 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; + } +} 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..57a8f54 --- /dev/null +++ b/clients/src/main/java/kafka/common/metrics/Metrics.java @@ -0,0 +1,91 @@ +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 all the probes and metrics + */ +public class Metrics { + + private final MetricConfig config; + private final ConcurrentMap metrics; + private final ConcurrentMap sensors; + private final List reporters; + private final Time time; + + public Metrics() { + this(new MetricConfig()); + } + + public Metrics(MetricConfig defaultConfig) { + this(defaultConfig, new ArrayList(0), new SystemTime()); + } + + public Metrics(MetricConfig defaultConfig, List reporters, Time time) { + this.config = defaultConfig; + this.sensors = new ConcurrentHashMap(); + this.metrics = new ConcurrentHashMap(); + this.reporters = Utils.notNull(reporters); + this.time = time; + for (MetricsReporter reporter : reporters) + reporter.init(new ArrayList()); + } + + public Sensor sensor(String name, Sensor... parents) { + return sensor(name, null, parents); + } + + 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; + } + + public KafkaMetric addMetric(String name, Measurable measurable) { + return addMetric(name, null, measurable); + } + + public KafkaMetric addMetric(String name, MetricConfig config, Measurable measurable) { + return addMetric(name, "", config, measurable); + } + + public synchronized KafkaMetric addMetric(String name, String description, MetricConfig config, Measurable measurable) { + KafkaMetric m = new KafkaMetric(this, + Utils.notNull(name), + Utils.notNull(description), + Utils.notNull(measurable), + config == null ? this.config : config, + time); + registerMetric(m); + return m; + } + + public synchronized void addReporter(MetricsReporter reporter) { + Utils.notNull(reporter).init(new ArrayList(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); + } + + public Map metrics() { + return this.metrics; + } + +} 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..493a0a9 --- /dev/null +++ b/clients/src/main/java/kafka/common/metrics/MetricsReporter.java @@ -0,0 +1,13 @@ +package kafka.common.metrics; + +import java.util.List; + +public interface MetricsReporter { + + public void init(List metrics); + + public void metricChange(KafkaMetric metric); + + public void close(); + +} 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..8d00b40 --- /dev/null +++ b/clients/src/main/java/kafka/common/metrics/Quota.java @@ -0,0 +1,33 @@ +package kafka.common.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); + } + +} 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..0d447c3 --- /dev/null +++ b/clients/src/main/java/kafka/common/metrics/QuotaViolationException.java @@ -0,0 +1,13 @@ +package kafka.common.metrics; + +import kafka.common.KafkaException; + +public class QuotaViolationException extends KafkaException { + + private static final long serialVersionUID = 1L; + + public QuotaViolationException(String m) { + super(m); + } + +} 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..dfe1444 --- /dev/null +++ b/clients/src/main/java/kafka/common/metrics/Sensor.java @@ -0,0 +1,116 @@ +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; + +public final class Sensor { + + private final Metrics registry; + private final String name; + private final Sensor[] parents; + private final List stats; + private final List 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(); + this.stats = new ArrayList(); + this.config = config; + this.time = time; + checkForest(new HashSet()); + } + + private void checkForest(Set 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); + } + + public String name() { + return this.name; + } + + public void record() { + record(1.0); + } + + 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()); + } + } + } + + public void add(CompoundStat stat) { + add(stat, null); + } + + 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); + } + } + + public void add(String name, String description, MeasurableStat stat) { + add(name, description, stat, null); + } + + public void add(String name, MeasurableStat stat) { + add(name, stat, null); + } + + public void add(String name, MeasurableStat stat, MetricConfig config) { + add(name, "", stat, config); + } + + 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 metrics() { + return Collections.unmodifiableList(this.metrics); + } + +} 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..db48fdc --- /dev/null +++ b/clients/src/main/java/kafka/common/metrics/Stat.java @@ -0,0 +1,7 @@ +package kafka.common.metrics; + +public interface Stat { + + public void record(MetricConfig config, double value, long time); + +} 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..5dd305d --- /dev/null +++ b/clients/src/main/java/kafka/common/metrics/stats/Avg.java @@ -0,0 +1,30 @@ +package kafka.common.metrics.stats; + +import java.util.List; + +import kafka.common.metrics.MetricConfig; + +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 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; + } + +} 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..2d60e09 --- /dev/null +++ b/clients/src/main/java/kafka/common/metrics/stats/Count.java @@ -0,0 +1,26 @@ +package kafka.common.metrics.stats; + +import java.util.List; + +import kafka.common.metrics.MetricConfig; + +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 samples, MetricConfig config, long now) { + double total = 0.0; + for (int i = 0; i < samples.size(); i++) + total += samples.get(i).value; + return total; + } + +} 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)); + } + } + } + +} 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..0ebebb8 --- /dev/null +++ b/clients/src/main/java/kafka/common/metrics/stats/Max.java @@ -0,0 +1,26 @@ +package kafka.common.metrics.stats; + +import java.util.List; + +import kafka.common.metrics.MetricConfig; + +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 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; + } + +} 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..802efa7 --- /dev/null +++ b/clients/src/main/java/kafka/common/metrics/stats/Min.java @@ -0,0 +1,26 @@ +package kafka.common.metrics.stats; + +import java.util.List; + +import kafka.common.metrics.MetricConfig; + +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 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; + } + +} 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; + } + +} 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..df221ca --- /dev/null +++ b/clients/src/main/java/kafka/common/metrics/stats/Percentiles.java @@ -0,0 +1,69 @@ +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; + +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 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 stats() { + List ms = new ArrayList(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); + } + +} 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..d87141e --- /dev/null +++ b/clients/src/main/java/kafka/common/metrics/stats/Rate.java @@ -0,0 +1,79 @@ +package kafka.common.metrics.stats; + +import java.util.List; +import java.util.concurrent.TimeUnit; + +import kafka.common.metrics.MeasurableStat; +import kafka.common.metrics.MetricConfig; + +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 samples, MetricConfig config, long now) { + double total = 0.0; + for (int i = 0; i < samples.size(); i++) + total += samples.get(i).value; + return total; + } + + } +} 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..a34634a --- /dev/null +++ b/clients/src/main/java/kafka/common/metrics/stats/SampledStat.java @@ -0,0 +1,96 @@ +package kafka.common.metrics.stats; + +import java.util.ArrayList; +import java.util.List; + +import kafka.common.metrics.MeasurableStat; +import kafka.common.metrics.MetricConfig; + +public abstract class SampledStat implements MeasurableStat { + + private double initialValue; + private int current = 0; + private List samples; + + public SampledStat(double initialValue) { + this.initialValue = initialValue; + this.samples = new ArrayList(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 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(); + } + } + +} 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..22195ba --- /dev/null +++ b/clients/src/main/java/kafka/common/metrics/stats/Total.java @@ -0,0 +1,28 @@ +package kafka.common.metrics.stats; + +import kafka.common.metrics.MeasurableStat; +import kafka.common.metrics.MetricConfig; + +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; + } + +} diff --git a/clients/src/main/java/kafka/common/metrics/stats/WindowedStat.java b/clients/src/main/java/kafka/common/metrics/stats/WindowedStat.java new file mode 100644 index 0000000..74fbe13 --- /dev/null +++ b/clients/src/main/java/kafka/common/metrics/stats/WindowedStat.java @@ -0,0 +1,83 @@ +package kafka.common.metrics.stats; + +import kafka.common.metrics.MeasurableStat; +import kafka.common.metrics.MetricConfig; + +public abstract class WindowedStat implements MeasurableStat { + + protected WindowedScalar current; + protected WindowedScalar shadow; + + protected WindowedStat(WindowedScalar current, WindowedScalar shadow) { + this.current = current; + this.shadow = shadow; + } + + public WindowedStat(double initialValue) { + this(new WindowedScalar(initialValue), new WindowedScalar(initialValue)); + } + + @Override + public void record(MetricConfig config, double value, long time) { + if (this.current.halfComplete(config.timeWindowNs(), config.eventWindow(), time)) + this.shadow.reset(time); + if (this.current.complete(config.timeWindowNs(), config.eventWindow(), time)) + swap(time); + this.current.update(next(this.current, value, time), time); + this.shadow.update(next(this.shadow, value, time), time); + } + + private void swap(long time) { + WindowedScalar s = this.current; + this.current = this.shadow; + this.shadow = s; + this.shadow.reset(time); + } + + @Override + public double measure(MetricConfig config, long now) { + return current.value; + } + + public abstract double next(WindowedScalar stat, double newValue, long time); + + protected static final class WindowedScalar { + + final double initialValue; + long lastWindow; + long eventCount; + double value; + + private WindowedScalar(double initialValue, long lastWindow, long eventCount, double value) { + this.initialValue = initialValue; + this.lastWindow = lastWindow; + this.eventCount = eventCount; + this.value = value; + } + + protected WindowedScalar(double initialValue) { + this(initialValue, 0L, Long.MAX_VALUE, initialValue); + } + + public boolean halfComplete(long timeWindow, long eventWindow, long now) { + return this.lastWindow - now >= timeWindow / 2 || eventCount >= eventWindow / 2; + } + + public boolean complete(long timeWindow, long eventWindow, long now) { + return now - this.lastWindow >= timeWindow || eventCount >= eventWindow; + } + + public void update(double value, long time) { + this.eventCount++; + this.value = value; + } + + public void reset(long time) { + this.lastWindow = time; + this.eventCount = 0; + this.value = initialValue; + } + + } + +} 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; + } + +} 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; + } + +} 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; + } + +} 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; + } + +} 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; + +} 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 sends) throws IOException; + + /** + * The list of sends that completed on the last {@link #poll(long, List) poll()} call. + */ + public List completedSends(); + + /** + * The list of receives that completed on the last {@link #poll(long, List) poll()} call. + */ + public List completedReceives(); + + /** + * The list of connections that finished disconnecting on the last {@link #poll(long, List) poll()} + * call. + */ + public List disconnected(); + + /** + * The list of connections that completed their connection on the last {@link #poll(long, List) poll()} + * call. + */ + public List connected(); + +} \ No newline at end of file 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. + *

+ * This class works with {@link NetworkSend} and {@link NetworkReceive} to transmit size-delimited network requests and + * responses. + *

+ * A connection can be added to the selector associated with an integer id by doing + * + *

+ * selector.connect(42, new InetSocketAddress("google.com", server.port), 64000, 64000);
+ * 
+ * + * 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 poll() call. + * + *
+ * List<NetworkRequest> requestsToSend = Arrays.asList(new NetworkRequest(0, myBytes), new NetworkRequest(1, myOtherBytes));
+ * selector.poll(TIMEOUT_MS, requestsToSend);
+ * 
+ * + * The selector maintains several lists that are reset by each call to poll() which are available via + * various getters. These are reset by each call to poll(). + * + * This class is not thread safe! + */ +public class Selector implements Selectable { + + private final java.nio.channels.Selector selector; + private final Map keys; + private final List completedSends; + private final List completedReceives; + private final List disconnected; + private final List 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(); + this.completedSends = new ArrayList(); + this.completedReceives = new ArrayList(); + this.connected = new ArrayList(); + this.disconnected = new ArrayList(); + } + + /** + * Begin connecting to the given address and add the connection to this selector associated with the given id + * number. + *

+ * 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. + *

+ * 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 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 keys = this.selector.selectedKeys(); + Iterator 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 completedSends() { + return this.completedSends; + } + + @Override + public List completedReceives() { + return this.completedReceives; + } + + @Override + public List disconnected() { + return this.disconnected; + } + + @Override + public List 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; + } + } + +} 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; + +} 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 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, Errors> classToError = new HashMap, Errors>(); + private static Map codeToError = new HashMap(); + 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; + } +} 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 brokers = new ArrayList(); + 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 partitions = new ArrayList(); + 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; + } + +} 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."); + } + +} 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[]."); + } + } +} 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; + } + +} 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 fieldsByName; + + public Schema(Field... fs) { + this.fields = new Field[fs.length]; + this.fieldsByName = new HashMap(); + 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 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); + } + +} 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(); + } + +} 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."); + } + }; + +} 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); + } +} 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); + } + +} 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 + ")"; + } +} 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 iterator() { + return new RecordsIterator(this.buffer); + } + + /* TODO: allow reuse of the buffer used for iteration */ + public static class RecordsIterator extends AbstractIterator { + 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)); + } + } + +} 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(); + } + +} 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 { + + 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(); + +} 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(); + } +} 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; + } + +} 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))); + } + +} 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 The type of thing we are iterating over + */ +public abstract class AbstractIterator implements Iterator { + + 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; + } + } + +} 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 implements ConcurrentMap { + + private volatile Map map; + + public CopyOnWriteMap() { + this.map = Collections.emptyMap(); + } + + public CopyOnWriteMap(Map 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> entrySet() { + return map.entrySet(); + } + + @Override + public V get(Object k) { + return map.get(k); + } + + @Override + public boolean isEmpty() { + return map.isEmpty(); + } + + @Override + public Set keySet() { + return map.keySet(); + } + + @Override + public int size() { + return map.size(); + } + + @Override + public Collection values() { + return map.values(); + } + + @Override + public synchronized void clear() { + this.map = Collections.emptyMap(); + } + + @Override + public synchronized V put(K k, V v) { + Map copy = new HashMap(this.map); + V prev = copy.put(k, v); + this.map = Collections.unmodifiableMap(copy); + return prev; + } + + @Override + public synchronized void putAll(Map entries) { + Map copy = new HashMap(this.map); + copy.putAll(entries); + this.map = Collections.unmodifiableMap(copy); + } + + @Override + public synchronized V remove(Object key) { + Map copy = new HashMap(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; + } + } + +} 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, + 0xCB3FF55E, + 0x006326FB, + 0xF135942E, + 0x3A69478B, + 0xBCFD3525, + 0x77A1E680, + 0x6AA4D638, + 0xA1F8059D, + 0x276C7733, + 0xEC30A496, + 0x191C11EE, + 0xD240C24B, + 0x54D4B0E5, + 0x9F886340, + 0x828D53F8, + 0x49D1805D, + 0xCF45F2F3, + 0x04192156, + 0xF54F9383, + 0x3E134026, + 0xB8873288, + 0x73DBE12D, + 0x6EDED195, + 0xA5820230, + 0x2316709E, + 0xE84AA33B, + 0x1ACA1375, + 0xD196C0D0, + 0x5702B27E, + 0x9C5E61DB, + 0x815B5163, + 0x4A0782C6, + 0xCC93F068, + 0x07CF23CD, + 0xF6999118, + 0x3DC542BD, + 0xBB513013, + 0x700DE3B6, + 0x6D08D30E, + 0xA65400AB, + 0x20C07205, + 0xEB9CA1A0, + 0x11E81EB4, + 0xDAB4CD11, + 0x5C20BFBF, + 0x977C6C1A, + 0x8A795CA2, + 0x41258F07, + 0xC7B1FDA9, + 0x0CED2E0C, + 0xFDBB9CD9, + 0x36E74F7C, + 0xB0733DD2, + 0x7B2FEE77, + 0x662ADECF, + 0xAD760D6A, + 0x2BE27FC4, + 0xE0BEAC61, + 0x123E1C2F, + 0xD962CF8A, + 0x5FF6BD24, + 0x94AA6E81, + 0x89AF5E39, + 0x42F38D9C, + 0xC467FF32, + 0x0F3B2C97, + 0xFE6D9E42, + 0x35314DE7, + 0xB3A53F49, + 0x78F9ECEC, + 0x65FCDC54, + 0xAEA00FF1, + 0x28347D5F, + 0xE368AEFA, + 0x16441B82, + 0xDD18C827, + 0x5B8CBA89, + 0x90D0692C, + 0x8DD55994, + 0x46898A31, + 0xC01DF89F, + 0x0B412B3A, + 0xFA1799EF, + 0x314B4A4A, + 0xB7DF38E4, + 0x7C83EB41, + 0x6186DBF9, + 0xAADA085C, + 0x2C4E7AF2, + 0xE712A957, + 0x15921919, + 0xDECECABC, + 0x585AB812, + 0x93066BB7, + 0x8E035B0F, + 0x455F88AA, + 0xC3CBFA04, + 0x089729A1, + 0xF9C19B74, + 0x329D48D1, + 0xB4093A7F, + 0x7F55E9DA, + 0x6250D962, + 0xA90C0AC7, + 0x2F987869, + 0xE4C4ABCC, + /* T8_6 */ + 0x00000000, + 0xA6770BB4, + 0x979F1129, + 0x31E81A9D, + 0xF44F2413, + 0x52382FA7, + 0x63D0353A, + 0xC5A73E8E, + 0x33EF4E67, + 0x959845D3, + 0xA4705F4E, + 0x020754FA, + 0xC7A06A74, + 0x61D761C0, + 0x503F7B5D, + 0xF64870E9, + 0x67DE9CCE, + 0xC1A9977A, + 0xF0418DE7, + 0x56368653, + 0x9391B8DD, + 0x35E6B369, + 0x040EA9F4, + 0xA279A240, + 0x5431D2A9, + 0xF246D91D, + 0xC3AEC380, + 0x65D9C834, + 0xA07EF6BA, + 0x0609FD0E, + 0x37E1E793, + 0x9196EC27, + 0xCFBD399C, + 0x69CA3228, + 0x582228B5, + 0xFE552301, + 0x3BF21D8F, + 0x9D85163B, + 0xAC6D0CA6, + 0x0A1A0712, + 0xFC5277FB, + 0x5A257C4F, + 0x6BCD66D2, + 0xCDBA6D66, + 0x081D53E8, + 0xAE6A585C, + 0x9F8242C1, + 0x39F54975, + 0xA863A552, + 0x0E14AEE6, + 0x3FFCB47B, + 0x998BBFCF, + 0x5C2C8141, + 0xFA5B8AF5, + 0xCBB39068, + 0x6DC49BDC, + 0x9B8CEB35, + 0x3DFBE081, + 0x0C13FA1C, + 0xAA64F1A8, + 0x6FC3CF26, + 0xC9B4C492, + 0xF85CDE0F, + 0x5E2BD5BB, + 0x440B7579, + 0xE27C7ECD, + 0xD3946450, + 0x75E36FE4, + 0xB044516A, + 0x16335ADE, + 0x27DB4043, + 0x81AC4BF7, + 0x77E43B1E, + 0xD19330AA, + 0xE07B2A37, + 0x460C2183, + 0x83AB1F0D, + 0x25DC14B9, + 0x14340E24, + 0xB2430590, + 0x23D5E9B7, + 0x85A2E203, + 0xB44AF89E, + 0x123DF32A, + 0xD79ACDA4, + 0x71EDC610, + 0x4005DC8D, + 0xE672D739, + 0x103AA7D0, + 0xB64DAC64, + 0x87A5B6F9, + 0x21D2BD4D, + 0xE47583C3, + 0x42028877, + 0x73EA92EA, + 0xD59D995E, + 0x8BB64CE5, + 0x2DC14751, + 0x1C295DCC, + 0xBA5E5678, + 0x7FF968F6, + 0xD98E6342, + 0xE86679DF, + 0x4E11726B, + 0xB8590282, + 0x1E2E0936, + 0x2FC613AB, + 0x89B1181F, + 0x4C162691, + 0xEA612D25, + 0xDB8937B8, + 0x7DFE3C0C, + 0xEC68D02B, + 0x4A1FDB9F, + 0x7BF7C102, + 0xDD80CAB6, + 0x1827F438, + 0xBE50FF8C, + 0x8FB8E511, + 0x29CFEEA5, + 0xDF879E4C, + 0x79F095F8, + 0x48188F65, + 0xEE6F84D1, + 0x2BC8BA5F, + 0x8DBFB1EB, + 0xBC57AB76, + 0x1A20A0C2, + 0x8816EAF2, + 0x2E61E146, + 0x1F89FBDB, + 0xB9FEF06F, + 0x7C59CEE1, + 0xDA2EC555, + 0xEBC6DFC8, + 0x4DB1D47C, + 0xBBF9A495, + 0x1D8EAF21, + 0x2C66B5BC, + 0x8A11BE08, + 0x4FB68086, + 0xE9C18B32, + 0xD82991AF, + 0x7E5E9A1B, + 0xEFC8763C, + 0x49BF7D88, + 0x78576715, + 0xDE206CA1, + 0x1B87522F, + 0xBDF0599B, + 0x8C184306, + 0x2A6F48B2, + 0xDC27385B, + 0x7A5033EF, + 0x4BB82972, + 0xEDCF22C6, + 0x28681C48, + 0x8E1F17FC, + 0xBFF70D61, + 0x198006D5, + 0x47ABD36E, + 0xE1DCD8DA, + 0xD034C247, + 0x7643C9F3, + 0xB3E4F77D, + 0x1593FCC9, + 0x247BE654, + 0x820CEDE0, + 0x74449D09, + 0xD23396BD, + 0xE3DB8C20, + 0x45AC8794, + 0x800BB91A, + 0x267CB2AE, + 0x1794A833, + 0xB1E3A387, + 0x20754FA0, + 0x86024414, + 0xB7EA5E89, + 0x119D553D, + 0xD43A6BB3, + 0x724D6007, + 0x43A57A9A, + 0xE5D2712E, + 0x139A01C7, + 0xB5ED0A73, + 0x840510EE, + 0x22721B5A, + 0xE7D525D4, + 0x41A22E60, + 0x704A34FD, + 0xD63D3F49, + 0xCC1D9F8B, + 0x6A6A943F, + 0x5B828EA2, + 0xFDF58516, + 0x3852BB98, + 0x9E25B02C, + 0xAFCDAAB1, + 0x09BAA105, + 0xFFF2D1EC, + 0x5985DA58, + 0x686DC0C5, + 0xCE1ACB71, + 0x0BBDF5FF, + 0xADCAFE4B, + 0x9C22E4D6, + 0x3A55EF62, + 0xABC30345, + 0x0DB408F1, + 0x3C5C126C, + 0x9A2B19D8, + 0x5F8C2756, + 0xF9FB2CE2, + 0xC813367F, + 0x6E643DCB, + 0x982C4D22, + 0x3E5B4696, + 0x0FB35C0B, + 0xA9C457BF, + 0x6C636931, + 0xCA146285, + 0xFBFC7818, + 0x5D8B73AC, + 0x03A0A617, + 0xA5D7ADA3, + 0x943FB73E, + 0x3248BC8A, + 0xF7EF8204, + 0x519889B0, + 0x6070932D, + 0xC6079899, + 0x304FE870, + 0x9638E3C4, + 0xA7D0F959, + 0x01A7F2ED, + 0xC400CC63, + 0x6277C7D7, + 0x539FDD4A, + 0xF5E8D6FE, + 0x647E3AD9, + 0xC209316D, + 0xF3E12BF0, + 0x55962044, + 0x90311ECA, + 0x3646157E, + 0x07AE0FE3, + 0xA1D90457, + 0x579174BE, + 0xF1E67F0A, + 0xC00E6597, + 0x66796E23, + 0xA3DE50AD, + 0x05A95B19, + 0x34414184, + 0x92364A30, + /* T8_7 */ + 0x00000000, + 0xCCAA009E, + 0x4225077D, + 0x8E8F07E3, + 0x844A0EFA, + 0x48E00E64, + 0xC66F0987, + 0x0AC50919, + 0xD3E51BB5, + 0x1F4F1B2B, + 0x91C01CC8, + 0x5D6A1C56, + 0x57AF154F, + 0x9B0515D1, + 0x158A1232, + 0xD92012AC, + 0x7CBB312B, + 0xB01131B5, + 0x3E9E3656, + 0xF23436C8, + 0xF8F13FD1, + 0x345B3F4F, + 0xBAD438AC, + 0x767E3832, + 0xAF5E2A9E, + 0x63F42A00, + 0xED7B2DE3, + 0x21D12D7D, + 0x2B142464, + 0xE7BE24FA, + 0x69312319, + 0xA59B2387, + 0xF9766256, + 0x35DC62C8, + 0xBB53652B, + 0x77F965B5, + 0x7D3C6CAC, + 0xB1966C32, + 0x3F196BD1, + 0xF3B36B4F, + 0x2A9379E3, + 0xE639797D, + 0x68B67E9E, + 0xA41C7E00, + 0xAED97719, + 0x62737787, + 0xECFC7064, + 0x205670FA, + 0x85CD537D, + 0x496753E3, + 0xC7E85400, + 0x0B42549E, + 0x01875D87, + 0xCD2D5D19, + 0x43A25AFA, + 0x8F085A64, + 0x562848C8, + 0x9A824856, + 0x140D4FB5, + 0xD8A74F2B, + 0xD2624632, + 0x1EC846AC, + 0x9047414F, + 0x5CED41D1, + 0x299DC2ED, + 0xE537C273, + 0x6BB8C590, + 0xA712C50E, + 0xADD7CC17, + 0x617DCC89, + 0xEFF2CB6A, + 0x2358CBF4, + 0xFA78D958, + 0x36D2D9C6, + 0xB85DDE25, + 0x74F7DEBB, + 0x7E32D7A2, + 0xB298D73C, + 0x3C17D0DF, + 0xF0BDD041, + 0x5526F3C6, + 0x998CF358, + 0x1703F4BB, + 0xDBA9F425, + 0xD16CFD3C, + 0x1DC6FDA2, + 0x9349FA41, + 0x5FE3FADF, + 0x86C3E873, + 0x4A69E8ED, + 0xC4E6EF0E, + 0x084CEF90, + 0x0289E689, + 0xCE23E617, + 0x40ACE1F4, + 0x8C06E16A, + 0xD0EBA0BB, + 0x1C41A025, + 0x92CEA7C6, + 0x5E64A758, + 0x54A1AE41, + 0x980BAEDF, + 0x1684A93C, + 0xDA2EA9A2, + 0x030EBB0E, + 0xCFA4BB90, + 0x412BBC73, + 0x8D81BCED, + 0x8744B5F4, + 0x4BEEB56A, + 0xC561B289, + 0x09CBB217, + 0xAC509190, + 0x60FA910E, + 0xEE7596ED, + 0x22DF9673, + 0x281A9F6A, + 0xE4B09FF4, + 0x6A3F9817, + 0xA6959889, + 0x7FB58A25, + 0xB31F8ABB, + 0x3D908D58, + 0xF13A8DC6, + 0xFBFF84DF, + 0x37558441, + 0xB9DA83A2, + 0x7570833C, + 0x533B85DA, + 0x9F918544, + 0x111E82A7, + 0xDDB48239, + 0xD7718B20, + 0x1BDB8BBE, + 0x95548C5D, + 0x59FE8CC3, + 0x80DE9E6F, + 0x4C749EF1, + 0xC2FB9912, + 0x0E51998C, + 0x04949095, + 0xC83E900B, + 0x46B197E8, + 0x8A1B9776, + 0x2F80B4F1, + 0xE32AB46F, + 0x6DA5B38C, + 0xA10FB312, + 0xABCABA0B, + 0x6760BA95, + 0xE9EFBD76, + 0x2545BDE8, + 0xFC65AF44, + 0x30CFAFDA, + 0xBE40A839, + 0x72EAA8A7, + 0x782FA1BE, + 0xB485A120, + 0x3A0AA6C3, + 0xF6A0A65D, + 0xAA4DE78C, + 0x66E7E712, + 0xE868E0F1, + 0x24C2E06F, + 0x2E07E976, + 0xE2ADE9E8, + 0x6C22EE0B, + 0xA088EE95, + 0x79A8FC39, + 0xB502FCA7, + 0x3B8DFB44, + 0xF727FBDA, + 0xFDE2F2C3, + 0x3148F25D, + 0xBFC7F5BE, + 0x736DF520, + 0xD6F6D6A7, + 0x1A5CD639, + 0x94D3D1DA, + 0x5879D144, + 0x52BCD85D, + 0x9E16D8C3, + 0x1099DF20, + 0xDC33DFBE, + 0x0513CD12, + 0xC9B9CD8C, + 0x4736CA6F, + 0x8B9CCAF1, + 0x8159C3E8, + 0x4DF3C376, + 0xC37CC495, + 0x0FD6C40B, + 0x7AA64737, + 0xB60C47A9, + 0x3883404A, + 0xF42940D4, + 0xFEEC49CD, + 0x32464953, + 0xBCC94EB0, + 0x70634E2E, + 0xA9435C82, + 0x65E95C1C, + 0xEB665BFF, + 0x27CC5B61, + 0x2D095278, + 0xE1A352E6, + 0x6F2C5505, + 0xA386559B, + 0x061D761C, + 0xCAB77682, + 0x44387161, + 0x889271FF, + 0x825778E6, + 0x4EFD7878, + 0xC0727F9B, + 0x0CD87F05, + 0xD5F86DA9, + 0x19526D37, + 0x97DD6AD4, + 0x5B776A4A, + 0x51B26353, + 0x9D1863CD, + 0x1397642E, + 0xDF3D64B0, + 0x83D02561, + 0x4F7A25FF, + 0xC1F5221C, + 0x0D5F2282, + 0x079A2B9B, + 0xCB302B05, + 0x45BF2CE6, + 0x89152C78, + 0x50353ED4, + 0x9C9F3E4A, + 0x121039A9, + 0xDEBA3937, + 0xD47F302E, + 0x18D530B0, + 0x965A3753, + 0x5AF037CD, + 0xFF6B144A, + 0x33C114D4, + 0xBD4E1337, + 0x71E413A9, + 0x7B211AB0, + 0xB78B1A2E, + 0x39041DCD, + 0xF5AE1D53, + 0x2C8E0FFF, + 0xE0240F61, + 0x6EAB0882, + 0xA201081C, + 0xA8C40105, + 0x646E019B, + 0xEAE10678, + 0x264B06E6 }; +} 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(); + } + }); + } + +} 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 + } + } + +} 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); + +} 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 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; + } + +} 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 EMPTY = new ArrayList(); + 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 sends = new ArrayList(); + 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 threads; + private final List sockets; + + public EchoServer() throws Exception { + this.port = TestUtils.choosePort(); + this.serverSocket = new ServerSocket(port); + this.threads = Collections.synchronizedList(new ArrayList()); + this.sockets = Collections.synchronizedList(new ArrayList()); + } + + 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(); + } + } + +} 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 threads = new ArrayList(); + 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(); + } + } + } + +} 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; + } + +} 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()); + } +} 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..3792586 --- /dev/null +++ b/clients/src/test/java/kafka/clients/producer/RecordAccumulatorTest.java @@ -0,0 +1,125 @@ +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.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); + + @Test + public void testFull() throws Exception { + long now = time.milliseconds(); + RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, 10L, false, 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 batches = accum.drain(asList(tp), Integer.MAX_VALUE); + assertEquals(1, batches.size()); + RecordBatch batch = batches.get(0); + Iterator 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 testLinger() throws Exception { + long lingerMs = 10L; + RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, lingerMs, false, 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 batches = accum.drain(asList(tp), Integer.MAX_VALUE); + assertEquals(1, batches.size()); + RecordBatch batch = batches.get(0); + Iterator 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, time); + int appends = 1024 / msgSize + 1; + List 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 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, time); + List threads = new ArrayList(); + 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 tps = accum.ready(now); + List 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(); + } + +} 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..67a44ab --- /dev/null +++ b/clients/src/test/java/kafka/clients/producer/RecordSendTest.java @@ -0,0 +1,74 @@ +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.errors.CorruptMessageException; +import kafka.common.errors.TimeoutException; + +import org.junit.Test; + +public class RecordSendTest { + + 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(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(baseOffset, error); + } catch (InterruptedException e) { + e.printStackTrace(); + } + } + }.start(); + return request; + } + +} 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..594e728 --- /dev/null +++ b/clients/src/test/java/kafka/clients/producer/SenderTest.java @@ -0,0 +1,90 @@ +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.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 RecordAccumulator accumulator = new RecordAccumulator(batchSize, 1024 * 1024, 0L, false, 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); + } + +} 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 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()); + } + + @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 m = new HashMap(); + 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 + } + } + } +} 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()); + } +} 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; + } + + } + +} 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..9766658 --- /dev/null +++ b/clients/src/test/java/kafka/common/metrics/stats/HistogramTest.java @@ -0,0 +1,57 @@ +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); + } + + // @Test + 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); + } + } + +} 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); + } + +} 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 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 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()); + } + } + } + +} 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 data() { + List values = new ArrayList(); + 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; + } + +} 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 l = new ArrayList(); + for (int i = 0; i < max; i++) + l.add(i); + ListIterator iter = new ListIterator(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 iter = new ListIterator(Arrays.asList()); + iter.next(); + } + + class ListIterator extends AbstractIterator { + private List list; + private int position = 0; + + public ListIterator(List l) { + this.list = l; + } + + public T makeNext() { + if (position < list.size()) + return list.get(position++); + else + return allDone(); + } + } +} 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); + } + +} 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)); + } +} 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..08804d9 --- /dev/null +++ b/clients/src/test/java/kafka/test/Microbenchmarks.java @@ -0,0 +1,102 @@ +package kafka.test; + +import java.util.Arrays; +import java.util.Random; +import java.util.concurrent.atomic.AtomicBoolean; + +import kafka.common.utils.SystemTime; + +public class Microbenchmarks { + + public static void main(String[] args) { + + 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(); + 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); + } + }.start(); + + new Thread() { + public void run() { + int counter = 0; + while (!done.get()) { + time.sleep(1); + synchronized (lock) { + counter += 1; + } + } + System.out.println("Counter: " + counter); + } + }.start(); + } + + 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; + } + +} 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 completedSends = new ArrayList(); + private final List completedReceives = new ArrayList(); + private final List disconnected = new ArrayList(); + private final List connected = new ArrayList(); + + 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 sends) throws IOException { + this.completedSends.addAll(sends); + time.sleep(timeout); + } + + @Override + public List completedSends() { + return completedSends; + } + + public void completeSend(NetworkSend send) { + this.completedSends.add(send); + } + + @Override + public List completedReceives() { + return completedReceives; + } + + public void completeReceive(NetworkReceive receive) { + this.completedReceives.add(receive); + } + + @Override + public List disconnected() { + return disconnected; + } + + @Override + public List connected() { + return connected; + } + +} 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(); + } + +} 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")) }