Page MenuHomeSoftware Heritage

No OneTemporary

This file is larger than 256 KB, so syntax highlighting was skipped.
diff --git a/clients/src/main/java/kafka/clients/producer/BufferExhaustedException.java b/clients/src/main/java/org/apache/kafka/clients/producer/BufferExhaustedException.java
similarity index 82%
rename from clients/src/main/java/kafka/clients/producer/BufferExhaustedException.java
rename to clients/src/main/java/org/apache/kafka/clients/producer/BufferExhaustedException.java
index d1aa4b62..f3ed4eaf 100644
--- a/clients/src/main/java/kafka/clients/producer/BufferExhaustedException.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/BufferExhaustedException.java
@@ -1,17 +1,17 @@
-package kafka.clients.producer;
+package org.apache.kafka.clients.producer;
-import kafka.common.KafkaException;
+import org.apache.kafka.common.KafkaException;
/**
* This exception is thrown if the producer is in non-blocking mode and the rate of data production exceeds the rate at
* which data can be sent for long enough for the alloted buffer to be exhausted.
*/
public class BufferExhaustedException extends KafkaException {
private static final long serialVersionUID = 1L;
public BufferExhaustedException(String message) {
super(message);
}
}
diff --git a/clients/src/main/java/kafka/clients/producer/Callback.java b/clients/src/main/java/org/apache/kafka/clients/producer/Callback.java
similarity index 95%
rename from clients/src/main/java/kafka/clients/producer/Callback.java
rename to clients/src/main/java/org/apache/kafka/clients/producer/Callback.java
index d287d78d..ad7d740b 100644
--- a/clients/src/main/java/kafka/clients/producer/Callback.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/Callback.java
@@ -1,18 +1,18 @@
-package kafka.clients.producer;
+package org.apache.kafka.clients.producer;
/**
* A callback interface that the user can implement to allow code to execute when the request is complete. This callback
* will generally execute in the background I/O thread so it should be fast.
*/
public interface Callback {
/**
* A callback method the user can implement to provide asynchronous handling of request completion. This method will
* be called when the record sent to the server has been acknowledged. Exactly one of the arguments will be
* non-null.
* @param metadata The metadata for the record that was sent (i.e. the partition and offset). Null if an error
* occurred.
* @param exception The exception thrown during processing of this record. Null if no error occurred.
*/
public void onCompletion(RecordMetadata metadata, Exception exception);
}
diff --git a/clients/src/main/java/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
similarity index 91%
rename from clients/src/main/java/kafka/clients/producer/KafkaProducer.java
rename to clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
index 1dd63fc2..dcc40d3f 100644
--- a/clients/src/main/java/kafka/clients/producer/KafkaProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java
@@ -1,286 +1,287 @@
-package kafka.clients.producer;
+package org.apache.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 java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
-import kafka.clients.producer.internals.FutureRecordMetadata;
-import kafka.clients.producer.internals.Metadata;
-import kafka.clients.producer.internals.Partitioner;
-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.PartitionInfo;
-import kafka.common.TopicPartition;
-import kafka.common.config.ConfigException;
-import kafka.common.errors.RecordTooLargeException;
-import kafka.common.metrics.JmxReporter;
-import kafka.common.metrics.MetricConfig;
-import kafka.common.metrics.Metrics;
-import kafka.common.metrics.MetricsReporter;
-import kafka.common.network.Selector;
-import kafka.common.record.CompressionType;
-import kafka.common.record.Record;
-import kafka.common.record.Records;
-import kafka.common.utils.KafkaThread;
-import kafka.common.utils.SystemTime;
+import org.apache.kafka.clients.producer.internals.FutureRecordMetadata;
+import org.apache.kafka.clients.producer.internals.Metadata;
+import org.apache.kafka.clients.producer.internals.Partitioner;
+import org.apache.kafka.clients.producer.internals.RecordAccumulator;
+import org.apache.kafka.clients.producer.internals.Sender;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.errors.RecordTooLargeException;
+import org.apache.kafka.common.metrics.JmxReporter;
+import org.apache.kafka.common.metrics.MetricConfig;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.metrics.MetricsReporter;
+import org.apache.kafka.common.network.Selector;
+import org.apache.kafka.common.record.CompressionType;
+import org.apache.kafka.common.record.Record;
+import org.apache.kafka.common.record.Records;
+import org.apache.kafka.common.utils.KafkaThread;
+import org.apache.kafka.common.utils.SystemTime;
+
/**
* A Kafka client that publishes records to the Kafka cluster.
* <P>
* The producer is <i>thread safe</i> and should generally be shared among all threads for best performance.
* <p>
* The producer manages a single background thread that does I/O as well as a TCP connection to each of the brokers it
* needs to communicate with. Failure to close the producer after use will leak these resources.
*/
public class KafkaProducer implements Producer {
private final Partitioner partitioner;
private final int maxRequestSize;
private final long metadataFetchTimeoutMs;
private final long totalMemorySize;
private final Metadata metadata;
private final RecordAccumulator accumulator;
private final Sender sender;
private final Metrics metrics;
private final Thread ioThread;
/**
* A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings
* are documented <a href="http://kafka.apache.org/documentation.html#producerconfigs">here</a>. Values can be
* either strings or Objects of the appropriate type (for example a numeric configuration would accept either the
* string "42" or the integer 42).
*/
public KafkaProducer(Map<String, Object> configs) {
this(new ProducerConfig(configs));
}
/**
* A producer is instantiated by providing a set of key-value pairs as configuration. Valid configuration strings
* are documented <a href="http://kafka.apache.org/documentation.html#producerconfigs">here</a>.
*/
public KafkaProducer(Properties properties) {
this(new ProducerConfig(properties));
}
private KafkaProducer(ProducerConfig config) {
this.metrics = new Metrics(new MetricConfig(),
Collections.singletonList((MetricsReporter) new JmxReporter("kafka.producer.")),
new SystemTime());
this.partitioner = new Partitioner();
this.metadataFetchTimeoutMs = config.getLong(ProducerConfig.METADATA_FETCH_TIMEOUT_CONFIG);
this.metadata = new Metadata();
this.maxRequestSize = config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG);
this.totalMemorySize = config.getLong(ProducerConfig.TOTAL_BUFFER_MEMORY_CONFIG);
this.accumulator = new RecordAccumulator(config.getInt(ProducerConfig.MAX_PARTITION_SIZE_CONFIG),
this.totalMemorySize,
config.getLong(ProducerConfig.LINGER_MS_CONFIG),
config.getBoolean(ProducerConfig.BLOCK_ON_BUFFER_FULL),
metrics,
new SystemTime());
List<InetSocketAddress> addresses = parseAndValidateAddresses(config.getList(ProducerConfig.BROKER_LIST_CONFIG));
this.metadata.update(Cluster.bootstrap(addresses), System.currentTimeMillis());
this.sender = new Sender(new Selector(),
this.metadata,
this.accumulator,
config.getString(ProducerConfig.CLIENT_ID_CONFIG),
config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG),
config.getLong(ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG),
(short) config.getInt(ProducerConfig.REQUIRED_ACKS_CONFIG),
config.getInt(ProducerConfig.REQUEST_TIMEOUT_CONFIG),
new SystemTime());
this.ioThread = new KafkaThread("kafka-network-thread", this.sender, true);
this.ioThread.start();
}
private static List<InetSocketAddress> parseAndValidateAddresses(List<String> urls) {
List<InetSocketAddress> addresses = new ArrayList<InetSocketAddress>();
for (String url : urls) {
if (url != null && url.length() > 0) {
String[] pieces = url.split(":");
if (pieces.length != 2)
throw new ConfigException("Invalid url in metadata.broker.list: " + url);
try {
InetSocketAddress address = new InetSocketAddress(pieces[0], Integer.parseInt(pieces[1]));
if (address.isUnresolved())
throw new ConfigException("DNS resolution failed for metadata bootstrap url: " + url);
addresses.add(address);
} catch (NumberFormatException e) {
throw new ConfigException("Invalid port in metadata.broker.list: " + url);
}
}
}
if (addresses.size() < 1)
throw new ConfigException("No bootstrap urls given in metadata.broker.list.");
return addresses;
}
/**
* Asynchronously send a record to a topic. Equivalent to {@link #send(ProducerRecord, Callback) send(record, null)}
*/
@Override
public Future<RecordMetadata> send(ProducerRecord record) {
return send(record, null);
}
/**
* Asynchronously send a record to a topic and invoke the provided callback when the send has been acknowledged.
* <p>
* The send is asynchronous and this method will return immediately once the record has been stored in the buffer of
* records waiting to be sent. This allows sending many records in parallel without blocking to wait for the
* response after each one.
* <p>
* The result of the send is a {@link RecordMetadata} specifying the partition the record was sent to and the offset
* it was assigned.
* <p>
* Since the send call is asynchronous it returns a {@link java.util.concurrent.Future Future} for the
* {@link RecordMetadata} that will be assigned to this record. Invoking {@link java.util.concurrent.Future#get()
* get()} on this future will result in the metadata for the record or throw any exception that occurred while
* sending the record.
* <p>
* If you want to simulate a simple blocking call you can do the following:
*
* <pre>
* producer.send(new ProducerRecord("the-topic", "key, "value")).get();
* </pre>
* <p>
* Those desiring fully non-blocking usage can make use of the {@link Callback} parameter to provide a callback that
* will be invoked when the request is complete.
*
* <pre>
* ProducerRecord record = new ProducerRecord("the-topic", "key, "value");
* producer.send(myRecord,
* new Callback() {
* public void onCompletion(RecordMetadata metadata, Exception e) {
* if(e != null)
* e.printStackTrace();
* System.out.println("The offset of the record we just sent is: " + metadata.offset());
* }
* });
* </pre>
*
* Callbacks for records being sent to the same partition are guaranteed to execute in order. That is, in the
* following example <code>callback1</code> is guaranteed to execute before <code>callback2</code>:
*
* <pre>
* producer.send(new ProducerRecord(topic, partition, key, value), callback1);
* producer.send(new ProducerRecord(topic, partition, key2, value2), callback2);
* </pre>
* <p>
* Note that callbacks will generally execute in the I/O thread of the producer and so should be reasonably fast or
* they will delay the sending of messages from other threads. If you want to execute blocking or computationally
* expensive callbacks it is recommended to use your own {@link java.util.concurrent.Executor} in the callback body
* to parallelize processing.
* <p>
* The producer manages a buffer of records waiting to be sent. This buffer has a hard limit on it's size, which is
* controlled by the configuration <code>total.memory.bytes</code>. If <code>send()</code> is called faster than the
* I/O thread can transfer data to the brokers the buffer will eventually run out of 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
* in cases where non-blocking usage is desired the setting <code>block.on.buffer.full=false</code> will cause the
* producer to instead throw an exception when buffer memory is exhausted.
*
* @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)
*/
@Override
public Future<RecordMetadata> send(ProducerRecord record, Callback callback) {
try {
Cluster cluster = metadata.fetch(record.topic(), this.metadataFetchTimeoutMs);
int partition = partitioner.partition(record, cluster);
ensureValidSize(record.key(), record.value());
TopicPartition tp = new TopicPartition(record.topic(), partition);
FutureRecordMetadata future = accumulator.append(tp, record.key(), record.value(), CompressionType.NONE, callback);
this.sender.wakeup();
return future;
} catch (Exception e) {
if (callback != null)
callback.onCompletion(null, e);
return new FutureFailure(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 RecordTooLargeException("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 RecordTooLargeException("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.");
}
public List<PartitionInfo> partitionsFor(String topic) {
return this.metadata.fetch(topic, this.metadataFetchTimeoutMs).partitionsFor(topic);
}
@Override
public Map<String, ? extends Metric> metrics() {
return Collections.unmodifiableMap(this.metrics.metrics());
}
/**
* Close this producer. This method blocks until all in-flight requests complete.
*/
@Override
public void close() {
this.sender.initiateClose();
try {
this.ioThread.join();
} catch (InterruptedException e) {
throw new KafkaException(e);
}
this.metrics.close();
}
private static class FutureFailure implements Future<RecordMetadata> {
private final ExecutionException exception;
public FutureFailure(Exception exception) {
this.exception = new ExecutionException(exception);
}
@Override
public boolean cancel(boolean interrupt) {
return false;
}
@Override
public RecordMetadata get() throws ExecutionException {
throw this.exception;
}
@Override
public RecordMetadata get(long timeout, TimeUnit unit) throws ExecutionException {
throw this.exception;
}
@Override
public boolean isCancelled() {
return false;
}
@Override
public boolean isDone() {
return true;
}
}
}
diff --git a/clients/src/main/java/kafka/clients/producer/MockProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
similarity index 93%
rename from clients/src/main/java/kafka/clients/producer/MockProducer.java
rename to clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
index ab83d5f7..76dbd53a 100644
--- a/clients/src/main/java/kafka/clients/producer/MockProducer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/MockProducer.java
@@ -1,199 +1,200 @@
-package kafka.clients.producer;
+package org.apache.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 java.util.concurrent.Future;
-import kafka.clients.producer.internals.FutureRecordMetadata;
-import kafka.clients.producer.internals.Partitioner;
-import kafka.clients.producer.internals.ProduceRequestResult;
-import kafka.common.Cluster;
-import kafka.common.Metric;
-import kafka.common.PartitionInfo;
-import kafka.common.TopicPartition;
+import org.apache.kafka.clients.producer.internals.FutureRecordMetadata;
+import org.apache.kafka.clients.producer.internals.Partitioner;
+import org.apache.kafka.clients.producer.internals.ProduceRequestResult;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.TopicPartition;
+
/**
* A mock of the producer interface you can use for testing code that uses Kafka.
* <p>
* By default this mock will synchronously complete each send call successfully. However it can be configured to allow
* the user to control the completion of the call and supply an optional error for the producer to throw.
*/
public class MockProducer implements Producer {
private final Cluster cluster;
private final Partitioner partitioner = new Partitioner();
private final List<ProducerRecord> sent;
private final Deque<Completion> completions;
private boolean autoComplete;
private Map<TopicPartition, Long> offsets;
/**
* Create a mock producer
*
* @param cluster The cluster holding metadata for this producer
* @param autoComplete If true automatically complete all requests successfully and execute the callback. Otherwise
* the user must call {@link #completeNext()} or {@link #errorNext(RuntimeException)} after
* {@link #send(ProducerRecord) send()} to complete the call and unblock the @{link
* java.util.concurrent.Future Future&lt;RecordMetadata&gt;} that is returned.
*/
public MockProducer(Cluster cluster, boolean autoComplete) {
this.cluster = cluster;
this.autoComplete = autoComplete;
this.offsets = new HashMap<TopicPartition, Long>();
this.sent = new ArrayList<ProducerRecord>();
this.completions = new ArrayDeque<Completion>();
}
/**
* Create a new mock producer with invented metadata the given autoComplete setting.
*
* Equivalent to {@link #MockProducer(Cluster, boolean) new MockProducer(null, autoComplete)}
*/
public MockProducer(boolean autoComplete) {
this(Cluster.empty(), autoComplete);
}
/**
* Create a new auto completing mock producer
*
* Equivalent to {@link #MockProducer(boolean) new MockProducer(true)}
*/
public MockProducer() {
this(true);
}
/**
* Adds the record to the list of sent records. The {@link RecordMetadata} returned will be immediately satisfied.
*
* @see #history()
*/
@Override
public synchronized Future<RecordMetadata> send(ProducerRecord record) {
return send(record, null);
}
/**
* Adds the record to the list of sent records.
*
* @see #history()
*/
@Override
public synchronized Future<RecordMetadata> send(ProducerRecord record, Callback callback) {
int partition = 0;
if (this.cluster.partitionsFor(record.topic()) != null)
partition = partitioner.partition(record, this.cluster);
ProduceRequestResult result = new ProduceRequestResult();
FutureRecordMetadata future = new FutureRecordMetadata(result, 0);
TopicPartition topicPartition = new TopicPartition(record.topic(), partition);
long offset = nextOffset(topicPartition);
Completion completion = new Completion(topicPartition, offset, new RecordMetadata(topicPartition, offset), result, callback);
this.sent.add(record);
if (autoComplete)
completion.complete(null);
else
this.completions.addLast(completion);
return future;
}
/**
* 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 List<PartitionInfo> partitionsFor(String topic) {
return this.cluster.partitionsFor(topic);
}
public Map<String, Metric> metrics() {
return Collections.emptyMap();
}
@Override
public void close() {
}
/**
* Get the list of sent records since the last call to {@link #clear()}
*/
public synchronized List<ProducerRecord> history() {
return new ArrayList<ProducerRecord>(this.sent);
}
/**
* Clear the stored history of sent records
*/
public synchronized void clear() {
this.sent.clear();
this.completions.clear();
}
/**
* Complete the earliest uncompleted call successfully.
*
* @return true if there was an uncompleted call to complete
*/
public synchronized boolean completeNext() {
return errorNext(null);
}
/**
* Complete the earliest uncompleted call with the given error.
*
* @return true if there was an uncompleted call to complete
*/
public synchronized boolean errorNext(RuntimeException e) {
Completion completion = this.completions.pollFirst();
if (completion != null) {
completion.complete(e);
return true;
} else {
return false;
}
}
private static class Completion {
private final long offset;
private final RecordMetadata metadata;
private final ProduceRequestResult result;
private final Callback callback;
private final TopicPartition topicPartition;
public Completion(TopicPartition topicPartition,
long offset,
RecordMetadata metadata,
ProduceRequestResult result,
Callback callback) {
this.metadata = metadata;
this.offset = offset;
this.result = result;
this.callback = callback;
this.topicPartition = topicPartition;
}
public void complete(RuntimeException e) {
result.done(topicPartition, e == null ? offset : -1L, e);
if (callback != null) {
if (e == null)
callback.onCompletion(metadata, null);
else
callback.onCompletion(null, e);
}
}
}
}
diff --git a/clients/src/main/java/kafka/clients/producer/Producer.java b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java
similarity index 90%
rename from clients/src/main/java/kafka/clients/producer/Producer.java
rename to clients/src/main/java/org/apache/kafka/clients/producer/Producer.java
index f149f3a9..7daea9f9 100644
--- a/clients/src/main/java/kafka/clients/producer/Producer.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/Producer.java
@@ -1,48 +1,49 @@
-package kafka.clients.producer;
+package org.apache.kafka.clients.producer;
import java.io.Closeable;
import java.util.List;
import java.util.Map;
import java.util.concurrent.Future;
-import kafka.common.Metric;
-import kafka.common.PartitionInfo;
+import org.apache.kafka.common.Metric;
+import org.apache.kafka.common.PartitionInfo;
+
/**
* The interface for the {@link KafkaProducer}
*
* @see KafkaProducer
* @see MockProducer
*/
public interface Producer extends Closeable {
/**
* 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 Future<RecordMetadata> send(ProducerRecord record);
/**
* Send a record and invoke the given callback when the record has been acknowledged by the server
*/
public Future<RecordMetadata> send(ProducerRecord record, Callback callback);
/**
* Get a list of partitions for the given topic for custom partition assignment. The partition metadata will change
* over time so this list should not be cached.
*/
public List<PartitionInfo> partitionsFor(String topic);
/**
* Return a map of metrics maintained by the producer
*/
public Map<String, ? extends Metric> metrics();
/**
* Close this producer
*/
public void close();
}
diff --git a/clients/src/main/java/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
similarity index 94%
rename from clients/src/main/java/kafka/clients/producer/ProducerConfig.java
rename to clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
index a94afc7d..77955a91 100644
--- a/clients/src/main/java/kafka/clients/producer/ProducerConfig.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java
@@ -1,130 +1,131 @@
-package kafka.clients.producer;
+package org.apache.kafka.clients.producer;
-import static kafka.common.config.ConfigDef.Range.atLeast;
-import static kafka.common.config.ConfigDef.Range.between;
+import static org.apache.kafka.common.config.ConfigDef.Range.atLeast;
+import static org.apache.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;
+import org.apache.kafka.common.config.AbstractConfig;
+import org.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigDef.Type;
+
/**
* The producer configuration keys
*/
public class ProducerConfig extends AbstractConfig {
private static final ConfigDef config;
/**
* A list of URLs to use for establishing the initial connection to the cluster. This list should be in the form
* <code>host1:port1,host2:port2,...</code>. These urls are just used for the initial connection to discover the
* full cluster membership (which may change dynamically) so this list need not contain the full set of servers (you
* may want more than one, though, in case a server is down).
*/
public static final String BROKER_LIST_CONFIG = "metadata.broker.list";
/**
* The amount of time to block waiting to fetch metadata about a topic the first time a record is sent to that
* topic.
*/
public static final String METADATA_FETCH_TIMEOUT_CONFIG = "metadata.fetch.timeout.ms";
/**
* The buffer size allocated for a partition. When records are received which are smaller than this size the
* producer will attempt to optimistically group them together until this size is reached.
*/
public static final String MAX_PARTITION_SIZE_CONFIG = "max.partition.bytes";
/**
* The total memory used by the producer to buffer records waiting to be sent to the server. If records are sent
* faster than they can be delivered to the server the producer will either block or throw an exception based on the
* preference specified by {@link #BLOCK_ON_BUFFER_FULL}.
*/
public static final String TOTAL_BUFFER_MEMORY_CONFIG = "total.memory.bytes";
/**
* The number of acknowledgments the producer requires from the server before considering a request complete.
*/
public static final String REQUIRED_ACKS_CONFIG = "request.required.acks";
/**
* The maximum amount of time the server will wait for acknowledgments from followers to meet the acknowledgment
* requirements the producer has specified. If the requested number of acknowledgments are not met an error will be
* returned.
*/
public static final String REQUEST_TIMEOUT_CONFIG = "request.timeout.ms";
/**
* The producer groups together any records that arrive in between request sends. Normally this occurs only under
* load when records arrive faster than they can be sent out. However the client can reduce the number of requests
* and increase throughput by adding a small amount of artificial delay to force more records to batch together.
* This setting gives an upper bound on this delay. If we get {@link #MAX_PARTITION_SIZE_CONFIG} worth of records
* for a partition it will be sent immediately regardless of this setting, however if we have fewer than this many
* bytes accumulated for this partition we will "linger" for the specified time waiting for more records to show up.
* This setting defaults to 0.
*/
public static final String LINGER_MS_CONFIG = "linger.ms";
/**
* Force a refresh of the cluster metadata after this period of time. This ensures that changes to the number of
* partitions or other settings will by taken up by producers without restart.
*/
public static final String METADATA_REFRESH_MS_CONFIG = "topic.metadata.refresh.interval.ms";
/**
* The id string to pass to the server when making requests. The purpose of this is to be able to track the source
* of requests beyond just ip/port by allowing a logical application name to be included.
*/
public static final String CLIENT_ID_CONFIG = "client.id";
/**
* The size of the TCP send buffer to use when sending data
*/
public static final String SEND_BUFFER_CONFIG = "send.buffer.bytes";
/**
* The maximum size of a request. This is also effectively a cap on the maximum record size. Note that the server
* has its own cap on record size which may be different from this.
*/
public static final String MAX_REQUEST_SIZE_CONFIG = "max.request.size";
/**
* The amount of time to wait before attempting to reconnect to a given host. This avoids repeated connecting to a
* host in a tight loop.
*/
public static final String RECONNECT_BACKOFF_MS_CONFIG = "reconnect.backoff.ms";
/**
* When our memory buffer is exhausted we must either stop accepting new records (block) or throw errors. By default
* this setting is true and we block, however users who want to guarantee we never block can turn this into an
* error.
*/
public static final String BLOCK_ON_BUFFER_FULL = "block.on.buffer.full";
public static final String ENABLE_JMX = "enable.jmx";
static {
/* TODO: add docs */
config = new ConfigDef().define(BROKER_LIST_CONFIG, Type.LIST, "blah blah")
.define(METADATA_FETCH_TIMEOUT_CONFIG, Type.LONG, 60 * 1000, atLeast(0), "blah blah")
.define(MAX_PARTITION_SIZE_CONFIG, Type.INT, 16384, atLeast(0), "blah blah")
.define(TOTAL_BUFFER_MEMORY_CONFIG, Type.LONG, 32 * 1024 * 1024L, atLeast(0L), "blah blah")
/* TODO: should be a string to handle acks=in-sync */
.define(REQUIRED_ACKS_CONFIG, Type.INT, 1, between(-1, Short.MAX_VALUE), "blah blah")
.define(REQUEST_TIMEOUT_CONFIG, Type.INT, 30 * 1000, atLeast(0), "blah blah")
.define(LINGER_MS_CONFIG, Type.LONG, 0, atLeast(0L), "blah blah")
.define(METADATA_REFRESH_MS_CONFIG, Type.LONG, 10 * 60 * 1000, atLeast(-1L), "blah blah")
.define(CLIENT_ID_CONFIG, Type.STRING, "", "blah blah")
.define(SEND_BUFFER_CONFIG, Type.INT, 128 * 1024, atLeast(0), "blah blah")
.define(MAX_REQUEST_SIZE_CONFIG, Type.INT, 1 * 1024 * 1024, atLeast(0), "blah blah")
.define(RECONNECT_BACKOFF_MS_CONFIG, Type.LONG, 10L, atLeast(0L), "blah blah")
.define(BLOCK_ON_BUFFER_FULL, Type.BOOLEAN, true, "blah blah")
.define(ENABLE_JMX, Type.BOOLEAN, true, "");
}
ProducerConfig(Map<? extends Object, ? extends Object> props) {
super(config, props);
}
}
diff --git a/clients/src/main/java/kafka/clients/producer/ProducerRecord.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java
similarity index 98%
rename from clients/src/main/java/kafka/clients/producer/ProducerRecord.java
rename to clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java
index b9c20bc6..10d1b96c 100644
--- a/clients/src/main/java/kafka/clients/producer/ProducerRecord.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerRecord.java
@@ -1,84 +1,84 @@
-package kafka.clients.producer;
+package org.apache.kafka.clients.producer;
/**
* A key/value pair to be sent to Kafka. This consists of a topic name to which the record is being sent, an optional
* partition number, and an optional key and value.
* <p>
* If a valid partition number is specified that partition will be used when sending the record. If no partition is
* specified but a key is present a partition will be chosen using a hash of the key. If neither key nor partition is
* present a partition will be assigned in a round-robin fashion.
*/
public final class ProducerRecord {
private final String topic;
private final Integer partition;
private final byte[] key;
private final byte[] value;
/**
* Creates a record to be sent to a specified topic and partition
*
* @param topic The topic the record will be appended to
* @param partition The partition to which the record should be sent
* @param key The key that will be included in the record
* @param value The record contents
*/
public ProducerRecord(String topic, Integer partition, byte[] key, byte[] value) {
if (topic == null)
throw new IllegalArgumentException("Topic cannot be null");
this.topic = topic;
this.partition = partition;
this.key = key;
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, byte[] key, byte[] value) {
this(topic, null, 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, byte[] 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 byte[] key() {
return key;
}
/**
* @return The value
*/
public byte[] value() {
return value;
}
/**
* The partition to which the record will be sent (or null if no partition was specified)
*/
public Integer partition() {
return partition;
}
}
diff --git a/clients/src/main/java/kafka/clients/producer/RecordMetadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java
similarity index 89%
rename from clients/src/main/java/kafka/clients/producer/RecordMetadata.java
rename to clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java
index 1486586e..5118b3a5 100644
--- a/clients/src/main/java/kafka/clients/producer/RecordMetadata.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java
@@ -1,39 +1,39 @@
-package kafka.clients.producer;
+package org.apache.kafka.clients.producer;
-import kafka.common.TopicPartition;
+import org.apache.kafka.common.TopicPartition;
/**
* The metadata for a record that has been acknowledged by the server
*/
public final class RecordMetadata {
private final long offset;
private final TopicPartition topicPartition;
public RecordMetadata(TopicPartition topicPartition, long offset) {
super();
this.offset = offset;
this.topicPartition = topicPartition;
}
/**
* The offset of the record in the topic/partition.
*/
public long offset() {
return this.offset;
}
/**
* The topic the record was appended to
*/
public String topic() {
return this.topicPartition.topic();
}
/**
* The partition the record was sent to
*/
public int partition() {
return this.topicPartition.partition();
}
}
diff --git a/clients/src/main/java/kafka/clients/producer/internals/BufferPool.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java
similarity index 98%
rename from clients/src/main/java/kafka/clients/producer/internals/BufferPool.java
rename to clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java
index c222ca0e..2548dad7 100644
--- a/clients/src/main/java/kafka/clients/producer/internals/BufferPool.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/BufferPool.java
@@ -1,223 +1,224 @@
-package kafka.clients.producer.internals;
+package org.apache.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;
+import org.apache.kafka.clients.producer.BufferExhaustedException;
+
/**
* A pool of ByteBuffers kept under a given memory limit. This class is fairly specific to the needs of the producer. In
* particular it has the following properties:
* <ol>
* <li>There is a special "poolable size" and buffers of this size are kept in a free list and recycled
* <li>It is fair. That is all memory is given to the longest waiting thread until it has sufficient memory. This
* prevents starvation or deadlock when a thread asks for a large chunk of memory and needs to block until multiple
* buffers are deallocated.
* </ol>
*/
public final class BufferPool {
private final long totalMemory;
private final int poolableSize;
private final boolean blockOnExhaustion;
private final ReentrantLock lock;
private final Deque<ByteBuffer> free;
private final Deque<Condition> waiters;
private long availableMemory;
/**
* Create a new buffer pool
*
* @param memory The maximum amount of memory that this buffer pool can allocate
* @param poolableSize The buffer size to cache in the free list rather than deallocating
* @param blockOnExhaustion This controls the behavior when the buffer pool is out of memory. If true the
* {@link #allocate(int)} call will block and wait for memory to be returned to the pool. If false
* {@link #allocate(int)} will throw an exception if the buffer is out of memory.
*/
public BufferPool(long memory, int poolableSize, boolean blockOnExhaustion) {
this.poolableSize = poolableSize;
this.blockOnExhaustion = blockOnExhaustion;
this.lock = new ReentrantLock();
this.free = new ArrayDeque<ByteBuffer>();
this.waiters = new ArrayDeque<Condition>();
this.totalMemory = memory;
this.availableMemory = memory;
}
/**
* Allocate a buffer of the given size
*
* @param size The buffer size to allocate in bytes
* @return The buffer
* @throws InterruptedException If the thread is interrupted while blocked
* @throws IllegalArgument if size is larger than the total memory controlled by the pool (and hence we would block
* forever)
* @throws BufferExhaustedException if the pool is in non-blocking mode and size exceeds the free memory in the pool
*/
public ByteBuffer allocate(int size) throws InterruptedException {
if (size > this.totalMemory)
throw new IllegalArgumentException("Attempt to allocate " + size
+ " bytes, but there is a hard limit of "
+ this.totalMemory
+ " on memory allocations.");
this.lock.lock();
try {
// check if we have a free buffer of the right size pooled
if (size == poolableSize && !this.free.isEmpty())
return this.free.pollFirst();
// now check if the request is immediately satisfiable with the
// memory on hand or if we need to block
int freeListSize = this.free.size() * this.poolableSize;
if (this.availableMemory + freeListSize >= size) {
// we have enough unallocated or pooled memory to immediately
// satisfy the request
freeUp(size);
this.availableMemory -= size;
lock.unlock();
return ByteBuffer.allocate(size);
} else if (!blockOnExhaustion) {
throw new BufferExhaustedException("You have exhausted the " + this.totalMemory
+ " bytes of memory you configured for the client and the client is configured to error"
+ " rather than block when memory is exhausted.");
} else {
// we are out of memory and will have to block
int accumulated = 0;
ByteBuffer buffer = null;
Condition moreMemory = this.lock.newCondition();
this.waiters.addLast(moreMemory);
// loop over and over until we have a buffer or have reserved
// enough memory to allocate one
while (accumulated < size) {
moreMemory.await();
// check if we can satisfy this request from the free list,
// otherwise allocate memory
if (accumulated == 0 && size == this.poolableSize && !this.free.isEmpty()) {
// just grab a buffer from the free list
buffer = this.free.pollFirst();
accumulated = size;
} else {
// we'll need to allocate memory, but we may only get
// part of what we need on this iteration
freeUp(size - accumulated);
int got = (int) Math.min(size - accumulated, this.availableMemory);
this.availableMemory -= got;
accumulated += got;
}
}
// remove the condition for this thread to let the next thread
// in line start getting memory
Condition removed = this.waiters.removeFirst();
if (removed != moreMemory)
throw new IllegalStateException("Wrong condition: this shouldn't happen.");
// signal any additional waiters if there is more memory left
// over for them
if (this.availableMemory > 0 || !this.free.isEmpty()) {
if (!this.waiters.isEmpty())
this.waiters.peekFirst().signal();
}
// unlock and return the buffer
lock.unlock();
if (buffer == null)
return ByteBuffer.allocate(size);
else
return buffer;
}
} finally {
if (lock.isHeldByCurrentThread())
lock.unlock();
}
}
/**
* Attempt to ensure we have at least the requested number of bytes of memory for allocation by deallocating pooled
* buffers (if needed)
*/
private void freeUp(int size) {
while (!this.free.isEmpty() && this.availableMemory < size)
this.availableMemory += this.free.pollLast().capacity();
}
/**
* Return buffers to the pool. If they are of the poolable size add them to the free list, otherwise just mark the
* memory as free.
*
* @param buffers The buffers to return
*/
public void deallocate(ByteBuffer... buffers) {
lock.lock();
try {
for (int i = 0; i < buffers.length; i++) {
int size = buffers[i].capacity();
if (size == this.poolableSize) {
buffers[i].clear();
this.free.add(buffers[i]);
} else {
this.availableMemory += size;
}
Condition moreMem = this.waiters.peekFirst();
if (moreMem != null)
moreMem.signal();
}
} finally {
lock.unlock();
}
}
/**
* the total free memory both unallocated and in the free list
*/
public long availableMemory() {
lock.lock();
try {
return this.availableMemory + this.free.size() * this.poolableSize;
} finally {
lock.unlock();
}
}
/**
* Get the unallocated memory (not in the free list or in use)
*/
public long unallocatedMemory() {
lock.lock();
try {
return this.availableMemory;
} finally {
lock.unlock();
}
}
/**
* The number of threads blocked waiting on memory
*/
public int queued() {
lock.lock();
try {
return this.waiters.size();
} finally {
lock.unlock();
}
}
/**
* The buffer size that will be retained in the free list after use
*/
public int poolableSize() {
return this.poolableSize;
}
/**
* The total memory managed by this pool
*/
public long totalMemory() {
return this.totalMemory;
}
}
\ No newline at end of file
diff --git a/clients/src/main/java/kafka/clients/producer/internals/FutureRecordMetadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java
similarity index 94%
rename from clients/src/main/java/kafka/clients/producer/internals/FutureRecordMetadata.java
rename to clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java
index 43b4c5dc..f4c79709 100644
--- a/clients/src/main/java/kafka/clients/producer/internals/FutureRecordMetadata.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/FutureRecordMetadata.java
@@ -1,63 +1,64 @@
-package kafka.clients.producer.internals;
+package org.apache.kafka.clients.producer.internals;
import java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
-import kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.clients.producer.RecordMetadata;
+
/**
* The future result of a record send
*/
public final class FutureRecordMetadata implements Future<RecordMetadata> {
private final ProduceRequestResult result;
private final long relativeOffset;
public FutureRecordMetadata(ProduceRequestResult result, long relativeOffset) {
this.result = result;
this.relativeOffset = relativeOffset;
}
@Override
public boolean cancel(boolean interrupt) {
return false;
}
@Override
public RecordMetadata get() throws InterruptedException, ExecutionException {
this.result.await();
return valueOrError();
}
@Override
public RecordMetadata get(long timeout, TimeUnit unit) throws InterruptedException, ExecutionException, TimeoutException {
boolean occurred = this.result.await(timeout, unit);
if (!occurred)
throw new TimeoutException("Timeout after waiting for " + TimeUnit.MILLISECONDS.convert(timeout, unit) + " ms.");
return valueOrError();
}
private RecordMetadata valueOrError() throws ExecutionException {
if (this.result.error() != null)
throw new ExecutionException(this.result.error());
else
return new RecordMetadata(result.topicPartition(), this.result.baseOffset() + this.relativeOffset);
}
public long relativeOffset() {
return this.relativeOffset;
}
@Override
public boolean isCancelled() {
return false;
}
@Override
public boolean isDone() {
return this.result.completed();
}
}
diff --git a/clients/src/main/java/kafka/clients/producer/internals/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
similarity index 95%
rename from clients/src/main/java/kafka/clients/producer/internals/Metadata.java
rename to clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
index f5f8b9b9..87c3cbaa 100644
--- a/clients/src/main/java/kafka/clients/producer/internals/Metadata.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Metadata.java
@@ -1,120 +1,121 @@
-package kafka.clients.producer.internals;
+package org.apache.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;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.errors.TimeoutException;
+
/**
* A class encapsulating some of the logic around metadata.
* <p>
* This class is shared by the client thread (for partitioning) and the background sender thread.
*
* Metadata is maintained for only a subset of topics, which can be added to over time. When we request metdata for a
* topic we don't have any metadata for it will trigger a metadata update.
*/
public final class Metadata {
private final long refreshBackoffMs;
private final long metadataExpireMs;
private long lastRefresh;
private Cluster cluster;
private boolean forceUpdate;
private final Set<String> topics;
/**
* Create a metadata instance with reasonable defaults
*/
public Metadata() {
this(100L, 60 * 60 * 1000L);
}
/**
* Create a new Metadata instance
* @param refreshBackoffMs The minimum amount of time that must expire between metadata refreshes to avoid busy
* polling
* @param metadataExpireMs The maximum amount of time that metadata can be retained without refresh
*/
public Metadata(long refreshBackoffMs, long metadataExpireMs) {
this.refreshBackoffMs = refreshBackoffMs;
this.metadataExpireMs = metadataExpireMs;
this.lastRefresh = 0L;
this.cluster = Cluster.empty();
this.forceUpdate = false;
this.topics = new HashSet<String>();
}
/**
* Get the current cluster info without blocking
*/
public synchronized Cluster fetch() {
return this.cluster;
}
/**
* Fetch cluster metadata including partitions for the given topic. If there is no metadata for the given topic,
* block waiting for an update.
* @param topic The topic we want metadata for
* @param maxWaitMs The maximum amount of time to block waiting for metadata
*/
public synchronized Cluster fetch(String topic, long maxWaitMs) {
List<PartitionInfo> partitions = null;
do {
partitions = cluster.partitionsFor(topic);
if (partitions == null) {
long begin = System.currentTimeMillis();
topics.add(topic);
forceUpdate = true;
try {
wait(maxWaitMs);
} catch (InterruptedException e) { /* this is fine, just try again */
}
long ellapsed = System.currentTimeMillis() - begin;
if (ellapsed > maxWaitMs)
throw new TimeoutException("Failed to update metadata after " + maxWaitMs + " ms.");
} else {
return cluster;
}
} while (true);
}
/**
* Does the current cluster info need to be updated? An update is needed if it has been at least refreshBackoffMs
* since our last update and either (1) an update has been requested or (2) the current metadata has expired (more
* than metadataExpireMs has passed since the last refresh)
*/
public synchronized boolean needsUpdate(long now) {
long msSinceLastUpdate = now - this.lastRefresh;
boolean updateAllowed = msSinceLastUpdate >= this.refreshBackoffMs;
boolean updateNeeded = this.forceUpdate || msSinceLastUpdate >= this.metadataExpireMs;
return updateAllowed && updateNeeded;
}
/**
* Force an update of the current cluster info
*/
public synchronized void forceUpdate() {
this.forceUpdate = true;
}
/**
* Get the list of topics we are currently maintaining metadata for
*/
public synchronized Set<String> topics() {
return new HashSet<String>(this.topics);
}
/**
* Update the cluster metadata
*/
public synchronized void update(Cluster cluster, long now) {
this.forceUpdate = false;
this.lastRefresh = now;
this.cluster = cluster;
notifyAll();
}
}
diff --git a/clients/src/main/java/kafka/clients/producer/internals/Partitioner.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java
similarity index 89%
rename from clients/src/main/java/kafka/clients/producer/internals/Partitioner.java
rename to clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java
index 6d2188e6..04fcae98 100644
--- a/clients/src/main/java/kafka/clients/producer/internals/Partitioner.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Partitioner.java
@@ -1,55 +1,56 @@
-package kafka.clients.producer.internals;
+package org.apache.kafka.clients.producer.internals;
import java.util.List;
import java.util.Random;
import java.util.concurrent.atomic.AtomicInteger;
-import kafka.clients.producer.ProducerRecord;
-import kafka.common.Cluster;
-import kafka.common.PartitionInfo;
-import kafka.common.utils.Utils;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.utils.Utils;
+
/**
* The default partitioning strategy:
* <ul>
* <li>If a partition is specified in the record, use it
* <li>If no partition is specified but a key is present choose a partition based on a hash of the key
* <li>If no partition or key is present choose a partition in a round-robin fashion
*/
public class Partitioner {
private final AtomicInteger counter = new AtomicInteger(new Random().nextInt());
/**
* Compute the partition for the given record.
*
* @param record The record being sent
* @param numPartitions The total number of partitions for the given topic
*/
public int partition(ProducerRecord record, Cluster cluster) {
List<PartitionInfo> partitions = cluster.partitionsFor(record.topic());
int numPartitions = partitions.size();
if (record.partition() != null) {
// they have given us a partition, use it
if (record.partition() < 0 || record.partition() >= numPartitions)
throw new IllegalArgumentException("Invalid partition given with record: " + record.partition()
+ " is not in the range [0..."
+ numPartitions
+ "].");
return record.partition();
} else if (record.key() == null) {
// choose the next available node in a round-robin fashion
for (int i = 0; i < numPartitions; i++) {
int partition = Utils.abs(counter.getAndIncrement()) % numPartitions;
if (partitions.get(partition).leader() != null)
return partition;
}
// no partitions are available, give a non-available partition
return Utils.abs(counter.getAndIncrement()) % numPartitions;
} else {
// hash the key to choose a partition
return Utils.abs(Utils.murmur2(record.key())) % numPartitions;
}
}
}
diff --git a/clients/src/main/java/kafka/clients/producer/internals/ProduceRequestResult.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java
similarity index 93%
rename from clients/src/main/java/kafka/clients/producer/internals/ProduceRequestResult.java
rename to clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java
index cdae00ab..ffeea1ac 100644
--- a/clients/src/main/java/kafka/clients/producer/internals/ProduceRequestResult.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProduceRequestResult.java
@@ -1,81 +1,82 @@
-package kafka.clients.producer.internals;
+package org.apache.kafka.clients.producer.internals;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.TimeUnit;
-import kafka.clients.producer.RecordMetadata;
-import kafka.common.TopicPartition;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.TopicPartition;
+
/**
* 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 RecordMetadata} instances that are batched together
* for the same partition in the request.
*/
public final class ProduceRequestResult {
private final CountDownLatch latch = new CountDownLatch(1);
private volatile TopicPartition topicPartition;
private volatile long baseOffset = -1L;
private volatile RuntimeException error;
public ProduceRequestResult() {
}
/**
* Mark this request as complete and unblock any threads waiting on its completion.
* @param topicPartition The topic and partition to which this record set was sent was sent
* @param baseOffset The base offset assigned to the record
* @param error The error that occurred if there was one, or null.
*/
public void done(TopicPartition topicPartition, long baseOffset, RuntimeException error) {
this.topicPartition = topicPartition;
this.baseOffset = baseOffset;
this.error = error;
this.latch.countDown();
}
/**
* Await the completion of this request
*/
public void await() throws InterruptedException {
latch.await();
}
/**
* 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) throws InterruptedException {
return latch.await(timeout, unit);
}
/**
* The base offset for the request (the first offset in the record set)
*/
public long baseOffset() {
return baseOffset;
}
/**
* The error thrown (generally on the server) while processing this request
*/
public RuntimeException error() {
return error;
}
/**
* The topic and partition to which the record was appended
*/
public TopicPartition topicPartition() {
return topicPartition;
}
/**
* 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/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
similarity index 91%
rename from clients/src/main/java/kafka/clients/producer/internals/RecordAccumulator.java
rename to clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
index c22939f4..18eff5d0 100644
--- a/clients/src/main/java/kafka/clients/producer/internals/RecordAccumulator.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java
@@ -1,234 +1,235 @@
-package kafka.clients.producer.internals;
+package org.apache.kafka.clients.producer.internals;
import java.nio.ByteBuffer;
import java.util.ArrayDeque;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Deque;
import java.util.List;
import java.util.Map;
import java.util.concurrent.ConcurrentMap;
-import kafka.clients.producer.Callback;
-import kafka.common.TopicPartition;
-import kafka.common.metrics.Measurable;
-import kafka.common.metrics.MetricConfig;
-import kafka.common.metrics.Metrics;
-import kafka.common.record.CompressionType;
-import kafka.common.record.MemoryRecords;
-import kafka.common.record.Record;
-import kafka.common.record.Records;
-import kafka.common.utils.CopyOnWriteMap;
-import kafka.common.utils.Time;
-import kafka.common.utils.Utils;
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.metrics.Measurable;
+import org.apache.kafka.common.metrics.MetricConfig;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.record.CompressionType;
+import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.record.Record;
+import org.apache.kafka.common.record.Records;
+import org.apache.kafka.common.utils.CopyOnWriteMap;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+
/**
- * This class acts as a queue that accumulates records into {@link kafka.common.record.MemoryRecords} instances to be
+ * This class acts as a queue that accumulates records into {@link org.apache.kafka.common.record.MemoryRecords} instances to be
* sent to the server.
* <p>
* The accumulator uses a bounded amount of memory and append calls will block when that memory is exhausted, unless
* this behavior is explicitly disabled.
*/
public final class RecordAccumulator {
private volatile boolean closed;
private int drainIndex;
private final int batchSize;
private final long lingerMs;
private final ConcurrentMap<TopicPartition, Deque<RecordBatch>> batches;
private final BufferPool free;
private final Time time;
/**
* Create a new record accumulator
*
- * @param batchSize The size to use when allocating {@link kafka.common.record.MemoryRecords} instances
+ * @param batchSize The size to use when allocating {@link org.apache.kafka.common.record.MemoryRecords} instances
* @param totalSize The maximum memory the record accumulator can use.
* @param lingerMs An artificial delay time to add before declaring a records instance that isn't full ready for
* sending. This allows time for more records to arrive. Setting a non-zero lingerMs will trade off some
* latency for potentially better throughput due to more batching (and hence fewer, larger requests).
* @param blockOnBufferFull If true block when we are out of memory; if false throw an exception when we are out of
* memory
* @param metrics The metrics
* @param time The time instance to use
*/
public RecordAccumulator(int batchSize, long totalSize, long lingerMs, boolean blockOnBufferFull, Metrics metrics, Time time) {
this.drainIndex = 0;
this.closed = false;
this.batchSize = batchSize;
this.lingerMs = lingerMs;
this.batches = new CopyOnWriteMap<TopicPartition, Deque<RecordBatch>>();
this.free = new BufferPool(totalSize, batchSize, blockOnBufferFull);
this.time = time;
registerMetrics(metrics);
}
private void registerMetrics(Metrics metrics) {
metrics.addMetric("blocked_threads",
"The number of user threads blocked waiting for buffer memory to enqueue their records",
new Measurable() {
public double measure(MetricConfig config, long now) {
return free.queued();
}
});
metrics.addMetric("buffer_total_bytes",
"The total amount of buffer memory that is available (not currently used for buffering records).",
new Measurable() {
public double measure(MetricConfig config, long now) {
return free.totalMemory();
}
});
metrics.addMetric("buffer_available_bytes",
"The total amount of buffer memory that is available (not currently used for buffering records).",
new Measurable() {
public double measure(MetricConfig config, long now) {
return free.availableMemory();
}
});
}
/**
* Add a record to the accumulator.
* <p>
* This method will block if sufficient memory isn't available for the record unless blocking has been disabled.
*
* @param tp The topic/partition to which this record is being sent
* @param key The key for the record
* @param value The value for the record
* @param compression The compression codec for the record
* @param callback The user-supplied callback to execute when the request is complete
*/
public FutureRecordMetadata append(TopicPartition tp, byte[] key, byte[] value, CompressionType compression, Callback callback) throws InterruptedException {
if (closed)
throw new IllegalStateException("Cannot send after the producer is closed.");
// check if we have an in-progress batch
Deque<RecordBatch> dq = dequeFor(tp);
synchronized (dq) {
RecordBatch batch = dq.peekLast();
if (batch != null) {
FutureRecordMetadata future = batch.tryAppend(key, value, compression, callback);
if (future != null)
return future;
}
}
// 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) {
FutureRecordMetadata future = first.tryAppend(key, value, compression, callback);
if (future != null) {
// Somebody else found us a batch, return the one we waited for! Hopefully this doesn't happen
// often...
free.deallocate(buffer);
return future;
}
}
RecordBatch batch = new RecordBatch(tp, new MemoryRecords(buffer), time.milliseconds());
FutureRecordMetadata future = Utils.notNull(batch.tryAppend(key, value, compression, callback));
dq.addLast(batch);
return future;
}
}
/**
* Get a list of topic-partitions which are ready to be sent.
* <p>
* A partition is ready if ANY of the following are true:
* <ol>
* <li>The record set is full
* <li>The record set has sat in the accumulator for at least lingerMs milliseconds
* <li>The accumulator is out of memory and threads are blocking waiting for data (in this case all partitions are
* immediately considered ready).
* <li>The accumulator has been closed
* </ol>
*/
public List<TopicPartition> ready(long now) {
List<TopicPartition> ready = new ArrayList<TopicPartition>();
boolean exhausted = this.free.queued() > 0;
for (Map.Entry<TopicPartition, Deque<RecordBatch>> entry : this.batches.entrySet()) {
Deque<RecordBatch> deque = entry.getValue();
synchronized (deque) {
RecordBatch batch = deque.peekFirst();
if (batch != null) {
boolean full = deque.size() > 1 || !batch.records.buffer().hasRemaining();
boolean expired = now - batch.created >= lingerMs;
if (full | expired | exhausted | closed)
ready.add(batch.topicPartition);
}
}
}
return ready;
}
/**
* Drain all the data for the given topic-partitions that will fit within the specified size. This method attempts
* to avoid choosing the same topic-partitions over and over.
*
* @param partitions The list of partitions to drain
* @param maxSize The maximum number of bytes to drain
* @return A list of {@link RecordBatch} for partitions specified with total size less than the requested maxSize.
* TODO: There may be a starvation issue due to iteration order
*/
public List<RecordBatch> drain(List<TopicPartition> partitions, int maxSize) {
if (partitions.isEmpty())
return Collections.emptyList();
int size = 0;
List<RecordBatch> ready = new ArrayList<RecordBatch>();
/* to make starvation less likely this loop doesn't start at 0 */
int start = drainIndex = drainIndex % partitions.size();
do {
TopicPartition tp = partitions.get(drainIndex);
Deque<RecordBatch> deque = dequeFor(tp);
if (deque != null) {
synchronized (deque) {
if (size + deque.peekFirst().records.sizeInBytes() > maxSize) {
return ready;
} else {
RecordBatch batch = deque.pollFirst();
size += batch.records.sizeInBytes();
ready.add(batch);
}
}
}
this.drainIndex = (this.drainIndex + 1) % partitions.size();
} while (start != drainIndex);
return ready;
}
/**
* Get the deque for the given topic-partition, creating it if necessary. Since new topics will only be added rarely
* we copy-on-write the hashmap
*/
private Deque<RecordBatch> dequeFor(TopicPartition tp) {
Deque<RecordBatch> d = this.batches.get(tp);
if (d != null)
return d;
this.batches.putIfAbsent(tp, new ArrayDeque<RecordBatch>());
return this.batches.get(tp);
}
/**
* Deallocate the list of record batches
*/
public void deallocate(Collection<RecordBatch> batches) {
ByteBuffer[] buffers = new ByteBuffer[batches.size()];
int i = 0;
for (RecordBatch batch : batches) {
buffers[i] = batch.records.buffer();
i++;
}
free.deallocate(buffers);
}
/**
* Close this accumulator and force all the record buffers to be drained
*/
public void close() {
this.closed = true;
}
}
diff --git a/clients/src/main/java/kafka/clients/producer/internals/RecordBatch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java
similarity index 89%
rename from clients/src/main/java/kafka/clients/producer/internals/RecordBatch.java
rename to clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java
index 633f4af3..6ba392e1 100644
--- a/clients/src/main/java/kafka/clients/producer/internals/RecordBatch.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java
@@ -1,86 +1,87 @@
-package kafka.clients.producer.internals;
+package org.apache.kafka.clients.producer.internals;
import java.util.ArrayList;
import java.util.List;
-import kafka.clients.producer.Callback;
-import kafka.clients.producer.RecordMetadata;
-import kafka.common.TopicPartition;
-import kafka.common.record.CompressionType;
-import kafka.common.record.MemoryRecords;
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.record.CompressionType;
+import org.apache.kafka.common.record.MemoryRecords;
+
/**
* A batch of records that is or will be sent.
*
* This class is not thread safe and external synchronization must be used when modifying it
*/
public final class RecordBatch {
public int recordCount = 0;
public final long created;
public final MemoryRecords records;
public final TopicPartition topicPartition;
private final ProduceRequestResult produceFuture;
private final List<Thunk> thunks;
public RecordBatch(TopicPartition tp, MemoryRecords records, long now) {
this.created = now;
this.records = records;
this.topicPartition = tp;
this.produceFuture = new ProduceRequestResult();
this.thunks = new ArrayList<Thunk>();
}
/**
* Append the record to the current record set and return the relative offset within that record set
*
* @return The RecordSend corresponding to this record or null if there isn't sufficient room.
*/
public FutureRecordMetadata 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);
FutureRecordMetadata future = new FutureRecordMetadata(this.produceFuture, this.recordCount);
if (callback != null)
thunks.add(new Thunk(callback, this.recordCount));
this.recordCount++;
return future;
}
}
/**
* Complete the request
*
* @param offset The offset
* @param errorCode The error code or 0 if no error
*/
public void done(long offset, RuntimeException exception) {
this.produceFuture.done(topicPartition, offset, exception);
// execute callbacks
for (int i = 0; i < this.thunks.size(); i++) {
try {
Thunk thunk = this.thunks.get(i);
if (exception == null)
thunk.callback.onCompletion(new RecordMetadata(topicPartition, this.produceFuture.baseOffset() + thunk.relativeOffset),
null);
else
thunk.callback.onCompletion(null, exception);
} 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 long relativeOffset;
public Thunk(Callback callback, long relativeOffset) {
this.callback = callback;
this.relativeOffset = relativeOffset;
}
}
}
\ No newline at end of file
diff --git a/clients/src/main/java/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
similarity index 96%
rename from clients/src/main/java/kafka/clients/producer/internals/Sender.java
rename to clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
index 5ac487bc..5a81f354 100644
--- a/clients/src/main/java/kafka/clients/producer/internals/Sender.java
+++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java
@@ -1,504 +1,505 @@
-package kafka.clients.producer.internals;
+package org.apache.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;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.NetworkException;
+import org.apache.kafka.common.network.NetworkReceive;
+import org.apache.kafka.common.network.NetworkSend;
+import org.apache.kafka.common.network.Selectable;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.requests.RequestHeader;
+import org.apache.kafka.common.requests.RequestSend;
+import org.apache.kafka.common.requests.ResponseHeader;
+import org.apache.kafka.common.utils.Time;
+
/**
* The background thread that handles the sending of produce requests to the Kafka cluster. This thread makes metadata
* requests to renew its view of the cluster and then sends produce requests to the appropriate nodes.
*/
public class Sender implements Runnable {
private final Map<Integer, NodeState> nodeState;
private final RecordAccumulator accumulator;
private final Selectable selector;
private final String clientId;
private final int maxRequestSize;
private final long reconnectBackoffMs;
private final short acks;
private final int requestTimeout;
private final InFlightRequests inFlightRequests;
private final Metadata metadata;
private final Time time;
private int correlation;
private boolean metadataFetchInProgress;
private volatile boolean running;
public Sender(Selectable selector,
Metadata metadata,
RecordAccumulator accumulator,
String clientId,
int maxRequestSize,
long reconnectBackoffMs,
short acks,
int requestTimeout,
Time time) {
this.nodeState = new HashMap<Integer, NodeState>();
this.accumulator = accumulator;
this.selector = selector;
this.maxRequestSize = maxRequestSize;
this.reconnectBackoffMs = reconnectBackoffMs;
this.metadata = metadata;
this.clientId = clientId;
this.running = true;
this.requestTimeout = requestTimeout;
this.acks = acks;
this.inFlightRequests = new InFlightRequests();
this.correlation = 0;
this.metadataFetchInProgress = false;
this.time = time;
}
/**
* The main run loop for the sender thread
*/
public void run() {
// main loop, runs until close is called
while (running) {
try {
run(time.milliseconds());
} catch (Exception e) {
e.printStackTrace();
}
}
// send anything left in the accumulator
int unsent = 0;
do {
try {
unsent = run(time.milliseconds());
} catch (Exception e) {
e.printStackTrace();
}
} while (unsent > 0);
// close all the connections
this.selector.close();
}
/**
* Run a single iteration of sending
*
* @param now The current time
* @return The total number of topic/partitions that had data ready (regardless of what we actually sent)
*/
public int run(long now) {
Cluster cluster = metadata.fetch();
// get the list of partitions with data ready to send
List<TopicPartition> ready = this.accumulator.ready(now);
// prune the list of ready topics to eliminate any that we aren't ready to send yet
List<TopicPartition> sendable = processReadyPartitions(cluster, ready, now);
// should we update our metadata?
List<NetworkSend> sends = new ArrayList<NetworkSend>(sendable.size());
InFlightRequest metadataReq = maybeMetadataRequest(cluster, now);
if (metadataReq != null) {
sends.add(metadataReq.request);
this.inFlightRequests.add(metadataReq);
}
// create produce requests
List<RecordBatch> batches = this.accumulator.drain(sendable, this.maxRequestSize);
List<InFlightRequest> requests = collate(cluster, batches);
for (int i = 0; i < requests.size(); i++) {
InFlightRequest request = requests.get(i);
this.inFlightRequests.add(request);
sends.add(request.request);
}
// do the I/O
try {
this.selector.poll(5L, sends);
} catch (IOException e) {
e.printStackTrace();
}
// handle responses, connections, and disconnections
handleSends(this.selector.completedSends());
handleResponses(this.selector.completedReceives(), now);
handleDisconnects(this.selector.disconnected());
handleConnects(this.selector.connected());
return ready.size();
}
private InFlightRequest maybeMetadataRequest(Cluster cluster, long now) {
if (this.metadataFetchInProgress || !metadata.needsUpdate(now))
return null;
Node node = cluster.nextNode();
NodeState state = nodeState.get(node.id());
if (state == null || (state.state == ConnectionState.DISCONNECTED && now - state.lastConnectAttempt > this.reconnectBackoffMs)) {
// we don't have a connection to this node right now, make one
initiateConnect(node, now);
return null;
} else if (state.state == ConnectionState.CONNECTED) {
this.metadataFetchInProgress = true;
return metadataRequest(node.id(), metadata.topics());
} else {
return null;
}
}
/**
* Start closing the sender (won't actually complete until all data is sent out)
*/
public void initiateClose() {
this.running = false;
this.accumulator.close();
}
/**
* Process the set of topic-partitions with data ready to send. If we have a connection to the appropriate node, add
* it to the returned set. For any partitions we have no connection to either make one, fetch the appropriate
* metdata to be able to do so
*/
private List<TopicPartition> processReadyPartitions(Cluster cluster, List<TopicPartition> ready, long now) {
List<TopicPartition> sendable = new ArrayList<TopicPartition>(ready.size());
for (TopicPartition tp : ready) {
Node node = cluster.leaderFor(tp);
if (node == null) {
// we don't know about this topic/partition or it has no leader, re-fetch metadata
metadata.forceUpdate();
} else {
NodeState state = nodeState.get(node.id());
// TODO: encapsulate this logic somehow
if (state == null || (state.state == ConnectionState.DISCONNECTED && now - state.lastConnectAttempt > this.reconnectBackoffMs)) {
// we don't have a connection to this node right now, make one
initiateConnect(node, now);
} else if (state.state == ConnectionState.CONNECTED && inFlightRequests.canSendMore(node.id())) {
sendable.add(tp);
}
}
}
return sendable;
}
/**
* Initiate a connection to the given node
*/
private void initiateConnect(Node node, long now) {
try {
selector.connect(node.id(), new InetSocketAddress(node.host(), node.port()), 64 * 1024 * 1024, 64 * 1024 * 1024); // TODO
// socket
// buffers
nodeState.put(node.id(), new NodeState(ConnectionState.CONNECTING, now));
} catch (IOException e) {
/* attempt failed, we'll try again after the backoff */
nodeState.put(node.id(), new NodeState(ConnectionState.DISCONNECTED, now));
/* maybe the problem is our metadata, update it */
metadata.forceUpdate();
}
}
/**
* Handle any closed connections
*/
private void handleDisconnects(List<Integer> disconnects) {
for (int node : disconnects) {
for (InFlightRequest request : this.inFlightRequests.clearAll(node)) {
if (request.batches != null) {
for (RecordBatch batch : request.batches.values())
batch.done(-1L, new NetworkException("The server disconnected unexpectedly without sending a response."));
this.accumulator.deallocate(request.batches.values());
}
NodeState state = this.nodeState.get(request.request.destination());
if (state != null)
state.state = ConnectionState.DISCONNECTED;
}
}
}
/**
* Record any connections that completed in our node state
*/
private void handleConnects(List<Integer> connects) {
for (Integer id : connects)
this.nodeState.get(id).state = ConnectionState.CONNECTED;
}
/**
* Process completed sends
*/
public void handleSends(List<NetworkSend> sends) {
/* if acks = 0 then the request is satisfied once sent */
for (NetworkSend send : sends) {
Deque<InFlightRequest> requests = this.inFlightRequests.requestQueue(send.destination());
InFlightRequest request = requests.peekFirst();
if (!request.expectResponse) {
requests.pollFirst();
if (request.request.header().apiKey() == ApiKeys.PRODUCE.id) {
for (RecordBatch batch : request.batches.values())
batch.done(-1L, Errors.NONE.exception());
this.accumulator.deallocate(request.batches.values());
}
}
}
}
/**
* Handle responses from the server
*/
private void handleResponses(List<NetworkReceive> receives, long now) {
for (NetworkReceive receive : receives) {
int source = receive.source();
InFlightRequest req = inFlightRequests.nextCompleted(source);
ResponseHeader header = ResponseHeader.parse(receive.payload());
short apiKey = req.request.header().apiKey();
Struct body = (Struct) ProtoUtils.currentResponseSchema(apiKey).read(receive.payload());
correlate(req.request.header(), header);
if (req.request.header().apiKey() == ApiKeys.PRODUCE.id)
handleProduceResponse(req, body);
else if (req.request.header().apiKey() == ApiKeys.METADATA.id)
handleMetadataResponse(body, now);
else
throw new IllegalStateException("Unexpected response type: " + req.request.header().apiKey());
}
}
private void handleMetadataResponse(Struct body, long now) {
this.metadataFetchInProgress = false;
Cluster cluster = ProtoUtils.parseMetadataResponse(body);
this.metadata.update(cluster, now);
}
/**
* Handle a produce response
*/
private void handleProduceResponse(InFlightRequest request, Struct response) {
for (Object topicResponse : (Object[]) response.get("responses")) {
Struct topicRespStruct = (Struct) topicResponse;
String topic = (String) topicRespStruct.get("topic");
for (Object partResponse : (Object[]) topicRespStruct.get("partition_responses")) {
Struct partRespStruct = (Struct) partResponse;
int partition = (Integer) partRespStruct.get("partition");
short errorCode = (Short) partRespStruct.get("error_code");
long offset = (Long) partRespStruct.get("base_offset");
RecordBatch batch = request.batches.get(new TopicPartition(topic, partition));
batch.done(offset, Errors.forCode(errorCode).exception());
}
}
this.accumulator.deallocate(request.batches.values());
}
/**
* Validate that the response corresponds to the request we expect or else explode
*/
private void correlate(RequestHeader requestHeader, ResponseHeader responseHeader) {
if (requestHeader.correlationId() != responseHeader.correlationId())
throw new IllegalStateException("Correlation id for response (" + responseHeader.correlationId()
+ ") does not match request ("
+ requestHeader.correlationId()
+ ")");
}
/**
* Create a metadata request for the given topics
*/
private InFlightRequest metadataRequest(int node, Set<String> topics) {
String[] ts = new String[topics.size()];
topics.toArray(ts);
Struct body = new Struct(ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id));
body.set("topics", topics.toArray());
RequestSend send = new RequestSend(node, new RequestHeader(ApiKeys.METADATA.id, clientId, correlation++), body);
return new InFlightRequest(true, send, null);
}
/**
* Collate the record batches into a list of produce requests on a per-node basis
*/
private List<InFlightRequest> collate(Cluster cluster, List<RecordBatch> batches) {
Map<Integer, List<RecordBatch>> collated = new HashMap<Integer, List<RecordBatch>>();
for (RecordBatch batch : batches) {
Node node = cluster.leaderFor(batch.topicPartition);
List<RecordBatch> found = collated.get(node.id());
if (found == null) {
found = new ArrayList<RecordBatch>();
collated.put(node.id(), found);
}
found.add(batch);
}
List<InFlightRequest> requests = new ArrayList<InFlightRequest>(collated.size());
for (Map.Entry<Integer, List<RecordBatch>> entry : collated.entrySet())
requests.add(produceRequest(entry.getKey(), acks, requestTimeout, entry.getValue()));
return requests;
}
/**
* Create a produce request from the given record batches
*/
private InFlightRequest produceRequest(int destination, short acks, int timeout, List<RecordBatch> batches) {
Map<TopicPartition, RecordBatch> batchesByPartition = new HashMap<TopicPartition, RecordBatch>();
Map<String, List<RecordBatch>> batchesByTopic = new HashMap<String, List<RecordBatch>>();
for (RecordBatch batch : batches) {
batchesByPartition.put(batch.topicPartition, batch);
List<RecordBatch> found = batchesByTopic.get(batch.topicPartition.topic());
if (found == null) {
found = new ArrayList<RecordBatch>();
batchesByTopic.put(batch.topicPartition.topic(), found);
}
found.add(batch);
}
Struct produce = new Struct(ProtoUtils.currentRequestSchema(ApiKeys.PRODUCE.id));
produce.set("acks", acks);
produce.set("timeout", timeout);
List<Struct> topicDatas = new ArrayList<Struct>(batchesByTopic.size());
for (Map.Entry<String, List<RecordBatch>> entry : batchesByTopic.entrySet()) {
Struct topicData = produce.instance("topic_data");
topicData.set("topic", entry.getKey());
List<RecordBatch> parts = entry.getValue();
Object[] partitionData = new Object[parts.size()];
for (int i = 0; i < parts.size(); i++) {
ByteBuffer buffer = parts.get(i).records.buffer();
buffer.flip();
Struct part = topicData.instance("data")
.set("partition", parts.get(i).topicPartition.partition())
.set("record_set", buffer);
partitionData[i] = part;
}
topicData.set("data", partitionData);
topicDatas.add(topicData);
}
produce.set("topic_data", topicDatas.toArray());
RequestHeader header = new RequestHeader(ApiKeys.PRODUCE.id, clientId, correlation++);
RequestSend send = new RequestSend(destination, header, produce);
return new InFlightRequest(acks != 0, send, batchesByPartition);
}
/**
* Wake up the selector associated with this send thread
*/
public void wakeup() {
this.selector.wakeup();
}
/**
* The states of a node connection
*/
private static enum ConnectionState {
DISCONNECTED, CONNECTING, CONNECTED
}
/**
* The state of a node
*/
private static final class NodeState {
private ConnectionState state;
private long lastConnectAttempt;
public NodeState(ConnectionState state, long lastConnectAttempt) {
this.state = state;
this.lastConnectAttempt = lastConnectAttempt;
}
public String toString() {
return "NodeState(" + state + ", " + lastConnectAttempt + ")";
}
}
/**
* An request that hasn't been fully processed yet
*/
private static final class InFlightRequest {
public boolean expectResponse;
public Map<TopicPartition, RecordBatch> batches;
public RequestSend request;
/**
* @param expectResponse Should we expect a response message or is this request complete once it is sent?
* @param request The request
* @param batches The record batches contained in the request if it is a produce request
*/
public InFlightRequest(boolean expectResponse, RequestSend request, Map<TopicPartition, RecordBatch> batches) {
this.batches = batches;
this.request = request;
this.expectResponse = expectResponse;
}
}
/**
* A set of outstanding request queues for each node that have not yet received responses
*/
private static final class InFlightRequests {
private final Map<Integer, Deque<InFlightRequest>> requests = new HashMap<Integer, Deque<InFlightRequest>>();
/**
* Add the given request to the queue for the node it was directed to
*/
public void add(InFlightRequest request) {
Deque<InFlightRequest> reqs = this.requests.get(request.request.destination());
if (reqs == null) {
reqs = new ArrayDeque<InFlightRequest>();
this.requests.put(request.request.destination(), reqs);
}
reqs.addFirst(request);
}
public Deque<InFlightRequest> requestQueue(int node) {
Deque<InFlightRequest> reqs = requests.get(node);
if (reqs == null || reqs.isEmpty())
throw new IllegalStateException("Response from server for which there are no in-flight requests.");
return reqs;
}
/**
* Get the oldest request (the one that that will be completed next) for the given node
*/
public InFlightRequest nextCompleted(int node) {
return requestQueue(node).pollLast();
}
/**
* Can we send more requests to this node?
*
* @param node Node in question
* @return true iff we have no requests still being sent to the given node
*/
public boolean canSendMore(int node) {
Deque<InFlightRequest> queue = requests.get(node);
return queue == null || queue.isEmpty() || queue.peekFirst().request.complete();
}
/**
* Clear out all the in-flight requests for the given node and return them
*
* @param node The node
* @return All the in-flight requests for that node that have been removed
*/
public Iterable<InFlightRequest> clearAll(int node) {
Deque<InFlightRequest> reqs = requests.get(node);
if (reqs == null) {
return Collections.emptyList();
} else {
return requests.remove(node);
}
}
}
}
diff --git a/clients/src/main/java/kafka/clients/tools/ProducerPerformance.java b/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java
similarity index 87%
rename from clients/src/main/java/kafka/clients/tools/ProducerPerformance.java
rename to clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java
index 973eb5e6..b5e792b7 100644
--- a/clients/src/main/java/kafka/clients/tools/ProducerPerformance.java
+++ b/clients/src/main/java/org/apache/kafka/clients/tools/ProducerPerformance.java
@@ -1,65 +1,66 @@
-package kafka.clients.tools;
+package org.apache.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.RecordMetadata;
-import kafka.common.record.Records;
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerConfig;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.common.record.Records;
+
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_records record_size");
System.exit(1);
}
String url = args[0];
int numRecords = Integer.parseInt(args[1]);
int recordSize = 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));
KafkaProducer producer = new KafkaProducer(props);
Callback callback = new Callback() {
public void onCompletion(RecordMetadata metadata, Exception e) {
if (e != null)
e.printStackTrace();
}
};
byte[] payload = new byte[recordSize];
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 < numRecords; i++) {
long sendStart = System.currentTimeMillis();
producer.send(record, callback);
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 * numRecords / (double) ellapsed;
double mbSec = msgsSec * (recordSize + Records.LOG_OVERHEAD) / (1024.0 * 1024.0);
System.out.printf("%d records sent in %d ms ms. %.2f records per second (%.2f mb/sec).", numRecords, ellapsed, msgsSec, mbSec);
producer.close();
}
}
diff --git a/clients/src/main/java/kafka/common/Cluster.java b/clients/src/main/java/org/apache/kafka/common/Cluster.java
similarity index 98%
rename from clients/src/main/java/kafka/common/Cluster.java
rename to clients/src/main/java/org/apache/kafka/common/Cluster.java
index 8d045d56..634895ca 100644
--- a/clients/src/main/java/kafka/common/Cluster.java
+++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java
@@ -1,123 +1,124 @@
-package kafka.common;
+package org.apache.kafka.common;
import java.net.InetSocketAddress;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.concurrent.atomic.AtomicInteger;
-import kafka.common.utils.Utils;
+import org.apache.kafka.common.utils.Utils;
+
/**
* A representation of a subset of the nodes, topics, and partitions in the Kafka cluster.
*/
public final class Cluster {
private final AtomicInteger counter = new AtomicInteger(0);
private final List<Node> nodes;
private final Map<TopicPartition, PartitionInfo> partitionsByTopicPartition;
private final Map<String, List<PartitionInfo>> partitionsByTopic;
/**
* Create a new cluster with the given nodes and partitions
* @param nodes The nodes in the cluster
* @param partitions Information about a subset of the topic-partitions this cluster hosts
*/
public Cluster(Collection<Node> nodes, Collection<PartitionInfo> partitions) {
// make a randomized, unmodifiable copy of the nodes
List<Node> copy = new ArrayList<Node>(nodes);
Collections.shuffle(copy);
this.nodes = Collections.unmodifiableList(copy);
// index the partitions by topic/partition for quick lookup
this.partitionsByTopicPartition = new HashMap<TopicPartition, PartitionInfo>(partitions.size());
for (PartitionInfo p : partitions)
this.partitionsByTopicPartition.put(new TopicPartition(p.topic(), p.partition()), p);
// index the partitions by topic and make the lists unmodifiable so we can handle them out in
// user-facing apis without risk of the client modifying the contents
HashMap<String, List<PartitionInfo>> parts = new HashMap<String, List<PartitionInfo>>();
for (PartitionInfo p : partitions) {
if (!parts.containsKey(p.topic()))
parts.put(p.topic(), new ArrayList<PartitionInfo>());
List<PartitionInfo> ps = parts.get(p.topic());
ps.add(p);
}
this.partitionsByTopic = new HashMap<String, List<PartitionInfo>>(parts.size());
for (Map.Entry<String, List<PartitionInfo>> entry : parts.entrySet())
this.partitionsByTopic.put(entry.getKey(), Collections.unmodifiableList(entry.getValue()));
}
/**
* Create an empty cluster instance with no nodes and no topic-partitions.
*/
public static Cluster empty() {
return new Cluster(new ArrayList<Node>(0), new ArrayList<PartitionInfo>(0));
}
/**
* Create a "bootstrap" cluster using the given list of host/ports
* @param addresses The addresses
* @return A cluster for these hosts/ports
*/
public static Cluster bootstrap(List<InetSocketAddress> addresses) {
List<Node> nodes = new ArrayList<Node>();
int nodeId = Integer.MIN_VALUE;
for (InetSocketAddress address : addresses)
nodes.add(new Node(nodeId++, address.getHostName(), address.getPort()));
return new Cluster(nodes, new ArrayList<PartitionInfo>(0));
}
/**
* @return The known set of nodes
*/
public List<Node> nodes() {
return this.nodes;
}
/**
* 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 info.leader();
}
/**
* Get the metadata for the specified partition
* @param topicPartition The topic and partition to fetch info for
* @return The metadata about the given topic and partition
*/
public PartitionInfo partition(TopicPartition topicPartition) {
return partitionsByTopicPartition.get(topicPartition);
}
/**
* Get the list of partitions for this topic
* @param topic The topic name
* @return A list of partitions
*/
public List<PartitionInfo> partitionsFor(String topic) {
return this.partitionsByTopic.get(topic);
}
/**
* Round-robin over the nodes in this cluster
*/
public Node nextNode() {
int size = nodes.size();
if (size == 0)
throw new IllegalStateException("No known nodes.");
int idx = Utils.abs(counter.getAndIncrement()) % size;
return this.nodes.get(idx);
}
}
diff --git a/clients/src/main/java/kafka/common/Configurable.java b/clients/src/main/java/org/apache/kafka/common/Configurable.java
similarity index 90%
rename from clients/src/main/java/kafka/common/Configurable.java
rename to clients/src/main/java/org/apache/kafka/common/Configurable.java
index 1af9dd45..37da3579 100644
--- a/clients/src/main/java/kafka/common/Configurable.java
+++ b/clients/src/main/java/org/apache/kafka/common/Configurable.java
@@ -1,15 +1,15 @@
-package kafka.common;
+package org.apache.kafka.common;
import java.util.Map;
/**
* A Mix-in style interface for classes that are instantiated by reflection and need to take configuration parameters
*/
public interface Configurable {
/**
* Configure this class with the given key-value pairs
*/
public void configure(Map<String, ?> configs);
}
diff --git a/clients/src/main/java/kafka/common/KafkaException.java b/clients/src/main/java/org/apache/kafka/common/KafkaException.java
similarity index 93%
rename from clients/src/main/java/kafka/common/KafkaException.java
rename to clients/src/main/java/org/apache/kafka/common/KafkaException.java
index 7182cace..d529a040 100644
--- a/clients/src/main/java/kafka/common/KafkaException.java
+++ b/clients/src/main/java/org/apache/kafka/common/KafkaException.java
@@ -1,26 +1,26 @@
-package kafka.common;
+package org.apache.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/org/apache/kafka/common/Metric.java
similarity index 92%
rename from clients/src/main/java/kafka/common/Metric.java
rename to clients/src/main/java/org/apache/kafka/common/Metric.java
index c29e3315..55800701 100644
--- a/clients/src/main/java/kafka/common/Metric.java
+++ b/clients/src/main/java/org/apache/kafka/common/Metric.java
@@ -1,23 +1,23 @@
-package kafka.common;
+package org.apache.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/org/apache/kafka/common/Node.java
similarity index 97%
rename from clients/src/main/java/kafka/common/Node.java
rename to clients/src/main/java/org/apache/kafka/common/Node.java
index 81fc907d..452cbb19 100644
--- a/clients/src/main/java/kafka/common/Node.java
+++ b/clients/src/main/java/org/apache/kafka/common/Node.java
@@ -1,76 +1,76 @@
-package kafka.common;
+package org.apache.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/org/apache/kafka/common/PartitionInfo.java
similarity index 97%
rename from clients/src/main/java/kafka/common/PartitionInfo.java
rename to clients/src/main/java/org/apache/kafka/common/PartitionInfo.java
index 0e50ed74..06babefd 100644
--- a/clients/src/main/java/kafka/common/PartitionInfo.java
+++ b/clients/src/main/java/org/apache/kafka/common/PartitionInfo.java
@@ -1,58 +1,58 @@
-package kafka.common;
+package org.apache.kafka.common;
/**
* Information about a topic-partition.
*/
public class PartitionInfo {
private final String topic;
private final int partition;
private final Node leader;
private final Node[] replicas;
private final Node[] inSyncReplicas;
public PartitionInfo(String topic, int partition, Node leader, Node[] replicas, Node[] 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 Node leader() {
return leader;
}
/**
* The complete set of replicas for this partition regardless of whether they are alive or up-to-date
*/
public Node[] 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 Node[] inSyncReplicas() {
return inSyncReplicas;
}
}
diff --git a/clients/src/main/java/kafka/common/TopicPartition.java b/clients/src/main/java/org/apache/kafka/common/TopicPartition.java
similarity index 97%
rename from clients/src/main/java/kafka/common/TopicPartition.java
rename to clients/src/main/java/org/apache/kafka/common/TopicPartition.java
index e7be96c6..7ac06040 100644
--- a/clients/src/main/java/kafka/common/TopicPartition.java
+++ b/clients/src/main/java/org/apache/kafka/common/TopicPartition.java
@@ -1,61 +1,61 @@
-package kafka.common;
+package org.apache.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/org/apache/kafka/common/config/AbstractConfig.java
similarity index 93%
rename from clients/src/main/java/kafka/common/config/AbstractConfig.java
rename to clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
index 5db302db..5d548d0e 100644
--- a/clients/src/main/java/kafka/common/config/AbstractConfig.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
@@ -1,93 +1,94 @@
-package kafka.common.config;
+package org.apache.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;
+import org.apache.kafka.common.Configurable;
+import org.apache.kafka.common.KafkaException;
+import org.apache.kafka.common.utils.Utils;
+
/**
* A convenient base class for configurations to extend.
* <p>
* This class holds both the original configuration that was provided as well as the parsed
*/
public class AbstractConfig {
private final Set<String> used;
private final Map<String, Object> values;
private final Map<String, ?> originals;
@SuppressWarnings("unchecked")
public AbstractConfig(ConfigDef definition, Map<?, ?> originals) {
/* check that all the keys are really strings */
for (Object key : originals.keySet())
if (!(key instanceof String))
throw new ConfigException(key.toString(), originals.get(key), "Key must be a string.");
this.originals = (Map<String, ?>) originals;
this.values = definition.parse(this.originals);
this.used = Collections.synchronizedSet(new HashSet<String>());
}
protected Object get(String key) {
if (!values.containsKey(key))
throw new ConfigException(String.format("Unknown configuration '%s'", key));
used.add(key);
return values.get(key);
}
public int getInt(String key) {
return (Integer) get(key);
}
public Long getLong(String key) {
return (Long) get(key);
}
@SuppressWarnings("unchecked")
public List<String> getList(String key) {
return (List<String>) get(key);
}
public boolean getBoolean(String key) {
return (Boolean) get(key);
}
public String getString(String key) {
return (String) get(key);
}
public Class<?> getClass(String key) {
return (Class<?>) get(key);
}
public Set<String> unused() {
Set<String> keys = new HashSet<String>(originals.keySet());
keys.remove(used);
return keys;
}
/**
* Get a configured instance of the give class specified by the given configuration key. If the object implements
* Configurable configure it using the configuration.
*
* @param key The configuration key for the class
* @param t The interface the class should implement
* @return A configured instance of the class
*/
public <T> T getConfiguredInstance(String key, Class<T> t) {
Class<?> c = getClass(key);
if (c == null)
return null;
Object o = Utils.newInstance(c);
if (!t.isInstance(o))
throw new KafkaException(c.getName() + " is not an instance of " + t.getName());
if (o instanceof Configurable)
((Configurable) o).configure(this.originals);
return t.cast(o);
}
}
diff --git a/clients/src/main/java/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
similarity index 99%
rename from clients/src/main/java/kafka/common/config/ConfigDef.java
rename to clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
index 2507c9cc..cc4bc481 100644
--- a/clients/src/main/java/kafka/common/config/ConfigDef.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
@@ -1,253 +1,253 @@
-package kafka.common.config;
+package org.apache.kafka.common.config;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
/**
* This class is used for specifying the set of expected configurations, their type, their defaults, their
* documentation, and any special validation logic used for checking the correctness of the values the user provides.
* <p>
* Usage of this class looks something like this:
*
* <pre>
* ConfigDef defs = new ConfigDef();
* defs.define(&quot;config_name&quot;, Type.STRING, &quot;default string value&quot;, &quot;This configuration is used for blah blah blah.&quot;);
* defs.define(&quot;another_config_name&quot;, Type.INT, 42, Range.atLeast(0), &quot;More documentation on this config&quot;);
*
* Properties props = new Properties();
* props.setProperty(&quot;config_name&quot;, &quot;some value&quot;);
* Map&lt;String, Object&gt; configs = defs.parse(props);
*
* String someConfig = (String) configs.get(&quot;config_name&quot;); // will return &quot;some value&quot;
* int anotherConfig = (Integer) configs.get(&quot;another_config_name&quot;); // will return default value of 42
* </pre>
*
* This class can be used stand-alone or in combination with {@link AbstractConfig} which provides some additional
* functionality for accessing configs.
*/
public class ConfigDef {
private static final Object NO_DEFAULT_VALUE = new Object();
private final Map<String, ConfigKey> configKeys = new HashMap<String, ConfigKey>();
/**
* Define a new configuration
* @param name The name of the config parameter
* @param type The type of the config
* @param defaultValue The default value to use if this config isn't present
* @param validator A validator to use in checking the correctness of the config
* @param documentation The documentation string for the config
* @return This ConfigDef so you can chain calls
*/
public ConfigDef define(String name, Type type, Object defaultValue, Validator validator, String documentation) {
if (configKeys.containsKey(name))
throw new ConfigException("Configuration " + name + " is defined twice.");
Object parsedDefault = defaultValue == NO_DEFAULT_VALUE ? NO_DEFAULT_VALUE : parseType(name, defaultValue, type);
configKeys.put(name, new ConfigKey(name, type, parsedDefault, validator, documentation));
return this;
}
/**
* Define a new configuration with no special validation logic
* @param name The name of the config parameter
* @param type The type of the config
* @param defaultValue The default value to use if this config isn't present
* @param documentation The documentation string for the config
* @return This ConfigDef so you can chain calls
*/
public ConfigDef define(String name, Type type, Object defaultValue, String documentation) {
return define(name, type, defaultValue, null, documentation);
}
/**
* Define a required parameter with no default value
* @param name The name of the config parameter
* @param type The type of the config
* @param validator A validator to use in checking the correctness of the config
* @param documentation The documentation string for the config
* @return This ConfigDef so you can chain calls
*/
public ConfigDef define(String name, Type type, Validator validator, String documentation) {
return define(name, type, NO_DEFAULT_VALUE, validator, documentation);
}
/**
* Define a required parameter with no default value and no special validation logic
* @param name The name of the config parameter
* @param type The type of the config
* @param documentation The documentation string for the config
* @return This ConfigDef so you can chain calls
*/
public ConfigDef define(String name, Type type, String documentation) {
return define(name, type, NO_DEFAULT_VALUE, null, documentation);
}
/**
* Parse and validate configs against this configuration definition. The input is a map of configs. It is expected
* that the keys of the map are strings, but the values can either be strings or they may already be of the
* appropriate type (int, string, etc). This will work equally well with either java.util.Properties instances or a
* programmatically constructed map.
* @param props The configs to parse and validate
* @return Parsed and validated configs. The key will be the config name and the value will be the value parsed into
* the appropriate type (int, string, etc)
*/
public Map<String, Object> parse(Map<?, ?> props) {
/* parse all known keys */
Map<String, Object> values = new HashMap<String, Object>();
for (ConfigKey key : configKeys.values()) {
Object value;
if (props.containsKey(key.name))
value = parseType(key.name, props.get(key.name), key.type);
else if (key.defaultValue == NO_DEFAULT_VALUE)
throw new ConfigException("Missing required configuration \"" + key.name + "\" which has no default value.");
else
value = key.defaultValue;
values.put(key.name, value);
}
return values;
}
/**
* Parse a value according to its expected type.
* @param name The config name
* @param value The config value
* @param type The expected type
* @return The parsed object
*/
private Object parseType(String name, Object value, Type type) {
try {
String trimmed = null;
if (value instanceof String)
trimmed = ((String) value).trim();
switch (type) {
case BOOLEAN:
if (value instanceof String)
return Boolean.parseBoolean(trimmed);
else if (value instanceof Boolean)
return value;
else
throw new ConfigException(name, value, "Expected value to be either true or false");
case STRING:
if (value instanceof String)
return trimmed;
else
throw new ConfigException(name, value, "Expected value to be a string, but it was a " + value.getClass().getName());
case INT:
if (value instanceof Integer) {
return (Integer) value;
} else if (value instanceof String) {
return Integer.parseInt(trimmed);
} else {
throw new ConfigException(name, value, "Expected value to be an number.");
}
case LONG:
if (value instanceof Integer)
return ((Integer) value).longValue();
if (value instanceof Long)
return (Long) value;
else if (value instanceof String)
return Long.parseLong(trimmed);
else
throw new ConfigException(name, value, "Expected value to be an number.");
case DOUBLE:
if (value instanceof Number)
return ((Number) value).doubleValue();
else if (value instanceof String)
return Double.parseDouble(trimmed);
else
throw new ConfigException(name, value, "Expected value to be an number.");
case LIST:
if (value instanceof List)
return (List<?>) value;
else if (value instanceof String)
return Arrays.asList(trimmed.split("\\s*,\\s*", -1));
else
throw new ConfigException(name, value, "Expected a comma seperated list.");
case CLASS:
if (value instanceof Class)
return (Class<?>) value;
else if (value instanceof String)
return Class.forName(trimmed);
else
throw new ConfigException(name, value, "Expected a Class instance or class name.");
default:
throw new IllegalStateException("Unknown type.");
}
} catch (NumberFormatException e) {
throw new ConfigException(name, value, "Not a number of type " + type);
} catch (ClassNotFoundException e) {
throw new ConfigException(name, value, "Class " + value + " could not be found.");
}
}
/**
* The config types
*/
public enum Type {
BOOLEAN, STRING, INT, LONG, DOUBLE, LIST, CLASS;
}
/**
* Validation logic the user may provide
*/
public interface Validator {
public void ensureValid(String name, Object o);
}
/**
* Validation logic for numeric ranges
*/
public static class Range implements Validator {
private final Number min;
private final Number max;
private Range(Number min, Number max) {
this.min = min;
this.max = max;
}
/**
* A numeric range that checks only the lower bound
* @param min The minimum acceptable value
*/
public static Range atLeast(Number min) {
return new Range(min, Double.MAX_VALUE);
}
/**
* A numeric range that checks both the upper and lower bound
*/
public static Range between(Number min, Number max) {
return new Range(min, max);
}
public void ensureValid(String name, Object o) {
Number n = (Number) o;
if (n.doubleValue() < min.doubleValue() || n.doubleValue() > max.doubleValue())
throw new ConfigException(name, o, "Value must be in the range [" + min + ", " + max + "]");
}
}
private static class ConfigKey {
public final String name;
public final Type type;
public final String documentation;
public final Object defaultValue;
public final Validator validator;
public ConfigKey(String name, Type type, Object defaultValue, Validator validator, String documentation) {
super();
this.name = name;
this.type = type;
this.defaultValue = defaultValue;
this.validator = validator;
if (this.validator != null)
this.validator.ensureValid(name, defaultValue);
this.documentation = documentation;
}
}
}
diff --git a/clients/src/main/java/kafka/common/config/ConfigException.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigException.java
similarity index 86%
rename from clients/src/main/java/kafka/common/config/ConfigException.java
rename to clients/src/main/java/org/apache/kafka/common/config/ConfigException.java
index fad141e2..c2a59cf3 100644
--- a/clients/src/main/java/kafka/common/config/ConfigException.java
+++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigException.java
@@ -1,24 +1,24 @@
-package kafka.common.config;
+package org.apache.kafka.common.config;
-import kafka.common.KafkaException;
+import org.apache.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/org/apache/kafka/common/errors/ApiException.java
similarity index 89%
rename from clients/src/main/java/kafka/common/errors/ApiException.java
rename to clients/src/main/java/org/apache/kafka/common/errors/ApiException.java
index 28f54114..46c40028 100644
--- a/clients/src/main/java/kafka/common/errors/ApiException.java
+++ b/clients/src/main/java/org/apache/kafka/common/errors/ApiException.java
@@ -1,35 +1,35 @@
-package kafka.common.errors;
+package org.apache.kafka.common.errors;
-import kafka.common.KafkaException;
+import org.apache.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/CorruptRecordException.java b/clients/src/main/java/org/apache/kafka/common/errors/CorruptRecordException.java
similarity index 92%
rename from clients/src/main/java/kafka/common/errors/CorruptRecordException.java
rename to clients/src/main/java/org/apache/kafka/common/errors/CorruptRecordException.java
index 492f2e35..448f6273 100644
--- a/clients/src/main/java/kafka/common/errors/CorruptRecordException.java
+++ b/clients/src/main/java/org/apache/kafka/common/errors/CorruptRecordException.java
@@ -1,23 +1,23 @@
-package kafka.common.errors;
+package org.apache.kafka.common.errors;
public class CorruptRecordException extends ApiException {
private static final long serialVersionUID = 1L;
public CorruptRecordException() {
super("This message has failed it's CRC checksum or is otherwise corrupt.");
}
public CorruptRecordException(String message) {
super(message);
}
public CorruptRecordException(Throwable cause) {
super(cause);
}
public CorruptRecordException(String message, Throwable cause) {
super(message, cause);
}
}
diff --git a/clients/src/main/java/kafka/common/errors/LeaderNotAvailableException.java b/clients/src/main/java/org/apache/kafka/common/errors/LeaderNotAvailableException.java
similarity index 91%
rename from clients/src/main/java/kafka/common/errors/LeaderNotAvailableException.java
rename to clients/src/main/java/org/apache/kafka/common/errors/LeaderNotAvailableException.java
index d7b86f8a..571fdd76 100644
--- a/clients/src/main/java/kafka/common/errors/LeaderNotAvailableException.java
+++ b/clients/src/main/java/org/apache/kafka/common/errors/LeaderNotAvailableException.java
@@ -1,19 +1,19 @@
-package kafka.common.errors;
+package org.apache.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/NetworkException.java b/clients/src/main/java/org/apache/kafka/common/errors/NetworkException.java
similarity index 91%
rename from clients/src/main/java/kafka/common/errors/NetworkException.java
rename to clients/src/main/java/org/apache/kafka/common/errors/NetworkException.java
index daedbf4e..14621e76 100644
--- a/clients/src/main/java/kafka/common/errors/NetworkException.java
+++ b/clients/src/main/java/org/apache/kafka/common/errors/NetworkException.java
@@ -1,23 +1,23 @@
-package kafka.common.errors;
+package org.apache.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/org/apache/kafka/common/errors/NotLeaderForPartitionException.java
similarity index 92%
rename from clients/src/main/java/kafka/common/errors/NotLeaderForPartitionException.java
rename to clients/src/main/java/org/apache/kafka/common/errors/NotLeaderForPartitionException.java
index 5d750fd4..2404afbe 100644
--- a/clients/src/main/java/kafka/common/errors/NotLeaderForPartitionException.java
+++ b/clients/src/main/java/org/apache/kafka/common/errors/NotLeaderForPartitionException.java
@@ -1,23 +1,23 @@
-package kafka.common.errors;
+package org.apache.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/org/apache/kafka/common/errors/OffsetMetadataTooLarge.java
similarity index 91%
rename from clients/src/main/java/kafka/common/errors/OffsetMetadataTooLarge.java
rename to clients/src/main/java/org/apache/kafka/common/errors/OffsetMetadataTooLarge.java
index ab9cd62e..e82dcc2a 100644
--- a/clients/src/main/java/kafka/common/errors/OffsetMetadataTooLarge.java
+++ b/clients/src/main/java/org/apache/kafka/common/errors/OffsetMetadataTooLarge.java
@@ -1,22 +1,22 @@
-package kafka.common.errors;
+package org.apache.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/org/apache/kafka/common/errors/OffsetOutOfRangeException.java
similarity index 91%
rename from clients/src/main/java/kafka/common/errors/OffsetOutOfRangeException.java
rename to clients/src/main/java/org/apache/kafka/common/errors/OffsetOutOfRangeException.java
index 93210cd8..fa0a673c 100644
--- a/clients/src/main/java/kafka/common/errors/OffsetOutOfRangeException.java
+++ b/clients/src/main/java/org/apache/kafka/common/errors/OffsetOutOfRangeException.java
@@ -1,22 +1,22 @@
-package kafka.common.errors;
+package org.apache.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/RecordTooLargeException.java b/clients/src/main/java/org/apache/kafka/common/errors/RecordTooLargeException.java
similarity index 91%
rename from clients/src/main/java/kafka/common/errors/RecordTooLargeException.java
rename to clients/src/main/java/org/apache/kafka/common/errors/RecordTooLargeException.java
index bef42932..f06065e4 100644
--- a/clients/src/main/java/kafka/common/errors/RecordTooLargeException.java
+++ b/clients/src/main/java/org/apache/kafka/common/errors/RecordTooLargeException.java
@@ -1,23 +1,23 @@
-package kafka.common.errors;
+package org.apache.kafka.common.errors;
public class RecordTooLargeException extends ApiException {
private static final long serialVersionUID = 1L;
public RecordTooLargeException() {
super();
}
public RecordTooLargeException(String message, Throwable cause) {
super(message, cause);
}
public RecordTooLargeException(String message) {
super(message);
}
public RecordTooLargeException(Throwable cause) {
super(cause);
}
}
diff --git a/clients/src/main/java/kafka/common/errors/RetryableException.java b/clients/src/main/java/org/apache/kafka/common/errors/RetryableException.java
similarity index 95%
rename from clients/src/main/java/kafka/common/errors/RetryableException.java
rename to clients/src/main/java/org/apache/kafka/common/errors/RetryableException.java
index 5aa8684d..53c5e8d3 100644
--- a/clients/src/main/java/kafka/common/errors/RetryableException.java
+++ b/clients/src/main/java/org/apache/kafka/common/errors/RetryableException.java
@@ -1,31 +1,31 @@
-package kafka.common.errors;
+package org.apache.kafka.common.errors;
/**
* A retryable exception is an exception that is safe to retry. To be retryable an exception should be
* <ol>
* <li>Transient, there is no point retrying a error due to a non-existant topic or message too large
* <li>Idempotent, the exception is known to not change any state on the server
* </ol>
* A client may choose to retry any request they like, but exceptions extending this class are always safe and sane to
* retry.
*/
public abstract class RetryableException extends ApiException {
private static final long serialVersionUID = 1L;
public RetryableException(String message, Throwable cause) {
super(message, cause);
}
public RetryableException(String message) {
super(message);
}
public RetryableException(Throwable cause) {
super(cause);
}
public RetryableException() {
}
}
diff --git a/clients/src/main/java/kafka/common/errors/TimeoutException.java b/clients/src/main/java/org/apache/kafka/common/errors/TimeoutException.java
similarity index 91%
rename from clients/src/main/java/kafka/common/errors/TimeoutException.java
rename to clients/src/main/java/org/apache/kafka/common/errors/TimeoutException.java
index da27a985..5e1bb671 100644
--- a/clients/src/main/java/kafka/common/errors/TimeoutException.java
+++ b/clients/src/main/java/org/apache/kafka/common/errors/TimeoutException.java
@@ -1,23 +1,23 @@
-package kafka.common.errors;
+package org.apache.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/org/apache/kafka/common/errors/UnknownServerException.java
similarity index 91%
rename from clients/src/main/java/kafka/common/errors/UnknownServerException.java
rename to clients/src/main/java/org/apache/kafka/common/errors/UnknownServerException.java
index d0b56d67..73fd755d 100644
--- a/clients/src/main/java/kafka/common/errors/UnknownServerException.java
+++ b/clients/src/main/java/org/apache/kafka/common/errors/UnknownServerException.java
@@ -1,22 +1,22 @@
-package kafka.common.errors;
+package org.apache.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/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.java
similarity index 92%
rename from clients/src/main/java/kafka/common/errors/UnknownTopicOrPartitionException.java
rename to clients/src/main/java/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.java
index 5c1ca128..2376bff3 100644
--- a/clients/src/main/java/kafka/common/errors/UnknownTopicOrPartitionException.java
+++ b/clients/src/main/java/org/apache/kafka/common/errors/UnknownTopicOrPartitionException.java
@@ -1,22 +1,22 @@
-package kafka.common.errors;
+package org.apache.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/org/apache/kafka/common/metrics/CompoundStat.java
similarity index 95%
rename from clients/src/main/java/kafka/common/metrics/CompoundStat.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/CompoundStat.java
index 5541e322..a421f659 100644
--- a/clients/src/main/java/kafka/common/metrics/CompoundStat.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/CompoundStat.java
@@ -1,40 +1,40 @@
-package kafka.common.metrics;
+package org.apache.kafka.common.metrics;
import java.util.List;
/**
* A compound stat is a stat where a single measurement and associated data structure feeds many metrics. This is the
* example for a histogram which has many associated percentiles.
*/
public interface CompoundStat extends Stat {
public List<NamedMeasurable> stats();
public static class NamedMeasurable {
private final String name;
private final String description;
private final Measurable stat;
public NamedMeasurable(String name, String description, Measurable stat) {
super();
this.name = name;
this.description = description;
this.stat = stat;
}
public String name() {
return name;
}
public String description() {
return description;
}
public Measurable stat() {
return stat;
}
}
}
diff --git a/clients/src/main/java/kafka/common/metrics/JmxReporter.java b/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java
similarity index 98%
rename from clients/src/main/java/kafka/common/metrics/JmxReporter.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java
index a0cee01e..57a6d984 100644
--- a/clients/src/main/java/kafka/common/metrics/JmxReporter.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/JmxReporter.java
@@ -1,184 +1,185 @@
-package kafka.common.metrics;
+package org.apache.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;
+import org.apache.kafka.common.KafkaException;
+
/**
* Register metrics in JMX as dynamic mbeans based on the metric names
*/
public class JmxReporter implements MetricsReporter {
private final String prefix;
private final Map<String, KafkaMbean> mbeans = new HashMap<String, KafkaMbean>();
public JmxReporter() {
this("");
}
/**
* Create a JMX reporter that prefixes all metrics with the given string.
*/
public JmxReporter(String prefix) {
this.prefix = prefix;
}
@Override
public synchronized void init(List<KafkaMetric> metrics) {
for (KafkaMetric metric : metrics)
addAttribute(metric);
for (KafkaMbean mbean : mbeans.values())
reregister(mbean);
}
@Override
public synchronized void metricChange(KafkaMetric metric) {
KafkaMbean mbean = addAttribute(metric);
reregister(mbean);
}
private KafkaMbean addAttribute(KafkaMetric metric) {
try {
String[] names = split(prefix + metric.name());
String qualifiedName = names[0] + "." + names[1];
if (!this.mbeans.containsKey(qualifiedName))
mbeans.put(qualifiedName, new KafkaMbean(names[0], names[1]));
KafkaMbean mbean = this.mbeans.get(qualifiedName);
mbean.setAttribute(names[2], metric);
return mbean;
} catch (JMException e) {
throw new KafkaException("Error creating mbean attribute " + metric.name(), e);
}
}
public synchronized void close() {
for (KafkaMbean mbean : this.mbeans.values())
unregister(mbean);
}
private void unregister(KafkaMbean mbean) {
MBeanServer server = ManagementFactory.getPlatformMBeanServer();
try {
if (server.isRegistered(mbean.name()))
server.unregisterMBean(mbean.name());
} catch (JMException e) {
throw new KafkaException("Error unregistering mbean", e);
}
}
private void reregister(KafkaMbean mbean) {
unregister(mbean);
try {
ManagementFactory.getPlatformMBeanServer().registerMBean(mbean, mbean.name());
} catch (JMException e) {
throw new KafkaException("Error registering mbean " + mbean.name(), e);
}
}
private String[] split(String name) {
int attributeStart = name.lastIndexOf('.');
if (attributeStart < 0)
throw new IllegalArgumentException("No MBean name in metric name: " + name);
String attributeName = name.substring(attributeStart + 1, name.length());
String remainder = name.substring(0, attributeStart);
int beanStart = remainder.lastIndexOf('.');
if (beanStart < 0)
return new String[] { "", remainder, attributeName };
String packageName = remainder.substring(0, beanStart);
String beanName = remainder.substring(beanStart + 1, remainder.length());
return new String[] { packageName, beanName, attributeName };
}
private static class KafkaMbean implements DynamicMBean {
private final String beanName;
private final ObjectName objectName;
private final Map<String, KafkaMetric> metrics;
public KafkaMbean(String packageName, String beanName) throws MalformedObjectNameException {
this.beanName = beanName;
this.metrics = new HashMap<String, KafkaMetric>();
this.objectName = new ObjectName(packageName + ":type=" + beanName);
}
public ObjectName name() {
return objectName;
}
public void setAttribute(String name, KafkaMetric metric) {
this.metrics.put(name, metric);
}
@Override
public Object getAttribute(String name) throws AttributeNotFoundException, MBeanException, ReflectionException {
if (this.metrics.containsKey(name))
return this.metrics.get(name).value();
else
throw new AttributeNotFoundException("Could not find attribute " + name);
}
@Override
public AttributeList getAttributes(String[] names) {
try {
AttributeList list = new AttributeList();
for (String name : names)
list.add(new Attribute(name, getAttribute(name)));
return list;
} catch (Exception e) {
e.printStackTrace();
return new AttributeList();
}
}
@Override
public MBeanInfo getMBeanInfo() {
MBeanAttributeInfo[] attrs = new MBeanAttributeInfo[metrics.size()];
int i = 0;
for (Map.Entry<String, KafkaMetric> entry : this.metrics.entrySet()) {
String attribute = entry.getKey();
KafkaMetric metric = entry.getValue();
attrs[i] = new MBeanAttributeInfo(attribute, double.class.getName(), metric.description(), true, false, false);
i += 1;
}
return new MBeanInfo(beanName, "", attrs, null, null, null);
}
@Override
public Object invoke(String name, Object[] params, String[] sig) throws MBeanException, ReflectionException {
throw new UnsupportedOperationException("Set not allowed.");
}
@Override
public void setAttribute(Attribute attribute) throws AttributeNotFoundException,
InvalidAttributeValueException,
MBeanException,
ReflectionException {
throw new UnsupportedOperationException("Set not allowed.");
}
@Override
public AttributeList setAttributes(AttributeList list) {
throw new UnsupportedOperationException("Set not allowed.");
}
}
}
diff --git a/clients/src/main/java/kafka/common/metrics/KafkaMetric.java b/clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java
similarity index 90%
rename from clients/src/main/java/kafka/common/metrics/KafkaMetric.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java
index 33212b01..260f23fb 100644
--- a/clients/src/main/java/kafka/common/metrics/KafkaMetric.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/KafkaMetric.java
@@ -1,55 +1,55 @@
-package kafka.common.metrics;
+package org.apache.kafka.common.metrics;
-import kafka.common.Metric;
-import kafka.common.utils.Time;
+import org.apache.kafka.common.Metric;
+import org.apache.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/org/apache/kafka/common/metrics/Measurable.java
similarity index 90%
rename from clients/src/main/java/kafka/common/metrics/Measurable.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/Measurable.java
index f5511ea0..90864050 100644
--- a/clients/src/main/java/kafka/common/metrics/Measurable.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/Measurable.java
@@ -1,16 +1,16 @@
-package kafka.common.metrics;
+package org.apache.kafka.common.metrics;
/**
* A measurable quantity that can be registered as a metric
*/
public interface Measurable {
/**
* Measure this quantity and return the result as a double
* @param config The configuration for this metric
* @param now The time the measurement is being taken
* @return The measured value
*/
public double measure(MetricConfig config, long now);
}
diff --git a/clients/src/main/java/kafka/common/metrics/MeasurableStat.java b/clients/src/main/java/org/apache/kafka/common/metrics/MeasurableStat.java
similarity index 54%
rename from clients/src/main/java/kafka/common/metrics/MeasurableStat.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/MeasurableStat.java
index 74d3bb4e..314f7264 100644
--- a/clients/src/main/java/kafka/common/metrics/MeasurableStat.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/MeasurableStat.java
@@ -1,10 +1,10 @@
-package kafka.common.metrics;
+package org.apache.kafka.common.metrics;
/**
* A MeasurableStat is a {@link Stat} that is also {@link Measurable} (i.e. can produce a single floating point value).
- * This is the interface used for most of the simple statistics such as {@link kafka.common.metrics.stats.Avg},
- * {@link kafka.common.metrics.stats.Max}, {@link kafka.common.metrics.stats.Count}, etc.
+ * This is the interface used for most of the simple statistics such as {@link org.apache.kafka.common.metrics.stats.Avg},
+ * {@link org.apache.kafka.common.metrics.stats.Max}, {@link org.apache.kafka.common.metrics.stats.Count}, etc.
*/
public interface MeasurableStat extends Stat, Measurable {
}
diff --git a/clients/src/main/java/kafka/common/metrics/MetricConfig.java b/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java
similarity index 97%
rename from clients/src/main/java/kafka/common/metrics/MetricConfig.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java
index 92f67f06..f4585845 100644
--- a/clients/src/main/java/kafka/common/metrics/MetricConfig.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/MetricConfig.java
@@ -1,71 +1,71 @@
-package kafka.common.metrics;
+package org.apache.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/org/apache/kafka/common/metrics/Metrics.java
similarity index 97%
rename from clients/src/main/java/kafka/common/metrics/Metrics.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java
index f2cb7828..c002ec18 100644
--- a/clients/src/main/java/kafka/common/metrics/Metrics.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/Metrics.java
@@ -1,190 +1,191 @@
-package kafka.common.metrics;
+package org.apache.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;
+import org.apache.kafka.common.utils.SystemTime;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+
/**
* A registry of sensors and metrics.
* <p>
* A metric is a named, numerical measurement. A sensor is a handle to record numerical measurements as they occur. Each
* Sensor has zero or more associated metrics. For example a Sensor might represent message sizes and we might associate
* with this sensor a metric for the average, maximum, or other statistics computed off the sequence of message sizes
* that are recorded by the sensor.
* <p>
* Usage looks something like this:
*
* <pre>
* // set up metrics:
* Metrics metrics = new Metrics(); // this is the global repository of metrics and sensors
* Sensor sensor = metrics.sensor(&quot;message-sizes&quot;);
* sensor.add(&quot;kafka.producer.message-sizes.avg&quot;, new Avg());
* sensor.add(&quot;kafka.producer.message-sizes.max&quot;, new Max());
*
* // as messages are sent we record the sizes
* sensor.record(messageSize);
* </pre>
*/
public class Metrics {
private final MetricConfig config;
private final ConcurrentMap<String, KafkaMetric> metrics;
private final ConcurrentMap<String, Sensor> sensors;
private final List<MetricsReporter> reporters;
private final Time time;
/**
* Create a metrics repository with no metric reporters and default configuration.
*/
public Metrics() {
this(new MetricConfig());
}
/**
* Create a metrics repository with no metric reporters and default configuration.
*/
public Metrics(Time time) {
this(new MetricConfig(), new ArrayList<MetricsReporter>(), time);
}
/**
* Create a metrics repository with no reporters and the given default config. This config will be used for any
* metric that doesn't override its own config.
* @param defaultConfig The default config to use for all metrics that don't override their config
*/
public Metrics(MetricConfig defaultConfig) {
this(defaultConfig, new ArrayList<MetricsReporter>(0), new SystemTime());
}
/**
* Create a metrics repository with a default config and the given metric reporters
* @param defaultConfig The default config
* @param reporters The metrics reporters
* @param time The time instance to use with the metrics
*/
public Metrics(MetricConfig defaultConfig, List<MetricsReporter> reporters, Time time) {
this.config = defaultConfig;
this.sensors = new ConcurrentHashMap<String, Sensor>();
this.metrics = new ConcurrentHashMap<String, KafkaMetric>();
this.reporters = Utils.notNull(reporters);
this.time = time;
for (MetricsReporter reporter : reporters)
reporter.init(new ArrayList<KafkaMetric>());
}
/**
* Create a sensor with the given unique name and zero or more parent sensors. All parent sensors will receive every
* value recorded with this sensor.
* @param name The name of the sensor
* @param parents The parent sensors
* @return The sensor that is created
*/
public Sensor sensor(String name, Sensor... parents) {
return sensor(name, null, parents);
}
/**
* Create a sensor with the given unique name and zero or more parent sensors. All parent sensors will receive every
* value recorded with this sensor.
* @param name The name of the sensor
* @param config A default configuration to use for this sensor for metrics that don't have their own config
* @param parents The parent sensors
* @return The sensor that is created
*/
public synchronized Sensor sensor(String name, MetricConfig config, Sensor... parents) {
Sensor s = this.sensors.get(Utils.notNull(name));
if (s == null) {
s = new Sensor(this, name, parents, config == null ? this.config : config, time);
this.sensors.put(name, s);
}
return s;
}
/**
* Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor.
* This is a way to expose existing values as metrics.
* @param name The name of the metric
* @param measurable The measurable that will be measured by this metric
*/
public void addMetric(String name, Measurable measurable) {
addMetric(name, "", measurable);
}
/**
* Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor.
* This is a way to expose existing values as metrics.
* @param name The name of the metric
* @param description A human-readable description to include in the metric
* @param measurable The measurable that will be measured by this metric
*/
public void addMetric(String name, String description, Measurable measurable) {
addMetric(name, description, null, measurable);
}
/**
* Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor.
* This is a way to expose existing values as metrics.
* @param name The name of the metric
* @param config The configuration to use when measuring this measurable
* @param measurable The measurable that will be measured by this metric
*/
public void addMetric(String name, MetricConfig config, Measurable measurable) {
addMetric(name, "", config, measurable);
}
/**
* Add a metric to monitor an object that implements measurable. This metric won't be associated with any sensor.
* This is a way to expose existing values as metrics.
* @param name The name of the metric
* @param description A human-readable description to include in the metric
* @param config The configuration to use when measuring this measurable
* @param measurable The measurable that will be measured by this metric
*/
public synchronized void addMetric(String name, String description, MetricConfig config, Measurable measurable) {
KafkaMetric m = new KafkaMetric(new Object(),
Utils.notNull(name),
Utils.notNull(description),
Utils.notNull(measurable),
config == null ? this.config : config,
time);
registerMetric(m);
}
/**
* Add a MetricReporter
*/
public synchronized void addReporter(MetricsReporter reporter) {
Utils.notNull(reporter).init(new ArrayList<KafkaMetric>(metrics.values()));
this.reporters.add(reporter);
}
synchronized void registerMetric(KafkaMetric metric) {
if (this.metrics.containsKey(metric.name()))
throw new IllegalArgumentException("A metric named '" + metric.name() + "' already exists, can't register another one.");
this.metrics.put(metric.name(), metric);
for (MetricsReporter reporter : reporters)
reporter.metricChange(metric);
}
/**
* Get all the metrics currently maintained indexed by metric name
*/
public Map<String, KafkaMetric> metrics() {
return this.metrics;
}
/**
* Close this metrics repository.
*/
public void close() {
for (MetricsReporter reporter : this.reporters)
reporter.close();
}
}
diff --git a/clients/src/main/java/kafka/common/metrics/MetricsReporter.java b/clients/src/main/java/org/apache/kafka/common/metrics/MetricsReporter.java
similarity index 93%
rename from clients/src/main/java/kafka/common/metrics/MetricsReporter.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/MetricsReporter.java
index bf0b39e6..a9690ea3 100644
--- a/clients/src/main/java/kafka/common/metrics/MetricsReporter.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/MetricsReporter.java
@@ -1,27 +1,27 @@
-package kafka.common.metrics;
+package org.apache.kafka.common.metrics;
import java.util.List;
/**
* A plugin interface to allow things to listen as new metrics are created so they can be reported
*/
public interface MetricsReporter {
/**
* This is called when the reporter is first registered to initially register all existing metrics
* @param metrics All currently existing metrics
*/
public void init(List<KafkaMetric> metrics);
/**
* This is called whenever a metric is updated or added
* @param metric
*/
public void metricChange(KafkaMetric metric);
/**
* Called when the metrics repository is closed.
*/
public void close();
}
diff --git a/clients/src/main/java/kafka/common/metrics/Quota.java b/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java
similarity index 94%
rename from clients/src/main/java/kafka/common/metrics/Quota.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/Quota.java
index 62782469..458528ea 100644
--- a/clients/src/main/java/kafka/common/metrics/Quota.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/Quota.java
@@ -1,36 +1,36 @@
-package kafka.common.metrics;
+package org.apache.kafka.common.metrics;
/**
* An upper or lower bound for metrics
*/
public final class Quota {
private final boolean upper;
private final double bound;
public Quota(double bound, boolean upper) {
this.bound = bound;
this.upper = upper;
}
public static Quota lessThan(double upperBound) {
return new Quota(upperBound, true);
}
public static Quota moreThan(double lowerBound) {
return new Quota(lowerBound, false);
}
public boolean isUpperBound() {
return this.upper;
}
public double bound() {
return this.bound;
}
public boolean acceptable(double value) {
return (upper && value <= bound) || (!upper && value >= bound);
}
}
diff --git a/clients/src/main/java/kafka/common/metrics/QuotaViolationException.java b/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java
similarity index 77%
rename from clients/src/main/java/kafka/common/metrics/QuotaViolationException.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java
index b9005cd5..a64586df 100644
--- a/clients/src/main/java/kafka/common/metrics/QuotaViolationException.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/QuotaViolationException.java
@@ -1,16 +1,16 @@
-package kafka.common.metrics;
+package org.apache.kafka.common.metrics;
-import kafka.common.KafkaException;
+import org.apache.kafka.common.KafkaException;
/**
* Thrown when a sensor records a value that causes a metric to go outside the bounds configured as its quota
*/
public class QuotaViolationException extends KafkaException {
private static final long serialVersionUID = 1L;
public QuotaViolationException(String m) {
super(m);
}
}
diff --git a/clients/src/main/java/kafka/common/metrics/Sensor.java b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java
similarity index 96%
rename from clients/src/main/java/kafka/common/metrics/Sensor.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java
index 9c118350..58b4252f 100644
--- a/clients/src/main/java/kafka/common/metrics/Sensor.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/Sensor.java
@@ -1,171 +1,172 @@
-package kafka.common.metrics;
+package org.apache.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;
+import org.apache.kafka.common.metrics.CompoundStat.NamedMeasurable;
+import org.apache.kafka.common.utils.Time;
+import org.apache.kafka.common.utils.Utils;
+
/**
* A sensor applies a continuous sequence of numerical values to a set of associated metrics. For example a sensor on
* message size would record a sequence of message sizes using the {@link #record(double)} api and would maintain a set
* of metrics about request sizes such as the average or max.
*/
public final class Sensor {
private final Metrics registry;
private final String name;
private final Sensor[] parents;
private final List<Stat> stats;
private final List<KafkaMetric> metrics;
private final MetricConfig config;
private final Time time;
Sensor(Metrics registry, String name, Sensor[] parents, MetricConfig config, Time time) {
super();
this.registry = registry;
this.name = Utils.notNull(name);
this.parents = parents;
this.metrics = new ArrayList<KafkaMetric>();
this.stats = new ArrayList<Stat>();
this.config = config;
this.time = time;
checkForest(new HashSet<Sensor>());
}
/* Validate that this sensor doesn't end up referencing itself */
private void checkForest(Set<Sensor> sensors) {
if (!sensors.add(this))
throw new IllegalArgumentException("Circular dependency in sensors: " + name() + " is its own parent.");
for (int i = 0; i < parents.length; i++)
parents[i].checkForest(sensors);
}
/**
* The name this sensor is registered with. This name will be unique among all registered sensors.
*/
public String name() {
return this.name;
}
/**
* Record an occurrence, this is just short-hand for {@link #record(double) record(1.0)}
*/
public void record() {
record(1.0);
}
/**
* Record a value with this sensor
* @param value The value to record
* @throws QuotaViolationException if recording this value moves a metric beyond its configured maximum or minimum
* bound
*/
public void record(double value) {
record(value, time.nanoseconds());
}
private void record(double value, long time) {
synchronized (this) {
// increment all the stats
for (int i = 0; i < this.stats.size(); i++)
this.stats.get(i).record(config, value, time);
checkQuotas(time);
}
for (int i = 0; i < parents.length; i++)
parents[i].record(value, time);
}
private void checkQuotas(long time) {
for (int i = 0; i < this.metrics.size(); i++) {
KafkaMetric metric = this.metrics.get(i);
MetricConfig config = metric.config();
if (config != null) {
Quota quota = config.quota();
if (quota != null)
if (!quota.acceptable(metric.value(time)))
throw new QuotaViolationException("Metric " + metric.name() + " is in violation of its quota of " + quota.bound());
}
}
}
/**
* Register a compound statistic with this sensor with no config override
*/
public void add(CompoundStat stat) {
add(stat, null);
}
/**
* Register a compound statistic with this sensor which yields multiple measurable quantities (like a histogram)
* @param stat The stat to register
* @param config The configuration for this stat. If null then the stat will use the default configuration for this
* sensor.
*/
public synchronized void add(CompoundStat stat, MetricConfig config) {
this.stats.add(Utils.notNull(stat));
for (NamedMeasurable m : stat.stats()) {
KafkaMetric metric = new KafkaMetric(this, m.name(), m.description(), m.stat(), config == null ? this.config : config, time);
this.registry.registerMetric(metric);
this.metrics.add(metric);
}
}
/**
* Add a metric with default configuration and no description. Equivalent to
* {@link Sensor#add(String, String, MeasurableStat, MetricConfig) add(name, "", stat, null)}
*
*/
public void add(String name, MeasurableStat stat) {
add(name, stat, null);
}
/**
* Add a metric with default configuration. Equivalent to
* {@link Sensor#add(String, String, MeasurableStat, MetricConfig) add(name, description, stat, null)}
*
*/
public void add(String name, String description, MeasurableStat stat) {
add(name, description, stat, null);
}
/**
* Add a metric to this sensor with no description. Equivalent to
* {@link Sensor#add(String, String, MeasurableStat, MetricConfig) add(name, "", stat, config)}
* @param name
* @param stat
* @param config
*/
public void add(String name, MeasurableStat stat, MetricConfig config) {
add(name, "", stat, config);
}
/**
* Register a metric with this sensor
* @param name The name of the metric
* @param description A description used when reporting the value
* @param stat The statistic to keep
* @param config A special configuration for this metric. If null use the sensor default configuration.
*/
public synchronized void add(String name, String description, MeasurableStat stat, MetricConfig config) {
KafkaMetric metric = new KafkaMetric(this,
Utils.notNull(name),
Utils.notNull(description),
Utils.notNull(stat),
config == null ? this.config : config,
time);
this.registry.registerMetric(metric);
this.metrics.add(metric);
this.stats.add(stat);
}
synchronized List<KafkaMetric> metrics() {
return Collections.unmodifiableList(this.metrics);
}
}
diff --git a/clients/src/main/java/kafka/common/metrics/Stat.java b/clients/src/main/java/org/apache/kafka/common/metrics/Stat.java
similarity index 90%
rename from clients/src/main/java/kafka/common/metrics/Stat.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/Stat.java
index 8844545a..11731383 100644
--- a/clients/src/main/java/kafka/common/metrics/Stat.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/Stat.java
@@ -1,16 +1,16 @@
-package kafka.common.metrics;
+package org.apache.kafka.common.metrics;
/**
* A Stat is a quanity such as average, max, etc that is computed off the stream of updates to a sensor
*/
public interface Stat {
/**
* Record the given value
* @param config The configuration to use for this metric
* @param value The value to record
* @param time The time this value occurred
*/
public void record(MetricConfig config, double value, long time);
}
diff --git a/clients/src/main/java/kafka/common/metrics/stats/Avg.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Avg.java
similarity index 87%
rename from clients/src/main/java/kafka/common/metrics/stats/Avg.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/stats/Avg.java
index b9d3d5dd..a2b12fa3 100644
--- a/clients/src/main/java/kafka/common/metrics/stats/Avg.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Avg.java
@@ -1,33 +1,34 @@
-package kafka.common.metrics.stats;
+package org.apache.kafka.common.metrics.stats;
import java.util.List;
-import kafka.common.metrics.MetricConfig;
+import org.apache.kafka.common.metrics.MetricConfig;
+
/**
* A {@link SampledStat} that maintains a simple average over its samples.
*/
public class Avg extends SampledStat {
public Avg() {
super(0.0);
}
@Override
protected void update(Sample sample, MetricConfig config, double value, long now) {
sample.value += value;
}
@Override
public double combine(List<Sample> samples, MetricConfig config, long now) {
double total = 0.0;
long count = 0;
for (int i = 0; i < samples.size(); i++) {
Sample s = samples.get(i);
total += s.value;
count += s.eventCount;
}
return total / count;
}
}
diff --git a/clients/src/main/java/kafka/common/metrics/stats/Count.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Count.java
similarity index 85%
rename from clients/src/main/java/kafka/common/metrics/stats/Count.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/stats/Count.java
index 3712e78a..6b962922 100644
--- a/clients/src/main/java/kafka/common/metrics/stats/Count.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Count.java
@@ -1,29 +1,30 @@
-package kafka.common.metrics.stats;
+package org.apache.kafka.common.metrics.stats;
import java.util.List;
-import kafka.common.metrics.MetricConfig;
+import org.apache.kafka.common.metrics.MetricConfig;
+
/**
* A {@link SampledStat} that maintains a simple count of what it has seen.
*/
public class Count extends SampledStat {
public Count() {
super(0);
}
@Override
protected void update(Sample sample, MetricConfig config, double value, long now) {
sample.value += 1.0;
}
@Override
public double combine(List<Sample> samples, MetricConfig config, long now) {
double total = 0.0;
for (int i = 0; i < samples.size(); i++)
total += samples.get(i).value;
return total;
}
}
diff --git a/clients/src/main/java/kafka/common/metrics/stats/Histogram.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Histogram.java
similarity index 98%
rename from clients/src/main/java/kafka/common/metrics/stats/Histogram.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/stats/Histogram.java
index 99225719..60e2669b 100644
--- a/clients/src/main/java/kafka/common/metrics/stats/Histogram.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Histogram.java
@@ -1,141 +1,141 @@
-package kafka.common.metrics.stats;
+package org.apache.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 == 0.0d)
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 float[] counts() {
return this.hist;
}
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/org/apache/kafka/common/metrics/stats/Max.java
similarity index 86%
rename from clients/src/main/java/kafka/common/metrics/stats/Max.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/stats/Max.java
index e7bd1d29..dc0e4db1 100644
--- a/clients/src/main/java/kafka/common/metrics/stats/Max.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Max.java
@@ -1,29 +1,30 @@
-package kafka.common.metrics.stats;
+package org.apache.kafka.common.metrics.stats;
import java.util.List;
-import kafka.common.metrics.MetricConfig;
+import org.apache.kafka.common.metrics.MetricConfig;
+
/**
* A {@link SampledStat} that gives the max over its samples.
*/
public final class Max extends SampledStat {
public Max() {
super(Double.NEGATIVE_INFINITY);
}
@Override
protected void update(Sample sample, MetricConfig config, double value, long now) {
sample.value = Math.max(sample.value, value);
}
@Override
public double combine(List<Sample> samples, MetricConfig config, long now) {
double max = Double.NEGATIVE_INFINITY;
for (int i = 0; i < samples.size(); i++)
max = Math.max(max, samples.get(i).value);
return max;
}
}
diff --git a/clients/src/main/java/kafka/common/metrics/stats/Min.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java
similarity index 86%
rename from clients/src/main/java/kafka/common/metrics/stats/Min.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java
index db0ab929..2883d985 100644
--- a/clients/src/main/java/kafka/common/metrics/stats/Min.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java
@@ -1,29 +1,30 @@
-package kafka.common.metrics.stats;
+package org.apache.kafka.common.metrics.stats;
import java.util.List;
-import kafka.common.metrics.MetricConfig;
+import org.apache.kafka.common.metrics.MetricConfig;
+
/**
* A {@link SampledStat} that gives the min over its samples.
*/
public class Min extends SampledStat {
public Min() {
super(Double.MIN_VALUE);
}
@Override
protected void update(Sample sample, MetricConfig config, double value, long now) {
sample.value = Math.min(sample.value, value);
}
@Override
public double combine(List<Sample> samples, MetricConfig config, long now) {
double max = Double.MAX_VALUE;
for (int i = 0; i < samples.size(); i++)
max = Math.min(max, samples.get(i).value);
return max;
}
}
diff --git a/clients/src/main/java/kafka/common/metrics/stats/Percentile.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentile.java
similarity index 93%
rename from clients/src/main/java/kafka/common/metrics/stats/Percentile.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentile.java
index 84320bb2..87768376 100644
--- a/clients/src/main/java/kafka/common/metrics/stats/Percentile.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentile.java
@@ -1,32 +1,32 @@
-package kafka.common.metrics.stats;
+package org.apache.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/org/apache/kafka/common/metrics/stats/Percentiles.java
similarity index 88%
rename from clients/src/main/java/kafka/common/metrics/stats/Percentiles.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java
index c3f8942b..02f50eb1 100644
--- a/clients/src/main/java/kafka/common/metrics/stats/Percentiles.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Percentiles.java
@@ -1,104 +1,105 @@
-package kafka.common.metrics.stats;
+package org.apache.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;
+import org.apache.kafka.common.metrics.CompoundStat;
+import org.apache.kafka.common.metrics.Measurable;
+import org.apache.kafka.common.metrics.MetricConfig;
+import org.apache.kafka.common.metrics.stats.Histogram.BinScheme;
+import org.apache.kafka.common.metrics.stats.Histogram.ConstantBinScheme;
+import org.apache.kafka.common.metrics.stats.Histogram.LinearBinScheme;
+
/**
* A compound stat that reports one or more percentiles
*/
public class Percentiles extends SampledStat implements CompoundStat {
public static enum BucketSizing {
CONSTANT, LINEAR
}
private final int buckets;
private final Percentile[] percentiles;
private final BinScheme binScheme;
public Percentiles(int sizeInBytes, double max, BucketSizing bucketing, Percentile... percentiles) {
this(sizeInBytes, 0.0, max, bucketing, percentiles);
}
public Percentiles(int sizeInBytes, double min, double max, BucketSizing bucketing, Percentile... percentiles) {
super(0.0);
this.percentiles = percentiles;
this.buckets = sizeInBytes / 4;
if (bucketing == BucketSizing.CONSTANT) {
this.binScheme = new ConstantBinScheme(buckets, min, max);
} else if (bucketing == BucketSizing.LINEAR) {
if (min != 0.0d)
throw new IllegalArgumentException("Linear bucket sizing requires min to be 0.0.");
this.binScheme = new LinearBinScheme(buckets, max);
} else {
throw new IllegalArgumentException("Unknown bucket type: " + bucketing);
}
}
@Override
public List<NamedMeasurable> stats() {
List<NamedMeasurable> ms = new ArrayList<NamedMeasurable>(this.percentiles.length);
for (Percentile percentile : this.percentiles) {
final double pct = percentile.percentile();
ms.add(new NamedMeasurable(percentile.name(), percentile.description(), new Measurable() {
public double measure(MetricConfig config, long now) {
return value(config, now, pct / 100.0);
}
}));
}
return ms;
}
public double value(MetricConfig config, long now, double quantile) {
timeoutObsoleteSamples(config, now);
float count = 0.0f;
for (Sample sample : this.samples)
count += sample.eventCount;
if (count == 0.0f)
return Double.NaN;
float sum = 0.0f;
float quant = (float) quantile;
for (int b = 0; b < buckets; b++) {
for (int s = 0; s < this.samples.size(); s++) {
HistogramSample sample = (HistogramSample) this.samples.get(s);
float[] hist = sample.histogram.counts();
sum += hist[b];
if (sum / count > quant)
return binScheme.fromBin(b);
}
}
return Double.POSITIVE_INFINITY;
}
public double combine(List<Sample> samples, MetricConfig config, long now) {
return value(config, now, 0.5);
}
@Override
protected HistogramSample newSample(long now) {
return new HistogramSample(this.binScheme, now);
}
@Override
protected void update(Sample sample, MetricConfig config, double value, long now) {
HistogramSample hist = (HistogramSample) sample;
hist.histogram.record(value);
}
private static class HistogramSample extends SampledStat.Sample {
private final Histogram histogram;
private HistogramSample(BinScheme scheme, long now) {
super(0.0, now);
this.histogram = new Histogram(scheme);
}
}
}
diff --git a/clients/src/main/java/kafka/common/metrics/stats/Rate.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java
similarity index 92%
rename from clients/src/main/java/kafka/common/metrics/stats/Rate.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java
index 3f24a92d..84d84e3b 100644
--- a/clients/src/main/java/kafka/common/metrics/stats/Rate.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java
@@ -1,85 +1,90 @@
-package kafka.common.metrics.stats;
+package org.apache.kafka.common.metrics.stats;
import java.util.List;
import java.util.concurrent.TimeUnit;
-import kafka.common.metrics.MeasurableStat;
-import kafka.common.metrics.MetricConfig;
+import org.apache.kafka.common.metrics.MeasurableStat;
+import org.apache.kafka.common.metrics.MetricConfig;
+
/**
* The rate of the given quanitity. By default this is the total observed over a set of samples from a sampled statistic
* divided by the ellapsed time over the sample windows. Alternative {@link SampledStat} implementations can be
* provided, however, to record the rate of occurences (e.g. the count of values measured over the time interval) or
* other such values.
*/
public class Rate implements MeasurableStat {
private final TimeUnit unit;
private final SampledStat stat;
+ public Rate() {
+ this(TimeUnit.SECONDS);
+ }
+
public Rate(TimeUnit unit) {
this(unit, new SampledTotal());
}
public Rate(TimeUnit unit, SampledStat stat) {
this.stat = stat;
this.unit = unit;
}
public String unitName() {
return unit.name().substring(0, unit.name().length() - 2).toLowerCase();
}
@Override
public void record(MetricConfig config, double value, long time) {
this.stat.record(config, value, time);
}
@Override
public double measure(MetricConfig config, long now) {
double ellapsed = convert(now - stat.oldest().lastWindow);
return stat.measure(config, now) / ellapsed;
}
private double convert(long time) {
switch (unit) {
case NANOSECONDS:
return time;
case MICROSECONDS:
return time / 1000.0;
case MILLISECONDS:
return time / (1000.0 * 1000.0);
case SECONDS:
return time / (1000.0 * 1000.0 * 1000.0);
case MINUTES:
return time / (60.0 * 1000.0 * 1000.0 * 1000.0);
case HOURS:
return time / (60.0 * 60.0 * 1000.0 * 1000.0 * 1000.0);
case DAYS:
return time / (24.0 * 60.0 * 60.0 * 1000.0 * 1000.0 * 1000.0);
default:
throw new IllegalStateException("Unknown unit: " + unit);
}
}
public static class SampledTotal extends SampledStat {
public SampledTotal() {
super(0.0d);
}
@Override
protected void update(Sample sample, MetricConfig config, double value, long now) {
sample.value += value;
}
@Override
public double combine(List<Sample> samples, MetricConfig config, long now) {
double total = 0.0;
for (int i = 0; i < samples.size(); i++)
total += samples.get(i).value;
return total;
}
}
}
diff --git a/clients/src/main/java/kafka/common/metrics/stats/SampledStat.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java
similarity index 95%
rename from clients/src/main/java/kafka/common/metrics/stats/SampledStat.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java
index e696af58..fc4e5a45 100644
--- a/clients/src/main/java/kafka/common/metrics/stats/SampledStat.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/SampledStat.java
@@ -1,110 +1,111 @@
-package kafka.common.metrics.stats;
+package org.apache.kafka.common.metrics.stats;
import java.util.ArrayList;
import java.util.List;
-import kafka.common.metrics.MeasurableStat;
-import kafka.common.metrics.MetricConfig;
+import org.apache.kafka.common.metrics.MeasurableStat;
+import org.apache.kafka.common.metrics.MetricConfig;
+
/**
* A SampledStat records a single scalar value measured over one or more samples. Each sample is recorded over a
* configurable window. The window can be defined by number of events or ellapsed time (or both, if both are given the
* window is complete when <i>either</i> the event count or ellapsed time criterion is met).
* <p>
* All the samples are combined to produce the measurement. When a window is complete the oldest sample is cleared and
* recycled to begin recording the next sample.
*
* Subclasses of this class define different statistics measured using this basic pattern.
*/
public abstract class SampledStat implements MeasurableStat {
private double initialValue;
private int current = 0;
protected List<Sample> samples;
public SampledStat(double initialValue) {
this.initialValue = initialValue;
this.samples = new ArrayList<Sample>(2);
}
@Override
public void record(MetricConfig config, double value, long now) {
Sample sample = current(now);
if (sample.isComplete(now, config))
sample = advance(config, now);
update(sample, config, value, now);
sample.eventCount += 1;
}
private Sample advance(MetricConfig config, long now) {
this.current = (this.current + 1) % config.samples();
if (this.current >= samples.size()) {
Sample sample = newSample(now);
this.samples.add(sample);
return sample;
} else {
Sample sample = current(now);
sample.reset(now);
return sample;
}
}
protected Sample newSample(long now) {
return new Sample(this.initialValue, now);
}
@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(newSample(now));
return this.samples.get(this.current);
}
public Sample oldest() {
return this.samples.get((this.current + 1) % this.samples.size());
}
protected abstract void update(Sample sample, MetricConfig config, double value, long now);
public abstract double combine(List<Sample> samples, MetricConfig config, long now);
/* Timeout any windows that have expired in the absense of any events */
protected 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/org/apache/kafka/common/metrics/stats/Total.java
similarity index 76%
rename from clients/src/main/java/kafka/common/metrics/stats/Total.java
rename to clients/src/main/java/org/apache/kafka/common/metrics/stats/Total.java
index c87b1ba2..562e7a6b 100644
--- a/clients/src/main/java/kafka/common/metrics/stats/Total.java
+++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Total.java
@@ -1,31 +1,31 @@
-package kafka.common.metrics.stats;
+package org.apache.kafka.common.metrics.stats;
-import kafka.common.metrics.MeasurableStat;
-import kafka.common.metrics.MetricConfig;
+import org.apache.kafka.common.metrics.MeasurableStat;
+import org.apache.kafka.common.metrics.MetricConfig;
/**
* An un-windowed cumulative total maintained over all time.
*/
public class Total implements MeasurableStat {
private double total;
public Total() {
this.total = 0.0;
}
public Total(double value) {
this.total = value;
}
@Override
public void record(MetricConfig config, double value, long time) {
this.total += value;
}
@Override
public double measure(MetricConfig config, long now) {
return this.total;
}
}
diff --git a/clients/src/main/java/kafka/common/network/ByteBufferReceive.java b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferReceive.java
similarity index 95%
rename from clients/src/main/java/kafka/common/network/ByteBufferReceive.java
rename to clients/src/main/java/org/apache/kafka/common/network/ByteBufferReceive.java
index 65a7c64d..21eb8064 100644
--- a/clients/src/main/java/kafka/common/network/ByteBufferReceive.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferReceive.java
@@ -1,45 +1,45 @@
-package kafka.common.network;
+package org.apache.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 {
long read = channel.read(buffers);
remaining += read;
return read;
}
public ByteBuffer[] reify() {
return buffers;
}
}
diff --git a/clients/src/main/java/kafka/common/network/ByteBufferSend.java b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java
similarity index 96%
rename from clients/src/main/java/kafka/common/network/ByteBufferSend.java
rename to clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java
index 43bf9639..04ec5aa3 100644
--- a/clients/src/main/java/kafka/common/network/ByteBufferSend.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/ByteBufferSend.java
@@ -1,54 +1,54 @@
-package kafka.common.network;
+package org.apache.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/org/apache/kafka/common/network/NetworkReceive.java
similarity index 97%
rename from clients/src/main/java/kafka/common/network/NetworkReceive.java
rename to clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java
index 68ae48e4..7801bd70 100644
--- a/clients/src/main/java/kafka/common/network/NetworkReceive.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/NetworkReceive.java
@@ -1,74 +1,74 @@
-package kafka.common.network;
+package org.apache.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/org/apache/kafka/common/network/NetworkSend.java
similarity index 95%
rename from clients/src/main/java/kafka/common/network/NetworkSend.java
rename to clients/src/main/java/org/apache/kafka/common/network/NetworkSend.java
index 4e4ac98a..c5e03169 100644
--- a/clients/src/main/java/kafka/common/network/NetworkSend.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/NetworkSend.java
@@ -1,26 +1,26 @@
-package kafka.common.network;
+package org.apache.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/org/apache/kafka/common/network/Receive.java
similarity index 95%
rename from clients/src/main/java/kafka/common/network/Receive.java
rename to clients/src/main/java/org/apache/kafka/common/network/Receive.java
index 40ee942a..981a7df2 100644
--- a/clients/src/main/java/kafka/common/network/Receive.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/Receive.java
@@ -1,35 +1,35 @@
-package kafka.common.network;
+package org.apache.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/org/apache/kafka/common/network/Selectable.java
similarity index 98%
rename from clients/src/main/java/kafka/common/network/Selectable.java
rename to clients/src/main/java/org/apache/kafka/common/network/Selectable.java
index 794fc608..85a606bd 100644
--- a/clients/src/main/java/kafka/common/network/Selectable.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/Selectable.java
@@ -1,68 +1,68 @@
-package kafka.common.network;
+package org.apache.kafka.common.network;
import java.io.IOException;
import java.net.InetSocketAddress;
import java.util.List;
/**
* An interface for asynchronous, multi-channel network I/O
*/
public interface Selectable {
/**
* Begin establishing a socket connection to the given address identified by the given address
* @param id The id for this connection
* @param address The address to connect to
* @param sendBufferSize The send buffer for the socket
* @param receiveBufferSize The receive buffer for the socket
* @throws IOException If we cannot begin connecting
*/
public void connect(int id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException;
/**
* Begin disconnecting the connection identified by the given id
*/
public void disconnect(int id);
/**
* Wakeup this selector if it is blocked on I/O
*/
public void wakeup();
/**
* Close this selector
*/
public void close();
/**
* Initiate any sends provided, and make progress on any other I/O operations in-flight (connections,
* disconnections, existing sends, and receives)
* @param timeout The amount of time to block if there is nothing to do
* @param sends The new sends to initiate
* @throws IOException
*/
public void poll(long timeout, List<NetworkSend> sends) throws IOException;
/**
* The list of sends that completed on the last {@link #poll(long, List<NetworkSend>) poll()} call.
*/
public List<NetworkSend> completedSends();
/**
* The list of receives that completed on the last {@link #poll(long, List<NetworkSend>) poll()} call.
*/
public List<NetworkReceive> completedReceives();
/**
* The list of connections that finished disconnecting on the last {@link #poll(long, List<NetworkSend>) poll()}
* call.
*/
public List<Integer> disconnected();
/**
* The list of connections that completed their connection on the last {@link #poll(long, List<NetworkSend>) poll()}
* call.
*/
public List<Integer> connected();
}
\ No newline at end of file
diff --git a/clients/src/main/java/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java
similarity index 99%
rename from clients/src/main/java/kafka/common/network/Selector.java
rename to clients/src/main/java/org/apache/kafka/common/network/Selector.java
index f53060cb..218f53eb 100644
--- a/clients/src/main/java/kafka/common/network/Selector.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java
@@ -1,349 +1,350 @@
-package kafka.common.network;
+package org.apache.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;
+import org.apache.kafka.common.KafkaException;
+
/**
* A selector interface for doing non-blocking multi-connection network I/O.
* <p>
* This class works with {@link NetworkSend} and {@link NetworkReceive} to transmit size-delimited network requests and
* responses.
* <p>
* A connection can be added to the selector associated with an integer id by doing
*
* <pre>
* selector.connect(42, new InetSocketAddress(&quot;google.com&quot;, server.port), 64000, 64000);
* </pre>
*
* The connect call does not block on the creation of the TCP connection, so the connect method only begins initiating
* the connection. The successful invocation of this method does not mean a valid connection has been established.
*
* Sending requests, receiving responses, processing connection completions, and disconnections on the existing
* connections are all done using the <code>poll()</code> call.
*
* <pre>
* List&lt;NetworkRequest&gt; requestsToSend = Arrays.asList(new NetworkRequest(0, myBytes), new NetworkRequest(1, myOtherBytes));
* selector.poll(TIMEOUT_MS, requestsToSend);
* </pre>
*
* The selector maintains several lists that are reset by each call to <code>poll()</code> which are available via
* various getters. These are reset by each call to <code>poll()</code>.
*
* This class is not thread safe!
*/
public class Selector implements Selectable {
private final java.nio.channels.Selector selector;
private final Map<Integer, SelectionKey> keys;
private final List<NetworkSend> completedSends;
private final List<NetworkReceive> completedReceives;
private final List<Integer> disconnected;
private final List<Integer> connected;
/**
* Create a new selector
*/
public Selector() {
try {
this.selector = java.nio.channels.Selector.open();
} catch (IOException e) {
throw new KafkaException(e);
}
this.keys = new HashMap<Integer, SelectionKey>();
this.completedSends = new ArrayList<NetworkSend>();
this.completedReceives = new ArrayList<NetworkReceive>();
this.connected = new ArrayList<Integer>();
this.disconnected = new ArrayList<Integer>();
}
/**
* Begin connecting to the given address and add the connection to this selector associated with the given id
* number.
* <p>
* Note that this call only initiates the connection, which will be completed on a future {@link #poll(long, List)}
* call. Check {@link #connected()} to see which (if any) connections have completed after a given poll call.
* @param id The id for the new connection
* @param address The address to connect to
* @param sendBufferSize The send buffer for the new connection
* @param receiveBufferSize The receive buffer for the new connection
* @throws IllegalStateException if there is already a connection for that id
* @throws UnresolvedAddressException if DNS resolution fails on the hostname
*/
@Override
public void connect(int id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException {
SocketChannel channel = SocketChannel.open();
channel.configureBlocking(false);
Socket socket = channel.socket();
socket.setKeepAlive(true);
socket.setSendBufferSize(sendBufferSize);
socket.setReceiveBufferSize(receiveBufferSize);
socket.setTcpNoDelay(true);
try {
channel.connect(address);
} catch (UnresolvedAddressException e) {
channel.close();
throw e;
}
SelectionKey key = channel.register(this.selector, SelectionKey.OP_CONNECT);
key.attach(new Transmissions(id));
if (this.keys.containsKey(key))
throw new IllegalStateException("There is already a connection for id " + id);
this.keys.put(id, key);
}
/**
* Disconnect any connections for the given id (if there are any). The disconnection is asynchronous and will not be
* processed until the next {@link #poll(long, List) poll()} call.
*/
@Override
public void disconnect(int id) {
SelectionKey key = this.keys.get(id);
if (key != null)
key.cancel();
}
/**
* Interrupt the selector if it is blocked waiting to do I/O.
*/
@Override
public void wakeup() {
this.selector.wakeup();
}
/**
* Close this selector and all associated connections
*/
@Override
public void close() {
for (SelectionKey key : this.selector.keys()) {
try {
close(key);
} catch (IOException e) {
e.printStackTrace();
}
}
try {
this.selector.close();
} catch (IOException e) {
e.printStackTrace();
}
}
/**
* Do whatever I/O can be done on each connection without blocking. This includes completing connections, completing
* disconnections, initiating new sends, or making progress on in-progress sends or receives.
* <p>
* The provided network sends will be started.
*
* When this call is completed the user can check for completed sends, receives, connections or disconnects using
* {@link #completedSends()}, {@link #completedReceives()}, {@link #connected()}, {@link #disconnected()}. These
* lists will be cleared at the beginning of each {@link #poll(long, List)} call and repopulated by the call if any
* completed I/O.
*
* @param timeout The amount of time to wait, in milliseconds. If negative, wait indefinitely.
* @param sends The list of new sends to begin
*
* @throws IllegalStateException If a send is given for which we have no existing connection or for which there is
* already an in-progress send
*/
@Override
public void poll(long timeout, List<NetworkSend> sends) throws IOException {
clear();
/* register for write interest on any new sends */
for (NetworkSend send : sends) {
SelectionKey key = keyForId(send.destination());
Transmissions transmissions = transmissions(key);
if (transmissions.hasSend())
throw new IllegalStateException("Attempt to begin a send operation with prior send operation still in progress.");
transmissions.send = send;
try {
key.interestOps(key.interestOps() | SelectionKey.OP_WRITE);
} catch (CancelledKeyException e) {
close(key);
}
}
/* check ready keys */
int readyKeys = select(timeout);
if (readyKeys > 0) {
Set<SelectionKey> keys = this.selector.selectedKeys();
Iterator<SelectionKey> iter = keys.iterator();
while (iter.hasNext()) {
SelectionKey key = iter.next();
iter.remove();
Transmissions transmissions = transmissions(key);
SocketChannel channel = channel(key);
try {
/*
* complete any connections that have finished their handshake
*/
if (key.isConnectable()) {
channel.finishConnect();
key.interestOps(key.interestOps() & ~SelectionKey.OP_CONNECT | SelectionKey.OP_READ);
this.connected.add(transmissions.id);
}
/* read from any connections that have readable data */
if (key.isReadable()) {
if (!transmissions.hasReceive())
transmissions.receive = new NetworkReceive(transmissions.id);
transmissions.receive.readFrom(channel);
if (transmissions.receive.complete()) {
transmissions.receive.payload().rewind();
this.completedReceives.add(transmissions.receive);
transmissions.clearReceive();
}
}
/*
* write to any sockets that have space in their buffer and for which we have data
*/
if (key.isWritable()) {
transmissions.send.writeTo(channel);
if (transmissions.send.remaining() <= 0) {
this.completedSends.add(transmissions.send);
transmissions.clearSend();
key.interestOps(key.interestOps() & ~SelectionKey.OP_WRITE);
}
}
/* cancel any defunct sockets */
if (!key.isValid())
close(key);
} catch (IOException e) {
e.printStackTrace();
close(key);
}
}
}
}
@Override
public List<NetworkSend> completedSends() {
return this.completedSends;
}
@Override
public List<NetworkReceive> completedReceives() {
return this.completedReceives;
}
@Override
public List<Integer> disconnected() {
return this.disconnected;
}
@Override
public List<Integer> connected() {
return this.connected;
}
/**
* Clear the results from the prior poll
*/
private void clear() {
this.completedSends.clear();
this.completedReceives.clear();
this.connected.clear();
this.disconnected.clear();
}
/**
* Check for data, waiting up to the given timeout.
*
* @param ms Length of time to wait, in milliseconds. If negative, wait indefinitely.
* @return The number of keys ready
* @throws IOException
*/
private int select(long ms) throws IOException {
if (ms == 0L)
return this.selector.selectNow();
else if (ms < 0L)
return this.selector.select();
else
return this.selector.select(ms);
}
/**
* Begin closing this connection
*/
private void close(SelectionKey key) throws IOException {
SocketChannel channel = channel(key);
Transmissions trans = transmissions(key);
if (trans != null)
this.disconnected.add(trans.id);
key.attach(null);
key.cancel();
channel.socket().close();
channel.close();
}
/**
* Get the selection key associated with this numeric id
*/
private SelectionKey keyForId(int id) {
SelectionKey key = this.keys.get(id);
if (key == null)
throw new IllegalStateException("Attempt to write to socket for which there is no open connection.");
return key;
}
/**
* Get the transmissions for the given connection
*/
private Transmissions transmissions(SelectionKey key) {
return (Transmissions) key.attachment();
}
/**
* Get the socket channel associated with this selection key
*/
private SocketChannel channel(SelectionKey key) {
return (SocketChannel) key.channel();
}
/**
* The id and in-progress send and receive associated with a connection
*/
private static class Transmissions {
public int id;
public NetworkSend send;
public NetworkReceive receive;
public Transmissions(int id) {
this.id = id;
}
public boolean hasSend() {
return this.send != null;
}
public void clearSend() {
this.send = null;
}
public boolean hasReceive() {
return this.receive != null;
}
public void clearReceive() {
this.receive = null;
}
}
}
diff --git a/clients/src/main/java/kafka/common/network/Send.java b/clients/src/main/java/org/apache/kafka/common/network/Send.java
similarity index 96%
rename from clients/src/main/java/kafka/common/network/Send.java
rename to clients/src/main/java/org/apache/kafka/common/network/Send.java
index e7ef68ad..68185e11 100644
--- a/clients/src/main/java/kafka/common/network/Send.java
+++ b/clients/src/main/java/org/apache/kafka/common/network/Send.java
@@ -1,41 +1,41 @@
-package kafka.common.network;
+package org.apache.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/org/apache/kafka/common/protocol/ApiKeys.java
similarity index 95%
rename from clients/src/main/java/kafka/common/protocol/ApiKeys.java
rename to clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
index 1e2f8bba..e1c87200 100644
--- a/clients/src/main/java/kafka/common/protocol/ApiKeys.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
@@ -1,35 +1,35 @@
-package kafka.common.protocol;
+package org.apache.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/org/apache/kafka/common/protocol/Errors.java
similarity index 81%
rename from clients/src/main/java/kafka/common/protocol/Errors.java
rename to clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
index 402a6c0f..323d4261 100644
--- a/clients/src/main/java/kafka/common/protocol/Errors.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
@@ -1,97 +1,98 @@
-package kafka.common.protocol;
+package org.apache.kafka.common.protocol;
import java.util.HashMap;
import java.util.Map;
-import kafka.common.errors.ApiException;
-import kafka.common.errors.CorruptRecordException;
-import kafka.common.errors.LeaderNotAvailableException;
-import kafka.common.errors.RecordTooLargeException;
-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;
+import org.apache.kafka.common.errors.ApiException;
+import org.apache.kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.common.errors.LeaderNotAvailableException;
+import org.apache.kafka.common.errors.NetworkException;
+import org.apache.kafka.common.errors.NotLeaderForPartitionException;
+import org.apache.kafka.common.errors.OffsetMetadataTooLarge;
+import org.apache.kafka.common.errors.OffsetOutOfRangeException;
+import org.apache.kafka.common.errors.RecordTooLargeException;
+import org.apache.kafka.common.errors.TimeoutException;
+import org.apache.kafka.common.errors.UnknownServerException;
+import org.apache.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 CorruptRecordException("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 RecordTooLargeException("The request included a message larger than the max message size the server will accept.")),
OFFSET_METADATA_TOO_LARGE(12, new OffsetMetadataTooLarge("The metadata field of the offset request was too large.")),
NETWORK_EXCEPTION(13, new NetworkException("The server disconnected before a response was received."));
private static Map<Class<?>, Errors> classToError = new HashMap<Class<?>, Errors>();
private static Map<Short, Errors> codeToError = new HashMap<Short, Errors>();
static {
for (Errors error : Errors.values()) {
codeToError.put(error.code(), error);
if (error.exception != null)
classToError.put(error.exception.getClass(), error);
}
}
private final short code;
private final ApiException exception;
private Errors(int code, ApiException exception) {
this.code = (short) code;
this.exception = exception;
}
/**
* An instance of the exception
*/
public ApiException exception() {
return this.exception;
}
/**
* The error code for the exception
*/
public short code() {
return this.code;
}
/**
* Throw the exception corresponding to this error if there is one
*/
public void maybeThrow() {
if (exception != null)
throw this.exception;
}
/**
* Throw the exception if there is one
*/
public static Errors forCode(short code) {
Errors error = codeToError.get(code);
return error == null ? UNKNOWN : error;
}
/**
* Return the error instance associated with this exception (or UKNOWN if there is none)
*/
public static Errors forException(Throwable t) {
Errors error = classToError.get(t.getClass());
return error == null ? UNKNOWN : error;
}
}
diff --git a/clients/src/main/java/kafka/common/protocol/ProtoUtils.java b/clients/src/main/java/org/apache/kafka/common/protocol/ProtoUtils.java
similarity index 93%
rename from clients/src/main/java/kafka/common/protocol/ProtoUtils.java
rename to clients/src/main/java/org/apache/kafka/common/protocol/ProtoUtils.java
index 576c24d0..b34bf79a 100644
--- a/clients/src/main/java/kafka/common/protocol/ProtoUtils.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/ProtoUtils.java
@@ -1,97 +1,98 @@
-package kafka.common.protocol;
+package org.apache.kafka.common.protocol;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import kafka.common.Cluster;
-import kafka.common.Node;
-import kafka.common.PartitionInfo;
-import kafka.common.protocol.types.Schema;
-import kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.PartitionInfo;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.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) {
Map<Integer, Node> brokers = new HashMap<Integer, Node>();
Object[] brokerStructs = (Object[]) response.get("brokers");
for (int i = 0; i < brokerStructs.length; i++) {
Struct broker = (Struct) brokerStructs[i];
int nodeId = (Integer) broker.get("node_id");
String host = (String) broker.get("host");
int port = (Integer) broker.get("port");
brokers.put(nodeId, new Node(nodeId, host, port));
}
List<PartitionInfo> partitions = new ArrayList<PartitionInfo>();
Object[] topicInfos = (Object[]) response.get("topic_metadata");
for (int i = 0; i < topicInfos.length; i++) {
Struct topicInfo = (Struct) topicInfos[i];
short topicError = topicInfo.getShort("topic_error_code");
if (topicError == Errors.NONE.code()) {
String topic = topicInfo.getString("topic");
Object[] partitionInfos = (Object[]) topicInfo.get("partition_metadata");
for (int j = 0; j < partitionInfos.length; j++) {
Struct partitionInfo = (Struct) partitionInfos[j];
short partError = partitionInfo.getShort("partition_error_code");
if (partError == Errors.NONE.code()) {
int partition = partitionInfo.getInt("partition_id");
int leader = partitionInfo.getInt("leader");
Node leaderNode = leader == -1 ? null : brokers.get(leader);
Object[] replicas = (Object[]) partitionInfo.get("replicas");
Node[] replicaNodes = new Node[replicas.length];
for (int k = 0; k < replicas.length; k++)
replicaNodes[k] = brokers.get(replicas[k]);
Object[] isr = (Object[]) partitionInfo.get("isr");
Node[] isrNodes = new Node[isr.length];
for (int k = 0; k < isr.length; k++)
isrNodes[k] = brokers.get(isr[k]);
partitions.add(new PartitionInfo(topic, partition, leaderNode, replicaNodes, isrNodes));
}
}
}
}
return new Cluster(brokers.values(), partitions);
}
}
diff --git a/clients/src/main/java/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
similarity index 94%
rename from clients/src/main/java/kafka/common/protocol/Protocol.java
rename to clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
index 49b60aa5..35d7be75 100644
--- a/clients/src/main/java/kafka/common/protocol/Protocol.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
@@ -1,130 +1,131 @@
-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;
+package org.apache.kafka.common.protocol;
+
+import static org.apache.kafka.common.protocol.types.Type.BYTES;
+import static org.apache.kafka.common.protocol.types.Type.INT16;
+import static org.apache.kafka.common.protocol.types.Type.INT32;
+import static org.apache.kafka.common.protocol.types.Type.INT64;
+import static org.apache.kafka.common.protocol.types.Type.STRING;
+
+import org.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.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("record_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/org/apache/kafka/common/protocol/types/ArrayOf.java
similarity index 96%
rename from clients/src/main/java/kafka/common/protocol/types/ArrayOf.java
rename to clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java
index 5daf95ba..6dfaea38 100644
--- a/clients/src/main/java/kafka/common/protocol/types/ArrayOf.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java
@@ -1,63 +1,63 @@
-package kafka.common.protocol.types;
+package org.apache.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/org/apache/kafka/common/protocol/types/Field.java
similarity index 96%
rename from clients/src/main/java/kafka/common/protocol/types/Field.java
rename to clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java
index d018a12a..7e971ada 100644
--- a/clients/src/main/java/kafka/common/protocol/types/Field.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java
@@ -1,48 +1,48 @@
-package kafka.common.protocol.types;
+package org.apache.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/org/apache/kafka/common/protocol/types/Schema.java
similarity index 98%
rename from clients/src/main/java/kafka/common/protocol/types/Schema.java
rename to clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java
index b7b1c750..0c11a04e 100644
--- a/clients/src/main/java/kafka/common/protocol/types/Schema.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Schema.java
@@ -1,134 +1,134 @@
-package kafka.common.protocol.types;
+package org.apache.kafka.common.protocol.types;
import java.nio.ByteBuffer;
import java.util.HashMap;
import java.util.Map;
/**
* The schema for a compound record definition
*/
public class Schema extends Type {
private final Field[] fields;
private final Map<String, Field> fieldsByName;
public Schema(Field... fs) {
this.fields = new Field[fs.length];
this.fieldsByName = new HashMap<String, Field>();
for (int i = 0; i < this.fields.length; i++) {
Field field = fs[i];
if (fieldsByName.containsKey(field.name))
throw new SchemaException("Schema contains a duplicate field: " + field.name);
this.fields[i] = new Field(i, field.name, field.type, field.doc, field.defaultValue, this);
this.fieldsByName.put(fs[i].name, this.fields[i]);
}
}
/**
* Write a struct to the buffer
*/
public void write(ByteBuffer buffer, Object o) {
Struct r = (Struct) o;
for (int i = 0; i < fields.length; i++) {
Field f = fields[i];
try {
Object value = f.type().validate(r.get(f));
f.type.write(buffer, value);
} catch (Exception e) {
throw new SchemaException("Error writing field '" + f.name + "': " + e.getMessage() == null ? e.getMessage() : e.getClass()
.getName());
}
}
}
/**
* Read a struct from the buffer
*/
public Object read(ByteBuffer buffer) {
Object[] objects = new Object[fields.length];
for (int i = 0; i < fields.length; i++)
objects[i] = fields[i].type.read(buffer);
return new Struct(this, objects);
}
/**
* The size of the given record
*/
public int sizeOf(Object o) {
int size = 0;
Struct r = (Struct) o;
for (int i = 0; i < fields.length; i++)
size += fields[i].type.sizeOf(r.get(fields[i]));
return size;
}
/**
* The number of fields in this schema
*/
public int numFields() {
return this.fields.length;
}
/**
* Get a field by its slot in the record array
*
* @param slot The slot at which this field sits
* @return The field
*/
public Field get(int slot) {
return this.fields[slot];
}
/**
* Get a field by its name
*
* @param name The name of the field
* @return The field
*/
public Field get(String name) {
return this.fieldsByName.get(name);
}
/**
* Get all the fields in this schema
*/
public Field[] fields() {
return this.fields;
}
/**
* Display a string representation of the schema
*/
public String toString() {
StringBuilder b = new StringBuilder();
b.append('{');
for (int i = 0; i < this.fields.length; i++) {
b.append(this.fields[i].name);
b.append(':');
b.append(this.fields[i].type());
if (i < this.fields.length - 1)
b.append(',');
}
b.append("}");
return b.toString();
}
@Override
public Struct validate(Object item) {
try {
Struct struct = (Struct) item;
for (int i = 0; i < this.fields.length; i++) {
Field field = this.fields[i];
try {
field.type.validate(struct.get(field));
} catch (SchemaException e) {
throw new SchemaException("Invalid value for field '" + field.name + "': " + e.getMessage());
}
}
return struct;
} catch (ClassCastException e) {
throw new SchemaException("Not a Struct.");
}
}
}
\ No newline at end of file
diff --git a/clients/src/main/java/kafka/common/protocol/types/SchemaException.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/SchemaException.java
similarity index 66%
rename from clients/src/main/java/kafka/common/protocol/types/SchemaException.java
rename to clients/src/main/java/org/apache/kafka/common/protocol/types/SchemaException.java
index a2a2d50c..83b19e61 100644
--- a/clients/src/main/java/kafka/common/protocol/types/SchemaException.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/SchemaException.java
@@ -1,13 +1,13 @@
-package kafka.common.protocol.types;
+package org.apache.kafka.common.protocol.types;
-import kafka.common.KafkaException;
+import org.apache.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/org/apache/kafka/common/protocol/types/Struct.java
similarity index 99%
rename from clients/src/main/java/kafka/common/protocol/types/Struct.java
rename to clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
index c83aefae..ffcfa33c 100644
--- a/clients/src/main/java/kafka/common/protocol/types/Struct.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java
@@ -1,227 +1,227 @@
-package kafka.common.protocol.types;
+package org.apache.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/org/apache/kafka/common/protocol/types/Type.java
similarity index 98%
rename from clients/src/main/java/kafka/common/protocol/types/Type.java
rename to clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java
index f4c93e3e..24ac060a 100644
--- a/clients/src/main/java/kafka/common/protocol/types/Type.java
+++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java
@@ -1,216 +1,217 @@
-package kafka.common.protocol.types;
+package org.apache.kafka.common.protocol.types;
import java.nio.ByteBuffer;
-import kafka.common.utils.Utils;
+import org.apache.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/org/apache/kafka/common/record/CompressionType.java
similarity index 96%
rename from clients/src/main/java/kafka/common/record/CompressionType.java
rename to clients/src/main/java/org/apache/kafka/common/record/CompressionType.java
index f6d9026f..00155be3 100644
--- a/clients/src/main/java/kafka/common/record/CompressionType.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/CompressionType.java
@@ -1,40 +1,40 @@
-package kafka.common.record;
+package org.apache.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/org/apache/kafka/common/record/InvalidRecordException.java
similarity index 82%
rename from clients/src/main/java/kafka/common/record/InvalidRecordException.java
rename to clients/src/main/java/org/apache/kafka/common/record/InvalidRecordException.java
index 97fbe504..889c3a58 100644
--- a/clients/src/main/java/kafka/common/record/InvalidRecordException.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/InvalidRecordException.java
@@ -1,11 +1,11 @@
-package kafka.common.record;
+package org.apache.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/org/apache/kafka/common/record/LogEntry.java
similarity index 92%
rename from clients/src/main/java/kafka/common/record/LogEntry.java
rename to clients/src/main/java/org/apache/kafka/common/record/LogEntry.java
index f5e99c9d..7dac9315 100644
--- a/clients/src/main/java/kafka/common/record/LogEntry.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/LogEntry.java
@@ -1,28 +1,28 @@
-package kafka.common.record;
+package org.apache.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/org/apache/kafka/common/record/MemoryRecords.java
similarity index 96%
rename from clients/src/main/java/kafka/common/record/MemoryRecords.java
rename to clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
index d3f84263..b0533610 100644
--- a/clients/src/main/java/kafka/common/record/MemoryRecords.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java
@@ -1,102 +1,103 @@
-package kafka.common.record;
+package org.apache.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;
+import org.apache.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 records in this set to the given channel */
public int writeTo(GatheringByteChannel channel) throws IOException {
return channel.write(buffer);
}
/**
* The size of this record set
*/
public int sizeInBytes() {
return this.buffer.position();
}
/**
* Get the byte buffer that backs this records instance
*/
public ByteBuffer buffer() {
return buffer.duplicate();
}
@Override
public Iterator<LogEntry> iterator() {
return new RecordsIterator(this.buffer);
}
/* TODO: allow reuse of the buffer used for iteration */
public static class RecordsIterator extends AbstractIterator<LogEntry> {
private final ByteBuffer buffer;
public RecordsIterator(ByteBuffer buffer) {
ByteBuffer copy = buffer.duplicate();
copy.flip();
this.buffer = copy;
}
@Override
protected LogEntry makeNext() {
if (buffer.remaining() < Records.LOG_OVERHEAD)
return allDone();
long offset = buffer.getLong();
int size = buffer.getInt();
if (size < 0)
throw new IllegalStateException("Record 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/org/apache/kafka/common/record/Record.java
similarity index 99%
rename from clients/src/main/java/kafka/common/record/Record.java
rename to clients/src/main/java/org/apache/kafka/common/record/Record.java
index b89accf3..cd83c674 100644
--- a/clients/src/main/java/kafka/common/record/Record.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/Record.java
@@ -1,286 +1,287 @@
-package kafka.common.record;
+package org.apache.kafka.common.record;
import java.nio.ByteBuffer;
-import kafka.common.utils.Utils;
+import org.apache.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 InvalidRecordException 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("Record(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/org/apache/kafka/common/record/Records.java
similarity index 95%
rename from clients/src/main/java/kafka/common/record/Records.java
rename to clients/src/main/java/org/apache/kafka/common/record/Records.java
index 6531ca02..4181ecbb 100644
--- a/clients/src/main/java/kafka/common/record/Records.java
+++ b/clients/src/main/java/org/apache/kafka/common/record/Records.java
@@ -1,29 +1,29 @@
-package kafka.common.record;
+package org.apache.kafka.common.record;
import java.io.IOException;
import java.nio.channels.GatheringByteChannel;
/**
* A binary format which consists of a 4 byte size, an 8 byte offset, and the record bytes. See {@link MemoryRecords}
* for the in-memory representation.
*/
public interface Records extends Iterable<LogEntry> {
int SIZE_LENGTH = 4;
int OFFSET_LENGTH = 8;
int LOG_OVERHEAD = SIZE_LENGTH + OFFSET_LENGTH;
/**
* Write these records to the given channel
* @param channel The channel to write to
* @return The number of bytes written
* @throws IOException If the write fails.
*/
public int writeTo(GatheringByteChannel channel) throws IOException;
/**
* The size of these records in bytes
*/
public int sizeInBytes();
}
diff --git a/clients/src/main/java/kafka/common/requests/RequestHeader.java b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java
similarity index 84%
rename from clients/src/main/java/kafka/common/requests/RequestHeader.java
rename to clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java
index 4ce67f80..ce937050 100644
--- a/clients/src/main/java/kafka/common/requests/RequestHeader.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestHeader.java
@@ -1,68 +1,69 @@
-package kafka.common.requests;
+package org.apache.kafka.common.requests;
-import static kafka.common.protocol.Protocol.REQUEST_HEADER;
+import static org.apache.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;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.Protocol;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.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/org/apache/kafka/common/requests/RequestSend.java
similarity index 84%
rename from clients/src/main/java/kafka/common/requests/RequestSend.java
rename to clients/src/main/java/org/apache/kafka/common/requests/RequestSend.java
index f6a9a86a..6b47343f 100644
--- a/clients/src/main/java/kafka/common/requests/RequestSend.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/RequestSend.java
@@ -1,38 +1,39 @@
-package kafka.common.requests;
+package org.apache.kafka.common.requests;
import java.nio.ByteBuffer;
-import kafka.common.network.NetworkSend;
-import kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.network.NetworkSend;
+import org.apache.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/org/apache/kafka/common/requests/ResponseHeader.java
similarity index 76%
rename from clients/src/main/java/kafka/common/requests/ResponseHeader.java
rename to clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java
index 1ef8e155..ad99a171 100644
--- a/clients/src/main/java/kafka/common/requests/ResponseHeader.java
+++ b/clients/src/main/java/org/apache/kafka/common/requests/ResponseHeader.java
@@ -1,45 +1,46 @@
-package kafka.common.requests;
+package org.apache.kafka.common.requests;
-import static kafka.common.protocol.Protocol.RESPONSE_HEADER;
+import static org.apache.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;
+import org.apache.kafka.common.protocol.Protocol;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.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/org/apache/kafka/common/utils/AbstractIterator.java
similarity index 97%
rename from clients/src/main/java/kafka/common/utils/AbstractIterator.java
rename to clients/src/main/java/org/apache/kafka/common/utils/AbstractIterator.java
index f3190d7d..107c2b69 100644
--- a/clients/src/main/java/kafka/common/utils/AbstractIterator.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/AbstractIterator.java
@@ -1,72 +1,72 @@
-package kafka.common.utils;
+package org.apache.kafka.common.utils;
import java.util.Iterator;
import java.util.NoSuchElementException;
/**
* A base class that simplifies implementing an iterator
* @param <T> The type of thing we are iterating over
*/
public abstract class AbstractIterator<T> implements Iterator<T> {
private static enum State {
READY, NOT_READY, DONE, FAILED
};
private State state = State.NOT_READY;
private T next;
@Override
public boolean hasNext() {
switch (state) {
case FAILED:
throw new IllegalStateException("Iterator is in failed state");
case DONE:
return false;
case READY:
return true;
default:
return maybeComputeNext();
}
}
@Override
public T next() {
if (!hasNext())
throw new NoSuchElementException();
state = State.NOT_READY;
if (next == null)
throw new IllegalStateException("Expected item but none found.");
return next;
}
@Override
public void remove() {
throw new UnsupportedOperationException("Removal not supported");
}
public T peek() {
if (!hasNext())
throw new NoSuchElementException();
return next;
}
protected T allDone() {
state = State.DONE;
return null;
}
protected abstract T makeNext();
private Boolean maybeComputeNext() {
state = State.FAILED;
next = makeNext();
if (state == State.DONE) {
return false;
} else {
state = State.READY;
return true;
}
}
}
diff --git a/clients/src/main/java/kafka/common/utils/CopyOnWriteMap.java b/clients/src/main/java/org/apache/kafka/common/utils/CopyOnWriteMap.java
similarity index 98%
rename from clients/src/main/java/kafka/common/utils/CopyOnWriteMap.java
rename to clients/src/main/java/org/apache/kafka/common/utils/CopyOnWriteMap.java
index e45df98f..10dbc688 100644
--- a/clients/src/main/java/kafka/common/utils/CopyOnWriteMap.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/CopyOnWriteMap.java
@@ -1,130 +1,130 @@
-package kafka.common.utils;
+package org.apache.kafka.common.utils;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
import java.util.Set;
import java.util.concurrent.ConcurrentMap;
/**
* A simple read-optimized map implementation that synchronizes only writes and does a fully copy on each modification
*/
public class CopyOnWriteMap<K, V> implements ConcurrentMap<K, V> {
private volatile Map<K, V> map;
public CopyOnWriteMap() {
this.map = Collections.emptyMap();
}
public CopyOnWriteMap(Map<K, V> map) {
this.map = Collections.unmodifiableMap(map);
}
@Override
public boolean containsKey(Object k) {
return map.containsKey(k);
}
@Override
public boolean containsValue(Object v) {
return map.containsValue(v);
}
@Override
public Set<java.util.Map.Entry<K, V>> entrySet() {
return map.entrySet();
}
@Override
public V get(Object k) {
return map.get(k);
}
@Override
public boolean isEmpty() {
return map.isEmpty();
}
@Override
public Set<K> keySet() {
return map.keySet();
}
@Override
public int size() {
return map.size();
}
@Override
public Collection<V> values() {
return map.values();
}
@Override
public synchronized void clear() {
this.map = Collections.emptyMap();
}
@Override
public synchronized V put(K k, V v) {
Map<K, V> copy = new HashMap<K, V>(this.map);
V prev = copy.put(k, v);
this.map = Collections.unmodifiableMap(copy);
return prev;
}
@Override
public synchronized void putAll(Map<? extends K, ? extends V> entries) {
Map<K, V> copy = new HashMap<K, V>(this.map);
copy.putAll(entries);
this.map = Collections.unmodifiableMap(copy);
}
@Override
public synchronized V remove(Object key) {
Map<K, V> copy = new HashMap<K, V>(this.map);
V prev = copy.remove(key);
this.map = Collections.unmodifiableMap(copy);
return prev;
}
@Override
public synchronized V putIfAbsent(K k, V v) {
if (!containsKey(k))
return put(k, v);
else
return get(k);
}
@Override
public synchronized boolean remove(Object k, Object v) {
if (containsKey(k) && get(k).equals(v)) {
remove(k);
return true;
} else {
return false;
}
}
@Override
public synchronized boolean replace(K k, V original, V replacement) {
if (containsKey(k) && get(k).equals(original)) {
put(k, replacement);
return true;
} else {
return false;
}
}
@Override
public synchronized V replace(K k, V v) {
if (containsKey(k)) {
return put(k, v);
} else {
return null;
}
}
}
diff --git a/clients/src/main/java/kafka/common/utils/Crc32.java b/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java
similarity index 99%
rename from clients/src/main/java/kafka/common/utils/Crc32.java
rename to clients/src/main/java/org/apache/kafka/common/utils/Crc32.java
index d929c8db..153c5a6d 100644
--- a/clients/src/main/java/kafka/common/utils/Crc32.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/Crc32.java
@@ -1,2169 +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;
+package org.apache.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/org/apache/kafka/common/utils/KafkaThread.java
similarity index 92%
rename from clients/src/main/java/kafka/common/utils/KafkaThread.java
rename to clients/src/main/java/org/apache/kafka/common/utils/KafkaThread.java
index f830abaa..96428a92 100644
--- a/clients/src/main/java/kafka/common/utils/KafkaThread.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/KafkaThread.java
@@ -1,18 +1,18 @@
-package kafka.common.utils;
+package org.apache.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/org/apache/kafka/common/utils/SystemTime.java
similarity index 92%
rename from clients/src/main/java/kafka/common/utils/SystemTime.java
rename to clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java
index c8ca09cc..a3a91a98 100644
--- a/clients/src/main/java/kafka/common/utils/SystemTime.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/SystemTime.java
@@ -1,26 +1,26 @@
-package kafka.common.utils;
+package org.apache.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/org/apache/kafka/common/utils/Time.java
similarity index 91%
rename from clients/src/main/java/kafka/common/utils/Time.java
rename to clients/src/main/java/org/apache/kafka/common/utils/Time.java
index 390d16f9..5d68915e 100644
--- a/clients/src/main/java/kafka/common/utils/Time.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/Time.java
@@ -1,23 +1,23 @@
-package kafka.common.utils;
+package org.apache.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/org/apache/kafka/common/utils/Utils.java
similarity index 98%
rename from clients/src/main/java/kafka/common/utils/Utils.java
rename to clients/src/main/java/org/apache/kafka/common/utils/Utils.java
index f132771d..146eb769 100644
--- a/clients/src/main/java/kafka/common/utils/Utils.java
+++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java
@@ -1,230 +1,231 @@
-package kafka.common.utils;
+package org.apache.kafka.common.utils;
import java.io.UnsupportedEncodingException;
import java.nio.ByteBuffer;
-import kafka.common.KafkaException;
+import org.apache.kafka.common.KafkaException;
+
public class Utils {
/**
* Turn the given UTF8 byte array into a string
*
* @param bytes The byte array
* @return The string
*/
public static String utf8(byte[] bytes) {
try {
return new String(bytes, "UTF8");
} catch (UnsupportedEncodingException e) {
throw new RuntimeException("This shouldn't happen.", e);
}
}
/**
* Turn a string into a utf8 byte[]
*
* @param string The string
* @return The byte[]
*/
public static byte[] utf8(String string) {
try {
return string.getBytes("UTF8");
} catch (UnsupportedEncodingException e) {
throw new RuntimeException("This shouldn't happen.", e);
}
}
/**
* Read an unsigned integer from the current position in the buffer, incrementing the position by 4 bytes
*
* @param buffer The buffer to read from
* @return The integer read, as a long to avoid signedness
*/
public static long readUnsignedInt(ByteBuffer buffer) {
return buffer.getInt() & 0xffffffffL;
}
/**
* Read an unsigned integer from the given position without modifying the buffers position
*
* @param buffer the buffer to read from
* @param index the index from which to read the integer
* @return The integer read, as a long to avoid signedness
*/
public static long readUnsignedInt(ByteBuffer buffer, int index) {
return buffer.getInt(index) & 0xffffffffL;
}
/**
* Write the given long value as a 4 byte unsigned integer. Overflow is ignored.
*
* @param buffer The buffer to write to
* @param value The value to write
*/
public static void writetUnsignedInt(ByteBuffer buffer, long value) {
buffer.putInt((int) (value & 0xffffffffL));
}
/**
* Write the given long value as a 4 byte unsigned integer. Overflow is ignored.
*
* @param buffer The buffer to write to
* @param index The position in the buffer at which to begin writing
* @param value The value to write
*/
public static void writeUnsignedInt(ByteBuffer buffer, int index, long value) {
buffer.putInt(index, (int) (value & 0xffffffffL));
}
/**
* Compute the CRC32 of the byte array
*
* @param bytes The array to compute the checksum for
* @return The CRC32
*/
public static long crc32(byte[] bytes) {
return crc32(bytes, 0, bytes.length);
}
/**
* Compute the CRC32 of the segment of the byte array given by the specificed size and offset
*
* @param bytes The bytes to checksum
* @param offset the offset at which to begin checksumming
* @param size the number of bytes to checksum
* @return The CRC32
*/
public static long crc32(byte[] bytes, int offset, int size) {
Crc32 crc = new Crc32();
crc.update(bytes, offset, size);
return crc.getValue();
}
/**
* Get the absolute value of the given number. If the number is Int.MinValue return 0. This is different from
* java.lang.Math.abs or scala.math.abs in that they return Int.MinValue (!).
*/
public static int abs(int n) {
return n & 0x7fffffff;
}
/**
* Get the length for UTF8-encoding a string without encoding it first
*
* @param s The string to calculate the length for
* @return The length when serialized
*/
public static int utf8Length(CharSequence s) {
int count = 0;
for (int i = 0, len = s.length(); i < len; i++) {
char ch = s.charAt(i);
if (ch <= 0x7F) {
count++;
} else if (ch <= 0x7FF) {
count += 2;
} else if (Character.isHighSurrogate(ch)) {
count += 4;
++i;
} else {
count += 3;
}
}
return count;
}
/**
* Read the given byte buffer into a byte array
*/
public static byte[] toArray(ByteBuffer buffer) {
return toArray(buffer, 0, buffer.limit());
}
/**
* Read a byte array from the given offset and size in the buffer
*/
public static byte[] toArray(ByteBuffer buffer, int offset, int size) {
byte[] dest = new byte[size];
if (buffer.hasArray()) {
System.arraycopy(buffer.array(), buffer.arrayOffset() + offset, dest, 0, size);
} else {
int pos = buffer.position();
buffer.get(dest);
buffer.position(pos);
}
return dest;
}
/**
* Check that the parameter t is not null
*
* @param t The object to check
* @return t if it isn't null
* @throws NullPointerException if t is null.
*/
public static <T> T notNull(T t) {
if (t == null)
throw new NullPointerException();
else
return t;
}
/**
* Instantiate the class
*/
public static Object newInstance(Class<?> c) {
try {
return c.newInstance();
} catch (IllegalAccessException e) {
throw new KafkaException("Could not instantiate class " + c.getName(), e);
} catch (InstantiationException e) {
throw new KafkaException("Could not instantiate class " + c.getName() + " Does it have a public no-argument constructor?", e);
}
}
/**
* Generates 32 bit murmur2 hash from byte array
* @param data byte array to hash
* @return 32 bit hash of the given array
*/
public static int murmur2(final byte[] data) {
int length = data.length;
int seed = 0x9747b28c;
// 'm' and 'r' are mixing constants generated offline.
// They're not really 'magic', they just happen to work well.
final int m = 0x5bd1e995;
final int r = 24;
// Initialize the hash to a random value
int h = seed ^ length;
int length4 = length / 4;
for (int i = 0; i < length4; i++) {
final int i4 = i * 4;
int k = (data[i4 + 0] & 0xff) + ((data[i4 + 1] & 0xff) << 8) + ((data[i4 + 2] & 0xff) << 16) + ((data[i4 + 3] & 0xff) << 24);
k *= m;
k ^= k >>> r;
k *= m;
h *= m;
h ^= k;
}
// Handle the last few bytes of the input array
switch (length % 4) {
case 3:
h ^= (data[(length & ~3) + 2] & 0xff) << 16;
case 2:
h ^= (data[(length & ~3) + 1] & 0xff) << 8;
case 1:
h ^= (data[length & ~3] & 0xff);
h *= m;
}
h ^= h >>> 13;
h *= m;
h ^= h >>> 15;
return h;
}
}
diff --git a/clients/src/test/java/kafka/clients/producer/BufferPoolTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java
similarity index 96%
rename from clients/src/test/java/kafka/clients/producer/BufferPoolTest.java
rename to clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java
index 70603c40..16be3521 100644
--- a/clients/src/test/java/kafka/clients/producer/BufferPoolTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/BufferPoolTest.java
@@ -1,170 +1,171 @@
-package kafka.clients.producer;
+package org.apache.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.apache.kafka.clients.producer.BufferExhaustedException;
+import org.apache.kafka.clients.producer.internals.BufferPool;
+import org.apache.kafka.test.TestUtils;
import org.junit.Test;
public class BufferPoolTest {
/**
* Test the simple non-blocking allocation paths
*/
@Test
public void testSimple() throws Exception {
int totalMemory = 64 * 1024;
int size = 1024;
BufferPool pool = new BufferPool(totalMemory, size, false);
ByteBuffer buffer = pool.allocate(size);
assertEquals("Buffer size should equal requested size.", size, buffer.limit());
assertEquals("Unallocated memory should have shrunk", totalMemory - size, pool.unallocatedMemory());
assertEquals("Available memory should have shrunk", totalMemory - size, pool.availableMemory());
buffer.putInt(1);
buffer.flip();
pool.deallocate(buffer);
assertEquals("All memory should be available", totalMemory, pool.availableMemory());
assertEquals("But now some is on the free list", totalMemory - size, pool.unallocatedMemory());
buffer = pool.allocate(size);
assertEquals("Recycled buffer should be cleared.", 0, buffer.position());
assertEquals("Recycled buffer should be cleared.", buffer.capacity(), buffer.limit());
pool.deallocate(buffer);
assertEquals("All memory should be available", totalMemory, pool.availableMemory());
assertEquals("Still a single buffer on the free list", totalMemory - size, pool.unallocatedMemory());
buffer = pool.allocate(2 * size);
pool.deallocate(buffer);
assertEquals("All memory should be available", totalMemory, pool.availableMemory());
assertEquals("Non-standard size didn't go to the free list.", totalMemory - size, pool.unallocatedMemory());
}
/**
* Test that we cannot try to allocate more memory then we have in the whole pool
*/
@Test(expected = IllegalArgumentException.class)
public void testCantAllocateMoreMemoryThanWeHave() throws Exception {
BufferPool pool = new BufferPool(1024, 512, true);
ByteBuffer buffer = pool.allocate(1024);
assertEquals(1024, buffer.limit());
pool.deallocate(buffer);
buffer = pool.allocate(1025);
}
@Test
public void testNonblockingMode() throws Exception {
BufferPool pool = new BufferPool(2, 1, false);
pool.allocate(1);
try {
pool.allocate(2);
fail("The buffer allocated more than it has!");
} catch (BufferExhaustedException e) {
// this is good
}
}
/**
* Test that delayed allocation blocks
*/
@Test
public void testDelayedAllocation() throws Exception {
BufferPool pool = new BufferPool(5 * 1024, 1024, true);
ByteBuffer buffer = pool.allocate(1024);
CountDownLatch doDealloc = asyncDeallocate(pool, buffer);
CountDownLatch allocation = asyncAllocate(pool, 5 * 1024);
assertEquals("Allocation shouldn't have happened yet, waiting on memory.", 1, allocation.getCount());
doDealloc.countDown(); // return the memory
allocation.await();
}
private CountDownLatch asyncDeallocate(final BufferPool pool, final ByteBuffer buffer) {
final CountDownLatch latch = new CountDownLatch(1);
new Thread() {
public void run() {
try {
latch.await();
} catch (InterruptedException e) {
e.printStackTrace();
}
pool.deallocate(buffer);
}
}.start();
return latch;
}
private CountDownLatch asyncAllocate(final BufferPool pool, final int size) {
final CountDownLatch completed = new CountDownLatch(1);
new Thread() {
public void run() {
try {
pool.allocate(size);
} catch (InterruptedException e) {
e.printStackTrace();
} finally {
completed.countDown();
}
}
}.start();
return completed;
}
/**
* This test creates lots of threads that hammer on the pool
*/
@Test
public void testStressfulSituation() throws Exception {
int numThreads = 10;
final int iterations = 50000;
final int poolableSize = 1024;
final int totalMemory = numThreads / 2 * poolableSize;
final BufferPool pool = new BufferPool(totalMemory, poolableSize, true);
List<StressTestThread> threads = new ArrayList<StressTestThread>();
for (int i = 0; i < numThreads; i++)
threads.add(new StressTestThread(pool, iterations));
for (StressTestThread thread : threads)
thread.start();
for (StressTestThread thread : threads)
thread.join();
for (StressTestThread thread : threads)
assertTrue("Thread should have completed all iterations successfully.", thread.success.get());
assertEquals(totalMemory, pool.availableMemory());
}
public static class StressTestThread extends Thread {
private final int iterations;
private final BufferPool pool;
public final AtomicBoolean success = new AtomicBoolean(false);
public StressTestThread(BufferPool pool, int iterations) {
this.iterations = iterations;
this.pool = pool;
}
public void run() {
try {
for (int i = 0; i < iterations; i++) {
int size;
if (TestUtils.random.nextBoolean())
// allocate poolable size
size = pool.poolableSize();
else
// allocate a random size
size = TestUtils.random.nextInt((int) pool.totalMemory());
ByteBuffer buffer = pool.allocate(size);
pool.deallocate(buffer);
}
success.set(true);
} catch (Exception e) {
e.printStackTrace();
}
}
}
}
diff --git a/clients/src/test/java/kafka/clients/producer/MetadataTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
similarity index 89%
rename from clients/src/test/java/kafka/clients/producer/MetadataTest.java
rename to clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
index dd45209f..a4683307 100644
--- a/clients/src/test/java/kafka/clients/producer/MetadataTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/MetadataTest.java
@@ -1,49 +1,49 @@
-package kafka.clients.producer;
+package org.apache.kafka.clients.producer;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
-import kafka.clients.producer.internals.Metadata;
-import kafka.common.Cluster;
-import kafka.test.TestUtils;
+import org.apache.kafka.clients.producer.internals.Metadata;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.test.TestUtils;
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(TestUtils.singletonCluster(topic, 1), 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 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/org/apache/kafka/clients/producer/MockProducerTest.java
similarity index 92%
rename from clients/src/test/java/kafka/clients/producer/MockProducerTest.java
rename to clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java
index 24b132f6..d7080556 100644
--- a/clients/src/test/java/kafka/clients/producer/MockProducerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/MockProducerTest.java
@@ -1,63 +1,66 @@
-package kafka.clients.producer;
+package org.apache.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 java.util.concurrent.ExecutionException;
import java.util.concurrent.Future;
+import org.apache.kafka.clients.producer.MockProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
import org.junit.Test;
public class MockProducerTest {
private String topic = "topic";
@Test
public void testAutoCompleteMock() throws Exception {
MockProducer producer = new MockProducer(true);
ProducerRecord record = new ProducerRecord(topic, "key".getBytes(), "value".getBytes());
Future<RecordMetadata> metadata = producer.send(record);
assertTrue("Send should be immediately complete", metadata.isDone());
assertFalse("Send should be successful", isError(metadata));
assertEquals("Offset should be 0", 0, metadata.get().offset());
assertEquals(topic, metadata.get().topic());
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());
}
@Test
public void testManualCompletion() throws Exception {
MockProducer producer = new MockProducer(false);
ProducerRecord record1 = new ProducerRecord("topic", "key1".getBytes(), "value1".getBytes());
ProducerRecord record2 = new ProducerRecord("topic", "key2".getBytes(), "value2".getBytes());
Future<RecordMetadata> md1 = producer.send(record1);
assertFalse("Send shouldn't have completed", md1.isDone());
Future<RecordMetadata> md2 = producer.send(record2);
assertFalse("Send shouldn't have completed", md2.isDone());
assertTrue("Complete the first request", producer.completeNext());
assertFalse("Requst should be successful", isError(md1));
assertFalse("Second request still incomplete", md2.isDone());
IllegalArgumentException e = new IllegalArgumentException("blah");
assertTrue("Complete the second request with an error", producer.errorNext(e));
try {
md2.get();
fail("Expected error to be thrown");
} catch (ExecutionException err) {
assertEquals(e, err.getCause());
}
assertFalse("No more requests to complete", producer.completeNext());
}
private boolean isError(Future<?> future) {
try {
future.get();
return false;
} catch (Exception e) {
return true;
}
}
}
diff --git a/clients/src/test/java/kafka/clients/producer/PartitionerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java
similarity index 87%
rename from clients/src/test/java/kafka/clients/producer/PartitionerTest.java
rename to clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java
index c18da761..1b37876d 100644
--- a/clients/src/test/java/kafka/clients/producer/PartitionerTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/PartitionerTest.java
@@ -1,54 +1,55 @@
-package kafka.clients.producer;
+package org.apache.kafka.clients.producer;
import static java.util.Arrays.asList;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.util.List;
-import kafka.clients.producer.internals.Partitioner;
-import kafka.common.Cluster;
-import kafka.common.Node;
-import kafka.common.PartitionInfo;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.internals.Partitioner;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.PartitionInfo;
import org.junit.Test;
public class PartitionerTest {
private byte[] key = "key".getBytes();
private byte[] value = "value".getBytes();
private Partitioner partitioner = new Partitioner();
private Node node0 = new Node(0, "localhost", 99);
private Node node1 = new Node(1, "localhost", 100);
private Node node2 = new Node(2, "localhost", 101);
private Node[] nodes = new Node[] { node0, node1, node2 };
private String topic = "test";
private List<PartitionInfo> partitions = asList(new PartitionInfo(topic, 0, node0, nodes, nodes),
new PartitionInfo(topic, 1, node1, nodes, nodes),
new PartitionInfo(topic, 2, null, nodes, nodes));
private Cluster cluster = new Cluster(asList(node0, node1, node2), partitions);
@Test
public void testUserSuppliedPartitioning() {
assertEquals("If the user supplies a partition we should use it.",
0,
partitioner.partition(new ProducerRecord("test", 0, key, value), cluster));
}
@Test
public void testKeyPartitionIsStable() {
int partition = partitioner.partition(new ProducerRecord("test", key, value), cluster);
assertEquals("Same key should yield same partition",
partition,
partitioner.partition(new ProducerRecord("test", key, "value2".getBytes()), cluster));
}
@Test
public void testRoundRobinWithDownNode() {
for (int i = 0; i < partitions.size(); i++) {
int part = partitioner.partition(new ProducerRecord("test", value), cluster);
assertTrue("We should never choose a leader-less node in round robin", part >= 0 && part < 2);
}
}
}
diff --git a/clients/src/test/java/kafka/clients/producer/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java
similarity index 91%
rename from clients/src/test/java/kafka/clients/producer/RecordAccumulatorTest.java
rename to clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java
index b1ab3615..d1ac656e 100644
--- a/clients/src/test/java/kafka/clients/producer/RecordAccumulatorTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/RecordAccumulatorTest.java
@@ -1,135 +1,135 @@
-package kafka.clients.producer;
+package org.apache.kafka.clients.producer;
import static java.util.Arrays.asList;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import java.nio.ByteBuffer;
import java.util.ArrayList;
import java.util.Iterator;
import java.util.List;
-import kafka.clients.producer.internals.RecordAccumulator;
-import kafka.clients.producer.internals.RecordBatch;
-import kafka.common.TopicPartition;
-import kafka.common.metrics.Metrics;
-import kafka.common.record.CompressionType;
-import kafka.common.record.LogEntry;
-import kafka.common.record.Record;
-import kafka.common.record.Records;
-import kafka.common.utils.MockTime;
+import org.apache.kafka.clients.producer.internals.RecordAccumulator;
+import org.apache.kafka.clients.producer.internals.RecordBatch;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.record.CompressionType;
+import org.apache.kafka.common.record.LogEntry;
+import org.apache.kafka.common.record.Record;
+import org.apache.kafka.common.record.Records;
+import org.apache.kafka.common.utils.MockTime;
import org.junit.Test;
public class RecordAccumulatorTest {
private TopicPartition tp = new TopicPartition("test", 0);
private MockTime time = new MockTime();
private byte[] key = "key".getBytes();
private byte[] value = "value".getBytes();
private int msgSize = Records.LOG_OVERHEAD + Record.recordSize(key, value);
private Metrics metrics = new Metrics(time);
@Test
public void testFull() throws Exception {
long now = time.milliseconds();
RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, 10L, false, metrics, time);
int appends = 1024 / msgSize;
for (int i = 0; i < appends; i++) {
accum.append(tp, key, value, CompressionType.NONE, null);
assertEquals("No partitions should be ready.", 0, accum.ready(now).size());
}
accum.append(tp, key, value, CompressionType.NONE, null);
assertEquals("Our partition should be ready", asList(tp), accum.ready(time.milliseconds()));
List<RecordBatch> batches = accum.drain(asList(tp), Integer.MAX_VALUE);
assertEquals(1, batches.size());
RecordBatch batch = batches.get(0);
Iterator<LogEntry> iter = batch.records.iterator();
for (int i = 0; i < appends; i++) {
LogEntry entry = iter.next();
assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key());
assertEquals("Values should match", ByteBuffer.wrap(value), entry.record().value());
}
assertFalse("No more records", iter.hasNext());
}
@Test
public void testAppendLarge() throws Exception {
int batchSize = 512;
RecordAccumulator accum = new RecordAccumulator(batchSize, 10 * 1024, 0L, false, metrics, time);
accum.append(tp, key, new byte[2 * batchSize], CompressionType.NONE, null);
assertEquals("Our partition should be ready", asList(tp), accum.ready(time.milliseconds()));
}
@Test
public void testLinger() throws Exception {
long lingerMs = 10L;
RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, lingerMs, false, metrics, time);
accum.append(tp, key, value, CompressionType.NONE, null);
assertEquals("No partitions should be ready", 0, accum.ready(time.milliseconds()).size());
time.sleep(10);
assertEquals("Our partition should be ready", asList(tp), accum.ready(time.milliseconds()));
List<RecordBatch> batches = accum.drain(asList(tp), Integer.MAX_VALUE);
assertEquals(1, batches.size());
RecordBatch batch = batches.get(0);
Iterator<LogEntry> iter = batch.records.iterator();
LogEntry entry = iter.next();
assertEquals("Keys should match", ByteBuffer.wrap(key), entry.record().key());
assertEquals("Values should match", ByteBuffer.wrap(value), entry.record().value());
assertFalse("No more records", iter.hasNext());
}
@Test
public void testPartialDrain() throws Exception {
RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, 10L, false, metrics, time);
int appends = 1024 / msgSize + 1;
List<TopicPartition> partitions = asList(new TopicPartition("test", 0), new TopicPartition("test", 1));
for (TopicPartition tp : partitions) {
for (int i = 0; i < appends; i++)
accum.append(tp, key, value, CompressionType.NONE, null);
}
assertEquals("Both partitions should be ready", 2, accum.ready(time.milliseconds()).size());
List<RecordBatch> batches = accum.drain(partitions, 1024);
assertEquals("But due to size bound only one partition should have been retrieved", 1, batches.size());
}
@Test
public void testStressfulSituation() throws Exception {
final int numThreads = 5;
final int msgs = 10000;
final int numParts = 10;
final RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, 0L, true, metrics, time);
List<Thread> threads = new ArrayList<Thread>();
for (int i = 0; i < numThreads; i++) {
threads.add(new Thread() {
public void run() {
for (int i = 0; i < msgs; i++) {
try {
accum.append(new TopicPartition("test", i % numParts), key, value, CompressionType.NONE, null);
} catch (Exception e) {
e.printStackTrace();
}
}
}
});
}
for (Thread t : threads)
t.start();
int read = 0;
long now = time.milliseconds();
while (read < numThreads * msgs) {
List<TopicPartition> tps = accum.ready(now);
List<RecordBatch> batches = accum.drain(tps, 5 * 1024);
for (RecordBatch batch : batches) {
for (LogEntry entry : batch.records)
read++;
}
accum.deallocate(batches);
}
for (Thread t : threads)
t.join();
}
}
diff --git a/clients/src/test/java/kafka/clients/producer/RecordSendTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java
similarity index 89%
rename from clients/src/test/java/kafka/clients/producer/RecordSendTest.java
rename to clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java
index 804c57b2..b2ce8fdc 100644
--- a/clients/src/test/java/kafka/clients/producer/RecordSendTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/RecordSendTest.java
@@ -1,78 +1,78 @@
-package kafka.clients.producer;
+package org.apache.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.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
-import kafka.clients.producer.internals.FutureRecordMetadata;
-import kafka.clients.producer.internals.ProduceRequestResult;
-import kafka.common.TopicPartition;
-import kafka.common.errors.CorruptRecordException;
+import org.apache.kafka.clients.producer.internals.FutureRecordMetadata;
+import org.apache.kafka.clients.producer.internals.ProduceRequestResult;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.errors.CorruptRecordException;
import org.junit.Test;
public class RecordSendTest {
private TopicPartition topicPartition = new TopicPartition("test", 0);
private long baseOffset = 45;
private long relOffset = 5;
/**
* Test that waiting on a request that never completes times out
*/
@Test
public void testTimeout() throws Exception {
ProduceRequestResult request = new ProduceRequestResult();
FutureRecordMetadata future = new FutureRecordMetadata(request, relOffset);
assertFalse("Request is not completed", future.isDone());
try {
future.get(5, TimeUnit.MILLISECONDS);
fail("Should have thrown exception.");
} catch (TimeoutException e) { /* this is good */
}
request.done(topicPartition, baseOffset, null);
assertTrue(future.isDone());
assertEquals(baseOffset + relOffset, future.get().offset());
}
/**
* Test that an asynchronous request will eventually throw the right exception
*/
@Test(expected = ExecutionException.class)
public void testError() throws Exception {
FutureRecordMetadata future = new FutureRecordMetadata(asyncRequest(baseOffset, new CorruptRecordException(), 50L), relOffset);
future.get();
}
/**
* Test that an asynchronous request will eventually return the right offset
*/
@Test
public void testBlocking() throws Exception {
FutureRecordMetadata future = new FutureRecordMetadata(asyncRequest(baseOffset, null, 50L), relOffset);
assertEquals(baseOffset + relOffset, future.get().offset());
}
/* create a new request result that will be completed after the given timeout */
public ProduceRequestResult asyncRequest(final long baseOffset, final RuntimeException error, final long timeout) {
final ProduceRequestResult request = new ProduceRequestResult();
new Thread() {
public void run() {
try {
sleep(timeout);
request.done(topicPartition, baseOffset, error);
} catch (InterruptedException e) {
e.printStackTrace();
}
}
}.start();
return request;
}
}
diff --git a/clients/src/test/java/kafka/clients/producer/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java
similarity index 75%
rename from clients/src/test/java/kafka/clients/producer/SenderTest.java
rename to clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java
index 87880950..98318b74 100644
--- a/clients/src/test/java/kafka/clients/producer/SenderTest.java
+++ b/clients/src/test/java/org/apache/kafka/clients/producer/SenderTest.java
@@ -1,87 +1,88 @@
-package kafka.clients.producer;
+package org.apache.kafka.clients.producer;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertTrue;
import java.nio.ByteBuffer;
import java.util.concurrent.Future;
-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.TopicPartition;
-import kafka.common.metrics.Metrics;
-import kafka.common.network.NetworkReceive;
-import kafka.common.protocol.ApiKeys;
-import kafka.common.protocol.Errors;
-import kafka.common.protocol.ProtoUtils;
-import kafka.common.protocol.types.Struct;
-import kafka.common.record.CompressionType;
-import kafka.common.requests.RequestSend;
-import kafka.common.requests.ResponseHeader;
-import kafka.common.utils.MockTime;
-import kafka.test.MockSelector;
-import kafka.test.TestUtils;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.kafka.clients.producer.internals.Metadata;
+import org.apache.kafka.clients.producer.internals.RecordAccumulator;
+import org.apache.kafka.clients.producer.internals.Sender;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.TopicPartition;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.network.NetworkReceive;
+import org.apache.kafka.common.protocol.ApiKeys;
+import org.apache.kafka.common.protocol.Errors;
+import org.apache.kafka.common.protocol.ProtoUtils;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.record.CompressionType;
+import org.apache.kafka.common.requests.RequestSend;
+import org.apache.kafka.common.requests.ResponseHeader;
+import org.apache.kafka.common.utils.MockTime;
+import org.apache.kafka.test.MockSelector;
+import org.apache.kafka.test.TestUtils;
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 = TestUtils.singletonCluster("test", 1);
private Metrics metrics = new Metrics(time);
private RecordAccumulator accumulator = new RecordAccumulator(batchSize, 1024 * 1024, 0L, false, metrics, time);
private Sender sender = new Sender(selector, metadata, this.accumulator, "", 1024 * 1024, 0L, (short) -1, 10000, time);
@Before
public void setup() {
metadata.update(cluster, time.milliseconds());
}
@Test
public void testSimple() throws Exception {
TopicPartition tp = new TopicPartition("test", 0);
Future<RecordMetadata> future = 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", future.isDone());
assertEquals(offset, future.get().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/org/apache/kafka/common/config/ConfigDefTest.java
similarity index 92%
rename from clients/src/test/java/kafka/common/config/ConfigDefTest.java
rename to clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java
index a6a91acd..051bad1f 100644
--- a/clients/src/test/java/kafka/common/config/ConfigDefTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java
@@ -1,88 +1,90 @@
-package kafka.common.config;
+package org.apache.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.apache.kafka.common.config.ConfigDef;
+import org.apache.kafka.common.config.ConfigException;
+import org.apache.kafka.common.config.ConfigDef.Range;
+import org.apache.kafka.common.config.ConfigDef.Type;
import org.junit.Test;
public class ConfigDefTest {
@Test
public void testBasicTypes() {
ConfigDef def = new ConfigDef().define("a", Type.INT, 5, Range.between(0, 14), "docs")
.define("b", Type.LONG, "docs")
.define("c", Type.STRING, "hello", "docs")
.define("d", Type.LIST, "docs")
.define("e", Type.DOUBLE, "docs")
.define("f", Type.CLASS, "docs")
.define("g", Type.BOOLEAN, "docs");
Properties props = new Properties();
props.put("a", "1 ");
props.put("b", 2);
props.put("d", " a , b, c");
props.put("e", 42.5d);
props.put("f", String.class.getName());
props.put("g", "true");
Map<String, Object> vals = def.parse(props);
assertEquals(1, vals.get("a"));
assertEquals(2L, vals.get("b"));
assertEquals("hello", vals.get("c"));
assertEquals(asList("a", "b", "c"), vals.get("d"));
assertEquals(42.5d, vals.get("e"));
assertEquals(String.class, vals.get("f"));
assertEquals(true, vals.get("g"));
}
@Test(expected = ConfigException.class)
public void testInvalidDefault() {
new ConfigDef().define("a", Type.INT, "hello", "docs");
}
@Test(expected = ConfigException.class)
public void testNullDefault() {
new ConfigDef().define("a", Type.INT, null, null, "docs");
}
@Test(expected = ConfigException.class)
public void testMissingRequired() {
new ConfigDef().define("a", Type.INT, "docs").parse(new HashMap<String, Object>());
}
@Test(expected = ConfigException.class)
public void testDefinedTwice() {
new ConfigDef().define("a", Type.STRING, "docs").define("a", Type.INT, "docs");
}
@Test
public void testBadInputs() {
testBadInputs(Type.INT, "hello", null, "42.5", 42.5, Long.MAX_VALUE, Long.toString(Long.MAX_VALUE), new Object());
testBadInputs(Type.LONG, "hello", null, "42.5", Long.toString(Long.MAX_VALUE) + "00", new Object());
testBadInputs(Type.DOUBLE, "hello", null, new Object());
testBadInputs(Type.STRING, new Object());
testBadInputs(Type.LIST, 53, new Object());
}
private void testBadInputs(Type type, Object... values) {
for (Object value : values) {
Map<String, Object> m = new HashMap<String, Object>();
m.put("name", value);
ConfigDef def = new ConfigDef().define("name", type, "docs");
try {
def.parse(m);
fail("Expected a config exception on bad input for value " + value);
} catch (ConfigException e) {
// this is good
}
}
}
}
diff --git a/clients/src/test/java/kafka/common/metrics/JmxReporterTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/JmxReporterTest.java
similarity index 65%
rename from clients/src/test/java/kafka/common/metrics/JmxReporterTest.java
rename to clients/src/test/java/org/apache/kafka/common/metrics/JmxReporterTest.java
index e2862610..7a46d84b 100644
--- a/clients/src/test/java/kafka/common/metrics/JmxReporterTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/metrics/JmxReporterTest.java
@@ -1,21 +1,24 @@
-package kafka.common.metrics;
+package org.apache.kafka.common.metrics;
-import kafka.common.metrics.stats.Avg;
-import kafka.common.metrics.stats.Total;
+import org.apache.kafka.common.metrics.JmxReporter;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.metrics.stats.Avg;
+import org.apache.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/org/apache/kafka/common/metrics/MetricsTest.java
similarity index 88%
rename from clients/src/test/java/kafka/common/metrics/MetricsTest.java
rename to clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java
index f66cc7fe..c339efaa 100644
--- a/clients/src/test/java/kafka/common/metrics/MetricsTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java
@@ -1,210 +1,218 @@
-package kafka.common.metrics;
+package org.apache.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.Metric;
-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.apache.kafka.common.Metric;
+import org.apache.kafka.common.metrics.JmxReporter;
+import org.apache.kafka.common.metrics.Measurable;
+import org.apache.kafka.common.metrics.MetricConfig;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.metrics.MetricsReporter;
+import org.apache.kafka.common.metrics.Quota;
+import org.apache.kafka.common.metrics.QuotaViolationException;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.metrics.stats.Avg;
+import org.apache.kafka.common.metrics.stats.Count;
+import org.apache.kafka.common.metrics.stats.Max;
+import org.apache.kafka.common.metrics.stats.Min;
+import org.apache.kafka.common.metrics.stats.Percentile;
+import org.apache.kafka.common.metrics.stats.Percentiles;
+import org.apache.kafka.common.metrics.stats.Rate;
+import org.apache.kafka.common.metrics.stats.Total;
+import org.apache.kafka.common.metrics.stats.Percentiles.BucketSizing;
+import org.apache.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
}
}
@Test
public void testPercentiles() {
int buckets = 100;
Percentiles percs = new Percentiles(4 * buckets,
0.0,
100.0,
BucketSizing.CONSTANT,
new Percentile("test.p25", 25),
new Percentile("test.p50", 50),
new Percentile("test.p75", 75));
MetricConfig config = new MetricConfig().eventWindow(50).samples(2);
Sensor sensor = metrics.sensor("test", config);
sensor.add(percs);
Metric p25 = this.metrics.metrics().get("test.p25");
Metric p50 = this.metrics.metrics().get("test.p50");
Metric p75 = this.metrics.metrics().get("test.p75");
// record two windows worth of sequential values
for (int i = 0; i < buckets; i++)
sensor.record(i);
assertEquals(25, p25.value(), 1.0);
assertEquals(50, p50.value(), 1.0);
assertEquals(75, p75.value(), 1.0);
for (int i = 0; i < buckets; i++)
sensor.record(0.0);
assertEquals(0.0, p25.value(), 1.0);
assertEquals(0.0, p50.value(), 1.0);
assertEquals(0.0, p75.value(), 1.0);
}
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/org/apache/kafka/common/metrics/stats/HistogramTest.java
similarity index 91%
rename from clients/src/test/java/kafka/common/metrics/stats/HistogramTest.java
rename to clients/src/test/java/org/apache/kafka/common/metrics/stats/HistogramTest.java
index 9c6a4ab0..f55682c2 100644
--- a/clients/src/test/java/kafka/common/metrics/stats/HistogramTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/metrics/stats/HistogramTest.java
@@ -1,86 +1,87 @@
-package kafka.common.metrics.stats;
+package org.apache.kafka.common.metrics.stats;
import static org.junit.Assert.assertEquals;
import java.util.Arrays;
import java.util.Random;
-import kafka.common.metrics.stats.Histogram.BinScheme;
-import kafka.common.metrics.stats.Histogram.ConstantBinScheme;
-import kafka.common.metrics.stats.Histogram.LinearBinScheme;
+import org.apache.kafka.common.metrics.stats.Histogram;
+import org.apache.kafka.common.metrics.stats.Histogram.BinScheme;
+import org.apache.kafka.common.metrics.stats.Histogram.ConstantBinScheme;
+import org.apache.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), 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(10, 10);
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 + EPS);
assertEquals("unbinning and rebinning the bin " + bin
+ " gave a different result ("
+ fromBin
+ " was placed in bin "
+ binAgain
+ " )", bin, binAgain);
}
}
public static void main(String[] args) {
Random random = new Random();
System.out.println("[-100, 100]:");
for (BinScheme scheme : Arrays.asList(new ConstantBinScheme(1000, -100, 100),
new ConstantBinScheme(100, -100, 100),
new ConstantBinScheme(10, -100, 100))) {
Histogram h = new Histogram(scheme);
for (int i = 0; i < 10000; i++)
h.record(200.0 * random.nextDouble() - 100.0);
for (double quantile = 0.0; quantile < 1.0; quantile += 0.05)
System.out.printf("%5.2f: %.1f, ", quantile, h.value(quantile));
System.out.println();
}
System.out.println("[0, 1000]");
for (BinScheme scheme : Arrays.asList(new LinearBinScheme(1000, 1000),
new LinearBinScheme(100, 1000),
new LinearBinScheme(10, 1000))) {
Histogram h = new Histogram(scheme);
for (int i = 0; i < 10000; i++)
h.record(1000.0 * random.nextDouble());
for (double quantile = 0.0; quantile < 1.0; quantile += 0.05)
System.out.printf("%5.2f: %.1f, ", quantile, h.value(quantile));
System.out.println();
}
}
}
diff --git a/clients/src/test/java/kafka/clients/common/network/SelectorTest.java b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java
similarity index 96%
rename from clients/src/test/java/kafka/clients/common/network/SelectorTest.java
rename to clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java
index 68bc9ee1..97a808bc 100644
--- a/clients/src/test/java/kafka/clients/common/network/SelectorTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/network/SelectorTest.java
@@ -1,292 +1,292 @@
-package kafka.clients.common.network;
+package org.apache.kafka.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.apache.kafka.common.network.NetworkReceive;
+import org.apache.kafka.common.network.NetworkSend;
+import org.apache.kafka.common.network.Selectable;
+import org.apache.kafka.common.network.Selector;
+import org.apache.kafka.common.utils.Utils;
+import org.apache.kafka.test.TestUtils;
import org.junit.After;
import org.junit.Before;
import org.junit.Test;
/**
* A set of tests for the selector. These use a test harness that runs a simple socket server that echos back responses.
*/
public class SelectorTest {
private static final List<NetworkSend> EMPTY = new ArrayList<NetworkSend>();
private static final int BUFFER_SIZE = 4 * 1024;
private EchoServer server;
private Selectable selector;
@Before
public void setup() throws Exception {
this.server = new EchoServer();
this.server.start();
this.selector = new Selector();
}
@After
public void teardown() throws Exception {
this.selector.close();
this.server.close();
}
/**
* Validate that when the server disconnects, a client send ends up with that node in the disconnected list.
*/
@Test
public void testServerDisconnect() throws Exception {
int node = 0;
// connect and do a simple request
blockingConnect(node);
assertEquals("hello", blockingRequest(node, "hello"));
// disconnect
this.server.closeConnections();
while (!selector.disconnected().contains(node))
selector.poll(1000L, EMPTY);
// reconnect and do another request
blockingConnect(node);
assertEquals("hello", blockingRequest(node, "hello"));
}
/**
* Validate that the client can intentionally disconnect and reconnect
*/
@Test
public void testClientDisconnect() throws Exception {
int node = 0;
blockingConnect(node);
selector.disconnect(node);
selector.poll(10, asList(createSend(node, "hello1")));
assertEquals("Request should not have succeeded", 0, selector.completedSends().size());
assertEquals("There should be a disconnect", 1, selector.disconnected().size());
assertTrue("The disconnect should be from our node", selector.disconnected().contains(node));
blockingConnect(node);
assertEquals("hello2", blockingRequest(node, "hello2"));
}
/**
* Sending a request with one already in flight should result in an exception
*/
@Test(expected = IllegalStateException.class)
public void testCantSendWithInProgress() throws Exception {
int node = 0;
blockingConnect(node);
selector.poll(1000L, asList(createSend(node, "test1"), createSend(node, "test2")));
}
/**
* Sending a request to a node without an existing connection should result in an exception
*/
@Test(expected = IllegalStateException.class)
public void testCantSendWithoutConnecting() throws Exception {
selector.poll(1000L, asList(createSend(0, "test")));
}
/**
* Sending a request to a node with a bad hostname should result in an exception during connect
*/
@Test(expected = UnresolvedAddressException.class)
public void testNoRouteToHost() throws Exception {
selector.connect(0, new InetSocketAddress("asdf.asdf.dsc", server.port), BUFFER_SIZE, BUFFER_SIZE);
}
/**
* Sending a request to a node not listening on that port should result in disconnection
*/
@Test
public void testConnectionRefused() throws Exception {
int node = 0;
selector.connect(node, new InetSocketAddress("localhost", TestUtils.choosePort()), BUFFER_SIZE, BUFFER_SIZE);
while (selector.disconnected().contains(node))
selector.poll(1000L, EMPTY);
}
/**
* Send multiple requests to several connections in parallel. Validate that responses are received in the order that
* requests were sent.
*/
@Test
public void testNormalOperation() throws Exception {
int conns = 5;
int reqs = 500;
// create connections
InetSocketAddress addr = new InetSocketAddress("localhost", server.port);
for (int i = 0; i < conns; i++)
selector.connect(i, addr, BUFFER_SIZE, BUFFER_SIZE);
// send echo requests and receive responses
int[] requests = new int[conns];
int[] responses = new int[conns];
int responseCount = 0;
List<NetworkSend> sends = new ArrayList<NetworkSend>();
for (int i = 0; i < conns; i++)
sends.add(createSend(i, i + "-" + 0));
// loop until we complete all requests
while (responseCount < conns * reqs) {
// do the i/o
selector.poll(0L, sends);
assertEquals("No disconnects should have occurred.", 0, selector.disconnected().size());
// handle any responses we may have gotten
for (NetworkReceive receive : selector.completedReceives()) {
String[] pieces = asString(receive).split("-");
assertEquals("Should be in the form 'conn-counter'", 2, pieces.length);
assertEquals("Check the source", receive.source(), Integer.parseInt(pieces[0]));
assertEquals("Check that the receive has kindly been rewound", 0, receive.payload().position());
assertEquals("Check the request counter", responses[receive.source()], Integer.parseInt(pieces[1]));
responses[receive.source()]++; // increment the expected counter
responseCount++;
}
// prepare new sends for the next round
sends.clear();
for (NetworkSend send : selector.completedSends()) {
int dest = send.destination();
requests[dest]++;
if (requests[dest] < reqs)
sends.add(createSend(dest, dest + "-" + requests[dest]));
}
}
}
/**
* Validate that we can send and receive a message larger than the receive and send buffer size
*/
@Test
public void testSendLargeRequest() throws Exception {
int node = 0;
blockingConnect(node);
String big = TestUtils.randomString(10 * BUFFER_SIZE);
assertEquals(big, blockingRequest(node, big));
}
/**
* Test sending an empty string
*/
@Test
public void testEmptyRequest() throws Exception {
int node = 0;
blockingConnect(node);
assertEquals("", blockingRequest(node, ""));
}
private String blockingRequest(int node, String s) throws IOException {
selector.poll(1000L, asList(createSend(node, s)));
while (true) {
selector.poll(1000L, EMPTY);
for (NetworkReceive receive : selector.completedReceives())
if (receive.source() == node)
return asString(receive);
}
}
/* connect and wait for the connection to complete */
private void blockingConnect(int node) throws IOException {
selector.connect(node, new InetSocketAddress("localhost", server.port), BUFFER_SIZE, BUFFER_SIZE);
while (!selector.connected().contains(node))
selector.poll(10000L, EMPTY);
}
private NetworkSend createSend(int node, String s) {
return new NetworkSend(node, ByteBuffer.wrap(s.getBytes()));
}
private String asString(NetworkReceive receive) {
return new String(Utils.toArray(receive.payload()));
}
/**
* A simple server that takes size delimited byte arrays and just echos them back to the sender.
*/
static class EchoServer extends Thread {
public final int port;
private final ServerSocket serverSocket;
private final List<Thread> threads;
private final List<Socket> sockets;
public EchoServer() throws Exception {
this.port = TestUtils.choosePort();
this.serverSocket = new ServerSocket(port);
this.threads = Collections.synchronizedList(new ArrayList<Thread>());
this.sockets = Collections.synchronizedList(new ArrayList<Socket>());
}
public void run() {
try {
while (true) {
final Socket socket = serverSocket.accept();
sockets.add(socket);
Thread thread = new Thread() {
public void run() {
try {
DataInputStream input = new DataInputStream(socket.getInputStream());
DataOutputStream output = new DataOutputStream(socket.getOutputStream());
while (socket.isConnected() && !socket.isClosed()) {
int size = input.readInt();
byte[] bytes = new byte[size];
input.readFully(bytes);
output.writeInt(size);
output.write(bytes);
output.flush();
}
} catch (IOException e) {
// ignore
} finally {
try {
socket.close();
} catch (IOException e) {
// ignore
}
}
}
};
thread.start();
threads.add(thread);
}
} catch (IOException e) {
// ignore
}
}
public void closeConnections() throws IOException {
for (Socket socket : sockets)
socket.close();
}
public void close() throws IOException, InterruptedException {
this.serverSocket.close();
closeConnections();
for (Thread t : threads)
t.join();
join();
}
}
}
diff --git a/clients/src/test/java/kafka/common/protocol/types/ProtocolSerializationTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java
similarity index 90%
rename from clients/src/test/java/kafka/common/protocol/types/ProtocolSerializationTest.java
rename to clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java
index 5204f3af..809a95bf 100644
--- a/clients/src/test/java/kafka/common/protocol/types/ProtocolSerializationTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java
@@ -1,96 +1,102 @@
-package kafka.common.protocol.types;
+package org.apache.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.apache.kafka.common.protocol.types.ArrayOf;
+import org.apache.kafka.common.protocol.types.Field;
+import org.apache.kafka.common.protocol.types.Schema;
+import org.apache.kafka.common.protocol.types.SchemaException;
+import org.apache.kafka.common.protocol.types.Struct;
+import org.apache.kafka.common.protocol.types.Type;
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/org/apache/kafka/common/record/MemoryRecordsTest.java
similarity index 86%
rename from clients/src/test/java/kafka/common/record/MemoryRecordsTest.java
rename to clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java
index 6906309e..5bf20209 100644
--- a/clients/src/test/java/kafka/common/record/MemoryRecordsTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java
@@ -1,44 +1,47 @@
-package kafka.common.record;
+package org.apache.kafka.common.record;
-import static kafka.common.utils.Utils.toArray;
+import static org.apache.kafka.common.utils.Utils.toArray;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertTrue;
import java.nio.ByteBuffer;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
+import org.apache.kafka.common.record.LogEntry;
+import org.apache.kafka.common.record.MemoryRecords;
+import org.apache.kafka.common.record.Record;
import org.junit.Test;
public class MemoryRecordsTest {
@Test
public void testIterator() {
MemoryRecords recs1 = new MemoryRecords(ByteBuffer.allocate(1024));
MemoryRecords recs2 = new MemoryRecords(ByteBuffer.allocate(1024));
List<Record> list = Arrays.asList(new Record("a".getBytes(), "1".getBytes()),
new Record("b".getBytes(), "2".getBytes()),
new Record("c".getBytes(), "3".getBytes()));
for (int i = 0; i < list.size(); i++) {
Record r = list.get(i);
recs1.append(i, r);
recs2.append(i, toArray(r.key()), toArray(r.value()), r.compressionType());
}
for (int iteration = 0; iteration < 2; iteration++) {
for (MemoryRecords recs : Arrays.asList(recs1, recs2)) {
Iterator<LogEntry> iter = recs.iterator();
for (int i = 0; i < list.size(); i++) {
assertTrue(iter.hasNext());
LogEntry entry = iter.next();
assertEquals((long) i, entry.offset());
assertEquals(list.get(i), entry.record());
}
assertFalse(iter.hasNext());
}
}
}
}
diff --git a/clients/src/test/java/kafka/common/record/RecordTest.java b/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java
similarity index 93%
rename from clients/src/test/java/kafka/common/record/RecordTest.java
rename to clients/src/test/java/org/apache/kafka/common/record/RecordTest.java
index 9c59c9b6..ff8cc210 100644
--- a/clients/src/test/java/kafka/common/record/RecordTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/record/RecordTest.java
@@ -1,87 +1,90 @@
-package kafka.common.record;
+package org.apache.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.apache.kafka.common.record.CompressionType;
+import org.apache.kafka.common.record.InvalidRecordException;
+import org.apache.kafka.common.record.Record;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import org.junit.runners.Parameterized.Parameters;
@RunWith(value = Parameterized.class)
public class RecordTest {
private ByteBuffer key;
private ByteBuffer value;
private CompressionType compression;
private Record record;
public RecordTest(byte[] key, byte[] value, CompressionType compression) {
this.key = key == null ? null : ByteBuffer.wrap(key);
this.value = value == null ? null : ByteBuffer.wrap(value);
this.compression = compression;
this.record = new Record(key, value, compression);
}
@Test
public void testFields() {
assertEquals(compression, record.compressionType());
assertEquals(key != null, record.hasKey());
assertEquals(key, record.key());
if (key != null)
assertEquals(key.limit(), record.keySize());
assertEquals(Record.CURRENT_MAGIC_VALUE, record.magic());
assertEquals(value, record.value());
if (value != null)
assertEquals(value.limit(), record.valueSize());
}
@Test
public void testChecksum() {
assertEquals(record.checksum(), record.computeChecksum());
assertTrue(record.isValid());
for (int i = Record.CRC_OFFSET + Record.CRC_LENGTH; i < record.size(); i++) {
Record copy = copyOf(record);
copy.buffer().put(i, (byte) 69);
assertFalse(copy.isValid());
try {
copy.ensureValid();
fail("Should fail the above test.");
} catch (InvalidRecordException e) {
// this is good
}
}
}
private Record copyOf(Record record) {
ByteBuffer buffer = ByteBuffer.allocate(record.size());
record.buffer().put(buffer);
buffer.rewind();
record.buffer().rewind();
return new Record(buffer);
}
@Test
public void testEquality() {
assertEquals(record, copyOf(record));
}
@Parameters
public static Collection<Object[]> data() {
List<Object[]> values = new ArrayList<Object[]>();
for (byte[] key : Arrays.asList(null, "".getBytes(), "key".getBytes()))
for (byte[] value : Arrays.asList(null, "".getBytes(), "value".getBytes()))
for (CompressionType compression : CompressionType.values())
values.add(new Object[] { key, value, compression });
return values;
}
}
diff --git a/clients/src/test/java/kafka/common/utils/AbstractIteratorTest.java b/clients/src/test/java/org/apache/kafka/common/utils/AbstractIteratorTest.java
similarity index 93%
rename from clients/src/test/java/kafka/common/utils/AbstractIteratorTest.java
rename to clients/src/test/java/org/apache/kafka/common/utils/AbstractIteratorTest.java
index 7662d38d..5dbf166f 100644
--- a/clients/src/test/java/kafka/common/utils/AbstractIteratorTest.java
+++ b/clients/src/test/java/org/apache/kafka/common/utils/AbstractIteratorTest.java
@@ -1,54 +1,55 @@
-package kafka.common.utils;
+package org.apache.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.apache.kafka.common.utils.AbstractIterator;
import org.junit.Test;
public class AbstractIteratorTest {
@Test
public void testIterator() {
int max = 10;
List<Integer> l = new ArrayList<Integer>();
for (int i = 0; i < max; i++)
l.add(i);
ListIterator<Integer> iter = new ListIterator<Integer>(l);
for (int i = 0; i < max; i++) {
Integer value = i;
assertEquals(value, iter.peek());
assertTrue(iter.hasNext());
assertEquals(value, iter.next());
}
assertFalse(iter.hasNext());
}
@Test(expected = NoSuchElementException.class)
public void testEmptyIterator() {
Iterator<Object> iter = new ListIterator<Object>(Arrays.asList());
iter.next();
}
class ListIterator<T> extends AbstractIterator<T> {
private List<T> list;
private int position = 0;
public ListIterator(List<T> l) {
this.list = l;
}
public T makeNext() {
if (position < list.size())
return list.get(position++);
else
return allDone();
}
}
}
diff --git a/clients/src/test/java/kafka/common/utils/MockTime.java b/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java
similarity index 86%
rename from clients/src/test/java/kafka/common/utils/MockTime.java
rename to clients/src/test/java/org/apache/kafka/common/utils/MockTime.java
index 095d4f61..8f54d218 100644
--- a/clients/src/test/java/kafka/common/utils/MockTime.java
+++ b/clients/src/test/java/org/apache/kafka/common/utils/MockTime.java
@@ -1,28 +1,30 @@
-package kafka.common.utils;
+package org.apache.kafka.common.utils;
import java.util.concurrent.TimeUnit;
+import org.apache.kafka.common.utils.Time;
+
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/org/apache/kafka/test/MetricsBench.java
similarity index 71%
rename from clients/src/test/java/kafka/test/MetricsBench.java
rename to clients/src/test/java/org/apache/kafka/test/MetricsBench.java
index 2b164bde..676cc423 100644
--- a/clients/src/test/java/kafka/test/MetricsBench.java
+++ b/clients/src/test/java/org/apache/kafka/test/MetricsBench.java
@@ -1,38 +1,39 @@
-package kafka.test;
+package org.apache.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;
+import org.apache.kafka.common.metrics.Metrics;
+import org.apache.kafka.common.metrics.Sensor;
+import org.apache.kafka.common.metrics.stats.Avg;
+import org.apache.kafka.common.metrics.stats.Count;
+import org.apache.kafka.common.metrics.stats.Max;
+import org.apache.kafka.common.metrics.stats.Percentile;
+import org.apache.kafka.common.metrics.stats.Percentiles;
+import org.apache.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/org/apache/kafka/test/Microbenchmarks.java
similarity index 97%
rename from clients/src/test/java/kafka/test/Microbenchmarks.java
rename to clients/src/test/java/org/apache/kafka/test/Microbenchmarks.java
index a0ddecbf..64d172da 100644
--- a/clients/src/test/java/kafka/test/Microbenchmarks.java
+++ b/clients/src/test/java/org/apache/kafka/test/Microbenchmarks.java
@@ -1,143 +1,144 @@
-package kafka.test;
+package org.apache.kafka.test;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.Random;
import java.util.concurrent.ConcurrentHashMap;
import java.util.concurrent.atomic.AtomicBoolean;
-import kafka.common.utils.CopyOnWriteMap;
-import kafka.common.utils.SystemTime;
+import org.apache.kafka.common.utils.CopyOnWriteMap;
+import org.apache.kafka.common.utils.SystemTime;
+
public class Microbenchmarks {
public static void main(String[] args) throws Exception {
final int iters = Integer.parseInt(args[0]);
double x = 0.0;
long start = System.nanoTime();
for (int i = 0; i < iters; i++)
x += Math.sqrt(x);
System.out.println(x);
System.out.println("sqrt: " + (System.nanoTime() - start) / (double) iters);
// test clocks
systemMillis(iters);
systemNanos(iters);
long total = 0;
start = System.nanoTime();
total += systemMillis(iters);
System.out.println("System.currentTimeMillis(): " + (System.nanoTime() - start) / iters);
start = System.nanoTime();
total += systemNanos(iters);
System.out.println("System.nanoTime(): " + (System.nanoTime() - start) / iters);
System.out.println(total);
// test random
int n = 0;
Random random = new Random();
start = System.nanoTime();
for (int i = 0; i < iters; i++) {
n += random.nextInt();
}
System.out.println(n);
System.out.println("random: " + (System.nanoTime() - start) / iters);
float[] floats = new float[1024];
for (int i = 0; i < floats.length; i++)
floats[i] = random.nextFloat();
Arrays.sort(floats);
int loc = 0;
start = System.nanoTime();
for (int i = 0; i < iters; i++)
loc += Arrays.binarySearch(floats, floats[i % floats.length]);
System.out.println(loc);
System.out.println("binary search: " + (System.nanoTime() - start) / iters);
final SystemTime time = new SystemTime();
final AtomicBoolean done = new AtomicBoolean(false);
final Object lock = new Object();
Thread t1 = new Thread() {
public void run() {
time.sleep(1);
int counter = 0;
long start = time.nanoseconds();
for (int i = 0; i < iters; i++) {
synchronized (lock) {
counter++;
}
}
System.out.println("synchronized: " + ((System.nanoTime() - start) / iters));
System.out.println(counter);
done.set(true);
}
};
Thread t2 = new Thread() {
public void run() {
int counter = 0;
while (!done.get()) {
time.sleep(1);
synchronized (lock) {
counter += 1;
}
}
System.out.println("Counter: " + counter);
}
};
t1.start();
t2.start();
t1.join();
t2.join();
Map<String, Integer> values = new HashMap<String, Integer>();
for (int i = 0; i < 100; i++)
values.put(Integer.toString(i), i);
System.out.println("HashMap:");
benchMap(2, 1000000, values);
System.out.println("ConcurentHashMap:");
benchMap(2, 1000000, new ConcurrentHashMap<String, Integer>(values));
System.out.println("CopyOnWriteMap:");
benchMap(2, 1000000, new CopyOnWriteMap<String, Integer>(values));
}
private static void benchMap(int numThreads, final int iters, final Map<String, Integer> map) throws Exception {
final List<String> keys = new ArrayList<String>(map.keySet());
final List<Thread> threads = new ArrayList<Thread>();
for (int i = 0; i < numThreads; i++) {
threads.add(new Thread() {
public void run() {
int sum = 0;
long start = System.nanoTime();
for (int j = 0; j < iters; j++)
map.get(keys.get(j % threads.size()));
System.out.println("Map access time: " + ((System.nanoTime() - start) / (double) iters));
}
});
}
for (Thread thread : threads)
thread.start();
for (Thread thread : threads)
thread.join();
}
private static long systemMillis(int iters) {
long total = 0;
for (int i = 0; i < iters; i++)
total += System.currentTimeMillis();
return total;
}
private static long systemNanos(int iters) {
long total = 0;
for (int i = 0; i < iters; i++)
total += System.currentTimeMillis();
return total;
}
}
diff --git a/clients/src/test/java/kafka/test/MockSelector.java b/clients/src/test/java/org/apache/kafka/test/MockSelector.java
similarity index 89%
rename from clients/src/test/java/kafka/test/MockSelector.java
rename to clients/src/test/java/org/apache/kafka/test/MockSelector.java
index 15508f4b..0c69c5fb 100644
--- a/clients/src/test/java/kafka/test/MockSelector.java
+++ b/clients/src/test/java/org/apache/kafka/test/MockSelector.java
@@ -1,87 +1,88 @@
-package kafka.test;
+package org.apache.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;
+import org.apache.kafka.common.network.NetworkReceive;
+import org.apache.kafka.common.network.NetworkSend;
+import org.apache.kafka.common.network.Selectable;
+import org.apache.kafka.common.utils.Time;
+
/**
* A fake selector to use for testing
*/
public class MockSelector implements Selectable {
private final Time time;
private final List<NetworkSend> completedSends = new ArrayList<NetworkSend>();
private final List<NetworkReceive> completedReceives = new ArrayList<NetworkReceive>();
private final List<Integer> disconnected = new ArrayList<Integer>();
private final List<Integer> connected = new ArrayList<Integer>();
public MockSelector(Time time) {
this.time = time;
}
@Override
public void connect(int id, InetSocketAddress address, int sendBufferSize, int receiveBufferSize) throws IOException {
this.connected.add(id);
}
@Override
public void disconnect(int id) {
this.disconnected.add(id);
}
@Override
public void wakeup() {
}
@Override
public void close() {
}
public void clear() {
this.completedSends.clear();
this.completedReceives.clear();
this.disconnected.clear();
this.connected.clear();
}
@Override
public void poll(long timeout, List<NetworkSend> sends) throws IOException {
this.completedSends.addAll(sends);
time.sleep(timeout);
}
@Override
public List<NetworkSend> completedSends() {
return completedSends;
}
public void completeSend(NetworkSend send) {
this.completedSends.add(send);
}
@Override
public List<NetworkReceive> completedReceives() {
return completedReceives;
}
public void completeReceive(NetworkReceive receive) {
this.completedReceives.add(receive);
}
@Override
public List<Integer> disconnected() {
return disconnected;
}
@Override
public List<Integer> connected() {
return connected;
}
}
diff --git a/clients/src/test/java/kafka/test/TestUtils.java b/clients/src/test/java/org/apache/kafka/test/TestUtils.java
similarity index 94%
rename from clients/src/test/java/kafka/test/TestUtils.java
rename to clients/src/test/java/org/apache/kafka/test/TestUtils.java
index 90c68507..541bc591 100644
--- a/clients/src/test/java/kafka/test/TestUtils.java
+++ b/clients/src/test/java/org/apache/kafka/test/TestUtils.java
@@ -1,95 +1,96 @@
-package kafka.test;
+package org.apache.kafka.test;
import static java.util.Arrays.asList;
import java.io.File;
import java.io.IOException;
import java.net.ServerSocket;
import java.util.ArrayList;
import java.util.List;
import java.util.Random;
-import kafka.common.Cluster;
-import kafka.common.Node;
-import kafka.common.PartitionInfo;
+import org.apache.kafka.common.Cluster;
+import org.apache.kafka.common.Node;
+import org.apache.kafka.common.PartitionInfo;
+
/**
* 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();
public static Cluster singletonCluster(String topic, int partitions) {
return clusterWith(1, topic, partitions);
}
public static Cluster clusterWith(int nodes, String topic, int partitions) {
Node[] ns = new Node[nodes];
for (int i = 0; i < nodes; i++)
ns[i] = new Node(0, "localhost", 1969);
List<PartitionInfo> parts = new ArrayList<PartitionInfo>();
for (int i = 0; i < partitions; i++)
parts.add(new PartitionInfo(topic, i, ns[i % ns.length], ns, ns));
return new Cluster(asList(ns), parts);
}
/**
* 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();
}
}

File Metadata

Mime Type
text/x-diff
Expires
Mon, Aug 18, 7:21 PM (1 w, 3 d ago)
Storage Engine
blob
Storage Format
Raw Data
Storage Handle
3250799

Event Timeline