From f6e35dec9bf330c3531fd95c6566070d4ddf0457 Mon Sep 17 00:00:00 2001 From: Anna Povzner Date: Mon, 7 Mar 2016 23:18:17 -0800 Subject: [PATCH 001/206] KAFKA-3201: Added rolling upgrade system tests from 0.8 and 0.9 to 0.10 MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Three main tests: 1. Setup: Producer (0.8) → Kafka Cluster → Consumer (0.8) First rolling bounce: Set inter.broker.protocol.version = 0.8 and message.format.version = 0.8 Second rolling bonus, use latest (default) inter.broker.protocol.version and message.format.version 2. Setup: Producer (0.9) → Kafka Cluster → Consumer (0.9) First rolling bounce: Set inter.broker.protocol.version = 0.9 and message.format.version = 0.9 Second rolling bonus, use latest (default) inter.broker.protocol.version and message.format.version 3. Setup: Producer (0.9) → Kafka Cluster → Consumer (0.9) First rolling bounce: Set inter.broker.protocol.version = 0.9 and message.format.version = 0.9 Second rolling bonus: use inter.broker.protocol.version = 0.10 and message.format.version = 0.9 Plus couple of variations of these tests using old/new consumer and no compression / snappy compression. Author: Anna Povzner Reviewers: Ewen Cheslack-Postava Closes #980 from apovzner/kafka-3201-02 --- .../sanity_checks/test_verifiable_producer.py | 8 ++- tests/kafkatest/services/console_consumer.py | 6 +- .../services/kafka/config_property.py | 2 + tests/kafkatest/services/kafka/version.py | 4 ++ .../tests/reassign_partitions_test.py | 2 +- tests/kafkatest/tests/upgrade_test.py | 70 +++++++++++++------ vagrant/base.sh | 1 + 7 files changed, 65 insertions(+), 28 deletions(-) diff --git a/tests/kafkatest/sanity_checks/test_verifiable_producer.py b/tests/kafkatest/sanity_checks/test_verifiable_producer.py index 4155279ca8329..e22d4222d5bd3 100644 --- a/tests/kafkatest/sanity_checks/test_verifiable_producer.py +++ b/tests/kafkatest/sanity_checks/test_verifiable_producer.py @@ -20,7 +20,7 @@ from kafkatest.services.zookeeper import ZookeeperService from kafkatest.services.kafka import KafkaService -from kafkatest.services.kafka.version import LATEST_0_8_2, TRUNK, KafkaVersion +from kafkatest.services.kafka.version import LATEST_0_8_2, LATEST_0_9, TRUNK, KafkaVersion from kafkatest.services.verifiable_producer import VerifiableProducer from kafkatest.utils import is_version @@ -45,6 +45,7 @@ def setUp(self): self.kafka.start() @parametrize(producer_version=str(LATEST_0_8_2)) + @parametrize(producer_version=str(LATEST_0_9)) @parametrize(producer_version=str(TRUNK)) def test_simple_run(self, producer_version=TRUNK): """ @@ -61,7 +62,10 @@ def test_simple_run(self, producer_version=TRUNK): # that this check works with TRUNK # When running VerifiableProducer 0.8.X, both trunk version and 0.8.X should show up because of the way # verifiable producer pulls in some trunk directories into its classpath - assert is_version(node, [node.version.vstring, TRUNK.vstring]) + if node.version <= LATEST_0_8_2: + assert is_version(node, [node.version.vstring, TRUNK.vstring]) + else: + assert is_version(node, [node.version.vstring]) self.producer.wait() num_produced = self.producer.num_acked diff --git a/tests/kafkatest/services/console_consumer.py b/tests/kafkatest/services/console_consumer.py index a3bc2fd37317c..e5f2196e18418 100644 --- a/tests/kafkatest/services/console_consumer.py +++ b/tests/kafkatest/services/console_consumer.py @@ -17,7 +17,7 @@ from ducktape.services.background_thread import BackgroundThreadService from kafkatest.services.kafka.directory import kafka_dir -from kafkatest.services.kafka.version import TRUNK, LATEST_0_8_2 +from kafkatest.services.kafka.version import TRUNK, LATEST_0_8_2, LATEST_0_9 from kafkatest.services.monitor.jmx import JmxMixin from kafkatest.services.security.security_config import SecurityConfig @@ -181,8 +181,8 @@ def start_cmd(self, node): if self.print_key: cmd += " --property print.key=true" - # LoggingMessageFormatter was introduced in 0.9.0.0 - if node.version > LATEST_0_8_2: + # LoggingMessageFormatter was introduced after 0.9 + if node.version > LATEST_0_9: cmd+=" --formatter kafka.tools.LoggingMessageFormatter" cmd += " 2>> %(stderr)s | tee -a %(stdout)s &" % args diff --git a/tests/kafkatest/services/kafka/config_property.py b/tests/kafkatest/services/kafka/config_property.py index cc685aad660e2..b2b1d05ba5316 100644 --- a/tests/kafkatest/services/kafka/config_property.py +++ b/tests/kafkatest/services/kafka/config_property.py @@ -40,6 +40,8 @@ ZOOKEEPER_CONNECT = "zookeeper.connect" ZOOKEEPER_CONNECTION_TIMEOUT_MS = "zookeeper.connection.timeout.ms" INTER_BROKER_PROTOCOL_VERSION = "inter.broker.protocol.version" +MESSAGE_FORMAT_VERSION = "message.format.version" + """ diff --git a/tests/kafkatest/services/kafka/version.py b/tests/kafkatest/services/kafka/version.py index 95f3448f8ecc2..761d91ba2b7e9 100644 --- a/tests/kafkatest/services/kafka/version.py +++ b/tests/kafkatest/services/kafka/version.py @@ -59,3 +59,7 @@ def __str__(self): V_0_8_2_2 = KafkaVersion("0.8.2.2") LATEST_0_8_2 = V_0_8_2_2 +# 0.9.0.X versions +V_0_9_0_0 = KafkaVersion("0.9.0.0") +V_0_9_0_1 = KafkaVersion("0.9.0.1") +LATEST_0_9 = V_0_9_0_1 diff --git a/tests/kafkatest/tests/reassign_partitions_test.py b/tests/kafkatest/tests/reassign_partitions_test.py index fc0b9d62ca45f..24ce097af6310 100644 --- a/tests/kafkatest/tests/reassign_partitions_test.py +++ b/tests/kafkatest/tests/reassign_partitions_test.py @@ -107,4 +107,4 @@ def test_reassign_partitions(self, bounce_brokers, security_protocol): self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic, new_consumer=new_consumer, consumer_timeout_ms=60000, message_validator=is_int) self.kafka.start() - self.run_produce_consume_validate(core_test_action=self.reassign_partitions(bounce_brokers)) + self.run_produce_consume_validate(core_test_action=lambda: self.reassign_partitions(bounce_brokers)) diff --git a/tests/kafkatest/tests/upgrade_test.py b/tests/kafkatest/tests/upgrade_test.py index 20be4f2441eb2..bec4b3f025bd2 100644 --- a/tests/kafkatest/tests/upgrade_test.py +++ b/tests/kafkatest/tests/upgrade_test.py @@ -13,9 +13,10 @@ # See the License for the specific language governing permissions and # limitations under the License. +from ducktape.mark import parametrize from kafkatest.services.zookeeper import ZookeeperService from kafkatest.services.kafka import KafkaService -from kafkatest.services.kafka.version import LATEST_0_8_2, TRUNK +from kafkatest.services.kafka.version import LATEST_0_8_2, LATEST_0_9, TRUNK, KafkaVersion from kafkatest.services.verifiable_producer import VerifiableProducer from kafkatest.services.console_consumer import ConsoleConsumer from kafkatest.services.kafka import config_property @@ -31,49 +32,74 @@ def __init__(self, test_context): def setUp(self): self.topic = "test_topic" self.zk = ZookeeperService(self.test_context, num_nodes=1) - self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zk, version=LATEST_0_8_2, topics={self.topic: { - "partitions": 3, - "replication-factor": 3, - 'configs': {"min.insync.replicas": 2}}}) self.zk.start() - self.kafka.start() # Producer and consumer self.producer_throughput = 10000 self.num_producers = 1 self.num_consumers = 1 - self.producer = VerifiableProducer( - self.test_context, self.num_producers, self.kafka, self.topic, - throughput=self.producer_throughput, version=LATEST_0_8_2) - - # TODO - reduce the timeout - self.consumer = ConsoleConsumer( - self.test_context, self.num_consumers, self.kafka, self.topic, - consumer_timeout_ms=30000, message_validator=is_int, version=LATEST_0_8_2) - def perform_upgrade(self): + def perform_upgrade(self, from_kafka_version, to_message_format_version=None): self.logger.info("First pass bounce - rolling upgrade") for node in self.kafka.nodes: self.kafka.stop_node(node) node.version = TRUNK - node.config[config_property.INTER_BROKER_PROTOCOL_VERSION] = "0.8.2.X" + node.config[config_property.INTER_BROKER_PROTOCOL_VERSION] = from_kafka_version + node.config[config_property.MESSAGE_FORMAT_VERSION] = from_kafka_version self.kafka.start_node(node) self.logger.info("Second pass bounce - remove inter.broker.protocol.version config") for node in self.kafka.nodes: self.kafka.stop_node(node) del node.config[config_property.INTER_BROKER_PROTOCOL_VERSION] + if to_message_format_version is None: + del node.config[config_property.MESSAGE_FORMAT_VERSION] + else: + node.config[config_property.MESSAGE_FORMAT_VERSION] = to_message_format_version self.kafka.start_node(node) - def test_upgrade(self): - """Test upgrade of Kafka broker cluster from 0.8.2 to 0.9.0 - - Start 3 node broker cluster on version 0.8.2 + @parametrize(from_kafka_version=str(LATEST_0_9), to_message_format_version=None, compression_types=["none"]) + @parametrize(from_kafka_version=str(LATEST_0_9), to_message_format_version=None, compression_types=["snappy"], new_consumer=True) + @parametrize(from_kafka_version=str(LATEST_0_9), to_message_format_version=str(LATEST_0_9), compression_types=["none"]) + @parametrize(from_kafka_version=str(LATEST_0_9), to_message_format_version=str(LATEST_0_9), compression_types=["snappy"], new_consumer=True) + @parametrize(from_kafka_version=str(LATEST_0_8_2), to_message_format_version=None, compression_types=["none"]) + @parametrize(from_kafka_version=str(LATEST_0_8_2), to_message_format_version=None, compression_types=["snappy"]) + def test_upgrade(self, from_kafka_version, to_message_format_version, compression_types, new_consumer=False): + """Test upgrade of Kafka broker cluster from 0.8.2 or 0.9.0 to 0.10 + + from_kafka_version is a Kafka version to upgrade from: either 0.8.2.X or 0.9 + + If to_message_format_version is None, it means that we will upgrade to default (latest) + message format version. It is possible to upgrade to 0.10 brokers but still use message + format version 0.9 + + - Start 3 node broker cluster on version 'from_kafka_version' - Start producer and consumer in the background - Perform two-phase rolling upgrade - - First phase: upgrade brokers to 0.9.0 with inter.broker.protocol.version set to 0.8.2.X - - Second phase: remove inter.broker.protocol.version config with rolling bounce + - First phase: upgrade brokers to 0.10 with inter.broker.protocol.version set to + from_kafka_version and message.format.version set to from_kafka_version + - Second phase: remove inter.broker.protocol.version config with rolling bounce; if + to_message_format_version is set to 0.9, set message.format.version to + to_message_format_version, otherwise remove message.format.version config - Finally, validate that every message acked by the producer was consumed by the consumer """ + self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zk, + version=KafkaVersion(from_kafka_version), + topics={self.topic: {"partitions": 3, "replication-factor": 3, + 'configs': {"min.insync.replicas": 2}}}) + self.kafka.start() + + self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, + self.topic, throughput=self.producer_throughput, + message_validator=is_int, + compression_types=compression_types, + version=KafkaVersion(from_kafka_version)) + + # TODO - reduce the timeout + self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, + self.topic, consumer_timeout_ms=30000, new_consumer=new_consumer, + message_validator=is_int, version=KafkaVersion(from_kafka_version)) - self.run_produce_consume_validate(core_test_action=self.perform_upgrade) + self.run_produce_consume_validate(core_test_action=lambda: self.perform_upgrade(from_kafka_version, + to_message_format_version)) diff --git a/vagrant/base.sh b/vagrant/base.sh index 8e1e1c648edde..d271f87ec5f84 100644 --- a/vagrant/base.sh +++ b/vagrant/base.sh @@ -63,6 +63,7 @@ get_kafka() { } get_kafka 0.8.2.2 +get_kafka 0.9.0.1 # For EC2 nodes, we want to use /mnt, which should have the local disk. On local # VMs, we can just create it if it doesn't exist and use it like we'd use From 5afa1660103df8aecbf7558e761418944fb5905a Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Tue, 8 Mar 2016 14:06:53 -0800 Subject: [PATCH 002/206] KAFKA-3197; Fix producer sending records out of order This patch reuse max.in.flight.request.per.connection. When it equals to one, we take it as user wants order protection. The current approach is make sure there is only one batch per partition on the fly. Author: Jiangjie Qin Reviewers: Aditya Auradkar , Jason Gustafson , Grant Henke , Ismael Juma , Joel Koshy , Jun Rao Closes #857 from becketqin/KAFKA-3197 --- .../kafka/clients/producer/KafkaProducer.java | 1 + .../producer/internals/RecordAccumulator.java | 84 ++++++++++++------- .../clients/producer/internals/Sender.java | 17 +++- .../internals/RecordAccumulatorTest.java | 37 +++++++- .../producer/internals/SenderTest.java | 51 +++++++++++ .../java/org/apache/kafka/test/TestUtils.java | 2 +- 6 files changed, 156 insertions(+), 36 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index a066512c9d51a..85ba9efd3c33e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -287,6 +287,7 @@ private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serial this.sender = new Sender(client, this.metadata, this.accumulator, + config.getInt(ProducerConfig.MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION) == 1, config.getInt(ProducerConfig.MAX_REQUEST_SIZE_CONFIG), (short) parseAcks(config.getString(ProducerConfig.ACKS_CONFIG)), config.getInt(ProducerConfig.RETRIES_CONFIG), diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index f1414f01472e2..beaa832d3889e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -59,7 +59,6 @@ public final class RecordAccumulator { private static final Logger log = LoggerFactory.getLogger(RecordAccumulator.class); private volatile boolean closed; - private int drainIndex; private final AtomicInteger flushesInProgress; private final AtomicInteger appendsInProgress; private final int batchSize; @@ -70,6 +69,9 @@ public final class RecordAccumulator { private final Time time; private final ConcurrentMap> batches; private final IncompleteRecordBatches incomplete; + // The following variables are only accessed by the sender thread, so we don't need to protect them. + private final Set muted; + private int drainIndex; /** @@ -105,6 +107,7 @@ public RecordAccumulator(int batchSize, String metricGrpName = "producer-metrics"; this.free = new BufferPool(totalSize, batchSize, metrics, time, metricGrpName); this.incomplete = new IncompleteRecordBatches(); + this.muted = new HashSet(); this.time = time; registerMetrics(metrics, metricGrpName); } @@ -213,7 +216,6 @@ public List abortExpiredBatches(int requestTimeout, Cluster cluster List expiredBatches = new ArrayList(); int count = 0; for (Map.Entry> entry : this.batches.entrySet()) { - TopicPartition topicAndPartition = entry.getKey(); Deque dq = entry.getValue(); synchronized (dq) { // iterate over the batches and expire them if they have stayed in accumulator for more than requestTimeOut @@ -259,14 +261,20 @@ public void reenqueue(RecordBatch batch, long now) { * partition will be ready; Also return the flag for whether there are any unknown leaders for the accumulated * partition batches. *

- * A destination node is ready to send data if ANY one of its partition is not backing off the send and ANY of the - * following are true : + * A destination node is ready to send data if: *

    - *
  1. The record set is full - *
  2. The record set has sat in the accumulator for at least lingerMs milliseconds - *
  3. The accumulator is out of memory and threads are blocking waiting for data (in this case all partitions are - * immediately considered ready). - *
  4. The accumulator has been closed + *
  5. There is at least one partition that is not backing off its send + *
  6. and those partitions are not muted (to prevent reordering if + * {@value org.apache.kafka.clients.producer.ProducerConfig#MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION} + * is set to one)
  7. + *
  8. and any of the following are true
  9. + *
      + *
    • The record set is full
    • + *
    • The record set has sat in the accumulator for at least lingerMs milliseconds
    • + *
    • The accumulator is out of memory and threads are blocking waiting for data (in this case all partitions + * are immediately considered ready).
    • + *
    • The accumulator has been closed
    • + *
    *
*/ public ReadyCheckResult ready(Cluster cluster, long nowMs) { @@ -282,7 +290,7 @@ public ReadyCheckResult ready(Cluster cluster, long nowMs) { Node leader = cluster.leaderFor(part); if (leader == null) { unknownLeadersExist = true; - } else if (!readyNodes.contains(leader)) { + } else if (!readyNodes.contains(leader) && !muted.contains(part)) { synchronized (deque) { RecordBatch batch = deque.peekFirst(); if (batch != null) { @@ -333,7 +341,10 @@ public boolean hasUnsent() { * @param now The current unix time in milliseconds * @return A list of {@link RecordBatch} for each node specified with total size less than the requested maxSize. */ - public Map> drain(Cluster cluster, Set nodes, int maxSize, long now) { + public Map> drain(Cluster cluster, + Set nodes, + int maxSize, + long now) { if (nodes.isEmpty()) return Collections.emptyMap(); @@ -346,25 +357,29 @@ public Map> drain(Cluster cluster, Set nodes, i int start = drainIndex = drainIndex % parts.size(); do { PartitionInfo part = parts.get(drainIndex); - Deque deque = dequeFor(new TopicPartition(part.topic(), part.partition())); - if (deque != null) { - synchronized (deque) { - RecordBatch first = deque.peekFirst(); - if (first != null) { - boolean backoff = first.attempts > 0 && first.lastAttemptMs + retryBackoffMs > now; - // Only drain the batch if it is not during backoff period. - if (!backoff) { - if (size + first.records.sizeInBytes() > maxSize && !ready.isEmpty()) { - // there is a rare case that a single batch size is larger than the request size due - // to compression; in this case we will still eventually send this batch in a single - // request - break; - } else { - RecordBatch batch = deque.pollFirst(); - batch.records.close(); - size += batch.records.sizeInBytes(); - ready.add(batch); - batch.drainedMs = now; + TopicPartition tp = new TopicPartition(part.topic(), part.partition()); + // Only proceed if the partition has no in-flight batches. + if (!muted.contains(tp)) { + Deque deque = dequeFor(new TopicPartition(part.topic(), part.partition())); + if (deque != null) { + synchronized (deque) { + RecordBatch first = deque.peekFirst(); + if (first != null) { + boolean backoff = first.attempts > 0 && first.lastAttemptMs + retryBackoffMs > now; + // Only drain the batch if it is not during backoff period. + if (!backoff) { + if (size + first.records.sizeInBytes() > maxSize && !ready.isEmpty()) { + // there is a rare case that a single batch size is larger than the request size due + // to compression; in this case we will still eventually send this batch in a single + // request + break; + } else { + RecordBatch batch = deque.pollFirst(); + batch.records.close(); + size += batch.records.sizeInBytes(); + ready.add(batch); + batch.drainedMs = now; + } } } } @@ -465,6 +480,14 @@ private void abortBatches() { } } + public void mutePartition(TopicPartition tp) { + muted.add(tp); + } + + public void unmutePartition(TopicPartition tp) { + muted.remove(tp); + } + /** * Close this accumulator and force all the record buffers to be drained */ @@ -532,4 +555,5 @@ public Iterable all() { } } } + } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index 9d24d07fc28a6..db8918c2a49dd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -67,6 +67,9 @@ public class Sender implements Runnable { /* the metadata for the client */ private final Metadata metadata; + /* the flag indicating whether the producer should guarantee the message order on the broker or not. */ + private final boolean guaranteeMessageOrder; + /* the maximum request size to attempt to send to the server */ private final int maxRequestSize; @@ -97,6 +100,7 @@ public class Sender implements Runnable { public Sender(KafkaClient client, Metadata metadata, RecordAccumulator accumulator, + boolean guaranteeMessageOrder, int maxRequestSize, short acks, int retries, @@ -107,6 +111,7 @@ public Sender(KafkaClient client, this.client = client; this.accumulator = accumulator; this.metadata = metadata; + this.guaranteeMessageOrder = guaranteeMessageOrder; this.maxRequestSize = maxRequestSize; this.running = true; this.acks = acks; @@ -164,7 +169,7 @@ public void run() { * @param now * The current POSIX time in milliseconds */ - public void run(long now) { + void run(long now) { Cluster cluster = metadata.fetch(); // get the list of partitions with data ready to send RecordAccumulator.ReadyCheckResult result = this.accumulator.ready(cluster, now); @@ -189,6 +194,13 @@ public void run(long now) { result.readyNodes, this.maxRequestSize, now); + if (guaranteeMessageOrder) { + // Mute all the partitions drained + for (List batchList : batches.values()) { + for (RecordBatch batch : batchList) + this.accumulator.mutePartition(batch.topicPartition); + } + } List expiredBatches = this.accumulator.abortExpiredBatches(this.requestTimeout, cluster, now); // update sensors @@ -304,6 +316,9 @@ private void completeBatch(RecordBatch batch, Errors error, long baseOffset, lon } if (error.exception() instanceof InvalidMetadataException) metadata.requestUpdate(); + // Unmute the completed partition. + if (guaranteeMessageOrder) + this.accumulator.unmutePartition(batch.topicPartition); } /** diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java index 0f95ee5d494db..366027286181d 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java @@ -14,8 +14,8 @@ import static java.util.Arrays.asList; import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; import java.nio.ByteBuffer; import java.util.ArrayList; @@ -39,8 +39,6 @@ import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.Records; import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.common.utils.SystemTime; -import org.apache.kafka.common.utils.Time; import org.junit.After; import org.junit.Test; @@ -299,7 +297,6 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { @Test public void testExpiredBatches() throws InterruptedException { - Time time = new SystemTime(); long now = time.milliseconds(); RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 10, 100L, metrics, time); int appends = 1024 / msgSize; @@ -317,4 +314,36 @@ public void testExpiredBatches() throws InterruptedException { List expiredBatches = accum.abortExpiredBatches(60, cluster, now); assertEquals(1, expiredBatches.size()); } + + @Test + public void testMutedPartitions() throws InterruptedException { + long now = time.milliseconds(); + RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 10, 100L, metrics, time); + int appends = 1024 / msgSize; + for (int i = 0; i < appends; i++) { + accum.append(tp1, 0L, key, value, null, maxBlockTimeMs); + assertEquals("No partitions should be ready.", 0, accum.ready(cluster, now).readyNodes.size()); + } + time.sleep(2000); + + // Test ready with muted partition + accum.mutePartition(tp1); + RecordAccumulator.ReadyCheckResult result = accum.ready(cluster, time.milliseconds()); + assertEquals("No node should be ready", 0, result.readyNodes.size()); + + // Test ready without muted partition + accum.unmutePartition(tp1); + result = accum.ready(cluster, time.milliseconds()); + assertTrue("The batch should be ready", result.readyNodes.size() > 0); + + // Test drain with muted partition + accum.mutePartition(tp1); + Map> drained = accum.drain(cluster, result.readyNodes, Integer.MAX_VALUE, time.milliseconds()); + assertEquals("No batch should have been drained", 0, drained.get(node1.id()).size()); + + // Test drain without muted partition. + accum.unmutePartition(tp1); + drained = accum.drain(cluster, result.readyNodes, Integer.MAX_VALUE, time.milliseconds()); + assertTrue("The batch should have been drained.", drained.get(node1.id()).size() > 0); + } } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java index b983de51f82fc..fb67747c6740f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/SenderTest.java @@ -33,6 +33,7 @@ import org.apache.kafka.common.metrics.KafkaMetric; import org.apache.kafka.common.metrics.MetricConfig; import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.types.Struct; import org.apache.kafka.common.record.CompressionType; @@ -75,6 +76,7 @@ public void setup() { sender = new Sender(client, metadata, this.accumulator, + true, MAX_REQUEST_SIZE, ACKS_ALL, MAX_RETRIES, @@ -134,6 +136,7 @@ public void testRetries() throws Exception { Sender sender = new Sender(client, metadata, this.accumulator, + false, MAX_REQUEST_SIZE, ACKS_ALL, maxRetries, @@ -178,6 +181,54 @@ public void testRetries() throws Exception { } } + @Test + public void testSendInOrder() throws Exception { + int maxRetries = 1; + Metrics m = new Metrics(); + try { + Sender sender = new Sender(client, + metadata, + this.accumulator, + true, + MAX_REQUEST_SIZE, + ACKS_ALL, + maxRetries, + m, + time, + "clientId", + REQUEST_TIMEOUT); + + // Create a two broker cluster, with partition 0 on broker 0 and partition 1 on broker 1 + Cluster cluster1 = TestUtils.clusterWith(2, "test", 2); + metadata.update(cluster1, time.milliseconds()); + + // Send the first message. + TopicPartition tp2 = new TopicPartition("test", 1); + accumulator.append(tp2, 0L, "key1".getBytes(), "value1".getBytes(), null, MAX_BLOCK_TIMEOUT); + sender.run(time.milliseconds()); // connect + sender.run(time.milliseconds()); // send produce request + String id = client.requests().peek().request().destination(); + assertEquals(ApiKeys.PRODUCE.id, client.requests().peek().request().header().apiKey()); + Node node = new Node(Integer.valueOf(id), "localhost", 0); + assertEquals(1, client.inFlightRequestCount()); + assertTrue("Client ready status should be true", client.isReady(node, 0L)); + + time.sleep(900); + // Now send another message to tp2 + accumulator.append(tp2, 0L, "key2".getBytes(), "value2".getBytes(), null, MAX_BLOCK_TIMEOUT); + + // Update metadata before sender receives response from broker 0. Now partition 2 moves to broker 0 + Cluster cluster2 = TestUtils.singletonCluster("test", 2); + metadata.update(cluster2, time.milliseconds()); + // Sender should not send the second message to node 0. + sender.run(time.milliseconds()); + assertEquals(1, client.inFlightRequestCount()); + } finally { + m.close(); + } + + } + private void completedWithError(Future future, Errors error) throws Exception { assertTrue("Request should be completed", future.isDone()); try { diff --git a/clients/src/test/java/org/apache/kafka/test/TestUtils.java b/clients/src/test/java/org/apache/kafka/test/TestUtils.java index 7b88f5edc35b8..7ffc54aa3c3eb 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestUtils.java @@ -52,7 +52,7 @@ public static Cluster singletonCluster(String topic, int 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); + ns[i] = new Node(i, "localhost", 1969); List parts = new ArrayList(); for (int i = 0; i < partitions; i++) parts.add(new PartitionInfo(topic, i, ns[i % ns.length], ns, ns)); From c4282371d954d7ae6decd32252d85f0d2a254e8c Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Tue, 8 Mar 2016 18:34:07 -0600 Subject: [PATCH 003/206] =?UTF-8?q?KAFKA-3343;=20Use=20NoTimestamp=20in=20?= =?UTF-8?q?GroupMetadataManager=20when=20message=20v0=20i=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …s used. Author: Jiangjie Qin Reviewers: Ismael Juma , Jun Rao Closes #1023 from becketqin/KAFKA-3343 --- .../coordinator/GroupMetadataManager.scala | 24 +++++++++++-------- 1 file changed, 14 insertions(+), 10 deletions(-) diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala index cbdb8543d92fb..2c0236ee0cd66 100644 --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala @@ -143,8 +143,9 @@ class GroupMetadataManager(val brokerId: Int, // if we crash or leaders move) since the new leaders will still expire the consumers with heartbeat and // retry removing this group. val groupPartition = partitionFor(group.groupId) + val (magicValue, timestamp) = getMessageFormatVersionAndTimestamp(groupPartition) val tombstone = new Message(bytes = null, key = GroupMetadataManager.groupMetadataKey(group.groupId), - timestamp = time.milliseconds(), magicValue = getMessageFormatVersion(groupPartition)) + timestamp = timestamp, magicValue = magicValue) val partitionOpt = replicaManager.getPartition(GroupCoordinator.GroupMetadataTopicName, groupPartition) partitionOpt.foreach { partition => @@ -169,12 +170,12 @@ class GroupMetadataManager(val brokerId: Int, def prepareStoreGroup(group: GroupMetadata, groupAssignment: Map[String, Array[Byte]], responseCallback: Short => Unit): DelayedStore = { + val (magicValue, timestamp) = getMessageFormatVersionAndTimestamp(partitionFor(group.groupId)) val message = new Message( key = GroupMetadataManager.groupMetadataKey(group.groupId), bytes = GroupMetadataManager.groupMetadataValue(group, groupAssignment), - timestamp = time.milliseconds(), - magicValue = getMessageFormatVersion(partitionFor(group.groupId)) - ) + timestamp = timestamp, + magicValue = magicValue) val groupMetadataPartition = new TopicPartition(GroupCoordinator.GroupMetadataTopicName, partitionFor(group.groupId)) @@ -253,11 +254,12 @@ class GroupMetadataManager(val brokerId: Int, // construct the message set to append val messages = filteredOffsetMetadata.map { case (topicAndPartition, offsetAndMetadata) => + val (magicValue, timestamp) = getMessageFormatVersionAndTimestamp(partitionFor(groupId)) new Message( key = GroupMetadataManager.offsetCommitKey(groupId, topicAndPartition.topic, topicAndPartition.partition), bytes = GroupMetadataManager.offsetCommitValue(offsetAndMetadata), - timestamp = time.milliseconds(), - magicValue = getMessageFormatVersion(partitionFor(groupId)) + timestamp = timestamp, + magicValue = magicValue ) }.toSeq @@ -557,8 +559,8 @@ class GroupMetadataManager(val brokerId: Int, val commitKey = GroupMetadataManager.offsetCommitKey(groupTopicAndPartition.group, groupTopicAndPartition.topicPartition.topic, groupTopicAndPartition.topicPartition.partition) - (offsetsPartition, new Message(bytes = null, key = commitKey, timestamp = time.milliseconds(), - magicValue = getMessageFormatVersion(offsetsPartition))) + val (magicValue, timestamp) = getMessageFormatVersionAndTimestamp(offsetsPartition) + (offsetsPartition, new Message(bytes = null, key = commitKey, timestamp = timestamp, magicValue = magicValue)) }.groupBy { case (partition, tombstone) => partition } // Append the tombstone messages to the offset partitions. It is okay if the replicas don't receive these (say, @@ -627,11 +629,13 @@ class GroupMetadataManager(val brokerId: Int, config.offsetsTopicNumPartitions } - private def getMessageFormatVersion(partition: Int): Byte = { + private def getMessageFormatVersionAndTimestamp(partition: Int): (Byte, Long) = { val groupMetadataTopicAndPartition = new TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, partition) - replicaManager.getMessageFormatVersion(groupMetadataTopicAndPartition).getOrElse { + val messageFormatVersion = replicaManager.getMessageFormatVersion(groupMetadataTopicAndPartition).getOrElse { throw new IllegalArgumentException(s"Message format version for partition $groupMetadataTopicPartitionCount not found") } + val timestamp = if (messageFormatVersion == Message.MagicValue_V0) Message.NoTimestamp else time.milliseconds() + (messageFormatVersion, timestamp) } /** From 39f1c3ff83879e3d00ea4567db15181c746deba2 Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Tue, 8 Mar 2016 21:40:40 -0800 Subject: [PATCH 004/206] MINOR: Ignoring streams tests until there is fix for KAFKA-3354 Per discussion with guozhangwang, `ignore` failing streams system tests until fix for KAFKA-3354 is checked in. Author: Geoff Anderson Reviewers: Guozhang Wang Closes #1031 from granders/ignore-streams-systest --- tests/kafkatest/tests/streams_bounce_test.py | 4 +++- tests/kafkatest/tests/streams_smoke_test.py | 4 +++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/tests/kafkatest/tests/streams_bounce_test.py b/tests/kafkatest/tests/streams_bounce_test.py index 2b9c4d6135b0f..552390999d478 100644 --- a/tests/kafkatest/tests/streams_bounce_test.py +++ b/tests/kafkatest/tests/streams_bounce_test.py @@ -13,9 +13,10 @@ # See the License for the specific language governing permissions and # limitations under the License. +from ducktape.mark import ignore + from kafkatest.tests.kafka_test import KafkaTest from kafkatest.services.streams import StreamsSmokeTestDriverService, StreamsSmokeTestJobRunnerService -from ducktape.utils.util import wait_until import time class StreamsBounceTest(KafkaTest): @@ -40,6 +41,7 @@ def __init__(self, test_context): self.driver = StreamsSmokeTestDriverService(test_context, self.kafka) self.processor1 = StreamsSmokeTestJobRunnerService(test_context, self.kafka) + @ignore def test_bounce(self): """ Start a smoke test client, then abort (kill -9) and restart it a few times. diff --git a/tests/kafkatest/tests/streams_smoke_test.py b/tests/kafkatest/tests/streams_smoke_test.py index 48e4db8badc82..ea05c5f4cd050 100644 --- a/tests/kafkatest/tests/streams_smoke_test.py +++ b/tests/kafkatest/tests/streams_smoke_test.py @@ -13,9 +13,10 @@ # See the License for the specific language governing permissions and # limitations under the License. +from ducktape.mark import ignore + from kafkatest.tests.kafka_test import KafkaTest from kafkatest.services.streams import StreamsSmokeTestDriverService, StreamsSmokeTestJobRunnerService -from ducktape.utils.util import wait_until import time class StreamsSmokeTest(KafkaTest): @@ -43,6 +44,7 @@ def __init__(self, test_context): self.processor3 = StreamsSmokeTestJobRunnerService(test_context, self.kafka) self.processor4 = StreamsSmokeTestJobRunnerService(test_context, self.kafka) + @ignore def test_streams(self): """ Start a few smoke test clients, then repeat start a new one, stop (cleanly) running one a few times. From 090d7220e2c0c4c8a39c388043c6032386ea024f Mon Sep 17 00:00:00 2001 From: Jeremy Custenborder Date: Tue, 8 Mar 2016 21:56:33 -0800 Subject: [PATCH 005/206] KAFKA-3237: Allow ConfigDef options with no default and validator Remove test cases testInvalidDefaultRange() and testInvalidDefaultString(). Defaults if not overridden will get checked on parse. Testing the defaults is unnecessary. This allows you to set that a parameter is required while setting a validator for that parameter. Added a test case testNullDefaultWithValidator that allows a null default with a validator for certain strings. Author: Jeremy Custenborder Reviewers: Grant Henke , Ewen Cheslack-Postava Closes #936 from jcustenborder/KAFKA-3237 --- .../org/apache/kafka/common/config/ConfigDef.java | 4 ++-- .../apache/kafka/common/config/ConfigDefTest.java | 13 +++++++++++++ 2 files changed, 15 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java index fe7bcce59f95a..703eb7c4b9bd0 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java @@ -48,7 +48,7 @@ */ public class ConfigDef { - private static final Object NO_DEFAULT_VALUE = new String(""); + public static final Object NO_DEFAULT_VALUE = new String(""); private final Map configKeys = new HashMap(); @@ -360,7 +360,7 @@ public ConfigKey(String name, Type type, Object defaultValue, Validator validato this.defaultValue = defaultValue; this.validator = validator; this.importance = importance; - if (this.validator != null) + if (this.validator != null && this.hasDefault()) this.validator.ensureValid(name, defaultValue); this.documentation = documentation; } diff --git a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java index cb6de24e966dd..fa0370b9bee81 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java @@ -151,6 +151,19 @@ public void testSslPasswords() { assertEquals(Password.HIDDEN, vals.get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG).toString()); } + @Test + public void testNullDefaultWithValidator() { + final String key = "enum_test"; + + ConfigDef def = new ConfigDef(); + def.define(key, Type.STRING, ConfigDef.NO_DEFAULT_VALUE, ValidString.in("ONE", "TWO", "THREE"), Importance.HIGH, "docs"); + + Properties props = new Properties(); + props.put(key, "ONE"); + Map vals = def.parse(props); + assertEquals("ONE", vals.get(key)); + } + private void testValidators(Type type, Validator validator, Object defaultVal, Object[] okValues, Object[] badValues) { ConfigDef def = new ConfigDef().define("name", type, defaultVal, validator, Importance.HIGH, "docs"); From 689309135b3d75c2c6e7b215aca2e8d42b435877 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Tue, 8 Mar 2016 22:53:34 -0800 Subject: [PATCH 006/206] MINOR: Clean up of ProducerConfig documentation, including correction for block.on.buffer.full default value is "false" and not "true" See: https://stackoverflow.com/questions/35578519/kafka-block-on-buffer-full-default-value and https://github.com/apache/kafka/blob/d5b43b19bb06e9cdc606312c8bcf87ed267daf44/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java#L232 Author: mjsax Reviewers: Ismael Juma , Grant Henke , Ewen Cheslack-Postava Closes #954 from mjsax/hotfix-docu --- .../clients/producer/ProducerConfig.java | 62 +++++++++++-------- 1 file changed, 37 insertions(+), 25 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java index ee2b142a70fde..5b7a296a903b1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerConfig.java @@ -59,7 +59,8 @@ public class ProducerConfig extends AbstractConfig { /** batch.size */ public static final String BATCH_SIZE_CONFIG = "batch.size"; - private static final String BATCH_SIZE_DOC = "The producer will attempt to batch records together into fewer requests whenever multiple records are being sent" + " to the same partition. This helps performance on both the client and the server. This configuration controls the " + private static final String BATCH_SIZE_DOC = "The producer will attempt to batch records together into fewer requests whenever multiple records are being sent" + + " to the same partition. This helps performance on both the client and the server. This configuration controls the " + "default batch size in bytes. " + "

" + "No attempt will be made to batch records larger than this size. " @@ -70,15 +71,6 @@ public class ProducerConfig extends AbstractConfig { + "batching entirely). A very large batch size may use memory a bit more wastefully as we will always allocate a " + "buffer of the specified batch size in anticipation of additional records."; - /** buffer.memory */ - public static final String BUFFER_MEMORY_CONFIG = "buffer.memory"; - private static final String BUFFER_MEMORY_DOC = "The total bytes of memory the producer can use 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 block.on.buffer.full. " - + "

" - + "This setting should correspond roughly to the total memory the producer will use, but is not a hard bound since " - + "not all memory the producer uses is used for buffering. Some additional memory will be used for compression (if " - + "compression is enabled) as well as for maintaining in-flight requests."; - /** acks */ public static final String ACKS_CONFIG = "acks"; private static final String ACKS_DOC = "The number of acknowledgments the producer requires the leader to have received before considering a request complete. This controls the " @@ -103,20 +95,22 @@ public class ProducerConfig extends AbstractConfig { */ @Deprecated public static final String TIMEOUT_CONFIG = "timeout.ms"; - private static final String TIMEOUT_DOC = "The configuration controls the maximum amount of time the server will wait for acknowledgments from followers to " + "meet the acknowledgment requirements the producer has specified with the acks configuration. If the " + private static final String TIMEOUT_DOC = "The configuration controls the maximum amount of time the server will wait for acknowledgments from followers to " + + "meet the acknowledgment requirements the producer has specified with the acks configuration. If the " + "requested number of acknowledgments are not met when the timeout elapses an error will be returned. This timeout " + "is measured on the server side and does not include the network latency of the request."; /** linger.ms */ public static final String LINGER_MS_CONFIG = "linger.ms"; - private static final String LINGER_MS_DOC = "The producer groups together any records that arrive in between request transmissions into a single batched request. " + "Normally this occurs only under load when records arrive faster than they can be sent out. However in some circumstances the client may want to " + private static final String LINGER_MS_DOC = "The producer groups together any records that arrive in between request transmissions into a single batched request. " + + "Normally this occurs only under load when records arrive faster than they can be sent out. However in some circumstances the client may want to " + "reduce the number of requests even under moderate load. This setting accomplishes this by adding a small amount " + "of artificial delay—that is, rather than immediately sending out a record the producer will wait for up to " + "the given delay to allow other records to be sent so that the sends can be batched together. This can be thought " + "of as analogous to Nagle's algorithm in TCP. This setting gives the upper bound on the delay for batching: once " - + "we get batch.size worth of records for a partition it will be sent immediately regardless of this " + + "we get " + BATCH_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 (i.e. no delay). Setting linger.ms=5, " + + "specified time waiting for more records to show up. This setting defaults to 0 (i.e. no delay). Setting " + LINGER_MS_CONFIG + "=5, " + "for example, would have the effect of reducing the number of requests sent but would add up to 5ms of latency to records sent in the absense of load."; /** client.id */ @@ -130,24 +124,47 @@ public class ProducerConfig extends AbstractConfig { /** max.request.size */ public static final String MAX_REQUEST_SIZE_CONFIG = "max.request.size"; - private static final String MAX_REQUEST_SIZE_DOC = "The maximum size of a request in bytes. 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. This setting will limit the number of record " + private static final String MAX_REQUEST_SIZE_DOC = "The maximum size of a request in bytes. 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. This setting will limit the number of record " + "batches the producer will send in a single request to avoid sending huge requests."; /** reconnect.backoff.ms */ public static final String RECONNECT_BACKOFF_MS_CONFIG = CommonClientConfigs.RECONNECT_BACKOFF_MS_CONFIG; + /** max.block.ms */ + public static final String MAX_BLOCK_MS_CONFIG = "max.block.ms"; + private static final String MAX_BLOCK_MS_DOC = "The configuration controls how long KafkaProducer.send() and KafkaProducer.partitionsFor() will block." + + "These methods can be blocked either because the buffer is full or metadata unavailable." + + "Blocking in the user-supplied serializers or partitioner will not be counted against this timeout."; + /** block.on.buffer.full */ /** * @deprecated This config will be removed in a future release. Also, the {@link #METADATA_FETCH_TIMEOUT_CONFIG} is no longer honored when this property is set to true. */ @Deprecated public static final String BLOCK_ON_BUFFER_FULL_CONFIG = "block.on.buffer.full"; - private static final String BLOCK_ON_BUFFER_FULL_DOC = "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 in some scenarios blocking is not desirable and it is better to " - + "immediately give an error. Setting this to false will accomplish that: the producer will throw a BufferExhaustedException if a record is sent and the buffer space is full."; + private static final String BLOCK_ON_BUFFER_FULL_DOC = "When our memory buffer is exhausted we must either stop accepting new records (block) or throw errors. " + + "By default this setting is false and the producer will throw a BufferExhaustedException if a record is sent and the buffer space is full. " + + "However in some scenarios getting an error is not desirable and it is better to block. Setting this to true will accomplish that." + + "If this property is set to true, parameter " + METADATA_FETCH_TIMEOUT_CONFIG + " is not longer honored." + + "

" + + "This parameter is deprecated and will be removed in a future release. " + + "Parameter " + MAX_BLOCK_MS_CONFIG + " should be used instead."; + + /** buffer.memory */ + public static final String BUFFER_MEMORY_CONFIG = "buffer.memory"; + private static final String BUFFER_MEMORY_DOC = "The total bytes of memory the producer can use 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 " + BLOCK_ON_BUFFER_FULL_CONFIG + ". " + + "

" + + "This setting should correspond roughly to the total memory the producer will use, but is not a hard bound since " + + "not all memory the producer uses is used for buffering. Some additional memory will be used for compression (if " + + "compression is enabled) as well as for maintaining in-flight requests."; /** retries */ public static final String RETRIES_CONFIG = "retries"; - private static final String RETRIES_DOC = "Setting a value greater than zero will cause the client to resend any record whose send fails with a potentially transient error." + " Note that this retry is no different than if the client resent the record upon receiving the " + private static final String RETRIES_DOC = "Setting a value greater than zero will cause the client to resend any record whose send fails with a potentially transient error." + + " Note that this retry is no different than if the client resent the record upon receiving the " + "error. Allowing retries will potentially change the ordering of records because if two records are " + "sent to a single partition, and the first fails and is retried but the second succeeds, then the second record " + "may appear first."; @@ -157,7 +174,8 @@ public class ProducerConfig extends AbstractConfig { /** compression.type */ public static final String COMPRESSION_TYPE_CONFIG = "compression.type"; - private static final String COMPRESSION_TYPE_DOC = "The compression type for all data generated by the producer. The default is none (i.e. no compression). Valid " + " values are none, gzip, snappy, or lz4. " + private static final String COMPRESSION_TYPE_DOC = "The compression type for all data generated by the producer. The default is none (i.e. no compression). Valid " + + " values are none, gzip, snappy, or lz4. " + "Compression is of full batches of data, so the efficacy of batching will also impact the compression ratio (more batching means better compression)."; /** metrics.sample.window.ms */ @@ -190,12 +208,6 @@ public class ProducerConfig extends AbstractConfig { public static final String PARTITIONER_CLASS_CONFIG = "partitioner.class"; private static final String PARTITIONER_CLASS_DOC = "Partitioner class that implements the Partitioner interface."; - /** max.block.ms */ - public static final String MAX_BLOCK_MS_CONFIG = "max.block.ms"; - private static final String MAX_BLOCK_MS_DOC = "The configuration controls how long {@link KafkaProducer#send()} and {@link KafkaProducer#partitionsFor} will block." - + "These methods can be blocked either because the buffer is full or metadata unavailable." - + "Blocking in the user-supplied serializers or partitioner will not be counted against this timeout."; - /** request.timeout.ms */ public static final String REQUEST_TIMEOUT_MS_CONFIG = CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG; private static final String REQUEST_TIMEOUT_MS_DOC = CommonClientConfigs.REQUEST_TIMEOUT_MS_DOC; From 324b0c85f603005dceee69033b8fbffc7ef95281 Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Tue, 8 Mar 2016 23:40:09 -0800 Subject: [PATCH 007/206] KAFKA-3279: Remove checks for JAAS system property JAAS configuration may be set using other methods and hence the check for System property doesn't always match where the actual configuration used by Kafka is loaded from. Author: Rajini Sivaram Reviewers: Ismael Juma , Sriharsha Chintalapani , Flavio Junqueira , Ewen Cheslack-Postava Closes #967 from rajinisivaram/KAFKA-3279 --- .../kafka/common/security/JaasUtils.java | 31 +++++++------------ .../kafka/common/security/kerberos/Login.java | 5 +-- .../security/auth/ZkAuthorizationTest.scala | 5 ++- 3 files changed, 18 insertions(+), 23 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java b/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java index 0467a0973d14e..ff5e0082f1771 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java @@ -21,7 +21,6 @@ import java.lang.reflect.InvocationTargetException; import java.lang.reflect.Method; import java.io.IOException; -import java.io.File; import org.apache.kafka.common.KafkaException; import org.slf4j.Logger; @@ -88,25 +87,17 @@ public static boolean isZkSecurityEnabled() { boolean zkSaslEnabled = Boolean.parseBoolean(System.getProperty(ZK_SASL_CLIENT, "true")); String zkLoginContextName = System.getProperty(ZK_LOGIN_CONTEXT_NAME_KEY, "Client"); - String loginConfigFile = System.getProperty(JAVA_LOGIN_CONFIG_PARAM); - if (loginConfigFile != null && loginConfigFile.length() > 0) { - File configFile = new File(loginConfigFile); - if (!configFile.canRead()) { - throw new KafkaException("File " + loginConfigFile + "cannot be read."); - } - - try { - Configuration loginConf = Configuration.getConfiguration(); - isSecurityEnabled = loginConf.getAppConfigurationEntry(zkLoginContextName) != null; - } catch (Exception e) { - throw new KafkaException(e); - } - if (isSecurityEnabled && !zkSaslEnabled) { - LOG.error("JAAS file is present, but system property " + - ZK_SASL_CLIENT + " is set to false, which disables " + - "SASL in the ZooKeeper client"); - throw new KafkaException("Exception while determining if ZooKeeper is secure"); - } + try { + Configuration loginConf = Configuration.getConfiguration(); + isSecurityEnabled = loginConf.getAppConfigurationEntry(zkLoginContextName) != null; + } catch (Exception e) { + throw new KafkaException("Exception while loading Zookeeper JAAS login context '" + zkLoginContextName + "'", e); + } + if (isSecurityEnabled && !zkSaslEnabled) { + LOG.error("JAAS configuration is present, but system property " + + ZK_SASL_CLIENT + " is set to false, which disables " + + "SASL in the ZooKeeper client"); + throw new KafkaException("Exception while determining if ZooKeeper is secure"); } return isSecurityEnabled; diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/Login.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/Login.java index e8afbe661cc77..2e1a056a80ff9 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/kerberos/Login.java +++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/Login.java @@ -286,11 +286,12 @@ public Subject subject() { private synchronized LoginContext login(final String loginContextName) throws LoginException { String jaasConfigFile = System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM); if (jaasConfigFile == null) { - throw new IllegalArgumentException("You must pass " + JaasUtils.JAVA_LOGIN_CONFIG_PARAM + " in secure mode."); + log.debug("System property '" + JaasUtils.JAVA_LOGIN_CONFIG_PARAM + "' is not set, using default JAAS configuration."); } AppConfigurationEntry[] configEntries = Configuration.getConfiguration().getAppConfigurationEntry(loginContextName); if (configEntries == null) { - String errorMessage = "Could not find a '" + loginContextName + "' entry in `" + jaasConfigFile + "`."; + String errorMessage = "Could not find a '" + loginContextName + "' entry in the JAAS configuration. System property '" + + JaasUtils.JAVA_LOGIN_CONFIG_PARAM + "' is " + (jaasConfigFile == null ? "not set" : jaasConfigFile); throw new IllegalArgumentException(errorMessage); } diff --git a/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala b/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala index 2d73f4d1994d7..6a533b3fdbc54 100644 --- a/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala +++ b/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala @@ -46,6 +46,7 @@ class ZkAuthorizationTest extends ZooKeeperTestHarness with Logging{ super.tearDown() System.clearProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM) System.clearProperty(authProvider) + Configuration.setConfiguration(null) } /** @@ -55,9 +56,11 @@ class ZkAuthorizationTest extends ZooKeeperTestHarness with Logging{ @Test def testIsZkSecurityEnabled() { assertTrue(JaasUtils.isZkSecurityEnabled()) + Configuration.setConfiguration(null) System.clearProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM) assertFalse(JaasUtils.isZkSecurityEnabled()) - try { + try { + Configuration.setConfiguration(null) System.setProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM, "no-such-file-exists.conf") JaasUtils.isZkSecurityEnabled() fail("Should have thrown an exception") From 59d19cf3e9dce777e5c7bb333c0836adf9da80e1 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 9 Mar 2016 09:24:18 -0800 Subject: [PATCH 008/206] MINOR: Update gradlew.bat as per latest gradle release When invoking `gradle` on a recent version, it updates `gradlew.bat` to fix a typo. It's an annoyance at development time as it causes a diff on whatever branch one is working on. Author: Ismael Juma Reviewers: Grant Henke , Ewen Cheslack-Postava Closes #1034 from ijuma/update-gradlew.bat --- gradlew.bat | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradlew.bat b/gradlew.bat index aec99730b4e8f..72d362dafd896 100644 --- a/gradlew.bat +++ b/gradlew.bat @@ -46,7 +46,7 @@ echo location of your Java installation. goto fail :init -@rem Get command-line arguments, handling Windowz variants +@rem Get command-line arguments, handling Windows variants if not "%OS%" == "Windows_NT" goto win9xME_args if "%@eval[2+2]" == "4" goto 4NT_args From 818080312364940e4a6e964c22ec54e133b8ec02 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 9 Mar 2016 14:19:57 -0800 Subject: [PATCH 009/206] KAFKA-3352: Avoid DNS reverse lookups By using `getHostString` (introduced in Java 7) instead of `getHostName`. Author: Ismael Juma Reviewers: Jason Gustafson, Grant Henke Closes #1030 from ijuma/kafka-3352-avoid-dns-reverse-look-ups --- .../java/org/apache/kafka/common/Cluster.java | 4 +- .../org/apache/kafka/common/ClusterTest.java | 42 +++++++++++++++++++ .../scala/kafka/network/SocketServer.scala | 4 +- 3 files changed, 46 insertions(+), 4 deletions(-) create mode 100644 clients/src/test/java/org/apache/kafka/common/ClusterTest.java diff --git a/clients/src/main/java/org/apache/kafka/common/Cluster.java b/clients/src/main/java/org/apache/kafka/common/Cluster.java index d86e3a4a1b63e..4f3735850fc11 100644 --- a/clients/src/main/java/org/apache/kafka/common/Cluster.java +++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java @@ -110,10 +110,10 @@ public static Cluster empty() { * @return A cluster for these hosts/ports */ public static Cluster bootstrap(List addresses) { - List nodes = new ArrayList(); + List nodes = new ArrayList<>(); int nodeId = -1; for (InetSocketAddress address : addresses) - nodes.add(new Node(nodeId--, address.getHostName(), address.getPort())); + nodes.add(new Node(nodeId--, address.getHostString(), address.getPort())); return new Cluster(nodes, new ArrayList(0), Collections.emptySet()); } diff --git a/clients/src/test/java/org/apache/kafka/common/ClusterTest.java b/clients/src/test/java/org/apache/kafka/common/ClusterTest.java new file mode 100644 index 0000000000000..c4b5d4609ae4b --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/ClusterTest.java @@ -0,0 +1,42 @@ +/** + * 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 org.apache.kafka.common; + +import org.apache.kafka.common.utils.Utils; +import org.junit.Test; + +import java.net.InetSocketAddress; +import java.util.Arrays; +import java.util.HashSet; +import java.util.Set; + +import static org.junit.Assert.assertEquals; + +public class ClusterTest { + + @Test + public void testBootstrap() { + String ipAddress = "140.211.11.105"; + String hostName = "www.example.com"; + Cluster cluster = Cluster.bootstrap(Arrays.asList( + new InetSocketAddress(ipAddress, 9002), + new InetSocketAddress(hostName, 9002) + )); + Set expectedHosts = Utils.mkSet(ipAddress, hostName); + Set actualHosts = new HashSet<>(); + for (Node node : cluster.nodes()) + actualHosts.add(node.host()); + assertEquals(expectedHosts, actualHosts); + } + +} diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index f6c30366c5d66..5c31ac656607a 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -303,10 +303,10 @@ private[kafka] class Acceptor(val endPoint: EndPoint, serverChannel.socket().setReceiveBufferSize(recvBufferSize) try { serverChannel.socket.bind(socketAddress) - info("Awaiting socket connections on %s:%d.".format(socketAddress.getHostName, serverChannel.socket.getLocalPort)) + info("Awaiting socket connections on %s:%d.".format(socketAddress.getHostString, serverChannel.socket.getLocalPort)) } catch { case e: SocketException => - throw new KafkaException("Socket server failed to bind to %s:%d: %s.".format(socketAddress.getHostName, port, e.getMessage), e) + throw new KafkaException("Socket server failed to bind to %s:%d: %s.".format(socketAddress.getHostString, port, e.getMessage), e) } serverChannel } From ee9995583ad74277464d3e9cb669a7c96c07f70b Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 9 Mar 2016 17:57:24 -0800 Subject: [PATCH 010/206] KAFKA-3357; Update to Scala 2.11.8 Scala 2.11.8 has been released with a number of bug fixes: * http://www.scala-lang.org/news/2.11.8/ There are a few important collection fixes: * https://issues.scala-lang.org/browse/SI-9497 * https://github.com/scala/scala/pull/4714 * https://github.com/scala/scala/pull/4693 And also some pattern matcher fixes. Author: Ismael Juma Reviewers: Grant Henke, Ashish Singh Closes #1032 from ijuma/kafka-3357-update-to-scala-2.11.8 --- README.md | 4 ++-- gradle/dependencies.gradle | 2 +- 2 files changed, 3 insertions(+), 3 deletions(-) diff --git a/README.md b/README.md index c2ed1a7037816..bc2ca3afb07e1 100644 --- a/README.md +++ b/README.md @@ -60,10 +60,10 @@ The release file can be found inside `./core/build/distributions/`. ### Cleaning the build ### ./gradlew clean -### Running a task on a particular version of Scala (either 2.10.6 or 2.11.7) ### +### Running a task on a particular version of Scala (either 2.10.6 or 2.11.8) ### *Note that if building the jars with a version other than 2.10, you need to set the `SCALA_BINARY_VERSION` variable or change it in `bin/kafka-run-class.sh` to run the quick start.* -You can pass either the major version (eg 2.11) or the full version (eg 2.11.7): +You can pass either the major version (eg 2.11) or the full version (eg 2.11.8): ./gradlew -PscalaVersion=2.11 jar ./gradlew -PscalaVersion=2.11 test diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index eebe0078e6275..3610430995e27 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -47,7 +47,7 @@ versions += [ // Add Scala version def defaultScala210Version = '2.10.6' -def defaultScala211Version = '2.11.7' +def defaultScala211Version = '2.11.8' if (hasProperty('scalaVersion')) { if (scalaVersion == '2.10') { versions["scala"] = defaultScala210Version From f0a4125d3edf4ff28399584e7800addc026f5d08 Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Wed, 9 Mar 2016 21:13:54 -0800 Subject: [PATCH 011/206] KAFKA-3361: Initial protocol documentation page and generation - Moves all generated docs under /docs/generated - Generates docs for Protocol, Errors, and ApiKeys - Adds new protocol.html page Author: Grant Henke Reviewers: Gwen Shapira Closes #970 from granthenke/protocol-doc-wip --- .gitignore | 5 +- build.gradle | 45 ++++- .../apache/kafka/common/protocol/ApiKeys.java | 26 +++ .../apache/kafka/common/protocol/Errors.java | 34 ++++ .../kafka/common/protocol/Protocol.java | 166 ++++++++++++++++++ .../kafka/common/protocol/types/Field.java | 4 + docs/configuration.html | 8 +- docs/protocol.html | 163 +++++++++++++++++ 8 files changed, 435 insertions(+), 16 deletions(-) create mode 100644 docs/protocol.html diff --git a/.gitignore b/.gitignore index 22d9c700b97d6..50e1f85f074d6 100644 --- a/.gitignore +++ b/.gitignore @@ -38,7 +38,4 @@ tests/results .ducktape tests/.ducktape -docs/producer_config.html -docs/consumer_config.html -docs/kafka_config.html -docs/connect_config.html +docs/generated/ diff --git a/build.gradle b/build.gradle index a8266de3cc93d..6375bfbaa8d0c 100644 --- a/build.gradle +++ b/build.gradle @@ -70,6 +70,7 @@ ext { userShowStandardStreams = project.hasProperty("showStandardStreams") ? showStandardStreams : null + generatedDocsDir = new File("${project.rootDir}/docs/generated") } apply from: "$rootDir/gradle/dependencies.gradle" @@ -348,25 +349,52 @@ project(':core') { into "$buildDir/dependant-libs-${versions.scala}" } - tasks.create(name: "genProducerConfigDocs", dependsOn:jar, type: JavaExec) { + + task genProtocolErrorDocs(type: JavaExec) { + classpath = sourceSets.main.runtimeClasspath + main = 'org.apache.kafka.common.protocol.Protocol' + if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() } + standardOutput = new File(generatedDocsDir, "protocol_errors.html").newOutputStream() + } + + task genProtocolApiKeyDocs(type: JavaExec) { + classpath = sourceSets.main.runtimeClasspath + main = 'org.apache.kafka.common.protocol.Protocol' + if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() } + standardOutput = new File(generatedDocsDir, "protocol_api_keys.html").newOutputStream() + } + + task genProtocolMessageDocs(type: JavaExec) { + classpath = sourceSets.main.runtimeClasspath + main = 'org.apache.kafka.common.protocol.Protocol' + if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() } + standardOutput = new File(generatedDocsDir, "protocol_messages.html").newOutputStream() + } + + task genProducerConfigDocs(type: JavaExec) { classpath = sourceSets.main.runtimeClasspath main = 'org.apache.kafka.clients.producer.ProducerConfig' - standardOutput = new File("$rootDir/docs/producer_config.html").newOutputStream() + if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() } + standardOutput = new File(generatedDocsDir, "producer_config.html").newOutputStream() } - tasks.create(name: "genConsumerConfigDocs", dependsOn:jar, type: JavaExec) { + task genConsumerConfigDocs(type: JavaExec) { classpath = sourceSets.main.runtimeClasspath main = 'org.apache.kafka.clients.consumer.ConsumerConfig' - standardOutput = new File("$rootDir/docs/consumer_config.html").newOutputStream() + if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() } + standardOutput = new File(generatedDocsDir, "consumer_config.html").newOutputStream() } - tasks.create(name: "genKafkaConfigDocs", dependsOn:jar, type: JavaExec) { + task genKafkaConfigDocs(type: JavaExec) { classpath = sourceSets.main.runtimeClasspath main = 'kafka.server.KafkaConfig' - standardOutput = new File("$rootDir/docs/kafka_config.html").newOutputStream() + if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() } + standardOutput = new File(generatedDocsDir, "kafka_config.html").newOutputStream() } - task siteDocsTar(dependsOn: ['genProducerConfigDocs', 'genConsumerConfigDocs', 'genKafkaConfigDocs', ':connect:runtime:genConnectConfigDocs'], type: Tar) { + task siteDocsTar(dependsOn: ['genProtocolErrorDocs', 'genProtocolApiKeyDocs', 'genProtocolMessageDocs', + 'genProducerConfigDocs', 'genConsumerConfigDocs', 'genKafkaConfigDocs', + ':connect:runtime:genConnectConfigDocs'], type: Tar) { classifier = 'site-docs' compression = Compression.GZIP from project.file("../docs") @@ -727,7 +755,8 @@ project(':connect:runtime') { tasks.create(name: "genConnectConfigDocs", dependsOn:jar, type: JavaExec) { classpath = sourceSets.main.runtimeClasspath main = 'org.apache.kafka.connect.runtime.distributed.DistributedConfig' - standardOutput = new File("$rootDir/docs/connect_config.html").newOutputStream() + if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() } + standardOutput = new File(generatedDocsDir, "connect_config.html").newOutputStream() } } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index 708e1f055ae71..e8fd3d3215e55 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -70,4 +70,30 @@ public static ApiKeys forId(int id) { "and `%s` (inclusive)", id, MIN_API_KEY, MAX_API_KEY)); return ID_TO_TYPE[id]; } + + private static String toHtml() { + final StringBuilder b = new StringBuilder(); + b.append("\n"); + b.append(""); + b.append("\n"); + b.append("\n"); + b.append(""); + for (ApiKeys key : ApiKeys.values()) { + b.append("\n"); + b.append(""); + b.append(""); + b.append("\n"); + } + b.append("
NameKey
"); + b.append(key.name); + b.append(""); + b.append(key.id); + b.append("
\n"); + return b.toString(); + } + + public static void main(String[] args) { + System.out.println(toHtml()); + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index e7098fc05fcdd..ab299af47486c 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -48,6 +48,7 @@ import org.apache.kafka.common.errors.RecordBatchTooLargeException; import org.apache.kafka.common.errors.RecordTooLargeException; import org.apache.kafka.common.errors.ReplicaNotAvailableException; +import org.apache.kafka.common.errors.RetriableException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.TopicAuthorizationException; import org.apache.kafka.common.errors.UnknownMemberIdException; @@ -208,4 +209,37 @@ public static Errors forException(Throwable t) { } return UNKNOWN; } + + private static String toHtml() { + final StringBuilder b = new StringBuilder(); + b.append("\n"); + b.append(""); + b.append("\n"); + b.append("\n"); + b.append("\n"); + b.append("\n"); + b.append("\n"); + for (Errors error : Errors.values()) { + b.append(""); + b.append(""); + b.append(""); + b.append(""); + b.append(""); + b.append("\n"); + } + b.append("
ErrorCodeRetriableDescription
"); + b.append(error.name()); + b.append(""); + b.append(error.code()); + b.append(""); + b.append(error.exception() != null && error.exception() instanceof RetriableException ? "True" : "False"); + b.append(""); + b.append(error.exception() != null ? error.exception().getMessage() : ""); + b.append("
\n"); + return b.toString(); + } + + public static void main(String[] args) { + System.out.println(toHtml()); + } } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index 3787d2cecf23c..a77bf8cbb8623 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -19,6 +19,12 @@ 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.Type; + +import java.util.LinkedHashMap; +import java.util.LinkedHashSet; +import java.util.Map; +import java.util.Set; import static org.apache.kafka.common.protocol.types.Type.BYTES; import static org.apache.kafka.common.protocol.types.Type.INT16; @@ -750,4 +756,164 @@ public class Protocol { + " but " + RESPONSES[api.id].length + " response versions."); } + private static String indentString(int size) { + StringBuilder b = new StringBuilder(size); + for (int i = 0; i < size; i++) + b.append(" "); + return b.toString(); + } + + private static void schemaToBnfHtml(Schema schema, StringBuilder b, int indentSize) { + final String indentStr = indentString(indentSize); + final Map subTypes = new LinkedHashMap<>(); + + // Top level fields + for (Field field: schema.fields()) { + if (field.type instanceof ArrayOf) { + b.append("["); + b.append(field.name); + b.append("] "); + Type innerType = ((ArrayOf) field.type).type(); + if (innerType instanceof Schema && !subTypes.containsKey(field.name)) + subTypes.put(field.name, (Schema) innerType); + } else if (field.type instanceof Schema) { + b.append(field.name); + b.append(" "); + if (!subTypes.containsKey(field.name)) + subTypes.put(field.name, (Schema) field.type); + } else { + b.append(field.name); + b.append(" "); + if (!subTypes.containsKey(field.name)) + subTypes.put(field.name, field.type); + } + } + b.append("\n"); + + // Sub Types/Schemas + for (Map.Entry entry: subTypes.entrySet()) { + if (entry.getValue() instanceof Schema) { + // Complex Schema Type + b.append(indentStr); + b.append(entry.getKey()); + b.append(" => "); + schemaToBnfHtml((Schema) entry.getValue(), b, indentSize + 2); + } else { + // Standard Field Type + b.append(indentStr); + b.append(entry.getKey()); + b.append(" => "); + b.append(entry.getValue()); + b.append("\n"); + } + } + } + + private static void populateSchemaFields(Schema schema, Set fields) { + for (Field field: schema.fields()) { + fields.add(field); + if (field.type instanceof ArrayOf) { + Type innerType = ((ArrayOf) field.type).type(); + if (innerType instanceof Schema) + populateSchemaFields((Schema) innerType, fields); + } else if (field.type instanceof Schema) + populateSchemaFields((Schema) field.type, fields); + } + } + + private static void schemaToFieldTableHtml(Schema schema, StringBuilder b) { + Set fields = new LinkedHashSet<>(); + populateSchemaFields(schema, fields); + + b.append("\n"); + b.append(""); + b.append("\n"); + b.append("\n"); + b.append(""); + for (Field field : fields) { + b.append("\n"); + b.append(""); + b.append(""); + b.append("\n"); + } + b.append("
FieldDescription
"); + b.append(field.name); + b.append(""); + b.append(field.doc); + b.append("
\n"); + } + + public static String toHtml() { + final StringBuilder b = new StringBuilder(); + b.append("

Headers:
\n"); + + b.append("
");
+        b.append("Request Header => ");
+        schemaToBnfHtml(REQUEST_HEADER, b, 2);
+        b.append("
\n"); + schemaToFieldTableHtml(REQUEST_HEADER, b); + + b.append("
");
+        b.append("Response Header => ");
+        schemaToBnfHtml(RESPONSE_HEADER, b, 2);
+        b.append("
\n"); + schemaToFieldTableHtml(RESPONSE_HEADER, b); + + for (ApiKeys key : ApiKeys.values()) { + // Key + b.append("
"); + b.append(key.name); + b.append(" API (Key: "); + b.append(key.id); + b.append("):
\n\n"); + // Requests + b.append("Requests:
\n"); + Schema[] requests = REQUESTS[key.id]; + for (int i = 0; i < requests.length; i++) { + Schema schema = requests[i]; + // Schema + if (schema != null) { + b.append("

"); + // Version header + b.append("

");
+                    b.append(key.name);
+                    b.append(" Request (Version: ");
+                    b.append(i);
+                    b.append(") => ");
+                    schemaToBnfHtml(requests[i], b, 2);
+                    b.append("
"); + schemaToFieldTableHtml(requests[i], b); + } + b.append("

\n"); + } + + // Responses + b.append("Responses:
\n"); + Schema[] responses = RESPONSES[key.id]; + for (int i = 0; i < responses.length; i++) { + Schema schema = responses[i]; + // Schema + if (schema != null) { + b.append("

"); + // Version header + b.append("

");
+                    b.append(key.name);
+                    b.append(" Response (Version: ");
+                    b.append(i);
+                    b.append(") => ");
+                    schemaToBnfHtml(responses[i], b, 2);
+                    b.append("
"); + schemaToFieldTableHtml(responses[i], b); + } + b.append("

\n"); + } + } + + return b.toString(); + } + + public static void main(String[] args) { + System.out.println(toHtml()); + } + } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java index 1eb119589c3d9..cab7bf47dd3f2 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Field.java @@ -66,4 +66,8 @@ public Type type() { return type; } + public Schema schema() { + return schema; + } + } diff --git a/docs/configuration.html b/docs/configuration.html index e2ecde5000ce1..a89778d3270fa 100644 --- a/docs/configuration.html +++ b/docs/configuration.html @@ -28,7 +28,7 @@

3.1 Broker Configs

Topic-level configurations and defaults are discussed in more detail below. - +

More details about broker configuration can be found in the scala class kafka.server.KafkaConfig.

@@ -150,7 +150,7 @@

3.1 Broker Configs

3.2 Producer Configs

Below is the configuration of the Java producer: - +

For those interested in the legacy Scala producer configs, information can be found @@ -330,7 +330,7 @@

3.3.1 Old Consumer Con

3.3.2 New Consumer Configs

Since 0.9.0.0 we have been working on a replacement for our existing simple and high-level consumers. The code is considered beta quality. Below is the configuration for the new consumer: - +

3.4 Kafka Connect Configs

- + diff --git a/docs/protocol.html b/docs/protocol.html new file mode 100644 index 0000000000000..98923aad53405 --- /dev/null +++ b/docs/protocol.html @@ -0,0 +1,163 @@ +

Kafka Wire Protocol

+ +

This document covers the wire protocol implemented in Kafka. It is meant to give a readable guide to the protocol that covers the available requests, their binary format, and the proper way to make use of them to implement a client. This document assumes you understand the basic design and terminology described here

+ + + +

Preliminaries

+ +
Network
+ +

Kafka uses a binary protocol over TCP. The protocol defines all apis as request response message pairs. All messages are size delimited and are made up of the following primitive types.

+ +

The client initiates a socket connection and then writes a sequence of request messages and reads back the corresponding response message. No handshake is required on connection or disconnection. TCP is happier if you maintain persistent connections used for many requests to amortize the cost of the TCP handshake, but beyond this penalty connecting is pretty cheap.

+ +

The client will likely need to maintain a connection to multiple brokers, as data is partitioned and the clients will need to talk to the server that has their data. However it should not generally be necessary to maintain multiple connections to a single broker from a single client instance (i.e. connection pooling).

+ +

The server guarantees that on a single TCP connection, requests will be processed in the order they are sent and responses will return in that order as well. The broker's request processing allows only a single in-flight request per connection in order to guarantee this ordering. Note that clients can (and ideally should) use non-blocking IO to implement request pipelining and achieve higher throughput. i.e., clients can send requests even while awaiting responses for preceding requests since the outstanding requests will be buffered in the underlying OS socket buffer. All requests are initiated by the client, and result in a corresponding response message from the server except where noted.

+ +

The server has a configurable maximum limit on request size and any request that exceeds this limit will result in the socket being disconnected.

+ +
Partitioning and bootstrapping
+ +

Kafka is a partitioned system so not all servers have the complete data set. Instead recall that topics are split into a pre-defined number of partitions, P, and each partition is replicated with some replication factor, N. Topic partitions themselves are just ordered "commit logs" numbered 0, 1, ..., P.

+ +

All systems of this nature have the question of how a particular piece of data is assigned to a particular partition. Kafka clients directly control this assignment, the brokers themselves enforce no particular semantics of which messages should be published to a particular partition. Rather, to publish messages the client directly addresses messages to a particular partition, and when fetching messages, fetches from a particular partition. If two clients want to use the same partitioning scheme they must use the same method to compute the mapping of key to partition.

+ +

These requests to publish or fetch data must be sent to the broker that is currently acting as the leader for a given partition. This condition is enforced by the broker, so a request for a particular partition to the wrong broker will result in an the NotLeaderForPartition error code (described below).

+ +

How can the client find out which topics exist, what partitions they have, and which brokers currently host those partitions so that it can direct its requests to the right hosts? This information is dynamic, so you can't just configure each client with some static mapping file. Instead all Kafka brokers can answer a metadata request that describes the current state of the cluster: what topics there are, which partitions those topics have, which broker is the leader for those partitions, and the host and port information for these brokers.

+ +

In other words, the client needs to somehow find one broker and that broker will tell the client about all the other brokers that exist and what partitions they host. This first broker may itself go down so the best practice for a client implementation is to take a list of two or three urls to bootstrap from. The user can then choose to use a load balancer or just statically configure two or three of their kafka hosts in the clients.

+ +

The client does not need to keep polling to see if the cluster has changed; it can fetch metadata once when it is instantiated cache that metadata until it receives an error indicating that the metadata is out of date. This error can come in two forms: (1) a socket error indicating the client cannot communicate with a particular broker, (2) an error code in the response to a request indicating that this broker no longer hosts the partition for which data was requested.

+
    +
  1. Cycle through a list of "bootstrap" kafka urls until we find one we can connect to. Fetch cluster metadata.
  2. +
  3. Process fetch or produce requests, directing them to the appropriate broker based on the topic/partitions they send to or fetch from.
  4. +
  5. If we get an appropriate error, refresh the metadata and try again.
  6. +
+ +
Partitioning Strategies
+ +

As mentioned above the assignment of messages to partitions is something the producing client controls. That said, how should this functionality be exposed to the end-user?

+ +

Partitioning really serves two purposes in Kafka:

+
    +
  1. It balances data and request load over brokers
  2. +
  3. It serves as a way to divvy up processing among consumer processes while allowing local state and preserving order within the partition. We call this semantic partitioning.
  4. +
+ +

For a given use case you may care about only one of these or both.

+ +

To accomplish simple load balancing a simple approach would be for the client to just round robin requests over all brokers. Another alternative, in an environment where there are many more producers than brokers, would be to have each client chose a single partition at random and publish to that. This later strategy will result in far fewer TCP connections.

+ +

Semantic partitioning means using some key in the message to assign messages to partitions. For example if you were processing a click message stream you might want to partition the stream by the user id so that all data for a particular user would go to a single consumer. To accomplish this the client can take a key associated with the message and use some hash of this key to choose the partition to which to deliver the message.

+ +
Batching
+ +

Our apis encourage batching small things together for efficiency. We have found this is a very significant performance win. Both our API to send messages and our API to fetch messages always work with a sequence of messages not a single message to encourage this. A clever client can make use of this and support an "asynchronous" mode in which it batches together messages sent individually and sends them in larger clumps. We go even further with this and allow the batching across multiple topics and partitions, so a produce request may contain data to append to many partitions and a fetch request may pull data from many partitions all at once.

+ +

The client implementer can choose to ignore this and send everything one at a time if they like.

+ +
Versioning and Compatibility
+ +

The protocol is designed to enable incremental evolution in a backward compatible fashion. Our versioning is on a per-api basis, each version consisting of a request and response pair. Each request contains an API key that identifies the API being invoked and a version number that indicates the format of the request and the expected format of the response.

+ +

The intention is that clients would implement a particular version of the protocol, and indicate this version in their requests. Our goal is primarily to allow API evolution in an environment where downtime is not allowed and clients and servers cannot all be changed at once.

+ +

The server will reject requests with a version it does not support, and will always respond to the client with exactly the protocol format it expects based on the version it included in its request. The intended upgrade path is that new features would first be rolled out on the server (with the older clients not making use of them) and then as newer clients are deployed these new features would gradually be taken advantage of.

+ +

Currently all versions are baselined at 0, as we evolve these APIs we will indicate the format for each version individually.

+ +

The Protocol

+ +
Protocol Primitive Types
+ +

The protocol is built out of the following primitive types.

+ +

Fixed Width Primitives

+ +

int8, int16, int32, int64 - Signed integers with the given precision (in bits) stored in big endian order.

+ +

Variable Length Primitives

+ +

bytes, string - These types consist of a signed integer giving a length N followed by N bytes of content. A length of -1 indicates null. string uses an int16 for its size, and bytes uses an int32.

+ +

Arrays

+ +

This is a notation for handling repeated structures. These will always be encoded as an int32 size containing the length N followed by N repetitions of the structure which can itself be made up of other primitive types. In the BNF grammars below we will show an array of a structure foo as [foo].

+ +
Notes on reading the request format grammars
+ +

The BNFs below give an exact context free grammar for the request and response binary format. The BNF is intentionally not compact in order to give human-readable name. As always in a BNF a sequence of productions indicates concatenation. When there are multiple possible productions these are separated with '|' and may be enclosed in parenthesis for grouping. The top-level definition is always given first and subsequent sub-parts are indented.

+ +
Common Request and Response Structure
+ +

All requests and responses originate from the following grammar which will be incrementally describe through the rest of this document:

+ +
+RequestOrResponse => Size (RequestMessage | ResponseMessage)
+Size => int32
+
+ + + + +
FieldDescription
message_sizeThe message_size field gives the size of the subsequent request or response message in bytes. The client can read requests by first reading this 4 byte size as an integer N, and then reading and parsing the subsequent N bytes of the request.
+ +
Message Sets
+ +

A description of the message set format can be found here. (KAFKA-3368)

+ +

Constants

+ +
Error Codes
+

We use numeric codes to indicate what problem occurred on the server. These can be translated by the client into exceptions or whatever the appropriate error handling mechanism in the client language. Here is a table of the error codes currently in use:

+ + +
Api Keys
+

The following are the numeric codes that the ApiKey in the request can take for each of the below request types.

+ + +

The Messages

+ +

This section gives details on each of the individual API Messages, their usage, their binary format, and the meaning of their fields.

+ + +

Some Common Philosophical Questions

+ +

Some people have asked why we don't use HTTP. There are a number of reasons, the best is that client implementors can make use of some of the more advanced TCP features--the ability to multiplex requests, the ability to simultaneously poll many connections, etc. We have also found HTTP libraries in many languages to be surprisingly shabby.

+ +

Others have asked if maybe we shouldn't support many different protocols. Prior experience with this was that it makes it very hard to add and test new features if they have to be ported across many protocol implementations. Our feeling is that most users don't really see multiple protocols as a feature, they just want a good reliable client in the language of their choice.

+ +

Another question is why we don't adopt XMPP, STOMP, AMQP or an existing protocol. The answer to this varies by protocol, but in general the problem is that the protocol does determine large parts of the implementation and we couldn't do what we are doing if we didn't have control over the protocol. Our belief is that it is possible to do better than existing messaging systems have in providing a truly distributed messaging system, and to do this we need to build something that works differently.

+ +

A final question is why we don't use a system like Protocol Buffers or Thrift to define our request messages. These packages excel at helping you to managing lots and lots of serialized messages. However we have only a few messages. Support across languages is somewhat spotty (depending on the package). Finally the mapping between binary log format and wire protocol is something we manage somewhat carefully and this would not be possible with these systems. Finally we prefer the style of versioning APIs explicitly and checking this to inferring new values as nulls as it allows more nuanced control of compatibility.

+ + From df41bc544aea91fd1e2d5258ebf1b99347700731 Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Wed, 9 Mar 2016 21:55:05 -0800 Subject: [PATCH 012/206] MINOR: KAFKA-3361 follow up Author: Grant Henke Reviewers: Gwen Shapira Closes #1037 from granthenke/protocol-minor --- build.gradle | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/build.gradle b/build.gradle index 6375bfbaa8d0c..c2bd2288504ab 100644 --- a/build.gradle +++ b/build.gradle @@ -352,14 +352,14 @@ project(':core') { task genProtocolErrorDocs(type: JavaExec) { classpath = sourceSets.main.runtimeClasspath - main = 'org.apache.kafka.common.protocol.Protocol' + main = 'org.apache.kafka.common.protocol.Errors' if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() } standardOutput = new File(generatedDocsDir, "protocol_errors.html").newOutputStream() } task genProtocolApiKeyDocs(type: JavaExec) { classpath = sourceSets.main.runtimeClasspath - main = 'org.apache.kafka.common.protocol.Protocol' + main = 'org.apache.kafka.common.protocol.ApiKeys' if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() } standardOutput = new File(generatedDocsDir, "protocol_api_keys.html").newOutputStream() } @@ -752,7 +752,7 @@ project(':connect:runtime') { dependsOn copyDependantLibs } - tasks.create(name: "genConnectConfigDocs", dependsOn:jar, type: JavaExec) { + task genConnectConfigDocs(type: JavaExec) { classpath = sourceSets.main.runtimeClasspath main = 'org.apache.kafka.connect.runtime.distributed.DistributedConfig' if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() } From 83f714bae11f9bca0f1cedb8a68e73a1dd637ade Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Wed, 9 Mar 2016 23:55:22 -0800 Subject: [PATCH 013/206] KAFKA-3344: Remove previous system test's leftover test-log4j.properties Author: Ashish Singh Reviewers: Ewen Cheslack-Postava Closes #1024 from SinghAsDev/KAFKA-3344 --- config/test-log4j.properties | 66 ------------------------------------ 1 file changed, 66 deletions(-) delete mode 100644 config/test-log4j.properties diff --git a/config/test-log4j.properties b/config/test-log4j.properties deleted file mode 100644 index e0bbc134233c9..0000000000000 --- a/config/test-log4j.properties +++ /dev/null @@ -1,66 +0,0 @@ -# Licensed to the Apache Software Foundation (ASF) under one or more -# contributor license agreements. See the NOTICE file distributed with -# this work for additional information regarding copyright ownership. -# The ASF licenses this file to You under the Apache License, Version 2.0 -# (the "License"); you may not use this file except in compliance with -# the License. You may obtain a copy of the License at -# -# http://www.apache.org/licenses/LICENSE-2.0 -# -# Unless required by applicable law or agreed to in writing, software -# distributed under the License is distributed on an "AS IS" BASIS, -# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. -# See the License for the specific language governing permissions and -# limitations under the License. -log4j.rootLogger=INFO, stdout - -log4j.appender.stdout=org.apache.log4j.ConsoleAppender -log4j.appender.stdout.layout=org.apache.log4j.PatternLayout -log4j.appender.stdout.layout.ConversionPattern=[%d] %p %m (%c)%n - -log4j.appender.kafkaAppender=org.apache.log4j.DailyRollingFileAppender -log4j.appender.kafkaAppender.DatePattern='.'yyyy-MM-dd-HH -log4j.appender.kafkaAppender.File=logs/server.log -log4j.appender.kafkaAppender.layout=org.apache.log4j.PatternLayout -log4j.appender.kafkaAppender.layout.ConversionPattern=[%d] %p %m (%c)%n - -log4j.appender.stateChangeAppender=org.apache.log4j.DailyRollingFileAppender -log4j.appender.stateChangeAppender.DatePattern='.'yyyy-MM-dd-HH -log4j.appender.stateChangeAppender.File=logs/state-change.log -log4j.appender.stateChangeAppender.layout=org.apache.log4j.PatternLayout -log4j.appender.stateChangeAppender.layout.ConversionPattern=[%d] %p %m (%c)%n - -log4j.appender.requestAppender=org.apache.log4j.DailyRollingFileAppender -log4j.appender.requestAppender.DatePattern='.'yyyy-MM-dd-HH -log4j.appender.requestAppender.File=logs/kafka-request.log -log4j.appender.requestAppender.layout=org.apache.log4j.PatternLayout -log4j.appender.requestAppender.layout.ConversionPattern=[%d] %p %m (%c)%n - -log4j.appender.controllerAppender=org.apache.log4j.DailyRollingFileAppender -log4j.appender.controllerAppender.DatePattern='.'yyyy-MM-dd-HH -log4j.appender.controllerAppender.File=logs/controller.log -log4j.appender.controllerAppender.layout=org.apache.log4j.PatternLayout -log4j.appender.controllerAppender.layout.ConversionPattern=[%d] %p %m (%c)%n - -# Turn on all our debugging info -#log4j.logger.kafka.producer.async.DefaultEventHandler=DEBUG, kafkaAppender -#log4j.logger.kafka.client.ClientUtils=DEBUG, kafkaAppender -log4j.logger.kafka.tools=DEBUG, kafkaAppender -log4j.logger.kafka.tools.ProducerPerformance$ProducerThread=DEBUG, kafkaAppender -#log4j.logger.org.I0Itec.zkclient.ZkClient=DEBUG -log4j.logger.kafka=INFO, kafkaAppender - -log4j.logger.kafka.network.RequestChannel$=TRACE, requestAppender -log4j.additivity.kafka.network.RequestChannel$=false - -#log4j.logger.kafka.network.Processor=TRACE, requestAppender -#log4j.logger.kafka.server.KafkaApis=TRACE, requestAppender -#log4j.additivity.kafka.server.KafkaApis=false -log4j.logger.kafka.request.logger=TRACE, requestAppender -log4j.additivity.kafka.request.logger=false - -log4j.logger.kafka.controller=TRACE, controllerAppender -log4j.additivity.kafka.controller=false - -log4j.logger.state.change.logger=TRACE, stateChangeAppender -log4j.additivity.state.change.logger=false From 1a451b4812ae2502bb6bc105e77eb00175a76ec5 Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Thu, 10 Mar 2016 10:44:58 -0800 Subject: [PATCH 014/206] MINOR: Add header and footer to protocol docs Because protocol.html is going to be in its own page it needs the header and footer included. Author: Grant Henke Reviewers: Gwen Shapira Closes #1043 from granthenke/protocol-docs-style --- docs/protocol.html | 21 ++++++++++++++++++++- 1 file changed, 20 insertions(+), 1 deletion(-) diff --git a/docs/protocol.html b/docs/protocol.html index 98923aad53405..cb359f17f6d70 100644 --- a/docs/protocol.html +++ b/docs/protocol.html @@ -1,3 +1,22 @@ + + + +

Kafka Wire Protocol

This document covers the wire protocol implemented in Kafka. It is meant to give a readable guide to the protocol that covers the available requests, their binary format, and the proper way to make use of them to implement a client. This document assumes you understand the basic design and terminology described here

@@ -160,4 +179,4 @@

Some Common Philosop

A final question is why we don't use a system like Protocol Buffers or Thrift to define our request messages. These packages excel at helping you to managing lots and lots of serialized messages. However we have only a few messages. Support across languages is somewhat spotty (depending on the package). Finally the mapping between binary log format and wire protocol is something we manage somewhat carefully and this would not be possible with these systems. Finally we prefer the style of versioning APIs explicitly and checking this to inferring new values as nulls as it allows more nuanced control of compatibility.

- + From 6eb061fa85de1b5346eb2652622c9c60f7f3baf1 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Thu, 10 Mar 2016 11:12:07 -0800 Subject: [PATCH 015/206] MINOR: Fix system test broken by change of consumer group tool output format MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit … format Author: Gwen Shapira Reviewers: Ewen Cheslack-Postava Closes #1039 from gwenshap/minor-consumer-groups --- tests/kafkatest/services/kafka/kafka.py | 2 +- tests/kafkatest/tests/consumer_group_command_test.py | 3 ++- 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index b9105df544dc6..358dacfb26a7e 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -458,7 +458,7 @@ def describe_consumer_group(self, group, node=None, new_consumer=False, command_ output = "" self.logger.debug(cmd) for line in node.account.ssh_capture(cmd): - if not (line.startswith("SLF4J") or line.startswith("GROUP, TOPIC") or line.startswith("Could not fetch offset")): + if not (line.startswith("SLF4J") or line.startswith("GROUP") or line.startswith("Could not fetch offset")): output += line self.logger.debug(output) return output diff --git a/tests/kafkatest/tests/consumer_group_command_test.py b/tests/kafkatest/tests/consumer_group_command_test.py index a7b43a15b9dfb..1424d96ef16e3 100644 --- a/tests/kafkatest/tests/consumer_group_command_test.py +++ b/tests/kafkatest/tests/consumer_group_command_test.py @@ -24,6 +24,7 @@ from kafkatest.services.security.security_config import SecurityConfig import os +import re TOPIC = "topic-consumer-group-command" @@ -80,7 +81,7 @@ def setup_and_verify(self, security_protocol, group=None): command_config_file = self.COMMAND_CONFIG_FILE if group: - wait_until(lambda: ("%s, topic-consumer-group-command, 0," % group) in self.kafka.describe_consumer_group(group=group, node=kafka_node, new_consumer=enable_new_consumer, command_config=command_config_file), timeout_sec=10, + wait_until(lambda: re.search("%s\s+topic-consumer-group-command\s+0"%group,self.kafka.describe_consumer_group(group=group, node=kafka_node, new_consumer=enable_new_consumer, command_config=command_config_file)), timeout_sec=10, err_msg="Timed out waiting to list expected consumer groups.") else: wait_until(lambda: "test-consumer-group" in self.kafka.list_consumer_groups(node=kafka_node, new_consumer=enable_new_consumer, command_config=command_config_file), timeout_sec=10, From e403b3c4bf8ca308fe180b093da20700f4db73c5 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Thu, 10 Mar 2016 11:29:08 -0800 Subject: [PATCH 016/206] KAFKA-3318: clean up consumer logging and error messages Author: Jason Gustafson Reviewers: Ismael Juma Closes #1036 from hachikuji/KAFKA-3318 --- .../internals/AbstractCoordinator.java | 120 +++++++++--------- .../internals/ConsumerCoordinator.java | 81 ++++++------ .../clients/consumer/internals/Fetcher.java | 6 +- .../java/org/apache/kafka/common/Node.java | 2 +- .../apache/kafka/common/protocol/Errors.java | 10 ++ 5 files changed, 115 insertions(+), 104 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index c6492bc66d2a6..c79d8e7d30393 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -174,7 +174,7 @@ protected abstract void onJoinComplete(int generation, */ public void ensureCoordinatorKnown() { while (coordinatorUnknown()) { - RequestFuture future = sendGroupMetadataRequest(); + RequestFuture future = sendGroupCoordinatorRequest(); client.poll(future); if (future.failed()) { @@ -216,7 +216,7 @@ public void ensureActiveGroup() { continue; } - RequestFuture future = performGroupJoin(); + RequestFuture future = sendJoinGroupRequest(); client.poll(future); if (future.succeeded()) { @@ -299,12 +299,12 @@ public void onFailure(RuntimeException e) { * elected leader by the coordinator. * @return A request future which wraps the assignment returned from the group leader */ - private RequestFuture performGroupJoin() { + private RequestFuture sendJoinGroupRequest() { if (coordinatorUnknown()) return RequestFuture.coordinatorNotAvailable(); // send a join group request to the coordinator - log.debug("(Re-)joining group {}", groupId); + log.info("(Re-)joining group {}", groupId); JoinGroupRequest request = new JoinGroupRequest( groupId, this.sessionTimeoutMs, @@ -312,8 +312,7 @@ private RequestFuture performGroupJoin() { protocolType(), metadata()); - // create the request for the coordinator - log.debug("Issuing request ({}: {}) to coordinator {}", ApiKeys.JOIN_GROUP, request, this.coordinator.id()); + log.debug("Sending JoinGroup ({}) to coordinator {}", request, this.coordinator); return client.send(coordinator, ApiKeys.JOIN_GROUP, request) .compose(new JoinGroupResponseHandler()); } @@ -328,10 +327,9 @@ public JoinGroupResponse parse(ClientResponse response) { @Override public void handle(JoinGroupResponse joinResponse, RequestFuture future) { - // process the response - short errorCode = joinResponse.errorCode(); - if (errorCode == Errors.NONE.code()) { - log.debug("Joined group: {}", joinResponse.toStruct()); + Errors error = Errors.forCode(joinResponse.errorCode()); + if (error == Errors.NONE) { + log.debug("Received successful join group response for group {}: {}", groupId, joinResponse.toStruct()); AbstractCoordinator.this.memberId = joinResponse.memberId(); AbstractCoordinator.this.generation = joinResponse.generationId(); AbstractCoordinator.this.rejoinNeeded = false; @@ -342,37 +340,33 @@ public void handle(JoinGroupResponse joinResponse, RequestFuture fut } else { onJoinFollower().chain(future); } - } else if (errorCode == Errors.GROUP_LOAD_IN_PROGRESS.code()) { - log.debug("Attempt to join group {} rejected since coordinator is loading the group.", groupId); + } else if (error == Errors.GROUP_LOAD_IN_PROGRESS) { + log.debug("Attempt to join group {} rejected since coordinator {} is loading the group.", groupId, + coordinator); // backoff and retry - future.raise(Errors.forCode(errorCode)); - } else if (errorCode == Errors.UNKNOWN_MEMBER_ID.code()) { + future.raise(error); + } else if (error == Errors.UNKNOWN_MEMBER_ID) { // reset the member id and retry immediately AbstractCoordinator.this.memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID; - log.info("Attempt to join group {} failed due to unknown member id, resetting and retrying.", - groupId); + log.debug("Attempt to join group {} failed due to unknown member id.", groupId); future.raise(Errors.UNKNOWN_MEMBER_ID); - } else if (errorCode == Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code() - || errorCode == Errors.NOT_COORDINATOR_FOR_GROUP.code()) { + } else if (error == Errors.GROUP_COORDINATOR_NOT_AVAILABLE + || error == Errors.NOT_COORDINATOR_FOR_GROUP) { // re-discover the coordinator and retry with backoff coordinatorDead(); - log.info("Attempt to join group {} failed due to obsolete coordinator information, retrying.", - groupId); - future.raise(Errors.forCode(errorCode)); - } else if (errorCode == Errors.INCONSISTENT_GROUP_PROTOCOL.code() - || errorCode == Errors.INVALID_SESSION_TIMEOUT.code() - || errorCode == Errors.INVALID_GROUP_ID.code()) { + log.debug("Attempt to join group {} failed due to obsolete coordinator information: {}", groupId, error.message()); + future.raise(error); + } else if (error == Errors.INCONSISTENT_GROUP_PROTOCOL + || error == Errors.INVALID_SESSION_TIMEOUT + || error == Errors.INVALID_GROUP_ID) { // log the error and re-throw the exception - Errors error = Errors.forCode(errorCode); - log.error("Attempt to join group {} failed due to: {}", - groupId, error.exception().getMessage()); + log.error("Attempt to join group {} failed due to fatal error: {}", groupId, error.message()); future.raise(error); - } else if (errorCode == Errors.GROUP_AUTHORIZATION_FAILED.code()) { + } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) { future.raise(new GroupAuthorizationException(groupId)); } else { // unexpected error, throw the exception - future.raise(new KafkaException("Unexpected error in join group response: " - + Errors.forCode(joinResponse.errorCode()).exception().getMessage())); + future.raise(new KafkaException("Unexpected error in join group response: " + error.message())); } } } @@ -381,7 +375,7 @@ private RequestFuture onJoinFollower() { // send follower's sync group with an empty assignment SyncGroupRequest request = new SyncGroupRequest(groupId, generation, memberId, Collections.emptyMap()); - log.debug("Issuing follower SyncGroup ({}: {}) to coordinator {}", ApiKeys.SYNC_GROUP, request, this.coordinator.id()); + log.debug("Sending follower SyncGroup for group {} to coordinator {}: {}", groupId, this.coordinator, request); return sendSyncGroupRequest(request); } @@ -392,7 +386,7 @@ private RequestFuture onJoinLeader(JoinGroupResponse joinResponse) { joinResponse.members()); SyncGroupRequest request = new SyncGroupRequest(groupId, generation, memberId, groupAssignment); - log.debug("Issuing leader SyncGroup ({}: {}) to coordinator {}", ApiKeys.SYNC_GROUP, request, this.coordinator.id()); + log.debug("Sending leader SyncGroup for group {} to coordinator {}: {}", groupId, this.coordinator, request); return sendSyncGroupRequest(request); } catch (RuntimeException e) { return RequestFuture.failure(e); @@ -418,7 +412,7 @@ public void handle(SyncGroupResponse syncResponse, RequestFuture future) { Errors error = Errors.forCode(syncResponse.errorCode()); if (error == Errors.NONE) { - log.debug("Received successful sync group response for group {}: {}", groupId, syncResponse.toStruct()); + log.info("Successfully joined group {} with generation {}", groupId, generation); sensors.syncLatency.record(response.requestLatencyMs()); future.complete(syncResponse.memberAssignment()); } else { @@ -426,20 +420,20 @@ public void handle(SyncGroupResponse syncResponse, if (error == Errors.GROUP_AUTHORIZATION_FAILED) { future.raise(new GroupAuthorizationException(groupId)); } else if (error == Errors.REBALANCE_IN_PROGRESS) { - log.info("SyncGroup for group {} failed due to coordinator rebalance, rejoining the group", groupId); + log.debug("SyncGroup for group {} failed due to coordinator rebalance", groupId); future.raise(error); } else if (error == Errors.UNKNOWN_MEMBER_ID || error == Errors.ILLEGAL_GENERATION) { - log.info("SyncGroup for group {} failed due to {}, rejoining the group", groupId, error); + log.debug("SyncGroup for group {} failed due to {}", groupId, error); AbstractCoordinator.this.memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID; future.raise(error); } else if (error == Errors.GROUP_COORDINATOR_NOT_AVAILABLE || error == Errors.NOT_COORDINATOR_FOR_GROUP) { - log.info("SyncGroup for group {} failed due to {}, will find new coordinator and rejoin", groupId, error); + log.debug("SyncGroup for group {} failed due to {}", groupId, error); coordinatorDead(); future.raise(error); } else { - future.raise(new KafkaException("Unexpected error from SyncGroup: " + error.exception().getMessage())); + future.raise(new KafkaException("Unexpected error from SyncGroup: " + error.message())); } } } @@ -450,7 +444,7 @@ public void handle(SyncGroupResponse syncResponse, * one of the brokers. The returned future should be polled to get the result of the request. * @return A request future which indicates the completion of the metadata request */ - private RequestFuture sendGroupMetadataRequest() { + private RequestFuture sendGroupCoordinatorRequest() { // initiate the group metadata request // find a node to ask about the coordinator Node node = this.client.leastLoadedNode(); @@ -460,7 +454,7 @@ private RequestFuture sendGroupMetadataRequest() { return RequestFuture.noBrokersAvailable(); } else { // create a group metadata request - log.debug("Issuing group metadata request to broker {}", node.id()); + log.debug("Sending coordinator request for group {} to broker {}", groupId, node); GroupCoordinatorRequest metadataRequest = new GroupCoordinatorRequest(this.groupId); return client.send(node, ApiKeys.GROUP_COORDINATOR, metadataRequest) .compose(new RequestFutureAdapter() { @@ -473,7 +467,7 @@ public void onSuccess(ClientResponse response, RequestFuture future) { } private void handleGroupMetadataResponse(ClientResponse resp, RequestFuture future) { - log.debug("Group metadata response {}", resp); + log.debug("Received group coordinator response {}", resp); if (!coordinatorUnknown()) { // We already found the coordinator, so ignore the request @@ -483,22 +477,24 @@ private void handleGroupMetadataResponse(ClientResponse resp, RequestFuture 0) heartbeatTask.reset(); future.complete(null); - } else if (errorCode == Errors.GROUP_AUTHORIZATION_FAILED.code()) { + } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) { future.raise(new GroupAuthorizationException(groupId)); } else { - future.raise(Errors.forCode(errorCode)); + future.raise(error); } } } @@ -524,7 +520,7 @@ public boolean coordinatorUnknown() { */ protected void coordinatorDead() { if (this.coordinator != null) { - log.info("Marking the coordinator {} dead.", this.coordinator.id()); + log.info("Marking the coordinator {} dead for group {}", this.coordinator, groupId); this.coordinator = null; } } @@ -566,7 +562,7 @@ public void onSuccess(Void value) {} @Override public void onFailure(RuntimeException e) { - log.info("LeaveGroup request failed with error", e); + log.debug("LeaveGroup request for group {} failed with error", groupId, e); } }); @@ -608,33 +604,33 @@ public HeartbeatResponse parse(ClientResponse response) { @Override public void handle(HeartbeatResponse heartbeatResponse, RequestFuture future) { sensors.heartbeatLatency.record(response.requestLatencyMs()); - short errorCode = heartbeatResponse.errorCode(); - if (errorCode == Errors.NONE.code()) { - log.debug("Received successful heartbeat response."); + Errors error = Errors.forCode(heartbeatResponse.errorCode()); + if (error == Errors.NONE) { + log.debug("Received successful heartbeat response for group {}", groupId); future.complete(null); - } else if (errorCode == Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code() - || errorCode == Errors.NOT_COORDINATOR_FOR_GROUP.code()) { - log.info("Attempt to heart beat failed since coordinator is either not started or not valid, marking it as dead."); + } else if (error == Errors.GROUP_COORDINATOR_NOT_AVAILABLE + || error == Errors.NOT_COORDINATOR_FOR_GROUP) { + log.debug("Attempt to heart beat failed for group {} since coordinator {} is either not started or not valid.", + groupId, coordinator); coordinatorDead(); - future.raise(Errors.forCode(errorCode)); - } else if (errorCode == Errors.REBALANCE_IN_PROGRESS.code()) { - log.info("Attempt to heart beat failed since the group is rebalancing, try to re-join group."); + future.raise(error); + } else if (error == Errors.REBALANCE_IN_PROGRESS) { + log.debug("Attempt to heart beat failed for group {} since it is rebalancing.", groupId); AbstractCoordinator.this.rejoinNeeded = true; future.raise(Errors.REBALANCE_IN_PROGRESS); - } else if (errorCode == Errors.ILLEGAL_GENERATION.code()) { - log.info("Attempt to heart beat failed since generation id is not legal, try to re-join group."); + } else if (error == Errors.ILLEGAL_GENERATION) { + log.debug("Attempt to heart beat failed for group {} since generation id is not legal.", groupId); AbstractCoordinator.this.rejoinNeeded = true; future.raise(Errors.ILLEGAL_GENERATION); - } else if (errorCode == Errors.UNKNOWN_MEMBER_ID.code()) { - log.info("Attempt to heart beat failed since member id is not valid, reset it and try to re-join group."); + } else if (error == Errors.UNKNOWN_MEMBER_ID) { + log.debug("Attempt to heart beat failed for group {} since member id is not valid.", groupId); memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID; AbstractCoordinator.this.rejoinNeeded = true; future.raise(Errors.UNKNOWN_MEMBER_ID); - } else if (errorCode == Errors.GROUP_AUTHORIZATION_FAILED.code()) { + } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) { future.raise(new GroupAuthorizationException(groupId)); } else { - future.raise(new KafkaException("Unexpected errorCode in heartbeat response: " - + Errors.forCode(errorCode).exception().getMessage())); + future.raise(new KafkaException("Unexpected error in heartbeat response: " + error.message())); } } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index aa39e11929116..b6b46c135a579 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -188,15 +188,15 @@ protected void onJoinComplete(int generation, // execute the user's callback after rebalance ConsumerRebalanceListener listener = subscriptions.listener(); - log.debug("Setting newly assigned partitions {}", subscriptions.assignedPartitions()); + log.info("Setting newly assigned partitions {} for group {}", subscriptions.assignedPartitions(), groupId); try { Set assigned = new HashSet<>(subscriptions.assignedPartitions()); listener.onPartitionsAssigned(assigned); } catch (WakeupException e) { throw e; } catch (Exception e) { - log.error("User provided listener " + listener.getClass().getName() - + " failed on partition assignment: ", e); + log.error("User provided listener {} for group {} failed on partition assignment", + listener.getClass().getName(), groupId, e); } } @@ -222,11 +222,12 @@ protected Map performAssignment(String leaderId, metadata.setTopics(this.subscriptions.groupSubscription()); client.ensureFreshMetadata(); - log.debug("Performing {} assignment for subscriptions {}", assignor.name(), subscriptions); + log.debug("Performing assignment for group {} using strategy {} with subscriptions {}", + groupId, assignor.name(), subscriptions); Map assignment = assignor.assign(metadata.fetch(), subscriptions); - log.debug("Finished assignment: {}", assignment); + log.debug("Finished assignment for group {}: {}", groupId, assignment); Map groupAssignment = new HashMap<>(); for (Map.Entry assignmentEntry : assignment.entrySet()) { @@ -244,15 +245,15 @@ protected void onJoinPrepare(int generation, String memberId) { // execute the user's callback before rebalance ConsumerRebalanceListener listener = subscriptions.listener(); - log.debug("Revoking previously assigned partitions {}", subscriptions.assignedPartitions()); + log.info("Revoking previously assigned partitions {} for group {}", subscriptions.assignedPartitions(), groupId); try { Set revoked = new HashSet<>(subscriptions.assignedPartitions()); listener.onPartitionsRevoked(revoked); } catch (WakeupException e) { throw e; } catch (Exception e) { - log.error("User provided listener " + listener.getClass().getName() - + " failed on partition revocation: ", e); + log.error("User provided listener {} for group {} failed on partition revocation", + listener.getClass().getName(), groupId, e); } subscriptions.needReassignment(); @@ -410,7 +411,7 @@ public void run(final long now) { return; if (coordinatorUnknown()) { - log.debug("Cannot auto-commit offsets now since the coordinator is unknown, will retry after backoff"); + log.debug("Cannot auto-commit offsets for group {} since the coordinator is unknown", groupId); client.schedule(this, now + retryBackoffMs); return; } @@ -423,10 +424,10 @@ public void onComplete(Map offsets, Exception if (exception == null) { reschedule(now + interval); } else if (exception instanceof SendFailedException) { - log.debug("Failed to send automatic offset commit, will retry immediately"); + log.debug("Failed to send automatic offset commit for group {}", groupId); reschedule(now); } else { - log.warn("Auto offset commit failed: {}", exception.getMessage()); + log.warn("Auto offset commit failed for group {}: {}", groupId, exception.getMessage()); reschedule(now + interval); } } @@ -447,7 +448,7 @@ private void maybeAutoCommitOffsetsSync() { throw e; } catch (Exception e) { // consistent with async auto-commit failures, we do not propagate the exception - log.warn("Auto offset commit failed: ", e.getMessage()); + log.warn("Auto offset commit failed for group {}: {}", groupId, e.getMessage()); } } } @@ -481,7 +482,7 @@ private RequestFuture sendOffsetCommitRequest(final Map futu Errors error = Errors.forCode(entry.getValue()); if (error == Errors.NONE) { - log.debug("Committed offset {} for partition {}", offset, tp); + log.debug("Group {} committed offset {} for partition {}", groupId, offset, tp); if (subscriptions.isAssigned(tp)) // update the local cache only if the partition is still assigned subscriptions.committed(tp, offsetAndMetadata); } else if (error == Errors.GROUP_AUTHORIZATION_FAILED) { - log.error("Unauthorized to commit for group {}", groupId); + log.error("Not authorized to commit offsets for group {}", groupId); future.raise(new GroupAuthorizationException(groupId)); return; } else if (error == Errors.TOPIC_AUTHORIZATION_FAILED) { @@ -533,18 +534,18 @@ public void handle(OffsetCommitResponse commitResponse, RequestFuture futu } else if (error == Errors.OFFSET_METADATA_TOO_LARGE || error == Errors.INVALID_COMMIT_OFFSET_SIZE) { // raise the error to the user - log.info("Offset commit for group {} failed on partition {} due to {}, will retry", groupId, tp, error); + log.debug("Offset commit for group {} failed on partition {}: {}", groupId, tp, error.message()); future.raise(error); return; } else if (error == Errors.GROUP_LOAD_IN_PROGRESS) { // just retry - log.info("Offset commit for group {} failed due to {}, will retry", groupId, error); + log.debug("Offset commit for group {} failed: {}", groupId, error.message()); future.raise(error); return; } else if (error == Errors.GROUP_COORDINATOR_NOT_AVAILABLE || error == Errors.NOT_COORDINATOR_FOR_GROUP || error == Errors.REQUEST_TIMED_OUT) { - log.info("Offset commit for group {} failed due to {}, will find new coordinator and retry", groupId, error); + log.debug("Offset commit for group {} failed: {}", groupId, error.message()); coordinatorDead(); future.raise(error); return; @@ -552,19 +553,24 @@ public void handle(OffsetCommitResponse commitResponse, RequestFuture futu || error == Errors.ILLEGAL_GENERATION || error == Errors.REBALANCE_IN_PROGRESS) { // need to re-join group - log.error("Error {} occurred while committing offsets for group {}", error, groupId); + log.debug("Offset commit for group {} failed: {}", groupId, error.message()); subscriptions.needReassignment(); - future.raise(new CommitFailedException("Commit cannot be completed due to group rebalance")); + future.raise(new CommitFailedException("Commit cannot be completed since the group has already " + + "rebalanced and assigned the partitions to another member. This means that the time " + + "between subsequent calls to poll() was longer than the configured session.timeout.ms, " + + "which typically implies that the poll loop is spending too much time message processing. " + + "You can address this either by increasing the session timeout or by reducing the maximum " + + "size of batches returned in poll() with max.poll.records.")); return; } else { - log.error("Error committing partition {} at offset {}: {}", tp, offset, error.exception().getMessage()); - future.raise(new KafkaException("Unexpected error in commit: " + error.exception().getMessage())); + log.error("Group {} failed to commit partition {} at offset {}: {}", groupId, tp, offset, error.message()); + future.raise(new KafkaException("Unexpected error in commit: " + error.message())); return; } } if (!unauthorizedTopics.isEmpty()) { - log.error("Unauthorized to commit to topics {}", unauthorizedTopics); + log.error("Not authorized to commit to topics {} for group {}", unauthorizedTopics, groupId); future.raise(new TopicAuthorizationException(unauthorizedTopics)); } else { future.complete(null); @@ -583,9 +589,9 @@ private RequestFuture> sendOffsetFetchReq if (coordinatorUnknown()) return RequestFuture.coordinatorNotAvailable(); - log.debug("Fetching committed offsets for partitions: {}", partitions); + log.debug("Group {} fetching committed offsets for partitions: {}", groupId, partitions); // construct the request - OffsetFetchRequest request = new OffsetFetchRequest(this.groupId, new ArrayList(partitions)); + OffsetFetchRequest request = new OffsetFetchRequest(this.groupId, new ArrayList<>(partitions)); // send the request with a callback return client.send(coordinator, ApiKeys.OFFSET_FETCH, request) @@ -606,31 +612,30 @@ public void handle(OffsetFetchResponse response, RequestFuture= 0) { // record the position with the offset (-1 indicates no committed offset to fetch) offsets.put(tp, new OffsetAndMetadata(data.offset, data.metadata)); } else { - log.debug("No committed offset for partition " + tp); + log.debug("Group {} has no committed offset for partition {}", groupId, tp); } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 7a1a72090a3a3..b4d5c02a08ac3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -501,12 +501,12 @@ private void handleListOffsetResponse(TopicPartition topicPartition, future.complete(offset); } else if (errorCode == Errors.NOT_LEADER_FOR_PARTITION.code() || errorCode == Errors.UNKNOWN_TOPIC_OR_PARTITION.code()) { - log.warn("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.", + log.debug("Attempt to fetch offsets for partition {} failed due to obsolete leadership information, retrying.", topicPartition); future.raise(Errors.forCode(errorCode)); } else { - log.error("Attempt to fetch offsets for partition {} failed due to: {}", - topicPartition, Errors.forCode(errorCode).exception().getMessage()); + log.warn("Attempt to fetch offsets for partition {} failed due to: {}", + topicPartition, Errors.forCode(errorCode).message()); future.raise(new StaleMetadataException()); } } diff --git a/clients/src/main/java/org/apache/kafka/common/Node.java b/clients/src/main/java/org/apache/kafka/common/Node.java index 644cd71c8cb4f..24cf6f4616f7a 100644 --- a/clients/src/main/java/org/apache/kafka/common/Node.java +++ b/clients/src/main/java/org/apache/kafka/common/Node.java @@ -96,7 +96,7 @@ public boolean equals(Object obj) { @Override public String toString() { - return "Node(" + id + ", " + host + ", " + port + ")"; + return host + ":" + port + " (id: " + idString + ")"; } } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index ab299af47486c..90be0144109af 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -182,6 +182,16 @@ public void maybeThrow() { } } + /** + * Get a friendly description of the error (if one is available). + * @return the error message + */ + public String message() { + if (exception != null) + return exception.getMessage(); + return toString(); + } + /** * Throw the exception if there is one */ From 04ef9c35479f5cf92df3b74a7379ce77dc809243 Mon Sep 17 00:00:00 2001 From: Eno Thereska Date: Thu, 10 Mar 2016 12:13:04 -0800 Subject: [PATCH 017/206] MINOR: Increased default EC2 instance size AWS instance size increased to m3.xlarge to allow all system tests to pass. ijuma ewencp have a look please. Author: Eno Thereska Reviewers: Gwen Shapira Closes #1046 from enothereska/minor-aws --- vagrant/aws/aws-example-Vagrantfile.local | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/vagrant/aws/aws-example-Vagrantfile.local b/vagrant/aws/aws-example-Vagrantfile.local index 176a33dcae39b..00b3d6164ee71 100644 --- a/vagrant/aws/aws-example-Vagrantfile.local +++ b/vagrant/aws/aws-example-Vagrantfile.local @@ -17,7 +17,7 @@ # Use this template Vagrantfile.local for running system tests on aws # To use it, move it to the base kafka directory and rename # it to Vagrantfile.local, and adjust variables as needed. -ec2_instance_type = "m3.medium" +ec2_instance_type = "m3.xlarge" num_zookeepers = 0 num_brokers = 0 num_workers = 9 From 9c4c5ae1cd15aa0afe3156e572362fbb40130573 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 10 Mar 2016 14:54:47 -0800 Subject: [PATCH 018/206] MINOR: Add unit test for internal topics Author: Guozhang Wang Reviewers: Yasuhiro Matsuda , Ewen Cheslack-Postava Closes #1047 from guozhangwang/KInternal --- .../internals/InternalTopicManager.java | 13 ++-- .../internals/StreamPartitionAssignor.java | 6 +- .../StreamPartitionAssignorTest.java | 66 +++++++++++++++++++ 3 files changed, 80 insertions(+), 5 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java index ce95bb08322ad..3725c4c4b19ca 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java @@ -73,6 +73,11 @@ public Object deserialize(byte[] bytes) { } } + public InternalTopicManager() { + this.zkClient = null; + this.replicationFactor = 0; + } + public InternalTopicManager(String zkConnect, int replicationFactor) { this.zkClient = new ZkClient(zkConnect, 30 * 1000, 30 * 1000, new ZKStringSerializer()); this.replicationFactor = replicationFactor; @@ -125,7 +130,7 @@ private List getBrokers() { } @SuppressWarnings("unchecked") - public Map> getTopicMetadata(String topic) { + private Map> getTopicMetadata(String topic) { String data = zkClient.readData(ZK_TOPIC_PATH + "/" + topic, true); if (data == null) return null; @@ -147,7 +152,7 @@ public Map> getTopicMetadata(String topic) { } } - public void createTopic(String topic, int numPartitions, int replicationFactor) throws ZkNodeExistsException { + private void createTopic(String topic, int numPartitions, int replicationFactor) throws ZkNodeExistsException { log.debug("Creating topic {} with {} partitions from ZK in partition assignor.", topic, numPartitions); List brokers = getBrokers(); @@ -183,13 +188,13 @@ public void createTopic(String topic, int numPartitions, int replicationFactor) } } - public void deleteTopic(String topic) throws ZkNodeExistsException { + private void deleteTopic(String topic) throws ZkNodeExistsException { log.debug("Deleting topic {} from ZK in partition assignor.", topic); zkClient.createPersistent(ZK_DELETE_TOPIC_PATH + "/" + topic, "", ZooDefs.Ids.OPEN_ACL_UNSAFE); } - public void addPartitions(String topic, int numPartitions, int replicationFactor, Map> existingAssignment) { + private void addPartitions(String topic, int numPartitions, int replicationFactor, Map> existingAssignment) { log.debug("Adding {} partitions topic {} from ZK with existing partitions assigned as {} in partition assignor.", topic, numPartitions, existingAssignment); List brokers = getBrokers(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java index 1b3bf101b6540..13f269b5fc9b3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java @@ -231,7 +231,7 @@ public Map assign(Cluster metadata, Map> entry : internalSourceTopicToTaskIds.entrySet()) { - String topic = streamThread.jobId + "-" + entry.getKey(); + String topic = entry.getKey(); // should have size 1 only int numPartitions = -1; @@ -455,4 +455,8 @@ public Set tasksForPartition(TopicPartition partition) { public Map> standbyTasks() { return standbyTasks; } + + public void setInternalTopicManager(InternalTopicManager internalTopicManager) { + this.internalTopicManager = internalTopicManager; + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java index 15b114a141026..9ff0af07af409 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java @@ -506,4 +506,70 @@ public void testOnAssignment() throws Exception { assertEquals(standbyTasks, partitionAssignor.standbyTasks()); } + @Test + public void testAssignWithInternalTopics() throws Exception { + StreamsConfig config = new StreamsConfig(configProps()); + + MockProducer producer = new MockProducer<>(true, serializer, serializer); + MockConsumer consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); + MockConsumer mockRestoreConsumer = new MockConsumer<>(OffsetResetStrategy.LATEST); + + TopologyBuilder builder = new TopologyBuilder(); + builder.addInternalTopic("topicX"); + builder.addSource("source1", "topic1"); + builder.addProcessor("processor1", new MockProcessorSupplier(), "source1"); + builder.addSink("sink1", "topicX", "processor1"); + builder.addSource("source2", "topicX"); + builder.addProcessor("processor2", new MockProcessorSupplier(), "source2"); + List topics = Utils.mkList("topic1", "topicX"); + Set allTasks = Utils.mkSet(task0, task1, task2); + + UUID uuid1 = UUID.randomUUID(); + UUID uuid2 = UUID.randomUUID(); + String client1 = "client1"; + + StreamThread thread10 = new StreamThread(builder, config, producer, consumer, mockRestoreConsumer, "test", client1, uuid1, new Metrics(), new SystemTime()); + + StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor(); + partitionAssignor.configure(config.getConsumerConfigs(thread10, "test", client1)); + MockInternalTopicManager internalTopicManager = new MockInternalTopicManager(mockRestoreConsumer); + partitionAssignor.setInternalTopicManager(internalTopicManager); + + Map subscriptions = new HashMap<>(); + Set emptyTasks = Collections.emptySet(); + subscriptions.put("consumer10", + new PartitionAssignor.Subscription(topics, new SubscriptionInfo(uuid1, emptyTasks, emptyTasks).encode())); + + Map assignments = partitionAssignor.assign(metadata, subscriptions); + + // check prepared internal topics + // TODO: we need to change it to 1 after fixing the prefix + assertEquals(2, internalTopicManager.readyTopics.size()); + assertEquals(allTasks.size(), (long) internalTopicManager.readyTopics.get("topicX")); + assertEquals(allTasks.size(), (long) internalTopicManager.readyTopics.get("test-topicX")); + } + + private class MockInternalTopicManager extends InternalTopicManager { + + public Map readyTopics = new HashMap<>(); + public MockConsumer restoreConsumer; + + public MockInternalTopicManager(MockConsumer restoreConsumer) { + super(); + + this.restoreConsumer = restoreConsumer; + } + + @Override + public void makeReady(String topic, int numPartitions) { + readyTopics.put(topic, numPartitions); + + List partitions = new ArrayList<>(); + for (int i = 0; i < numPartitions; i++) { + partitions.add(new PartitionInfo(topic, i, null, null, null)); + } + + restoreConsumer.updatePartitions(topic, partitions); + } + } } From 287cce251f7d5471946d550031147aaab4e7ca26 Mon Sep 17 00:00:00 2001 From: Manikumar reddy O Date: Fri, 11 Mar 2016 10:56:21 -0800 Subject: [PATCH 019/206] MINOR: update compression design doc to include lz4 protocol Author: Manikumar reddy O Reviewers: Ismael Juma, Guozhang Wang Closes #1040 from omkreddy/MINOR-DOC --- docs/design.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/design.html b/docs/design.html index 7f7ca05314e69..c04b772e2299c 100644 --- a/docs/design.html +++ b/docs/design.html @@ -100,7 +100,7 @@

End-to-end Batch Compr

Kafka supports this by allowing recursive message sets. A batch of messages can be clumped together compressed and sent to the server in this form. This batch of messages will be written in compressed form and will remain compressed in the log and will only be decompressed by the consumer.

-Kafka supports GZIP and Snappy compression protocols. More details on compression can be found here. +Kafka supports GZIP, Snappy and LZ4 compression protocols. More details on compression can be found here.

4.4 The Producer

From 764d8ca9eb0aba6099ba289a10f437e72b53ffec Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Fri, 11 Mar 2016 11:11:59 -0800 Subject: [PATCH 020/206] KAFKA-2073: migrate to client-side topic metadata request/response Author: Jason Gustafson Author: Ismael Juma Author: hachikuji Reviewers: Grant Henke, Ismael Juma, Gwen Shapira, Flavio Junquiera Closes #988 from hachikuji/KAFKA-2073 --- .../apache/kafka/clients/NetworkClient.java | 12 +- .../clients/consumer/internals/Fetcher.java | 7 +- .../java/org/apache/kafka/common/Node.java | 13 +- .../common/requests/MetadataRequest.java | 20 +- .../common/requests/MetadataResponse.java | 272 +++++++++++------- .../consumer/internals/FetcherTest.java | 43 +-- .../internals/DefaultPartitionerTest.java | 14 +- .../common/requests/RequestResponseTest.java | 22 +- .../main/scala/kafka/admin/AdminClient.scala | 5 +- .../main/scala/kafka/admin/AdminUtils.scala | 38 +-- .../main/scala/kafka/api/TopicMetadata.scala | 1 - .../kafka/api/TopicMetadataRequest.scala | 16 -- .../src/main/scala/kafka/cluster/Broker.scala | 16 +- .../scala/kafka/network/RequestChannel.scala | 1 - .../main/scala/kafka/server/KafkaApis.scala | 165 ++++++----- .../scala/kafka/server/MetadataCache.scala | 209 ++++++++------ .../scala/kafka/server/ReplicaManager.scala | 2 +- core/src/main/scala/kafka/utils/ZkUtils.scala | 2 +- .../RequestResponseSerializationTest.scala | 13 +- .../integration/BaseTopicMetadataTest.scala | 20 +- .../unit/kafka/producer/ProducerTest.scala | 2 +- .../unit/kafka/server/MetadataCacheTest.scala | 199 +++++++++++++ 22 files changed, 699 insertions(+), 393 deletions(-) create mode 100644 core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index d4c406902111d..4d01cdeb2e27d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -18,6 +18,7 @@ import org.apache.kafka.common.network.Selectable; import org.apache.kafka.common.network.Send; 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.MetadataRequest; @@ -35,6 +36,7 @@ import java.util.ArrayList; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Random; import java.util.Set; @@ -147,6 +149,9 @@ private NetworkClient(MetadataUpdater metadataUpdater, */ @Override public boolean ready(Node node, long now) { + if (node.isEmpty()) + throw new IllegalArgumentException("Cannot connect to empty node " + node); + if (isReady(node, now)) return true; @@ -578,9 +583,10 @@ private void handleResponse(RequestHeader header, Struct body, long now) { MetadataResponse response = new MetadataResponse(body); Cluster cluster = response.cluster(); // check if any topics metadata failed to get updated - if (response.errors().size() > 0) { - log.warn("Error while fetching metadata with correlation id {} : {}", header.correlationId(), response.errors()); - } + Map errors = response.errors(); + if (!errors.isEmpty()) + log.warn("Error while fetching metadata with correlation id {} : {}", header.correlationId(), errors); + // don't update the cluster if there are no valid nodes...the topic we want may still be in the process of being // created which means we will get errors and no nodes until it exists if (cluster.nodes().size() > 0) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index b4d5c02a08ac3..802a2f0b63e7b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -215,13 +215,14 @@ public Map> getTopicMetadata(List topics, lo throw new TopicAuthorizationException(unauthorizedTopics); boolean shouldRetry = false; - if (!response.errors().isEmpty()) { + Map errors = response.errors(); + if (!errors.isEmpty()) { // if there were errors, we need to check whether they were fatal or whether // we should just retry - log.debug("Topic metadata fetch included errors: {}", response.errors()); + log.debug("Topic metadata fetch included errors: {}", errors); - for (Map.Entry errorEntry : response.errors().entrySet()) { + for (Map.Entry errorEntry : errors.entrySet()) { String topic = errorEntry.getKey(); Errors error = errorEntry.getValue(); diff --git a/clients/src/main/java/org/apache/kafka/common/Node.java b/clients/src/main/java/org/apache/kafka/common/Node.java index 24cf6f4616f7a..6c3fd0bab93b7 100644 --- a/clients/src/main/java/org/apache/kafka/common/Node.java +++ b/clients/src/main/java/org/apache/kafka/common/Node.java @@ -17,6 +17,8 @@ */ public class Node { + private static final Node NO_NODE = new Node(-1, "", -1); + private final int id; private final String idString; private final String host; @@ -31,7 +33,16 @@ public Node(int id, String host, int port) { } public static Node noNode() { - return new Node(-1, "", -1); + return NO_NODE; + } + + /** + * Check whether this node is empty, which may be the case if noNode() is used as a placeholder + * in a response payload with an error. + * @return true if it is, false otherwise + */ + public boolean isEmpty() { + return host == null || host.isEmpty() || port < 0; } /** diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java index a6c249fd0bacc..92d8c6dc785a9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java @@ -12,9 +12,7 @@ */ package org.apache.kafka.common.requests; -import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; -import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; import org.apache.kafka.common.protocol.ProtoUtils; @@ -24,9 +22,7 @@ import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; import java.util.List; -import java.util.Map; public class MetadataRequest extends AbstractRequest { @@ -44,7 +40,7 @@ public MetadataRequest(List topics) { public MetadataRequest(Struct struct) { super(struct); Object[] topicArray = struct.getArray(TOPICS_KEY_NAME); - topics = new ArrayList(); + topics = new ArrayList<>(); for (Object topicObj: topicArray) { topics.add((String) topicObj); } @@ -52,16 +48,16 @@ public MetadataRequest(Struct struct) { @Override public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { - Map topicErrors = new HashMap(); - for (String topic : topics) { - topicErrors.put(topic, Errors.forException(e)); - } + List topicMetadatas = new ArrayList<>(); + Errors error = Errors.forException(e); + List partitions = Collections.emptyList(); + + for (String topic : topics) + topicMetadatas.add(new MetadataResponse.TopicMetadata(error, topic, partitions)); - Cluster cluster = new Cluster(Collections.emptyList(), Collections.emptyList(), - Collections.emptySet()); switch (versionId) { case 0: - return new MetadataResponse(cluster, topicErrors); + return new MetadataResponse(Collections.emptyList(), topicMetadatas); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.METADATA.id))); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java index 805b9e713b9da..13e0d8f7c5648 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -12,15 +12,6 @@ */ package org.apache.kafka.common.requests; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.Collections; -import java.util.HashMap; -import java.util.HashSet; -import java.util.List; -import java.util.Map; -import java.util.Set; - import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; @@ -30,11 +21,20 @@ import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; + public class MetadataResponse extends AbstractRequestResponse { private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.METADATA.id); private static final String BROKERS_KEY_NAME = "brokers"; - private static final String TOPIC_METATDATA_KEY_NAME = "topic_metadata"; + private static final String TOPIC_METADATA_KEY_NAME = "topic_metadata"; // broker level field names private static final String NODE_ID_KEY_NAME = "node_id"; @@ -71,18 +71,18 @@ public class MetadataResponse extends AbstractRequestResponse { private static final String REPLICAS_KEY_NAME = "replicas"; private static final String ISR_KEY_NAME = "isr"; - private final Cluster cluster; - private final Map errors; + private final Collection brokers; + private final List topicMetadata; - /** - * Constructor for MetadataResponse where there are errors for some of the topics, - * error data take precedence over cluster information for particular topic - */ - public MetadataResponse(Cluster cluster, Map errors) { + + public MetadataResponse(List brokers, List topicMetadata) { super(new Struct(CURRENT_SCHEMA)); - List brokerArray = new ArrayList(); - for (Node node : cluster.nodes()) { + this.brokers = brokers; + this.topicMetadata = topicMetadata; + + List brokerArray = new ArrayList<>(); + for (Node node : brokers) { Struct broker = struct.instance(BROKERS_KEY_NAME); broker.set(NODE_ID_KEY_NAME, node.id()); broker.set(HOST_KEY_NAME, node.host()); @@ -91,51 +91,39 @@ public MetadataResponse(Cluster cluster, Map errors) { } struct.set(BROKERS_KEY_NAME, brokerArray.toArray()); + List topicMetadataArray = new ArrayList<>(topicMetadata.size()); + for (TopicMetadata metadata : topicMetadata) { + Struct topicData = struct.instance(TOPIC_METADATA_KEY_NAME); + topicData.set(TOPIC_KEY_NAME, metadata.topic); + topicData.set(TOPIC_ERROR_CODE_KEY_NAME, metadata.error.code()); + + List partitionMetadataArray = new ArrayList<>(metadata.partitionMetadata.size()); + for (PartitionMetadata partitionMetadata : metadata.partitionMetadata()) { + Struct partitionData = topicData.instance(PARTITION_METADATA_KEY_NAME); + partitionData.set(PARTITION_ERROR_CODE_KEY_NAME, partitionMetadata.error.code()); + partitionData.set(PARTITION_KEY_NAME, partitionMetadata.partition); + partitionData.set(LEADER_KEY_NAME, partitionMetadata.leader.id()); + ArrayList replicas = new ArrayList<>(partitionMetadata.replicas.size()); + for (Node node : partitionMetadata.replicas) + replicas.add(node.id()); + partitionData.set(REPLICAS_KEY_NAME, replicas.toArray()); + ArrayList isr = new ArrayList<>(partitionMetadata.isr.size()); + for (Node node : partitionMetadata.isr) + isr.add(node.id()); + partitionData.set(ISR_KEY_NAME, isr.toArray()); + partitionMetadataArray.add(partitionData); - List topicArray = new ArrayList(); - for (Map.Entry errorEntry : errors.entrySet()) { - Struct topicData = struct.instance(TOPIC_METATDATA_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, errorEntry.getKey()); - topicData.set(TOPIC_ERROR_CODE_KEY_NAME, errorEntry.getValue().code()); - topicData.set(PARTITION_METADATA_KEY_NAME, new Struct[0]); - topicArray.add(topicData); - } - - for (String topic : cluster.topics()) { - if (!errors.containsKey(topic)) { - Struct topicData = struct.instance(TOPIC_METATDATA_KEY_NAME); - topicData.set(TOPIC_KEY_NAME, topic); - topicData.set(TOPIC_ERROR_CODE_KEY_NAME, Errors.NONE.code()); - List partitionArray = new ArrayList(); - for (PartitionInfo fetchPartitionData : cluster.partitionsForTopic(topic)) { - Struct partitionData = topicData.instance(PARTITION_METADATA_KEY_NAME); - partitionData.set(PARTITION_ERROR_CODE_KEY_NAME, Errors.NONE.code()); - partitionData.set(PARTITION_KEY_NAME, fetchPartitionData.partition()); - partitionData.set(LEADER_KEY_NAME, fetchPartitionData.leader().id()); - ArrayList replicas = new ArrayList(); - for (Node node : fetchPartitionData.replicas()) - replicas.add(node.id()); - partitionData.set(REPLICAS_KEY_NAME, replicas.toArray()); - ArrayList isr = new ArrayList(); - for (Node node : fetchPartitionData.inSyncReplicas()) - isr.add(node.id()); - partitionData.set(ISR_KEY_NAME, isr.toArray()); - partitionArray.add(partitionData); - } - topicData.set(PARTITION_METADATA_KEY_NAME, partitionArray.toArray()); - topicArray.add(topicData); } + topicData.set(PARTITION_METADATA_KEY_NAME, partitionMetadataArray.toArray()); + topicMetadataArray.add(topicData); } - struct.set(TOPIC_METATDATA_KEY_NAME, topicArray.toArray()); - - this.cluster = cluster; - this.errors = errors; + struct.set(TOPIC_METADATA_KEY_NAME, topicMetadataArray.toArray()); } public MetadataResponse(Struct struct) { super(struct); - Map errors = new HashMap(); - Map brokers = new HashMap(); + + Map brokers = new HashMap<>(); Object[] brokerStructs = (Object[]) struct.get(BROKERS_KEY_NAME); for (int i = 0; i < brokerStructs.length; i++) { Struct broker = (Struct) brokerStructs[i]; @@ -144,63 +132,155 @@ public MetadataResponse(Struct struct) { int port = broker.getInt(PORT_KEY_NAME); brokers.put(nodeId, new Node(nodeId, host, port)); } - List partitions = new ArrayList(); - Object[] topicInfos = (Object[]) struct.get(TOPIC_METATDATA_KEY_NAME); + + List topicMetadata = new ArrayList<>(); + Object[] topicInfos = (Object[]) struct.get(TOPIC_METADATA_KEY_NAME); for (int i = 0; i < topicInfos.length; i++) { Struct topicInfo = (Struct) topicInfos[i]; - short topicError = topicInfo.getShort(TOPIC_ERROR_CODE_KEY_NAME); + Errors topicError = Errors.forCode(topicInfo.getShort(TOPIC_ERROR_CODE_KEY_NAME)); String topic = topicInfo.getString(TOPIC_KEY_NAME); - if (topicError == Errors.NONE.code()) { - Object[] partitionInfos = (Object[]) topicInfo.get(PARTITION_METADATA_KEY_NAME); - for (int j = 0; j < partitionInfos.length; j++) { - Struct partitionInfo = (Struct) partitionInfos[j]; - int partition = partitionInfo.getInt(PARTITION_KEY_NAME); - int leader = partitionInfo.getInt(LEADER_KEY_NAME); - Node leaderNode = leader == -1 ? null : brokers.get(leader); - Object[] replicas = (Object[]) partitionInfo.get(REPLICAS_KEY_NAME); - 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_KEY_NAME); - 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)); - } - } else { - errors.put(topic, Errors.forCode(topicError)); + List partitionMetadata = new ArrayList<>(); + + Object[] partitionInfos = (Object[]) topicInfo.get(PARTITION_METADATA_KEY_NAME); + for (int j = 0; j < partitionInfos.length; j++) { + Struct partitionInfo = (Struct) partitionInfos[j]; + Errors partitionError = Errors.forCode(partitionInfo.getShort(PARTITION_ERROR_CODE_KEY_NAME)); + int partition = partitionInfo.getInt(PARTITION_KEY_NAME); + int leader = partitionInfo.getInt(LEADER_KEY_NAME); + Node leaderNode = leader == -1 ? null : brokers.get(leader); + Object[] replicas = (Object[]) partitionInfo.get(REPLICAS_KEY_NAME); + List replicaNodes = new ArrayList<>(replicas.length); + for (Object replicaNodeId : replicas) + replicaNodes.add(brokers.get(replicaNodeId)); + Object[] isr = (Object[]) partitionInfo.get(ISR_KEY_NAME); + List isrNodes = new ArrayList<>(isr.length); + for (Object isrNode : isr) + isrNodes.add(brokers.get(isrNode)); + partitionMetadata.add(new PartitionMetadata(partitionError, partition, leaderNode, replicaNodes, isrNodes)); } + + topicMetadata.add(new TopicMetadata(topicError, topic, partitionMetadata)); } - this.errors = errors; - this.cluster = new Cluster(brokers.values(), partitions, unauthorizedTopics(errors)); + this.brokers = brokers.values(); + this.topicMetadata = topicMetadata; } - private Set unauthorizedTopics(Map topicErrors) { - if (topicErrors.isEmpty()) - return Collections.emptySet(); + /** + * Get a map of the topics which had metadata errors + * @return the map + */ + public Map errors() { + Map errors = new HashMap<>(); + for (TopicMetadata metadata : topicMetadata) { + if (metadata.error != Errors.NONE) + errors.put(metadata.topic(), metadata.error); + } + return errors; + } + /** + * Get a snapshot of the cluster metadata from this response + * @return the cluster snapshot + */ + public Cluster cluster() { Set unauthorizedTopics = new HashSet<>(); - for (Map.Entry topicErrorEntry : topicErrors.entrySet()) { - if (topicErrorEntry.getValue() == Errors.TOPIC_AUTHORIZATION_FAILED) - unauthorizedTopics.add(topicErrorEntry.getKey()); + List partitions = new ArrayList<>(); + for (TopicMetadata metadata : topicMetadata) { + if (metadata.error == Errors.NONE) { + for (PartitionMetadata partitionMetadata : metadata.partitionMetadata) + partitions.add(new PartitionInfo( + metadata.topic, + partitionMetadata.partition, + partitionMetadata.leader, + partitionMetadata.replicas.toArray(new Node[0]), + partitionMetadata.isr.toArray(new Node[0]))); + } else if (metadata.error == Errors.TOPIC_AUTHORIZATION_FAILED) { + unauthorizedTopics.add(metadata.topic); + } } - return unauthorizedTopics; + + return new Cluster(this.brokers, partitions, unauthorizedTopics); } - public Map errors() { - return this.errors; + /** + * Get all brokers returned in metadata response + * @return the brokers + */ + public Collection brokers() { + return brokers; } - public boolean hasError(String topic) { - return this.errors.containsKey(topic); + public static MetadataResponse parse(ByteBuffer buffer) { + return new MetadataResponse(CURRENT_SCHEMA.read(buffer)); } - public Cluster cluster() { - return this.cluster; + public static class TopicMetadata { + private final Errors error; + private final String topic; + private final List partitionMetadata; + + public TopicMetadata(Errors error, + String topic, + List partitionMetadata) { + this.error = error; + this.topic = topic; + this.partitionMetadata = partitionMetadata; + } + + public Errors error() { + return error; + } + + public String topic() { + return topic; + } + + public List partitionMetadata() { + return partitionMetadata; + } + } - public static MetadataResponse parse(ByteBuffer buffer) { - return new MetadataResponse(CURRENT_SCHEMA.read(buffer)); + public static class PartitionMetadata { + private final Errors error; + private final int partition; + private final Node leader; + private final List replicas; + private final List isr; + + public PartitionMetadata(Errors error, + int partition, + Node leader, + List replicas, + List isr) { + this.error = error; + this.partition = partition; + this.leader = leader; + this.replicas = replicas; + this.isr = isr; + } + + public Errors error() { + return error; + } + + public int partition() { + return partition; + } + + public Node leader() { + return leader; + } + + public List replicas() { + return replicas; + } + + public List isr() { + return isr; + } + } + } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 823d04ee6d553..58c38412588ae 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -53,6 +53,7 @@ import org.junit.Test; import java.nio.ByteBuffer; +import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.HashMap; @@ -441,8 +442,7 @@ public void testUpdateFetchPositionDisconnect() { @Test public void testGetAllTopics() { // sending response before request, as getTopicMetadata is a blocking call - client.prepareResponse( - new MetadataResponse(cluster, Collections.emptyMap()).toStruct()); + client.prepareResponse(newMetadataResponse(topicName, Errors.NONE).toStruct()); Map> allTopics = fetcher.getAllTopicMetadata(5000L); @@ -453,7 +453,7 @@ public void testGetAllTopics() { public void testGetAllTopicsDisconnect() { // first try gets a disconnect, next succeeds client.prepareResponse(null, true); - client.prepareResponse(new MetadataResponse(cluster, Collections.emptyMap()).toStruct()); + client.prepareResponse(newMetadataResponse(topicName, Errors.NONE).toStruct()); Map> allTopics = fetcher.getAllTopicMetadata(5000L); assertEquals(cluster.topics().size(), allTopics.size()); } @@ -466,8 +466,7 @@ public void testGetAllTopicsTimeout() { @Test public void testGetAllTopicsUnauthorized() { - client.prepareResponse(new MetadataResponse(cluster, - Collections.singletonMap(topicName, Errors.TOPIC_AUTHORIZATION_FAILED)).toStruct()); + client.prepareResponse(newMetadataResponse(topicName, Errors.TOPIC_AUTHORIZATION_FAILED).toStruct()); try { fetcher.getAllTopicMetadata(10L); fail(); @@ -478,17 +477,13 @@ public void testGetAllTopicsUnauthorized() { @Test(expected = InvalidTopicException.class) public void testGetTopicMetadataInvalidTopic() { - client.prepareResponse(new MetadataResponse(cluster, - Collections.singletonMap(topicName, Errors.INVALID_TOPIC_EXCEPTION)).toStruct()); + client.prepareResponse(newMetadataResponse(topicName, Errors.INVALID_TOPIC_EXCEPTION).toStruct()); fetcher.getTopicMetadata(Collections.singletonList(topicName), 5000L); } @Test public void testGetTopicMetadataUnknownTopic() { - Cluster emptyCluster = new Cluster(this.cluster.nodes(), Collections.emptyList(), - Collections.emptySet()); - client.prepareResponse(new MetadataResponse(emptyCluster, - Collections.singletonMap(topicName, Errors.UNKNOWN_TOPIC_OR_PARTITION)).toStruct()); + client.prepareResponse(newMetadataResponse(topicName, Errors.UNKNOWN_TOPIC_OR_PARTITION).toStruct()); Map> topicMetadata = fetcher.getTopicMetadata(Collections.singletonList(topicName), 5000L); assertNull(topicMetadata.get(topicName)); @@ -496,12 +491,9 @@ public void testGetTopicMetadataUnknownTopic() { @Test public void testGetTopicMetadataLeaderNotAvailable() { - Cluster emptyCluster = new Cluster(this.cluster.nodes(), Collections.emptyList(), - Collections.emptySet()); - client.prepareResponse(new MetadataResponse(emptyCluster, - Collections.singletonMap(topicName, Errors.LEADER_NOT_AVAILABLE)).toStruct()); - client.prepareResponse(new MetadataResponse(this.cluster, - Collections.emptyMap()).toStruct()); + client.prepareResponse(newMetadataResponse(topicName, Errors.LEADER_NOT_AVAILABLE).toStruct()); + client.prepareResponse(newMetadataResponse(topicName, Errors.NONE).toStruct()); + Map> topicMetadata = fetcher.getTopicMetadata(Collections.singletonList(topicName), 5000L); assertTrue(topicMetadata.containsKey(topicName)); } @@ -565,6 +557,23 @@ private Struct fetchResponse(ByteBuffer buffer, short error, long hw, int thrott return response.toStruct(); } + private MetadataResponse newMetadataResponse(String topic, Errors error) { + List partitionsMetadata = new ArrayList<>(); + if (error == Errors.NONE) { + for (PartitionInfo partitionInfo : cluster.partitionsForTopic(topic)) { + partitionsMetadata.add(new MetadataResponse.PartitionMetadata( + Errors.NONE, + partitionInfo.partition(), + partitionInfo.leader(), + Arrays.asList(partitionInfo.replicas()), + Arrays.asList(partitionInfo.inSyncReplicas()))); + } + } + + MetadataResponse.TopicMetadata topicMetadata = new MetadataResponse.TopicMetadata(error, topic, partitionsMetadata); + return new MetadataResponse(cluster.nodes(), Arrays.asList(topicMetadata)); + } + private Fetcher createFetcher(int maxPollRecords, SubscriptionState subscriptions, Metrics metrics) { diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/DefaultPartitionerTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/DefaultPartitionerTest.java index 7a5cef6886a28..fd8a5bc3b0d4c 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/DefaultPartitionerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/DefaultPartitionerTest.java @@ -12,19 +12,19 @@ */ package org.apache.kafka.clients.producer.internals; -import static java.util.Arrays.asList; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.util.Collections; -import java.util.List; - import org.apache.kafka.clients.producer.Partitioner; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; import org.junit.Test; +import java.util.Collections; +import java.util.List; + +import static java.util.Arrays.asList; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + public class DefaultPartitionerTest { private byte[] keyBytes = "key".getBytes(); private Partitioner partitioner = new DefaultPartitioner(); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index edeaf639016a7..30238378f7b3f 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -14,9 +14,7 @@ package org.apache.kafka.common.requests; import org.apache.kafka.common.BrokerEndPoint; -import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; -import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.UnknownServerException; import org.apache.kafka.common.protocol.ApiKeys; @@ -278,16 +276,16 @@ private AbstractRequest createMetadataRequest() { private AbstractRequestResponse createMetadataResponse() { Node node = new Node(1, "host1", 1001); - Node[] replicas = new Node[1]; - replicas[0] = node; - Node[] isr = new Node[1]; - isr[0] = node; - Cluster cluster = new Cluster(Arrays.asList(node), Arrays.asList(new PartitionInfo("topic1", 1, node, replicas, isr)), - Collections.emptySet()); - - Map errors = new HashMap(); - errors.put("topic2", Errors.LEADER_NOT_AVAILABLE); - return new MetadataResponse(cluster, errors); + List replicas = Arrays.asList(node); + List isr = Arrays.asList(node); + + List allTopicMetadata = new ArrayList<>(); + allTopicMetadata.add(new MetadataResponse.TopicMetadata(Errors.NONE, "topic1", + Arrays.asList(new MetadataResponse.PartitionMetadata(Errors.NONE, 1, node, replicas, isr)))); + allTopicMetadata.add(new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, "topic2", + Collections.emptyList())); + + return new MetadataResponse(Arrays.asList(node), allTopicMetadata); } private AbstractRequest createOffsetCommitRequest() { diff --git a/core/src/main/scala/kafka/admin/AdminClient.scala b/core/src/main/scala/kafka/admin/AdminClient.scala index a8d9964c8dede..b8573153e5b8d 100644 --- a/core/src/main/scala/kafka/admin/AdminClient.scala +++ b/core/src/main/scala/kafka/admin/AdminClient.scala @@ -92,8 +92,9 @@ class AdminClient(val time: Time, val request = new MetadataRequest(List[String]()) val responseBody = sendAnyNode(ApiKeys.METADATA, request) val response = new MetadataResponse(responseBody) - if (!response.errors().isEmpty) - debug(s"Metadata request contained errors: ${response.errors()}") + val errors = response.errors() + if (!errors.isEmpty) + debug(s"Metadata request contained errors: ${errors}") response.cluster().nodes().asScala.toList } diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index a868400efec64..3fb44d320afa3 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -18,23 +18,25 @@ package kafka.admin import kafka.common._ -import kafka.cluster.{BrokerEndPoint, Broker} +import kafka.cluster.Broker import kafka.log.LogConfig import kafka.server.ConfigType import kafka.utils._ import kafka.utils.ZkUtils._ -import kafka.api.{TopicMetadata, PartitionMetadata} import java.util.Random import java.util.Properties +import org.apache.kafka.common.Node import org.apache.kafka.common.errors.{ReplicaNotAvailableException, InvalidTopicException, LeaderNotAvailableException} import org.apache.kafka.common.protocol.{Errors, SecurityProtocol} +import org.apache.kafka.common.requests.MetadataResponse import scala.Predef._ import scala.collection._ -import mutable.ListBuffer +import scala.collection.JavaConverters._ import scala.collection.mutable +import mutable.ListBuffer import collection.Map import collection.Set @@ -390,15 +392,18 @@ object AdminUtils extends Logging { def fetchAllEntityConfigs(zkUtils: ZkUtils, entityType: String): Map[String, Properties] = zkUtils.getAllEntitiesWithConfig(entityType).map(entity => (entity, fetchEntityConfig(zkUtils, entityType, entity))).toMap - def fetchTopicMetadataFromZk(topic: String, zkUtils: ZkUtils): TopicMetadata = + def fetchTopicMetadataFromZk(topic: String, zkUtils: ZkUtils): MetadataResponse.TopicMetadata = fetchTopicMetadataFromZk(topic, zkUtils, new mutable.HashMap[Int, Broker]) - def fetchTopicMetadataFromZk(topics: Set[String], zkUtils: ZkUtils): Set[TopicMetadata] = { + def fetchTopicMetadataFromZk(topics: Set[String], zkUtils: ZkUtils): Set[MetadataResponse.TopicMetadata] = { val cachedBrokerInfo = new mutable.HashMap[Int, Broker]() topics.map(topic => fetchTopicMetadataFromZk(topic, zkUtils, cachedBrokerInfo)) } - private def fetchTopicMetadataFromZk(topic: String, zkUtils: ZkUtils, cachedBrokerInfo: mutable.HashMap[Int, Broker], protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): TopicMetadata = { + private def fetchTopicMetadataFromZk(topic: String, + zkUtils: ZkUtils, + cachedBrokerInfo: mutable.HashMap[Int, Broker], + protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): MetadataResponse.TopicMetadata = { if(zkUtils.pathExists(getTopicPath(topic))) { val topicPartitionAssignment = zkUtils.getPartitionAssignmentForTopics(List(topic)).get(topic).get val sortedPartitions = topicPartitionAssignment.toList.sortWith((m1, m2) => m1._1 < m2._1) @@ -409,22 +414,22 @@ object AdminUtils extends Logging { val leader = zkUtils.getLeaderForPartition(topic, partition) debug("replicas = " + replicas + ", in sync replicas = " + inSyncReplicas + ", leader = " + leader) - var leaderInfo: Option[BrokerEndPoint] = None - var replicaInfo: Seq[BrokerEndPoint] = Nil - var isrInfo: Seq[BrokerEndPoint] = Nil + var leaderInfo: Node = Node.noNode() + var replicaInfo: Seq[Node] = Nil + var isrInfo: Seq[Node] = Nil try { leaderInfo = leader match { case Some(l) => try { - Some(getBrokerInfoFromCache(zkUtils, cachedBrokerInfo, List(l)).head.getBrokerEndPoint(protocol)) + getBrokerInfoFromCache(zkUtils, cachedBrokerInfo, List(l)).head.getNode(protocol) } catch { case e: Throwable => throw new LeaderNotAvailableException("Leader not available for partition [%s,%d]".format(topic, partition), e) } case None => throw new LeaderNotAvailableException("No leader exists for partition " + partition) } try { - replicaInfo = getBrokerInfoFromCache(zkUtils, cachedBrokerInfo, replicas).map(_.getBrokerEndPoint(protocol)) - isrInfo = getBrokerInfoFromCache(zkUtils, cachedBrokerInfo, inSyncReplicas).map(_.getBrokerEndPoint(protocol)) + replicaInfo = getBrokerInfoFromCache(zkUtils, cachedBrokerInfo, replicas).map(_.getNode(protocol)) + isrInfo = getBrokerInfoFromCache(zkUtils, cachedBrokerInfo, inSyncReplicas).map(_.getNode(protocol)) } catch { case e: Throwable => throw new ReplicaNotAvailableException(e) } @@ -434,18 +439,17 @@ object AdminUtils extends Logging { if(isrInfo.size < inSyncReplicas.size) throw new ReplicaNotAvailableException("In Sync Replica information not available for following brokers: " + inSyncReplicas.filterNot(isrInfo.map(_.id).contains(_)).mkString(",")) - new PartitionMetadata(partition, leaderInfo, replicaInfo, isrInfo, Errors.NONE.code) + new MetadataResponse.PartitionMetadata(Errors.NONE, partition, leaderInfo, replicaInfo.asJava, isrInfo.asJava) } catch { case e: Throwable => debug("Error while fetching metadata for partition [%s,%d]".format(topic, partition), e) - new PartitionMetadata(partition, leaderInfo, replicaInfo, isrInfo, - Errors.forException(e).code) + new MetadataResponse.PartitionMetadata(Errors.forException(e), partition, leaderInfo, replicaInfo.asJava, isrInfo.asJava) } } - new TopicMetadata(topic, partitionMetadata) + new MetadataResponse.TopicMetadata(Errors.NONE, topic, partitionMetadata.toList.asJava) } else { // topic doesn't exist, send appropriate error code - new TopicMetadata(topic, Seq.empty[PartitionMetadata], Errors.UNKNOWN_TOPIC_OR_PARTITION.code) + new MetadataResponse.TopicMetadata(Errors.UNKNOWN_TOPIC_OR_PARTITION, topic, java.util.Collections.emptyList()) } } diff --git a/core/src/main/scala/kafka/api/TopicMetadata.scala b/core/src/main/scala/kafka/api/TopicMetadata.scala index b0e41ec575c49..ae5ea582d063e 100644 --- a/core/src/main/scala/kafka/api/TopicMetadata.scala +++ b/core/src/main/scala/kafka/api/TopicMetadata.scala @@ -21,7 +21,6 @@ import kafka.cluster.BrokerEndPoint import java.nio.ByteBuffer import kafka.api.ApiUtils._ import kafka.utils.Logging -import kafka.common._ import org.apache.kafka.common.protocol.Errors object TopicMetadata { diff --git a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala index be135869d86f9..0654e3de22640 100644 --- a/core/src/main/scala/kafka/api/TopicMetadataRequest.scala +++ b/core/src/main/scala/kafka/api/TopicMetadataRequest.scala @@ -30,22 +30,6 @@ import scala.collection.mutable.ListBuffer object TopicMetadataRequest extends Logging { val CurrentVersion = 0.shortValue val DefaultClientId = "" - - /** - * TopicMetadataRequest has the following format - - * number of topics (4 bytes) list of topics (2 bytes + topic.length per topic) detailedMetadata (2 bytes) timestamp (8 bytes) count (4 bytes) - */ - - def readFrom(buffer: ByteBuffer): TopicMetadataRequest = { - val versionId = buffer.getShort - val correlationId = buffer.getInt - val clientId = readShortString(buffer) - val numTopics = readIntInRange(buffer, "number of topics", (0, Int.MaxValue)) - val topics = new ListBuffer[String]() - for(i <- 0 until numTopics) - topics += readShortString(buffer) - new TopicMetadataRequest(versionId, correlationId, clientId, topics.toList) - } } case class TopicMetadataRequest(versionId: Short, diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index b56cae97b2379..7340f148af1ec 100755 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -21,6 +21,7 @@ import java.nio.ByteBuffer import kafka.common.{BrokerEndPointNotAvailableException, BrokerNotAvailableException, KafkaException} import kafka.utils.Json +import org.apache.kafka.common.Node import org.apache.kafka.common.protocol.SecurityProtocol /** @@ -103,7 +104,7 @@ object Broker { } } -case class Broker(id: Int, endPoints: Map[SecurityProtocol, EndPoint]) { +case class Broker(id: Int, endPoints: collection.Map[SecurityProtocol, EndPoint]) { override def toString: String = id + " : " + endPoints.values.mkString("(",",",")") @@ -133,13 +134,14 @@ case class Broker(id: Int, endPoints: Map[SecurityProtocol, EndPoint]) { endPoints.contains(protocolType) } + def getNode(protocolType: SecurityProtocol): Node = { + val endpoint = endPoints.getOrElse(protocolType, throw new BrokerEndPointNotAvailableException("End point %s not found for broker %d".format(protocolType,id))) + new Node(id, endpoint.host, endpoint.port) + } + def getBrokerEndPoint(protocolType: SecurityProtocol): BrokerEndPoint = { - val endpoint = endPoints.get(protocolType) - endpoint match { - case Some(endpoint) => new BrokerEndPoint(id, endpoint.host, endpoint.port) - case None => - throw new BrokerEndPointNotAvailableException("End point %s not found for broker %d".format(protocolType,id)) - } + val endpoint = endPoints.getOrElse(protocolType, throw new BrokerEndPointNotAvailableException("End point %s not found for broker %d".format(protocolType,id))) + new BrokerEndPoint(id, endpoint.host, endpoint.port) } } diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index 916c438496b00..1105802e0890d 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -63,7 +63,6 @@ object RequestChannel extends Logging { // o.a.k.common.requests.AbstractRequest.getRequest() private val keyToNameAndDeserializerMap: Map[Short, (ByteBuffer) => RequestOrResponse]= Map(ApiKeys.FETCH.id -> FetchRequest.readFrom, - ApiKeys.METADATA.id -> TopicMetadataRequest.readFrom, ApiKeys.CONTROLLED_SHUTDOWN_KEY.id -> ControlledShutdownRequest.readFrom ) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 8f3a2ad458dc2..5f9ec8ba4cee4 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -19,6 +19,7 @@ package kafka.server import java.nio.ByteBuffer import java.lang.{Long => JLong, Short => JShort} +import java.util.Properties import kafka.admin.AdminUtils import kafka.api._ @@ -40,7 +41,7 @@ import org.apache.kafka.common.requests.{ListOffsetRequest, ListOffsetResponse, DescribeGroupsRequest, DescribeGroupsResponse, HeartbeatRequest, HeartbeatResponse, JoinGroupRequest, JoinGroupResponse, LeaveGroupRequest, LeaveGroupResponse, ResponseHeader, ResponseSend, SyncGroupRequest, SyncGroupResponse, LeaderAndIsrRequest, LeaderAndIsrResponse, StopReplicaRequest, StopReplicaResponse, ProduceRequest, ProduceResponse, UpdateMetadataRequest, UpdateMetadataResponse, -OffsetCommitRequest, OffsetCommitResponse, OffsetFetchRequest, OffsetFetchResponse} +MetadataRequest, MetadataResponse, OffsetCommitRequest, OffsetCommitResponse, OffsetFetchRequest, OffsetFetchResponse} import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.{TopicPartition, Node} @@ -258,7 +259,7 @@ class KafkaApis(val requestChannel: RequestChannel, case (topicPartition, partitionData) => val topicDirs = new ZKGroupTopicDirs(offsetCommitRequest.groupId, topicPartition.topic) try { - if (metadataCache.getTopicMetadata(Set(topicPartition.topic), request.securityProtocol).size <= 0) + if (!metadataCache.hasTopicMetadata(topicPartition.topic)) (topicPartition, Errors.UNKNOWN_TOPIC_OR_PARTITION.code) else if (partitionData.metadata != null && partitionData.metadata.length > config.offsetMetadataMaxSize) (topicPartition, Errors.OFFSET_METADATA_TOO_LARGE.code) @@ -618,84 +619,105 @@ class KafkaApis(val requestChannel: RequestChannel, ret.toSeq.sortBy(- _) } - private def getTopicMetadata(topics: Set[String], securityProtocol: SecurityProtocol): Seq[TopicMetadata] = { + private def createTopic(topic: String, + numPartitions: Int, + replicationFactor: Int, + properties: Properties = new Properties()): MetadataResponse.TopicMetadata = { + try { + AdminUtils.createTopic(zkUtils, topic, numPartitions, replicationFactor, properties) + info("Auto creation of topic %s with %d partitions and replication factor %d is successful" + .format(topic, numPartitions, replicationFactor)) + new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, topic, java.util.Collections.emptyList()) + } catch { + case e: TopicExistsException => // let it go, possibly another broker created this topic + new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, topic, java.util.Collections.emptyList()) + case itex: InvalidTopicException => + new MetadataResponse.TopicMetadata(Errors.INVALID_TOPIC_EXCEPTION, topic, java.util.Collections.emptyList()) + } + } + + private def createGroupMetadataTopic(): MetadataResponse.TopicMetadata = { + val aliveBrokers = metadataCache.getAliveBrokers + val offsetsTopicReplicationFactor = + if (aliveBrokers.nonEmpty) + Math.min(config.offsetsTopicReplicationFactor.toInt, aliveBrokers.length) + else + config.offsetsTopicReplicationFactor.toInt + createTopic(GroupCoordinator.GroupMetadataTopicName, config.offsetsTopicPartitions, + offsetsTopicReplicationFactor, coordinator.offsetsTopicConfigs) + } + + private def getOrCreateGroupMetadataTopic(securityProtocol: SecurityProtocol): MetadataResponse.TopicMetadata = { + val topicMetadata = metadataCache.getTopicMetadata(Set(GroupCoordinator.GroupMetadataTopicName), securityProtocol) + topicMetadata.headOption.getOrElse(createGroupMetadataTopic()) + } + + private def getTopicMetadata(topics: Set[String], securityProtocol: SecurityProtocol): Seq[MetadataResponse.TopicMetadata] = { val topicResponses = metadataCache.getTopicMetadata(topics, securityProtocol) - if (topics.size > 0 && topicResponses.size != topics.size) { + if (topics.isEmpty || topicResponses.size == topics.size) { + topicResponses + } else { val nonExistentTopics = topics -- topicResponses.map(_.topic).toSet val responsesForNonExistentTopics = nonExistentTopics.map { topic => - if (topic == GroupCoordinator.GroupMetadataTopicName || config.autoCreateTopicsEnable) { - try { - if (topic == GroupCoordinator.GroupMetadataTopicName) { - val aliveBrokers = metadataCache.getAliveBrokers - val offsetsTopicReplicationFactor = - if (aliveBrokers.length > 0) - Math.min(config.offsetsTopicReplicationFactor.toInt, aliveBrokers.length) - else - config.offsetsTopicReplicationFactor.toInt - AdminUtils.createTopic(zkUtils, topic, config.offsetsTopicPartitions, - offsetsTopicReplicationFactor, - coordinator.offsetsTopicConfigs) - info("Auto creation of topic %s with %d partitions and replication factor %d is successful!" - .format(topic, config.offsetsTopicPartitions, offsetsTopicReplicationFactor)) - } - else { - AdminUtils.createTopic(zkUtils, topic, config.numPartitions, config.defaultReplicationFactor) - info("Auto creation of topic %s with %d partitions and replication factor %d is successful!" - .format(topic, config.numPartitions, config.defaultReplicationFactor)) - } - new TopicMetadata(topic, Seq.empty[PartitionMetadata], Errors.LEADER_NOT_AVAILABLE.code) - } catch { - case e: TopicExistsException => // let it go, possibly another broker created this topic - new TopicMetadata(topic, Seq.empty[PartitionMetadata], Errors.LEADER_NOT_AVAILABLE.code) - case itex: InvalidTopicException => - new TopicMetadata(topic, Seq.empty[PartitionMetadata], Errors.INVALID_TOPIC_EXCEPTION.code) - } + if (topic == GroupCoordinator.GroupMetadataTopicName) { + createGroupMetadataTopic() + } else if (config.autoCreateTopicsEnable) { + createTopic(topic, config.numPartitions, config.defaultReplicationFactor) } else { - new TopicMetadata(topic, Seq.empty[PartitionMetadata], Errors.UNKNOWN_TOPIC_OR_PARTITION.code) + new MetadataResponse.TopicMetadata(Errors.UNKNOWN_TOPIC_OR_PARTITION, topic, java.util.Collections.emptyList()) } } - topicResponses.appendAll(responsesForNonExistentTopics) + topicResponses ++ responsesForNonExistentTopics } - topicResponses } /** * Handle a topic metadata request */ def handleTopicMetadataRequest(request: RequestChannel.Request) { - val metadataRequest = request.requestObj.asInstanceOf[TopicMetadataRequest] - - //if topics is empty -> fetch all topics metadata but filter out the topic response that are not authorized - val topics = if (metadataRequest.topics.isEmpty) { - val topicResponses = metadataCache.getTopicMetadata(metadataRequest.topics.toSet, request.securityProtocol) - topicResponses.map(_.topic).filter(topic => authorize(request.session, Describe, new Resource(Topic, topic))).toSet + val metadataRequest = request.body.asInstanceOf[MetadataRequest] + + val topics = metadataRequest.topics.asScala.toSet + var (authorizedTopics, unauthorizedTopics) = if (metadataRequest.topics.isEmpty) { + //if topics is empty -> fetch all topics metadata but filter out the topic response that are not authorized + val authorized = metadataCache.getAllTopics() + .filter(topic => authorize(request.session, Describe, new Resource(Topic, topic))) + (authorized, mutable.Set[String]()) } else { - metadataRequest.topics.toSet + topics.partition(topic => authorize(request.session, Describe, new Resource(Topic, topic))) } - //when topics is empty this will be a duplicate authorization check but given this should just be a cache lookup, it should not matter. - var (authorizedTopics, unauthorizedTopics) = topics.partition(topic => authorize(request.session, Describe, new Resource(Topic, topic))) - - if (!authorizedTopics.isEmpty) { - val topicResponses = metadataCache.getTopicMetadata(authorizedTopics, request.securityProtocol) - if (config.autoCreateTopicsEnable && topicResponses.size != authorizedTopics.size) { - val nonExistentTopics: Set[String] = topics -- topicResponses.map(_.topic).toSet - authorizer.foreach { - az => if (!az.authorize(request.session, Create, Resource.ClusterResource)) { - authorizedTopics --= nonExistentTopics - unauthorizedTopics ++= nonExistentTopics + if (authorizedTopics.nonEmpty) { + val nonExistingTopics = metadataCache.getNonExistingTopics(authorizedTopics) + if (config.autoCreateTopicsEnable && nonExistingTopics.nonEmpty) { + authorizer.foreach { az => + if (!az.authorize(request.session, Create, Resource.ClusterResource)) { + authorizedTopics --= nonExistingTopics + unauthorizedTopics ++= nonExistingTopics } } } } - val unauthorizedTopicMetaData = unauthorizedTopics.map(topic => new TopicMetadata(topic, Seq.empty[PartitionMetadata], Errors.TOPIC_AUTHORIZATION_FAILED.code)) + val unauthorizedTopicMetadata = unauthorizedTopics.map(topic => + new MetadataResponse.TopicMetadata(Errors.TOPIC_AUTHORIZATION_FAILED, topic, java.util.Collections.emptyList())) + + val topicMetadata = if (authorizedTopics.isEmpty) + Seq.empty[MetadataResponse.TopicMetadata] + else + getTopicMetadata(authorizedTopics, request.securityProtocol) - val topicMetadata = if (authorizedTopics.isEmpty) Seq.empty[TopicMetadata] else getTopicMetadata(authorizedTopics, request.securityProtocol) val brokers = metadataCache.getAliveBrokers - trace("Sending topic metadata %s and brokers %s for correlation id %d to client %s".format(topicMetadata.mkString(","), brokers.mkString(","), metadataRequest.correlationId, metadataRequest.clientId)) - val response = new TopicMetadataResponse(brokers.map(_.getBrokerEndPoint(request.securityProtocol)), topicMetadata ++ unauthorizedTopicMetaData, metadataRequest.correlationId) - requestChannel.sendResponse(new RequestChannel.Response(request, new RequestOrResponseSend(request.connectionId, response))) + + trace("Sending topic metadata %s and brokers %s for correlation id %d to client %s".format(topicMetadata.mkString(","), + brokers.mkString(","), request.header.correlationId, request.header.clientId)) + + val responseHeader = new ResponseHeader(request.header.correlationId) + val responseBody = new MetadataResponse( + brokers.map(_.getNode(request.securityProtocol)).asJava, + (topicMetadata ++ unauthorizedTopicMetadata).asJava + ) + requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, responseHeader, responseBody))) } /* @@ -725,7 +747,7 @@ class KafkaApis(val requestChannel: RequestChannel, val responseInfo = authorizedTopicPartitions.map { topicPartition => val topicDirs = new ZKGroupTopicDirs(offsetFetchRequest.groupId, topicPartition.topic) try { - if (metadataCache.getTopicMetadata(Set(topicPartition.topic), request.securityProtocol).isEmpty) + if (!metadataCache.hasTopicMetadata(topicPartition.topic)) (topicPartition, unknownTopicPartitionResponse) else { val payloadOpt = zkUtils.readDataMaybeNull(s"${topicDirs.consumerOffsetDir}/${topicPartition.partition}")._1 @@ -769,16 +791,21 @@ class KafkaApis(val requestChannel: RequestChannel, val partition = coordinator.partitionFor(groupCoordinatorRequest.groupId) // get metadata (and create the topic if necessary) - val offsetsTopicMetadata = getTopicMetadata(Set(GroupCoordinator.GroupMetadataTopicName), request.securityProtocol).head - val coordinatorEndpoint = offsetsTopicMetadata.partitionsMetadata.find(_.partitionId == partition).flatMap { - partitionMetadata => partitionMetadata.leader - } + val offsetsTopicMetadata = getOrCreateGroupMetadataTopic(request.securityProtocol) - val responseBody = coordinatorEndpoint match { - case None => - new GroupCoordinatorResponse(Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code, Node.noNode()) - case Some(endpoint) => - new GroupCoordinatorResponse(Errors.NONE.code, new Node(endpoint.id, endpoint.host, endpoint.port)) + val responseBody = if (offsetsTopicMetadata.error != Errors.NONE) { + new GroupCoordinatorResponse(Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code, Node.noNode) + } else { + val coordinatorEndpoint = offsetsTopicMetadata.partitionMetadata().asScala + .find(_.partition == partition) + .map(_.leader()) + + coordinatorEndpoint match { + case Some(endpoint) if !endpoint.isEmpty => + new GroupCoordinatorResponse(Errors.NONE.code, endpoint) + case _ => + new GroupCoordinatorResponse(Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code, Node.noNode) + } } trace("Sending consumer metadata %s for correlation id %d to client %s." @@ -788,8 +815,6 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleDescribeGroupRequest(request: RequestChannel.Request) { - import JavaConverters._ - val describeRequest = request.body.asInstanceOf[DescribeGroupsRequest] val responseHeader = new ResponseHeader(request.header.correlationId) @@ -814,8 +839,6 @@ class KafkaApis(val requestChannel: RequestChannel, } def handleListGroupsRequest(request: RequestChannel.Request) { - import JavaConverters._ - val responseHeader = new ResponseHeader(request.header.correlationId) val responseBody = if (!authorize(request.session, Describe, Resource.ClusterResource)) { ListGroupsResponse.fromError(Errors.CLUSTER_AUTHORIZATION_FAILED) diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 4be795dbbbc7a..1fdd717044c2f 100755 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -17,124 +17,151 @@ package kafka.server -import kafka.cluster.{EndPoint, BrokerEndPoint, Broker} -import kafka.common.TopicAndPartition +import java.util.EnumMap +import java.util.concurrent.locks.ReentrantReadWriteLock -import kafka.api._ -import kafka.controller.KafkaController.StateChangeLogger -import kafka.controller.LeaderIsrAndControllerEpoch -import org.apache.kafka.common.errors.{ReplicaNotAvailableException, LeaderNotAvailableException} -import org.apache.kafka.common.protocol.{Errors, SecurityProtocol} -import org.apache.kafka.common.requests.UpdateMetadataRequest -import org.apache.kafka.common.requests.UpdateMetadataRequest.PartitionState import scala.collection.{Seq, Set, mutable} import scala.collection.JavaConverters._ -import kafka.utils.Logging +import kafka.cluster.{Broker, EndPoint} +import kafka.api._ +import kafka.common.TopicAndPartition +import kafka.controller.{KafkaController, LeaderIsrAndControllerEpoch} import kafka.utils.CoreUtils._ - -import java.util.concurrent.locks.ReentrantReadWriteLock - +import kafka.utils.Logging +import org.apache.kafka.common.{Node, TopicPartition} +import org.apache.kafka.common.protocol.{Errors, SecurityProtocol} +import org.apache.kafka.common.requests.UpdateMetadataRequest.PartitionState +import org.apache.kafka.common.requests.{MetadataResponse, UpdateMetadataRequest} /** * A cache for the state (e.g., current leader) of each partition. This cache is updated through * UpdateMetadataRequest from the controller. Every broker maintains the same cache, asynchronously. */ private[server] class MetadataCache(brokerId: Int) extends Logging { - private val cache: mutable.Map[String, mutable.Map[Int, PartitionStateInfo]] = - new mutable.HashMap[String, mutable.Map[Int, PartitionStateInfo]]() - private var aliveBrokers: Map[Int, Broker] = Map() + private val stateChangeLogger = KafkaController.stateChangeLogger + private val cache = mutable.Map[String, mutable.Map[Int, PartitionStateInfo]]() + private val aliveBrokers = mutable.Map[Int, Broker]() + private val aliveNodes = mutable.Map[Int, collection.Map[SecurityProtocol, Node]]() private val partitionMetadataLock = new ReentrantReadWriteLock() this.logIdent = "[Kafka Metadata Cache on broker %d] ".format(brokerId) - def getTopicMetadata(topics: Set[String], protocol: SecurityProtocol) = { + private def getAliveEndpoints(brokers: Iterable[Int], protocol: SecurityProtocol): Seq[Node] = { + val result = new mutable.ArrayBuffer[Node](math.min(aliveBrokers.size, brokers.size)) + brokers.foreach { brokerId => + getAliveEndpoint(brokerId, protocol).foreach(result +=) + } + result + } - val isAllTopics = topics.isEmpty - val topicsRequested = if(isAllTopics) cache.keySet else topics - val topicResponses: mutable.ListBuffer[TopicMetadata] = new mutable.ListBuffer[TopicMetadata] + private def getAliveEndpoint(brokerId: Int, protocol: SecurityProtocol): Option[Node] = + aliveNodes.get(brokerId).flatMap(_.get(protocol)) + + private def getPartitionMetadata(topic: String, protocol: SecurityProtocol): Option[Iterable[MetadataResponse.PartitionMetadata]] = { + cache.get(topic).map { partitions => + partitions.map { case (partitionId, partitionState) => + val topicPartition = TopicAndPartition(topic, partitionId) + + val leaderAndIsr = partitionState.leaderIsrAndControllerEpoch.leaderAndIsr + val maybeLeader = getAliveEndpoint(leaderAndIsr.leader, protocol) + + val replicas = partitionState.allReplicas + val replicaInfo = getAliveEndpoints(replicas, protocol) + + maybeLeader match { + case None => + debug("Error while fetching metadata for %s: leader not available".format(topicPartition)) + new MetadataResponse.PartitionMetadata(Errors.LEADER_NOT_AVAILABLE, partitionId, Node.noNode(), + replicaInfo.asJava, java.util.Collections.emptyList()) + + case Some(leader) => + val isr = leaderAndIsr.isr + val isrInfo = getAliveEndpoints(isr, protocol) + + if (replicaInfo.size < replicas.size) { + debug("Error while fetching metadata for %s: replica information not available for following brokers %s" + .format(topicPartition, replicas.filterNot(replicaInfo.map(_.id).contains).mkString(","))) + + new MetadataResponse.PartitionMetadata(Errors.REPLICA_NOT_AVAILABLE, partitionId, leader, + replicaInfo.asJava, isrInfo.asJava) + } else if (isrInfo.size < isr.size) { + debug("Error while fetching metadata for %s: in sync replica information not available for following brokers %s" + .format(topicPartition, isr.filterNot(isrInfo.map(_.id).contains).mkString(","))) + new MetadataResponse.PartitionMetadata(Errors.REPLICA_NOT_AVAILABLE, partitionId, leader, + replicaInfo.asJava, isrInfo.asJava) + } else { + new MetadataResponse.PartitionMetadata(Errors.NONE, partitionId, leader, replicaInfo.asJava, + isrInfo.asJava) + } + } + } + } + } + + def getTopicMetadata(topics: Set[String], protocol: SecurityProtocol): Seq[MetadataResponse.TopicMetadata] = { inReadLock(partitionMetadataLock) { - for (topic <- topicsRequested) { - if (isAllTopics || cache.contains(topic)) { - val partitionStateInfos = cache(topic) - val partitionMetadata = partitionStateInfos.map { - case (partitionId, partitionState) => - val replicas = partitionState.allReplicas - val replicaInfo: Seq[BrokerEndPoint] = replicas.map(aliveBrokers.getOrElse(_, null)).filter(_ != null).toSeq.map(_.getBrokerEndPoint(protocol)) - var leaderInfo: Option[BrokerEndPoint] = None - var leaderBrokerInfo: Option[Broker] = None - var isrInfo: Seq[BrokerEndPoint] = Nil - val leaderIsrAndEpoch = partitionState.leaderIsrAndControllerEpoch - val leader = leaderIsrAndEpoch.leaderAndIsr.leader - val isr = leaderIsrAndEpoch.leaderAndIsr.isr - val topicPartition = TopicAndPartition(topic, partitionId) - try { - leaderBrokerInfo = aliveBrokers.get(leader) - if (!leaderBrokerInfo.isDefined) - throw new LeaderNotAvailableException("Leader not available for %s".format(topicPartition)) - else - leaderInfo = Some(leaderBrokerInfo.get.getBrokerEndPoint(protocol)) - isrInfo = isr.map(aliveBrokers.getOrElse(_, null)).filter(_ != null).map(_.getBrokerEndPoint(protocol)) - if (replicaInfo.size < replicas.size) - throw new ReplicaNotAvailableException("Replica information not available for following brokers: " + - replicas.filterNot(replicaInfo.map(_.id).contains(_)).mkString(",")) - if (isrInfo.size < isr.size) - throw new ReplicaNotAvailableException("In Sync Replica information not available for following brokers: " + - isr.filterNot(isrInfo.map(_.id).contains(_)).mkString(",")) - new PartitionMetadata(partitionId, leaderInfo, replicaInfo, isrInfo, Errors.NONE.code) - } catch { - case e: Throwable => - debug("Error while fetching metadata for %s: %s".format(topicPartition, e.toString)) - new PartitionMetadata(partitionId, leaderInfo, replicaInfo, isrInfo, - Errors.forException(e).code) - } - } - topicResponses += new TopicMetadata(topic, partitionMetadata.toSeq) + val topicsRequested = if (topics.isEmpty) cache.keySet else topics + topicsRequested.toSeq.flatMap { topic => + getPartitionMetadata(topic, protocol).map { partitionMetadata => + new MetadataResponse.TopicMetadata(Errors.NONE, topic, partitionMetadata.toBuffer.asJava) } } } - topicResponses } - def getAliveBrokers = { + def hasTopicMetadata(topic: String): Boolean = { + inReadLock(partitionMetadataLock) { + cache.contains(topic) + } + } + + def getAllTopics(): Set[String] = { + inReadLock(partitionMetadataLock) { + cache.keySet.toSet + } + } + + def getNonExistingTopics(topics: Set[String]): Set[String] = { + inReadLock(partitionMetadataLock) { + topics -- cache.keySet + } + } + + def getAliveBrokers: Seq[Broker] = { inReadLock(partitionMetadataLock) { aliveBrokers.values.toSeq } } - def addOrUpdatePartitionInfo(topic: String, - partitionId: Int, - stateInfo: PartitionStateInfo) { + private def addOrUpdatePartitionInfo(topic: String, + partitionId: Int, + stateInfo: PartitionStateInfo) { inWriteLock(partitionMetadataLock) { - cache.get(topic) match { - case Some(infos) => infos.put(partitionId, stateInfo) - case None => { - val newInfos: mutable.Map[Int, PartitionStateInfo] = new mutable.HashMap[Int, PartitionStateInfo] - cache.put(topic, newInfos) - newInfos.put(partitionId, stateInfo) - } - } + val infos = cache.getOrElseUpdate(topic, mutable.Map()) + infos(partitionId) = stateInfo } } def getPartitionInfo(topic: String, partitionId: Int): Option[PartitionStateInfo] = { inReadLock(partitionMetadataLock) { - cache.get(topic) match { - case Some(partitionInfos) => partitionInfos.get(partitionId) - case None => None - } + cache.get(topic).flatMap(_.get(partitionId)) } } - def updateCache(correlationId: Int, updateMetadataRequest: UpdateMetadataRequest, - brokerId: Int, - stateChangeLogger: StateChangeLogger) { + def updateCache(correlationId: Int, updateMetadataRequest: UpdateMetadataRequest) { inWriteLock(partitionMetadataLock) { - aliveBrokers = updateMetadataRequest.liveBrokers.asScala.map { broker => - val endPoints = broker.endPoints.asScala.map { case (protocol, ep) => - (protocol, EndPoint(ep.host, ep.port, protocol)) - }.toMap - (broker.id, Broker(broker.id, endPoints)) - }.toMap + aliveNodes.clear() + aliveBrokers.clear() + updateMetadataRequest.liveBrokers.asScala.foreach { broker => + val nodes = new EnumMap[SecurityProtocol, Node](classOf[SecurityProtocol]) + val endPoints = new EnumMap[SecurityProtocol, EndPoint](classOf[SecurityProtocol]) + broker.endPoints.asScala.foreach { case (protocol, ep) => + endPoints.put(protocol, EndPoint(ep.host, ep.port, protocol)) + nodes.put(protocol, new Node(broker.id, ep.host, ep.port)) + } + aliveBrokers(broker.id) = Broker(broker.id, endPoints.asScala) + aliveNodes(broker.id) = nodes.asScala + } updateMetadataRequest.partitionStates.asScala.foreach { case (tp, info) => if (info.leader == LeaderAndIsr.LeaderDuringDelete) { @@ -167,16 +194,12 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { } } - private def removePartitionInfo(topic: String, partitionId: Int) = { - cache.get(topic) match { - case Some(infos) => { - infos.remove(partitionId) - if(infos.isEmpty) { - cache.remove(topic) - } - true - } - case None => false - } + private def removePartitionInfo(topic: String, partitionId: Int): Boolean = { + cache.get(topic).map { infos => + infos.remove(partitionId) + if (infos.isEmpty) cache.remove(topic) + true + }.getOrElse(false) } + } diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index e388d98046c15..56553134f247f 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -582,7 +582,7 @@ class ReplicaManager(val config: KafkaConfig, stateChangeLogger.warn(stateControllerEpochErrorMessage) throw new ControllerMovedException(stateControllerEpochErrorMessage) } else { - metadataCache.updateCache(correlationId, updateMetadataRequest, localBrokerId, stateChangeLogger) + metadataCache.updateCache(correlationId, updateMetadataRequest) controllerEpoch = updateMetadataRequest.controllerEpoch } } diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index ccc86dfec5e1e..f39ed014cec9b 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -264,7 +264,7 @@ class ZkUtils(val zkClient: ZkClient, * @param advertisedEndpoints * @param jmxPort */ - def registerBrokerInZk(id: Int, host: String, port: Int, advertisedEndpoints: immutable.Map[SecurityProtocol, EndPoint], jmxPort: Int) { + def registerBrokerInZk(id: Int, host: String, port: Int, advertisedEndpoints: collection.Map[SecurityProtocol, EndPoint], jmxPort: Int) { val brokerIdPath = BrokerIdsPath + "/" + id val timestamp = SystemTime.milliseconds.toString diff --git a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala index fafc4b0aafc26..ca9dac424db21 100644 --- a/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala +++ b/core/src/test/scala/unit/kafka/api/RequestResponseSerializationTest.scala @@ -145,14 +145,6 @@ object SerializationTestUtils { ) } - def createTestTopicMetadataRequest: TopicMetadataRequest = { - new TopicMetadataRequest(1, 1, "client 1", Seq(topic1, topic2)) - } - - def createTestTopicMetadataResponse: TopicMetadataResponse = { - new TopicMetadataResponse(brokers.map(_.getBrokerEndPoint(SecurityProtocol.PLAINTEXT)).toSeq, Seq(topicmetaData1, topicmetaData2), 1) - } - def createTestOffsetCommitRequestV2: OffsetCommitRequest = { new OffsetCommitRequest( groupId = "group 1", @@ -217,8 +209,6 @@ class RequestResponseSerializationTest extends JUnitSuite { private val fetchRequest = SerializationTestUtils.createTestFetchRequest private val offsetRequest = SerializationTestUtils.createTestOffsetRequest private val offsetResponse = SerializationTestUtils.createTestOffsetResponse - private val topicMetadataRequest = SerializationTestUtils.createTestTopicMetadataRequest - private val topicMetadataResponse = SerializationTestUtils.createTestTopicMetadataResponse private val offsetCommitRequestV0 = SerializationTestUtils.createTestOffsetCommitRequestV0 private val offsetCommitRequestV1 = SerializationTestUtils.createTestOffsetCommitRequestV1 private val offsetCommitRequestV2 = SerializationTestUtils.createTestOffsetCommitRequestV2 @@ -234,8 +224,7 @@ class RequestResponseSerializationTest extends JUnitSuite { val requestsAndResponses = collection.immutable.Seq(producerRequest, producerResponse, - fetchRequest, offsetRequest, offsetResponse, topicMetadataRequest, - topicMetadataResponse, + fetchRequest, offsetRequest, offsetResponse, offsetCommitRequestV0, offsetCommitRequestV1, offsetCommitRequestV2, offsetCommitResponse, offsetFetchRequest, offsetFetchResponse, consumerMetadataRequest, consumerMetadataResponse, diff --git a/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala b/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala index 2400cfbb672cb..7c9f3aee22bf8 100644 --- a/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala +++ b/core/src/test/scala/unit/kafka/integration/BaseTopicMetadataTest.scala @@ -18,10 +18,9 @@ package kafka.integration import java.io.File -import java.nio.ByteBuffer import kafka.admin.AdminUtils -import kafka.api.{TopicMetadataRequest, TopicMetadataResponse} +import kafka.api.TopicMetadataResponse import kafka.client.ClientUtils import kafka.cluster.{Broker, BrokerEndPoint} import kafka.server.{KafkaConfig, KafkaServer, NotRunning} @@ -63,23 +62,6 @@ abstract class BaseTopicMetadataTest extends ZooKeeperTestHarness { super.tearDown() } - @Test - def testTopicMetadataRequest { - // create topic - val topic = "test" - AdminUtils.createTopic(zkUtils, topic, 1, 1) - - // create a topic metadata request - val topicMetadataRequest = new TopicMetadataRequest(List(topic), 0) - - val serializedMetadataRequest = ByteBuffer.allocate(topicMetadataRequest.sizeInBytes + 2) - topicMetadataRequest.writeTo(serializedMetadataRequest) - serializedMetadataRequest.rewind() - val deserializedMetadataRequest = TopicMetadataRequest.readFrom(serializedMetadataRequest) - - assertEquals(topicMetadataRequest, deserializedMetadataRequest) - } - @Test def testBasicTopicMetadata { // create topic diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala index 12b3583d01e5a..de19f6f1b87e7 100755 --- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -353,7 +353,7 @@ class ProducerTest extends ZooKeeperTestHarness with Logging{ // create topic AdminUtils.createTopic(zkUtils, "new-topic", 2, 1) TestUtils.waitUntilTrue(() => - AdminUtils.fetchTopicMetadataFromZk("new-topic", zkUtils).errorCode != Errors.UNKNOWN_TOPIC_OR_PARTITION.code, + AdminUtils.fetchTopicMetadataFromZk("new-topic", zkUtils).error != Errors.UNKNOWN_TOPIC_OR_PARTITION, "Topic new-topic not created after timeout", waitTime = zookeeper.tickTime) TestUtils.waitUntilLeaderIsElectedOrChanged(zkUtils, "new-topic", 0) diff --git a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala new file mode 100644 index 0000000000000..f3f0c8709f143 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala @@ -0,0 +1,199 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.server + +import java.util +import util.Arrays.asList + +import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.protocol.{Errors, SecurityProtocol} +import org.apache.kafka.common.requests.UpdateMetadataRequest +import org.apache.kafka.common.requests.UpdateMetadataRequest.{PartitionState, Broker, EndPoint} +import org.junit.Test +import org.junit.Assert._ + +import scala.collection.JavaConverters._ + +class MetadataCacheTest { + + private def asSet[T](elems: T*): util.Set[T] = new util.HashSet(elems.asJava) + + @Test + def getTopicMetadataNonExistingTopics() { + val topic = "topic" + val cache = new MetadataCache(1) + val topicMetadata = cache.getTopicMetadata(Set(topic), SecurityProtocol.PLAINTEXT) + assertTrue(topicMetadata.isEmpty) + } + + @Test + def getTopicMetadata() { + val topic = "topic" + + val cache = new MetadataCache(1) + + val zkVersion = 3 + val controllerId = 2 + val controllerEpoch = 1 + val brokers = Set( + new Broker(0, Map(SecurityProtocol.PLAINTEXT -> new EndPoint("foo", 9092)).asJava), + new Broker(1, Map(SecurityProtocol.PLAINTEXT -> new EndPoint("foo", 9092)).asJava), + new Broker(2, Map(SecurityProtocol.PLAINTEXT -> new EndPoint("foo", 9092)).asJava)) + + val partitionStates = Map( + new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, 0, 0, asList(0), zkVersion, asSet(0)), + new TopicPartition(topic, 1) -> new PartitionState(controllerEpoch, 1, 1, asList(1), zkVersion, asSet(1)), + new TopicPartition(topic, 2) -> new PartitionState(controllerEpoch, 2, 2, asList(2), zkVersion, asSet(2))) + + val updateMetadataRequest = new UpdateMetadataRequest(controllerId, controllerEpoch, partitionStates.asJava, brokers.asJava) + cache.updateCache(15, updateMetadataRequest) + + val topicMetadatas = cache.getTopicMetadata(Set(topic), SecurityProtocol.PLAINTEXT) + assertEquals(1, topicMetadatas.size) + + val topicMetadata = topicMetadatas.head + assertEquals(Errors.NONE, topicMetadata.error) + assertEquals(topic, topicMetadata.topic) + + val partitionMetadatas = topicMetadata.partitionMetadata.asScala.sortBy(_.partition) + assertEquals(3, partitionMetadatas.size) + + for (i <- 0 to 2) { + val partitionMetadata = partitionMetadatas(i) + assertEquals(Errors.NONE, partitionMetadata.error) + assertEquals(i, partitionMetadata.partition) + assertEquals(i, partitionMetadata.leader.id) + assertEquals(List(i), partitionMetadata.isr.asScala.map(_.id)) + assertEquals(List(i), partitionMetadata.replicas.asScala.map(_.id)) + } + } + + @Test + def getTopicMetadataPartitionLeaderNotAvailable() { + val topic = "topic" + + val cache = new MetadataCache(1) + + val zkVersion = 3 + val controllerId = 2 + val controllerEpoch = 1 + val brokers = Set(new Broker(0, Map(SecurityProtocol.PLAINTEXT -> new EndPoint("foo", 9092)).asJava)) + + val leader = 1 + val leaderEpoch = 1 + val partitionStates = Map( + new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, asList(0), zkVersion, asSet(0))) + + val updateMetadataRequest = new UpdateMetadataRequest(controllerId, controllerEpoch, partitionStates.asJava, brokers.asJava) + cache.updateCache(15, updateMetadataRequest) + + val topicMetadatas = cache.getTopicMetadata(Set(topic), SecurityProtocol.PLAINTEXT) + assertEquals(1, topicMetadatas.size) + + val topicMetadata = topicMetadatas.head + assertEquals(Errors.NONE, topicMetadata.error) + + val partitionMetadatas = topicMetadata.partitionMetadata + assertEquals(1, partitionMetadatas.size) + + val partitionMetadata = partitionMetadatas.get(0) + assertEquals(0, partitionMetadata.partition) + assertEquals(Errors.LEADER_NOT_AVAILABLE, partitionMetadata.error) + assertTrue(partitionMetadata.isr.isEmpty) + assertEquals(1, partitionMetadata.replicas.size) + assertEquals(0, partitionMetadata.replicas.get(0).id) + } + + @Test + def getTopicMetadataReplicaNotAvailable() { + val topic = "topic" + + val cache = new MetadataCache(1) + + val zkVersion = 3 + val controllerId = 2 + val controllerEpoch = 1 + val brokers = Set(new Broker(0, Map(SecurityProtocol.PLAINTEXT -> new EndPoint("foo", 9092)).asJava)) + + // replica 1 is not available + val leader = 0 + val leaderEpoch = 0 + val replicas = asSet[Integer](0, 1) + val isr = asList[Integer](0) + + val partitionStates = Map( + new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas)) + + val updateMetadataRequest = new UpdateMetadataRequest(controllerId, controllerEpoch, partitionStates.asJava, brokers.asJava) + cache.updateCache(15, updateMetadataRequest) + + val topicMetadatas = cache.getTopicMetadata(Set(topic), SecurityProtocol.PLAINTEXT) + assertEquals(1, topicMetadatas.size) + + val topicMetadata = topicMetadatas.head + assertEquals(Errors.NONE, topicMetadata.error) + + val partitionMetadatas = topicMetadata.partitionMetadata + assertEquals(1, partitionMetadatas.size) + + val partitionMetadata = partitionMetadatas.get(0) + assertEquals(0, partitionMetadata.partition) + assertEquals(Errors.REPLICA_NOT_AVAILABLE, partitionMetadata.error) + assertEquals(Set(0), partitionMetadata.replicas.asScala.map(_.id).toSet) + assertEquals(Set(0), partitionMetadata.isr.asScala.map(_.id).toSet) + } + + @Test + def getTopicMetadataIsrNotAvailable() { + val topic = "topic" + + val cache = new MetadataCache(1) + + val zkVersion = 3 + val controllerId = 2 + val controllerEpoch = 1 + val brokers = Set(new Broker(0, Map(SecurityProtocol.PLAINTEXT -> new EndPoint("foo", 9092)).asJava)) + + // replica 1 is not available + val leader = 0 + val leaderEpoch = 0 + val replicas = asSet[Integer](0) + val isr = asList[Integer](0, 1) + + val partitionStates = Map( + new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, zkVersion, replicas)) + + val updateMetadataRequest = new UpdateMetadataRequest(controllerId, controllerEpoch, partitionStates.asJava, brokers.asJava) + cache.updateCache(15, updateMetadataRequest) + + val topicMetadatas = cache.getTopicMetadata(Set(topic), SecurityProtocol.PLAINTEXT) + assertEquals(1, topicMetadatas.size) + + val topicMetadata = topicMetadatas.head + assertEquals(Errors.NONE, topicMetadata.error) + + val partitionMetadatas = topicMetadata.partitionMetadata + assertEquals(1, partitionMetadatas.size) + + val partitionMetadata = partitionMetadatas.get(0) + assertEquals(0, partitionMetadata.partition) + assertEquals(Errors.REPLICA_NOT_AVAILABLE, partitionMetadata.error) + assertEquals(Set(0), partitionMetadata.replicas.asScala.map(_.id).toSet) + assertEquals(Set(0), partitionMetadata.isr.asScala.map(_.id).toSet) + } + +} From 02d4da5f64989b41358cdfd94d95b94fb4e20198 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Fri, 11 Mar 2016 11:22:15 -0800 Subject: [PATCH 021/206] KAFKA-2960 KAFKA-1148; Clear purgatory for partitions before becoming follower Author: Jiangjie Qin Reviewers: Aditya Auradkar , Ismael Juma , Joel Koshy , Jun Rao , Guozhang Wang Closes #1018 from becketqin/KAFKA-2960 --- .../scala/kafka/server/ReplicaManager.scala | 8 +- .../kafka/server/ReplicaManagerTest.scala | 109 +++++++++++++++--- 2 files changed, 100 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 56553134f247f..de58e569ddf72 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -737,7 +737,8 @@ class ReplicaManager(val config: KafkaConfig, * 2. Mark the replicas as followers so that no more data can be added from the producer clients. * 3. Stop fetchers for these partitions so that no more data can be added by the replica fetcher threads. * 4. Truncate the log and checkpoint offsets for these partitions. - * 5. If the broker is not shutting down, add the fetcher to the new leaders. + * 5. Clear the produce and fetch requests in the purgatory + * 6. If the broker is not shutting down, add the fetcher to the new leaders. * * The ordering of doing these steps make sure that the replicas in transition will not * take any more messages before checkpointing offsets so that all messages before the checkpoint @@ -800,6 +801,11 @@ class ReplicaManager(val config: KafkaConfig, } logManager.truncateTo(partitionsToMakeFollower.map(partition => (new TopicAndPartition(partition), partition.getOrCreateReplica().highWatermark.messageOffset)).toMap) + partitionsToMakeFollower.foreach { partition => + val topicPartitionOperationKey = new TopicPartitionOperationKey(partition.topic, partition.partitionId) + tryCompleteDelayedProduce(topicPartitionOperationKey) + tryCompleteDelayedFetch(topicPartitionOperationKey) + } partitionsToMakeFollower.foreach { partition => stateChangeLogger.trace(("Broker %d truncated logs and checkpointed recovery boundaries for partition [%s,%d] as part of " + diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index 32085f6ae0d7e..a5a8df1e32aaf 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -18,25 +18,28 @@ package kafka.server -import kafka.api.SerializationTestUtils -import kafka.message.{Message, ByteBufferMessageSet} -import kafka.utils.{ZkUtils, MockScheduler, MockTime, TestUtils} -import org.apache.kafka.common.requests.ProduceRequest - -import java.util.concurrent.atomic.AtomicBoolean import java.io.File +import java.util.concurrent.atomic.AtomicBoolean +import kafka.api.{FetchResponsePartitionData, PartitionFetchInfo} +import kafka.cluster.Broker +import kafka.common.TopicAndPartition +import kafka.message.{ByteBufferMessageSet, Message} +import kafka.utils.{MockScheduler, MockTime, TestUtils, ZkUtils} +import org.I0Itec.zkclient.ZkClient import org.apache.kafka.common.metrics.Metrics -import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.requests.LeaderAndIsrRequest +import org.apache.kafka.common.requests.LeaderAndIsrRequest.PartitionState import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.utils.{MockTime => JMockTime} +import org.apache.kafka.common.{BrokerEndPoint, TopicPartition} import org.easymock.EasyMock -import org.I0Itec.zkclient.ZkClient +import org.junit.Assert.{assertEquals, assertTrue} import org.junit.Test -import scala.collection.Map import scala.collection.JavaConverters._ +import scala.collection.Map class ReplicaManagerTest { @@ -47,9 +50,9 @@ class ReplicaManagerTest { val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) val config = KafkaConfig.fromProps(props) val zkClient = EasyMock.createMock(classOf[ZkClient]) - val zkUtils = ZkUtils(zkClient, false) + val zkUtils = ZkUtils(zkClient, isZkSecurityEnabled = false) val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) - val time: MockTime = new MockTime() + val time = new MockTime() val jTime = new JMockTime val metrics = new Metrics val rm = new ReplicaManager(config, metrics, time, jTime, zkUtils, new MockScheduler(time), mockLogMgr, @@ -73,7 +76,7 @@ class ReplicaManagerTest { val zkClient = EasyMock.createMock(classOf[ZkClient]) val zkUtils = ZkUtils(zkClient, false) val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) - val time: MockTime = new MockTime() + val time = new MockTime() val jTime = new JMockTime val metrics = new Metrics val rm = new ReplicaManager(config, metrics, time, jTime, zkUtils, new MockScheduler(time), mockLogMgr, @@ -84,7 +87,7 @@ class ReplicaManagerTest { rm.checkpointHighWatermarks() } finally { // shutdown the replica manager upon test completion - rm.shutdown(false) + rm.shutdown(checkpointHW = false) metrics.close() } } @@ -94,9 +97,9 @@ class ReplicaManagerTest { val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) val config = KafkaConfig.fromProps(props) val zkClient = EasyMock.createMock(classOf[ZkClient]) - val zkUtils = ZkUtils(zkClient, false) + val zkUtils = ZkUtils(zkClient, isZkSecurityEnabled = false) val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) - val time: MockTime = new MockTime() + val time = new MockTime() val jTime = new JMockTime val metrics = new Metrics val rm = new ReplicaManager(config, metrics, time, jTime, zkUtils, new MockScheduler(time), mockLogMgr, @@ -112,10 +115,84 @@ class ReplicaManagerTest { messagesPerPartition = Map(new TopicPartition("test1", 0) -> new ByteBufferMessageSet(new Message("first message".getBytes))), responseCallback = callback) } finally { - rm.shutdown(false) + rm.shutdown(checkpointHW = false) metrics.close() } TestUtils.verifyNonDaemonThreadsStatus(this.getClass.getName) } + + @Test + def testClearPurgatoryOnBecomingFollower() { + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) + props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath) + val config = KafkaConfig.fromProps(props) + val zkClient = EasyMock.createMock(classOf[ZkClient]) + val zkUtils = ZkUtils(zkClient, isZkSecurityEnabled = false) + val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) + val time = new MockTime() + val jTime = new JMockTime + val metrics = new Metrics + val rm = new ReplicaManager(config, metrics, time, jTime, zkUtils, new MockScheduler(time), mockLogMgr, + new AtomicBoolean(false)) + + try { + var produceCallbackFired = false + def produceCallback(responseStatus: Map[TopicPartition, PartitionResponse]) = { + assertEquals("Should give NotLeaderForPartitionException", Errors.NOT_LEADER_FOR_PARTITION.code, responseStatus.values.head.errorCode) + produceCallbackFired = true + } + + var fetchCallbackFired = false + def fetchCallback(responseStatus: Map[TopicAndPartition, FetchResponsePartitionData]) = { + assertEquals("Should give NotLeaderForPartitionException", Errors.NOT_LEADER_FOR_PARTITION.code, responseStatus.values.head.error) + fetchCallbackFired = true + } + + val aliveBrokers = Seq(new Broker(0, "host0", 0), new Broker(1, "host1", 1)) + val metadataCache = EasyMock.createMock(classOf[MetadataCache]) + EasyMock.expect(metadataCache.getAliveBrokers).andReturn(aliveBrokers).anyTimes() + EasyMock.replay(metadataCache) + + val brokerList : java.util.List[Integer] = Seq[Integer](0, 1).asJava + val brokerSet : java.util.Set[Integer] = Set[Integer](0, 1).asJava + + val partition = rm.getOrCreatePartition(topic, 0) + partition.getOrCreateReplica(0) + // Make this replica the leader. + val leaderAndIsrRequest1 = new LeaderAndIsrRequest(0, 0, + collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerSet)).asJava, + Set(new BrokerEndPoint(0, "host1", 0), new BrokerEndPoint(1, "host2", 1)).asJava) + rm.becomeLeaderOrFollower(0, leaderAndIsrRequest1, metadataCache, (_, _) => {}) + rm.getLeaderReplicaIfLocal(topic, 0) + + // Append a message. + rm.appendMessages( + timeout = 1000, + requiredAcks = -1, + internalTopicsAllowed = false, + messagesPerPartition = Map(new TopicPartition(topic, 0) -> new ByteBufferMessageSet(new Message("first message".getBytes))), + responseCallback = produceCallback) + + // Fetch some messages + rm.fetchMessages( + timeout = 1000, + replicaId = -1, + fetchMinBytes = 100000, + fetchInfo = collection.immutable.Map(new TopicAndPartition(topic, 0) -> new PartitionFetchInfo(0, 100000)), + responseCallback = fetchCallback) + + // Make this replica the follower + val leaderAndIsrRequest2 = new LeaderAndIsrRequest(0, 0, + collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 1, 1, brokerList, 0, brokerSet)).asJava, + Set(new BrokerEndPoint(0, "host1", 0), new BrokerEndPoint(1, "host2", 1)).asJava) + rm.becomeLeaderOrFollower(1, leaderAndIsrRequest2, metadataCache, (_, _) => {}) + + assertTrue(produceCallbackFired) + assertTrue(fetchCallbackFired) + } finally { + rm.shutdown(checkpointHW = false) + metrics.close() + } + } } From a162f6bf66d0d21505c8d11942f84be446616491 Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Fri, 11 Mar 2016 12:17:45 -0800 Subject: [PATCH 022/206] KAFKA-3380; Add system test for GetOffsetShell tool Author: Ashish Singh Reviewers: Gwen Shapira Closes #1048 from SinghAsDev/KAFKA-3380 --- tests/kafkatest/services/kafka/kafka.py | 20 +++- .../kafkatest/tests/get_offset_shell_test.py | 91 +++++++++++++++++++ 2 files changed, 110 insertions(+), 1 deletion(-) create mode 100644 tests/kafkatest/tests/get_offset_shell_test.py diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index 358dacfb26a7e..788d41bb4f12a 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -490,4 +490,22 @@ def controller(self): controller_idx = int(controller_info["brokerid"]) self.logger.info("Controller's ID: %d" % (controller_idx)) - return self.get_node(controller_idx) \ No newline at end of file + return self.get_node(controller_idx) + + def get_offset_shell(self, topic, partitions, max_wait_ms, offsets, time): + node = self.nodes[0] + + cmd = "/opt/%s/bin/" % kafka_dir(node) + cmd += "kafka-run-class.sh kafka.tools.GetOffsetShell" + cmd += " --topic %s --broker-list %s --max-wait-ms %s --offsets %s --time %s" % (topic, self.bootstrap_servers(self.security_protocol), max_wait_ms, offsets, time) + + if partitions: + cmd += ' --partitions %s' % partitions + + cmd += " 2>> /mnt/get_offset_shell.log | tee -a /mnt/get_offset_shell.log &" + output = "" + self.logger.debug(cmd) + for line in node.account.ssh_capture(cmd): + output += line + self.logger.debug(output) + return output \ No newline at end of file diff --git a/tests/kafkatest/tests/get_offset_shell_test.py b/tests/kafkatest/tests/get_offset_shell_test.py new file mode 100644 index 0000000000000..38bd9dc46d8fe --- /dev/null +++ b/tests/kafkatest/tests/get_offset_shell_test.py @@ -0,0 +1,91 @@ +# 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. + + +from ducktape.utils.util import wait_until +from ducktape.tests.test import Test +from kafkatest.services.verifiable_producer import VerifiableProducer + +from kafkatest.services.zookeeper import ZookeeperService +from kafkatest.services.kafka import KafkaService +from kafkatest.services.console_consumer import ConsoleConsumer +from kafkatest.services.security.security_config import SecurityConfig + +TOPIC = "topic-get-offset-shell" +MAX_MESSAGES = 100 +NUM_PARTITIONS = 1 +REPLICATION_FACTOR = 1 + +class GetOffsetShellTest(Test): + """ + Tests GetOffsetShell tool + """ + def __init__(self, test_context): + super(GetOffsetShellTest, self).__init__(test_context) + self.num_zk = 1 + self.num_brokers = 1 + self.messages_received_count = 0 + self.topics = { + TOPIC: {'partitions': NUM_PARTITIONS, 'replication-factor': REPLICATION_FACTOR} + } + + self.zk = ZookeeperService(test_context, self.num_zk) + + + + def setUp(self): + self.zk.start() + + def start_kafka(self, security_protocol, interbroker_security_protocol): + self.kafka = KafkaService( + self.test_context, self.num_brokers, + self.zk, security_protocol=security_protocol, + interbroker_security_protocol=interbroker_security_protocol, topics=self.topics) + self.kafka.start() + + def start_producer(self): + # This will produce to kafka cluster + self.producer = VerifiableProducer(self.test_context, num_nodes=1, kafka=self.kafka, topic=TOPIC, throughput=1000, max_messages=MAX_MESSAGES) + self.producer.start() + current_acked = self.producer.num_acked + wait_until(lambda: self.producer.num_acked >= current_acked + MAX_MESSAGES, timeout_sec=10, + err_msg="Timeout awaiting messages to be produced and acked") + + def start_consumer(self, security_protocol): + enable_new_consumer = security_protocol != SecurityConfig.PLAINTEXT + self.consumer = ConsoleConsumer(self.test_context, num_nodes=self.num_brokers, kafka=self.kafka, topic=TOPIC, + consumer_timeout_ms=1000, new_consumer=enable_new_consumer) + self.consumer.start() + + def test_get_offset_shell(self, security_protocol='PLAINTEXT'): + """ + Tests if GetOffsetShell is getting offsets correctly + :return: None + """ + self.start_kafka(security_protocol, security_protocol) + self.start_producer() + + # Assert that offset fetched without any consumers consuming is 0 + assert self.kafka.get_offset_shell(TOPIC, None, 1000, 1, -1), "%s:%s:%s" % (TOPIC, NUM_PARTITIONS - 1, 0) + + self.start_consumer(security_protocol) + + node = self.consumer.nodes[0] + + wait_until(lambda: self.consumer.alive(node), timeout_sec=10, backoff_sec=.2, err_msg="Consumer was too slow to start") + + # Assert that offset is correctly indicated by GetOffsetShell tool + wait_until(lambda: "%s:%s:%s" % (TOPIC, NUM_PARTITIONS - 1, MAX_MESSAGES) in self.kafka.get_offset_shell(TOPIC, None, 1000, 1, -1), timeout_sec=10, + err_msg="Timed out waiting to reach expected offset.") \ No newline at end of file From c9311d5f4ec3b135cb6c0f87008da946863daaa2 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 11 Mar 2016 17:53:32 -0800 Subject: [PATCH 023/206] KAFKA-3047: Explicit offset assignment in Log.append can corrupt the log This fix was suggested by Maciek Makowski, who also reported the problem. Author: Ismael Juma Reviewers: Guozhang Wang Closes #1029 from ijuma/KAFKA-3047-log-append-can-corrupt-the-log --- core/src/main/scala/kafka/log/Log.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index fd176b1e9eaa3..8c956f7378f58 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -328,11 +328,11 @@ class Log(val dir: File, try { // they are valid, insert them in the log lock synchronized { - appendInfo.firstOffset = nextOffsetMetadata.messageOffset if (assignOffsets) { // assign offsets to the message set val offset = new LongRef(nextOffsetMetadata.messageOffset) + appendInfo.firstOffset = offset.value val now = time.milliseconds val (validatedMessages, messageSizesMaybeChanged) = try { validMessages.validateMessagesAndAssignOffsets(offset, From ac7b2e95d342972e3499d203bc23e1675e90c591 Mon Sep 17 00:00:00 2001 From: Manikumar reddy O Date: Sun, 13 Mar 2016 20:32:47 -0700 Subject: [PATCH 024/206] KAFKA-2551; Update Unclean leader election docs Author: Manikumar reddy O Reviewers: Ismael Juma , Sriharsha Chintalapani Closes #1054 from omkreddy/KAFKA-2551 --- docs/design.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/design.html b/docs/design.html index c04b772e2299c..ad40431eeb18c 100644 --- a/docs/design.html +++ b/docs/design.html @@ -240,7 +240,7 @@

Unclean leader ele
  • Choose the first replica (not necessarily in the ISR) that comes back to life as the leader.

    -This is a simple tradeoff between availability and consistency. If we wait for replicas in the ISR, then we will remain unavailable as long as those replicas are down. If such replicas were destroyed or their data was lost, then we are permanently down. If, on the other hand, a non-in-sync replica comes back to life and we allow it to become leader, then its log becomes the source of truth even though it is not guaranteed to have every committed message. In our current release we choose the second strategy and favor choosing a potentially inconsistent replica when all replicas in the ISR are dead. In the future, we would like to make this configurable to better support use cases where downtime is preferable to inconsistency. +This is a simple tradeoff between availability and consistency. If we wait for replicas in the ISR, then we will remain unavailable as long as those replicas are down. If such replicas were destroyed or their data was lost, then we are permanently down. If, on the other hand, a non-in-sync replica comes back to life and we allow it to become leader, then its log becomes the source of truth even though it is not guaranteed to have every committed message. By default Kafka chooses the second strategy and favor choosing a potentially inconsistent replica when all replicas in the ISR are dead. This behavior can be disabled using configuration property unclean.leader.election.enable, to support use cases where downtime is preferable to inconsistency.

    This dilemma is not specific to Kafka. It exists in any quorum-based scheme. For example in a majority voting scheme, if a majority of servers suffer a permanent failure, then you must either choose to lose 100% of your data or violate consistency by taking what remains on an existing server as your new source of truth. From c1a56c6839e77f3de5266315a92b236a379ec857 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 14 Mar 2016 14:50:24 -0700 Subject: [PATCH 025/206] KAFKA-3395: prefix job id to internal topic names guozhangwang Author: Yasuhiro Matsuda Reviewers: Guozhang Wang Closes #1062 from ymatsuda/k3395 --- .../streams/processor/TopologyBuilder.java | 76 +++++++++++++------ .../internals/StreamPartitionAssignor.java | 6 +- .../processor/internals/StreamThread.java | 6 +- .../kstream/internals/KStreamImplTest.java | 2 +- .../processor/TopologyBuilderTest.java | 28 ++++--- .../internals/ProcessorTopologyTest.java | 2 +- .../StreamPartitionAssignorTest.java | 6 +- .../processor/internals/StreamThreadTest.java | 6 +- .../apache/kafka/test/KStreamTestDriver.java | 4 +- .../test/ProcessorTopologyTestDriver.java | 2 +- tests/kafkatest/tests/streams_bounce_test.py | 1 - tests/kafkatest/tests/streams_smoke_test.py | 1 - 12 files changed, 84 insertions(+), 56 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java index 7e3cab9054e39..6e5aec5413b56 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java @@ -85,7 +85,7 @@ private static abstract class NodeFactory { this.name = name; } - public abstract ProcessorNode build(); + public abstract ProcessorNode build(String jobId); } private static class ProcessorNodeFactory extends NodeFactory { @@ -105,7 +105,7 @@ public void addStateStore(String stateStoreName) { @SuppressWarnings("unchecked") @Override - public ProcessorNode build() { + public ProcessorNode build(String jobId) { return new ProcessorNode(name, supplier.get(), stateStoreNames); } } @@ -124,12 +124,12 @@ private SourceNodeFactory(String name, String[] topics, Deserializer keyDeserial @SuppressWarnings("unchecked") @Override - public ProcessorNode build() { + public ProcessorNode build(String jobId) { return new SourceNode(name, keyDeserializer, valDeserializer); } } - private static class SinkNodeFactory extends NodeFactory { + private class SinkNodeFactory extends NodeFactory { public final String[] parents; public final String topic; private Serializer keySerializer; @@ -147,8 +147,13 @@ private SinkNodeFactory(String name, String[] parents, String topic, Serializer @SuppressWarnings("unchecked") @Override - public ProcessorNode build() { - return new SinkNode(name, topic, keySerializer, valSerializer, partitioner); + public ProcessorNode build(String jobId) { + if (internalTopicNames.contains(topic)) { + // prefix the job id to the internal topic name + return new SinkNode(name, jobId + "-" + topic, keySerializer, valSerializer, partitioner); + } else { + return new SinkNode(name, topic, keySerializer, valSerializer, partitioner); + } } } @@ -491,7 +496,7 @@ private void connectProcessorAndStateStore(String processorName, String stateSto * * @return groups of topic names */ - public Map topicGroups() { + public Map topicGroups(String jobId) { Map topicGroups = new HashMap<>(); if (nodeGroups == null) @@ -506,27 +511,35 @@ public Map topicGroups() { // if the node is a source node, add to the source topics String[] topics = nodeToSourceTopics.get(node); if (topics != null) { - sourceTopics.addAll(Arrays.asList(topics)); - // if some of the topics are internal, add them to the internal topics for (String topic : topics) { - if (this.internalTopicNames.contains(topic)) - internalSourceTopics.add(topic); + if (this.internalTopicNames.contains(topic)) { + // prefix the job id to the internal topic name + String internalTopic = jobId + "-" + topic; + internalSourceTopics.add(internalTopic); + sourceTopics.add(internalTopic); + } else { + sourceTopics.add(topic); + } } } // if the node is a sink node, add to the sink topics String topic = nodeToSinkTopic.get(node); - if (topic != null) - sinkTopics.add(topic); + if (topic != null) { + if (internalTopicNames.contains(topic)) { + // prefix the job id to the change log topic name + sinkTopics.add(jobId + "-" + topic); + } else { + sinkTopics.add(topic); + } + } // if the node is connected to a state, add to the state topics for (StateStoreFactory stateFactory : stateFactories.values()) { - - // we store the changelog topic here without the job id prefix - // since it is within a single job and is only used for if (stateFactory.isInternal && stateFactory.users.contains(node)) { - stateChangelogTopics.add(stateFactory.supplier.name() + ProcessorStateManager.STATE_CHANGELOG_TOPIC_SUFFIX); + // prefix the job id to the change log topic name + stateChangelogTopics.add(jobId + "-" + stateFactory.supplier.name() + ProcessorStateManager.STATE_CHANGELOG_TOPIC_SUFFIX); } } } @@ -586,7 +599,7 @@ private Map> makeNodeGroups() { return nodeGroups; } - + /** * Asserts that the streams of the specified source nodes must be copartitioned. * @@ -624,7 +637,7 @@ public Collection> copartitionGroups() { * * @see org.apache.kafka.streams.KafkaStreams#KafkaStreams(TopologyBuilder, org.apache.kafka.streams.StreamsConfig) */ - public ProcessorTopology build(Integer topicGroupId) { + public ProcessorTopology build(String jobId, Integer topicGroupId) { Set nodeGroup; if (topicGroupId != null) { nodeGroup = nodeGroups().get(topicGroupId); @@ -632,11 +645,11 @@ public ProcessorTopology build(Integer topicGroupId) { // when nodeGroup is null, we build the full topology. this is used in some tests. nodeGroup = null; } - return build(nodeGroup); + return build(jobId, nodeGroup); } @SuppressWarnings("unchecked") - private ProcessorTopology build(Set nodeGroup) { + private ProcessorTopology build(String jobId, Set nodeGroup) { List processorNodes = new ArrayList<>(nodeFactories.size()); Map processorMap = new HashMap<>(); Map topicSourceMap = new HashMap<>(); @@ -645,7 +658,7 @@ private ProcessorTopology build(Set nodeGroup) { // create processor nodes in a topological order ("nodeFactories" is already topologically sorted) for (NodeFactory factory : nodeFactories.values()) { if (nodeGroup == null || nodeGroup.contains(factory.name)) { - ProcessorNode node = factory.build(); + ProcessorNode node = factory.build(jobId); processorNodes.add(node); processorMap.put(node.name(), node); @@ -660,7 +673,12 @@ private ProcessorTopology build(Set nodeGroup) { } } else if (factory instanceof SourceNodeFactory) { for (String topic : ((SourceNodeFactory) factory).topics) { - topicSourceMap.put(topic, (SourceNode) node); + if (internalTopicNames.contains(topic)) { + // prefix the job id to the internal topic name + topicSourceMap.put(jobId + "-" + topic, (SourceNode) node); + } else { + topicSourceMap.put(topic, (SourceNode) node); + } } } else if (factory instanceof SinkNodeFactory) { for (String parent : ((SinkNodeFactory) factory).parents) { @@ -679,7 +697,15 @@ private ProcessorTopology build(Set nodeGroup) { * Get the names of topics that are to be consumed by the source nodes created by this builder. * @return the unmodifiable set of topic names used by source nodes, which changes as new sources are added; never null */ - public Set sourceTopics() { - return Collections.unmodifiableSet(sourceTopicNames); + public Set sourceTopics(String jobId) { + Set topics = new HashSet<>(); + for (String topic : sourceTopicNames) { + if (internalTopicNames.contains(topic)) { + topics.add(jobId + "-" + topic); + } else { + topics.add(topic); + } + } + return Collections.unmodifiableSet(topics); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java index 13f269b5fc9b3..266df3ed09900 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java @@ -117,7 +117,7 @@ public void configure(Map configs) { streamThread = (StreamThread) o; streamThread.partitionAssignor(this); - this.topicGroups = streamThread.builder.topicGroups(); + this.topicGroups = streamThread.builder.topicGroups(streamThread.jobId); if (configs.containsKey(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG)) { internalTopicManager = new InternalTopicManager( @@ -350,7 +350,7 @@ public Map assign(Cluster metadata, Map> entry : topicToTaskIds.entrySet()) { - String topic = streamThread.jobId + "-" + entry.getKey(); + String topic = entry.getKey(); // the expected number of partitions is the max value of TaskId.partition + 1 int numPartitions = 0; @@ -445,7 +445,7 @@ private void ensureCopartitioning(Set copartitionGroup, Set inte /* For Test Only */ public Set tasksForState(String stateName) { - return stateChangelogTopicToTaskIds.get(stateName + ProcessorStateManager.STATE_CHANGELOG_TOPIC_SUFFIX); + return stateChangelogTopicToTaskIds.get(ProcessorStateManager.storeChangelogTopic(streamThread.jobId, stateName)); } public Set tasksForPartition(TopicPartition partition) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 4ce86ac91ca2e..e9343e0afeb82 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -173,7 +173,7 @@ public StreamThread(TopologyBuilder builder, this.jobId = jobId; this.config = config; this.builder = builder; - this.sourceTopics = builder.sourceTopics(); + this.sourceTopics = builder.sourceTopics(jobId); this.clientId = clientId; this.processId = processId; this.partitionGrouper = config.getConfiguredInstance(StreamsConfig.PARTITION_GROUPER_CLASS_CONFIG, PartitionGrouper.class); @@ -580,7 +580,7 @@ public Set cachedTasks() { protected StreamTask createStreamTask(TaskId id, Collection partitions) { sensors.taskCreationSensor.record(); - ProcessorTopology topology = builder.build(id.topicGroupId); + ProcessorTopology topology = builder.build(jobId, id.topicGroupId); return new StreamTask(id, jobId, partitions, topology, consumer, producer, restoreConsumer, config, sensors); } @@ -650,7 +650,7 @@ private void closeOne(AbstractTask task) { protected StandbyTask createStandbyTask(TaskId id, Collection partitions) { sensors.taskCreationSensor.record(); - ProcessorTopology topology = builder.build(id.topicGroupId); + ProcessorTopology topology = builder.build(jobId, id.topicGroupId); if (!topology.stateStoreSuppliers().isEmpty()) { return new StandbyTask(id, jobId, partitions, topology, consumer, restoreConsumer, config, sensors); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java index 1ce56ff78827a..3d3a9e3d6c4d2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java @@ -137,6 +137,6 @@ public Integer apply(Integer value1, Integer value2) { 1 + // to 2 + // through 1, // process - builder.build(null).processors().size()); + builder.build("X", null).processors().size()); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java index 0635bd20283b1..9af313a958fa8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/TopologyBuilderTest.java @@ -142,8 +142,14 @@ public void testSourceTopics() { builder.addSource("source-1", "topic-1"); builder.addSource("source-2", "topic-2"); builder.addSource("source-3", "topic-3"); + builder.addInternalTopic("topic-3"); - assertEquals(3, builder.sourceTopics().size()); + Set expected = new HashSet(); + expected.add("topic-1"); + expected.add("topic-2"); + expected.add("X-topic-3"); + + assertEquals(expected, builder.sourceTopics("X")); } @Test(expected = TopologyBuilderException.class) @@ -184,13 +190,13 @@ public void testAddStateStore() { StateStoreSupplier supplier = new MockStateStoreSupplier("store-1", false); builder.addStateStore(supplier); - suppliers = builder.build(null).stateStoreSuppliers(); + suppliers = builder.build("X", null).stateStoreSuppliers(); assertEquals(0, suppliers.size()); builder.addSource("source-1", "topic-1"); builder.addProcessor("processor-1", new MockProcessorSupplier(), "source-1"); builder.connectProcessorAndStateStores("processor-1", "store-1"); - suppliers = builder.build(null).stateStoreSuppliers(); + suppliers = builder.build("X", null).stateStoreSuppliers(); assertEquals(1, suppliers.size()); assertEquals(supplier.name(), suppliers.get(0).name()); } @@ -212,7 +218,7 @@ public void testTopicGroups() { builder.addProcessor("processor-3", new MockProcessorSupplier(), "source-3", "source-4"); - Map topicGroups = builder.topicGroups(); + Map topicGroups = builder.topicGroups("X"); Map expectedTopicGroups = new HashMap<>(); expectedTopicGroups.put(0, new TopicsInfo(Collections.emptySet(), mkSet("topic-1", "topic-1x", "topic-2"), Collections.emptySet(), Collections.emptySet())); @@ -250,12 +256,12 @@ public void testTopicGroupsByStateStore() { builder.addStateStore(supplier); builder.connectProcessorAndStateStores("processor-5", "store-3"); - Map topicGroups = builder.topicGroups(); + Map topicGroups = builder.topicGroups("X"); Map expectedTopicGroups = new HashMap<>(); - expectedTopicGroups.put(0, new TopicsInfo(Collections.emptySet(), mkSet("topic-1", "topic-1x", "topic-2"), Collections.emptySet(), mkSet("store-1" + ProcessorStateManager.STATE_CHANGELOG_TOPIC_SUFFIX))); - expectedTopicGroups.put(1, new TopicsInfo(Collections.emptySet(), mkSet("topic-3", "topic-4"), Collections.emptySet(), mkSet("store-2" + ProcessorStateManager.STATE_CHANGELOG_TOPIC_SUFFIX))); - expectedTopicGroups.put(2, new TopicsInfo(Collections.emptySet(), mkSet("topic-5"), Collections.emptySet(), mkSet("store-3" + ProcessorStateManager.STATE_CHANGELOG_TOPIC_SUFFIX))); + expectedTopicGroups.put(0, new TopicsInfo(Collections.emptySet(), mkSet("topic-1", "topic-1x", "topic-2"), Collections.emptySet(), mkSet(ProcessorStateManager.storeChangelogTopic("X", "store-1")))); + expectedTopicGroups.put(1, new TopicsInfo(Collections.emptySet(), mkSet("topic-3", "topic-4"), Collections.emptySet(), mkSet(ProcessorStateManager.storeChangelogTopic("X", "store-2")))); + expectedTopicGroups.put(2, new TopicsInfo(Collections.emptySet(), mkSet("topic-5"), Collections.emptySet(), mkSet(ProcessorStateManager.storeChangelogTopic("X", "store-3")))); assertEquals(3, topicGroups.size()); assertEquals(expectedTopicGroups, topicGroups); @@ -275,9 +281,9 @@ public void testBuild() { builder.addProcessor("processor-2", new MockProcessorSupplier(), "source-2", "processor-1"); builder.addProcessor("processor-3", new MockProcessorSupplier(), "source-3", "source-4"); - ProcessorTopology topology0 = builder.build(0); - ProcessorTopology topology1 = builder.build(1); - ProcessorTopology topology2 = builder.build(2); + ProcessorTopology topology0 = builder.build("X", 0); + ProcessorTopology topology1 = builder.build("X", 1); + ProcessorTopology topology2 = builder.build("X", 2); assertEquals(mkSet("source-1", "source-2", "processor-1", "processor-2"), nodeNames(topology0.processors())); assertEquals(mkSet("source-3", "source-4", "processor-3"), nodeNames(topology1.processors())); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java index 40cce93a79a69..c8115b86896cd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java @@ -97,7 +97,7 @@ public void testTopologyMetadata() { builder.addSink("sink-1", "topic-3", "processor-1"); builder.addSink("sink-2", "topic-4", "processor-1", "processor-2"); - final ProcessorTopology topology = builder.build(null); + final ProcessorTopology topology = builder.build("X", null); assertEquals(6, topology.processors().size()); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java index 9ff0af07af409..7f37bdafaab97 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java @@ -521,7 +521,7 @@ public void testAssignWithInternalTopics() throws Exception { builder.addSink("sink1", "topicX", "processor1"); builder.addSource("source2", "topicX"); builder.addProcessor("processor2", new MockProcessorSupplier(), "source2"); - List topics = Utils.mkList("topic1", "topicX"); + List topics = Utils.mkList("topic1", "test-topicX"); Set allTasks = Utils.mkSet(task0, task1, task2); UUID uuid1 = UUID.randomUUID(); @@ -543,9 +543,7 @@ public void testAssignWithInternalTopics() throws Exception { Map assignments = partitionAssignor.assign(metadata, subscriptions); // check prepared internal topics - // TODO: we need to change it to 1 after fixing the prefix - assertEquals(2, internalTopicManager.readyTopics.size()); - assertEquals(allTasks.size(), (long) internalTopicManager.readyTopics.get("topicX")); + assertEquals(1, internalTopicManager.readyTopics.size()); assertEquals(allTasks.size(), (long) internalTopicManager.readyTopics.get("test-topicX")); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index e0727476b8082..eaaf842b4f221 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -171,7 +171,7 @@ public void testPartitionAssignmentChange() throws Exception { StreamThread thread = new StreamThread(builder, config, producer, consumer, mockRestoreConsumer, jobId, clientId, processId, new Metrics(), new SystemTime()) { @Override protected StreamTask createStreamTask(TaskId id, Collection partitionsForTask) { - ProcessorTopology topology = builder.build(id.topicGroupId); + ProcessorTopology topology = builder.build("X", id.topicGroupId); return new TestStreamTask(id, jobId, partitionsForTask, topology, consumer, producer, mockRestoreConsumer, config); } }; @@ -298,7 +298,7 @@ public void maybeClean() { @Override protected StreamTask createStreamTask(TaskId id, Collection partitionsForTask) { - ProcessorTopology topology = builder.build(id.topicGroupId); + ProcessorTopology topology = builder.build("X", id.topicGroupId); return new TestStreamTask(id, jobId, partitionsForTask, topology, consumer, producer, mockRestoreConsumer, config); } }; @@ -420,7 +420,7 @@ public void maybeCommit() { @Override protected StreamTask createStreamTask(TaskId id, Collection partitionsForTask) { - ProcessorTopology topology = builder.build(id.topicGroupId); + ProcessorTopology topology = builder.build("X", id.topicGroupId); return new TestStreamTask(id, jobId, partitionsForTask, topology, consumer, producer, mockRestoreConsumer, config); } }; diff --git a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java index c0c5c3920969b..edbcb4a9faaf4 100644 --- a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java @@ -53,7 +53,7 @@ public KStreamTestDriver(KStreamBuilder builder, File stateDir, Serializer keySerializer, Deserializer keyDeserializer, Serializer valSerializer, Deserializer valDeserializer) { - this.topology = builder.build(null); + this.topology = builder.build("X", null); this.stateDir = stateDir; this.context = new MockProcessorContext(this, stateDir, keySerializer, keyDeserializer, valSerializer, valDeserializer, new MockRecordCollector()); @@ -127,7 +127,7 @@ private class MockRecordCollector extends RecordCollector { public MockRecordCollector() { super(null); } - + @Override public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer, StreamPartitioner partitioner) { diff --git a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java index 34fd10c57991f..cf17dbe99a74b 100644 --- a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java @@ -146,7 +146,7 @@ public class ProcessorTopologyTestDriver { */ public ProcessorTopologyTestDriver(StreamsConfig config, TopologyBuilder builder, String... storeNames) { id = new TaskId(0, 0); - topology = builder.build(null); + topology = builder.build("X", null); // Set up the consumer and producer ... consumer = new MockConsumer<>(OffsetResetStrategy.EARLIEST); diff --git a/tests/kafkatest/tests/streams_bounce_test.py b/tests/kafkatest/tests/streams_bounce_test.py index 552390999d478..d6746419462d8 100644 --- a/tests/kafkatest/tests/streams_bounce_test.py +++ b/tests/kafkatest/tests/streams_bounce_test.py @@ -41,7 +41,6 @@ def __init__(self, test_context): self.driver = StreamsSmokeTestDriverService(test_context, self.kafka) self.processor1 = StreamsSmokeTestJobRunnerService(test_context, self.kafka) - @ignore def test_bounce(self): """ Start a smoke test client, then abort (kill -9) and restart it a few times. diff --git a/tests/kafkatest/tests/streams_smoke_test.py b/tests/kafkatest/tests/streams_smoke_test.py index ea05c5f4cd050..e3c465af32f9d 100644 --- a/tests/kafkatest/tests/streams_smoke_test.py +++ b/tests/kafkatest/tests/streams_smoke_test.py @@ -44,7 +44,6 @@ def __init__(self, test_context): self.processor3 = StreamsSmokeTestJobRunnerService(test_context, self.kafka) self.processor4 = StreamsSmokeTestJobRunnerService(test_context, self.kafka) - @ignore def test_streams(self): """ Start a few smoke test clients, then repeat start a new one, stop (cleanly) running one a few times. From cf40acc2b17cf6d32cee712dad6c88a851ae0715 Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Mon, 14 Mar 2016 15:09:47 -0700 Subject: [PATCH 026/206] MINOR: Remove unused method, redundant in interface definition and add final for object used in sychronization guozhangwang Very minor cleanup. Author: Liquan Pei Reviewers: Guozhang Wang Closes #1063 from Ishiihara/minor-cleanup --- .../processor/internals/ProcessorStateManager.java | 12 ------------ .../processor/internals/PunctuationQueue.java | 2 +- .../streams/processor/internals/RecordCollector.java | 4 ++-- 3 files changed, 3 insertions(+), 15 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index bae30e35a4dc9..c8f289e07dbdc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -137,8 +137,6 @@ private static FileLock lockStateDirectory(FileChannel channel) throws IOExcepti } } - - public File baseDir() { return this.baseDir; } @@ -305,16 +303,6 @@ public StateStore getStore(String name) { return stores.get(name); } - public void cleanup() throws IOException { - // clean up any unknown files in the state directory - for (File file : this.baseDir.listFiles()) { - if (!this.stores.containsKey(file.getName())) { - log.info("Deleting state directory {}", file.getAbsolutePath()); - file.delete(); - } - } - } - public void flush() { if (!this.stores.isEmpty()) { log.debug("Flushing stores."); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java index b4b7afe4f03e3..d7d7eee90ef78 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java @@ -21,7 +21,7 @@ public class PunctuationQueue { - private PriorityQueue pq = new PriorityQueue<>(); + private final PriorityQueue pq = new PriorityQueue<>(); public void schedule(PunctuationSchedule sched) { synchronized (pq) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java index cd8af9b6371d2..eb731be59a18e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/RecordCollector.java @@ -37,12 +37,12 @@ public class RecordCollector { /** * A supplier of a {@link RecordCollector} instance. */ - public static interface Supplier { + public interface Supplier { /** * Get the record collector. * @return the record collector */ - public RecordCollector recordCollector(); + RecordCollector recordCollector(); } private static final Logger log = LoggerFactory.getLogger(RecordCollector.class); From ffbe624e6f4906c55080508fab19ac60dc93761e Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Mon, 14 Mar 2016 19:13:26 -0700 Subject: [PATCH 027/206] KAFKA-3373; add 'log' prefix to configurations in KIP-31/32 Author: Jiangjie Qin Reviewers: Gwen Shapira Closes #1049 from becketqin/KAFKA-3373 --- core/src/main/scala/kafka/log/LogConfig.scala | 12 +++---- .../scala/kafka/server/ConfigHandler.scala | 4 +-- .../main/scala/kafka/server/KafkaConfig.scala | 33 ++++++++++--------- .../main/scala/kafka/server/KafkaServer.scala | 6 ++-- .../unit/kafka/server/KafkaConfigTest.scala | 4 +-- .../unit/kafka/server/LogOffsetTest.scala | 24 +++++++------- docs/upgrade.html | 4 +-- .../services/kafka/config_property.py | 2 +- tests/kafkatest/tests/upgrade_test.py | 6 ++-- 9 files changed, 48 insertions(+), 47 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index a76dce784b1d7..ffec85a95edea 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -47,9 +47,9 @@ object Defaults { val MinInSyncReplicas = kafka.server.Defaults.MinInSyncReplicas val CompressionType = kafka.server.Defaults.CompressionType val PreAllocateEnable = kafka.server.Defaults.LogPreAllocateEnable - val MessageFormatVersion = kafka.server.Defaults.MessageFormatVersion - val MessageTimestampType = kafka.server.Defaults.MessageTimestampType - val MessageTimestampDifferenceMaxMs = kafka.server.Defaults.MessageTimestampDifferenceMaxMs + val MessageFormatVersion = kafka.server.Defaults.LogMessageFormatVersion + val MessageTimestampType = kafka.server.Defaults.LogMessageTimestampType + val MessageTimestampDifferenceMaxMs = kafka.server.Defaults.LogMessageTimestampDifferenceMaxMs } case class LogConfig(props: java.util.Map[_, _]) extends AbstractConfig(LogConfig.configDef, props, false) { @@ -110,9 +110,9 @@ object LogConfig { val MinInSyncReplicasProp = "min.insync.replicas" val CompressionTypeProp = "compression.type" val PreAllocateEnableProp = "preallocate" - val MessageFormatVersionProp = KafkaConfig.MessageFormatVersionProp - val MessageTimestampTypeProp = KafkaConfig.MessageTimestampTypeProp - val MessageTimestampDifferenceMaxMsProp = KafkaConfig.MessageTimestampDifferenceMaxMsProp + val MessageFormatVersionProp = "message.format.version" + val MessageTimestampTypeProp = "message.timestamp.type" + val MessageTimestampDifferenceMaxMsProp = "message.timestamp.difference.max.ms" val SegmentSizeDoc = "The hard maximum for the size of a segment file in the log" val SegmentMsDoc = "The soft maximum on the amount of time before a new log segment is rolled" diff --git a/core/src/main/scala/kafka/server/ConfigHandler.scala b/core/src/main/scala/kafka/server/ConfigHandler.scala index 4bdd308e2c03e..ab1d7825e1832 100644 --- a/core/src/main/scala/kafka/server/ConfigHandler.scala +++ b/core/src/main/scala/kafka/server/ConfigHandler.scala @@ -48,8 +48,8 @@ class TopicConfigHandler(private val logManager: LogManager, kafkaConfig: KafkaC warn(s"Log configuration ${LogConfig.MessageFormatVersionProp} is ignored for `$topic` because `$versionString` " + s"is not compatible with Kafka inter-broker protocol version `${kafkaConfig.interBrokerProtocolVersionString}`") Some(LogConfig.MessageFormatVersionProp) - } - else None + } else + None } val logs = logManager.logsByTopicPartition.filterKeys(_.topic == topic).values.toBuffer diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index d13c87229f84c..8d14edd17f844 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -23,6 +23,7 @@ import kafka.api.ApiVersion import kafka.cluster.EndPoint import kafka.consumer.ConsumerConfig import kafka.coordinator.OffsetConfig +import kafka.log.LogConfig import kafka.message.{BrokerCompressionCodec, CompressionCodec, Message, MessageSet} import kafka.utils.CoreUtils import org.apache.kafka.clients.CommonClientConfigs @@ -95,12 +96,12 @@ object Defaults { val LogFlushOffsetCheckpointIntervalMs = 60000 val LogPreAllocateEnable = false // lazy val as `InterBrokerProtocolVersion` is defined later - lazy val MessageFormatVersion = InterBrokerProtocolVersion + lazy val LogMessageFormatVersion = InterBrokerProtocolVersion + val LogMessageTimestampType = "CreateTime" + val LogMessageTimestampDifferenceMaxMs = Long.MaxValue val NumRecoveryThreadsPerDataDir = 1 val AutoCreateTopicsEnable = true val MinInSyncReplicas = 1 - val MessageTimestampType = "CreateTime" - val MessageTimestampDifferenceMaxMs = Long.MaxValue /** ********* Replication configuration ***********/ val ControllerSocketTimeoutMs = RequestTimeoutMs @@ -184,6 +185,8 @@ object Defaults { object KafkaConfig { + private val LogConfigPrefix = "log." + def main(args: Array[String]) { System.out.println(configDef.toHtmlTable) } @@ -255,12 +258,12 @@ object KafkaConfig { val LogFlushIntervalMsProp = "log.flush.interval.ms" val LogFlushOffsetCheckpointIntervalMsProp = "log.flush.offset.checkpoint.interval.ms" val LogPreAllocateProp = "log.preallocate" - val MessageFormatVersionProp = "message.format.version" + val LogMessageFormatVersionProp = LogConfigPrefix + LogConfig.MessageFormatVersionProp + val LogMessageTimestampTypeProp = LogConfigPrefix + LogConfig.MessageTimestampTypeProp + val LogMessageTimestampDifferenceMaxMsProp = LogConfigPrefix + LogConfig.MessageTimestampDifferenceMaxMsProp val NumRecoveryThreadsPerDataDirProp = "num.recovery.threads.per.data.dir" val AutoCreateTopicsEnableProp = "auto.create.topics.enable" val MinInSyncReplicasProp = "min.insync.replicas" - val MessageTimestampTypeProp = "message.timestamp.type" - val MessageTimestampDifferenceMaxMsProp = "message.timestamp.difference.max.ms" /** ********* Replication configuration ***********/ val ControllerSocketTimeoutMsProp = "controller.socket.timeout.ms" val DefaultReplicationFactorProp = "default.replication.factor" @@ -607,9 +610,9 @@ object KafkaConfig { .define(NumRecoveryThreadsPerDataDirProp, INT, Defaults.NumRecoveryThreadsPerDataDir, atLeast(1), HIGH, NumRecoveryThreadsPerDataDirDoc) .define(AutoCreateTopicsEnableProp, BOOLEAN, Defaults.AutoCreateTopicsEnable, HIGH, AutoCreateTopicsEnableDoc) .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), HIGH, MinInSyncReplicasDoc) - .define(MessageFormatVersionProp, STRING, Defaults.MessageFormatVersion, MEDIUM, MessageFormatVersionDoc) - .define(MessageTimestampTypeProp, STRING, Defaults.MessageTimestampType, in("CreateTime", "LogAppendTime"), MEDIUM, MessageTimestampTypeDoc) - .define(MessageTimestampDifferenceMaxMsProp, LONG, Defaults.MessageTimestampDifferenceMaxMs, atLeast(0), MEDIUM, MessageTimestampDifferenceMaxMsDoc) + .define(LogMessageFormatVersionProp, STRING, Defaults.LogMessageFormatVersion, MEDIUM, MessageFormatVersionDoc) + .define(LogMessageTimestampTypeProp, STRING, Defaults.LogMessageTimestampType, in("CreateTime", "LogAppendTime"), MEDIUM, MessageTimestampTypeDoc) + .define(LogMessageTimestampDifferenceMaxMsProp, LONG, Defaults.LogMessageTimestampDifferenceMaxMs, atLeast(0), MEDIUM, MessageTimestampDifferenceMaxMsDoc) /** ********* Replication configuration ***********/ .define(ControllerSocketTimeoutMsProp, INT, Defaults.ControllerSocketTimeoutMs, MEDIUM, ControllerSocketTimeoutMsDoc) @@ -802,10 +805,10 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean) extends Abstra val logPreAllocateEnable: java.lang.Boolean = getBoolean(KafkaConfig.LogPreAllocateProp) // We keep the user-provided String as `ApiVersion.apply` can choose a slightly different version (eg if `0.10.0` // is passed, `0.10.0-IV0` may be picked) - val messageFormatVersionString = getString(KafkaConfig.MessageFormatVersionProp) - val messageFormatVersion = ApiVersion(messageFormatVersionString) - val messageTimestampType = TimestampType.forName(getString(KafkaConfig.MessageTimestampTypeProp)) - val messageTimestampDifferenceMaxMs = getLong(KafkaConfig.MessageTimestampDifferenceMaxMsProp) + val logMessageFormatVersionString = getString(KafkaConfig.LogMessageFormatVersionProp) + val logMessageFormatVersion = ApiVersion(logMessageFormatVersionString) + val logMessageTimestampType = TimestampType.forName(getString(KafkaConfig.LogMessageTimestampTypeProp)) + val logMessageTimestampDifferenceMaxMs = getLong(KafkaConfig.LogMessageTimestampDifferenceMaxMsProp) /** ********* Replication configuration ***********/ val controllerSocketTimeoutMs: Int = getInt(KafkaConfig.ControllerSocketTimeoutMsProp) @@ -986,7 +989,7 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean) extends Abstra s"${KafkaConfig.AdvertisedListenersProp} protocols must be equal to or a subset of ${KafkaConfig.ListenersProp} protocols. " + s"Found ${advertisedListeners.keySet}. The valid options based on currently configured protocols are ${listeners.keySet}" ) - require(interBrokerProtocolVersion >= messageFormatVersion, - s"message.format.version $messageFormatVersionString cannot be used when inter.broker.protocol.version is set to $interBrokerProtocolVersionString") + require(interBrokerProtocolVersion >= logMessageFormatVersion, + s"log.message.format.version $logMessageFormatVersionString cannot be used when inter.broker.protocol.version is set to $interBrokerProtocolVersionString") } } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 2203df92071c1..2f5441ac12115 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -75,9 +75,9 @@ object KafkaServer { logProps.put(LogConfig.CompressionTypeProp, kafkaConfig.compressionType) logProps.put(LogConfig.UncleanLeaderElectionEnableProp, kafkaConfig.uncleanLeaderElectionEnable) logProps.put(LogConfig.PreAllocateEnableProp, kafkaConfig.logPreAllocateEnable) - logProps.put(LogConfig.MessageFormatVersionProp, kafkaConfig.messageFormatVersion.version) - logProps.put(LogConfig.MessageTimestampTypeProp, kafkaConfig.messageTimestampType.name) - logProps.put(LogConfig.MessageTimestampDifferenceMaxMsProp, kafkaConfig.messageTimestampDifferenceMaxMs) + logProps.put(LogConfig.MessageFormatVersionProp, kafkaConfig.logMessageFormatVersion.version) + logProps.put(LogConfig.MessageTimestampTypeProp, kafkaConfig.logMessageTimestampType.name) + logProps.put(LogConfig.MessageTimestampDifferenceMaxMsProp, kafkaConfig.logMessageTimestampDifferenceMaxMs) logProps } } diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index c5a0079013417..7524e6a6f8726 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -283,14 +283,14 @@ class KafkaConfigTest { props.put(KafkaConfig.InterBrokerProtocolVersionProp, "0.8.2.0") // We need to set the message format version to make the configuration valid. - props.put(KafkaConfig.MessageFormatVersionProp, "0.8.2.0") + props.put(KafkaConfig.LogMessageFormatVersionProp, "0.8.2.0") val conf2 = KafkaConfig.fromProps(props) assertEquals(KAFKA_0_8_2, conf2.interBrokerProtocolVersion) // check that 0.8.2.0 is the same as 0.8.2.1 props.put(KafkaConfig.InterBrokerProtocolVersionProp, "0.8.2.1") // We need to set the message format version to make the configuration valid - props.put(KafkaConfig.MessageFormatVersionProp, "0.8.2.1") + props.put(KafkaConfig.LogMessageFormatVersionProp, "0.8.2.1") val conf3 = KafkaConfig.fromProps(props) assertEquals(KAFKA_0_8_2, conf3.interBrokerProtocolVersion) diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index 5c2092c1af58b..8c86a7b5956af 100755 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -18,21 +18,20 @@ package kafka.server import java.io.File +import java.util.{Properties, Random} + +import kafka.admin.AdminUtils +import kafka.api.{FetchRequestBuilder, OffsetRequest, PartitionOffsetRequestInfo} +import kafka.common.TopicAndPartition +import kafka.consumer.SimpleConsumer +import kafka.message.{ByteBufferMessageSet, Message, NoCompressionCodec} +import kafka.utils.TestUtils._ import kafka.utils._ -import org.apache.kafka.common.protocol.Errors +import kafka.zk.ZooKeeperTestHarness import org.apache.kafka.common.TopicPartition +import org.apache.kafka.common.protocol.Errors import org.junit.Assert._ -import java.util.{Random, Properties} -import kafka.consumer.SimpleConsumer -import kafka.message.{NoCompressionCodec, ByteBufferMessageSet, Message} -import kafka.zk.ZooKeeperTestHarness -import kafka.admin.AdminUtils -import kafka.api.{ApiVersion, PartitionOffsetRequestInfo, FetchRequestBuilder, OffsetRequest} -import kafka.utils.TestUtils._ -import kafka.common.TopicAndPartition -import org.junit.After -import org.junit.Before -import org.junit.Test +import org.junit.{After, Before, Test} class LogOffsetTest extends ZooKeeperTestHarness { val random = new Random() @@ -206,7 +205,6 @@ class LogOffsetTest extends ZooKeeperTestHarness { props.put("log.retention.check.interval.ms", (5*1000*60).toString) props.put("log.segment.bytes", logSize.toString) props.put("zookeeper.connect", zkConnect.toString) - props.put("message.format.version", "0.10.0") props } diff --git a/docs/upgrade.html b/docs/upgrade.html index 863a6face5d9d..15ea3ae045cf1 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -26,7 +26,7 @@

    Upgrading from 0.8.x or 0.9.x to 0.10.
    1. Update server.properties file on all brokers and add the following property: inter.broker.protocol.version=CURRENT_KAFKA_VERSION (e.g. 0.8.2 or 0.9.0.0). - We recommend that users set message.format.version=CURRENT_KAFKA_VERSION as well to avoid a performance regression + We recommend that users set log.message.format.version=CURRENT_KAFKA_VERSION as well to avoid a performance regression during upgrade. See potential performance impact during upgrade for the details.
    2. Upgrade the brokers. This can be done a broker at a time by simply bringing it down, updating the code, and restarting it.
    3. @@ -41,7 +41,7 @@

      Upgrading from 0.8.x or 0.9.x to 0.10.
      Potential performance impact during upgrade to 0.10.0.0

      The message format in 0.10.0 includes a new timestamp field and uses relative offsets for compressed messages. - The on disk message format can be configured through message.format.version in the server.properties file. + The on disk message format can be configured through log.message.format.version in the server.properties file. The default on-disk message format is 0.10.0. If a consumer client is on a version before 0.10.0.0, it only understands message formats before 0.10.0. In this case, the broker is able to convert messages from the 0.10.0 format to an earlier format before sending the response to the consumer on an older version. However, the broker can't use zero-copy transfer in this case. diff --git a/tests/kafkatest/services/kafka/config_property.py b/tests/kafkatest/services/kafka/config_property.py index b2b1d05ba5316..8f30f1331ed23 100644 --- a/tests/kafkatest/services/kafka/config_property.py +++ b/tests/kafkatest/services/kafka/config_property.py @@ -40,7 +40,7 @@ ZOOKEEPER_CONNECT = "zookeeper.connect" ZOOKEEPER_CONNECTION_TIMEOUT_MS = "zookeeper.connection.timeout.ms" INTER_BROKER_PROTOCOL_VERSION = "inter.broker.protocol.version" -MESSAGE_FORMAT_VERSION = "message.format.version" +MESSAGE_FORMAT_VERSION = "log.message.format.version" diff --git a/tests/kafkatest/tests/upgrade_test.py b/tests/kafkatest/tests/upgrade_test.py index bec4b3f025bd2..9926f11ee0e18 100644 --- a/tests/kafkatest/tests/upgrade_test.py +++ b/tests/kafkatest/tests/upgrade_test.py @@ -78,10 +78,10 @@ def test_upgrade(self, from_kafka_version, to_message_format_version, compressio - Start producer and consumer in the background - Perform two-phase rolling upgrade - First phase: upgrade brokers to 0.10 with inter.broker.protocol.version set to - from_kafka_version and message.format.version set to from_kafka_version + from_kafka_version and log.message.format.version set to from_kafka_version - Second phase: remove inter.broker.protocol.version config with rolling bounce; if - to_message_format_version is set to 0.9, set message.format.version to - to_message_format_version, otherwise remove message.format.version config + to_message_format_version is set to 0.9, set log.message.format.version to + to_message_format_version, otherwise remove log.message.format.version config - Finally, validate that every message acked by the producer was consumed by the consumer """ self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zk, From 241c3ebb2803f1e09306fb06f20a66e7a60ca3c8 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Mon, 14 Mar 2016 19:14:36 -0700 Subject: [PATCH 028/206] KAFKA-3375; Suppress deprecated warnings where reasonable and tweak compiler settings * Fix and suppress number of unchecked warnings (except for Kafka Streams) * Add `SafeVarargs` annotation to fix warnings * Suppress unfixable deprecation warnings * Replace deprecated by non-deprecated usage where possible * Avoid reflective calls via structural types in Scala * Tweak compiler settings for scalac and javac Once we drop Java 7 and Scala 2.10, we can tweak the compiler settings further so that they warn us about more things. Author: Ismael Juma Reviewers: Grant Henke, Gwen Shapira, Guozhang Wang Closes #1042 from ijuma/kafka-3375-suppress-depreccated-tweak-compiler --- build.gradle | 18 ++++++++++++++++++ .../clients/consumer/ConsumerRecords.java | 2 ++ .../internals/ConsumerCoordinator.java | 4 ++-- .../consumer/internals/RequestFuture.java | 6 +++--- .../kafka/clients/producer/KafkaProducer.java | 2 +- .../common/network/SaslChannelBuilder.java | 1 + .../common/security/kerberos/LoginManager.java | 2 +- .../kafka/common/security/ssl/SslFactory.java | 3 ++- .../org/apache/kafka/common/utils/Utils.java | 2 ++ .../common/requests/RequestResponseTest.java | 1 + .../org/apache/kafka/connect/data/Struct.java | 2 ++ .../kafka/connect/json/JsonConverter.java | 2 +- .../apache/kafka/connect/runtime/Worker.java | 1 + .../connect/storage/KafkaConfigStorage.java | 1 + .../storage/KafkaStatusBackingStore.java | 2 ++ .../storage/OffsetStorageReaderImpl.java | 1 + .../kafka/connect/storage/OffsetUtils.java | 1 + .../runtime/WorkerSinkTaskThreadedTest.java | 1 + .../connect/runtime/WorkerSourceTaskTest.java | 1 + .../rest/resources/ConnectorsResourceTest.java | 1 + .../standalone/StandaloneHerderTest.java | 1 + .../storage/KafkaConfigStorageTest.java | 1 + .../storage/KafkaOffsetBackingStoreTest.java | 1 + .../storage/KafkaStatusBackingStoreTest.java | 1 + .../util/ByteArrayProducerRecordEquals.java | 1 + .../integration/kafka/api/QuotasTest.scala | 1 - .../scala/kafka/tools/TestLogCleaning.scala | 4 ++-- .../ZkNodeChangeNotificationListenerTest.scala | 8 ++++---- 28 files changed, 56 insertions(+), 16 deletions(-) diff --git a/build.gradle b/build.gradle index c2bd2288504ab..321fc3f9b8d30 100644 --- a/build.gradle +++ b/build.gradle @@ -103,6 +103,12 @@ subprojects { sourceCompatibility = 1.7 + compileJava { + options.encoding = 'UTF-8' + // Add unchecked once we drop support for Java 7 as @SuppressWarnings("unchecked") is too buggy in Java 7 + options.compilerArgs << "-Xlint:deprecation" + } + if (JavaVersion.current().isJava8Compatible()) { tasks.withType(Javadoc) { // disable the crazy super-strict doclint tool in Java 8 @@ -220,6 +226,18 @@ subprojects { tasks.withType(ScalaCompile) { scalaCompileOptions.useAnt = false + scalaCompileOptions.additionalParameters = [ + "-deprecation", + "-unchecked", + "-encoding", "utf8", + "-target:jvm-${sourceCompatibility}".toString(), + "-Xlog-reflective-calls", + "-feature", + "-language:postfixOps", + "-language:implicitConversions", + "-language:existentials" + ] + configure(scalaCompileOptions.forkOptions) { memoryMaximumSize = '1g' jvmArgs = ['-XX:MaxPermSize=512m', '-Xss2m'] diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java index 8ee9be28dc742..3d7ec60438dce 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java @@ -28,6 +28,8 @@ * partition returned by a {@link Consumer#poll(long)} operation. */ public class ConsumerRecords implements Iterable> { + + @SuppressWarnings("unchecked") public static final ConsumerRecords EMPTY = new ConsumerRecords<>(Collections.EMPTY_MAP); private final Map>> records; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index b6b46c135a579..2ae1437336af3 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -68,7 +68,7 @@ public final class ConsumerCoordinator extends AbstractCoordinator { private final OffsetCommitCallback defaultOffsetCommitCallback; private final boolean autoCommitEnabled; private final AutoCommitTask autoCommitTask; - private final ConsumerInterceptors interceptors; + private final ConsumerInterceptors interceptors; /** * Initialize the coordination manager. @@ -87,7 +87,7 @@ public ConsumerCoordinator(ConsumerNetworkClient client, OffsetCommitCallback defaultOffsetCommitCallback, boolean autoCommitEnabled, long autoCommitIntervalMs, - ConsumerInterceptors interceptors) { + ConsumerInterceptors interceptors) { super(client, groupId, sessionTimeoutMs, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java index 7be99bd2d7097..71c16faf95275 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/RequestFuture.java @@ -42,7 +42,7 @@ public class RequestFuture { private boolean isDone = false; private T value; private RuntimeException exception; - private List> listeners = new ArrayList>(); + private List> listeners = new ArrayList<>(); /** @@ -129,12 +129,12 @@ public void raise(Errors error) { } private void fireSuccess() { - for (RequestFutureListener listener: listeners) + for (RequestFutureListener listener : listeners) listener.onSuccess(value); } private void fireFailure() { - for (RequestFutureListener listener: listeners) + for (RequestFutureListener listener : listeners) listener.onFailure(exception); } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 85ba9efd3c33e..c87973ad1a2db 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -202,7 +202,7 @@ public KafkaProducer(Properties properties, Serializer keySerializer, Seriali keySerializer, valueSerializer); } - @SuppressWarnings("unchecked") + @SuppressWarnings({"unchecked", "deprecation"}) private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serializer valueSerializer) { try { log.trace("Starting the Kafka producer"); diff --git a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java index b3db4e1bd9a32..0cd5bfe9005de 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java @@ -61,6 +61,7 @@ public void configure(Map configs) throws KafkaException { defaultRealm = ""; } + @SuppressWarnings("unchecked") List principalToLocalRules = (List) configs.get(SaslConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES); if (principalToLocalRules != null) kerberosShortNamer = KerberosShortNamer.fromUnparsedRules(defaultRealm, principalToLocalRules); diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/LoginManager.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/LoginManager.java index cf68d2058c0be..e163ba8cf70c6 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/kerberos/LoginManager.java +++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/LoginManager.java @@ -31,7 +31,7 @@ public class LoginManager { - private static final EnumMap CACHED_INSTANCES = new EnumMap(LoginType.class); + private static final EnumMap CACHED_INSTANCES = new EnumMap<>(LoginType.class); private final Login login; private final String serviceName; diff --git a/clients/src/main/java/org/apache/kafka/common/security/ssl/SslFactory.java b/clients/src/main/java/org/apache/kafka/common/security/ssl/SslFactory.java index 0d4d2ce362d19..d0fe2e8694fce 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/ssl/SslFactory.java +++ b/clients/src/main/java/org/apache/kafka/common/security/ssl/SslFactory.java @@ -69,11 +69,12 @@ public void configure(Map configs) throws KafkaException { this.protocol = (String) configs.get(SslConfigs.SSL_PROTOCOL_CONFIG); this.provider = (String) configs.get(SslConfigs.SSL_PROVIDER_CONFIG); - + @SuppressWarnings("unchecked") List cipherSuitesList = (List) configs.get(SslConfigs.SSL_CIPHER_SUITES_CONFIG); if (cipherSuitesList != null) this.cipherSuites = cipherSuitesList.toArray(new String[cipherSuitesList.size()]); + @SuppressWarnings("unchecked") List enabledProtocolsList = (List) configs.get(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG); if (enabledProtocolsList != null) this.enabledProtocols = enabledProtocolsList.toArray(new String[enabledProtocolsList.size()]); diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index daef458d8bfd5..4c4225bdcf89a 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -578,6 +578,7 @@ public static ByteBuffer ensureCapacity(ByteBuffer existingBuffer, int newLength * @param the type of element * @return Set */ + @SafeVarargs public static Set mkSet(T... elems) { return new HashSet<>(Arrays.asList(elems)); } @@ -588,6 +589,7 @@ public static Set mkSet(T... elems) { * @param the type of element * @return List */ + @SafeVarargs public static List mkList(T... elems) { return Arrays.asList(elems); } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 30238378f7b3f..7ccf07980ae24 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -370,6 +370,7 @@ private AbstractRequestResponse createLeaderAndIsrResponse() { return new LeaderAndIsrResponse(Errors.NONE.code(), responses); } + @SuppressWarnings("deprecation") private AbstractRequest createUpdateMetadataRequest(int version) { Map partitionStates = new HashMap<>(); List isr = Arrays.asList(1, 2); diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/Struct.java b/connect/api/src/main/java/org/apache/kafka/connect/data/Struct.java index 4ca37c3366cd3..a598259128c94 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/Struct.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/Struct.java @@ -172,6 +172,7 @@ public byte[] getBytes(String fieldName) { /** * Equivalent to calling {@link #get(String)} and casting the result to a List. */ + @SuppressWarnings("unchecked") public List getArray(String fieldName) { return (List) getCheckType(fieldName, Schema.Type.ARRAY); } @@ -179,6 +180,7 @@ public List getArray(String fieldName) { /** * Equivalent to calling {@link #get(String)} and casting the result to a Map. */ + @SuppressWarnings("unchecked") public Map getMap(String fieldName) { return (Map) getCheckType(fieldName, Schema.Type.MAP); } diff --git a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java index a70caddfa6b40..d9a685953d2d5 100644 --- a/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java +++ b/connect/json/src/main/java/org/apache/kafka/connect/json/JsonConverter.java @@ -425,7 +425,7 @@ private ObjectNode asJsonSchema(Schema schema) { ObjectNode jsonSchemaParams = JsonNodeFactory.instance.objectNode(); for (Map.Entry prop : schema.parameters().entrySet()) jsonSchemaParams.put(prop.getKey(), prop.getValue()); - jsonSchema.put(JsonSchema.SCHEMA_PARAMETERS_FIELD_NAME, jsonSchemaParams); + jsonSchema.set(JsonSchema.SCHEMA_PARAMETERS_FIELD_NAME, jsonSchemaParams); } if (schema.defaultValue() != null) jsonSchema.set(JsonSchema.SCHEMA_DEFAULT_FIELD_NAME, convertToJson(schema, schema.defaultValue())); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java index 4c0d016e1eab9..aa574935d31af 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java @@ -189,6 +189,7 @@ public boolean isSinkConnector(String connName) { return SinkConnector.class.isAssignableFrom(workerConnector.delegate.getClass()); } + @SuppressWarnings("unchecked") private Class getConnectorClass(String connectorAlias) { // Avoid the classpath scan if the full class name was provided try { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigStorage.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigStorage.java index 7f2fb830ac5b3..08c528c30c53f 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigStorage.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigStorage.java @@ -375,6 +375,7 @@ private KafkaBasedLog createKafkaBasedLog(String topic, Map(topic, producerProps, consumerProps, consumedCallback, new SystemTime()); } + @SuppressWarnings("unchecked") private final Callback> consumedCallback = new Callback>() { @Override public void onCompletion(Throwable error, ConsumerRecord record) { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaStatusBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaStatusBackingStore.java index eb9a48c4b1a18..d24645e4b6e8e 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaStatusBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaStatusBackingStore.java @@ -300,6 +300,7 @@ private ConnectorStatus parseConnectorStatus(String connector, byte[] data) { return null; } + @SuppressWarnings("unchecked") Map statusMap = (Map) schemaAndValue.value(); TaskStatus.State state = TaskStatus.State.valueOf((String) statusMap.get(STATE_KEY_NAME)); String trace = (String) statusMap.get(TRACE_KEY_NAME); @@ -319,6 +320,7 @@ private TaskStatus parseTaskStatus(ConnectorTaskId taskId, byte[] data) { log.error("Invalid connector status type {}", schemaAndValue.value().getClass()); return null; } + @SuppressWarnings("unchecked") Map statusMap = (Map) schemaAndValue.value(); TaskStatus.State state = TaskStatus.State.valueOf((String) statusMap.get(STATE_KEY_NAME)); String trace = (String) statusMap.get(TRACE_KEY_NAME); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReaderImpl.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReaderImpl.java index 23c1019aba553..b404de2205956 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReaderImpl.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetStorageReaderImpl.java @@ -55,6 +55,7 @@ public Map offset(Map partition) { } @Override + @SuppressWarnings("unchecked") public Map, Map> offsets(Collection> partitions) { // Serialize keys so backing store can work with them Map> serializedToOriginal = new HashMap<>(partitions.size()); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetUtils.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetUtils.java index f31715a88d63a..b457b128653b6 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetUtils.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/OffsetUtils.java @@ -24,6 +24,7 @@ import java.util.Map; public class OffsetUtils { + @SuppressWarnings("unchecked") public static void validateFormat(Object offsetData) { if (offsetData == null) return; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java index ac10d595248d6..1099d7a2740b9 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java @@ -535,6 +535,7 @@ public ConsumerRecords answer() throws Throwable { return capturedRecords; } + @SuppressWarnings("unchecked") private IExpectationSetters expectOnePoll() { // Currently the SinkTask's put() method will not be invoked unless we provide some data, so instead of // returning empty data, we return one record. The expectation is that the data will be ignored by the diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java index 404be0ba68ac1..9b0133a6fb14b 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java @@ -365,6 +365,7 @@ public List answer() throws Throwable { return latch; } + @SuppressWarnings("unchecked") private void expectSendRecordSyncFailure(Throwable error) throws InterruptedException { expectConvertKeyValue(false); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java index 1feab0dbbf63e..4659ae8667611 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java @@ -56,6 +56,7 @@ @RunWith(PowerMockRunner.class) @PrepareForTest(RestServer.class) @PowerMockIgnore("javax.management.*") +@SuppressWarnings("unchecked") public class ConnectorsResourceTest { // Note trailing / and that we do *not* use LEADER_URL to construct our reference values. This checks that we handle // URL construction properly, avoiding //, which will mess up routing in the REST server diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java index 07d0e3dcb6697..3959ff8f767d2 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java @@ -63,6 +63,7 @@ import static org.junit.Assert.fail; @RunWith(PowerMockRunner.class) +@SuppressWarnings("unchecked") public class StandaloneHerderTest { private static final String CONNECTOR_NAME = "test"; private static final List TOPICS_LIST = Arrays.asList("topic1", "topic2"); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigStorageTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigStorageTest.java index f95704c223e7d..5e79a8d3d1b95 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigStorageTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigStorageTest.java @@ -62,6 +62,7 @@ @RunWith(PowerMockRunner.class) @PrepareForTest(KafkaConfigStorage.class) @PowerMockIgnore("javax.management.*") +@SuppressWarnings("unchecked") public class KafkaConfigStorageTest { private static final String TOPIC = "connect-configs"; private static final Map DEFAULT_CONFIG_STORAGE_PROPS = new HashMap<>(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java index aa929421283d5..38e0f7b1b9bc7 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaOffsetBackingStoreTest.java @@ -59,6 +59,7 @@ @RunWith(PowerMockRunner.class) @PrepareForTest(KafkaOffsetBackingStore.class) @PowerMockIgnore("javax.management.*") +@SuppressWarnings("unchecked") public class KafkaOffsetBackingStoreTest { private static final String TOPIC = "connect-offsets"; private static final Map DEFAULT_PROPS = new HashMap<>(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaStatusBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaStatusBackingStoreTest.java index 8acd31f7d63f5..45ccdd50e9f49 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaStatusBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaStatusBackingStoreTest.java @@ -45,6 +45,7 @@ import static org.easymock.EasyMock.newCapture; import static org.junit.Assert.assertEquals; +@SuppressWarnings("unchecked") public class KafkaStatusBackingStoreTest extends EasyMockSupport { private static final String STATUS_TOPIC = "status-topic"; diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/util/ByteArrayProducerRecordEquals.java b/connect/runtime/src/test/java/org/apache/kafka/connect/util/ByteArrayProducerRecordEquals.java index 4d17ac40da3c0..bcfcc23db6716 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/util/ByteArrayProducerRecordEquals.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/util/ByteArrayProducerRecordEquals.java @@ -36,6 +36,7 @@ public ByteArrayProducerRecordEquals(ProducerRecord record) { } @Override + @SuppressWarnings("unchecked") public boolean matches(Object argument) { if (!(argument instanceof ProducerRecord)) return false; diff --git a/core/src/test/scala/integration/kafka/api/QuotasTest.scala b/core/src/test/scala/integration/kafka/api/QuotasTest.scala index 23be1208af10b..b6a0ae5a6d6fa 100644 --- a/core/src/test/scala/integration/kafka/api/QuotasTest.scala +++ b/core/src/test/scala/integration/kafka/api/QuotasTest.scala @@ -71,7 +71,6 @@ class QuotasTest extends KafkaServerTestHarness { val producerProps = new Properties() producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) producerProps.put(ProducerConfig.ACKS_CONFIG, "0") - producerProps.put(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "false") producerProps.put(ProducerConfig.BUFFER_MEMORY_CONFIG, producerBufferSize.toString) producerProps.put(ProducerConfig.CLIENT_ID_CONFIG, producerId1) producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, diff --git a/core/src/test/scala/kafka/tools/TestLogCleaning.scala b/core/src/test/scala/kafka/tools/TestLogCleaning.scala index dcbfbe1c1fc14..2e288ecd3c033 100755 --- a/core/src/test/scala/kafka/tools/TestLogCleaning.scala +++ b/core/src/test/scala/kafka/tools/TestLogCleaning.scala @@ -247,7 +247,7 @@ object TestLogCleaning { dups: Int, percentDeletes: Int): File = { val producerProps = new Properties - producerProps.setProperty(ProducerConfig.BLOCK_ON_BUFFER_FULL_CONFIG, "true") + producerProps.setProperty(ProducerConfig.MAX_BLOCK_MS_CONFIG, Long.MaxValue.toString) producerProps.setProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerUrl) producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer") @@ -316,4 +316,4 @@ case class TestRecord(val topic: String, val key: Int, val value: Long, val dele def this(line: String) = this(line.split("\t")) override def toString() = topic + "\t" + key + "\t" + value + "\t" + (if(delete) "d" else "u") def topicAndKey = topic + key -} \ No newline at end of file +} diff --git a/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala b/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala index 50496f0adf655..8d48609bafec0 100644 --- a/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala +++ b/core/src/test/scala/unit/kafka/common/ZkNodeChangeNotificationListenerTest.scala @@ -27,9 +27,9 @@ class ZkNodeChangeNotificationListenerTest extends KafkaServerTestHarness { @Test def testProcessNotification() { + @volatile var notification: String = null + @volatile var invocationCount = 0 val notificationHandler = new NotificationHandler { - @volatile var notification: String = _ - @volatile var invocationCount: Integer = 0 override def processNotification(notificationMessage: String): Unit = { notification = notificationMessage invocationCount += 1 @@ -48,7 +48,7 @@ class ZkNodeChangeNotificationListenerTest extends KafkaServerTestHarness { zkUtils.createSequentialPersistentPath(seqNodePath, notificationMessage1) - TestUtils.waitUntilTrue(() => notificationHandler.invocationCount == 1 && notificationHandler.notification == notificationMessage1, "failed to send/process notification message in the timeout period.") + TestUtils.waitUntilTrue(() => invocationCount == 1 && notification == notificationMessage1, "failed to send/process notification message in the timeout period.") /*There is no easy way to test that purging. Even if we mock kafka time with MockTime, the purging compares kafka time with the time stored in zookeeper stat and the embeded zookeeper server does not provide a way to mock time. so to test purging we will have to use SystemTime.sleep(changeExpirationMs + 1) issue a write and check @@ -56,6 +56,6 @@ class ZkNodeChangeNotificationListenerTest extends KafkaServerTestHarness { depending on how threads get scheduled.*/ zkUtils.createSequentialPersistentPath(seqNodePath, notificationMessage2) - TestUtils.waitUntilTrue(() => notificationHandler.invocationCount == 2 && notificationHandler.notification == notificationMessage2, "failed to send/process notification message in the timeout period.") + TestUtils.waitUntilTrue(() => invocationCount == 2 && notification == notificationMessage2, "failed to send/process notification message in the timeout period.") } } From deb2b004cb983baa9e2fbff47b39a6f9a243623d Mon Sep 17 00:00:00 2001 From: MayureshGharat Date: Mon, 14 Mar 2016 19:22:05 -0700 Subject: [PATCH 029/206] KAFKA-3013: Display the topic-partition in the exception message for expired batches in recordAccumulator Added topic-partition information to the exception message on batch expiry in RecordAccumulator Author: MayureshGharat Reviewers: Gwen Shapira, Lin Dong, Ismael Juma Closes #695 from MayureshGharat/kafka-3013 --- .../apache/kafka/clients/producer/internals/RecordBatch.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java index 7b5fbbe0736dd..eb7bbb3cefbaa 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java @@ -143,7 +143,7 @@ public boolean maybeExpire(int requestTimeout, long now, long lingerMs) { if ((this.records.isFull() && requestTimeout < (now - this.lastAppendTime)) || requestTimeout < (now - (this.lastAttemptMs + lingerMs))) { expire = true; this.records.close(); - this.done(-1L, Record.NO_TIMESTAMP, new TimeoutException("Batch Expired")); + this.done(-1L, Record.NO_TIMESTAMP, new TimeoutException("Batch containing " + recordCount + " record(s) expired due to timeout while requesting metadata from brokers for " + topicPartition)); } return expire; From 951e30adc6d4a0ed37dcc3fde0050ca5faff146d Mon Sep 17 00:00:00 2001 From: Allen Wang Date: Tue, 15 Mar 2016 10:03:03 -0700 Subject: [PATCH 030/206] KAFKA-1215; Rack-Aware replica assignment option Please see https://cwiki.apache.org/confluence/display/KAFKA/KIP-36+Rack+aware+replica+assignment for the overall design. The update to TopicMetadataRequest/TopicMetadataResponse will be done in a different PR. Author: Allen Wang Author: Ismael Juma Reviewers: Ismael Juma , Jason Gustafson , Grant Henke , Jun Rao Closes #132 from allenxwang/KAFKA-1215 --- .../kafka/common/protocol/Protocol.java | 22 +- .../kafka/common/protocol/types/Struct.java | 13 +- .../requests/UpdateMetadataRequest.java | 47 ++-- .../common/requests/RequestResponseTest.java | 43 ++-- .../main/scala/kafka/admin/AdminUtils.scala | 235 +++++++++++++++--- .../scala/kafka/admin/BrokerMetadata.scala | 23 ++ .../scala/kafka/admin/RackAwareMode.scala | 42 ++++ .../admin/ReassignPartitionsCommand.scala | 37 ++- .../main/scala/kafka/admin/TopicCommand.scala | 5 +- .../src/main/scala/kafka/cluster/Broker.scala | 112 ++++----- .../controller/ControllerChannelManager.scala | 39 +-- .../main/scala/kafka/server/KafkaApis.scala | 4 +- .../main/scala/kafka/server/KafkaConfig.scala | 10 + .../scala/kafka/server/KafkaHealthcheck.scala | 13 +- .../main/scala/kafka/server/KafkaServer.scala | 2 +- .../scala/kafka/server/MetadataCache.scala | 2 +- core/src/main/scala/kafka/utils/ZkUtils.scala | 74 +++--- .../api/RackAwareAutoTopicCreationTest.scala | 65 +++++ .../unit/kafka/admin/AdminRackAwareTest.scala | 196 +++++++++++++++ .../scala/unit/kafka/admin/AdminTest.scala | 47 +++- .../unit/kafka/admin/RackAwareTest.scala | 82 ++++++ .../admin/ReassignPartitionsCommandTest.scala | 51 ++++ .../unit/kafka/admin/TopicCommandTest.scala | 32 ++- .../kafka/cluster/BrokerEndPointTest.scala | 14 -- .../unit/kafka/server/KafkaConfigTest.scala | 2 +- .../scala/unit/kafka/utils/TestUtils.scala | 33 ++- docs/upgrade.html | 5 + 27 files changed, 1000 insertions(+), 250 deletions(-) create mode 100644 core/src/main/scala/kafka/admin/BrokerMetadata.scala create mode 100644 core/src/main/scala/kafka/admin/RackAwareMode.scala create mode 100644 core/src/test/scala/integration/kafka/api/RackAwareAutoTopicCreationTest.scala create mode 100644 core/src/test/scala/unit/kafka/admin/AdminRackAwareTest.scala create mode 100644 core/src/test/scala/unit/kafka/admin/RackAwareTest.scala create mode 100644 core/src/test/scala/unit/kafka/admin/ReassignPartitionsCommandTest.scala diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index a77bf8cbb8623..e32d0b6cd4353 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -697,8 +697,26 @@ public class Protocol { public static final Schema UPDATE_METADATA_RESPONSE_V1 = UPDATE_METADATA_RESPONSE_V0; - public static final Schema[] UPDATE_METADATA_REQUEST = new Schema[] {UPDATE_METADATA_REQUEST_V0, UPDATE_METADATA_REQUEST_V1}; - public static final Schema[] UPDATE_METADATA_RESPONSE = new Schema[] {UPDATE_METADATA_RESPONSE_V0, UPDATE_METADATA_RESPONSE_V1}; + public static final Schema UPDATE_METADATA_REQUEST_PARTITION_STATE_V2 = UPDATE_METADATA_REQUEST_PARTITION_STATE_V1; + + public static final Schema UPDATE_METADATA_REQUEST_END_POINT_V2 = UPDATE_METADATA_REQUEST_END_POINT_V1; + + public static final Schema UPDATE_METADATA_REQUEST_BROKER_V2 = + new Schema(new Field("id", INT32, "The broker id."), + new Field("end_points", new ArrayOf(UPDATE_METADATA_REQUEST_END_POINT_V2)), + new Field("rack", NULLABLE_STRING, "The rack")); + + public static final Schema UPDATE_METADATA_REQUEST_V2 = + new Schema(new Field("controller_id", INT32, "The controller id."), + new Field("controller_epoch", INT32, "The controller epoch."), + new Field("partition_states", new ArrayOf(UPDATE_METADATA_REQUEST_PARTITION_STATE_V2)), + new Field("live_brokers", new ArrayOf(UPDATE_METADATA_REQUEST_BROKER_V2))); + + public static final Schema UPDATE_METADATA_RESPONSE_V2 = UPDATE_METADATA_RESPONSE_V1; + + + public static final Schema[] UPDATE_METADATA_REQUEST = new Schema[] {UPDATE_METADATA_REQUEST_V0, UPDATE_METADATA_REQUEST_V1, UPDATE_METADATA_REQUEST_V2}; + public static final Schema[] UPDATE_METADATA_RESPONSE = new Schema[] {UPDATE_METADATA_RESPONSE_V0, UPDATE_METADATA_RESPONSE_V1, UPDATE_METADATA_RESPONSE_V2}; /* an array of all requests and responses with all schema versions; a null value in the inner array means that the * particular version is not supported */ diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java index 4902f25d1757f..79f0638c11e21 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java @@ -311,7 +311,10 @@ public int hashCode() { for (Object arrayItem: arrayObject) result = prime * result + arrayItem.hashCode(); } else { - result = prime * result + this.get(f).hashCode(); + Object field = this.get(f); + if (field != null) { + result = prime * result + field.hashCode(); + } } } return result; @@ -330,11 +333,13 @@ public boolean equals(Object obj) { return false; for (int i = 0; i < this.values.length; i++) { Field f = this.schema.get(i); - Boolean result; + boolean result; if (f.type() instanceof ArrayOf) { - result = Arrays.equals((Object []) this.get(f), (Object []) other.get(f)); + result = Arrays.equals((Object[]) this.get(f), (Object[]) other.get(f)); } else { - result = this.get(f).equals(other.get(f)); + Object thisField = this.get(f); + Object otherField = other.get(f); + result = (thisField == null && otherField == null) || thisField.equals(otherField); } if (!result) return false; diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java index d8d8013871124..4c3d0a74740d9 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java @@ -49,16 +49,22 @@ public PartitionState(int controllerEpoch, int leader, int leaderEpoch, List endPoints; + public final String rack; - public Broker(int id, Map endPoints) { + public Broker(int id, Map endPoints, String rack) { this.id = id; this.endPoints = endPoints; + this.rack = rack; + } + + @Deprecated + public Broker(int id, Map endPoints) { + this(id, endPoints, null); } } @@ -91,6 +97,7 @@ public EndPoint(String host, int port) { // Broker key names private static final String BROKER_ID_KEY_NAME = "id"; private static final String ENDPOINTS_KEY_NAME = "end_points"; + private static final String RACK_KEY_NAME = "rack"; // EndPoint key names private static final String HOST_KEY_NAME = "host"; @@ -117,20 +124,20 @@ private static Set brokerEndPointsToBrokers(Set brokerEn for (BrokerEndPoint brokerEndPoint : brokerEndPoints) { Map endPoints = Collections.singletonMap(SecurityProtocol.PLAINTEXT, new EndPoint(brokerEndPoint.host(), brokerEndPoint.port())); - brokers.add(new Broker(brokerEndPoint.id(), endPoints)); + brokers.add(new Broker(brokerEndPoint.id(), endPoints, null)); } return brokers; } /** - * Constructor for version 1. + * Constructor for version 2. */ public UpdateMetadataRequest(int controllerId, int controllerEpoch, Map partitionStates, Set liveBrokers) { - this(1, controllerId, controllerEpoch, partitionStates, liveBrokers); + this(2, controllerId, controllerEpoch, partitionStates, liveBrokers); } - private UpdateMetadataRequest(int version, int controllerId, int controllerEpoch, Map partitionStates, Set liveBrokers) { super(new Struct(ProtoUtils.requestSchema(ApiKeys.UPDATE_METADATA_KEY.id, version))); struct.set(CONTROLLER_ID_KEY_NAME, controllerId); @@ -173,6 +180,9 @@ private UpdateMetadataRequest(int version, int controllerId, int controllerEpoch } brokerData.set(ENDPOINTS_KEY_NAME, endPointsData.toArray()); + if (version >= 2) { + brokerData.set(RACK_KEY_NAME, broker.rack); + } } brokersData.add(brokerData); @@ -226,8 +236,8 @@ public UpdateMetadataRequest(Struct struct) { int port = brokerData.getInt(PORT_KEY_NAME); Map endPoints = new HashMap<>(1); endPoints.put(SecurityProtocol.PLAINTEXT, new EndPoint(host, port)); - liveBrokers.add(new Broker(brokerId, endPoints)); - } else { // V1 + liveBrokers.add(new Broker(brokerId, endPoints, null)); + } else { // V1 or V2 Map endPoints = new HashMap<>(); for (Object endPointDataObj : brokerData.getArray(ENDPOINTS_KEY_NAME)) { Struct endPointData = (Struct) endPointDataObj; @@ -236,11 +246,13 @@ public UpdateMetadataRequest(Struct struct) { short protocolTypeId = endPointData.getShort(SECURITY_PROTOCOL_TYPE_KEY_NAME); endPoints.put(SecurityProtocol.forId(protocolTypeId), new EndPoint(host, port)); } - liveBrokers.add(new Broker(brokerId, endPoints)); + String rack = null; + if (brokerData.hasField(RACK_KEY_NAME)) { // V2 + rack = brokerData.getString(RACK_KEY_NAME); + } + liveBrokers.add(new Broker(brokerId, endPoints, rack)); } - } - controllerId = struct.getInt(CONTROLLER_ID_KEY_NAME); controllerEpoch = struct.getInt(CONTROLLER_EPOCH_KEY_NAME); this.partitionStates = partitionStates; @@ -249,14 +261,11 @@ public UpdateMetadataRequest(Struct struct) { @Override public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { - switch (versionId) { - case 0: - case 1: - return new UpdateMetadataResponse(Errors.forException(e).code()); - default: - throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", - versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id))); - } + if (versionId <= 2) + return new UpdateMetadataResponse(Errors.forException(e).code()); + else + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.UPDATE_METADATA_KEY.id))); } public int controllerId() { diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 7ccf07980ae24..b556b4647468e 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -86,8 +86,9 @@ public void testSerialization() throws Exception { createStopReplicaRequest(), createStopReplicaRequest().getErrorResponse(0, new UnknownServerException()), createStopReplicaResponse(), - createUpdateMetadataRequest(1), - createUpdateMetadataRequest(1).getErrorResponse(1, new UnknownServerException()), + createUpdateMetadataRequest(2, "rack1"), + createUpdateMetadataRequest(2, null), + createUpdateMetadataRequest(2, "rack1").getErrorResponse(2, new UnknownServerException()), createUpdateMetadataResponse(), createLeaderAndIsrRequest(), createLeaderAndIsrRequest().getErrorResponse(0, new UnknownServerException()), @@ -97,8 +98,11 @@ public void testSerialization() throws Exception { for (AbstractRequestResponse req : requestResponseList) checkSerialization(req, null); - checkSerialization(createUpdateMetadataRequest(0), 0); - checkSerialization(createUpdateMetadataRequest(0).getErrorResponse(0, new UnknownServerException()), 0); + checkSerialization(createUpdateMetadataRequest(0, null), 0); + checkSerialization(createUpdateMetadataRequest(0, null).getErrorResponse(0, new UnknownServerException()), 0); + checkSerialization(createUpdateMetadataRequest(1, null), 1); + checkSerialization(createUpdateMetadataRequest(1, "rack1"), 1); + checkSerialization(createUpdateMetadataRequest(1, null).getErrorResponse(1, new UnknownServerException()), 1); } private void checkSerialization(AbstractRequestResponse req, Integer version) throws Exception { @@ -120,7 +124,7 @@ private void checkSerialization(AbstractRequestResponse req, Integer version) th @Test public void produceResponseVersionTest() { - Map responseData = new HashMap(); + Map responseData = new HashMap<>(); responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE.code(), 10000, Record.NO_TIMESTAMP)); ProduceResponse v0Response = new ProduceResponse(responseData); ProduceResponse v1Response = new ProduceResponse(responseData, 10, 1); @@ -138,7 +142,7 @@ public void produceResponseVersionTest() { @Test public void fetchResponseVersionTest() { - Map responseData = new HashMap(); + Map responseData = new HashMap<>(); responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData(Errors.NONE.code(), 1000000, ByteBuffer.allocate(10))); FetchResponse v0Response = new FetchResponse(responseData); @@ -192,14 +196,14 @@ private AbstractRequestResponse createGroupCoordinatorResponse() { } private AbstractRequest createFetchRequest() { - Map fetchData = new HashMap(); + Map fetchData = new HashMap<>(); fetchData.put(new TopicPartition("test1", 0), new FetchRequest.PartitionData(100, 1000000)); fetchData.put(new TopicPartition("test2", 0), new FetchRequest.PartitionData(200, 1000000)); return new FetchRequest(-1, 100, 100000, fetchData); } private AbstractRequestResponse createFetchResponse() { - Map responseData = new HashMap(); + Map responseData = new HashMap<>(); responseData.put(new TopicPartition("test", 0), new FetchResponse.PartitionData(Errors.NONE.code(), 1000000, ByteBuffer.allocate(10))); return new FetchResponse(responseData, 0); } @@ -259,13 +263,13 @@ private AbstractRequestResponse createLeaveGroupResponse() { } private AbstractRequest createListOffsetRequest() { - Map offsetData = new HashMap(); + Map offsetData = new HashMap<>(); offsetData.put(new TopicPartition("test", 0), new ListOffsetRequest.PartitionData(1000000L, 10)); return new ListOffsetRequest(-1, offsetData); } private AbstractRequestResponse createListOffsetResponse() { - Map responseData = new HashMap(); + Map responseData = new HashMap<>(); responseData.put(new TopicPartition("test", 0), new ListOffsetResponse.PartitionData(Errors.NONE.code(), Arrays.asList(100L))); return new ListOffsetResponse(responseData); } @@ -289,13 +293,13 @@ private AbstractRequestResponse createMetadataResponse() { } private AbstractRequest createOffsetCommitRequest() { - Map commitData = new HashMap(); + Map commitData = new HashMap<>(); commitData.put(new TopicPartition("test", 0), new OffsetCommitRequest.PartitionData(100, "")); return new OffsetCommitRequest("group1", 100, "consumer1", 1000000, commitData); } private AbstractRequestResponse createOffsetCommitResponse() { - Map responseData = new HashMap(); + Map responseData = new HashMap<>(); responseData.put(new TopicPartition("test", 0), Errors.NONE.code()); return new OffsetCommitResponse(responseData); } @@ -305,19 +309,19 @@ private AbstractRequest createOffsetFetchRequest() { } private AbstractRequestResponse createOffsetFetchResponse() { - Map responseData = new HashMap(); + Map responseData = new HashMap<>(); responseData.put(new TopicPartition("test", 0), new OffsetFetchResponse.PartitionData(100L, "", Errors.NONE.code())); return new OffsetFetchResponse(responseData); } private AbstractRequest createProduceRequest() { - Map produceData = new HashMap(); + Map produceData = new HashMap<>(); produceData.put(new TopicPartition("test", 0), ByteBuffer.allocate(10)); return new ProduceRequest((short) 1, 5000, produceData); } private AbstractRequestResponse createProduceResponse() { - Map responseData = new HashMap(); + Map responseData = new HashMap<>(); responseData.put(new TopicPartition("test", 0), new ProduceResponse.PartitionResponse(Errors.NONE.code(), 10000, Record.NO_TIMESTAMP)); return new ProduceResponse(responseData, 0); } @@ -371,7 +375,7 @@ private AbstractRequestResponse createLeaderAndIsrResponse() { } @SuppressWarnings("deprecation") - private AbstractRequest createUpdateMetadataRequest(int version) { + private AbstractRequest createUpdateMetadataRequest(int version, String rack) { Map partitionStates = new HashMap<>(); List isr = Arrays.asList(1, 2); List replicas = Arrays.asList(1, 2, 3, 4); @@ -397,11 +401,10 @@ private AbstractRequest createUpdateMetadataRequest(int version) { endPoints2.put(SecurityProtocol.PLAINTEXT, new UpdateMetadataRequest.EndPoint("host1", 1244)); endPoints2.put(SecurityProtocol.SSL, new UpdateMetadataRequest.EndPoint("host2", 1234)); - Set liveBrokers = new HashSet<>(Arrays.asList(new UpdateMetadataRequest.Broker(0, endPoints1), - new UpdateMetadataRequest.Broker(1, endPoints2) + Set liveBrokers = new HashSet<>(Arrays.asList(new UpdateMetadataRequest.Broker(0, endPoints1, rack), + new UpdateMetadataRequest.Broker(1, endPoints2, rack) )); - - return new UpdateMetadataRequest(1, 10, partitionStates, liveBrokers); + return new UpdateMetadataRequest(version, 1, 10, partitionStates, liveBrokers); } } diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 3fb44d320afa3..24174bea95461 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -19,7 +19,6 @@ package kafka.admin import kafka.common._ import kafka.cluster.Broker - import kafka.log.LogConfig import kafka.server.ConfigType import kafka.utils._ @@ -32,14 +31,12 @@ import org.apache.kafka.common.errors.{ReplicaNotAvailableException, InvalidTopi import org.apache.kafka.common.protocol.{Errors, SecurityProtocol} import org.apache.kafka.common.requests.MetadataResponse -import scala.Predef._ import scala.collection._ -import scala.collection.JavaConverters._ -import scala.collection.mutable +import JavaConverters._ import mutable.ListBuffer +import scala.collection.mutable import collection.Map import collection.Set - import org.I0Itec.zkclient.exception.ZkNodeExistsException object AdminUtils extends Logging { @@ -48,11 +45,13 @@ object AdminUtils extends Logging { val EntityConfigChangeZnodePrefix = "config_change_" /** - * There are 2 goals of replica assignment: + * There are 3 goals of replica assignment: + * * 1. Spread the replicas evenly among brokers. * 2. For partitions assigned to a particular broker, their other replicas are spread over the other brokers. + * 3. If all brokers have rack information, assign the replicas for each partition to different racks if possible * - * To achieve this goal, we: + * To achieve this goal for replica assignment without considering racks, we: * 1. Assign the first replica of each partition by round-robin, starting from a random position in the broker list. * 2. Assign the remaining replicas of each partition with an increasing shift. * @@ -64,39 +63,177 @@ object AdminUtils extends Logging { * p8 p9 p5 p6 p7 (2nd replica) * p3 p4 p0 p1 p2 (3nd replica) * p7 p8 p9 p5 p6 (3nd replica) + * + * To create rack aware assignment, this API will first create a rack alternated broker list. For example, + * from this brokerID -> rack mapping: + * + * 0 -> "rack1", 1 -> "rack3", 2 -> "rack3", 3 -> "rack2", 4 -> "rack2", 5 -> "rack1" + * + * The rack alternated list will be: + * + * 0, 3, 1, 5, 4, 2 + * + * Then an easy round-robin assignment can be applied. Assume 6 partitions with replication factor of 3, the assignment + * will be: + * + * 0 -> 0,3,1 + * 1 -> 3,1,5 + * 2 -> 1,5,4 + * 3 -> 5,4,2 + * 4 -> 4,2,0 + * 5 -> 2,0,3 + * + * Once it has completed the first round-robin, if there are more partitions to assign, the algorithm will start + * shifting the followers. This is to ensure we will not always get the same set of sequences. + * In this case, if there is another partition to assign (partition #6), the assignment will be: + * + * 6 -> 0,4,2 (instead of repeating 0,3,1 as partition 0) + * + * The rack aware assignment always chooses the 1st replica of the partition using round robin on the rack alternated + * broker list. For rest of the replicas, it will be biased towards brokers on racks that do not have + * any replica assignment, until every rack has a replica. Then the assignment will go back to round-robin on + * the broker list. + * + * As the result, if the number of replicas is equal to or greater than the number of racks, it will ensure that + * each rack will get at least one replica. Otherwise, each rack will get at most one replica. In a perfect + * situation where the number of replicas is the same as the number of racks and each rack has the same number of + * brokers, it guarantees that the replica distribution is even across brokers and racks. + * + * @return a Map from partition id to replica ids + * @throws AdminOperationException If rack information is supplied but it is incomplete, or if it is not possible to + * assign each replica to a unique rack. + * */ - def assignReplicasToBrokers(brokerList: Seq[Int], + def assignReplicasToBrokers(brokerMetadatas: Seq[BrokerMetadata], nPartitions: Int, replicationFactor: Int, fixedStartIndex: Int = -1, - startPartitionId: Int = -1) - : Map[Int, Seq[Int]] = { + startPartitionId: Int = -1): Map[Int, Seq[Int]] = { if (nPartitions <= 0) throw new AdminOperationException("number of partitions must be larger than 0") if (replicationFactor <= 0) throw new AdminOperationException("replication factor must be larger than 0") - if (replicationFactor > brokerList.size) - throw new AdminOperationException("replication factor: " + replicationFactor + - " larger than available brokers: " + brokerList.size) - val ret = new mutable.HashMap[Int, List[Int]]() - val startIndex = if (fixedStartIndex >= 0) fixedStartIndex else rand.nextInt(brokerList.size) - var currentPartitionId = if (startPartitionId >= 0) startPartitionId else 0 + if (replicationFactor > brokerMetadatas.size) + throw new AdminOperationException(s"replication factor: $replicationFactor larger than available brokers: ${brokerMetadatas.size}") + if (brokerMetadatas.forall(_.rack.isEmpty)) + assignReplicasToBrokersRackUnaware(nPartitions, replicationFactor, brokerMetadatas.map(_.id), fixedStartIndex, + startPartitionId) + else { + if (brokerMetadatas.exists(_.rack.isEmpty)) + throw new AdminOperationException("Not all brokers have rack information for replica rack aware assignment") + assignReplicasToBrokersRackAware(nPartitions, replicationFactor, brokerMetadatas, fixedStartIndex, + startPartitionId) + } + } - var nextReplicaShift = if (fixedStartIndex >= 0) fixedStartIndex else rand.nextInt(brokerList.size) - for (i <- 0 until nPartitions) { - if (currentPartitionId > 0 && (currentPartitionId % brokerList.size == 0)) + private def assignReplicasToBrokersRackUnaware(nPartitions: Int, + replicationFactor: Int, + brokerList: Seq[Int], + fixedStartIndex: Int, + startPartitionId: Int): Map[Int, Seq[Int]] = { + val ret = mutable.Map[Int, Seq[Int]]() + val brokerArray = brokerList.toArray + val startIndex = if (fixedStartIndex >= 0) fixedStartIndex else rand.nextInt(brokerArray.length) + var currentPartitionId = math.max(0, startPartitionId) + var nextReplicaShift = if (fixedStartIndex >= 0) fixedStartIndex else rand.nextInt(brokerArray.length) + for (_ <- 0 until nPartitions) { + if (currentPartitionId > 0 && (currentPartitionId % brokerArray.length == 0)) nextReplicaShift += 1 - val firstReplicaIndex = (currentPartitionId + startIndex) % brokerList.size - var replicaList = List(brokerList(firstReplicaIndex)) + val firstReplicaIndex = (currentPartitionId + startIndex) % brokerArray.length + val replicaBuffer = mutable.ArrayBuffer(brokerArray(firstReplicaIndex)) for (j <- 0 until replicationFactor - 1) - replicaList ::= brokerList(replicaIndex(firstReplicaIndex, nextReplicaShift, j, brokerList.size)) - ret.put(currentPartitionId, replicaList.reverse) - currentPartitionId = currentPartitionId + 1 + replicaBuffer += brokerArray(replicaIndex(firstReplicaIndex, nextReplicaShift, j, brokerArray.length)) + ret.put(currentPartitionId, replicaBuffer) + currentPartitionId += 1 } - ret.toMap + ret + } + + private def assignReplicasToBrokersRackAware(nPartitions: Int, + replicationFactor: Int, + brokerMetadatas: Seq[BrokerMetadata], + fixedStartIndex: Int, + startPartitionId: Int): Map[Int, Seq[Int]] = { + val brokerRackMap = brokerMetadatas.collect { case BrokerMetadata(id, Some(rack)) => + id -> rack + }.toMap + val numRacks = brokerRackMap.values.toSet.size + val arrangedBrokerList = getRackAlternatedBrokerList(brokerRackMap) + val numBrokers = arrangedBrokerList.size + val ret = mutable.Map[Int, Seq[Int]]() + val startIndex = if (fixedStartIndex >= 0) fixedStartIndex else rand.nextInt(arrangedBrokerList.size) + var currentPartitionId = math.max(0, startPartitionId) + var nextReplicaShift = if (fixedStartIndex >= 0) fixedStartIndex else rand.nextInt(arrangedBrokerList.size) + for (_ <- 0 until nPartitions) { + if (currentPartitionId > 0 && (currentPartitionId % arrangedBrokerList.size == 0)) + nextReplicaShift += 1 + val firstReplicaIndex = (currentPartitionId + startIndex) % arrangedBrokerList.size + val leader = arrangedBrokerList(firstReplicaIndex) + val replicaBuffer = mutable.ArrayBuffer(leader) + val racksWithReplicas = mutable.Set(brokerRackMap(leader)) + val brokersWithReplicas = mutable.Set(leader) + var k = 0 + for (_ <- 0 until replicationFactor - 1) { + var done = false + while (!done) { + val broker = arrangedBrokerList(replicaIndex(firstReplicaIndex, nextReplicaShift * numRacks, k, arrangedBrokerList.size)) + val rack = brokerRackMap(broker) + // Skip this broker if + // 1. there is already a broker in the same rack that has assigned a replica AND there is one or more racks + // that do not have any replica, or + // 2. the broker has already assigned a replica AND there is one or more brokers that do not have replica assigned + if ((!racksWithReplicas.contains(rack) || racksWithReplicas.size == numRacks) + && (!brokersWithReplicas.contains(broker) || brokersWithReplicas.size == numBrokers)) { + replicaBuffer += broker + racksWithReplicas += rack + brokersWithReplicas += broker + done = true + } + k += 1 + } + } + ret.put(currentPartitionId, replicaBuffer) + currentPartitionId += 1 + } + ret } + /** + * Given broker and rack information, returns a list of brokers alternated by the rack. Assume + * this is the rack and its brokers: + * + * rack1: 0, 1, 2 + * rack2: 3, 4, 5 + * rack3: 6, 7, 8 + * + * This API would return the list of 0, 3, 6, 1, 4, 7, 2, 5, 8 + * + * This is essential to make sure that the assignReplicasToBrokers API can use such list and + * assign replicas to brokers in a simple round-robin fashion, while ensuring an even + * distribution of leader and replica counts on each broker and that replicas are + * distributed to all racks. + */ + private[admin] def getRackAlternatedBrokerList(brokerRackMap: Map[Int, String]): IndexedSeq[Int] = { + val brokersIteratorByRack = getInverseMap(brokerRackMap).map { case (rack, brokers) => + (rack, brokers.toIterator) + } + val racks = brokersIteratorByRack.keys.toArray.sorted + val result = new mutable.ArrayBuffer[Int] + var rackIndex = 0 + while (result.size < brokerRackMap.size) { + val rackIterator = brokersIteratorByRack(racks(rackIndex)) + if (rackIterator.hasNext) + result += rackIterator.next() + rackIndex = (rackIndex + 1) % racks.length + } + result + } + private[admin] def getInverseMap(brokerRackMap: Map[Int, String]): Map[String, Seq[Int]] = { + brokerRackMap.toSeq.map { case (id, rack) => (rack, id) } + .groupBy { case (rack, _) => rack } + .map { case (rack, rackAndIdList) => (rack, rackAndIdList.map { case (_, id) => id }.sorted) } + } /** * Add partitions to existing topic with optional replica assignment * @@ -110,7 +247,8 @@ object AdminUtils extends Logging { topic: String, numPartitions: Int = 1, replicaAssignmentStr: String = "", - checkBrokerAvailable: Boolean = true) { + checkBrokerAvailable: Boolean = true, + rackAwareMode: RackAwareMode = RackAwareMode.Enforced) { val existingPartitionsReplicaList = zkUtils.getReplicaAssignmentForTopics(List(topic)) if (existingPartitionsReplicaList.size == 0) throw new AdminOperationException("The topic %s does not exist".format(topic)) @@ -124,16 +262,16 @@ object AdminUtils extends Logging { throw new AdminOperationException("The number of partitions for a topic can only be increased") // create the new partition replication list - val brokerList = zkUtils.getSortedBrokerList() - val newPartitionReplicaList = if (replicaAssignmentStr == null || replicaAssignmentStr == "") { - var startIndex = brokerList.indexWhere(_ >= existingReplicaListForPartitionZero.head) - if(startIndex < 0) { - startIndex = 0 + val brokerMetadatas = getBrokerMetadatas(zkUtils, rackAwareMode) + val newPartitionReplicaList = + if (replicaAssignmentStr == null || replicaAssignmentStr == "") { + val startIndex = math.max(0, brokerMetadatas.indexWhere(_.id >= existingReplicaListForPartitionZero.head)) + AdminUtils.assignReplicasToBrokers(brokerMetadatas, partitionsToAdd, existingReplicaListForPartitionZero.size, + startIndex, existingPartitionsReplicaList.size) } - AdminUtils.assignReplicasToBrokers(brokerList, partitionsToAdd, existingReplicaListForPartitionZero.size, startIndex, existingPartitionsReplicaList.size) - } - else - getManualReplicaAssignment(replicaAssignmentStr, brokerList.toSet, existingPartitionsReplicaList.size, checkBrokerAvailable) + else + getManualReplicaAssignment(replicaAssignmentStr, brokerMetadatas.map(_.id).toSet, + existingPartitionsReplicaList.size, checkBrokerAvailable) // check if manual assignment has the right replication factor val unmatchedRepFactorList = newPartitionReplicaList.values.filter(p => (p.size != existingReplicaListForPartitionZero.size)) @@ -237,13 +375,32 @@ object AdminUtils extends Logging { def topicExists(zkUtils: ZkUtils, topic: String): Boolean = zkUtils.zkClient.exists(getTopicPath(topic)) + def getBrokerMetadatas(zkUtils: ZkUtils, rackAwareMode: RackAwareMode = RackAwareMode.Enforced, + brokerList: Option[Seq[Int]] = None): Seq[BrokerMetadata] = { + val allBrokers = zkUtils.getAllBrokersInCluster() + val brokers = brokerList.map(brokerIds => allBrokers.filter(b => brokerIds.contains(b.id))).getOrElse(allBrokers) + val brokersWithRack = brokers.filter(_.rack.nonEmpty) + if (rackAwareMode == RackAwareMode.Enforced && brokersWithRack.nonEmpty && brokersWithRack.size < brokers.size) { + throw new AdminOperationException("Not all brokers have rack information. Add --disable-rack-aware in command line" + + " to make replica assignment without rack information.") + } + val brokerMetadatas = rackAwareMode match { + case RackAwareMode.Disabled => brokers.map(broker => BrokerMetadata(broker.id, None)) + case RackAwareMode.Safe if brokersWithRack.size < brokers.size => + brokers.map(broker => BrokerMetadata(broker.id, None)) + case _ => brokers.map(broker => BrokerMetadata(broker.id, broker.rack)) + } + brokerMetadatas.sortBy(_.id) + } + def createTopic(zkUtils: ZkUtils, topic: String, partitions: Int, replicationFactor: Int, - topicConfig: Properties = new Properties) { - val brokerList = zkUtils.getSortedBrokerList() - val replicaAssignment = AdminUtils.assignReplicasToBrokers(brokerList, partitions, replicationFactor) + topicConfig: Properties = new Properties, + rackAwareMode: RackAwareMode = RackAwareMode.Enforced) { + val brokerMetadatas = getBrokerMetadatas(zkUtils, rackAwareMode) + val replicaAssignment = AdminUtils.assignReplicasToBrokers(brokerMetadatas, partitions, replicationFactor) AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, replicaAssignment, topicConfig) } @@ -304,6 +461,7 @@ object AdminUtils extends Logging { /** * Update the config for a client and create a change notification so the change will propagate to other brokers + * * @param zkUtils Zookeeper utilities used to write the config to ZK * @param clientId: The clientId for which configs are being changed * @param configs: The final set of configs that will be applied to the topic. If any new configs need to be added or @@ -316,6 +474,7 @@ object AdminUtils extends Logging { /** * Update the config for an existing topic and create a change notification so the change will propagate to other brokers + * * @param zkUtils Zookeeper utilities used to write the config to ZK * @param topic: The topic for which configs are being changed * @param configs: The final set of configs that will be applied to the topic. If any new configs need to be added or diff --git a/core/src/main/scala/kafka/admin/BrokerMetadata.scala b/core/src/main/scala/kafka/admin/BrokerMetadata.scala new file mode 100644 index 0000000000000..86831e376e5bd --- /dev/null +++ b/core/src/main/scala/kafka/admin/BrokerMetadata.scala @@ -0,0 +1,23 @@ +/** + * 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.admin + +/** + * Broker metadata used by admin tools. + * + * @param id an integer that uniquely identifies this broker + * @param rack the rack of the broker, which is used to in rack aware partition assignment for fault tolerance. + * Examples: "RACK1", "us-east-1d" + */ +case class BrokerMetadata(id: Int, rack: Option[String]) diff --git a/core/src/main/scala/kafka/admin/RackAwareMode.scala b/core/src/main/scala/kafka/admin/RackAwareMode.scala new file mode 100644 index 0000000000000..45555b60bfcea --- /dev/null +++ b/core/src/main/scala/kafka/admin/RackAwareMode.scala @@ -0,0 +1,42 @@ +/** + * 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.admin + +/** + * Mode to control how rack aware replica assignment will be executed + */ +object RackAwareMode { + + /** + * Ignore all rack information in replica assignment. This is an optional mode used in command line. + */ + case object Disabled extends RackAwareMode + + /** + * Assume every broker has rack, or none of the brokers has rack. If only partial brokers have rack, fail fast + * in replica assignment. This is the default mode in command line tools (TopicCommand and ReassignPartitionsCommand). + */ + case object Enforced extends RackAwareMode + + /** + * Use rack information if every broker has a rack. Otherwise, fallback to Disabled mode. This is used in auto topic + * creation. + */ + case object Safe extends RackAwareMode +} + +sealed trait RackAwareMode diff --git a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala index 13e423d4c23d7..446ab9f5374c8 100755 --- a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala +++ b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala @@ -91,23 +91,33 @@ object ReassignPartitionsCommand extends Logging { if (duplicateReassignments.nonEmpty) throw new AdminCommandFailedException("Broker list contains duplicate entries: %s".format(duplicateReassignments.mkString(","))) val topicsToMoveJsonString = Utils.readFileAsString(topicsToMoveJsonFile) + val disableRackAware = opts.options.has(opts.disableRackAware) + val (proposedAssignments, currentAssignments) = generateAssignment(zkUtils, brokerListToReassign, topicsToMoveJsonString, disableRackAware) + println("Current partition replica assignment\n\n%s".format(zkUtils.getPartitionReassignmentZkData(currentAssignments))) + println("Proposed partition reassignment configuration\n\n%s".format(zkUtils.getPartitionReassignmentZkData(proposedAssignments))) + } + + def generateAssignment(zkUtils: ZkUtils, brokerListToReassign: Seq[Int], topicsToMoveJsonString: String, disableRackAware: Boolean): (Map[TopicAndPartition, Seq[Int]], Map[TopicAndPartition, Seq[Int]]) = { val topicsToReassign = zkUtils.parseTopicsData(topicsToMoveJsonString) val duplicateTopicsToReassign = CoreUtils.duplicates(topicsToReassign) if (duplicateTopicsToReassign.nonEmpty) throw new AdminCommandFailedException("List of topics to reassign contains duplicate entries: %s".format(duplicateTopicsToReassign.mkString(","))) - val topicPartitionsToReassign = zkUtils.getReplicaAssignmentForTopics(topicsToReassign) - - var partitionsToBeReassigned : Map[TopicAndPartition, Seq[Int]] = new mutable.HashMap[TopicAndPartition, List[Int]]() - val groupedByTopic = topicPartitionsToReassign.groupBy(tp => tp._1.topic) - groupedByTopic.foreach { topicInfo => - val assignedReplicas = AdminUtils.assignReplicasToBrokers(brokerListToReassign, topicInfo._2.size, - topicInfo._2.head._2.size) - partitionsToBeReassigned ++= assignedReplicas.map(replicaInfo => (TopicAndPartition(topicInfo._1, replicaInfo._1) -> replicaInfo._2)) + val currentAssignment = zkUtils.getReplicaAssignmentForTopics(topicsToReassign) + + val groupedByTopic = currentAssignment.groupBy { case (tp, _) => tp.topic } + val rackAwareMode = if (disableRackAware) RackAwareMode.Disabled else RackAwareMode.Enforced + val brokerMetadatas = AdminUtils.getBrokerMetadatas(zkUtils, rackAwareMode, Some(brokerListToReassign)) + + val partitionsToBeReassigned = mutable.Map[TopicAndPartition, Seq[Int]]() + groupedByTopic.foreach { case (topic, assignment) => + val (_, replicas) = assignment.head + val assignedReplicas = AdminUtils.assignReplicasToBrokers(brokerMetadatas, assignment.size, replicas.size) + partitionsToBeReassigned ++= assignedReplicas.map { case (partition, replicas) => + (TopicAndPartition(topic, partition) -> replicas) + } } - val currentPartitionReplicaAssignment = zkUtils.getReplicaAssignmentForTopics(partitionsToBeReassigned.map(_._1.topic).toSeq) - println("Current partition replica assignment\n\n%s" - .format(zkUtils.getPartitionReassignmentZkData(currentPartitionReplicaAssignment))) - println("Proposed partition reassignment configuration\n\n%s".format(zkUtils.getPartitionReassignmentZkData(partitionsToBeReassigned))) + + (partitionsToBeReassigned, currentAssignment) } def executeAssignment(zkUtils: ZkUtils, opts: ReassignPartitionsCommandOptions) { @@ -200,7 +210,8 @@ object ReassignPartitionsCommand extends Logging { .withRequiredArg .describedAs("brokerlist") .ofType(classOf[String]) - + val disableRackAware = parser.accepts("disable-rack-aware", "Disable rack aware replica assignment") + if(args.length == 0) CommandLineUtils.printUsageAndDie(parser, "This command moves topic partitions between replicas.") diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index d4212c551f549..e89e09d2a3bf9 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -105,7 +105,9 @@ object TopicCommand extends Logging { val partitions = opts.options.valueOf(opts.partitionsOpt).intValue val replicas = opts.options.valueOf(opts.replicationFactorOpt).intValue warnOnMaxMessagesChange(configs, replicas) - AdminUtils.createTopic(zkUtils, topic, partitions, replicas, configs) + val rackAwareMode = if (opts.options.has(opts.disableRackAware)) RackAwareMode.Disabled + else RackAwareMode.Enforced + AdminUtils.createTopic(zkUtils, topic, partitions, replicas, configs, rackAwareMode) } println("Created topic \"%s\".".format(topic)) } catch { @@ -324,6 +326,7 @@ object TopicCommand extends Logging { val ifNotExistsOpt = parser.accepts("if-not-exists", "if set when creating topics, the action will only execute if the topic does not already exist") + val disableRackAware = parser.accepts("disable-rack-aware", "Disable rack aware replica assignment") val options = parser.parse(args : _*) val allTopicLevelOpts: Set[OptionSpec[_]] = Set(alterOpt, createOpt, describeOpt, listOpt, deleteOpt) diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index 7340f148af1ec..77b85e01b4020 100755 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -19,6 +19,7 @@ package kafka.cluster import java.nio.ByteBuffer +import kafka.api.ApiUtils._ import kafka.common.{BrokerEndPointNotAvailableException, BrokerNotAvailableException, KafkaException} import kafka.utils.Json import org.apache.kafka.common.Node @@ -32,26 +33,41 @@ import org.apache.kafka.common.protocol.SecurityProtocol object Broker { /** - * Create a broker object from id and JSON string. - * @param id - * @param brokerInfoString - * - * Version 1 JSON schema for a broker is: - * {"version":1, - * "host":"localhost", - * "port":9092 - * "jmx_port":9999, - * "timestamp":"2233345666" } - * - * The current JSON schema for a broker is: - * {"version":2, - * "host","localhost", - * "port",9092 - * "jmx_port":9999, - * "timestamp":"2233345666", - * "endpoints": ["PLAINTEXT://host1:9092", - * "SSL://host1:9093"] - */ + * Create a broker object from id and JSON string. + * + * @param id + * @param brokerInfoString + * + * Version 1 JSON schema for a broker is: + * { + * "version":1, + * "host":"localhost", + * "port":9092 + * "jmx_port":9999, + * "timestamp":"2233345666" + * } + * + * Version 2 JSON schema for a broker is: + * { + * "version":2, + * "host":"localhost", + * "port":9092 + * "jmx_port":9999, + * "timestamp":"2233345666", + * "endpoints":["PLAINTEXT://host1:9092", "SSL://host1:9093"] + * } + * + * Version 3 (current) JSON schema for a broker is: + * { + * "version":3, + * "host":"localhost", + * "port":9092 + * "jmx_port":9999, + * "timestamp":"2233345666", + * "endpoints":["PLAINTEXT://host1:9092", "SSL://host1:9093"], + * "rack":"dc1" + * } + */ def createBroker(id: Int, brokerInfoString: String): Broker = { if (brokerInfoString == null) throw new BrokerNotAvailableException(s"Broker id $id does not exist") @@ -75,9 +91,8 @@ object Broker { (ep.protocolType, ep) }.toMap } - - - new Broker(id, endpoints) + val rack = brokerInfo.get("rack").filter(_ != null).map(_.asInstanceOf[String]) + new Broker(id, endpoints, rack) case None => throw new BrokerNotAvailableException(s"Broker id $id does not exist") } @@ -86,61 +101,34 @@ object Broker { throw new KafkaException(s"Failed to parse the broker info from zookeeper: $brokerInfoString", t) } } - - /** - * - * @param buffer Containing serialized broker. - * Current serialization is: - * id (int), number of endpoints (int), serialized endpoints - * @return broker object - */ - def readFrom(buffer: ByteBuffer): Broker = { - val id = buffer.getInt - val numEndpoints = buffer.getInt - - val endpoints = List.range(0, numEndpoints).map(i => EndPoint.readFrom(buffer)) - .map(ep => ep.protocolType -> ep).toMap - new Broker(id, endpoints) - } } -case class Broker(id: Int, endPoints: collection.Map[SecurityProtocol, EndPoint]) { +case class Broker(id: Int, endPoints: collection.Map[SecurityProtocol, EndPoint], rack: Option[String]) { - override def toString: String = id + " : " + endPoints.values.mkString("(",",",")") + override def toString: String = + s"$id : ${endPoints.values.mkString("(",",",")")} : ${rack.orNull}" + + def this(id: Int, endPoints: Map[SecurityProtocol, EndPoint]) = { + this(id, endPoints, None) + } def this(id: Int, host: String, port: Int, protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT) = { - this(id, Map(protocol -> EndPoint(host, port, protocol))) + this(id, Map(protocol -> EndPoint(host, port, protocol)), None) } def this(bep: BrokerEndPoint, protocol: SecurityProtocol) = { this(bep.id, bep.host, bep.port, protocol) } - - def writeTo(buffer: ByteBuffer) { - buffer.putInt(id) - buffer.putInt(endPoints.size) - for(endpoint <- endPoints.values) { - endpoint.writeTo(buffer) - } - } - - def sizeInBytes: Int = - 4 + /* broker id*/ - 4 + /* number of endPoints */ - endPoints.values.map(_.sizeInBytes).sum /* end points */ - - def supportsChannel(protocolType: SecurityProtocol): Unit = { - endPoints.contains(protocolType) - } - def getNode(protocolType: SecurityProtocol): Node = { - val endpoint = endPoints.getOrElse(protocolType, throw new BrokerEndPointNotAvailableException("End point %s not found for broker %d".format(protocolType,id))) + val endpoint = endPoints.getOrElse(protocolType, + throw new BrokerEndPointNotAvailableException(s"End point with security protocol $protocolType not found for broker $id")) new Node(id, endpoint.host, endpoint.port) } def getBrokerEndPoint(protocolType: SecurityProtocol): BrokerEndPoint = { - val endpoint = endPoints.getOrElse(protocolType, throw new BrokerEndPointNotAvailableException("End point %s not found for broker %d".format(protocolType,id))) + val endpoint = endPoints.getOrElse(protocolType, + throw new BrokerEndPointNotAvailableException(s"End point with security protocol $protocolType not found for broker $id")) new BrokerEndPoint(id, endpoint.host, endpoint.port) } diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index 3b1a458e6be1f..ea156fa66fd24 100755 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -16,24 +16,25 @@ */ package kafka.controller -import kafka.api.{LeaderAndIsr, KAFKA_0_9_0, PartitionStateInfo} +import java.net.SocketTimeoutException +import java.util.concurrent.{BlockingQueue, LinkedBlockingQueue} + +import kafka.api._ +import kafka.cluster.Broker +import kafka.common.{KafkaException, TopicAndPartition} +import kafka.server.KafkaConfig import kafka.utils._ -import org.apache.kafka.clients.{ClientResponse, ClientRequest, ManualMetadataUpdater, NetworkClient} -import org.apache.kafka.common.{BrokerEndPoint, TopicPartition, Node} +import org.apache.kafka.clients.{ClientRequest, ClientResponse, ManualMetadataUpdater, NetworkClient} import org.apache.kafka.common.metrics.Metrics -import org.apache.kafka.common.network.{LoginType, Selectable, ChannelBuilders, Selector, NetworkReceive, Mode} -import org.apache.kafka.common.protocol.{SecurityProtocol, ApiKeys} -import org.apache.kafka.common.requests._ +import org.apache.kafka.common.network.{ChannelBuilders, LoginType, Mode, NetworkReceive, Selectable, Selector} +import org.apache.kafka.common.protocol.{ApiKeys, SecurityProtocol} +import org.apache.kafka.common.requests.{UpdateMetadataRequest, _} import org.apache.kafka.common.utils.Time -import collection.mutable.HashMap -import kafka.cluster.Broker -import java.net.{SocketTimeoutException} -import java.util.concurrent.{LinkedBlockingQueue, BlockingQueue} -import kafka.server.KafkaConfig -import collection.mutable -import kafka.common.{KafkaException, TopicAndPartition} -import collection.Set -import collection.JavaConverters._ +import org.apache.kafka.common.{BrokerEndPoint, Node, TopicPartition} + +import scala.collection.JavaConverters._ +import scala.collection.{Set, mutable} +import scala.collection.mutable.HashMap class ControllerChannelManager(controllerContext: ControllerContext, config: KafkaConfig, time: Time, metrics: Metrics, threadNamePrefix: Option[String] = None) extends Logging { protected val brokerStateInfo = new HashMap[Int, ControllerBrokerStateInfo] @@ -380,7 +381,9 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging topicPartition -> partitionState } - val version = if (controller.config.interBrokerProtocolVersion >= KAFKA_0_9_0) (1: Short) else (0: Short) + val version = if (controller.config.interBrokerProtocolVersion >= KAFKA_0_10_0_IV0) 2: Short + else if (controller.config.interBrokerProtocolVersion >= KAFKA_0_9_0) 1: Short + else 0: Short val updateMetadataRequest = if (version == 0) { @@ -395,9 +398,9 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging val endPoints = broker.endPoints.map { case (securityProtocol, endPoint) => securityProtocol -> new UpdateMetadataRequest.EndPoint(endPoint.host, endPoint.port) } - new UpdateMetadataRequest.Broker(broker.id, endPoints.asJava) + new UpdateMetadataRequest.Broker(broker.id, endPoints.asJava, broker.rack.orNull) } - new UpdateMetadataRequest(controllerId, controllerEpoch, partitionStates.asJava, liveBrokers.asJava) + new UpdateMetadataRequest(version, controllerId, controllerEpoch, partitionStates.asJava, liveBrokers.asJava) } controller.sendRequest(broker, ApiKeys.UPDATE_METADATA_KEY, Some(version), updateMetadataRequest, null) diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 5f9ec8ba4cee4..452f721c0a583 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import java.lang.{Long => JLong, Short => JShort} import java.util.Properties -import kafka.admin.AdminUtils +import kafka.admin.{RackAwareMode, AdminUtils} import kafka.api._ import kafka.cluster.Partition import kafka.common._ @@ -624,7 +624,7 @@ class KafkaApis(val requestChannel: RequestChannel, replicationFactor: Int, properties: Properties = new Properties()): MetadataResponse.TopicMetadata = { try { - AdminUtils.createTopic(zkUtils, topic, numPartitions, replicationFactor, properties) + AdminUtils.createTopic(zkUtils, topic, numPartitions, replicationFactor, properties, RackAwareMode.Safe) info("Auto creation of topic %s with %d partitions and replication factor %d is successful" .format(topic, numPartitions, replicationFactor)) new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, topic, java.util.Collections.emptyList()) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 8d14edd17f844..9c2487673e262 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -222,6 +222,8 @@ object KafkaConfig { val MaxConnectionsPerIpProp = "max.connections.per.ip" val MaxConnectionsPerIpOverridesProp = "max.connections.per.ip.overrides" val ConnectionsMaxIdleMsProp = "connections.max.idle.ms" + /***************** rack configuration *************/ + val RackProp = "broker.rack" /** ********* Log Configuration ***********/ val NumPartitionsProp = "num.partitions" val LogDirsProp = "log.dirs" @@ -388,6 +390,8 @@ object KafkaConfig { val MaxConnectionsPerIpDoc = "The maximum number of connections we allow from each ip address" val MaxConnectionsPerIpOverridesDoc = "Per-ip or hostname overrides to the default maximum number of connections" val ConnectionsMaxIdleMsDoc = "Idle connections timeout: the server socket processor threads close the connections that idle more than this" + /************* Rack Configuration **************/ + val RackDoc = "Rack of the broker. This will be used in rack aware replication assignment for fault tolerance. Examples: `RACK1`, `us-east-1d`" /** ********* Log Configuration ***********/ val NumPartitionsDoc = "The default number of log partitions per topic" val LogDirDoc = "The directory in which the log data is kept (supplemental for " + LogDirsProp + " property)" @@ -571,6 +575,9 @@ object KafkaConfig { .define(MaxConnectionsPerIpOverridesProp, STRING, Defaults.MaxConnectionsPerIpOverrides, MEDIUM, MaxConnectionsPerIpOverridesDoc) .define(ConnectionsMaxIdleMsProp, LONG, Defaults.ConnectionsMaxIdleMs, MEDIUM, ConnectionsMaxIdleMsDoc) + /************ Rack Configuration ******************/ + .define(RackProp, STRING, null, MEDIUM, RackDoc) + /** ********* Log Configuration ***********/ .define(NumPartitionsProp, INT, Defaults.NumPartitions, atLeast(1), MEDIUM, NumPartitionsDoc) .define(LogDirProp, STRING, Defaults.LogDir, HIGH, LogDirDoc) @@ -771,6 +778,9 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean) extends Abstra getMap(KafkaConfig.MaxConnectionsPerIpOverridesProp, getString(KafkaConfig.MaxConnectionsPerIpOverridesProp)).map { case (k, v) => (k, v.toInt)} val connectionsMaxIdleMs = getLong(KafkaConfig.ConnectionsMaxIdleMsProp) + /***************** rack configuration **************/ + val rack = Option(getString(KafkaConfig.RackProp)) + /** ********* Log Configuration ***********/ val autoCreateTopicsEnable = getBoolean(KafkaConfig.AutoCreateTopicsEnableProp) val numPartitions = getInt(KafkaConfig.NumPartitionsProp) diff --git a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala index 928ff43e11932..2598e6dbee604 100644 --- a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala +++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala @@ -17,12 +17,14 @@ package kafka.server +import java.net.InetAddress + +import kafka.api.ApiVersion import kafka.cluster.EndPoint import kafka.utils._ +import org.I0Itec.zkclient.IZkStateListener import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.zookeeper.Watcher.Event.KeeperState -import org.I0Itec.zkclient.{IZkStateListener, ZkClient, ZkConnection} -import java.net.InetAddress /** @@ -35,7 +37,9 @@ import java.net.InetAddress */ class KafkaHealthcheck(private val brokerId: Int, private val advertisedEndpoints: Map[SecurityProtocol, EndPoint], - private val zkUtils: ZkUtils) extends Logging { + private val zkUtils: ZkUtils, + private val rack: Option[String], + private val interBrokerProtocolVersion: ApiVersion) extends Logging { val brokerIdPath = ZkUtils.BrokerIdsPath + "/" + brokerId val sessionExpireListener = new SessionExpireListener @@ -61,7 +65,8 @@ class KafkaHealthcheck(private val brokerId: Int, // only PLAINTEXT is supported as default // if the broker doesn't listen on PLAINTEXT protocol, an empty endpoint will be registered and older clients will be unable to connect val plaintextEndpoint = updatedEndpoints.getOrElse(SecurityProtocol.PLAINTEXT, new EndPoint(null,-1,null)) - zkUtils.registerBrokerInZk(brokerId, plaintextEndpoint.host, plaintextEndpoint.port, updatedEndpoints, jmxPort) + zkUtils.registerBrokerInZk(brokerId, plaintextEndpoint.host, plaintextEndpoint.port, updatedEndpoints, jmxPort, rack, + interBrokerProtocolVersion) } /** diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index 2f5441ac12115..e29494baa1d99 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -239,7 +239,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr else (protocol, endpoint) } - kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, listeners, zkUtils) + kafkaHealthcheck = new KafkaHealthcheck(config.brokerId, listeners, zkUtils, config.rack, config.interBrokerProtocolVersion) kafkaHealthcheck.startup() // Now that the broker id is successfully registered via KafkaHealthcheck, checkpoint it diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 1fdd717044c2f..6df261c7482c9 100755 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -159,7 +159,7 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { endPoints.put(protocol, EndPoint(ep.host, ep.port, protocol)) nodes.put(protocol, new Node(broker.id, ep.host, ep.port)) } - aliveBrokers(broker.id) = Broker(broker.id, endPoints.asScala) + aliveBrokers(broker.id) = Broker(broker.id, endPoints.asScala, Option(broker.rack)) aliveNodes(broker.id) = nodes.asScala } diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index f39ed014cec9b..99c8196a03897 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -18,31 +18,26 @@ package kafka.utils import java.util.concurrent.CountDownLatch + +import kafka.admin._ +import kafka.api.{ApiVersion, KAFKA_0_10_0_IV0, LeaderAndIsr} import kafka.cluster._ +import kafka.common.{KafkaException, NoEpochForPartitionException, TopicAndPartition} import kafka.consumer.{ConsumerThreadId, TopicCount} +import kafka.controller.{KafkaController, LeaderIsrAndControllerEpoch, ReassignedPartitionsContext} import kafka.server.ConfigType -import org.I0Itec.zkclient.{ZkClient,ZkConnection} -import org.I0Itec.zkclient.exception.{ZkException, ZkNodeExistsException, ZkNoNodeException, - ZkMarshallingError, ZkBadVersionException} +import kafka.utils.ZkUtils._ +import org.I0Itec.zkclient.exception.{ZkBadVersionException, ZkException, ZkMarshallingError, ZkNoNodeException, ZkNodeExistsException} import org.I0Itec.zkclient.serialize.ZkSerializer +import org.I0Itec.zkclient.{ZkClient, ZkConnection} import org.apache.kafka.common.config.ConfigException import org.apache.kafka.common.protocol.SecurityProtocol -import org.apache.zookeeper.ZooDefs -import scala.collection._ -import kafka.api.LeaderAndIsr -import org.apache.zookeeper.data.{ACL, Stat} -import kafka.admin._ -import kafka.common.{KafkaException, NoEpochForPartitionException} -import kafka.controller.ReassignedPartitionsContext -import kafka.controller.KafkaController -import kafka.controller.LeaderIsrAndControllerEpoch -import kafka.common.TopicAndPartition -import kafka.utils.ZkUtils._ -import org.apache.zookeeper.AsyncCallback.{DataCallback,StringCallback} -import org.apache.zookeeper.CreateMode -import org.apache.zookeeper.KeeperException +import org.apache.zookeeper.AsyncCallback.{DataCallback, StringCallback} import org.apache.zookeeper.KeeperException.Code -import org.apache.zookeeper.ZooKeeper +import org.apache.zookeeper.data.{ACL, Stat} +import org.apache.zookeeper.{CreateMode, KeeperException, ZooDefs, ZooKeeper} + +import scala.collection._ object ZkUtils { val ConsumersPath = "/consumers" @@ -256,19 +251,43 @@ class ZkUtils(val zkClient: ZkClient, } /** - * Register brokers with v2 json format (which includes multiple endpoints). + * Register brokers with v3 json format (which includes multiple endpoints and rack) if + * the apiVersion is 0.10.0.X or above. Register the broker with v2 json format otherwise. + * Due to KAFKA-3100, 0.9.0.0 broker and old clients will break if JSON version is above 2. + * We include v2 to make it possible for the broker to migrate from 0.9.0.0 to 0.10.0.X without having to upgrade + * to 0.9.0.1 first (clients have to be upgraded to 0.9.0.1 in any case). + * * This format also includes default endpoints for compatibility with older clients. - * @param id - * @param host - * @param port - * @param advertisedEndpoints - * @param jmxPort + * + * @param id broker ID + * @param host broker host name + * @param port broker port + * @param advertisedEndpoints broker end points + * @param jmxPort jmx port + * @param rack broker rack + * @param apiVersion Kafka version the broker is running as */ - def registerBrokerInZk(id: Int, host: String, port: Int, advertisedEndpoints: collection.Map[SecurityProtocol, EndPoint], jmxPort: Int) { + def registerBrokerInZk(id: Int, + host: String, + port: Int, + advertisedEndpoints: collection.Map[SecurityProtocol, EndPoint], + jmxPort: Int, + rack: Option[String], + apiVersion: ApiVersion) { val brokerIdPath = BrokerIdsPath + "/" + id val timestamp = SystemTime.milliseconds.toString - val brokerInfo = Json.encode(Map("version" -> 2, "host" -> host, "port" -> port, "endpoints"->advertisedEndpoints.values.map(_.connectionString).toArray, "jmx_port" -> jmxPort, "timestamp" -> timestamp)) + val version = if (apiVersion >= KAFKA_0_10_0_IV0) 3 else 2 + var jsonMap = Map("version" -> version, + "host" -> host, + "port" -> port, + "endpoints" -> advertisedEndpoints.values.map(_.connectionString).toArray, + "jmx_port" -> jmxPort, + "timestamp" -> timestamp + ) + rack.foreach(rack => if (version >= 3) jsonMap += ("rack" -> rack)) + + val brokerInfo = Json.encode(jsonMap) registerBrokerInZk(brokerIdPath, brokerInfo) info("Registered broker %d at path %s with addresses: %s".format(id, brokerIdPath, advertisedEndpoints.mkString(","))) @@ -745,6 +764,7 @@ class ZkUtils(val zkClient: ZkClient, /** * This API takes in a broker id, queries zookeeper for the broker metadata and returns the metadata for that broker * or throws an exception if the broker dies before the query to zookeeper finishes + * * @param brokerId The broker id * @return An optional Broker object encapsulating the broker metadata */ @@ -768,7 +788,6 @@ class ZkUtils(val zkClient: ZkClient, case e: ZkNoNodeException => { createParentPath(BrokerSequenceIdPath, acls) try { - import scala.collection.JavaConversions._ zkClient.createPersistent(BrokerSequenceIdPath, "", acls) 0 } catch { @@ -880,7 +899,6 @@ class ZKConfig(props: VerifiableProperties) { object ZkPath { @volatile private var isNamespacePresent: Boolean = false - import scala.collection.JavaConversions._ def checkNamespace(client: ZkClient) { if(isNamespacePresent) diff --git a/core/src/test/scala/integration/kafka/api/RackAwareAutoTopicCreationTest.scala b/core/src/test/scala/integration/kafka/api/RackAwareAutoTopicCreationTest.scala new file mode 100644 index 0000000000000..a2f2041e7aa8b --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/RackAwareAutoTopicCreationTest.scala @@ -0,0 +1,65 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.api + +import java.util.Properties + +import kafka.admin.{RackAwareMode, AdminUtils, RackAwareTest} +import kafka.integration.KafkaServerTestHarness +import kafka.server.KafkaConfig +import kafka.utils.TestUtils +import org.apache.kafka.clients.producer.ProducerRecord +import org.junit.Assert._ +import org.junit.Test +import scala.collection.Map + +class RackAwareAutoTopicCreationTest extends KafkaServerTestHarness with RackAwareTest { + val numServers = 4 + val numPartitions = 8 + val replicationFactor = 2 + val overridingProps = new Properties() + overridingProps.put(KafkaConfig.NumPartitionsProp, numPartitions.toString) + overridingProps.put(KafkaConfig.DefaultReplicationFactorProp, replicationFactor.toString) + + def generateConfigs() = + (0 until numServers) map { node => + TestUtils.createBrokerConfig(node, zkConnect, enableControlledShutdown = false, rack = Some((node / 2).toString)) + } map (KafkaConfig.fromProps(_, overridingProps)) + + private val topic = "topic" + + @Test + def testAutoCreateTopic() { + val producer = TestUtils.createNewProducer(brokerList, retries = 5) + try { + // Send a message to auto-create the topic + val record = new ProducerRecord(topic, null, "key".getBytes, "value".getBytes) + assertEquals("Should have offset 0", 0L, producer.send(record).get.offset) + + // double check that the topic is created with leader elected + TestUtils.waitUntilLeaderIsElectedOrChanged(zkUtils, topic, 0) + val assignment = zkUtils.getReplicaAssignmentForTopics(Seq(topic)).map { case (topicPartition, replicas) => + topicPartition.partition -> replicas + } + val brokerMetadatas = AdminUtils.getBrokerMetadatas(zkUtils, RackAwareMode.Enforced) + val expectedMap = Map(0 -> "0", 1 -> "0", 2 -> "1", 3 -> "1") + assertEquals(expectedMap, brokerMetadatas.map(b => b.id -> b.rack.get).toMap) + checkReplicaDistribution(assignment, expectedMap, numServers, numPartitions, replicationFactor) + } finally producer.close() + } +} + diff --git a/core/src/test/scala/unit/kafka/admin/AdminRackAwareTest.scala b/core/src/test/scala/unit/kafka/admin/AdminRackAwareTest.scala new file mode 100644 index 0000000000000..27ff4d4e34ad5 --- /dev/null +++ b/core/src/test/scala/unit/kafka/admin/AdminRackAwareTest.scala @@ -0,0 +1,196 @@ +/** + * 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.admin + +import kafka.utils.{Logging, TestUtils} +import kafka.zk.ZooKeeperTestHarness +import org.junit.Assert._ +import org.junit.Test + +import scala.collection.{Map, Seq} + +class AdminRackAwareTest extends RackAwareTest with Logging { + + @Test + def testGetRackAlternatedBrokerListAndAssignReplicasToBrokers() { + val rackMap = Map(0 -> "rack1", 1 -> "rack3", 2 -> "rack3", 3 -> "rack2", 4 -> "rack2", 5 -> "rack1") + val newList = AdminUtils.getRackAlternatedBrokerList(rackMap) + assertEquals(List(0, 3, 1, 5, 4, 2), newList) + val anotherList = AdminUtils.getRackAlternatedBrokerList(rackMap - 5) + assertEquals(List(0, 3, 1, 4, 2), anotherList) + val assignment = AdminUtils.assignReplicasToBrokers(toBrokerMetadata(rackMap), 7, 3, 0, 0) + val expected = Map(0 -> List(0, 3, 1), + 1 -> List(3, 1, 5), + 2 -> List(1, 5, 4), + 3 -> List(5, 4, 2), + 4 -> List(4, 2, 0), + 5 -> List(2, 0, 3), + 6 -> List(0, 4, 2)) + assertEquals(expected, assignment) + } + + @Test + def testAssignmentWithRackAware() { + val brokerRackMapping = Map(0 -> "rack1", 1 -> "rack2", 2 -> "rack2", 3 -> "rack3", 4 -> "rack3", 5 -> "rack1") + val numPartitions = 6 + val replicationFactor = 3 + val assignment = AdminUtils.assignReplicasToBrokers(toBrokerMetadata(brokerRackMapping), numPartitions, + replicationFactor, 2, 0) + checkReplicaDistribution(assignment, brokerRackMapping, brokerRackMapping.size, numPartitions, + replicationFactor) + } + + @Test + def testAssignmentWithRackAwareWithRandomStartIndex() { + val brokerRackMapping = Map(0 -> "rack1", 1 -> "rack2", 2 -> "rack2", 3 -> "rack3", 4 -> "rack3", 5 -> "rack1") + val numPartitions = 6 + val replicationFactor = 3 + val assignment = AdminUtils.assignReplicasToBrokers(toBrokerMetadata(brokerRackMapping), numPartitions, + replicationFactor) + checkReplicaDistribution(assignment, brokerRackMapping, brokerRackMapping.size, numPartitions, + replicationFactor) + } + + @Test + def testAssignmentWithRackAwareWithUnevenReplicas() { + val brokerRackMapping = Map(0 -> "rack1", 1 -> "rack2", 2 -> "rack2", 3 -> "rack3", 4 -> "rack3", 5 -> "rack1") + val numPartitions = 13 + val replicationFactor = 3 + val assignment = AdminUtils.assignReplicasToBrokers(toBrokerMetadata(brokerRackMapping), numPartitions, + replicationFactor, 0, 0) + checkReplicaDistribution(assignment, brokerRackMapping, brokerRackMapping.size, numPartitions, + replicationFactor, verifyLeaderDistribution = false, verifyReplicasDistribution = false) + } + + @Test + def testAssignmentWithRackAwareWithUnevenRacks() { + val brokerRackMapping = Map(0 -> "rack1", 1 -> "rack1", 2 -> "rack2", 3 -> "rack3", 4 -> "rack3", 5 -> "rack1") + val numPartitions = 12 + val replicationFactor = 3 + val assignment = AdminUtils.assignReplicasToBrokers(toBrokerMetadata(brokerRackMapping), numPartitions, + replicationFactor) + checkReplicaDistribution(assignment, brokerRackMapping, brokerRackMapping.size, numPartitions, + replicationFactor, verifyReplicasDistribution = false) + } + + @Test + def testAssignmentWith2ReplicasRackAware() { + val brokerRackMapping = Map(0 -> "rack1", 1 -> "rack2", 2 -> "rack2", 3 -> "rack3", 4 -> "rack3", 5 -> "rack1") + val numPartitions = 12 + val replicationFactor = 2 + val assignment = AdminUtils.assignReplicasToBrokers(toBrokerMetadata(brokerRackMapping), numPartitions, + replicationFactor) + checkReplicaDistribution(assignment, brokerRackMapping, brokerRackMapping.size, numPartitions, + replicationFactor) + } + + @Test + def testRackAwareExpansion() { + val brokerRackMapping = Map(6 -> "rack1", 7 -> "rack2", 8 -> "rack2", 9 -> "rack3", 10 -> "rack3", 11 -> "rack1") + val numPartitions = 12 + val replicationFactor = 2 + val assignment = AdminUtils.assignReplicasToBrokers(toBrokerMetadata(brokerRackMapping), numPartitions, + replicationFactor, startPartitionId = 12) + checkReplicaDistribution(assignment, brokerRackMapping, brokerRackMapping.size, numPartitions, + replicationFactor) + } + + @Test + def testAssignmentWith2ReplicasRackAwareWith6Partitions() { + val brokerRackMapping = Map(0 -> "rack1", 1 -> "rack2", 2 -> "rack2", 3 -> "rack3", 4 -> "rack3", 5 -> "rack1") + val numPartitions = 6 + val replicationFactor = 2 + val assignment = AdminUtils.assignReplicasToBrokers(toBrokerMetadata(brokerRackMapping), numPartitions, + replicationFactor) + checkReplicaDistribution(assignment, brokerRackMapping, brokerRackMapping.size, numPartitions, + replicationFactor) + } + + @Test + def testAssignmentWith2ReplicasRackAwareWith6PartitionsAnd3Brokers() { + val brokerRackMapping = Map(0 -> "rack1", 1 -> "rack2", 4 -> "rack3") + val numPartitions = 3 + val replicationFactor = 2 + val assignment = AdminUtils.assignReplicasToBrokers(toBrokerMetadata(brokerRackMapping), numPartitions, replicationFactor) + checkReplicaDistribution(assignment, brokerRackMapping, brokerRackMapping.size, numPartitions, replicationFactor) + } + + @Test + def testLargeNumberPartitionsAssignment() { + val numPartitions = 96 + val replicationFactor = 3 + val brokerRackMapping = Map(0 -> "rack1", 1 -> "rack2", 2 -> "rack2", 3 -> "rack3", 4 -> "rack3", 5 -> "rack1", + 6 -> "rack1", 7 -> "rack2", 8 -> "rack2", 9 -> "rack3", 10 -> "rack1", 11 -> "rack3") + val assignment = AdminUtils.assignReplicasToBrokers(toBrokerMetadata(brokerRackMapping), numPartitions, + replicationFactor) + checkReplicaDistribution(assignment, brokerRackMapping, brokerRackMapping.size, numPartitions, + replicationFactor) + } + + @Test + def testMoreReplicasThanRacks() { + val numPartitions = 6 + val replicationFactor = 5 + val brokerRackMapping = Map(0 -> "rack1", 1 -> "rack2", 2 -> "rack2", 3 -> "rack3", 4 -> "rack3", 5 -> "rack2") + val assignment = AdminUtils.assignReplicasToBrokers(toBrokerMetadata(brokerRackMapping), numPartitions, replicationFactor) + assertEquals(List.fill(assignment.size)(replicationFactor), assignment.values.map(_.size)) + val distribution = getReplicaDistribution(assignment, brokerRackMapping) + for (partition <- 0 until numPartitions) + assertEquals(3, distribution.partitionRacks(partition).toSet.size) + } + + @Test + def testLessReplicasThanRacks() { + val numPartitions = 6 + val replicationFactor = 2 + val brokerRackMapping = Map(0 -> "rack1", 1 -> "rack2", 2 -> "rack2", 3 -> "rack3", 4 -> "rack3", 5 -> "rack2") + val assignment = AdminUtils.assignReplicasToBrokers(toBrokerMetadata(brokerRackMapping), numPartitions, + replicationFactor) + assertEquals(List.fill(assignment.size)(replicationFactor), assignment.values.map(_.size)) + val distribution = getReplicaDistribution(assignment, brokerRackMapping) + for (partition <- 0 to 5) + assertEquals(2, distribution.partitionRacks(partition).toSet.size) + } + + @Test + def testSingleRack() { + val numPartitions = 6 + val replicationFactor = 3 + val brokerRackMapping = Map(0 -> "rack1", 1 -> "rack1", 2 -> "rack1", 3 -> "rack1", 4 -> "rack1", 5 -> "rack1") + val assignment = AdminUtils.assignReplicasToBrokers(toBrokerMetadata(brokerRackMapping), numPartitions, replicationFactor) + assertEquals(List.fill(assignment.size)(replicationFactor), assignment.values.map(_.size)) + val distribution = getReplicaDistribution(assignment, brokerRackMapping) + for (partition <- 0 until numPartitions) + assertEquals(1, distribution.partitionRacks(partition).toSet.size) + for (broker <- brokerRackMapping.keys) + assertEquals(1, distribution.brokerLeaderCount(broker)) + } + + @Test + def testSkipBrokerWithReplicaAlreadyAssigned() { + val rackInfo = Map(0 -> "a", 1 -> "b", 2 -> "c", 3 -> "a", 4 -> "a") + val brokerList = 0 to 4 + val numPartitions = 6 + val replicationFactor = 4 + val brokerMetadatas = toBrokerMetadata(rackInfo) + assertEquals(brokerList, brokerMetadatas.map(_.id)) + val assignment = AdminUtils.assignReplicasToBrokers(brokerMetadatas, numPartitions, replicationFactor, + fixedStartIndex = 2) + checkReplicaDistribution(assignment, rackInfo, 5, 6, 4, + verifyRackAware = false, verifyLeaderDistribution = false, verifyReplicasDistribution = false) + } +} diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala index 7c2577c992aa9..8910e096d8497 100755 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -33,20 +33,20 @@ import TestUtils._ import scala.collection.{Map, immutable} -class AdminTest extends ZooKeeperTestHarness with Logging { +class AdminTest extends ZooKeeperTestHarness with Logging with RackAwareTest { @Test def testReplicaAssignment() { - val brokerList = List(0, 1, 2, 3, 4) + val brokerMetadatas = (0 to 4).map(new BrokerMetadata(_, None)) // test 0 replication factor intercept[AdminOperationException] { - AdminUtils.assignReplicasToBrokers(brokerList, 10, 0) + AdminUtils.assignReplicasToBrokers(brokerMetadatas, 10, 0) } // test wrong replication factor intercept[AdminOperationException] { - AdminUtils.assignReplicasToBrokers(brokerList, 10, 6) + AdminUtils.assignReplicasToBrokers(brokerMetadatas, 10, 6) } // correct assignment @@ -62,9 +62,8 @@ class AdminTest extends ZooKeeperTestHarness with Logging { 8 -> List(3, 0, 1), 9 -> List(4, 1, 2)) - val actualAssignment = AdminUtils.assignReplicasToBrokers(brokerList, 10, 3, 0) - val e = (expectedAssignment.toList == actualAssignment.toList) - assertTrue(expectedAssignment.toList == actualAssignment.toList) + val actualAssignment = AdminUtils.assignReplicasToBrokers(brokerMetadatas, 10, 3, 0) + assertEquals(expectedAssignment, actualAssignment) } @Test @@ -314,7 +313,8 @@ class AdminTest extends ZooKeeperTestHarness with Logging { val partition = 1 val preferredReplica = 0 // create brokers - val serverConfigs = TestUtils.createBrokerConfigs(3, zkConnect, false).map(KafkaConfig.fromProps) + val brokerRack = Map(0 -> "rack0", 1 -> "rack1", 2 -> "rack2") + val serverConfigs = TestUtils.createBrokerConfigs(3, zkConnect, false, rackInfo = brokerRack).map(KafkaConfig.fromProps) // create the topic AdminUtils.createOrUpdateTopicPartitionAssignmentPathInZK(zkUtils, topic, expectedReplicaAssignment) val servers = serverConfigs.reverseMap(s => TestUtils.createServer(s)) @@ -452,4 +452,35 @@ class AdminTest extends ZooKeeperTestHarness with Logging { server.config.logDirs.foreach(CoreUtils.rm(_)) } } + + @Test + def testGetBrokerMetadatas() { + // broker 4 has no rack information + val brokerList = 0 to 5 + val rackInfo = Map(0 -> "rack1", 1 -> "rack2", 2 -> "rack2", 3 -> "rack1", 5 -> "rack3") + val brokerMetadatas = toBrokerMetadata(rackInfo, brokersWithoutRack = brokerList.filterNot(rackInfo.keySet)) + TestUtils.createBrokersInZk(brokerMetadatas, zkUtils) + + val processedMetadatas1 = AdminUtils.getBrokerMetadatas(zkUtils, RackAwareMode.Disabled) + assertEquals(brokerList, processedMetadatas1.map(_.id)) + assertEquals(List.fill(brokerList.size)(None), processedMetadatas1.map(_.rack)) + + val processedMetadatas2 = AdminUtils.getBrokerMetadatas(zkUtils, RackAwareMode.Safe) + assertEquals(brokerList, processedMetadatas2.map(_.id)) + assertEquals(List.fill(brokerList.size)(None), processedMetadatas2.map(_.rack)) + + intercept[AdminOperationException] { + AdminUtils.getBrokerMetadatas(zkUtils, RackAwareMode.Enforced) + } + + val partialList = List(0, 1, 2, 3, 5) + val processedMetadatas3 = AdminUtils.getBrokerMetadatas(zkUtils, RackAwareMode.Enforced, Some(partialList)) + assertEquals(partialList, processedMetadatas3.map(_.id)) + assertEquals(partialList.map(rackInfo), processedMetadatas3.flatMap(_.rack)) + + val numPartitions = 3 + AdminUtils.createTopic(zkUtils, "foo", numPartitions, 2, rackAwareMode = RackAwareMode.Safe) + val assignment = zkUtils.getReplicaAssignmentForTopics(Seq("foo")) + assertEquals(numPartitions, assignment.size) + } } diff --git a/core/src/test/scala/unit/kafka/admin/RackAwareTest.scala b/core/src/test/scala/unit/kafka/admin/RackAwareTest.scala new file mode 100644 index 0000000000000..facc7458333dc --- /dev/null +++ b/core/src/test/scala/unit/kafka/admin/RackAwareTest.scala @@ -0,0 +1,82 @@ +/** + * 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.admin + +import scala.collection.{Map, Seq, mutable} +import org.junit.Assert._ + +trait RackAwareTest { + + def checkReplicaDistribution(assignment: Map[Int, Seq[Int]], + brokerRackMapping: Map[Int, String], + numBrokers: Int, + numPartitions: Int, + replicationFactor: Int, + verifyRackAware: Boolean = true, + verifyLeaderDistribution: Boolean = true, + verifyReplicasDistribution: Boolean = true) { + // always verify that no broker will be assigned for more than one replica + for ((_, brokerList) <- assignment) { + assertEquals("More than one replica is assigned to same broker for the same partition", brokerList.toSet.size, brokerList.size) + } + val distribution = getReplicaDistribution(assignment, brokerRackMapping) + + if (verifyRackAware) { + val partitionRackMap = distribution.partitionRacks + assertEquals("More than one replica of the same partition is assigned to the same rack", + List.fill(numPartitions)(replicationFactor), partitionRackMap.values.toList.map(_.distinct.size)) + } + + if (verifyLeaderDistribution) { + val leaderCount = distribution.brokerLeaderCount + val leaderCountPerBroker = numPartitions / numBrokers + assertEquals("Preferred leader count is not even for brokers", List.fill(numBrokers)(leaderCountPerBroker), leaderCount.values.toList) + } + + if (verifyReplicasDistribution) { + val replicasCount = distribution.brokerReplicasCount + val numReplicasPerBroker = numPartitions * replicationFactor / numBrokers + assertEquals("Replica count is not even for broker", List.fill(numBrokers)(numReplicasPerBroker), replicasCount.values.toList) + } + } + + def getReplicaDistribution(assignment: Map[Int, Seq[Int]], brokerRackMapping: Map[Int, String]): ReplicaDistributions = { + val leaderCount = mutable.Map[Int, Int]() + val partitionCount = mutable.Map[Int, Int]() + val partitionRackMap = mutable.Map[Int, List[String]]() + assignment.foreach { case (partitionId, replicaList) => + val leader = replicaList.head + leaderCount(leader) = leaderCount.getOrElse(leader, 0) + 1 + for (brokerId <- replicaList) { + partitionCount(brokerId) = partitionCount.getOrElse(brokerId, 0) + 1 + val rack = brokerRackMapping.getOrElse(brokerId, sys.error(s"No mapping found for $brokerId in `brokerRackMapping`")) + partitionRackMap(partitionId) = rack :: partitionRackMap.getOrElse(partitionId, List()) + } + } + ReplicaDistributions(partitionRackMap, leaderCount, partitionCount) + } + + def toBrokerMetadata(rackMap: Map[Int, String], brokersWithoutRack: Seq[Int] = Seq.empty): Seq[BrokerMetadata] = + rackMap.toSeq.map { case (brokerId, rack) => + BrokerMetadata(brokerId, Some(rack)) + } ++ brokersWithoutRack.map { brokerId => + BrokerMetadata(brokerId, None) + }.sortBy(_.id) + +} + +case class ReplicaDistributions(partitionRacks: Map[Int, Seq[String]], brokerLeaderCount: Map[Int, Int], brokerReplicasCount: Map[Int, Int]) diff --git a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsCommandTest.scala b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsCommandTest.scala new file mode 100644 index 0000000000000..0f71a19fcd5dd --- /dev/null +++ b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsCommandTest.scala @@ -0,0 +1,51 @@ +/** + * 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.admin + +import kafka.utils.{Logging, TestUtils} +import kafka.zk.ZooKeeperTestHarness +import org.junit.Test + +class ReassignPartitionsCommandTest extends ZooKeeperTestHarness with Logging with RackAwareTest { + + @Test + def testRackAwareReassign() { + val rackInfo = Map(0 -> "rack1", 1 -> "rack2", 2 -> "rack2", 3 -> "rack1", 4 -> "rack3", 5 -> "rack3") + TestUtils.createBrokersInZk(toBrokerMetadata(rackInfo), zkUtils) + + val numPartitions = 18 + val replicationFactor = 3 + + // create a non rack aware assignment topic first + val createOpts = new kafka.admin.TopicCommand.TopicCommandOptions(Array( + "--partitions", numPartitions.toString, + "--replication-factor", replicationFactor.toString, + "--disable-rack-aware", + "--topic", "foo")) + kafka.admin.TopicCommand.createTopic(zkUtils, createOpts) + + val topicJson = """{"topics": [{"topic": "foo"}], "version":1}""" + val (proposedAssignment, currentAssignment) = ReassignPartitionsCommand.generateAssignment(zkUtils, + rackInfo.keys.toSeq.sorted, topicJson, disableRackAware = false) + + val assignment = proposedAssignment map { case (topicPartition, replicas) => + (topicPartition.partition, replicas) + } + checkReplicaDistribution(assignment, rackInfo, rackInfo.size, numPartitions, replicationFactor) + } + +} diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala index d554b02809d86..b42aaf4d2ef99 100644 --- a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala @@ -27,7 +27,7 @@ import kafka.admin.TopicCommand.TopicCommandOptions import kafka.utils.ZkUtils._ import kafka.coordinator.GroupCoordinator -class TopicCommandTest extends ZooKeeperTestHarness with Logging { +class TopicCommandTest extends ZooKeeperTestHarness with Logging with RackAwareTest { @Test def testConfigPreservationAcrossPartitionAlteration() { @@ -157,4 +157,34 @@ class TopicCommandTest extends ZooKeeperTestHarness with Logging { Array("--partitions", numPartitions.toString, "--replication-factor", "1", "--topic", topic, "--if-not-exists")) TopicCommand.createTopic(zkUtils, createNotExistsOpts) } + + @Test + def testCreateAlterTopicWithRackAware() { + val rackInfo = Map(0 -> "rack1", 1 -> "rack2", 2 -> "rack2", 3 -> "rack1", 4 -> "rack3", 5 -> "rack3") + TestUtils.createBrokersInZk(toBrokerMetadata(rackInfo), zkUtils) + + val numPartitions = 18 + val replicationFactor = 3 + val createOpts = new TopicCommandOptions(Array( + "--partitions", numPartitions.toString, + "--replication-factor", replicationFactor.toString, + "--topic", "foo")) + TopicCommand.createTopic(zkUtils, createOpts) + + var assignment = zkUtils.getReplicaAssignmentForTopics(Seq("foo")).map { case (tp, replicas) => + tp.partition -> replicas + } + checkReplicaDistribution(assignment, rackInfo, rackInfo.size, numPartitions, replicationFactor) + + val alteredNumPartitions = 36 + // verify that adding partitions will also be rack aware + val alterOpts = new TopicCommandOptions(Array( + "--partitions", alteredNumPartitions.toString, + "--topic", "foo")) + TopicCommand.alterTopic(zkUtils, alterOpts) + assignment = zkUtils.getReplicaAssignmentForTopics(Seq("foo")).map { case (tp, replicas) => + tp.partition -> replicas + } + checkReplicaDistribution(assignment, rackInfo, rackInfo.size, alteredNumPartitions, replicationFactor) + } } diff --git a/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala b/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala index 905612cb801af..400d6d6d67a29 100644 --- a/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala +++ b/core/src/test/scala/unit/kafka/cluster/BrokerEndPointTest.scala @@ -27,20 +27,6 @@ import scala.collection.mutable class BrokerEndPointTest extends Logging { - @Test - def testSerDe() { - - val endpoint = new EndPoint("myhost", 9092, SecurityProtocol.PLAINTEXT) - val listEndPoints = Map(SecurityProtocol.PLAINTEXT -> endpoint) - val origBroker = new Broker(1, listEndPoints) - val brokerBytes = ByteBuffer.allocate(origBroker.sizeInBytes) - - origBroker.writeTo(brokerBytes) - - val newBroker = Broker.readFrom(brokerBytes.flip().asInstanceOf[ByteBuffer]) - assert(origBroker == newBroker) - } - @Test def testHashAndEquals() { val endpoint1 = new EndPoint("myhost", 9092, SecurityProtocol.PLAINTEXT) diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index 7524e6a6f8726..fa240d2efd1b8 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -530,7 +530,7 @@ class KafkaConfigTest { case KafkaConfig.MetricNumSamplesProp => assertPropertyInvalid(getBaseProperties, name, "not_a_number", "-1", "0") case KafkaConfig.MetricSampleWindowMsProp => assertPropertyInvalid(getBaseProperties, name, "not_a_number", "-1", "0") case KafkaConfig.MetricReporterClassesProp => // ignore string - + case KafkaConfig.RackProp => // ignore string //SSL Configs case KafkaConfig.PrincipalBuilderClassProp => case KafkaConfig.SslProtocolProp => // ignore string diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 252308304d445..49fb85fb11c33 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -27,29 +27,27 @@ import java.security.cert.X509Certificate import javax.net.ssl.X509TrustManager import charset.Charset -import kafka.security.auth.{Resource, Authorizer, Acl} +import kafka.security.auth.{Acl, Authorizer, Resource} import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.kafka.common.utils.Utils._ import org.apache.kafka.test.TestSslUtils import scala.collection.mutable.{ArrayBuffer, ListBuffer} - import kafka.server._ import kafka.producer._ import kafka.message._ import kafka.api._ -import kafka.cluster.Broker -import kafka.consumer.{ConsumerTimeoutException, KafkaStream, ConsumerConfig} -import kafka.serializer.{StringEncoder, DefaultEncoder, Encoder} +import kafka.cluster.{Broker, EndPoint} +import kafka.consumer.{ConsumerConfig, ConsumerTimeoutException, KafkaStream} +import kafka.serializer.{DefaultEncoder, Encoder, StringEncoder} import kafka.common.TopicAndPartition import kafka.admin.AdminUtils import kafka.producer.ProducerConfig import kafka.log._ import kafka.utils.ZkUtils._ - import org.junit.Assert._ import org.apache.kafka.clients.producer.KafkaProducer -import org.apache.kafka.clients.consumer.{RangeAssignor, KafkaConsumer} +import org.apache.kafka.clients.consumer.{KafkaConsumer, RangeAssignor} import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.common.network.Mode @@ -154,11 +152,12 @@ object TestUtils extends Logging { enablePlaintext: Boolean = true, enableSsl: Boolean = false, enableSaslPlaintext: Boolean = false, - enableSaslSsl: Boolean = false): Seq[Properties] = { + enableSaslSsl: Boolean = false, + rackInfo: Map[Int, String] = Map()): Seq[Properties] = { (0 until numConfigs).map { node => createBrokerConfig(node, zkConnect, enableControlledShutdown, enableDeleteTopic, RandomPort, interBrokerSecurityProtocol, trustStoreFile, enablePlaintext = enablePlaintext, enableSsl = enableSsl, - enableSaslPlaintext = enableSaslPlaintext, enableSaslSsl = enableSaslSsl) + enableSaslPlaintext = enableSaslPlaintext, enableSaslSsl = enableSaslSsl, rack = rackInfo.get(node)) } } @@ -180,7 +179,7 @@ object TestUtils extends Logging { enablePlaintext: Boolean = true, enableSaslPlaintext: Boolean = false, saslPlaintextPort: Int = RandomPort, enableSsl: Boolean = false, sslPort: Int = RandomPort, - enableSaslSsl: Boolean = false, saslSslPort: Int = RandomPort) + enableSaslSsl: Boolean = false, saslSslPort: Int = RandomPort, rack: Option[String] = None) : Properties = { def shouldEnable(protocol: SecurityProtocol) = interBrokerSecurityProtocol.fold(false)(_ == protocol) @@ -210,6 +209,7 @@ object TestUtils extends Logging { props.put("delete.topic.enable", enableDeleteTopic.toString) props.put("controlled.shutdown.retry.backoff.ms", "100") props.put("log.cleaner.dedupe.buffer.size", "2097152") + rack.foreach(props.put("broker.rack", _)) if (protocolAndPorts.exists { case (protocol, _) => usesSslTransportLayer(protocol) }) props.putAll(sslConfigs(Mode.SERVER, false, trustStoreFile, s"server$nodeId")) @@ -591,9 +591,16 @@ object TestUtils extends Logging { } } - def createBrokersInZk(zkUtils: ZkUtils, ids: Seq[Int]): Seq[Broker] = { - val brokers = ids.map(id => new Broker(id, "localhost", 6667, SecurityProtocol.PLAINTEXT)) - brokers.foreach(b => zkUtils.registerBrokerInZk(b.id, "localhost", 6667, b.endPoints, jmxPort = -1)) + def createBrokersInZk(zkUtils: ZkUtils, ids: Seq[Int]): Seq[Broker] = + createBrokersInZk(ids.map(kafka.admin.BrokerMetadata(_, None)), zkUtils) + + def createBrokersInZk(brokerMetadatas: Seq[kafka.admin.BrokerMetadata], zkUtils: ZkUtils): Seq[Broker] = { + val brokers = brokerMetadatas.map { b => + val protocol = SecurityProtocol.PLAINTEXT + Broker(b.id, Map(protocol -> EndPoint("localhost", 6667, protocol)).toMap, b.rack) + } + brokers.foreach(b => zkUtils.registerBrokerInZk(b.id, "localhost", 6667, b.endPoints, jmxPort = -1, + rack = b.rack, ApiVersion.latestVersion)) brokers } diff --git a/docs/upgrade.html b/docs/upgrade.html index 15ea3ae045cf1..ba3d0248718ae 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -21,6 +21,11 @@

      Upgrading from 0.8.x or 0.9.x to 0.10. 0.10.0.0 has potential breaking changes (please review before upgrading) and there may be a performance impact during the upgrade. Because new protocols are introduced, it is important to upgrade your Kafka clusters before upgrading your clients. +

      +Notes to clients with version 0.9.0.0: Due to a bug introduced in 0.9.0.0, +clients that depend on ZooKeeper (old Scala high-level Consumer and MirrorMaker if used with the old consumer) will not +work with 0.10.0.x brokers. Therefore, 0.9.0.0 clients should be upgraded to 0.9.0.1 before brokers are upgraded to +0.10.0.x. This step is not necessary for 0.8.X or 0.9.0.1 clients.

      For a rolling upgrade:

      From 355076cd262dd071287a1c8586ae0f9635e218e3 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 15 Mar 2016 12:08:26 -0700 Subject: [PATCH 031/206] MINOR: kstream/ktable counting method with default long serdes guozhangwang miguno Author: Yasuhiro Matsuda Reviewers: Michael G. Noll, Guozhang Wang Closes #1065 from ymatsuda/count_serdes --- .../examples/pageview/PageViewTypedJob.java | 3 +-- .../examples/pageview/PageViewUntypedJob.java | 3 +-- .../examples/wordcount/WordCountJob.java | 3 +-- .../apache/kafka/streams/kstream/KStream.java | 9 ++------ .../apache/kafka/streams/kstream/KTable.java | 2 -- .../kstream/internals/KStreamImpl.java | 21 +++++++++---------- .../streams/kstream/internals/KTableImpl.java | 9 ++++---- .../streams/smoketest/SmokeTestClient.java | 8 ++----- 8 files changed, 22 insertions(+), 36 deletions(-) diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedJob.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedJob.java index 6a105fdb9c625..1fcb4039b1af6 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedJob.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedJob.java @@ -151,8 +151,7 @@ public KeyValue apply(String user, PageViewByRegion vi } }) .countByKey(HoppingWindows.of("GeoPageViewsWindow").with(7 * 24 * 60 * 60 * 1000), - stringSerializer, longSerializer, - stringDeserializer, longDeserializer) + stringSerializer, stringDeserializer) // TODO: we can merge ths toStream().map(...) with a single toStream(...) .toStream() .map(new KeyValueMapper, Long, KeyValue>() { diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedJob.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedJob.java index e890589577156..fb1a55dc8bfad 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedJob.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedJob.java @@ -108,8 +108,7 @@ public KeyValue apply(String user, JsonNode viewRegion) { } }) .countByKey(HoppingWindows.of("GeoPageViewsWindow").with(7 * 24 * 60 * 60 * 1000), - stringSerializer, longSerializer, - stringDeserializer, longDeserializer) + stringSerializer, stringDeserializer) // TODO: we can merge ths toStream().map(...) with a single toStream(...) .toStream() .map(new KeyValueMapper, Long, KeyValue>() { diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountJob.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountJob.java index 82d216efb690f..d1f8d8647a58d 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountJob.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountJob.java @@ -66,7 +66,6 @@ public static void main(String[] args) throws Exception { final Serializer stringSerializer = new StringSerializer(); final Deserializer stringDeserializer = new StringDeserializer(); final Serializer longSerializer = new LongSerializer(); - final Deserializer longDeserializer = new LongDeserializer(); KStream source = builder.stream("streams-file-input"); @@ -82,7 +81,7 @@ public KeyValue apply(String key, String value) { return new KeyValue(value, value); } }) - .countByKey(stringSerializer, longSerializer, stringDeserializer, longDeserializer, "Counts"); + .countByKey(stringSerializer, stringDeserializer, "Counts"); counts.to("streams-wordcount-output", stringSerializer, longSerializer); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index 6426af9d9153b..1640bdec000c0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -329,18 +329,13 @@ KTable aggregateByKey(Initializer initializer, */ KTable, Long> countByKey(Windows windows, Serializer keySerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer aggValueDeserializer); + Deserializer keyDeserializer); /** * Count number of messages of this stream by key without a window basis, and hence - * return a ever updating counting table - * + * return a ever updating counting table. */ KTable countByKey(Serializer keySerializer, - Serializer aggValueSerializer, Deserializer keyDeserializer, - Deserializer aggValueDeserializer, String name); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index 485bb20624ded..b44ed21cf8a84 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -193,10 +193,8 @@ KTable aggregate(Initializer initializer, KTable count(KeyValueMapper selector, Serializer keySerializer, Serializer valueSerializer, - Serializer aggValueSerializer, Deserializer keyDeserializer, Deserializer valueDeserializer, - Deserializer aggValueDeserializer, String name); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index cff082cc25dad..884933b259b6b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -18,6 +18,8 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.LongDeserializer; +import org.apache.kafka.common.serialization.LongSerializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Initializer; @@ -91,6 +93,8 @@ public class KStreamImpl extends AbstractStream implements KStream sourceNodes) { super(topology, name, sourceNodes); @@ -501,9 +505,7 @@ public KTable aggregateByKey(Initializer initializer, @Override public KTable, Long> countByKey(Windows windows, Serializer keySerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer aggValueDeserializer) { + Deserializer keyDeserializer) { return this.aggregateByKey( new Initializer() { @Override @@ -516,16 +518,13 @@ public Long apply() { public Long apply(K aggKey, V value, Long aggregate) { return aggregate + 1L; } - }, windows, keySerializer, aggValueSerializer, keyDeserializer, aggValueDeserializer); + }, windows, keySerializer, LONG_SERIALIZER, keyDeserializer, LONG_DESERIALIZER); } @Override - public KTable countByKey(Serializer keySerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer aggValueDeserializer, - String name) { - + public KTable countByKey(Serializer keySerializer, + Deserializer keyDeserializer, + String name) { return this.aggregateByKey( new Initializer() { @Override @@ -538,6 +537,6 @@ public Long apply() { public Long apply(K aggKey, V value, Long aggregate) { return aggregate + 1L; } - }, keySerializer, aggValueSerializer, keyDeserializer, aggValueDeserializer, name); + }, keySerializer, LONG_SERIALIZER, keyDeserializer, LONG_DESERIALIZER, name); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index b82582bf1724e..d63fcc82bde5f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -18,6 +18,8 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.LongDeserializer; +import org.apache.kafka.common.serialization.LongSerializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Initializer; @@ -75,6 +77,8 @@ public class KTableImpl extends AbstractStream implements KTable processorSupplier; @@ -302,12 +306,9 @@ public KTable aggregate(Initializer initializer, public KTable count(final KeyValueMapper selector, Serializer keySerializer, Serializer valueSerializer, - Serializer aggValueSerializer, Deserializer keyDeserializer, Deserializer valueDeserializer, - Deserializer aggValueDeserializer, String name) { - return this.aggregate( new Initializer() { @Override @@ -331,7 +332,7 @@ public KeyValue apply(K key, V value) { return new KeyValue<>(selector.apply(key, value), value); } }, - keySerializer, valueSerializer, aggValueSerializer, keyDeserializer, valueDeserializer, aggValueDeserializer, name); + keySerializer, valueSerializer, LONG_SERIALIZER, keyDeserializer, valueDeserializer, LONG_DESERIALIZER, name); } @Override diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java index fec447fca690f..6cb45f33490bb 100644 --- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java +++ b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java @@ -191,9 +191,7 @@ public Long apply(String aggKey, Integer value, Long aggregate) { data.countByKey( UnlimitedWindows.of("uwin-cnt"), stringSerializer, - longSerializer, - stringDeserializer, - longDeserializer + stringDeserializer ).toStream().map( new Unwindow() ).to("cnt", stringSerializer, longSerializer); @@ -224,9 +222,7 @@ public Double apply(Long value1, Long value2) { data.countByKey( TumblingWindows.of("tumbling-win-cnt").with(WINDOW_SIZE), stringSerializer, - longSerializer, - stringDeserializer, - longDeserializer + stringDeserializer ).toStream().map( new KeyValueMapper, Long, KeyValue>() { @Override From 90c623d10f1c081e41f151eb28524f0fb57609c7 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 15 Mar 2016 12:15:15 -0700 Subject: [PATCH 032/206] MINOR: Add test that verifies fix for KAFKA-3047 Also clean-up `LogTest` a little. Author: Ismael Juma Reviewers: Guozhang Wang Closes #1071 from ijuma/kafka-3047-explicit-offset-assignment-corrupt-log-test --- .../test/scala/unit/kafka/log/LogTest.scala | 55 ++++++++++++------- 1 file changed, 34 insertions(+), 21 deletions(-) diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 5446eec51c42a..c2eb8179fbe20 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -321,7 +321,7 @@ class LogTest extends JUnitSuite { for(i <- 0 until messagesToAppend) log.append(TestUtils.singleMessageSet(i.toString.getBytes)) - var currOffset = log.logEndOffset + val currOffset = log.logEndOffset assertEquals(currOffset, messagesToAppend) // time goes by; the log file is deleted @@ -765,6 +765,19 @@ class LogTest extends JUnitSuite { assertTrue("Message payload should be null.", messageSet.head.message.isNull) } + @Test(expected = classOf[IllegalArgumentException]) + def testAppendWithOutOfOrderOffsetsThrowsException() { + val log = new Log(logDir, + LogConfig(), + recoveryPoint = 0L, + time.scheduler, + time) + val messages = (0 until 2).map(id => new Message(id.toString.getBytes)).toArray + messages.foreach(message => log.append(new ByteBufferMessageSet(message))) + val invalidMessage = new ByteBufferMessageSet(new Message(1.toString.getBytes)) + log.append(invalidMessage, assignOffsets = false) + } + @Test def testCorruptLog() { // append some messages to create some segments @@ -836,9 +849,9 @@ class LogTest extends JUnitSuite { @Test def testParseTopicPartitionName() { - val topic: String = "test_topic" - val partition:String = "143" - val dir: File = new File(logDir + topicPartitionName(topic, partition)) + val topic = "test_topic" + val partition = "143" + val dir = new File(logDir + topicPartitionName(topic, partition)) val topicAndPartition = Log.parseTopicPartitionName(dir) assertEquals(topic, topicAndPartition.asTuple._1) assertEquals(partition.toInt, topicAndPartition.asTuple._2) @@ -847,8 +860,8 @@ class LogTest extends JUnitSuite { @Test def testParseTopicPartitionNameForEmptyName() { try { - val dir: File = new File("") - val topicAndPartition = Log.parseTopicPartitionName(dir) + val dir = new File("") + Log.parseTopicPartitionName(dir) fail("KafkaException should have been thrown for dir: " + dir.getCanonicalPath) } catch { case e: Exception => // its GOOD! @@ -859,7 +872,7 @@ class LogTest extends JUnitSuite { def testParseTopicPartitionNameForNull() { try { val dir: File = null - val topicAndPartition = Log.parseTopicPartitionName(dir) + Log.parseTopicPartitionName(dir) fail("KafkaException should have been thrown for dir: " + dir) } catch { case e: Exception => // its GOOD! @@ -868,11 +881,11 @@ class LogTest extends JUnitSuite { @Test def testParseTopicPartitionNameForMissingSeparator() { - val topic: String = "test_topic" - val partition:String = "1999" - val dir: File = new File(logDir + File.separator + topic + partition) + val topic = "test_topic" + val partition = "1999" + val dir = new File(logDir + File.separator + topic + partition) try { - val topicAndPartition = Log.parseTopicPartitionName(dir) + Log.parseTopicPartitionName(dir) fail("KafkaException should have been thrown for dir: " + dir.getCanonicalPath) } catch { case e: Exception => // its GOOD! @@ -881,11 +894,11 @@ class LogTest extends JUnitSuite { @Test def testParseTopicPartitionNameForMissingTopic() { - val topic: String = "" - val partition:String = "1999" - val dir: File = new File(logDir + topicPartitionName(topic, partition)) + val topic = "" + val partition = "1999" + val dir = new File(logDir + topicPartitionName(topic, partition)) try { - val topicAndPartition = Log.parseTopicPartitionName(dir) + Log.parseTopicPartitionName(dir) fail("KafkaException should have been thrown for dir: " + dir.getCanonicalPath) } catch { case e: Exception => // its GOOD! @@ -894,18 +907,18 @@ class LogTest extends JUnitSuite { @Test def testParseTopicPartitionNameForMissingPartition() { - val topic: String = "test_topic" - val partition:String = "" - val dir: File = new File(logDir + topicPartitionName(topic, partition)) + val topic = "test_topic" + val partition = "" + val dir = new File(logDir + topicPartitionName(topic, partition)) try { - val topicAndPartition = Log.parseTopicPartitionName(dir) + Log.parseTopicPartitionName(dir) fail("KafkaException should have been thrown for dir: " + dir.getCanonicalPath) } catch { case e: Exception => // its GOOD! } } - def topicPartitionName(topic: String, partition: String): String = { + def topicPartitionName(topic: String, partition: String): String = File.separator + topic + "-" + partition - } + } From fd6efbe0b7f18beb506f112350b30b7209a2f5cc Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 15 Mar 2016 13:37:33 -0700 Subject: [PATCH 033/206] MINOR: Remove unused import in `WordCountJob` to fix checkstyle failure Author: Ismael Juma Reviewers: Ewen Cheslack-Postava Closes #1077 from ijuma/fix-streams-checkstyle-failure --- .../apache/kafka/streams/examples/wordcount/WordCountJob.java | 1 - 1 file changed, 1 deletion(-) diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountJob.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountJob.java index d1f8d8647a58d..2b51a4401569c 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountJob.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountJob.java @@ -18,7 +18,6 @@ package org.apache.kafka.streams.examples.wordcount; import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.LongDeserializer; import org.apache.kafka.common.serialization.LongSerializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.serialization.StringSerializer; From 6eacc0de303e4d29e083b89c1f53615c1dfa291e Mon Sep 17 00:00:00 2001 From: Jeremy Custenborder Date: Tue, 15 Mar 2016 14:32:22 -0700 Subject: [PATCH 034/206] KAFKA-3260 - Added SourceTask.commitRecord Added commitRecord(SourceRecord record) to SourceTask. This method is called during the callback from producer.send() when the message has been sent successfully. Added commitTaskRecord(SourceRecord record) to WorkerSourceTask to handle calling commitRecord on the SourceTask. Updated tests for calls to commitRecord. Author: Jeremy Custenborder Reviewers: Jason Gustafson , Ewen Cheslack-Postava Closes #950 from jcustenborder/KAFKA-3260 --- .../apache/kafka/connect/source/SourceTask.java | 16 ++++++++++++++++ .../kafka/connect/runtime/WorkerSourceTask.java | 9 +++++++++ .../connect/runtime/WorkerSourceTaskTest.java | 17 +++++++++++++++++ 3 files changed, 42 insertions(+) diff --git a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java index 802fcdd6bbec2..c508ec1dd5382 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/source/SourceTask.java @@ -79,4 +79,20 @@ public void commit() throws InterruptedException { * {@link java.nio.channels.Selector#wakeup() wakeup()} to interrupt any ongoing requests. */ public abstract void stop(); + + /** + *

      + * Commit an individual {@link SourceRecord} when the callback from the producer client is received. + *

      + *

      + * SourceTasks are not required to implement this functionality; Kafka Connect will record offsets + * automatically. This hook is provided for systems that also need to store offsets internally + * in their own system. + *

      + * @param record {@link SourceRecord} that was successfully sent via the producer. + * @throws InterruptedException + */ + public void commitRecord(SourceRecord record) throws InterruptedException { + // This space intentionally left blank. + } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java index 0014be8e26b0c..3a43f968a102a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java @@ -207,6 +207,7 @@ public void onCompletion(RecordMetadata recordMetadata, Exception e) { log.trace("Wrote record successfully: topic {} partition {} offset {}", recordMetadata.topic(), recordMetadata.partition(), recordMetadata.offset()); + commitTaskRecord(record); } recordSent(producerRecord); } @@ -226,6 +227,14 @@ public void onCompletion(RecordMetadata recordMetadata, Exception e) { return true; } + private void commitTaskRecord(SourceRecord record) { + try { + task.commitRecord(record); + } catch (InterruptedException e) { + log.error("Exception thrown", e); + } + } + private synchronized void recordSent(final ProducerRecord record) { ProducerRecord removed = outstandingMessages.remove(record); // While flushing, we may also see callbacks for items in the backlog diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java index 9b0133a6fb14b..ece298569e07e 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java @@ -133,6 +133,8 @@ public void testPollsInBackground() throws Exception { final CountDownLatch pollLatch = expectPolls(10); // In this test, we don't flush, so nothing goes any further than the offset writer + expectCommitRecord(10); + sourceTask.stop(); EasyMock.expectLastCall(); expectOffsetFlush(true); @@ -203,10 +205,13 @@ public void testCommit() throws Exception { sourceTask.stop(); EasyMock.expectLastCall(); expectOffsetFlush(true); + + expectCommitRecord(1); statusListener.onShutdown(taskId); EasyMock.expectLastCall(); + PowerMock.replayAll(); workerTask.initialize(EMPTY_TASK_PROPS); @@ -233,6 +238,7 @@ public void testCommitFailure() throws Exception { // We'll wait for some data, then trigger a flush final CountDownLatch pollLatch = expectPolls(1); + expectCommitRecord(1); expectOffsetFlush(true); sourceTask.stop(); @@ -254,6 +260,13 @@ public void testCommitFailure() throws Exception { PowerMock.verifyAll(); } + private void expectCommitRecord(int count) throws Exception { + for (int i = 0; i < count; i++) { + sourceTask.commitRecord(EasyMock.anyObject(SourceRecord.class)); + EasyMock.expectLastCall(); + } + } + @Test public void testSendRecordsConvertsData() throws Exception { createWorkerTask(); @@ -264,6 +277,8 @@ public void testSendRecordsConvertsData() throws Exception { Capture> sent = expectSendRecordAnyTimes(); + expectCommitRecord(records.size()); + PowerMock.replayAll(); Whitebox.setInternalState(workerTask, "toSend", records); @@ -292,6 +307,8 @@ public void testSendRecordsRetries() throws Exception { expectSendRecordOnce(true); expectSendRecordOnce(false); + expectCommitRecord(3); + PowerMock.replayAll(); // Try to send 3, make first pass, second fail. Should save last two From 6834b91e349f2ed4b8ff5c97eb3b0c390bc27648 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Wed, 16 Mar 2016 08:24:18 -0700 Subject: [PATCH 035/206] MINOR: KAFKA-3260 follow up, fix commitRecord calls in tests Author: Ewen Cheslack-Postava Reviewers: Guozhang Wang Closes #1080 from ewencp/minor-kafka-3260-followup-tests --- .../connect/runtime/WorkerSourceTaskTest.java | 25 ++++++------------- 1 file changed, 8 insertions(+), 17 deletions(-) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java index ece298569e07e..3dd07a699fe9c 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java @@ -133,8 +133,6 @@ public void testPollsInBackground() throws Exception { final CountDownLatch pollLatch = expectPolls(10); // In this test, we don't flush, so nothing goes any further than the offset writer - expectCommitRecord(10); - sourceTask.stop(); EasyMock.expectLastCall(); expectOffsetFlush(true); @@ -205,8 +203,6 @@ public void testCommit() throws Exception { sourceTask.stop(); EasyMock.expectLastCall(); expectOffsetFlush(true); - - expectCommitRecord(1); statusListener.onShutdown(taskId); EasyMock.expectLastCall(); @@ -238,7 +234,6 @@ public void testCommitFailure() throws Exception { // We'll wait for some data, then trigger a flush final CountDownLatch pollLatch = expectPolls(1); - expectCommitRecord(1); expectOffsetFlush(true); sourceTask.stop(); @@ -259,14 +254,7 @@ public void testCommitFailure() throws Exception { PowerMock.verifyAll(); } - - private void expectCommitRecord(int count) throws Exception { - for (int i = 0; i < count; i++) { - sourceTask.commitRecord(EasyMock.anyObject(SourceRecord.class)); - EasyMock.expectLastCall(); - } - } - + @Test public void testSendRecordsConvertsData() throws Exception { createWorkerTask(); @@ -277,8 +265,6 @@ public void testSendRecordsConvertsData() throws Exception { Capture> sent = expectSendRecordAnyTimes(); - expectCommitRecord(records.size()); - PowerMock.replayAll(); Whitebox.setInternalState(workerTask, "toSend", records); @@ -307,8 +293,6 @@ public void testSendRecordsRetries() throws Exception { expectSendRecordOnce(true); expectSendRecordOnce(false); - expectCommitRecord(3); - PowerMock.replayAll(); // Try to send 3, make first pass, second fail. Should save last two @@ -439,6 +423,13 @@ public Future answer() throws Throwable { else expect.andAnswer(expectResponse); + // 3. As a result of a successful producer send callback, we'll notify the source task of the record commit + sourceTask.commitRecord(EasyMock.anyObject(SourceRecord.class)); + if (anyTimes) + EasyMock.expectLastCall().anyTimes(); + else + EasyMock.expectLastCall(); + return sent; } From b0f811804fb818792dfc55298bd11eacd19dec5c Mon Sep 17 00:00:00 2001 From: Eno Thereska Date: Wed, 16 Mar 2016 08:47:38 -0700 Subject: [PATCH 036/206] KAFKA-3371: ClientCompatibilityTest system test failing becketqin have a look if this looks reasonable to you. Thanks. Author: Eno Thereska Reviewers: Geoff Anderson , Ewen Cheslack-Postava Closes #1051 from enothereska/kafka-3371 --- tests/kafkatest/tests/compatibility_test.py | 102 -------------------- 1 file changed, 102 deletions(-) delete mode 100644 tests/kafkatest/tests/compatibility_test.py diff --git a/tests/kafkatest/tests/compatibility_test.py b/tests/kafkatest/tests/compatibility_test.py deleted file mode 100644 index bc00b294679a2..0000000000000 --- a/tests/kafkatest/tests/compatibility_test.py +++ /dev/null @@ -1,102 +0,0 @@ -# Copyright 2015 Confluent Inc. -# -# Licensed 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. - -from ducktape.tests.test import Test - -from kafkatest.services.zookeeper import ZookeeperService -from kafkatest.services.kafka import KafkaService -from kafkatest.services.kafka.version import LATEST_0_8_2, TRUNK -from kafkatest.services.verifiable_producer import VerifiableProducer -from kafkatest.services.console_consumer import ConsoleConsumer -from kafkatest.utils import is_int - - -class ClientCompatibilityTest(Test): - - def __init__(self, test_context): - super(ClientCompatibilityTest, self).__init__(test_context=test_context) - - def setUp(self): - self.topic = "test_topic" - self.zk = ZookeeperService(self.test_context, num_nodes=1) - self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zk, version=LATEST_0_8_2, topics={self.topic: { - "partitions": 3, - "replication-factor": 3, - 'configs': {"min.insync.replicas": 2}}}) - self.zk.start() - self.kafka.start() - - # Producer and consumer - self.producer_throughput = 10000 - self.num_producers = 1 - self.num_consumers = 1 - - def test_producer_back_compatibility(self): - """Run 0.9.X java producer against 0.8.X brokers. - This test documents the fact that java producer v0.9.0.0 and later won't run against 0.8.X brokers - the broker responds to a V1 produce request with a V0 fetch response; the client then tries to parse this V0 - produce response as a V1 produce response, resulting in a BufferUnderflowException - """ - self.producer = VerifiableProducer( - self.test_context, self.num_producers, self.kafka, self.topic, max_messages=100, - throughput=self.producer_throughput, version=TRUNK) - - node = self.producer.nodes[0] - try: - self.producer.start() - self.producer.wait() - raise Exception("0.9.X java producer should not run successfully against 0.8.X broker") - except: - # Expected - pass - finally: - self.producer.kill_node(node, clean_shutdown=False) - - self.logger.info("Grepping producer log for expected error type") - node.account.ssh("egrep -m 1 %s %s" % ("\"org\.apache\.kafka\.common\.protocol\.types\.SchemaException.*throttle_time_ms.*: java\.nio\.BufferUnderflowException\"", self.producer.LOG_FILE), allow_fail=False) - - def test_consumer_back_compatibility(self): - """Run the scala 0.8.X consumer against an 0.9.X cluster. - Expect 0.8.X scala consumer to fail with buffer underflow. This error is the same as when an 0.9.X producer - is run against an 0.8.X broker: the broker responds to a V1 fetch request with a V0 fetch response; the - client then tries to parse this V0 fetch response as a V1 fetch response, resulting in a BufferUnderflowException - """ - num_messages = 10 - self.producer = VerifiableProducer( - self.test_context, self.num_producers, self.kafka, self.topic, max_messages=num_messages, - throughput=self.producer_throughput, version=LATEST_0_8_2) - - self.consumer = ConsoleConsumer( - self.test_context, self.num_consumers, self.kafka, self.topic, group_id="consumer-09X", - consumer_timeout_ms=10000, message_validator=is_int, version=TRUNK) - - self.old_consumer = ConsoleConsumer( - self.test_context, self.num_consumers, self.kafka, self.topic, group_id="consumer-08X", - consumer_timeout_ms=10000, message_validator=is_int, version=LATEST_0_8_2) - - self.producer.run() - self.consumer.run() - self.old_consumer.run() - - consumed = len(self.consumer.messages_consumed[1]) - old_consumed = len(self.old_consumer.messages_consumed[1]) - assert old_consumed == num_messages, "Expected 0.8.X scala consumer to consume %d, but only got %d" % (num_messages, old_consumed) - assert consumed == 0, "Expected 0.9.X scala consumer to fail to consume any messages, but got %d" % consumed - - self.logger.info("Grepping consumer log for expected error type") - node = self.consumer.nodes[0] - node.account.ssh("egrep -m 1 %s %s" % ("\"java\.nio\.BufferUnderflowException\"", self.consumer.LOG_FILE), allow_fail=False) - - - From cd8bd606cbf63f37eaee448ffbbab01ce3e0e7db Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Wed, 16 Mar 2016 15:07:37 -0700 Subject: [PATCH 037/206] KAFKA-3403: Upgrade ZkClient to 0.8 This ZkClient version adds authentication validation and a conditional delete method needed for other patches Author: Grant Henke Reviewers: Ismael Juma, Gwen Shapira Closes #1084 from granthenke/zkclient-08 --- gradle/dependencies.gradle | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 3610430995e27..aa1d3f9bdf8f4 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -41,7 +41,7 @@ versions += [ scalaParserCombinators: "1.0.4", slf4j: "1.7.18", snappy: "1.1.2.1", - zkclient: "0.7", + zkclient: "0.8", zookeeper: "3.4.6", ] From 858047a12ba3a7d426178c63226dd2c7509f20dd Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 16 Mar 2016 16:15:55 -0700 Subject: [PATCH 038/206] KAFKA-3402; Restore behaviour of MetadataCache.getTopicMetadata when unsupported security protocol is received Author: Ismael Juma Reviewers: Jason Gustafson, Grant Henke Closes #1073 from ijuma/kafka-3402-restore-get-topic-metadata-behaviour --- .../scala/kafka/server/MetadataCache.scala | 7 +- .../kafka/api/AuthorizerIntegrationTest.scala | 2 +- .../unit/kafka/server/MetadataCacheTest.scala | 90 ++++++++++++++----- 3 files changed, 72 insertions(+), 27 deletions(-) diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 6df261c7482c9..b23ecbe75b9d1 100755 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -24,7 +24,7 @@ import scala.collection.{Seq, Set, mutable} import scala.collection.JavaConverters._ import kafka.cluster.{Broker, EndPoint} import kafka.api._ -import kafka.common.TopicAndPartition +import kafka.common.{BrokerEndPointNotAvailableException, TopicAndPartition} import kafka.controller.{KafkaController, LeaderIsrAndControllerEpoch} import kafka.utils.CoreUtils._ import kafka.utils.Logging @@ -55,7 +55,10 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { } private def getAliveEndpoint(brokerId: Int, protocol: SecurityProtocol): Option[Node] = - aliveNodes.get(brokerId).flatMap(_.get(protocol)) + aliveNodes.get(brokerId).map { nodeMap => + nodeMap.getOrElse(protocol, + throw new BrokerEndPointNotAvailableException(s"Broker `$brokerId` does not support security protocol `$protocol`")) + } private def getPartitionMetadata(topic: String, protocol: SecurityProtocol): Option[Iterable[MetadataResponse.PartitionMetadata]] = { cache.get(topic).map { partitions => diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index 33027e7fce42c..b09c541334d44 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -187,7 +187,7 @@ class AuthorizerIntegrationTest extends KafkaServerTestHarness { private def createUpdateMetadataRequest = { val partitionState = Map(tp -> new requests.UpdateMetadataRequest.PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Set(brokerId).asJava)).asJava val brokers = Set(new requests.UpdateMetadataRequest.Broker(brokerId, - Map(SecurityProtocol.PLAINTEXT -> new requests.UpdateMetadataRequest.EndPoint("localhost", 0)).asJava)).asJava + Map(SecurityProtocol.PLAINTEXT -> new requests.UpdateMetadataRequest.EndPoint("localhost", 0)).asJava, null)).asJava new requests.UpdateMetadataRequest(brokerId, Int.MaxValue, partitionState, brokers) } diff --git a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala index f3f0c8709f143..dcc310fffae3e 100644 --- a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala +++ b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala @@ -19,10 +19,11 @@ package kafka.server import java.util import util.Arrays.asList +import kafka.common.BrokerEndPointNotAvailableException import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.{Errors, SecurityProtocol} import org.apache.kafka.common.requests.UpdateMetadataRequest -import org.apache.kafka.common.requests.UpdateMetadataRequest.{PartitionState, Broker, EndPoint} +import org.apache.kafka.common.requests.UpdateMetadataRequest.{Broker, EndPoint, PartitionState} import org.junit.Test import org.junit.Assert._ @@ -49,10 +50,18 @@ class MetadataCacheTest { val zkVersion = 3 val controllerId = 2 val controllerEpoch = 1 - val brokers = Set( - new Broker(0, Map(SecurityProtocol.PLAINTEXT -> new EndPoint("foo", 9092)).asJava), - new Broker(1, Map(SecurityProtocol.PLAINTEXT -> new EndPoint("foo", 9092)).asJava), - new Broker(2, Map(SecurityProtocol.PLAINTEXT -> new EndPoint("foo", 9092)).asJava)) + + def securityProtocolToEndPoint(brokerId: Int): Map[SecurityProtocol, EndPoint] = { + val host = s"foo-$brokerId" + Map( + SecurityProtocol.PLAINTEXT -> new EndPoint(host, 9092), + SecurityProtocol.SSL -> new EndPoint(host, 9093) + ) + } + + val brokers = (0 to 2).map { brokerId => + new Broker(brokerId, securityProtocolToEndPoint(brokerId).asJava, "rack1") + }.toSet val partitionStates = Map( new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, 0, 0, asList(0), zkVersion, asSet(0)), @@ -62,24 +71,32 @@ class MetadataCacheTest { val updateMetadataRequest = new UpdateMetadataRequest(controllerId, controllerEpoch, partitionStates.asJava, brokers.asJava) cache.updateCache(15, updateMetadataRequest) - val topicMetadatas = cache.getTopicMetadata(Set(topic), SecurityProtocol.PLAINTEXT) - assertEquals(1, topicMetadatas.size) + for (securityProtocol <- Seq(SecurityProtocol.PLAINTEXT, SecurityProtocol.SSL)) { + val topicMetadatas = cache.getTopicMetadata(Set(topic), securityProtocol) + assertEquals(1, topicMetadatas.size) + + val topicMetadata = topicMetadatas.head + assertEquals(Errors.NONE, topicMetadata.error) + assertEquals(topic, topicMetadata.topic) + + val partitionMetadatas = topicMetadata.partitionMetadata.asScala.sortBy(_.partition) + assertEquals(3, partitionMetadatas.size) + + for (i <- 0 to 2) { + val partitionMetadata = partitionMetadatas(i) + assertEquals(Errors.NONE, partitionMetadata.error) + assertEquals(i, partitionMetadata.partition) + val leader = partitionMetadata.leader + assertEquals(i, leader.id) + val endPoint = securityProtocolToEndPoint(partitionMetadata.leader.id)(securityProtocol) + assertEquals(endPoint.host, leader.host) + assertEquals(endPoint.port, leader.port) + assertEquals(List(i), partitionMetadata.isr.asScala.map(_.id)) + assertEquals(List(i), partitionMetadata.replicas.asScala.map(_.id)) + } - val topicMetadata = topicMetadatas.head - assertEquals(Errors.NONE, topicMetadata.error) - assertEquals(topic, topicMetadata.topic) - - val partitionMetadatas = topicMetadata.partitionMetadata.asScala.sortBy(_.partition) - assertEquals(3, partitionMetadatas.size) - - for (i <- 0 to 2) { - val partitionMetadata = partitionMetadatas(i) - assertEquals(Errors.NONE, partitionMetadata.error) - assertEquals(i, partitionMetadata.partition) - assertEquals(i, partitionMetadata.leader.id) - assertEquals(List(i), partitionMetadata.isr.asScala.map(_.id)) - assertEquals(List(i), partitionMetadata.replicas.asScala.map(_.id)) } + } @Test @@ -91,7 +108,7 @@ class MetadataCacheTest { val zkVersion = 3 val controllerId = 2 val controllerEpoch = 1 - val brokers = Set(new Broker(0, Map(SecurityProtocol.PLAINTEXT -> new EndPoint("foo", 9092)).asJava)) + val brokers = Set(new Broker(0, Map(SecurityProtocol.PLAINTEXT -> new EndPoint("foo", 9092)).asJava, null)) val leader = 1 val leaderEpoch = 1 @@ -127,7 +144,7 @@ class MetadataCacheTest { val zkVersion = 3 val controllerId = 2 val controllerEpoch = 1 - val brokers = Set(new Broker(0, Map(SecurityProtocol.PLAINTEXT -> new EndPoint("foo", 9092)).asJava)) + val brokers = Set(new Broker(0, Map(SecurityProtocol.PLAINTEXT -> new EndPoint("foo", 9092)).asJava, null)) // replica 1 is not available val leader = 0 @@ -166,7 +183,7 @@ class MetadataCacheTest { val zkVersion = 3 val controllerId = 2 val controllerEpoch = 1 - val brokers = Set(new Broker(0, Map(SecurityProtocol.PLAINTEXT -> new EndPoint("foo", 9092)).asJava)) + val brokers = Set(new Broker(0, Map(SecurityProtocol.PLAINTEXT -> new EndPoint("foo", 9092)).asJava, "rack1")) // replica 1 is not available val leader = 0 @@ -196,4 +213,29 @@ class MetadataCacheTest { assertEquals(Set(0), partitionMetadata.isr.asScala.map(_.id).toSet) } + @Test + def getTopicMetadataWithNonSupportedSecurityProtocol() { + val topic = "topic" + val cache = new MetadataCache(1) + val brokers = Set(new Broker(0, Map(SecurityProtocol.PLAINTEXT -> new EndPoint("foo", 9092)).asJava, "")) + val controllerEpoch = 1 + val leader = 0 + val leaderEpoch = 0 + val replicas = asSet[Integer](0) + val isr = asList[Integer](0, 1) + val partitionStates = Map( + new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas)) + val updateMetadataRequest = new UpdateMetadataRequest(2, controllerEpoch, partitionStates.asJava, brokers.asJava) + cache.updateCache(15, updateMetadataRequest) + + try { + val result = cache.getTopicMetadata(Set(topic), SecurityProtocol.SSL) + fail(s"Exception should be thrown by `getTopicMetadata` with non-supported SecurityProtocol, $result was returned instead") + } + catch { + case e: BrokerEndPointNotAvailableException => //expected + } + + } + } From 9a836d0154efe6ea1effc688567186cb56265bf4 Mon Sep 17 00:00:00 2001 From: Anna Povzner Date: Wed, 16 Mar 2016 17:29:29 -0700 Subject: [PATCH 039/206] KAFKA-3303; Pass partial record metadata to ProducerInterceptor.onAcknowledgement on error This is a KIP-42 followup. Currently, If sending the record fails before it gets to the server, ProducerInterceptor.onAcknowledgement() is called with metadata == null, and non-null exception. However, it is useful to pass topic and partition, if known, to ProducerInterceptor.onAcknowledgement() as well. This patch ensures that ProducerInterceptor.onAcknowledgement() gets record metadata with topic and maybe partition. If partition is not set in 'record' and KafkaProducer.send() fails before partition gets assigned, then ProducerInterceptor.onAcknowledgement() gets RecordMetadata with partition == -1. Only time when ProducerInterceptor.onAcknowledgement() gets null record metadata is when the client passes null record to KafkaProducer.send(). Author: Anna Povzner Reviewers: Ismael Juma , Ashish Singh , Jun Rao Closes #1015 from apovzner/kip42-3 --- .../kafka/clients/producer/KafkaProducer.java | 36 ++++++++---- .../clients/producer/ProducerInterceptor.java | 7 ++- .../clients/producer/RecordMetadata.java | 5 ++ .../internals/ProducerInterceptors.java | 35 ++++++++++- .../internals/ProducerInterceptorsTest.java | 58 +++++++++++++++++++ .../kafka/test/MockProducerInterceptor.java | 9 ++- .../kafka/api/PlaintextConsumerTest.scala | 5 +- 7 files changed, 138 insertions(+), 17 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index c87973ad1a2db..6acc0599e8518 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -427,9 +427,7 @@ public Future send(ProducerRecord record) { public Future send(ProducerRecord record, Callback callback) { // intercept the record, which can be potentially modified; this method does not throw exceptions ProducerRecord interceptedRecord = this.interceptors == null ? record : this.interceptors.onSend(record); - // producer callback will make sure to call both 'callback' and interceptor callback - Callback interceptCallback = this.interceptors == null ? callback : new InterceptorCallback<>(callback, this.interceptors); - return doSend(interceptedRecord, interceptCallback); + return doSend(interceptedRecord, callback); } /** @@ -437,6 +435,7 @@ public Future send(ProducerRecord record, Callback callbac * See {@link #send(ProducerRecord, Callback)} for details. */ private Future doSend(ProducerRecord record, Callback callback) { + TopicPartition tp = null; try { // first make sure the metadata for the topic is available long waitedOnMetadataMs = waitOnMetadata(record.topic(), this.maxBlockTimeMs); @@ -460,10 +459,12 @@ private Future doSend(ProducerRecord record, Callback call int partition = partition(record, serializedKey, serializedValue, metadata.fetch()); int serializedSize = Records.LOG_OVERHEAD + Record.recordSize(serializedKey, serializedValue); ensureValidRecordSize(serializedSize); - TopicPartition tp = new TopicPartition(record.topic(), partition); + tp = new TopicPartition(record.topic(), partition); long timestamp = record.timestamp() == null ? time.milliseconds() : record.timestamp(); log.trace("Sending record {} with callback {} to topic {} partition {}", record, callback, record.topic(), partition); - RecordAccumulator.RecordAppendResult result = accumulator.append(tp, timestamp, serializedKey, serializedValue, callback, remainingWaitMs); + // producer callback will make sure to call both 'callback' and interceptor callback + Callback interceptCallback = this.interceptors == null ? callback : new InterceptorCallback<>(callback, this.interceptors, tp); + RecordAccumulator.RecordAppendResult result = accumulator.append(tp, timestamp, serializedKey, serializedValue, interceptCallback, remainingWaitMs); if (result.batchIsFull || result.newBatchCreated) { log.trace("Waking up the sender since topic {} partition {} is either full or getting a new batch", record.topic(), partition); this.sender.wakeup(); @@ -477,27 +478,29 @@ private Future doSend(ProducerRecord record, Callback call if (callback != null) callback.onCompletion(null, e); this.errors.record(); + if (this.interceptors != null) + this.interceptors.onSendError(record, tp, e); return new FutureFailure(e); } catch (InterruptedException e) { this.errors.record(); if (this.interceptors != null) - this.interceptors.onAcknowledgement(null, e); + this.interceptors.onSendError(record, tp, e); throw new InterruptException(e); } catch (BufferExhaustedException e) { this.errors.record(); this.metrics.sensor("buffer-exhausted-records").record(); if (this.interceptors != null) - this.interceptors.onAcknowledgement(null, e); + this.interceptors.onSendError(record, tp, e); throw e; } catch (KafkaException e) { this.errors.record(); if (this.interceptors != null) - this.interceptors.onAcknowledgement(null, e); + this.interceptors.onSendError(record, tp, e); throw e; } catch (Exception e) { // we notify interceptor about all exceptions, since onSend is called before anything else in this method if (this.interceptors != null) - this.interceptors.onAcknowledgement(null, e); + this.interceptors.onSendError(record, tp, e); throw e; } } @@ -763,15 +766,24 @@ public boolean isDone() { private static class InterceptorCallback implements Callback { private final Callback userCallback; private final ProducerInterceptors interceptors; + private final TopicPartition tp; - public InterceptorCallback(Callback userCallback, ProducerInterceptors interceptors) { + public InterceptorCallback(Callback userCallback, ProducerInterceptors interceptors, + TopicPartition tp) { this.userCallback = userCallback; this.interceptors = interceptors; + this.tp = tp; } public void onCompletion(RecordMetadata metadata, Exception exception) { - if (this.interceptors != null) - this.interceptors.onAcknowledgement(metadata, exception); + if (this.interceptors != null) { + if (metadata == null) { + this.interceptors.onAcknowledgement(new RecordMetadata(tp, -1, -1, Record.NO_TIMESTAMP, -1, -1, -1), + exception); + } else { + this.interceptors.onAcknowledgement(metadata, exception); + } + } if (this.userCallback != null) this.userCallback.onCompletion(metadata, exception); } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java index aa18fdc736c2b..e835a69501b2b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/ProducerInterceptor.java @@ -76,7 +76,12 @@ public interface ProducerInterceptor extends Configurable { * This method will generally execute in the background I/O thread, so the implementation should be reasonably fast. * Otherwise, sending of messages from other threads could be delayed. * - * @param metadata The metadata for the record that was sent (i.e. the partition and offset). Null if an error occurred. + * @param metadata The metadata for the record that was sent (i.e. the partition and offset). + * If an error occurred, metadata will contain only valid topic and maybe + * partition. If partition is not given in ProducerRecord and an error occurs + * before partition gets assigned, then partition will be set to RecordMetadata.NO_PARTITION. + * The metadata may be null if the client passed null record to + * {@link org.apache.kafka.clients.producer.KafkaProducer#send(ProducerRecord)}. * @param exception The exception thrown during processing of this record. Null if no error occurred. */ public void onAcknowledgement(RecordMetadata metadata, Exception exception); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java index c60a53d916209..988da16addf97 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java @@ -23,6 +23,11 @@ */ public final class RecordMetadata { + /** + * Partition value for record without partition assigned + */ + public static final int UNKNOWN_PARTITION = -1; + private final long offset; // The timestamp of the message. // If LogAppendTime is used for the topic, the timestamp will be the timestamp returned by the broker. diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerInterceptors.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerInterceptors.java index 9343a2e188988..8466d3a85752d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerInterceptors.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ProducerInterceptors.java @@ -20,6 +20,8 @@ import org.apache.kafka.clients.producer.ProducerInterceptor; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.record.Record; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -76,7 +78,8 @@ public ProducerRecord onSend(ProducerRecord record) { * * This method does not throw exceptions. Exceptions thrown by any of interceptor methods are caught and ignored. * - * @param metadata The metadata for the record that was sent (i.e. the partition and offset). Null if an error occurred. + * @param metadata The metadata for the record that was sent (i.e. the partition and offset). + * If an error occurred, metadata will only contain valid topic and maybe partition. * @param exception The exception thrown during processing of this record. Null if no error occurred. */ public void onAcknowledgement(RecordMetadata metadata, Exception exception) { @@ -90,6 +93,36 @@ public void onAcknowledgement(RecordMetadata metadata, Exception exception) { } } + /** + * This method is called when sending the record fails in {@link ProducerInterceptor#onSend + * (ProducerRecord)} method. This method calls {@link ProducerInterceptor#onAcknowledgement(RecordMetadata, Exception)} + * method for each interceptor + * + * @param record The record from client + * @param interceptTopicPartition The topic/partition for the record if an error occurred + * after partition gets assigned; the topic part of interceptTopicPartition is the same as in record. + * @param exception The exception thrown during processing of this record. + */ + public void onSendError(ProducerRecord record, TopicPartition interceptTopicPartition, Exception exception) { + for (ProducerInterceptor interceptor : this.interceptors) { + try { + if (record == null && interceptTopicPartition == null) { + interceptor.onAcknowledgement(null, exception); + } else { + if (interceptTopicPartition == null) { + interceptTopicPartition = new TopicPartition(record.topic(), + record.partition() == null ? RecordMetadata.UNKNOWN_PARTITION : record.partition()); + } + interceptor.onAcknowledgement(new RecordMetadata(interceptTopicPartition, -1, -1, Record.NO_TIMESTAMP, -1, -1, -1), + exception); + } + } catch (Exception e) { + // do not propagate interceptor exceptions, just log + log.warn("Error executing interceptor onAcknowledgement callback", e); + } + } + } + /** * Closes every interceptor in a container. */ diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java index 5a32ddaac63e4..2135eb2cf3048 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/ProducerInterceptorsTest.java @@ -30,6 +30,9 @@ public class ProducerInterceptorsTest { private final TopicPartition tp = new TopicPartition("test", 0); private final ProducerRecord producerRecord = new ProducerRecord<>("test", 0, 1, "value"); private int onAckCount = 0; + private int onErrorAckCount = 0; + private int onErrorAckWithTopicSetCount = 0; + private int onErrorAckWithTopicPartitionSetCount = 0; private int onSendCount = 0; private class AppendProducerInterceptor implements ProducerInterceptor { @@ -59,6 +62,16 @@ public ProducerRecord onSend(ProducerRecord re @Override public void onAcknowledgement(RecordMetadata metadata, Exception exception) { onAckCount++; + if (exception != null) { + onErrorAckCount++; + // the length check is just to call topic() method and let it throw an exception + // if RecordMetadata.TopicPartition is null + if (metadata != null && metadata.topic().length() >= 0) { + onErrorAckWithTopicSetCount++; + if (metadata.partition() >= 0) + onErrorAckWithTopicPartitionSetCount++; + } + } if (throwExceptionOnAck) throw new KafkaException("Injected exception in AppendProducerInterceptor.onAcknowledgement"); } @@ -143,5 +156,50 @@ public void testOnAcknowledgementChain() { interceptors.close(); } + + @Test + public void testOnAcknowledgementWithErrorChain() { + List> interceptorList = new ArrayList<>(); + AppendProducerInterceptor interceptor1 = new AppendProducerInterceptor("One"); + interceptorList.add(interceptor1); + ProducerInterceptors interceptors = new ProducerInterceptors<>(interceptorList); + + // verify that metadata contains both topic and partition + interceptors.onSendError(producerRecord, + new TopicPartition(producerRecord.topic(), producerRecord.partition()), + new KafkaException("Test")); + assertEquals(1, onErrorAckCount); + assertEquals(1, onErrorAckWithTopicPartitionSetCount); + + // verify that metadata contains both topic and partition (because record already contains partition) + interceptors.onSendError(producerRecord, null, new KafkaException("Test")); + assertEquals(2, onErrorAckCount); + assertEquals(2, onErrorAckWithTopicPartitionSetCount); + + // if producer record does not contain partition, interceptor should get partition == -1 + ProducerRecord record2 = new ProducerRecord<>("test2", null, 1, "value"); + interceptors.onSendError(record2, null, new KafkaException("Test")); + assertEquals(3, onErrorAckCount); + assertEquals(3, onErrorAckWithTopicSetCount); + assertEquals(2, onErrorAckWithTopicPartitionSetCount); + + // if producer record does not contain partition, but topic/partition is passed to + // onSendError, then interceptor should get valid partition + int reassignedPartition = producerRecord.partition() + 1; + interceptors.onSendError(record2, + new TopicPartition(record2.topic(), reassignedPartition), + new KafkaException("Test")); + assertEquals(4, onErrorAckCount); + assertEquals(4, onErrorAckWithTopicSetCount); + assertEquals(3, onErrorAckWithTopicPartitionSetCount); + + // if both record and topic/partition are null, interceptor should not receive metadata + interceptors.onSendError(null, null, new KafkaException("Test")); + assertEquals(5, onErrorAckCount); + assertEquals(4, onErrorAckWithTopicSetCount); + assertEquals(3, onErrorAckWithTopicPartitionSetCount); + + interceptors.close(); + } } diff --git a/clients/src/test/java/org/apache/kafka/test/MockProducerInterceptor.java b/clients/src/test/java/org/apache/kafka/test/MockProducerInterceptor.java index cee12473dbeca..9e4d0de113dd7 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockProducerInterceptor.java +++ b/clients/src/test/java/org/apache/kafka/test/MockProducerInterceptor.java @@ -32,6 +32,7 @@ public class MockProducerInterceptor implements ProducerInterceptor onSend(ProducerRecord reco @Override public void onAcknowledgement(RecordMetadata metadata, Exception exception) { - if (exception != null) + if (exception != null) { ON_ERROR_COUNT.incrementAndGet(); - else if (metadata != null) + if (metadata != null) { + ON_ERROR_WITH_METADATA_COUNT.incrementAndGet(); + } + } else if (metadata != null) ON_SUCCESS_COUNT.incrementAndGet(); } @@ -81,5 +85,6 @@ public static void resetCounters() { ONSEND_COUNT.set(0); ON_SUCCESS_COUNT.set(0); ON_ERROR_COUNT.set(0); + ON_ERROR_WITH_METADATA_COUNT.set(0); } } diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 9bdbf6d021c11..801447956c725 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -568,7 +568,10 @@ class PlaintextConsumerTest extends BaseConsumerTest { testProducer.send(null, null) fail("Should not allow sending a null record") } catch { - case e: Throwable => assertEquals("Interceptor should be notified about exception", 1, MockProducerInterceptor.ON_ERROR_COUNT.intValue()) // this is ok + case e: Throwable => { + assertEquals("Interceptor should be notified about exception", 1, MockProducerInterceptor.ON_ERROR_COUNT.intValue()) + assertEquals("Interceptor should not receive metadata with an exception when record is null", 0, MockProducerInterceptor.ON_ERROR_WITH_METADATA_COUNT.intValue()) + } } // create consumer with interceptor From 958e10c87ce293c3bf59bb9840eaaae915eff25e Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Thu, 17 Mar 2016 10:41:48 -0700 Subject: [PATCH 040/206] KAFKA-3411: Streams: stop using "job" terminology, rename job.id to application.id guozhangwang ymatsuda : please review. Author: Michael G. Noll Reviewers: Guozhang Wang Closes #1081 from miguno/KAFKA-3411 --- .../pageview/JsonTimestampExtractor.java | 8 ++-- ...ewTypedJob.java => PageViewTypedDemo.java} | 10 ++--- ...typedJob.java => PageViewUntypedDemo.java} | 10 ++--- .../pipe/{PipeJob.java => PipeDemo.java} | 6 +-- .../{WordCountJob.java => WordCountDemo.java} | 6 +-- ...orJob.java => WordCountProcessorDemo.java} | 6 +-- .../apache/kafka/streams/KafkaStreams.java | 12 +++--- .../apache/kafka/streams/StreamsConfig.java | 14 +++---- .../streams/processor/PartitionGrouper.java | 8 ++-- .../streams/processor/ProcessorContext.java | 6 +-- .../streams/processor/TopologyBuilder.java | 42 +++++++++---------- .../processor/internals/AbstractTask.java | 16 +++---- .../internals/ProcessorContextImpl.java | 9 +--- .../internals/ProcessorStateManager.java | 14 +++---- .../internals/StandbyContextImpl.java | 14 +++---- .../processor/internals/StandbyTask.java | 12 ++---- .../internals/StreamPartitionAssignor.java | 4 +- .../processor/internals/StreamTask.java | 6 +-- .../processor/internals/StreamThread.java | 28 ++++++------- .../state/internals/StoreChangeLogger.java | 2 +- .../kafka/streams/StreamsConfigTest.java | 7 ++-- .../internals/ProcessorStateManagerTest.java | 28 ++++++------- .../internals/ProcessorTopologyTest.java | 2 +- .../processor/internals/StandbyTaskTest.java | 20 ++++----- .../StreamPartitionAssignorTest.java | 2 +- .../processor/internals/StreamTaskTest.java | 6 +-- .../processor/internals/StreamThreadTest.java | 34 +++++++-------- .../streams/smoketest/SmokeTestClient.java | 2 +- .../kafka/test/MockProcessorContext.java | 4 +- .../test/ProcessorTopologyTestDriver.java | 6 +-- 30 files changed, 163 insertions(+), 181 deletions(-) rename streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/{PageViewTypedJob.java => PageViewTypedDemo.java} (95%) rename streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/{PageViewUntypedJob.java => PageViewUntypedDemo.java} (94%) rename streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/{PipeJob.java => PipeDemo.java} (94%) rename streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/{WordCountJob.java => WordCountDemo.java} (96%) rename streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/{WordCountProcessorJob.java => WordCountProcessorDemo.java} (96%) diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonTimestampExtractor.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonTimestampExtractor.java index 6443193ef0c75..63e8377d6520b 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonTimestampExtractor.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/JsonTimestampExtractor.java @@ -29,12 +29,12 @@ public class JsonTimestampExtractor implements TimestampExtractor { @Override public long extract(ConsumerRecord record) { - if (record.value() instanceof PageViewTypedJob.PageView) { - return ((PageViewTypedJob.PageView) record.value()).timestamp; + if (record.value() instanceof PageViewTypedDemo.PageView) { + return ((PageViewTypedDemo.PageView) record.value()).timestamp; } - if (record.value() instanceof PageViewTypedJob.UserProfile) { - return ((PageViewTypedJob.UserProfile) record.value()).timestamp; + if (record.value() instanceof PageViewTypedDemo.UserProfile) { + return ((PageViewTypedDemo.UserProfile) record.value()).timestamp; } if (record.value() instanceof JsonNode) { diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedJob.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java similarity index 95% rename from streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedJob.java rename to streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java index 1fcb4039b1af6..4f9de291c3406 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedJob.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java @@ -18,8 +18,6 @@ package org.apache.kafka.streams.examples.pageview; import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.LongDeserializer; -import org.apache.kafka.common.serialization.LongSerializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.serialization.StringDeserializer; @@ -50,7 +48,7 @@ * Before running this example you must create the source topic (e.g. via bin/kafka-topics.sh --create ...) * and write some data to it (e.g. via bin-kafka-console-producer.sh). Otherwise you won't see any data arriving in the output topic. */ -public class PageViewTypedJob { +public class PageViewTypedDemo { // POJO classes static public class PageView { @@ -82,7 +80,7 @@ static public class RegionCount { public static void main(String[] args) throws Exception { Properties props = new Properties(); - props.put(StreamsConfig.JOB_ID_CONFIG, "streams-pageview-typed"); + props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pageview-typed"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); @@ -98,8 +96,6 @@ public static void main(String[] args) throws Exception { final Serializer stringSerializer = new StringSerializer(); final Deserializer stringDeserializer = new StringDeserializer(); - final Serializer longSerializer = new LongSerializer(); - final Deserializer longDeserializer = new LongDeserializer(); // TODO: the following can be removed with a serialization factory Map serdeProps = new HashMap<>(); @@ -175,7 +171,7 @@ public KeyValue apply(Windowed ke KafkaStreams streams = new KafkaStreams(builder, props); streams.start(); - // usually the streaming job would be ever running, + // usually the stream application would be running forever, // in this example we just let it run for some time and stop since the input data is finite. Thread.sleep(5000L); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedJob.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java similarity index 94% rename from streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedJob.java rename to streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java index fb1a55dc8bfad..9377095b2128d 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedJob.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java @@ -21,8 +21,6 @@ import com.fasterxml.jackson.databind.node.JsonNodeFactory; import com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.LongDeserializer; -import org.apache.kafka.common.serialization.LongSerializer; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.common.serialization.StringDeserializer; @@ -54,11 +52,11 @@ * Before running this example you must create the source topic (e.g. via bin/kafka-topics.sh --create ...) * and write some data to it (e.g. via bin-kafka-console-producer.sh). Otherwise you won't see any data arriving in the output topic. */ -public class PageViewUntypedJob { +public class PageViewUntypedDemo { public static void main(String[] args) throws Exception { Properties props = new Properties(); - props.put(StreamsConfig.JOB_ID_CONFIG, "streams-pageview-untyped"); + props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pageview-untyped"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); @@ -74,8 +72,6 @@ public static void main(String[] args) throws Exception { final Serializer stringSerializer = new StringSerializer(); final Deserializer stringDeserializer = new StringDeserializer(); - final Serializer longSerializer = new LongSerializer(); - final Deserializer longDeserializer = new LongDeserializer(); final Serializer jsonSerializer = new JsonSerializer(); final Deserializer jsonDeserializer = new JsonDeserializer(); @@ -131,7 +127,7 @@ public KeyValue apply(Windowed key, Long value) { KafkaStreams streams = new KafkaStreams(builder, props); streams.start(); - // usually the streaming job would be ever running, + // usually the stream application would be running forever, // in this example we just let it run for some time and stop since the input data is finite. Thread.sleep(5000L); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeJob.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java similarity index 94% rename from streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeJob.java rename to streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java index 8885ca2aa9a88..c37c68ac3f29a 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeJob.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java @@ -35,11 +35,11 @@ * Before running this example you must create the source topic (e.g. via bin/kafka-topics.sh --create ...) * and write some data to it (e.g. via bin-kafka-console-producer.sh). Otherwise you won't see any data arriving in the output topic. */ -public class PipeJob { +public class PipeDemo { public static void main(String[] args) throws Exception { Properties props = new Properties(); - props.put(StreamsConfig.JOB_ID_CONFIG, "streams-pipe"); + props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pipe"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); props.put(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class); @@ -56,7 +56,7 @@ public static void main(String[] args) throws Exception { KafkaStreams streams = new KafkaStreams(builder, props); streams.start(); - // usually the streaming job would be ever running, + // usually the stream application would be running forever, // in this example we just let it run for some time and stop since the input data is finite. Thread.sleep(5000L); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountJob.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java similarity index 96% rename from streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountJob.java rename to streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java index 2b51a4401569c..03d514292ac12 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountJob.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java @@ -45,11 +45,11 @@ * Before running this example you must create the source topic (e.g. via bin/kafka-topics.sh --create ...) * and write some data to it (e.g. via bin-kafka-console-producer.sh). Otherwise you won't see any data arriving in the output topic. */ -public class WordCountJob { +public class WordCountDemo { public static void main(String[] args) throws Exception { Properties props = new Properties(); - props.put(StreamsConfig.JOB_ID_CONFIG, "streams-wordcount"); + props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); @@ -87,7 +87,7 @@ public KeyValue apply(String key, String value) { KafkaStreams streams = new KafkaStreams(builder, props); streams.start(); - // usually the streaming job would be ever running, + // usually the stream application would be running forever, // in this example we just let it run for some time and stop since the input data is finite. Thread.sleep(5000L); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorJob.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java similarity index 96% rename from streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorJob.java rename to streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java index cb82656ccaad8..b651b3aa40210 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorJob.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java @@ -43,7 +43,7 @@ * Before running this example you must create the source topic (e.g. via bin/kafka-topics.sh --create ...) * and write some data to it (e.g. via bin-kafka-console-producer.sh). Otherwise you won't see any data arriving in the output topic. */ -public class WordCountProcessorJob { +public class WordCountProcessorDemo { private static class MyProcessorSupplier implements ProcessorSupplier { @@ -105,7 +105,7 @@ public void close() { public static void main(String[] args) throws Exception { Properties props = new Properties(); - props.put(StreamsConfig.JOB_ID_CONFIG, "streams-wordcount-processor"); + props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount-processor"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); @@ -128,7 +128,7 @@ public static void main(String[] args) throws Exception { KafkaStreams streams = new KafkaStreams(builder, props); streams.start(); - // usually the streaming job would be ever running, + // usually the stream application would be running forever, // in this example we just let it run for some time and stop since the input data is finite. Thread.sleep(5000L); diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index 15d6d8b57fdda..20958e4a4730c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -46,7 +46,7 @@ * The {@link KafkaStreams} class manages the lifecycle of a Kafka Streams instance. One stream instance can contain one or * more threads specified in the configs for the processing work. *

      - * A {@link KafkaStreams} instance can co-ordinate with any other instances with the same job ID (whether in this same process, on other processes + * A {@link KafkaStreams} instance can co-ordinate with any other instances with the same application ID (whether in this same process, on other processes * on this machine, or on remote machines) as a single (possibly distributed) stream processing client. These instances will divide up the work * based on the assignment of the input topic partitions so that all partitions are being * consumed. If instances are added or failed, all instances will rebelance the partition assignment among themselves @@ -59,7 +59,7 @@ * A simple example might look like this: *

        *    Map<String, Object> props = new HashMap<>();
      - *    props.put(StreamsConfig.JOB_ID_CONFIG, "my-job");
      + *    props.put(StreamsConfig.APPLICATION_ID_CONFIG, "my-stream-processing-application");
        *    props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
        *    props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
        *    props.put(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
      @@ -113,12 +113,12 @@ public KafkaStreams(TopologyBuilder builder, StreamsConfig config) {
       
               this.processId = UUID.randomUUID();
       
      -        // JobId is a required config and hence should always have value
      -        String jobId = config.getString(StreamsConfig.JOB_ID_CONFIG);
      +        // The application ID is a required config and hence should always have value
      +        String applicationId = config.getString(StreamsConfig.APPLICATION_ID_CONFIG);
       
               String clientId = config.getString(StreamsConfig.CLIENT_ID_CONFIG);
               if (clientId.length() <= 0)
      -            clientId = jobId + "-" + STREAM_CLIENT_ID_SEQUENCE.getAndIncrement();
      +            clientId = applicationId + "-" + STREAM_CLIENT_ID_SEQUENCE.getAndIncrement();
       
               List reporters = config.getConfiguredInstances(StreamsConfig.METRIC_REPORTER_CLASSES_CONFIG,
                       MetricsReporter.class);
      @@ -132,7 +132,7 @@ public KafkaStreams(TopologyBuilder builder, StreamsConfig config) {
       
               this.threads = new StreamThread[config.getInt(StreamsConfig.NUM_STREAM_THREADS_CONFIG)];
               for (int i = 0; i < this.threads.length; i++) {
      -            this.threads[i] = new StreamThread(builder, config, jobId, clientId, processId, metrics, time);
      +            this.threads[i] = new StreamThread(builder, config, applicationId, clientId, processId, metrics, time);
               }
           }
       
      diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
      index c4b8ffe5d7a4c..52fdbd4583920 100644
      --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
      +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java
      @@ -83,13 +83,13 @@ public class StreamsConfig extends AbstractConfig {
           public static final String PARTITION_GROUPER_CLASS_CONFIG = "partition.grouper";
           private static final String PARTITION_GROUPER_CLASS_DOC = "Partition grouper class that implements the PartitionGrouper interface.";
       
      -    /** job.id */
      -    public static final String JOB_ID_CONFIG = "job.id";
      -    public static final String JOB_ID_DOC = "An id string to identify for the stream job. It is used as 1) the default client-id prefix, 2) the group-id for membership management, 3) the changelog topic prefix.";
      +    /** application.id */
      +    public static final String APPLICATION_ID_CONFIG = "application.id";
      +    public static final String APPLICATION_ID_DOC = "An identifier for the stream processing application. Must be unique within the Kafka cluster. It is used as 1) the default client-id prefix, 2) the group-id for membership management, 3) the changelog topic prefix.";
       
           /** replication.factor */
           public static final String REPLICATION_FACTOR_CONFIG = "replication.factor";
      -    public static final String REPLICATION_FACTOR_DOC = "The replication factor for change log topics and repartition topics created by the job.";
      +    public static final String REPLICATION_FACTOR_DOC = "The replication factor for change log topics and repartition topics created by the stream processing application.";
       
           /** key.serializer */
           public static final String KEY_SERIALIZER_CLASS_CONFIG = ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG;
      @@ -124,10 +124,10 @@ public class StreamsConfig extends AbstractConfig {
           private static final String WALLCLOCK_TIMESTAMP_EXTRACTOR = "org.apache.kafka.streams.processor.internals.WallclockTimestampExtractor";
       
           static {
      -        CONFIG = new ConfigDef().define(JOB_ID_CONFIG,      // required with no default value
      +        CONFIG = new ConfigDef().define(APPLICATION_ID_CONFIG,      // required with no default value
                                               Type.STRING,
                                               Importance.HIGH,
      -                                        StreamsConfig.JOB_ID_DOC)
      +                                        StreamsConfig.APPLICATION_ID_DOC)
                                       .define(BOOTSTRAP_SERVERS_CONFIG,       // required with no default value
                                               Type.STRING,
                                               Importance.HIGH,
      @@ -297,7 +297,7 @@ public Map getProducerConfigs(String clientId) {
           }
       
           private void removeStreamsSpecificConfigs(Map props) {
      -        props.remove(StreamsConfig.JOB_ID_CONFIG);
      +        props.remove(StreamsConfig.APPLICATION_ID_CONFIG);
               props.remove(StreamsConfig.STATE_DIR_CONFIG);
               props.remove(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG);
               props.remove(StreamsConfig.NUM_STREAM_THREADS_CONFIG);
      diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/PartitionGrouper.java b/streams/src/main/java/org/apache/kafka/streams/processor/PartitionGrouper.java
      index ae9844de41a95..0c94084752510 100644
      --- a/streams/src/main/java/org/apache/kafka/streams/processor/PartitionGrouper.java
      +++ b/streams/src/main/java/org/apache/kafka/streams/processor/PartitionGrouper.java
      @@ -28,7 +28,8 @@
        *
        * This grouper also acts as the stream task creation function along with partition distribution
        * such that each generated partition group is assigned with a distinct {@link TaskId};
      - * the created task ids will then be assigned to Kafka Streams instances that host the stream job.
      + * the created task ids will then be assigned to Kafka Streams instances that host the stream
      + * processing application.
        */
       public interface PartitionGrouper {
       
      @@ -37,9 +38,10 @@ public interface PartitionGrouper {
            * expected to be processed together must be in the same group. DefaultPartitionGrouper implements this
            * interface. See {@link DefaultPartitionGrouper} for more information.
            *
      -     * @param topicGroups The map from the {@link TopologyBuilder#topicGroups() topic group} id to topics
      +     * @param topicGroups The map from the {@link TopologyBuilder#topicGroups(String)} topic group} id to topics
            * @param metadata Metadata of the consuming cluster
            * @return a map of task ids to groups of partitions
            */
           Map> partitionGroups(Map> topicGroups, Cluster metadata);
      -}
      +
      +}
      \ No newline at end of file
      diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java
      index 79376ba6ae845..e9d5252b1e31d 100644
      --- a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java
      +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java
      @@ -29,11 +29,11 @@
       public interface ProcessorContext {
       
           /**
      -     * Returns the job id
      +     * Returns the application id
            *
      -     * @return the job id
      +     * @return the application id
            */
      -    String jobId();
      +    String applicationId();
       
           /**
            * Returns the task id
      diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java
      index 6e5aec5413b56..ab7122ba1faaf 100644
      --- a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java
      +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java
      @@ -85,7 +85,7 @@ private static abstract class NodeFactory {
                   this.name = name;
               }
       
      -        public abstract ProcessorNode build(String jobId);
      +        public abstract ProcessorNode build(String applicationId);
           }
       
           private static class ProcessorNodeFactory extends NodeFactory {
      @@ -105,7 +105,7 @@ public void addStateStore(String stateStoreName) {
       
               @SuppressWarnings("unchecked")
               @Override
      -        public ProcessorNode build(String jobId) {
      +        public ProcessorNode build(String applicationId) {
                   return new ProcessorNode(name, supplier.get(), stateStoreNames);
               }
           }
      @@ -124,7 +124,7 @@ private SourceNodeFactory(String name, String[] topics, Deserializer keyDeserial
       
               @SuppressWarnings("unchecked")
               @Override
      -        public ProcessorNode build(String jobId) {
      +        public ProcessorNode build(String applicationId) {
                   return new SourceNode(name, keyDeserializer, valDeserializer);
               }
           }
      @@ -147,10 +147,10 @@ private SinkNodeFactory(String name, String[] parents, String topic, Serializer
       
               @SuppressWarnings("unchecked")
               @Override
      -        public ProcessorNode build(String jobId) {
      +        public ProcessorNode build(String applicationId) {
                   if (internalTopicNames.contains(topic)) {
      -                // prefix the job id to the internal topic name
      -                return new SinkNode(name, jobId + "-" + topic, keySerializer, valSerializer, partitioner);
      +                // prefix the internal topic name with the application id
      +                return new SinkNode(name, applicationId + "-" + topic, keySerializer, valSerializer, partitioner);
                   } else {
                       return new SinkNode(name, topic, keySerializer, valSerializer, partitioner);
                   }
      @@ -496,7 +496,7 @@ private void connectProcessorAndStateStore(String processorName, String stateSto
            *
            * @return groups of topic names
            */
      -    public Map topicGroups(String jobId) {
      +    public Map topicGroups(String applicationId) {
               Map topicGroups = new HashMap<>();
       
               if (nodeGroups == null)
      @@ -514,8 +514,8 @@ public Map topicGroups(String jobId) {
                           // if some of the topics are internal, add them to the internal topics
                           for (String topic : topics) {
                               if (this.internalTopicNames.contains(topic)) {
      -                            // prefix the job id to the internal topic name
      -                            String internalTopic = jobId + "-" + topic;
      +                            // prefix the internal topic name with the application id
      +                            String internalTopic = applicationId + "-" + topic;
                                   internalSourceTopics.add(internalTopic);
                                   sourceTopics.add(internalTopic);
                               } else {
      @@ -528,8 +528,8 @@ public Map topicGroups(String jobId) {
                       String topic = nodeToSinkTopic.get(node);
                       if (topic != null) {
                           if (internalTopicNames.contains(topic)) {
      -                        // prefix the job id to the change log topic name
      -                        sinkTopics.add(jobId + "-" + topic);
      +                        // prefix the change log topic name with the application id
      +                        sinkTopics.add(applicationId + "-" + topic);
                           } else {
                               sinkTopics.add(topic);
                           }
      @@ -538,8 +538,8 @@ public Map topicGroups(String jobId) {
                       // if the node is connected to a state, add to the state topics
                       for (StateStoreFactory stateFactory : stateFactories.values()) {
                           if (stateFactory.isInternal && stateFactory.users.contains(node)) {
      -                        // prefix the job id to the change log topic name
      -                        stateChangelogTopics.add(jobId + "-" + stateFactory.supplier.name() + ProcessorStateManager.STATE_CHANGELOG_TOPIC_SUFFIX);
      +                        // prefix the change log topic name with the application id
      +                        stateChangelogTopics.add(applicationId + "-" + stateFactory.supplier.name() + ProcessorStateManager.STATE_CHANGELOG_TOPIC_SUFFIX);
                           }
                       }
                   }
      @@ -637,7 +637,7 @@ public Collection> copartitionGroups() {
            *
            * @see org.apache.kafka.streams.KafkaStreams#KafkaStreams(TopologyBuilder, org.apache.kafka.streams.StreamsConfig)
            */
      -    public ProcessorTopology build(String jobId, Integer topicGroupId) {
      +    public ProcessorTopology build(String applicationId, Integer topicGroupId) {
               Set nodeGroup;
               if (topicGroupId != null) {
                   nodeGroup = nodeGroups().get(topicGroupId);
      @@ -645,11 +645,11 @@ public ProcessorTopology build(String jobId, Integer topicGroupId) {
                   // when nodeGroup is null, we build the full topology. this is used in some tests.
                   nodeGroup = null;
               }
      -        return build(jobId, nodeGroup);
      +        return build(applicationId, nodeGroup);
           }
       
           @SuppressWarnings("unchecked")
      -    private ProcessorTopology build(String jobId, Set nodeGroup) {
      +    private ProcessorTopology build(String applicationId, Set nodeGroup) {
               List processorNodes = new ArrayList<>(nodeFactories.size());
               Map processorMap = new HashMap<>();
               Map topicSourceMap = new HashMap<>();
      @@ -658,7 +658,7 @@ private ProcessorTopology build(String jobId, Set nodeGroup) {
               // create processor nodes in a topological order ("nodeFactories" is already topologically sorted)
               for (NodeFactory factory : nodeFactories.values()) {
                   if (nodeGroup == null || nodeGroup.contains(factory.name)) {
      -                ProcessorNode node = factory.build(jobId);
      +                ProcessorNode node = factory.build(applicationId);
                       processorNodes.add(node);
                       processorMap.put(node.name(), node);
       
      @@ -674,8 +674,8 @@ private ProcessorTopology build(String jobId, Set nodeGroup) {
                       } else if (factory instanceof SourceNodeFactory) {
                           for (String topic : ((SourceNodeFactory) factory).topics) {
                               if (internalTopicNames.contains(topic)) {
      -                            // prefix the job id to the internal topic name
      -                            topicSourceMap.put(jobId + "-" + topic, (SourceNode) node);
      +                            // prefix the internal topic name with the application id
      +                            topicSourceMap.put(applicationId + "-" + topic, (SourceNode) node);
                               } else {
                                   topicSourceMap.put(topic, (SourceNode) node);
                               }
      @@ -697,11 +697,11 @@ private ProcessorTopology build(String jobId, Set nodeGroup) {
            * Get the names of topics that are to be consumed by the source nodes created by this builder.
            * @return the unmodifiable set of topic names used by source nodes, which changes as new sources are added; never null
            */
      -    public Set sourceTopics(String jobId) {
      +    public Set sourceTopics(String applicationId) {
               Set topics = new HashSet<>();
               for (String topic : sourceTopicNames) {
                   if (internalTopicNames.contains(topic)) {
      -                topics.add(jobId + "-" + topic);
      +                topics.add(applicationId + "-" + topic);
                   } else {
                       topics.add(topic);
                   }
      diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java
      index 8ff72bcd8a0ed..b3b653754e8ae 100644
      --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java
      +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java
      @@ -37,7 +37,7 @@
       
       public abstract class AbstractTask {
           protected final TaskId id;
      -    protected final String jobId;
      +    protected final String applicationId;
           protected final ProcessorTopology topology;
           protected final Consumer consumer;
           protected final ProcessorStateManager stateMgr;
      @@ -45,7 +45,7 @@ public abstract class AbstractTask {
           protected ProcessorContext processorContext;
       
           protected AbstractTask(TaskId id,
      -                           String jobId,
      +                           String applicationId,
                                  Collection partitions,
                                  ProcessorTopology topology,
                                  Consumer consumer,
      @@ -53,17 +53,17 @@ protected AbstractTask(TaskId id,
                                  StreamsConfig config,
                                  boolean isStandby) {
               this.id = id;
      -        this.jobId = jobId;
      +        this.applicationId = applicationId;
               this.partitions = new HashSet<>(partitions);
               this.topology = topology;
               this.consumer = consumer;
       
               // create the processor state manager
               try {
      -            File jobStateDir = StreamThread.makeStateDir(jobId, config.getString(StreamsConfig.STATE_DIR_CONFIG));
      -            File stateFile = new File(jobStateDir.getCanonicalPath(), id.toString());
      +            File applicationStateDir = StreamThread.makeStateDir(applicationId, config.getString(StreamsConfig.STATE_DIR_CONFIG));
      +            File stateFile = new File(applicationStateDir.getCanonicalPath(), id.toString());
                   // if partitions is null, this is a standby task
      -            this.stateMgr = new ProcessorStateManager(jobId, id.partition, partitions, stateFile, restoreConsumer, isStandby);
      +            this.stateMgr = new ProcessorStateManager(applicationId, id.partition, partitions, stateFile, restoreConsumer, isStandby);
               } catch (IOException e) {
                   throw new ProcessorStateException("Error while creating the state manager", e);
               }
      @@ -83,8 +83,8 @@ public final TaskId id() {
               return id;
           }
       
      -    public final String jobId() {
      -        return jobId;
      +    public final String applicationId() {
      +        return applicationId;
           }
       
           public final Set partitions() {
      diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
      index c4acc01aea9bc..f6e43d0f18a90 100644
      --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
      +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java
      @@ -27,15 +27,10 @@
       import org.apache.kafka.streams.processor.StateRestoreCallback;
       import org.apache.kafka.streams.processor.TaskId;
       
      -import org.slf4j.Logger;
      -import org.slf4j.LoggerFactory;
      -
       import java.io.File;
       
       public class ProcessorContextImpl implements ProcessorContext, RecordCollector.Supplier {
       
      -    private static final Logger log = LoggerFactory.getLogger(ProcessorContextImpl.class);
      -
           private final TaskId id;
           private final StreamTask task;
           private final StreamsMetrics metrics;
      @@ -84,8 +79,8 @@ public TaskId taskId() {
           }
       
           @Override
      -    public String jobId() {
      -        return task.jobId();
      +    public String applicationId() {
      +        return task.applicationId();
           }
       
           @Override
      diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java
      index c8f289e07dbdc..df8516ceb8fd8 100644
      --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java
      +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java
      @@ -51,7 +51,7 @@ public class ProcessorStateManager {
           public static final String CHECKPOINT_FILE_NAME = ".checkpoint";
           public static final String LOCK_FILE_NAME = ".lock";
       
      -    private final String jobId;
      +    private final String applicationId;
           private final int defaultPartition;
           private final Map partitionForTopic;
           private final File baseDir;
      @@ -65,8 +65,8 @@ public class ProcessorStateManager {
           private final boolean isStandby;
           private final Map restoreCallbacks; // used for standby tasks, keyed by state topic name
       
      -    public ProcessorStateManager(String jobId, int defaultPartition, Collection sources, File baseDir, Consumer restoreConsumer, boolean isStandby) throws IOException {
      -        this.jobId = jobId;
      +    public ProcessorStateManager(String applicationId, int defaultPartition, Collection sources, File baseDir, Consumer restoreConsumer, boolean isStandby) throws IOException {
      +        this.applicationId = applicationId;
               this.defaultPartition = defaultPartition;
               this.partitionForTopic = new HashMap<>();
               for (TopicPartition source : sources) {
      @@ -104,8 +104,8 @@ private static void createStateDirectory(File stateDir) throws IOException {
               }
           }
       
      -    public static String storeChangelogTopic(String jobId, String storeName) {
      -        return jobId + "-" + storeName + STATE_CHANGELOG_TOPIC_SUFFIX;
      +    public static String storeChangelogTopic(String applicationId, String storeName) {
      +        return applicationId + "-" + storeName + STATE_CHANGELOG_TOPIC_SUFFIX;
           }
       
           public static FileLock lockStateDirectory(File stateDir) throws IOException {
      @@ -154,7 +154,7 @@ public void register(StateStore store, boolean loggingEnabled, StateRestoreCallb
               // check that the underlying change log topic exist or not
               String topic;
               if (loggingEnabled)
      -            topic = storeChangelogTopic(this.jobId, store.name());
      +            topic = storeChangelogTopic(this.applicationId, store.name());
               else topic = store.name();
       
               // block until the partition is ready for this state changelog topic or time has elapsed
      @@ -325,7 +325,7 @@ public void close(Map ackedOffsets) throws IOException {
                       for (String storeName : stores.keySet()) {
                           TopicPartition part;
                           if (loggingEnabled.contains(storeName))
      -                        part = new TopicPartition(storeChangelogTopic(jobId, storeName), getPartition(storeName));
      +                        part = new TopicPartition(storeChangelogTopic(applicationId, storeName), getPartition(storeName));
                           else
                               part = new TopicPartition(storeName, getPartition(storeName));
       
      diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java
      index 82633b470c38c..0bcae18270e21 100644
      --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java
      +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java
      @@ -25,17 +25,13 @@
       import org.apache.kafka.streams.processor.StateRestoreCallback;
       import org.apache.kafka.streams.processor.StateStore;
       import org.apache.kafka.streams.processor.TaskId;
      -import org.slf4j.Logger;
      -import org.slf4j.LoggerFactory;
       
       import java.io.File;
       
       public class StandbyContextImpl implements ProcessorContext, RecordCollector.Supplier {
       
      -    private static final Logger log = LoggerFactory.getLogger(StandbyContextImpl.class);
      -
           private final TaskId id;
      -    private final String jobId;
      +    private final String applicationId;
           private final StreamsMetrics metrics;
           private final ProcessorStateManager stateMgr;
       
      @@ -47,12 +43,12 @@ public class StandbyContextImpl implements ProcessorContext, RecordCollector.Sup
           private boolean initialized;
       
           public StandbyContextImpl(TaskId id,
      -                              String jobId,
      +                              String applicationId,
                                     StreamsConfig config,
                                     ProcessorStateManager stateMgr,
                                     StreamsMetrics metrics) {
               this.id = id;
      -        this.jobId = jobId;
      +        this.applicationId = applicationId;
               this.metrics = metrics;
               this.stateMgr = stateMgr;
       
      @@ -78,8 +74,8 @@ public TaskId taskId() {
           }
       
           @Override
      -    public String jobId() {
      -        return jobId;
      +    public String applicationId() {
      +        return applicationId;
           }
       
           @Override
      diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java
      index da454cb9264e7..f19d5a3fe0b73 100644
      --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java
      +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyTask.java
      @@ -23,8 +23,6 @@
       import org.apache.kafka.streams.StreamsConfig;
       import org.apache.kafka.streams.StreamsMetrics;
       import org.apache.kafka.streams.processor.TaskId;
      -import org.slf4j.Logger;
      -import org.slf4j.LoggerFactory;
       
       import java.util.Collection;
       import java.util.Collections;
      @@ -36,15 +34,13 @@
        */
       public class StandbyTask extends AbstractTask {
       
      -    private static final Logger log = LoggerFactory.getLogger(StandbyTask.class);
      -
           private final Map checkpointedOffsets;
       
           /**
            * Create {@link StandbyTask} with its assigned partitions
            *
            * @param id                    the ID of this task
      -     * @param jobId                 the ID of the job
      +     * @param applicationId         the ID of the stream processing application
            * @param partitions            the collection of assigned {@link TopicPartition}
            * @param topology              the instance of {@link ProcessorTopology}
            * @param consumer              the instance of {@link Consumer}
      @@ -53,17 +49,17 @@ public class StandbyTask extends AbstractTask {
            * @param metrics               the {@link StreamsMetrics} created by the thread
            */
           public StandbyTask(TaskId id,
      -                       String jobId,
      +                       String applicationId,
                              Collection partitions,
                              ProcessorTopology topology,
                              Consumer consumer,
                              Consumer restoreConsumer,
                              StreamsConfig config,
                              StreamsMetrics metrics) {
      -        super(id, jobId, partitions, topology, consumer, restoreConsumer, config, true);
      +        super(id, applicationId, partitions, topology, consumer, restoreConsumer, config, true);
       
               // initialize the topology with its own context
      -        this.processorContext = new StandbyContextImpl(id, jobId, config, stateMgr, metrics);
      +        this.processorContext = new StandbyContextImpl(id, applicationId, config, stateMgr, metrics);
       
               initializeStateStores();
       
      diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java
      index 266df3ed09900..a6b82af02e28c 100644
      --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java
      +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java
      @@ -117,7 +117,7 @@ public void configure(Map configs) {
               streamThread = (StreamThread) o;
               streamThread.partitionAssignor(this);
       
      -        this.topicGroups = streamThread.builder.topicGroups(streamThread.jobId);
      +        this.topicGroups = streamThread.builder.topicGroups(streamThread.applicationId);
       
               if (configs.containsKey(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG)) {
                   internalTopicManager = new InternalTopicManager(
      @@ -445,7 +445,7 @@ private void ensureCopartitioning(Set copartitionGroup, Set inte
       
           /* For Test Only */
           public Set tasksForState(String stateName) {
      -        return stateChangelogTopicToTaskIds.get(ProcessorStateManager.storeChangelogTopic(streamThread.jobId, stateName));
      +        return stateChangelogTopicToTaskIds.get(ProcessorStateManager.storeChangelogTopic(streamThread.applicationId, stateName));
           }
       
           public Set tasksForPartition(TopicPartition partition) {
      diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
      index 4d663240b9f30..54a25c1aabfe1 100644
      --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
      +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java
      @@ -61,7 +61,7 @@ public class StreamTask extends AbstractTask implements Punctuator {
            * Create {@link StreamTask} with its assigned partitions
            *
            * @param id                    the ID of this task
      -     * @param jobId                 the ID of the job
      +     * @param applicationId         the ID of the stream processing application
            * @param partitions            the collection of assigned {@link TopicPartition}
            * @param topology              the instance of {@link ProcessorTopology}
            * @param consumer              the instance of {@link Consumer}
      @@ -71,7 +71,7 @@ public class StreamTask extends AbstractTask implements Punctuator {
            * @param metrics               the {@link StreamsMetrics} created by the thread
            */
           public StreamTask(TaskId id,
      -                      String jobId,
      +                      String applicationId,
                             Collection partitions,
                             ProcessorTopology topology,
                             Consumer consumer,
      @@ -79,7 +79,7 @@ public StreamTask(TaskId id,
                             Consumer restoreConsumer,
                             StreamsConfig config,
                             StreamsMetrics metrics) {
      -        super(id, jobId, partitions, topology, consumer, restoreConsumer, config, false);
      +        super(id, applicationId, partitions, topology, consumer, restoreConsumer, config, false);
               this.punctuationQueue = new PunctuationQueue();
               this.maxBufferedSize = config.getInt(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG);
       
      diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
      index e9343e0afeb82..491c8126fcc4c 100644
      --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
      +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java
      @@ -72,7 +72,7 @@ public class StreamThread extends Thread {
           private static final AtomicInteger STREAM_THREAD_ID_SEQUENCE = new AtomicInteger(1);
       
           public final PartitionGrouper partitionGrouper;
      -    public final String jobId;
      +    public final String applicationId;
           public final String clientId;
           public final UUID processId;
       
      @@ -106,12 +106,12 @@ public class StreamThread extends Thread {
           private Map>> standbyRecords;
           private boolean processStandbyRecords = false;
       
      -    static File makeStateDir(String jobId, String baseDirName) {
      +    static File makeStateDir(String applicationId, String baseDirName) {
               File baseDir = new File(baseDirName);
               if (!baseDir.exists())
                   baseDir.mkdir();
       
      -        File stateDir = new File(baseDir, jobId);
      +        File stateDir = new File(baseDir, applicationId);
               if (!stateDir.exists())
                   stateDir.mkdir();
       
      @@ -150,12 +150,12 @@ public void onPartitionsRevoked(Collection assignment) {
       
           public StreamThread(TopologyBuilder builder,
                               StreamsConfig config,
      -                        String jobId,
      +                        String applicationId,
                               String clientId,
                               UUID processId,
                               Metrics metrics,
                               Time time) {
      -        this(builder, config, null , null, null, jobId, clientId, processId, metrics, time);
      +        this(builder, config, null , null, null, applicationId, clientId, processId, metrics, time);
           }
       
           StreamThread(TopologyBuilder builder,
      @@ -163,17 +163,17 @@ public StreamThread(TopologyBuilder builder,
                        Producer producer,
                        Consumer consumer,
                        Consumer restoreConsumer,
      -                 String jobId,
      +                 String applicationId,
                        String clientId,
                        UUID processId,
                        Metrics metrics,
                        Time time) {
               super("StreamThread-" + STREAM_THREAD_ID_SEQUENCE.getAndIncrement());
       
      -        this.jobId = jobId;
      +        this.applicationId = applicationId;
               this.config = config;
               this.builder = builder;
      -        this.sourceTopics = builder.sourceTopics(jobId);
      +        this.sourceTopics = builder.sourceTopics(applicationId);
               this.clientId = clientId;
               this.processId = processId;
               this.partitionGrouper = config.getConfiguredInstance(StreamsConfig.PARTITION_GROUPER_CLASS_CONFIG, PartitionGrouper.class);
      @@ -194,7 +194,7 @@ public StreamThread(TopologyBuilder builder,
               this.standbyRecords = new HashMap<>();
       
               // read in task specific config values
      -        this.stateDir = makeStateDir(this.jobId, this.config.getString(StreamsConfig.STATE_DIR_CONFIG));
      +        this.stateDir = makeStateDir(this.applicationId, this.config.getString(StreamsConfig.STATE_DIR_CONFIG));
               this.pollTimeMs = config.getLong(StreamsConfig.POLL_MS_CONFIG);
               this.commitTimeMs = config.getLong(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG);
               this.cleanTimeMs = config.getLong(StreamsConfig.STATE_CLEANUP_DELAY_MS_CONFIG);
      @@ -224,7 +224,7 @@ private Producer createProducer() {
           private Consumer createConsumer() {
               String threadName = this.getName();
               log.info("Creating consumer client for stream thread [" + threadName + "]");
      -        return new KafkaConsumer<>(config.getConsumerConfigs(this, this.jobId, this.clientId + "-" + threadName),
      +        return new KafkaConsumer<>(config.getConsumerConfigs(this, this.applicationId, this.clientId + "-" + threadName),
                       new ByteArrayDeserializer(),
                       new ByteArrayDeserializer());
           }
      @@ -580,9 +580,9 @@ public Set cachedTasks() {
           protected StreamTask createStreamTask(TaskId id, Collection partitions) {
               sensors.taskCreationSensor.record();
       
      -        ProcessorTopology topology = builder.build(jobId, id.topicGroupId);
      +        ProcessorTopology topology = builder.build(applicationId, id.topicGroupId);
       
      -        return new StreamTask(id, jobId, partitions, topology, consumer, producer, restoreConsumer, config, sensors);
      +        return new StreamTask(id, applicationId, partitions, topology, consumer, producer, restoreConsumer, config, sensors);
           }
       
           private void addStreamTasks(Collection assignment) {
      @@ -650,10 +650,10 @@ private void closeOne(AbstractTask task) {
           protected StandbyTask createStandbyTask(TaskId id, Collection partitions) {
               sensors.taskCreationSensor.record();
       
      -        ProcessorTopology topology = builder.build(jobId, id.topicGroupId);
      +        ProcessorTopology topology = builder.build(applicationId, id.topicGroupId);
       
               if (!topology.stateStoreSuppliers().isEmpty()) {
      -            return new StandbyTask(id, jobId, partitions, topology, consumer, restoreConsumer, config, sensors);
      +            return new StandbyTask(id, applicationId, partitions, topology, consumer, restoreConsumer, config, sensors);
               } else {
                   return null;
               }
      diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java
      index aac4d8521328b..4229f940a00c6 100644
      --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java
      +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java
      @@ -57,7 +57,7 @@ public StoreChangeLogger(String storeName, ProcessorContext context, Serdes serialization, int maxDirty, int maxRemoved) {
      -        this.topic = ProcessorStateManager.storeChangelogTopic(context.jobId(), storeName);
      +        this.topic = ProcessorStateManager.storeChangelogTopic(context.applicationId(), storeName);
               this.context = context;
               this.partition = partition;
               this.serialization = serialization;
      diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
      index f0276ab42604d..83ebe48f1c1e3 100644
      --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
      +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java
      @@ -41,7 +41,7 @@ public class StreamsConfigTest {
       
           @Before
           public void setUp() {
      -        props.put(StreamsConfig.JOB_ID_CONFIG, "streams-config-test");
      +        props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-config-test");
               props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092");
               props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class);
               props.put(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class);
      @@ -59,9 +59,10 @@ public void testGetProducerConfigs() throws Exception {
       
           @Test
           public void testGetConsumerConfigs() throws Exception {
      -        Map returnedProps = streamsConfig.getConsumerConfigs(streamThreadPlaceHolder, "example-job", "client");
      +        Map returnedProps =
      +            streamsConfig.getConsumerConfigs(streamThreadPlaceHolder, "example-application", "client");
               assertEquals(returnedProps.get(ConsumerConfig.CLIENT_ID_CONFIG), "client-consumer");
      -        assertEquals(returnedProps.get(ConsumerConfig.GROUP_ID_CONFIG), "example-job");
      +        assertEquals(returnedProps.get(ConsumerConfig.GROUP_ID_CONFIG), "example-application");
       
           }
       
      diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java
      index 14cb4935b5a36..1d0a969865606 100644
      --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java
      +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java
      @@ -183,12 +183,12 @@ public synchronized void seekToEnd(TopicPartition... partitions) {
           }
       
           private final Set noPartitions = Collections.emptySet();
      -    private final String jobId = "test-job";
      +    private final String applicationId = "test-application";
           private final String stateDir = "test";
           private final String persistentStoreName = "persistentStore";
           private final String nonPersistentStoreName = "nonPersistentStore";
      -    private final String persistentStoreTopicName = ProcessorStateManager.storeChangelogTopic(jobId, persistentStoreName);
      -    private final String nonPersistentStoreTopicName = ProcessorStateManager.storeChangelogTopic(jobId, nonPersistentStoreName);
      +    private final String persistentStoreTopicName = ProcessorStateManager.storeChangelogTopic(applicationId, persistentStoreName);
      +    private final String nonPersistentStoreTopicName = ProcessorStateManager.storeChangelogTopic(applicationId, nonPersistentStoreName);
       
           @Test
           public void testLockStateDirectory() throws IOException {
      @@ -197,7 +197,7 @@ public void testLockStateDirectory() throws IOException {
                   FileLock lock;
       
                   // the state manager locks the directory
      -            ProcessorStateManager stateMgr = new ProcessorStateManager(jobId, 1, noPartitions, baseDir, new MockRestoreConsumer(), false);
      +            ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, 1, noPartitions, baseDir, new MockRestoreConsumer(), false);
       
                   try {
                       // this should not get the lock
      @@ -226,7 +226,7 @@ public void testNoTopic() throws IOException {
               try {
                   MockStateStoreSupplier.MockStateStore mockStateStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false);
       
      -            ProcessorStateManager stateMgr = new ProcessorStateManager(jobId, 1, noPartitions, baseDir, new MockRestoreConsumer(), false);
      +            ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, 1, noPartitions, baseDir, new MockRestoreConsumer(), false);
                   try {
                       stateMgr.register(mockStateStore, true, mockStateStore.stateRestoreCallback);
                   } finally {
      @@ -258,7 +258,7 @@ public void testRegisterPersistentStore() throws IOException {
       
                   MockStateStoreSupplier.MockStateStore persistentStore = new MockStateStoreSupplier.MockStateStore("persistentStore", true); // persistent store
       
      -            ProcessorStateManager stateMgr = new ProcessorStateManager(jobId, 2, noPartitions, baseDir, restoreConsumer, false);
      +            ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, 2, noPartitions, baseDir, restoreConsumer, false);
                   try {
                       restoreConsumer.reset();
       
      @@ -311,7 +311,7 @@ public void testRegisterNonPersistentStore() throws IOException {
       
                   MockStateStoreSupplier.MockStateStore nonPersistentStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false); // non persistent store
       
      -            ProcessorStateManager stateMgr = new ProcessorStateManager(jobId, 2, noPartitions, baseDir, restoreConsumer, false);
      +            ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, 2, noPartitions, baseDir, restoreConsumer, false);
                   try {
                       restoreConsumer.reset();
       
      @@ -351,9 +351,9 @@ public void testChangeLogOffsets() throws IOException {
                   String storeName2 = "store2";
                   String storeName3 = "store3";
       
      -            String storeTopicName1 = ProcessorStateManager.storeChangelogTopic(jobId, storeName1);
      -            String storeTopicName2 = ProcessorStateManager.storeChangelogTopic(jobId, storeName2);
      -            String storeTopicName3 = ProcessorStateManager.storeChangelogTopic(jobId, storeName3);
      +            String storeTopicName1 = ProcessorStateManager.storeChangelogTopic(applicationId, storeName1);
      +            String storeTopicName2 = ProcessorStateManager.storeChangelogTopic(applicationId, storeName2);
      +            String storeTopicName3 = ProcessorStateManager.storeChangelogTopic(applicationId, storeName3);
       
                   OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(baseDir, ProcessorStateManager.CHECKPOINT_FILE_NAME));
                   checkpoint.write(Collections.singletonMap(new TopicPartition(storeTopicName1, 0), lastCheckpointedOffset));
      @@ -386,7 +386,7 @@ public void testChangeLogOffsets() throws IOException {
       
                   // if there is an source partition, inherit the partition id
                   Set sourcePartitions = Utils.mkSet(new TopicPartition(storeTopicName3, 1));
      -            ProcessorStateManager stateMgr = new ProcessorStateManager(jobId, 0, sourcePartitions, baseDir, restoreConsumer, true); // standby
      +            ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, 0, sourcePartitions, baseDir, restoreConsumer, true); // standby
                   try {
                       restoreConsumer.reset();
       
      @@ -425,7 +425,7 @@ public void testGetStore() throws IOException {
       
                   MockStateStoreSupplier.MockStateStore mockStateStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false);
       
      -            ProcessorStateManager stateMgr = new ProcessorStateManager(jobId, 1, noPartitions, baseDir, restoreConsumer, false);
      +            ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, 1, noPartitions, baseDir, restoreConsumer, false);
                   try {
                       stateMgr.register(mockStateStore, true, mockStateStore.stateRestoreCallback);
       
      @@ -462,12 +462,12 @@ public void testClose() throws IOException {
                   HashMap ackedOffsets = new HashMap<>();
                   ackedOffsets.put(new TopicPartition(persistentStoreTopicName, 1), 123L);
                   ackedOffsets.put(new TopicPartition(nonPersistentStoreTopicName, 1), 456L);
      -            ackedOffsets.put(new TopicPartition(ProcessorStateManager.storeChangelogTopic(jobId, "otherTopic"), 1), 789L);
      +            ackedOffsets.put(new TopicPartition(ProcessorStateManager.storeChangelogTopic(applicationId, "otherTopic"), 1), 789L);
       
                   MockStateStoreSupplier.MockStateStore persistentStore = new MockStateStoreSupplier.MockStateStore(persistentStoreName, true);
                   MockStateStoreSupplier.MockStateStore nonPersistentStore = new MockStateStoreSupplier.MockStateStore(nonPersistentStoreName, false);
       
      -            ProcessorStateManager stateMgr = new ProcessorStateManager(jobId, 1, noPartitions, baseDir, restoreConsumer, false);
      +            ProcessorStateManager stateMgr = new ProcessorStateManager(applicationId, 1, noPartitions, baseDir, restoreConsumer, false);
                   try {
                       // make sure the checkpoint file is deleted
                       assertFalse(checkpointFile.exists());
      diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java
      index c8115b86896cd..12210cca8673a 100644
      --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java
      +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java
      @@ -67,7 +67,7 @@ public void setup() {
               // Create a new directory in which we'll put all of the state for this test, enabling running tests in parallel ...
               File localState = StateTestUtils.tempDir();
               Properties props = new Properties();
      -        props.setProperty(StreamsConfig.JOB_ID_CONFIG, "processor-topology-test");
      +        props.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "processor-topology-test");
               props.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9091");
               props.setProperty(StreamsConfig.STATE_DIR_CONFIG, localState.getAbsolutePath());
               props.setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, CustomTimestampExtractor.class.getName());
      diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java
      index 295f0dd416325..21bdaffbd8edf 100644
      --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java
      +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java
      @@ -57,11 +57,11 @@ public class StandbyTaskTest {
       
           private final Serializer intSerializer = new IntegerSerializer();
       
      -    private final String jobId = "test-job";
      +    private final String applicationId = "test-application";
           private final String storeName1 = "store1";
           private final String storeName2 = "store2";
      -    private final String storeChangelogTopicName1 = ProcessorStateManager.storeChangelogTopic(jobId, storeName1);
      -    private final String storeChangelogTopicName2 = ProcessorStateManager.storeChangelogTopic(jobId, storeName2);
      +    private final String storeChangelogTopicName1 = ProcessorStateManager.storeChangelogTopic(applicationId, storeName1);
      +    private final String storeChangelogTopicName2 = ProcessorStateManager.storeChangelogTopic(applicationId, storeName2);
       
           private final TopicPartition partition1 = new TopicPartition(storeChangelogTopicName1, 1);
           private final TopicPartition partition2 = new TopicPartition(storeChangelogTopicName2, 1);
      @@ -94,7 +94,7 @@ private StreamsConfig createConfig(final File baseDir) throws Exception {
                       setProperty(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer");
                       setProperty(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer");
                       setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, "org.apache.kafka.test.MockTimestampExtractor");
      -                setProperty(StreamsConfig.JOB_ID_CONFIG, jobId);
      +                setProperty(StreamsConfig.APPLICATION_ID_CONFIG, applicationId);
                       setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171");
                       setProperty(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3");
                       setProperty(StreamsConfig.STATE_DIR_CONFIG, baseDir.getCanonicalPath());
      @@ -133,7 +133,7 @@ public void testStorePartitions() throws Exception {
               File baseDir = Files.createTempDirectory("test").toFile();
               try {
                   StreamsConfig config = createConfig(baseDir);
      -            StandbyTask task = new StandbyTask(taskId, jobId, topicPartitions, topology, consumer, restoreStateConsumer, config, null);
      +            StandbyTask task = new StandbyTask(taskId, applicationId, topicPartitions, topology, consumer, restoreStateConsumer, config, null);
       
                   assertEquals(Utils.mkSet(partition2), new HashSet<>(task.changeLogPartitions()));
       
      @@ -148,7 +148,7 @@ public void testUpdateNonPersistentStore() throws Exception {
               File baseDir = Files.createTempDirectory("test").toFile();
               try {
                   StreamsConfig config = createConfig(baseDir);
      -            StandbyTask task = new StandbyTask(taskId, jobId, topicPartitions, topology, consumer, restoreStateConsumer, config, null);
      +            StandbyTask task = new StandbyTask(taskId, applicationId, topicPartitions, topology, consumer, restoreStateConsumer, config, null);
       
                   restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions()));
       
      @@ -167,7 +167,7 @@ public void testUpdate() throws Exception {
               File baseDir = Files.createTempDirectory("test").toFile();
               try {
                   StreamsConfig config = createConfig(baseDir);
      -            StandbyTask task = new StandbyTask(taskId, jobId, topicPartitions, topology, consumer, restoreStateConsumer, config, null);
      +            StandbyTask task = new StandbyTask(taskId, applicationId, topicPartitions, topology, consumer, restoreStateConsumer, config, null);
       
                   restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions()));
       
      @@ -201,7 +201,7 @@ public void testUpdate() throws Exception {
       
                   task.close();
       
      -            File taskDir = new File(StreamThread.makeStateDir(jobId, baseDir.getCanonicalPath()), taskId.toString());
      +            File taskDir = new File(StreamThread.makeStateDir(applicationId, baseDir.getCanonicalPath()), taskId.toString());
                   OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(taskDir, ProcessorStateManager.CHECKPOINT_FILE_NAME));
                   Map offsets = checkpoint.read();
       
      @@ -230,7 +230,7 @@ public void testUpdateKTable() throws Exception {
                   ));
       
                   StreamsConfig config = createConfig(baseDir);
      -            StandbyTask task = new StandbyTask(taskId, jobId, ktablePartitions, ktableTopology, consumer, restoreStateConsumer, config, null);
      +            StandbyTask task = new StandbyTask(taskId, applicationId, ktablePartitions, ktableTopology, consumer, restoreStateConsumer, config, null);
       
                   restoreStateConsumer.assign(new ArrayList<>(task.changeLogPartitions()));
       
      @@ -299,7 +299,7 @@ public void testUpdateKTable() throws Exception {
       
                   task.close();
       
      -            File taskDir = new File(StreamThread.makeStateDir(jobId, baseDir.getCanonicalPath()), taskId.toString());
      +            File taskDir = new File(StreamThread.makeStateDir(applicationId, baseDir.getCanonicalPath()), taskId.toString());
                   OffsetCheckpoint checkpoint = new OffsetCheckpoint(new File(taskDir, ProcessorStateManager.CHECKPOINT_FILE_NAME));
                   Map offsets = checkpoint.read();
       
      diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java
      index 7f37bdafaab97..a5990bdcf692d 100644
      --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java
      +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java
      @@ -94,7 +94,7 @@ private Properties configProps() {
                       setProperty(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer");
                       setProperty(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer");
                       setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, "org.apache.kafka.test.MockTimestampExtractor");
      -                setProperty(StreamsConfig.JOB_ID_CONFIG, "stream-partition-assignor-test");
      +                setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "stream-partition-assignor-test");
                       setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171");
                       setProperty(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3");
                   }
      diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
      index 1f401dbdb7ed2..f2ade6ba8c15b 100644
      --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
      +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java
      @@ -78,7 +78,7 @@ private StreamsConfig createConfig(final File baseDir) throws Exception {
                       setProperty(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer");
                       setProperty(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer");
                       setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, "org.apache.kafka.test.MockTimestampExtractor");
      -                setProperty(StreamsConfig.JOB_ID_CONFIG, "stream-task-test");
      +                setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "stream-task-test");
                       setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171");
                       setProperty(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3");
                       setProperty(StreamsConfig.STATE_DIR_CONFIG, baseDir.getCanonicalPath());
      @@ -105,7 +105,7 @@ public void testProcessOrder() throws Exception {
               File baseDir = Files.createTempDirectory("test").toFile();
               try {
                   StreamsConfig config = createConfig(baseDir);
      -            StreamTask task = new StreamTask(new TaskId(0, 0), "jobId", partitions, topology, consumer, producer, restoreStateConsumer, config, null);
      +            StreamTask task = new StreamTask(new TaskId(0, 0), "applicationId", partitions, topology, consumer, producer, restoreStateConsumer, config, null);
       
                   task.addRecords(partition1, records(
                           new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
      @@ -156,7 +156,7 @@ public void testPauseResume() throws Exception {
               File baseDir = Files.createTempDirectory("test").toFile();
               try {
                   StreamsConfig config = createConfig(baseDir);
      -            StreamTask task = new StreamTask(new TaskId(1, 1), "jobId", partitions, topology, consumer, producer, restoreStateConsumer, config, null);
      +            StreamTask task = new StreamTask(new TaskId(1, 1), "applicationId", partitions, topology, consumer, producer, restoreStateConsumer, config, null);
       
                   task.addRecords(partition1, records(
                           new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue),
      diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
      index eaaf842b4f221..b201c078dec57 100644
      --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
      +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java
      @@ -60,7 +60,7 @@
       public class StreamThreadTest {
       
           private final String clientId = "clientId";
      -    private final String jobId = "stream-thread-test";
      +    private final String applicationId = "stream-thread-test";
           private final UUID processId = UUID.randomUUID();
       
           private TopicPartition t1p1 = new TopicPartition("topic1", 1);
      @@ -118,7 +118,7 @@ private Properties configProps() {
                       setProperty(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer");
                       setProperty(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer");
                       setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, "org.apache.kafka.test.MockTimestampExtractor");
      -                setProperty(StreamsConfig.JOB_ID_CONFIG, jobId);
      +                setProperty(StreamsConfig.APPLICATION_ID_CONFIG, applicationId);
                       setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171");
                       setProperty(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3");
                   }
      @@ -129,14 +129,14 @@ private static class TestStreamTask extends StreamTask {
               public boolean committed = false;
       
               public TestStreamTask(TaskId id,
      -                              String jobId,
      +                              String applicationId,
                                     Collection partitions,
                                     ProcessorTopology topology,
                                     Consumer consumer,
                                     Producer producer,
                                     Consumer restoreConsumer,
                                     StreamsConfig config) {
      -            super(id, jobId, partitions, topology, consumer, producer, restoreConsumer, config, null);
      +            super(id, applicationId, partitions, topology, consumer, producer, restoreConsumer, config, null);
               }
       
               @Override
      @@ -168,11 +168,11 @@ public void testPartitionAssignmentChange() throws Exception {
               builder.addSource("source3", "topic3");
               builder.addProcessor("processor", new MockProcessorSupplier(), "source2", "source3");
       
      -        StreamThread thread = new StreamThread(builder, config, producer, consumer, mockRestoreConsumer, jobId, clientId, processId, new Metrics(), new SystemTime()) {
      +        StreamThread thread = new StreamThread(builder, config, producer, consumer, mockRestoreConsumer, applicationId, clientId, processId, new Metrics(), new SystemTime()) {
                   @Override
                   protected StreamTask createStreamTask(TaskId id, Collection partitionsForTask) {
                       ProcessorTopology topology = builder.build("X", id.topicGroupId);
      -                return new TestStreamTask(id, jobId, partitionsForTask, topology, consumer, producer, mockRestoreConsumer, config);
      +                return new TestStreamTask(id, applicationId, partitionsForTask, topology, consumer, producer, mockRestoreConsumer, config);
                   }
               };
       
      @@ -271,12 +271,12 @@ public void testMaybeClean() throws Exception {
       
                   StreamsConfig config = new StreamsConfig(props);
       
      -            File jobDir = new File(baseDir, jobId);
      -            jobDir.mkdir();
      -            File stateDir1 = new File(jobDir, task1.toString());
      -            File stateDir2 = new File(jobDir, task2.toString());
      -            File stateDir3 = new File(jobDir, task3.toString());
      -            File extraDir = new File(jobDir, "X");
      +            File applicationDir = new File(baseDir, applicationId);
      +            applicationDir.mkdir();
      +            File stateDir1 = new File(applicationDir, task1.toString());
      +            File stateDir2 = new File(applicationDir, task2.toString());
      +            File stateDir3 = new File(applicationDir, task3.toString());
      +            File extraDir = new File(applicationDir, "X");
                   stateDir1.mkdir();
                   stateDir2.mkdir();
                   stateDir3.mkdir();
      @@ -290,7 +290,7 @@ public void testMaybeClean() throws Exception {
                   TopologyBuilder builder = new TopologyBuilder();
                   builder.addSource("source1", "topic1");
       
      -            StreamThread thread = new StreamThread(builder, config, producer, consumer, mockRestoreConsumer, jobId, clientId,  processId, new Metrics(), mockTime) {
      +            StreamThread thread = new StreamThread(builder, config, producer, consumer, mockRestoreConsumer, applicationId, clientId,  processId, new Metrics(), mockTime) {
                       @Override
                       public void maybeClean() {
                           super.maybeClean();
      @@ -299,7 +299,7 @@ public void maybeClean() {
                       @Override
                       protected StreamTask createStreamTask(TaskId id, Collection partitionsForTask) {
                           ProcessorTopology topology = builder.build("X", id.topicGroupId);
      -                    return new TestStreamTask(id, jobId, partitionsForTask, topology, consumer, producer, mockRestoreConsumer, config);
      +                    return new TestStreamTask(id, applicationId, partitionsForTask, topology, consumer, producer, mockRestoreConsumer, config);
                       }
                   };
       
      @@ -412,7 +412,7 @@ public void testMaybeCommit() throws Exception {
                   TopologyBuilder builder = new TopologyBuilder();
                   builder.addSource("source1", "topic1");
       
      -            StreamThread thread = new StreamThread(builder, config, producer, consumer, mockRestoreConsumer, jobId, clientId,  processId, new Metrics(), mockTime) {
      +            StreamThread thread = new StreamThread(builder, config, producer, consumer, mockRestoreConsumer, applicationId, clientId,  processId, new Metrics(), mockTime) {
                       @Override
                       public void maybeCommit() {
                           super.maybeCommit();
      @@ -421,7 +421,7 @@ public void maybeCommit() {
                       @Override
                       protected StreamTask createStreamTask(TaskId id, Collection partitionsForTask) {
                           ProcessorTopology topology = builder.build("X", id.topicGroupId);
      -                    return new TestStreamTask(id, jobId, partitionsForTask, topology, consumer, producer, mockRestoreConsumer, config);
      +                    return new TestStreamTask(id, applicationId, partitionsForTask, topology, consumer, producer, mockRestoreConsumer, config);
                       }
                   };
       
      @@ -482,7 +482,7 @@ private void initPartitionGrouper(StreamsConfig config, StreamThread thread) {
       
               StreamPartitionAssignor partitionAssignor = new StreamPartitionAssignor();
       
      -        partitionAssignor.configure(config.getConsumerConfigs(thread, thread.jobId, thread.clientId));
      +        partitionAssignor.configure(config.getConsumerConfigs(thread, thread.applicationId, thread.clientId));
       
               Map assignments =
                       partitionAssignor.assign(metadata, Collections.singletonMap("client", subscription));
      diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java
      index 6cb45f33490bb..063eafeb04f5d 100644
      --- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java
      +++ b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java
      @@ -81,7 +81,7 @@ public void close() {
       
           private static KafkaStreams createKafkaStreams(File stateDir, String kafka, String zookeeper) {
               Properties props = new Properties();
      -        props.put(StreamsConfig.JOB_ID_CONFIG, "SmokeTest");
      +        props.put(StreamsConfig.APPLICATION_ID_CONFIG, "SmokeTest");
               props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString());
               props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka);
               props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, zookeeper);
      diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java
      index d597fd231cc4c..b463669595bd7 100644
      --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java
      +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java
      @@ -106,8 +106,8 @@ public TaskId taskId() {
           }
       
           @Override
      -    public String jobId() {
      -        return "mockJob";
      +    public String applicationId() {
      +        return "mockApplication";
           }
       
           @Override
      diff --git a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java
      index cf17dbe99a74b..a2948a260706a 100644
      --- a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java
      +++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java
      @@ -126,7 +126,7 @@ public class ProcessorTopologyTestDriver {
       
           private final Serializer bytesSerializer = new ByteArraySerializer();
       
      -    private final String jobId = "test-driver-job";
      +    private final String applicationId = "test-driver-application";
       
           private final TaskId id;
           private final ProcessorTopology topology;
      @@ -167,7 +167,7 @@ public List partitionsFor(String topic) {
               }
       
               task = new StreamTask(id,
      -            jobId,
      +            applicationId,
                   partitionsByTopic.values(),
                   topology,
                   consumer,
      @@ -334,7 +334,7 @@ public synchronized long position(TopicPartition partition) {
               };
               // For each store name ...
               for (String storeName : storeNames) {
      -            String topicName = ProcessorStateManager.storeChangelogTopic(jobId, storeName);
      +            String topicName = ProcessorStateManager.storeChangelogTopic(applicationId, storeName);
                   // Set up the restore-state topic ...
                   // consumer.subscribe(new TopicPartition(topicName, 1));
                   // Set up the partition that matches the ID (which is what ProcessorStateManager expects) ...
      
      From dce06766da245ca95951c9c7e82d6a113db7cb13 Mon Sep 17 00:00:00 2001
      From: Drausin Wulsin 
      Date: Thu, 17 Mar 2016 10:52:33 -0700
      Subject: [PATCH 041/206] KAFKA-3392: ConsumerRecords iterator throws
       NoSuchElementException when a TopicPartition is empty
      
      This contribution is my original work, and I license it under the project's open source license.
      
      CC jkreps
      
      Author: Drausin Wulsin 
      Author: John Doe 
      
      Reviewers: Jason Gustafson
      
      Closes #1055 from drausin/bugfix/consumer-records-iterator
      ---
       .../clients/consumer/ConsumerRecords.java     |  2 +-
       .../clients/consumer/ConsumerRecordsTest.java | 58 +++++++++++++++++++
       2 files changed, 59 insertions(+), 1 deletion(-)
       create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordsTest.java
      
      diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java
      index 3d7ec60438dce..5b83f0c677c66 100644
      --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java
      +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecords.java
      @@ -103,7 +103,7 @@ public Iterator> iterator() {
                       Iterator> current;
       
                       public ConsumerRecord makeNext() {
      -                    if (current == null || !current.hasNext()) {
      +                    while (current == null || !current.hasNext()) {
                               if (iters.hasNext())
                                   current = iters.next().iterator();
                               else
      diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordsTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordsTest.java
      new file mode 100644
      index 0000000000000..d68a341a55315
      --- /dev/null
      +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordsTest.java
      @@ -0,0 +1,58 @@
      +/**
      + * 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 org.apache.kafka.clients.consumer;
      +
      +import static org.junit.Assert.assertEquals;
      +
      +import java.util.ArrayList;
      +import java.util.Arrays;
      +import java.util.Iterator;
      +import java.util.LinkedHashMap;
      +import java.util.List;
      +import java.util.Map;
      +import org.apache.kafka.common.TopicPartition;
      +import org.apache.kafka.common.record.TimestampType;
      +import org.junit.Test;
      +
      +public class ConsumerRecordsTest {
      +
      +    @Test
      +    public void iterator() throws Exception {
      +
      +        Map>> records = new LinkedHashMap<>();
      +
      +        String topic = "topic";
      +        records.put(new TopicPartition(topic, 0), new ArrayList>());
      +        ConsumerRecord record1 = new ConsumerRecord<>(topic, 1, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, 1, "value1");
      +        ConsumerRecord record2 = new ConsumerRecord<>(topic, 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, 2, "value2");
      +        records.put(new TopicPartition(topic, 1), Arrays.asList(record1, record2));
      +        records.put(new TopicPartition(topic, 2), new ArrayList>());
      +
      +        ConsumerRecords consumerRecords = new ConsumerRecords<>(records);
      +        Iterator> iter = consumerRecords.iterator();
      +
      +        int c = 0;
      +        for (; iter.hasNext(); c++) {
      +            ConsumerRecord record = iter.next();
      +            assertEquals(1, record.partition());
      +            assertEquals(topic, record.topic());
      +            assertEquals(c, record.offset());
      +        }
      +        assertEquals(2, c);
      +    }
      +}
      \ No newline at end of file
      
      From 30e78fa00650b258f3ab5ef6c9bdf5ca137289c0 Mon Sep 17 00:00:00 2001
      From: edoardo 
      Date: Thu, 17 Mar 2016 12:33:47 -0700
      Subject: [PATCH 042/206] KAFKA-2832: Add a consumer config option to exclude
       internal topics
      
      A new consumer config option 'exclude.internal.topics' was added to
      allow excluding internal topics when wildcards are used to specify
      consumers.
      The new option takes a boolean value, with a default 'false' value (i.e.
      no exclusion).
      
      This patch is co-authored with rajinisivaram edoardocomar mimaison
      
      Author: edoardo 
      Author: Vahid Hashemian 
      
      Reviewers: Ismael Juma, Jun Rao, Gwen Shapira
      
      Closes #1082 from edoardocomar/KAFKA-2832
      ---
       .../clients/consumer/ConsumerConfig.java      | 11 ++++++
       .../kafka/clients/consumer/KafkaConsumer.java |  3 +-
       .../internals/ConsumerCoordinator.java        |  9 ++++-
       .../common/internals/TopicConstants.java      | 33 +++++++++++++++++
       .../internals/ConsumerCoordinatorTest.java    | 37 +++++++++++++++----
       .../main/scala/kafka/admin/TopicCommand.scala |  7 ++--
       core/src/main/scala/kafka/common/Topic.scala  |  2 -
       .../scala/kafka/consumer/TopicFilter.scala    |  5 ++-
       .../kafka/coordinator/GroupCoordinator.scala  |  3 --
       .../coordinator/GroupMetadataManager.scala    | 30 +++++++--------
       .../main/scala/kafka/server/KafkaApis.scala   | 11 +++---
       .../scala/kafka/server/ReplicaManager.scala   |  5 +--
       .../kafka/api/AuthorizerIntegrationTest.scala |  5 +--
       .../kafka/api/BaseConsumerTest.scala          |  7 +---
       .../kafka/api/IntegrationTestHarness.scala    |  4 +-
       .../api/ProducerFailureHandlingTest.scala     |  4 +-
       .../unit/kafka/admin/TopicCommandTest.scala   |  9 +++--
       .../unit/kafka/consumer/TopicFilterTest.scala |  9 +++--
       .../GroupCoordinatorResponseTest.scala        | 14 +++----
       19 files changed, 135 insertions(+), 73 deletions(-)
       create mode 100644 clients/src/main/java/org/apache/kafka/common/internals/TopicConstants.java
      
      diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
      index bd9efc39671a1..9101307cc0c5a 100644
      --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
      +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java
      @@ -172,6 +172,12 @@ public class ConsumerConfig extends AbstractConfig {
           public static final String MAX_POLL_RECORDS_CONFIG = "max.poll.records";
           private static final String MAX_POLL_RECORDS_DOC = "The maximum number of records returned in a single call to poll().";
       
      +    /** exclude.internal.topics */
      +    public static final String EXCLUDE_INTERNAL_TOPICS_CONFIG = "exclude.internal.topics";
      +    private static final String EXCLUDE_INTERNAL_TOPICS_DOC = "Whether records from internal topics (such as offsets) should be exposed to the consumer. "
      +                                                            + "If set to true the only way to receive records from an internal topic is subscribing to it.";
      +    public static final boolean EXCLUDE_INTERNAL_TOPICS_DEFAULT = true;
      +    
           static {
               CONFIG = new ConfigDef().define(BOOTSTRAP_SERVERS_CONFIG,
                                               Type.LIST,
      @@ -316,6 +322,11 @@ public class ConsumerConfig extends AbstractConfig {
                                               atLeast(1),
                                               Importance.MEDIUM,
                                               MAX_POLL_RECORDS_DOC)
      +                                .define(EXCLUDE_INTERNAL_TOPICS_CONFIG,
      +                                        Type.BOOLEAN,
      +                                        EXCLUDE_INTERNAL_TOPICS_DEFAULT,
      +                                        Importance.MEDIUM,
      +                                        EXCLUDE_INTERNAL_TOPICS_DOC)
       
                                       // security support
                                       .define(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG,
      diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
      index d7c8e14cea0cb..804a16036101e 100644
      --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
      +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java
      @@ -612,7 +612,8 @@ private KafkaConsumer(ConsumerConfig config,
                           new ConsumerCoordinator.DefaultOffsetCommitCallback(),
                           config.getBoolean(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG),
                           config.getLong(ConsumerConfig.AUTO_COMMIT_INTERVAL_MS_CONFIG),
      -                    this.interceptors);
      +                    this.interceptors,
      +                    config.getBoolean(ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_CONFIG));
                   if (keyDeserializer == null) {
                       this.keyDeserializer = config.getConfiguredInstance(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG,
                               Deserializer.class);
      diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
      index 2ae1437336af3..cf935309cd383 100644
      --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
      +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java
      @@ -26,6 +26,7 @@
       import org.apache.kafka.common.errors.GroupAuthorizationException;
       import org.apache.kafka.common.errors.TopicAuthorizationException;
       import org.apache.kafka.common.errors.WakeupException;
      +import org.apache.kafka.common.internals.TopicConstants;
       import org.apache.kafka.common.metrics.Measurable;
       import org.apache.kafka.common.metrics.MetricConfig;
       import org.apache.kafka.common.metrics.Metrics;
      @@ -69,6 +70,7 @@ public final class ConsumerCoordinator extends AbstractCoordinator {
           private final boolean autoCommitEnabled;
           private final AutoCommitTask autoCommitTask;
           private final ConsumerInterceptors interceptors;
      +    private final boolean excludeInternalTopics;
       
           /**
            * Initialize the coordination manager.
      @@ -87,7 +89,8 @@ public ConsumerCoordinator(ConsumerNetworkClient client,
                                      OffsetCommitCallback defaultOffsetCommitCallback,
                                      boolean autoCommitEnabled,
                                      long autoCommitIntervalMs,
      -                               ConsumerInterceptors interceptors) {
      +                               ConsumerInterceptors interceptors,
      +                               boolean excludeInternalTopics) {
               super(client,
                       groupId,
                       sessionTimeoutMs,
      @@ -110,6 +113,7 @@ public ConsumerCoordinator(ConsumerNetworkClient client,
               this.autoCommitTask = autoCommitEnabled ? new AutoCommitTask(autoCommitIntervalMs) : null;
               this.sensors = new ConsumerCoordinatorMetrics(metrics, metricGrpPrefix);
               this.interceptors = interceptors;
      +        this.excludeInternalTopics = excludeInternalTopics;
           }
       
           @Override
      @@ -140,7 +144,8 @@ public void onMetadataUpdate(Cluster cluster) {
                           final List topicsToSubscribe = new ArrayList<>();
       
                           for (String topic : cluster.topics())
      -                        if (subscriptions.getSubscribedPattern().matcher(topic).matches())
      +                        if (subscriptions.getSubscribedPattern().matcher(topic).matches() &&
      +                                !(excludeInternalTopics && TopicConstants.INTERNAL_TOPICS.contains(topic)))
                                   topicsToSubscribe.add(topic);
       
                           subscriptions.changeSubscription(topicsToSubscribe);
      diff --git a/clients/src/main/java/org/apache/kafka/common/internals/TopicConstants.java b/clients/src/main/java/org/apache/kafka/common/internals/TopicConstants.java
      new file mode 100644
      index 0000000000000..5d6b9928fff98
      --- /dev/null
      +++ b/clients/src/main/java/org/apache/kafka/common/internals/TopicConstants.java
      @@ -0,0 +1,33 @@
      +/**
      + * Licensed to the Apache Software Foundation (ASF) under one or more
      + * contributor license agreements.  See the NOTICE file distributed with
      + * this work for additional information regarding copyright ownership.
      + * The ASF licenses this file to You under the Apache License, Version 2.0
      + * (the "License"); you may not use this file except in compliance with
      + * the License.  You may obtain a copy of the License at
      + *
      + *    http://www.apache.org/licenses/LICENSE-2.0
      + *
      + * Unless required by applicable law or agreed to in writing, software
      + * distributed under the License is distributed on an "AS IS" BASIS,
      + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
      + * See the License for the specific language governing permissions and
      + * limitations under the License.
      + */
      +package org.apache.kafka.common.internals;
      +
      +import java.util.Arrays;
      +import java.util.Collection;
      +import java.util.Collections;
      +import java.util.HashSet;
      +
      +public final class TopicConstants {
      +
      +    //avoid instantiation
      +    private TopicConstants() {
      +    }
      +
      +    // TODO: we store both group metadata and offset data here despite the topic name being offsets only
      +    public static final String GROUP_METADATA_TOPIC_NAME = "__consumer_offsets";
      +    public static final Collection INTERNAL_TOPICS = Collections.unmodifiableSet(new HashSet(Arrays.asList(GROUP_METADATA_TOPIC_NAME)));
      +}
      diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
      index 0b8a1626c52aa..260ee7a081ed1 100644
      --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
      +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java
      @@ -20,6 +20,7 @@
       import org.apache.kafka.clients.Metadata;
       import org.apache.kafka.clients.MockClient;
       import org.apache.kafka.clients.consumer.CommitFailedException;
      +import org.apache.kafka.clients.consumer.ConsumerConfig;
       import org.apache.kafka.clients.consumer.ConsumerRebalanceListener;
       import org.apache.kafka.clients.consumer.OffsetAndMetadata;
       import org.apache.kafka.clients.consumer.OffsetCommitCallback;
      @@ -34,6 +35,7 @@
       import org.apache.kafka.common.errors.DisconnectException;
       import org.apache.kafka.common.errors.GroupAuthorizationException;
       import org.apache.kafka.common.errors.OffsetMetadataTooLarge;
      +import org.apache.kafka.common.internals.TopicConstants;
       import org.apache.kafka.common.metrics.Metrics;
       import org.apache.kafka.common.protocol.Errors;
       import org.apache.kafka.common.protocol.types.Struct;
      @@ -63,6 +65,7 @@
       import java.util.List;
       import java.util.Map;
       import java.util.concurrent.atomic.AtomicBoolean;
      +import java.util.regex.Pattern;
       
       import static org.junit.Assert.assertEquals;
       import static org.junit.Assert.assertFalse;
      @@ -107,7 +110,7 @@ public void setup() {
               this.partitionAssignor.clear();
       
               client.setNode(node);
      -        this.coordinator = buildCoordinator(metrics, assignors);
      +        this.coordinator = buildCoordinator(metrics, assignors, ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_DEFAULT);
           }
       
           @After
      @@ -263,7 +266,7 @@ public void testCoordinatorDisconnect() {
           }
       
           @Test(expected = ApiException.class)
      -    public void testJoinGroupInvalidGroupId() {
      +    public void testJoinGroupInvalidGroupId() { 
               final String consumerId = "leader";
       
               subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener);
      @@ -509,7 +512,7 @@ public boolean matches(ClientRequest request) {
           }
       
           @Test
      -    public void testMetadataChangeTriggersRebalance() {
      +    public void testMetadataChangeTriggersRebalance() { 
               final String consumerId = "consumer";
       
               subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener);
      @@ -532,6 +535,25 @@ public void testMetadataChangeTriggersRebalance() {
               assertTrue(subscriptions.partitionAssignmentNeeded());
           }
       
      +    @Test
      +    public void testExcludeInternalTopicsConfigOption() { 
      +        subscriptions.subscribe(Pattern.compile(".*"), rebalanceListener);
      +
      +        metadata.update(TestUtils.singletonCluster(TopicConstants.GROUP_METADATA_TOPIC_NAME, 2), time.milliseconds());
      +
      +        assertFalse(subscriptions.partitionAssignmentNeeded());
      +    }
      +
      +    @Test
      +    public void testIncludeInternalTopicsConfigOption() {
      +        coordinator = buildCoordinator(new Metrics(), assignors, false);
      +        subscriptions.subscribe(Pattern.compile(".*"), rebalanceListener);
      +
      +        metadata.update(TestUtils.singletonCluster(TopicConstants.GROUP_METADATA_TOPIC_NAME, 2), time.milliseconds());
      +
      +        assertTrue(subscriptions.partitionAssignmentNeeded());
      +    }
      +    
           @Test
           public void testRejoinGroup() {
               subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener);
      @@ -882,7 +904,7 @@ public void testProtocolMetadataOrder() {
               RangeAssignor range = new RangeAssignor();
       
               try (Metrics metrics = new Metrics(time)) {
      -            ConsumerCoordinator coordinator = buildCoordinator(metrics, Arrays.asList(roundRobin, range));
      +            ConsumerCoordinator coordinator = buildCoordinator(metrics, Arrays.asList(roundRobin, range), ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_DEFAULT);
                   List metadata = coordinator.metadata();
                   assertEquals(2, metadata.size());
                   assertEquals(roundRobin.name(), metadata.get(0).name());
      @@ -890,7 +912,7 @@ public void testProtocolMetadataOrder() {
               }
       
               try (Metrics metrics = new Metrics(time)) {
      -            ConsumerCoordinator coordinator = buildCoordinator(metrics, Arrays.asList(range, roundRobin));
      +            ConsumerCoordinator coordinator = buildCoordinator(metrics, Arrays.asList(range, roundRobin), ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_DEFAULT);
                   List metadata = coordinator.metadata();
                   assertEquals(2, metadata.size());
                   assertEquals(range.name(), metadata.get(0).name());
      @@ -898,7 +920,7 @@ public void testProtocolMetadataOrder() {
               }
           }
       
      -    private ConsumerCoordinator buildCoordinator(Metrics metrics, List assignors) {
      +    private ConsumerCoordinator buildCoordinator(Metrics metrics, List assignors, boolean excludeInternalTopics) {
               return new ConsumerCoordinator(
                       consumerClient,
                       groupId,
      @@ -914,7 +936,8 @@ private ConsumerCoordinator buildCoordinator(Metrics metrics, List
             try {
      -        if (Topic.InternalTopics.contains(topic)) {
      +        if (TopicConstants.INTERNAL_TOPICS.contains(topic)) {
                 throw new AdminOperationException("Topic %s is a kafka internal topic and is not allowed to be marked for deletion.".format(topic))
               } else {
                 zkUtils.createPersistentPath(getDeleteTopicPath(topic))
      diff --git a/core/src/main/scala/kafka/common/Topic.scala b/core/src/main/scala/kafka/common/Topic.scala
      index 55d2bdb492246..930d0e441568c 100644
      --- a/core/src/main/scala/kafka/common/Topic.scala
      +++ b/core/src/main/scala/kafka/common/Topic.scala
      @@ -25,8 +25,6 @@ object Topic {
         private val maxNameLength = 255
         private val rgx = new Regex(legalChars + "+")
       
      -  val InternalTopics = Set(GroupCoordinator.GroupMetadataTopicName)
      -
         def validate(topic: String) {
           if (topic.length <= 0)
             throw new org.apache.kafka.common.errors.InvalidTopicException("topic name is illegal, can't be empty")
      diff --git a/core/src/main/scala/kafka/consumer/TopicFilter.scala b/core/src/main/scala/kafka/consumer/TopicFilter.scala
      index 5a13540699be4..b89968e86cdde 100644
      --- a/core/src/main/scala/kafka/consumer/TopicFilter.scala
      +++ b/core/src/main/scala/kafka/consumer/TopicFilter.scala
      @@ -21,6 +21,7 @@ package kafka.consumer
       import kafka.utils.Logging
       import java.util.regex.{PatternSyntaxException, Pattern}
       import kafka.common.Topic
      +import org.apache.kafka.common.internals.TopicConstants
       
       
       sealed abstract class TopicFilter(rawRegex: String) extends Logging {
      @@ -47,7 +48,7 @@ sealed abstract class TopicFilter(rawRegex: String) extends Logging {
       
       case class Whitelist(rawRegex: String) extends TopicFilter(rawRegex) {
         override def isTopicAllowed(topic: String, excludeInternalTopics: Boolean) = {
      -    val allowed = topic.matches(regex) && !(Topic.InternalTopics.contains(topic) && excludeInternalTopics)
      +    val allowed = topic.matches(regex) && !(TopicConstants.INTERNAL_TOPICS.contains(topic) && excludeInternalTopics)
       
           debug("%s %s".format(
             topic, if (allowed) "allowed" else "filtered"))
      @@ -60,7 +61,7 @@ case class Whitelist(rawRegex: String) extends TopicFilter(rawRegex) {
       
       case class Blacklist(rawRegex: String) extends TopicFilter(rawRegex) {
         override def isTopicAllowed(topic: String, excludeInternalTopics: Boolean) = {
      -    val allowed = (!topic.matches(regex)) && !(Topic.InternalTopics.contains(topic) && excludeInternalTopics)
      +    val allowed = (!topic.matches(regex)) && !(TopicConstants.INTERNAL_TOPICS.contains(topic) && excludeInternalTopics)
       
           debug("%s %s".format(
             topic, if (allowed) "allowed" else "filtered"))
      diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
      index 36d7bbb969276..30a3a78f6b131 100644
      --- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
      +++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
      @@ -725,9 +725,6 @@ object GroupCoordinator {
         val EmptyGroup = GroupSummary(NoState, NoProtocolType, NoProtocol, NoMembers)
         val DeadGroup = GroupSummary(Dead.toString, NoProtocolType, NoProtocol, NoMembers)
       
      -  // TODO: we store both group metadata and offset data here despite the topic name being offsets only
      -  val GroupMetadataTopicName = "__consumer_offsets"
      -
         def apply(config: KafkaConfig,
                   zkUtils: ZkUtils,
                   replicaManager: ReplicaManager,
      diff --git a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala
      index 2c0236ee0cd66..c6bc44ef5efb5 100644
      --- a/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala
      +++ b/core/src/main/scala/kafka/coordinator/GroupMetadataManager.scala
      @@ -18,7 +18,6 @@
       package kafka.coordinator
       
       import java.util.concurrent.locks.ReentrantReadWriteLock
      -
       import kafka.utils.CoreUtils._
       import org.apache.kafka.common.TopicPartition
       import org.apache.kafka.common.protocol.Errors
      @@ -40,14 +39,13 @@ import kafka.metrics.KafkaMetricsGroup
       import kafka.common.TopicAndPartition
       import kafka.common.MessageFormatter
       import kafka.server.ReplicaManager
      -
       import scala.collection._
       import java.io.PrintStream
       import java.nio.ByteBuffer
       import java.util.concurrent.atomic.AtomicBoolean
       import java.util.concurrent.TimeUnit
      -
       import com.yammer.metrics.core.Gauge
      +import org.apache.kafka.common.internals.TopicConstants
       
       case class DelayedStore(messageSet: Map[TopicPartition, MessageSet],
                               callback: Map[TopicPartition, PartitionResponse] => Unit)
      @@ -147,9 +145,9 @@ class GroupMetadataManager(val brokerId: Int,
             val tombstone = new Message(bytes = null, key = GroupMetadataManager.groupMetadataKey(group.groupId),
               timestamp = timestamp, magicValue = magicValue)
       
      -      val partitionOpt = replicaManager.getPartition(GroupCoordinator.GroupMetadataTopicName, groupPartition)
      +      val partitionOpt = replicaManager.getPartition(TopicConstants.GROUP_METADATA_TOPIC_NAME, groupPartition)
             partitionOpt.foreach { partition =>
      -        val appendPartition = TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, groupPartition)
      +        val appendPartition = TopicAndPartition(TopicConstants.GROUP_METADATA_TOPIC_NAME, groupPartition)
       
               trace("Marking group %s as deleted.".format(group.groupId))
       
      @@ -177,7 +175,7 @@ class GroupMetadataManager(val brokerId: Int,
             timestamp = timestamp,
             magicValue = magicValue)
       
      -    val groupMetadataPartition = new TopicPartition(GroupCoordinator.GroupMetadataTopicName, partitionFor(group.groupId))
      +    val groupMetadataPartition = new TopicPartition(TopicConstants.GROUP_METADATA_TOPIC_NAME, partitionFor(group.groupId))
       
           val groupMetadataMessageSet = Map(groupMetadataPartition ->
             new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, message))
      @@ -263,7 +261,7 @@ class GroupMetadataManager(val brokerId: Int,
             )
           }.toSeq
       
      -    val offsetTopicPartition = new TopicPartition(GroupCoordinator.GroupMetadataTopicName, partitionFor(groupId))
      +    val offsetTopicPartition = new TopicPartition(TopicConstants.GROUP_METADATA_TOPIC_NAME, partitionFor(groupId))
       
           val offsetsAndMetadataMessageSet = Map(offsetTopicPartition ->
             new ByteBufferMessageSet(config.offsetsTopicCompressionCodec, messages:_*))
      @@ -351,7 +349,7 @@ class GroupMetadataManager(val brokerId: Int,
          */
         def loadGroupsForPartition(offsetsPartition: Int,
                                    onGroupLoaded: GroupMetadata => Unit) {
      -    val topicPartition = TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, offsetsPartition)
      +    val topicPartition = TopicAndPartition(TopicConstants.GROUP_METADATA_TOPIC_NAME, offsetsPartition)
           scheduler.schedule(topicPartition.toString, loadGroupsAndOffsets)
       
           def loadGroupsAndOffsets() {
      @@ -470,7 +468,7 @@ class GroupMetadataManager(val brokerId: Int,
          */
         def removeGroupsForPartition(offsetsPartition: Int,
                                      onGroupUnloaded: GroupMetadata => Unit) {
      -    val topicPartition = TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, offsetsPartition)
      +    val topicPartition = TopicAndPartition(TopicConstants.GROUP_METADATA_TOPIC_NAME, offsetsPartition)
           scheduler.schedule(topicPartition.toString, removeGroupsAndOffsets)
       
           def removeGroupsAndOffsets() {
      @@ -507,10 +505,10 @@ class GroupMetadataManager(val brokerId: Int,
             }
       
             if (numOffsetsRemoved > 0) info("Removed %d cached offsets for %s on follower transition."
      -        .format(numOffsetsRemoved, TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, offsetsPartition)))
      +        .format(numOffsetsRemoved, TopicAndPartition(TopicConstants.GROUP_METADATA_TOPIC_NAME, offsetsPartition)))
       
             if (numGroupsRemoved > 0) info("Removed %d cached groups for %s on follower transition."
      -        .format(numGroupsRemoved, TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, offsetsPartition)))
      +        .format(numGroupsRemoved, TopicAndPartition(TopicConstants.GROUP_METADATA_TOPIC_NAME, offsetsPartition)))
           }
         }
       
      @@ -566,9 +564,9 @@ class GroupMetadataManager(val brokerId: Int,
             // Append the tombstone messages to the offset partitions. It is okay if the replicas don't receive these (say,
             // if we crash or leaders move) since the new leaders will get rid of expired offsets during their own purge cycles.
             tombstonesForPartition.flatMap { case (offsetsPartition, tombstones) =>
      -        val partitionOpt = replicaManager.getPartition(GroupCoordinator.GroupMetadataTopicName, offsetsPartition)
      +        val partitionOpt = replicaManager.getPartition(TopicConstants.GROUP_METADATA_TOPIC_NAME, offsetsPartition)
               partitionOpt.map { partition =>
      -          val appendPartition = TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, offsetsPartition)
      +          val appendPartition = TopicAndPartition(TopicConstants.GROUP_METADATA_TOPIC_NAME, offsetsPartition)
                 val messages = tombstones.map(_._2).toSeq
       
                 trace("Marked %d offsets in %s for deletion.".format(messages.size, appendPartition))
      @@ -593,7 +591,7 @@ class GroupMetadataManager(val brokerId: Int,
         }
       
         private def getHighWatermark(partitionId: Int): Long = {
      -    val partitionOpt = replicaManager.getPartition(GroupCoordinator.GroupMetadataTopicName, partitionId)
      +    val partitionOpt = replicaManager.getPartition(TopicConstants.GROUP_METADATA_TOPIC_NAME, partitionId)
       
           val hw = partitionOpt.map { partition =>
             partition.leaderReplicaIfLocal().map(_.highWatermark.messageOffset).getOrElse(-1L)
      @@ -621,7 +619,7 @@ class GroupMetadataManager(val brokerId: Int,
          * If the topic does not exist, the configured partition count is returned.
          */
         private def getOffsetsTopicPartitionCount = {
      -    val topic = GroupCoordinator.GroupMetadataTopicName
      +    val topic = TopicConstants.GROUP_METADATA_TOPIC_NAME
           val topicData = zkUtils.getPartitionAssignmentForTopics(Seq(topic))
           if (topicData(topic).nonEmpty)
             topicData(topic).size
      @@ -630,7 +628,7 @@ class GroupMetadataManager(val brokerId: Int,
         }
       
         private def getMessageFormatVersionAndTimestamp(partition: Int): (Byte, Long) = {
      -    val groupMetadataTopicAndPartition = new TopicAndPartition(GroupCoordinator.GroupMetadataTopicName, partition)
      +    val groupMetadataTopicAndPartition = new TopicAndPartition(TopicConstants.GROUP_METADATA_TOPIC_NAME, partition)
           val messageFormatVersion = replicaManager.getMessageFormatVersion(groupMetadataTopicAndPartition).getOrElse {
             throw new IllegalArgumentException(s"Message format version for partition $groupMetadataTopicPartitionCount not found")
           }
      diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala
      index 452f721c0a583..0fb4d74cab576 100644
      --- a/core/src/main/scala/kafka/server/KafkaApis.scala
      +++ b/core/src/main/scala/kafka/server/KafkaApis.scala
      @@ -45,6 +45,7 @@ MetadataRequest, MetadataResponse, OffsetCommitRequest, OffsetCommitResponse, Of
       import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
       import org.apache.kafka.common.utils.Utils
       import org.apache.kafka.common.{TopicPartition, Node}
      +import org.apache.kafka.common.internals.TopicConstants
       
       import scala.collection._
       import scala.collection.JavaConverters._
      @@ -129,11 +130,11 @@ class KafkaApis(val requestChannel: RequestChannel,
               // this callback is invoked under the replica state change lock to ensure proper order of
               // leadership changes
               updatedLeaders.foreach { partition =>
      -          if (partition.topic == GroupCoordinator.GroupMetadataTopicName)
      +          if (partition.topic == TopicConstants.GROUP_METADATA_TOPIC_NAME)
                   coordinator.handleGroupImmigration(partition.partitionId)
               }
               updatedFollowers.foreach { partition =>
      -          if (partition.topic == GroupCoordinator.GroupMetadataTopicName)
      +          if (partition.topic == TopicConstants.GROUP_METADATA_TOPIC_NAME)
                   coordinator.handleGroupEmigration(partition.partitionId)
               }
             }
      @@ -643,12 +644,12 @@ class KafkaApis(val requestChannel: RequestChannel,
               Math.min(config.offsetsTopicReplicationFactor.toInt, aliveBrokers.length)
             else
               config.offsetsTopicReplicationFactor.toInt
      -    createTopic(GroupCoordinator.GroupMetadataTopicName, config.offsetsTopicPartitions,
      +    createTopic(TopicConstants.GROUP_METADATA_TOPIC_NAME, config.offsetsTopicPartitions,
             offsetsTopicReplicationFactor, coordinator.offsetsTopicConfigs)
         }
       
         private def getOrCreateGroupMetadataTopic(securityProtocol: SecurityProtocol): MetadataResponse.TopicMetadata = {
      -    val topicMetadata = metadataCache.getTopicMetadata(Set(GroupCoordinator.GroupMetadataTopicName), securityProtocol)
      +    val topicMetadata = metadataCache.getTopicMetadata(Set(TopicConstants.GROUP_METADATA_TOPIC_NAME), securityProtocol)
           topicMetadata.headOption.getOrElse(createGroupMetadataTopic())
         }
       
      @@ -659,7 +660,7 @@ class KafkaApis(val requestChannel: RequestChannel,
           } else {
             val nonExistentTopics = topics -- topicResponses.map(_.topic).toSet
             val responsesForNonExistentTopics = nonExistentTopics.map { topic =>
      -        if (topic == GroupCoordinator.GroupMetadataTopicName) {
      +        if (topic == TopicConstants.GROUP_METADATA_TOPIC_NAME) {
                 createGroupMetadataTopic()
               } else if (config.autoCreateTopicsEnable) {
                 createTopic(topic, config.numPartitions, config.defaultReplicationFactor)
      diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala
      index de58e569ddf72..f050e27c7e13f 100644
      --- a/core/src/main/scala/kafka/server/ReplicaManager.scala
      +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
      @@ -19,7 +19,6 @@ package kafka.server
       import java.io.{File, IOException}
       import java.util.concurrent.TimeUnit
       import java.util.concurrent.atomic.{AtomicBoolean, AtomicLong}
      -
       import com.yammer.metrics.core.Gauge
       import kafka.api._
       import kafka.cluster.{Partition, Replica}
      @@ -38,9 +37,9 @@ import org.apache.kafka.common.protocol.Errors
       import org.apache.kafka.common.requests.{LeaderAndIsrRequest, StopReplicaRequest, UpdateMetadataRequest}
       import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
       import org.apache.kafka.common.utils.{Time => JTime}
      -
       import scala.collection._
       import scala.collection.JavaConverters._
      +import org.apache.kafka.common.internals.TopicConstants
       
       /*
        * Result metadata of a log append operation on the log
      @@ -395,7 +394,7 @@ class ReplicaManager(val config: KafkaConfig,
             BrokerTopicStats.getBrokerAllTopicsStats().totalProduceRequestRate.mark()
       
             // reject appending to internal topics if it is not allowed
      -      if (Topic.InternalTopics.contains(topicPartition.topic) && !internalTopicsAllowed) {
      +      if (TopicConstants.INTERNAL_TOPICS.contains(topicPartition.topic) && !internalTopicsAllowed) {
               (topicPartition, LogAppendResult(
                 LogAppendInfo.UnknownLogAppendInfo,
                 Some(new InvalidTopicException("Cannot append to internal topic %s".format(topicPartition.topic)))))
      diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
      index b09c541334d44..fad76575939d6 100644
      --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
      +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala
      @@ -17,7 +17,6 @@ import java.net.Socket
       import java.nio.ByteBuffer
       import java.util.concurrent.ExecutionException
       import java.util.{ArrayList, Collections, Properties}
      -
       import kafka.cluster.EndPoint
       import kafka.common.TopicAndPartition
       import kafka.coordinator.GroupCoordinator
      @@ -34,10 +33,10 @@ import org.apache.kafka.common.security.auth.KafkaPrincipal
       import org.apache.kafka.common.{BrokerEndPoint, TopicPartition, requests}
       import org.junit.Assert._
       import org.junit.{After, Assert, Before, Test}
      -
       import scala.collection.JavaConverters._
       import scala.collection.mutable
       import scala.collection.mutable.Buffer
      +import org.apache.kafka.common.internals.TopicConstants
       
       class AuthorizerIntegrationTest extends KafkaServerTestHarness {
         val topic = "topic"
      @@ -143,7 +142,7 @@ class AuthorizerIntegrationTest extends KafkaServerTestHarness {
             consumers += TestUtils.createNewConsumer(TestUtils.getBrokerListStrFromServers(servers), groupId = group, securityProtocol = SecurityProtocol.PLAINTEXT)
       
           // create the consumer offset topic
      -    TestUtils.createTopic(zkUtils, GroupCoordinator.GroupMetadataTopicName,
      +    TestUtils.createTopic(zkUtils, TopicConstants.GROUP_METADATA_TOPIC_NAME,
             1,
             1,
             servers,
      diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
      index 684b38f3f002c..f576be575c377 100644
      --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
      +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
      @@ -13,23 +13,20 @@
       package kafka.api
       
       import java.util
      -
       import kafka.coordinator.GroupCoordinator
       import org.apache.kafka.clients.consumer._
       import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord}
       import org.apache.kafka.common.record.TimestampType
       import org.apache.kafka.common.serialization.ByteArrayDeserializer
       import org.apache.kafka.common.{PartitionInfo, TopicPartition}
      -
       import kafka.utils.{TestUtils, Logging, ShutdownableThread}
       import kafka.server.KafkaConfig
      -
       import java.util.ArrayList
       import org.junit.Assert._
       import org.junit.{Before, Test}
      -
       import scala.collection.JavaConverters._
       import scala.collection.mutable.Buffer
      +import org.apache.kafka.common.internals.TopicConstants
       
       /**
        * Integration tests for the new consumer that cover basic usage as well as server failures
      @@ -196,7 +193,7 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging {
           // get metadata for the topic
           var parts: Seq[PartitionInfo] = null
           while (parts == null)
      -      parts = consumer0.partitionsFor(GroupCoordinator.GroupMetadataTopicName).asScala
      +      parts = consumer0.partitionsFor(TopicConstants.GROUP_METADATA_TOPIC_NAME).asScala
           assertEquals(1, parts.size)
           assertNotNull(parts(0).leader())
       
      diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
      index b4f31c4c580c0..d0680b8309bc6 100644
      --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
      +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala
      @@ -24,10 +24,10 @@ import java.util.Properties
       import org.apache.kafka.clients.producer.KafkaProducer
       import kafka.server.KafkaConfig
       import kafka.integration.KafkaServerTestHarness
      -
       import org.junit.{After, Before}
       import scala.collection.mutable.Buffer
       import kafka.coordinator.GroupCoordinator
      +import org.apache.kafka.common.internals.TopicConstants
       
       /**
        * A helper class for writing integration tests that involve producers, consumers, and servers
      @@ -75,7 +75,7 @@ trait IntegrationTestHarness extends KafkaServerTestHarness {
           }
       
           // create the consumer offset topic
      -    TestUtils.createTopic(zkUtils, GroupCoordinator.GroupMetadataTopicName,
      +    TestUtils.createTopic(zkUtils, TopicConstants.GROUP_METADATA_TOPIC_NAME,
             serverConfig.getProperty(KafkaConfig.OffsetsTopicPartitionsProp).toInt,
             serverConfig.getProperty(KafkaConfig.OffsetsTopicReplicationFactorProp).toInt,
             servers,
      diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
      index 63a6b6fa3eead..2bb203db4d91b 100644
      --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
      +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala
      @@ -19,7 +19,6 @@ package kafka.api
       
       import java.util.concurrent.{ExecutionException, TimeUnit, TimeoutException}
       import java.util.{Properties, Random}
      -
       import kafka.common.Topic
       import kafka.consumer.SimpleConsumer
       import kafka.integration.KafkaServerTestHarness
      @@ -31,6 +30,7 @@ import org.apache.kafka.common.KafkaException
       import org.apache.kafka.common.errors.{InvalidTopicException, NotEnoughReplicasAfterAppendException, NotEnoughReplicasException}
       import org.junit.Assert._
       import org.junit.{After, Before, Test}
      +import org.apache.kafka.common.internals.TopicConstants
       
       class ProducerFailureHandlingTest extends KafkaServerTestHarness {
         private val producerBufferSize = 30000
      @@ -198,7 +198,7 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness {
         @Test
         def testCannotSendToInternalTopic() {
           val thrown = intercept[ExecutionException] {
      -      producer2.send(new ProducerRecord[Array[Byte],Array[Byte]](Topic.InternalTopics.head, "test".getBytes, "test".getBytes)).get
      +      producer2.send(new ProducerRecord[Array[Byte],Array[Byte]](TopicConstants.INTERNAL_TOPICS.iterator.next, "test".getBytes, "test".getBytes)).get
           }
           assertTrue("Unexpected exception while sending to an invalid topic " + thrown.getCause, thrown.getCause.isInstanceOf[InvalidTopicException])
         }
      diff --git a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala
      index b42aaf4d2ef99..e0107da4f3c37 100644
      --- a/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala
      +++ b/core/src/test/scala/unit/kafka/admin/TopicCommandTest.scala
      @@ -26,6 +26,7 @@ import kafka.server.ConfigType
       import kafka.admin.TopicCommand.TopicCommandOptions
       import kafka.utils.ZkUtils._
       import kafka.coordinator.GroupCoordinator
      +import org.apache.kafka.common.internals.TopicConstants
       
       class TopicCommandTest extends ZooKeeperTestHarness with Logging with RackAwareTest {
       
      @@ -86,12 +87,12 @@ class TopicCommandTest extends ZooKeeperTestHarness with Logging with RackAwareT
           // create the offset topic
           val createOffsetTopicOpts = new TopicCommandOptions(Array("--partitions", numPartitionsOriginal.toString,
             "--replication-factor", "1",
      -      "--topic", GroupCoordinator.GroupMetadataTopicName))
      +      "--topic", TopicConstants.GROUP_METADATA_TOPIC_NAME))
           TopicCommand.createTopic(zkUtils, createOffsetTopicOpts)
       
      -    // try to delete the GroupCoordinator.GroupMetadataTopicName and make sure it doesn't
      -    val deleteOffsetTopicOpts = new TopicCommandOptions(Array("--topic", GroupCoordinator.GroupMetadataTopicName))
      -    val deleteOffsetTopicPath = getDeleteTopicPath(GroupCoordinator.GroupMetadataTopicName)
      +    // try to delete the TopicConstants.GROUP_METADATA_TOPIC_NAME and make sure it doesn't
      +    val deleteOffsetTopicOpts = new TopicCommandOptions(Array("--topic", TopicConstants.GROUP_METADATA_TOPIC_NAME))
      +    val deleteOffsetTopicPath = getDeleteTopicPath(TopicConstants.GROUP_METADATA_TOPIC_NAME)
           assertFalse("Delete path for topic shouldn't exist before deletion.", zkUtils.zkClient.exists(deleteOffsetTopicPath))
           intercept[AdminOperationException] {
               TopicCommand.deleteTopic(zkUtils, deleteOffsetTopicOpts)
      diff --git a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala
      index 1e8d04eff1b1b..0e0a06a6daf72 100644
      --- a/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala
      +++ b/core/src/test/scala/unit/kafka/consumer/TopicFilterTest.scala
      @@ -22,6 +22,7 @@ import org.junit.Assert._
       import org.scalatest.junit.JUnitSuite
       import org.junit.Test
       import kafka.coordinator.GroupCoordinator
      +import org.apache.kafka.common.internals.TopicConstants
       
       
       class TopicFilterTest extends JUnitSuite {
      @@ -37,8 +38,8 @@ class TopicFilterTest extends JUnitSuite {
       
           val topicFilter2 = new Whitelist(".+")
           assertTrue(topicFilter2.isTopicAllowed("alltopics", excludeInternalTopics = true))
      -    assertFalse(topicFilter2.isTopicAllowed(GroupCoordinator.GroupMetadataTopicName, excludeInternalTopics = true))
      -    assertTrue(topicFilter2.isTopicAllowed(GroupCoordinator.GroupMetadataTopicName, excludeInternalTopics = false))
      +    assertFalse(topicFilter2.isTopicAllowed(TopicConstants.GROUP_METADATA_TOPIC_NAME, excludeInternalTopics = true))
      +    assertTrue(topicFilter2.isTopicAllowed(TopicConstants.GROUP_METADATA_TOPIC_NAME, excludeInternalTopics = false))
       
           val topicFilter3 = new Whitelist("white_listed-topic.+")
           assertTrue(topicFilter3.isTopicAllowed("white_listed-topic1", excludeInternalTopics = true))
      @@ -57,8 +58,8 @@ class TopicFilterTest extends JUnitSuite {
           assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = true))
           assertFalse(topicFilter1.isTopicAllowed("black1", excludeInternalTopics = false))
       
      -    assertFalse(topicFilter1.isTopicAllowed(GroupCoordinator.GroupMetadataTopicName, excludeInternalTopics = true))
      -    assertTrue(topicFilter1.isTopicAllowed(GroupCoordinator.GroupMetadataTopicName, excludeInternalTopics = false))
      +    assertFalse(topicFilter1.isTopicAllowed(TopicConstants.GROUP_METADATA_TOPIC_NAME, excludeInternalTopics = true))
      +    assertTrue(topicFilter1.isTopicAllowed(TopicConstants.GROUP_METADATA_TOPIC_NAME, excludeInternalTopics = false))
         }
       
         @Test
      diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
      index 50fa09e009786..acdb660ca687e 100644
      --- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
      +++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
      @@ -19,7 +19,6 @@ package kafka.coordinator
       
       import org.apache.kafka.common.record.Record
       import org.junit.Assert._
      -
       import kafka.common.{OffsetAndMetadata, TopicAndPartition}
       import kafka.message.{Message, MessageSet}
       import kafka.server.{ReplicaManager, KafkaConfig}
      @@ -32,12 +31,11 @@ import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
       import org.easymock.{Capture, IAnswer, EasyMock}
       import org.junit.{After, Before, Test}
       import org.scalatest.junit.JUnitSuite
      -
       import java.util.concurrent.TimeUnit
      -
       import scala.collection._
       import scala.concurrent.duration.Duration
       import scala.concurrent.{Await, Future, Promise}
      +import org.apache.kafka.common.internals.TopicConstants
       
       /**
        * Test GroupCoordinator responses
      @@ -81,12 +79,12 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
       
           // make two partitions of the group topic to make sure some partitions are not owned by the coordinator
           val ret = mutable.Map[String, Map[Int, Seq[Int]]]()
      -    ret += (GroupCoordinator.GroupMetadataTopicName -> Map(0 -> Seq(1), 1 -> Seq(1)))
      +    ret += (TopicConstants.GROUP_METADATA_TOPIC_NAME -> Map(0 -> Seq(1), 1 -> Seq(1)))
       
           replicaManager = EasyMock.createNiceMock(classOf[ReplicaManager])
       
           zkUtils = EasyMock.createNiceMock(classOf[ZkUtils])
      -    EasyMock.expect(zkUtils.getPartitionAssignmentForTopics(Seq(GroupCoordinator.GroupMetadataTopicName))).andReturn(ret)
      +    EasyMock.expect(zkUtils.getPartitionAssignmentForTopics(Seq(TopicConstants.GROUP_METADATA_TOPIC_NAME))).andReturn(ret)
           EasyMock.replay(zkUtils)
       
           groupCoordinator = GroupCoordinator(KafkaConfig.fromProps(props), zkUtils, replicaManager, new SystemTime)
      @@ -834,7 +832,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
             EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MessageSet]],
             EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
             override def answer = capturedArgument.getValue.apply(
      -        Map(new TopicPartition(GroupCoordinator.GroupMetadataTopicName, groupPartitionId) ->
      +        Map(new TopicPartition(TopicConstants.GROUP_METADATA_TOPIC_NAME, groupPartitionId) ->
                 new PartitionResponse(Errors.NONE.code, 0L, Record.NO_TIMESTAMP)
               )
             )})
      @@ -911,7 +909,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
             EasyMock.anyObject().asInstanceOf[Map[TopicPartition, MessageSet]],
             EasyMock.capture(capturedArgument))).andAnswer(new IAnswer[Unit] {
             override def answer = capturedArgument.getValue.apply(
      -        Map(new TopicPartition(GroupCoordinator.GroupMetadataTopicName, groupPartitionId) ->
      +        Map(new TopicPartition(TopicConstants.GROUP_METADATA_TOPIC_NAME, groupPartitionId) ->
                 new PartitionResponse(Errors.NONE.code, 0L, Record.NO_TIMESTAMP)
               )
             )})
      @@ -925,7 +923,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
         private def leaveGroup(groupId: String, consumerId: String): LeaveGroupCallbackParams = {
           val (responseFuture, responseCallback) = setupHeartbeatCallback
       
      -    EasyMock.expect(replicaManager.getPartition(GroupCoordinator.GroupMetadataTopicName, groupPartitionId)).andReturn(None)
      +    EasyMock.expect(replicaManager.getPartition(TopicConstants.GROUP_METADATA_TOPIC_NAME, groupPartitionId)).andReturn(None)
           EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes()
           EasyMock.replay(replicaManager)
       
      
      From a1eb12d7c6ad9422b9cf24b670d1b4c11227b03e Mon Sep 17 00:00:00 2001
      From: Eno Thereska 
      Date: Thu, 17 Mar 2016 13:17:01 -0700
      Subject: [PATCH 043/206] KAFKA-3188: Compatibility test for old and new
       clients with 0.10 broker
      
      apovzner becketqin please have a look if you can. Thanks.
      
      Author: Eno Thereska 
      
      Reviewers: Anna Povzner, Gwen Shapira
      
      Closes #1059 from enothereska/kafka-3188-compatibility
      ---
       .../services/kafka/config_property.py         |  2 +-
       .../tests/compatibility_test_new_broker.py    | 78 +++++++++++++++++++
       2 files changed, 79 insertions(+), 1 deletion(-)
       create mode 100644 tests/kafkatest/tests/compatibility_test_new_broker.py
      
      diff --git a/tests/kafkatest/services/kafka/config_property.py b/tests/kafkatest/services/kafka/config_property.py
      index 8f30f1331ed23..e1801efa92937 100644
      --- a/tests/kafkatest/services/kafka/config_property.py
      +++ b/tests/kafkatest/services/kafka/config_property.py
      @@ -41,7 +41,7 @@
       ZOOKEEPER_CONNECTION_TIMEOUT_MS = "zookeeper.connection.timeout.ms"
       INTER_BROKER_PROTOCOL_VERSION = "inter.broker.protocol.version"
       MESSAGE_FORMAT_VERSION = "log.message.format.version"
      -
      +MESSAGE_TIMESTAMP_TYPE = "message.timestamp.type"
       
       
       """
      diff --git a/tests/kafkatest/tests/compatibility_test_new_broker.py b/tests/kafkatest/tests/compatibility_test_new_broker.py
      new file mode 100644
      index 0000000000000..2c261df6c6dfd
      --- /dev/null
      +++ b/tests/kafkatest/tests/compatibility_test_new_broker.py
      @@ -0,0 +1,78 @@
      +# Copyright 2015 Confluent Inc.
      +#
      +# Licensed 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.
      +
      +from ducktape.tests.test import Test
      +from ducktape.mark import parametrize
      +from ducktape.utils.util import wait_until
      +from kafkatest.services.zookeeper import ZookeeperService
      +from kafkatest.services.kafka import KafkaService
      +from kafkatest.services.kafka.version import LATEST_0_9, LATEST_0_8_2, TRUNK, KafkaVersion
      +from kafkatest.services.verifiable_producer import VerifiableProducer
      +from kafkatest.services.console_consumer import ConsoleConsumer
      +from kafkatest.utils import is_int
      +from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest
      +from kafkatest.services.kafka import config_property
      +
      +# Compatibility tests for moving to a new broker (e.g., 0.10.x) and using a mix of old and new clients (e.g., 0.9.x)
      +class ClientCompatibilityTestNewBroker(ProduceConsumeValidateTest):
      +
      +    def __init__(self, test_context):
      +        super(ClientCompatibilityTestNewBroker, self).__init__(test_context=test_context)
      +
      +    def setUp(self):
      +        self.topic = "test_topic"
      +        self.zk = ZookeeperService(self.test_context, num_nodes=1)
      +            
      +        self.zk.start()
      +
      +        # Producer and consumer
      +        self.producer_throughput = 10000
      +        self.num_producers = 1
      +        self.num_consumers = 1
      +        self.messages_per_producer = 1000
      +
      +    @parametrize(producer_version=str(LATEST_0_8_2), consumer_version=str(LATEST_0_8_2), compression_types=["none"], timestamp_type=None)
      +    @parametrize(producer_version=str(LATEST_0_8_2), consumer_version=str(LATEST_0_9), compression_types=["none"], timestamp_type=None)
      +    @parametrize(producer_version=str(LATEST_0_9), consumer_version=str(TRUNK), compression_types=["none"], timestamp_type=None)
      +    @parametrize(producer_version=str(TRUNK), consumer_version=str(LATEST_0_9), compression_types=["none"], timestamp_type=None)
      +    @parametrize(producer_version=str(LATEST_0_9), consumer_version=str(TRUNK), compression_types=["snappy"], new_consumer=True, timestamp_type=None)
      +    @parametrize(producer_version=str(TRUNK), consumer_version=str(LATEST_0_9), compression_types=["snappy"], new_consumer=True, timestamp_type=str("CreateTime"))
      +    @parametrize(producer_version=str(TRUNK), consumer_version=str(TRUNK), compression_types=["snappy"], new_consumer=True, timestamp_type=str("LogAppendTime"))
      +    @parametrize(producer_version=str(LATEST_0_9), consumer_version=str(LATEST_0_9), compression_types=["snappy"], new_consumer=True, timestamp_type=str("LogAppendTime"))
      +    @parametrize(producer_version=str(TRUNK), consumer_version=str(TRUNK), compression_types=["none"], timestamp_type=str("LogAppendTime"))
      +    def test_compatibility(self, producer_version, consumer_version, compression_types, new_consumer=False, timestamp_type=None):
      +       
      +        self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zk, version=TRUNK, topics={self.topic: {
      +                                                                    "partitions": 3,
      +                                                                    "replication-factor": 3,
      +                                                                    'configs': {"min.insync.replicas": 2}}})
      +        for node in self.kafka.nodes:
      +            if timestamp_type is not None:
      +                node.config[config_property.MESSAGE_TIMESTAMP_TYPE] = timestamp_type
      +        self.kafka.start()
      +         
      +        self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka,
      +                                           self.topic, throughput=self.producer_throughput,
      +                                           message_validator=is_int,
      +                                           compression_types=compression_types,
      +                                           version=KafkaVersion(producer_version))
      +
      +        self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka,
      +                                        self.topic, consumer_timeout_ms=30000, new_consumer=new_consumer,
      +                                        message_validator=is_int, version=KafkaVersion(consumer_version))
      +
      +        self.run_produce_consume_validate(lambda: wait_until(
      +            lambda: self.producer.each_produced_at_least(self.messages_per_producer) == True,
      +            timeout_sec=120, backoff_sec=1,
      +            err_msg="Producer did not produce all messages in reasonable amount of time"))
      
      From c07d017227f319250e5c373f8a6f504874ecfbf2 Mon Sep 17 00:00:00 2001
      From: Liquan Pei 
      Date: Thu, 17 Mar 2016 13:26:02 -0700
      Subject: [PATCH 044/206] KAFKA-3315: Add REST and Connector API to expose
       connector configuration
      
      Author: Liquan Pei 
      
      Reviewers: Ewen Cheslack-Postava 
      
      Closes #964 from Ishiihara/expose-connector-config
      ---
       build.gradle                                  |   1 +
       .../kafka/common/config/AbstractConfig.java   |  26 +-
       .../apache/kafka/common/config/Config.java    |  28 +
       .../apache/kafka/common/config/ConfigDef.java | 652 ++++++++++++++++--
       .../kafka/common/config/ConfigValue.java      | 113 +++
       .../kafka/common/config/ConfigDefTest.java    | 202 +++++-
       .../kafka/connect/connector/Connector.java    |  21 +
       .../ConnectorReconfigurationTest.java         |   9 +-
       .../connect/file/FileStreamSinkConnector.java |  11 +
       .../file/FileStreamSourceConnector.java       |  12 +
       .../kafka/connect/runtime/AbstractHerder.java | 109 ++-
       .../connect/runtime/ConnectorConfig.java      |  18 +-
       .../apache/kafka/connect/runtime/Herder.java  |   7 +
       .../apache/kafka/connect/runtime/Worker.java  |   5 +
       .../distributed/DistributedHerder.java        |   6 +-
       .../connect/runtime/rest/RestServer.java      |   8 +-
       .../runtime/rest/entities/ConfigInfo.java     |  66 ++
       .../runtime/rest/entities/ConfigInfos.java    | 102 +++
       .../runtime/rest/entities/ConfigKeyInfo.java  | 171 +++++
       .../rest/entities/ConfigValueInfo.java        | 106 +++
       .../runtime/rest/entities/ConnectorInfo.java  |   5 +-
       .../resources/ConnectorPluginsResource.java   |  49 ++
       .../rest/resources/ConnectorsResource.java    |  16 +-
       .../runtime/standalone/StandaloneHerder.java  |   4 +-
       .../tools/VerifiableSinkConnector.java        |   6 +
       .../tools/VerifiableSourceConnector.java      |   6 +
       .../connect/runtime/AbstractHerderTest.java   |  10 +-
       .../kafka/connect/runtime/WorkerTest.java     |  12 +-
       .../ConnectorPluginsResourceTest.java         | 165 +++++
       .../resources/ConnectorsResourceTest.java     |   4 +-
       30 files changed, 1836 insertions(+), 114 deletions(-)
       create mode 100644 clients/src/main/java/org/apache/kafka/common/config/Config.java
       create mode 100644 clients/src/main/java/org/apache/kafka/common/config/ConfigValue.java
       create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigInfo.java
       create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigInfos.java
       create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigKeyInfo.java
       create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigValueInfo.java
       create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java
       create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java
      
      diff --git a/build.gradle b/build.gradle
      index 321fc3f9b8d30..4b84ba577947b 100644
      --- a/build.gradle
      +++ b/build.gradle
      @@ -543,6 +543,7 @@ project(':clients') {
           include "**/org/apache/kafka/common/*"
           include "**/org/apache/kafka/common/errors/*"
           include "**/org/apache/kafka/common/serialization/*"
      +    include "**/org/apache/kafka/common/config/*"
         }
       }
       
      diff --git a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
      index b44f72c855ccd..f833d7e784b05 100644
      --- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
      +++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java
      @@ -12,6 +12,13 @@
        */
       package org.apache.kafka.common.config;
       
      +import org.apache.kafka.common.Configurable;
      +import org.apache.kafka.common.KafkaException;
      +import org.apache.kafka.common.config.types.Password;
      +import org.apache.kafka.common.utils.Utils;
      +import org.slf4j.Logger;
      +import org.slf4j.LoggerFactory;
      +
       import java.util.ArrayList;
       import java.util.Collections;
       import java.util.HashMap;
      @@ -20,13 +27,6 @@
       import java.util.Map;
       import java.util.Set;
       
      -import org.apache.kafka.common.Configurable;
      -import org.apache.kafka.common.KafkaException;
      -import org.apache.kafka.common.config.types.Password;
      -import org.apache.kafka.common.utils.Utils;
      -import org.slf4j.Logger;
      -import org.slf4j.LoggerFactory;
      -
       /**
        * A convenient base class for configurations to extend.
        * 

      @@ -46,7 +46,7 @@ public class AbstractConfig { private final Map values; @SuppressWarnings("unchecked") - public AbstractConfig(ConfigDef definition, Map originals, Boolean doLog) { + public AbstractConfig(ConfigDef definition, Map originals, boolean doLog) { /* check that all the keys are really strings */ for (Object key : originals.keySet()) if (!(key instanceof String)) @@ -62,6 +62,12 @@ public AbstractConfig(ConfigDef definition, Map originals) { this(definition, originals, true); } + public AbstractConfig(Map parsedConfig) { + this.values = parsedConfig; + this.originals = new HashMap<>(); + this.used = Collections.synchronizedSet(new HashSet()); + } + protected Object get(String key) { if (!values.containsKey(key)) throw new ConfigException(String.format("Unknown configuration '%s'", key)); @@ -94,7 +100,7 @@ public List getList(String key) { return (List) get(key); } - public boolean getBoolean(String key) { + public Boolean getBoolean(String key) { return (Boolean) get(key); } @@ -125,7 +131,7 @@ public Map originals() { /** * Get all the original settings, ensuring that all values are of type String. * @return the original settings - * @throw ClassCastException if any of the values are not strings + * @throws ClassCastException if any of the values are not strings */ public Map originalsStrings() { Map copy = new RecordingMap<>(); diff --git a/clients/src/main/java/org/apache/kafka/common/config/Config.java b/clients/src/main/java/org/apache/kafka/common/config/Config.java new file mode 100644 index 0000000000000..ce5ee17abdb0b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/config/Config.java @@ -0,0 +1,28 @@ +/** + * 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 org.apache.kafka.common.config; + +import java.util.List; + +public class Config { + private List configValues; + + public Config(List configValues) { + this.configValues = configValues; + } + + public List configValues() { + return configValues; + } + +} diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java index 703eb7c4b9bd0..881cb0bf9353f 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java @@ -12,45 +12,66 @@ */ package org.apache.kafka.common.config; +import org.apache.kafka.common.config.types.Password; +import org.apache.kafka.common.utils.Utils; + import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; import java.util.Comparator; import java.util.HashMap; +import java.util.HashSet; +import java.util.Iterator; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; -import org.apache.kafka.common.config.types.Password; -import org.apache.kafka.common.utils.Utils; - /** - * 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. + * This class is used for specifying the set of expected configurations. For each configuration, you can specify + * the name, the type, the default value, the documentation, the group information, the order in the group, + * the width of the configuration value and the name suitable for display in the UI. + * + * You can provide special validation logic used for single configuration validation by overriding {@link Validator}. + * + * Moreover, you can specify the dependents of a configuration. The valid values and visibility of a configuration + * may change according to the values of other configurations. You can override {@link Recommender} to get valid + * values and set visibility of a configuration given the current configuration values. + * *

      - * Usage of this class looks something like this: + * To use the class: *

      *

        * ConfigDef defs = new ConfigDef();
      - * defs.define("config_name", Type.STRING, "default string value", "This configuration is used for blah blah blah.");
      - * defs.define("another_config_name", Type.INT, 42, Range.atLeast(0), "More documentation on this config");
        *
      - * Properties props = new Properties();
      - * props.setProperty("config_name", "some value");
      + * defs.define("config_with_default", Type.STRING, "default string value", "Configuration with default value.");
      + * defs.define("config_with_validator", Type.INT, 42, Range.atLeast(0), "Configuration with user provided validator.");
      + * defs.define("config_with_dependents", Type.INT, "Configuration with dependents.", "group", 1, "Config With Dependents", Arrays.asList("config_with_default;","config_with_validator"));
      + *
      + * Map<String, String> props = new HashMap<>();
      + * props.put("config_with_default", "some value");
      + * props.put("config_with_dependents", "some other value");
      + * // will return "some value"
        * Map<String, Object> configs = defs.parse(props);
      + * String someConfig = (String) configs.get("config_with_default");
      + * // will return default value of 42
      + * int anotherConfig = (Integer) configs.get("config_with_validator");
        *
      - * String someConfig = (String) configs.get("config_name"); // will return "some value"
      - * int anotherConfig = (Integer) configs.get("another_config_name"); // will return default value of 42
      + * To validate the full configuration, use:
      + * List<Config> configs = def.validate(props);
      + * The {@link Config} contains updated configuration information given the current configuration values.
        * 
      *

      - * This class can be used stand-alone or in combination with {@link AbstractConfig} which provides some additional + * This class can be used standalone or in combination with {@link AbstractConfig} which provides some additional * functionality for accessing configs. */ public class ConfigDef { public static final Object NO_DEFAULT_VALUE = new String(""); - private final Map configKeys = new HashMap(); + private final Map configKeys = new HashMap<>(); + private final List groups = new LinkedList<>(); + private Set configsWithNoParent; /** * Returns unmodifiable set of properties names defined in this {@linkplain ConfigDef} @@ -63,26 +84,256 @@ public Set names() { /** * 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 importance The importance of this config: is this something you will likely need to change. - * @param documentation The documentation string for the config + * @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 the validator to use in checking the correctness of the config + * @param importance the importance of this config + * @param documentation the documentation string for the config + * @param group the group this config belongs to + * @param orderInGroup the order of this config in the group + * @param width the width of the config + * @param displayName the name suitable for display + * @param dependents the configurations that are dependents of this configuration + * @param recommender the recommender provides valid values given the parent configuration values * @return This ConfigDef so you can chain calls */ - public ConfigDef define(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation) { - if (configKeys.containsKey(name)) + public ConfigDef define(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation, + String group, int orderInGroup, Width width, String displayName, List dependents, Recommender recommender) { + if (configKeys.containsKey(name)) { throw new ConfigException("Configuration " + name + " is defined twice."); + } + if (group != null && !groups.contains(group)) { + groups.add(group); + } Object parsedDefault = defaultValue == NO_DEFAULT_VALUE ? NO_DEFAULT_VALUE : parseType(name, defaultValue, type); - configKeys.put(name, new ConfigKey(name, type, parsedDefault, validator, importance, documentation)); + configKeys.put(name, new ConfigKey(name, type, parsedDefault, validator, importance, documentation, group, orderInGroup, width, displayName, dependents, recommender)); return this; } + /** + * Define a new configuration with no custom recommender + * @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 the validator to use in checking the correctness of the config + * @param importance the importance of this config + * @param documentation the documentation string for the config + * @param group the group this config belongs to + * @param orderInGroup the order of this config in the group + * @param width the width of the config + * @param displayName the name suitable for display + * @param dependents the configurations that are dependents of this configuration + * @return This ConfigDef so you can chain calls + */ + public ConfigDef define(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation, + String group, int orderInGroup, Width width, String displayName, List dependents) { + return define(name, type, defaultValue, validator, importance, documentation, group, orderInGroup, width, displayName, dependents, null); + } + + /** + * Define a new configuration with no dependents + * @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 the validator to use in checking the correctness of the config + * @param importance the importance of this config + * @param documentation the documentation string for the config + * @param group the group this config belongs to + * @param orderInGroup the order of this config in the group + * @param width the width of the config + * @param displayName the name suitable for display + * @param recommender the recommender provides valid values given the parent configuration values + * @return This ConfigDef so you can chain calls + */ + public ConfigDef define(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation, + String group, int orderInGroup, Width width, String displayName, Recommender recommender) { + return define(name, type, defaultValue, validator, importance, documentation, group, orderInGroup, width, displayName, Collections.emptyList(), recommender); + } + + /** + * Define a new configuration with no dependents and no custom recommender + * @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 the validator to use in checking the correctness of the config + * @param importance the importance of this config + * @param documentation the documentation string for the config + * @param group the group this config belongs to + * @param orderInGroup the order of this config in the group + * @param width the width of the config + * @param displayName the name suitable for display + * @return This ConfigDef so you can chain calls + */ + public ConfigDef define(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation, + String group, int orderInGroup, Width width, String displayName) { + return define(name, type, defaultValue, validator, importance, documentation, group, orderInGroup, width, displayName, Collections.emptyList()); + } + + /** + * 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 importance the importance of this config + * @param documentation the documentation string for the config + * @param group the group this config belongs to + * @param orderInGroup the order of this config in the group + * @param width the width of the config + * @param displayName the name suitable for display + * @param dependents the configurations that are dependents of this configuration + * @param recommender the recommender provides valid values given the parent configuration values + * @return This ConfigDef so you can chain calls + */ + public ConfigDef define(String name, Type type, Object defaultValue, Importance importance, String documentation, + String group, int orderInGroup, Width width, String displayName, List dependents, Recommender recommender) { + return define(name, type, defaultValue, null, importance, documentation, group, orderInGroup, width, displayName, dependents, recommender); + } + + /** + * Define a new configuration with no special validation logic and no custom recommender + * @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 importance the importance of this config + * @param documentation the documentation string for the config + * @param group the group this config belongs to + * @param orderInGroup the order of this config in the group + * @param width the width of the config + * @param displayName the name suitable for display + * @param dependents the configurations that are dependents of this configuration + * @return This ConfigDef so you can chain calls + */ + public ConfigDef define(String name, Type type, Object defaultValue, Importance importance, String documentation, + String group, int orderInGroup, Width width, String displayName, List dependents) { + return define(name, type, defaultValue, null, importance, documentation, group, orderInGroup, width, displayName, dependents, null); + } + + /** + * Define a new configuration with no special validation logic and no custom recommender + * @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 importance the importance of this config + * @param documentation the documentation string for the config + * @param group the group this config belongs to + * @param orderInGroup the order of this config in the group + * @param width the width of the config + * @param displayName the name suitable for display + * @param recommender the recommender provides valid values given the parent configuration values + * @return This ConfigDef so you can chain calls + */ + public ConfigDef define(String name, Type type, Object defaultValue, Importance importance, String documentation, + String group, int orderInGroup, Width width, String displayName, Recommender recommender) { + return define(name, type, defaultValue, null, importance, documentation, group, orderInGroup, width, displayName, Collections.emptyList(), recommender); + } + + /** + * Define a new configuration with no special validation logic, not dependents and no custom recommender + * @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 importance the importance of this config + * @param documentation the documentation string for the config + * @param group the group this config belongs to + * @param orderInGroup the order of this config in the group + * @param width the width of the config + * @param displayName the name suitable for display + * @return This ConfigDef so you can chain calls + */ + public ConfigDef define(String name, Type type, Object defaultValue, Importance importance, String documentation, + String group, int orderInGroup, Width width, String displayName) { + return define(name, type, defaultValue, null, importance, documentation, group, orderInGroup, width, displayName, Collections.emptyList()); + } + + /** + * Define a new configuration 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 importance the importance of this config + * @param documentation the documentation string for the config + * @param group the group this config belongs to + * @param orderInGroup the order of this config in the group + * @param width the width of the config + * @param displayName the name suitable for display + * @param dependents the configurations that are dependents of this configuration + * @param recommender the recommender provides valid values given the parent configuration value + * @return This ConfigDef so you can chain calls + */ + public ConfigDef define(String name, Type type, Importance importance, String documentation, String group, int orderInGroup, + Width width, String displayName, List dependents, Recommender recommender) { + return define(name, type, NO_DEFAULT_VALUE, null, importance, documentation, group, orderInGroup, width, displayName, dependents, recommender); + } + + /** + * Define a new configuration with no default value, no special validation logic and no custom recommender + * @param name the name of the config parameter + * @param type the type of the config + * @param importance the importance of this config + * @param documentation the documentation string for the config + * @param group the group this config belongs to + * @param orderInGroup the order of this config in the group + * @param width the width of the config + * @param displayName the name suitable for display + * @param dependents the configurations that are dependents of this configuration + * @return This ConfigDef so you can chain calls + */ + public ConfigDef define(String name, Type type, Importance importance, String documentation, String group, int orderInGroup, + Width width, String displayName, List dependents) { + return define(name, type, NO_DEFAULT_VALUE, null, importance, documentation, group, orderInGroup, width, displayName, dependents, null); + } + + /** + * Define a new configuration with no default value, no special validation logic and no custom recommender + * @param name the name of the config parameter + * @param type the type of the config + * @param importance the importance of this config + * @param documentation the documentation string for the config + * @param group the group this config belongs to + * @param orderInGroup the order of this config in the group + * @param width the width of the config + * @param displayName the name suitable for display + * @param recommender the recommender provides valid values given the parent configuration value + * @return This ConfigDef so you can chain calls + */ + public ConfigDef define(String name, Type type, Importance importance, String documentation, String group, int orderInGroup, + Width width, String displayName, Recommender recommender) { + return define(name, type, NO_DEFAULT_VALUE, null, importance, documentation, group, orderInGroup, width, displayName, Collections.emptyList(), recommender); + } + + /** + * Define a new configuration with no default value, no special validation logic, no dependents and no custom recommender + * @param name the name of the config parameter + * @param type the type of the config + * @param importance the importance of this config + * @param documentation the documentation string for the config + * @param group the group this config belongs to + * @param orderInGroup the order of this config in the group + * @param width the width of the config + * @param displayName the name suitable for display + * @return This ConfigDef so you can chain calls + */ + public ConfigDef define(String name, Type type, Importance importance, String documentation, String group, int orderInGroup, + Width width, String displayName) { + return define(name, type, NO_DEFAULT_VALUE, null, importance, documentation, group, orderInGroup, width, displayName, Collections.emptyList()); + } + + /** + * Define a new configuration with no group, no order in group, no width, no display name, no dependents and no custom recommender + * @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 the validator to use in checking the correctness of the config + * @param importance the importance of this 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, Importance importance, String documentation) { + return define(name, type, defaultValue, validator, importance, documentation, null, -1, Width.NONE, name); + } + /** * 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 @@ -96,7 +347,6 @@ public ConfigDef define(String name, Type type, Object defaultValue, Importance /** * Define a new configuration 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 importance The importance of this config: is this something you will likely need to change. @@ -107,6 +357,22 @@ public ConfigDef define(String name, Type type, Importance importance, String do return define(name, type, NO_DEFAULT_VALUE, null, importance, documentation); } + /** + * Get the configuration keys + * @return a map containing all configuration keys + */ + public Map configKeys() { + return configKeys; + } + + /** + * Get the groups for the configuration + * @return a list of group names + */ + public List groups() { + return groups; + } + /** * Add standard SSL client configuration options. * @return this @@ -131,34 +397,188 @@ public ConfigDef withClientSaslSupport() { * 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 + * @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) + * the appropriate type (int, string, etc). */ public Map parse(Map props) { - /* parse all known keys */ - Map values = new HashMap(); + // Check all configurations are defined + List undefinedConfigKeys = undefinedDependentConfigs(); + if (!undefinedConfigKeys.isEmpty()) { + String joined = Utils.join(undefinedConfigKeys, ","); + throw new ConfigException("Some configurations in are referred in the dependents, but not defined: " + joined); + } + // parse all known keys + Map values = new HashMap<>(); for (ConfigKey key : configKeys.values()) { Object value; // props map contains setting - assign ConfigKey value - if (props.containsKey(key.name)) + if (props.containsKey(key.name)) { value = parseType(key.name, props.get(key.name), key.type); - // props map doesn't contain setting, the key is required because no default value specified - its an error - else if (key.defaultValue == NO_DEFAULT_VALUE) + // props map doesn't contain setting, the key is required because no default value specified - its an error + } else if (key.defaultValue == NO_DEFAULT_VALUE) { throw new ConfigException("Missing required configuration \"" + key.name + "\" which has no default value."); - // otherwise assign setting its default value - else + } else { + // otherwise assign setting its default value value = key.defaultValue; - if (key.validator != null) + } + if (key.validator != null) { key.validator.ensureValid(key.name, value); + } values.put(key.name, value); } return values; } + /** + * Validate the current configuration values with the configuration definition. + * @param props the current configuration values + * @return List of Config, each Config contains the updated configuration information given + * the current configuration values. + */ + public List validate(Map props) { + Map configValues = new HashMap<>(); + for (String name: configKeys.keySet()) { + configValues.put(name, new ConfigValue(name)); + } + + List undefinedConfigKeys = undefinedDependentConfigs(); + for (String undefinedConfigKey: undefinedConfigKeys) { + ConfigValue undefinedConfigValue = new ConfigValue(undefinedConfigKey); + undefinedConfigValue.addErrorMessage(undefinedConfigKey + " is referred in the dependents, but not defined."); + undefinedConfigValue.visible(false); + configValues.put(undefinedConfigKey, undefinedConfigValue); + } + + Map parsed = parseForValidate(props, configValues); + return validate(parsed, configValues); + } + + // package accessible for testing + Map parseForValidate(Map props, Map configValues) { + Map parsed = new HashMap<>(); + Set configsWithNoParent = getConfigsWithNoParent(); + for (String name: configsWithNoParent) { + parseForValidate(name, props, parsed, configValues); + } + return parsed; + } + + + private List validate(Map parsed, Map configValues) { + Set configsWithNoParent = getConfigsWithNoParent(); + for (String name: configsWithNoParent) { + validate(name, parsed, configValues); + } + return new LinkedList<>(configValues.values()); + } + + private List undefinedDependentConfigs() { + Set undefinedConfigKeys = new HashSet<>(); + for (String configName: configKeys.keySet()) { + ConfigKey configKey = configKeys.get(configName); + List dependents = configKey.dependents; + for (String dependent: dependents) { + if (!configKeys.containsKey(dependent)) { + undefinedConfigKeys.add(dependent); + } + } + } + return new LinkedList<>(undefinedConfigKeys); + } + + private Set getConfigsWithNoParent() { + if (this.configsWithNoParent != null) { + return this.configsWithNoParent; + } + Set configsWithParent = new HashSet<>(); + + for (ConfigKey configKey: configKeys.values()) { + List dependents = configKey.dependents; + configsWithParent.addAll(dependents); + } + + Set configs = new HashSet<>(configKeys.keySet()); + configs.removeAll(configsWithParent); + this.configsWithNoParent = configs; + return configs; + } + + private void parseForValidate(String name, Map props, Map parsed, Map configs) { + if (!configKeys.containsKey(name)) { + return; + } + ConfigKey key = configKeys.get(name); + ConfigValue config = configs.get(name); + + Object value = null; + if (props.containsKey(key.name)) { + try { + value = parseType(key.name, props.get(key.name), key.type); + } catch (ConfigException e) { + config.addErrorMessage(e.getMessage()); + } + } else if (key.defaultValue == NO_DEFAULT_VALUE) { + config.addErrorMessage("Missing required configuration \"" + key.name + "\" which has no default value."); + } else { + value = key.defaultValue; + } + + if (key.validator != null) { + try { + key.validator.ensureValid(key.name, value); + } catch (ConfigException e) { + config.addErrorMessage(e.getMessage()); + } + } + config.value(value); + parsed.put(name, value); + for (String dependent: key.dependents) { + parseForValidate(dependent, props, parsed, configs); + } + } + + private void validate(String name, Map parsed, Map configs) { + if (!configKeys.containsKey(name)) { + return; + } + ConfigKey key = configKeys.get(name); + ConfigValue config = configs.get(name); + Object value = parsed.get(name); + List recommendedValues; + if (key.recommender != null) { + try { + recommendedValues = key.recommender.validValues(name, parsed); + List originalRecommendedValues = config.recommendedValues(); + + if (!originalRecommendedValues.isEmpty()) { + Set originalRecommendedValueSet = new HashSet<>(originalRecommendedValues); + Iterator it = recommendedValues.iterator(); + while (it.hasNext()) { + Object o = it.next(); + if (!originalRecommendedValueSet.contains(o)) { + it.remove(); + } + } + } + config.recommendedValues(recommendedValues); + if (value != null && !recommendedValues.isEmpty() && !recommendedValues.contains(value)) { + config.addErrorMessage("Invalid value for configuration " + key.name); + } + config.visible(key.recommender.visible(name, parsed)); + } catch (ConfigException e) { + config.addErrorMessage(e.getMessage()); + } + } + + configs.put(name, config); + for (String dependent: key.dependents) { + validate(dependent, parsed, configs); + } + } + /** * Parse a value according to its expected type. - * * @param name The config name * @param value The config value * @param type The expected type @@ -263,15 +683,56 @@ public enum Type { BOOLEAN, STRING, INT, SHORT, LONG, DOUBLE, LIST, CLASS, PASSWORD } + /** + * The importance level for a configuration + */ public enum Importance { HIGH, MEDIUM, LOW } /** - * Validation logic the user may provide + * The width of a configuration value + */ + public enum Width { + NONE, SHORT, MEDIUM, LONG + } + + /** + * This is used by the {@link #validate(Map)} to get valid values for a configuration given the current + * configuration values in order to perform full configuration validation and visibility modification. + * In case that there are dependencies between configurations, the valid values and visibility + * for a configuration may change given the values of other configurations. + */ + public interface Recommender { + + /** + * The valid values for the configuration given the current configuration values. + * @param name The name of the configuration + * @param parsedConfig The parsed configuration values + * @return The list of valid values. To function properly, the returned objects should have the type + * defined for the configuration using the recommender. + */ + List validValues(String name, Map parsedConfig); + + /** + * Set the visibility of the configuration given the current configuration values. + * @param name The name of the configuration + * @param parsedConfig The parsed configuration values + * @return The visibility of the configuration + */ + boolean visible(String name, Map parsedConfig); + } + + /** + * Validation logic the user may provide to perform single configuration validation. */ public interface Validator { - public void ensureValid(String name, Object o); + /** + * Perform single configuration validation. + * @param name The name of the configuration + * @param value The value of the configuration + */ + void ensureValid(String name, Object value); } /** @@ -345,16 +806,24 @@ public String toString() { } } - private static class ConfigKey { + public static class ConfigKey { public final String name; public final Type type; public final String documentation; public final Object defaultValue; public final Validator validator; public final Importance importance; - - public ConfigKey(String name, Type type, Object defaultValue, Validator validator, Importance importance, String documentation) { - super(); + public final String group; + public final int orderInGroup; + public final Width width; + public final String displayName; + public final List dependents; + public final Recommender recommender; + + public ConfigKey(String name, Type type, Object defaultValue, Validator validator, + Importance importance, String documentation, String group, + int orderInGroup, Width width, String displayName, + List dependents, Recommender recommender) { this.name = name; this.type = type; this.defaultValue = defaultValue; @@ -363,34 +832,21 @@ public ConfigKey(String name, Type type, Object defaultValue, Validator validato if (this.validator != null && this.hasDefault()) this.validator.ensureValid(name, defaultValue); this.documentation = documentation; + this.dependents = dependents; + this.group = group; + this.orderInGroup = orderInGroup; + this.width = width; + this.displayName = displayName; + this.recommender = recommender; } public boolean hasDefault() { return this.defaultValue != NO_DEFAULT_VALUE; } - } public String toHtmlTable() { - // sort first required fields, then by importance, then name - List configs = new ArrayList(this.configKeys.values()); - Collections.sort(configs, new Comparator() { - public int compare(ConfigDef.ConfigKey k1, ConfigDef.ConfigKey k2) { - // first take anything with no default value (therefore required) - if (!k1.hasDefault() && k2.hasDefault()) - return -1; - else if (!k2.hasDefault() && k1.hasDefault()) - return 1; - - // then sort by importance - int cmp = k1.importance.compareTo(k2.importance); - if (cmp == 0) - // then sort in alphabetical order - return k1.name.compareTo(k2.name); - else - return cmp; - } - }); + List configs = sortedConfigs(); StringBuilder b = new StringBuilder(); b.append("\n"); b.append("\n"); @@ -434,4 +890,74 @@ else if (def.type == Type.STRING && def.defaultValue.toString().isEmpty()) b.append("
      "); return b.toString(); } + + /** + * Get the configs formatted with reStructuredText, suitable for embedding in Sphinx + * documentation. + */ + public String toRst() { + List configs = sortedConfigs(); + StringBuilder b = new StringBuilder(); + + for (ConfigKey def : configs) { + b.append("``"); + b.append(def.name); + b.append("``\n"); + for (String docLine : def.documentation.split("\n")) { + if (docLine.length() == 0) { + continue; + } + b.append(" "); + b.append(docLine); + b.append("\n\n"); + } + b.append(" * Type: "); + b.append(def.type.toString().toLowerCase()); + b.append("\n"); + if (def.defaultValue != null) { + b.append(" * Default: "); + if (def.type == Type.STRING) { + b.append("\""); + b.append(def.defaultValue); + b.append("\""); + } else { + b.append(def.defaultValue); + } + b.append("\n"); + } + b.append(" * Importance: "); + b.append(def.importance.toString().toLowerCase()); + b.append("\n\n"); + } + return b.toString(); + } + + /** + * Get a list of configs sorted into "natural" order: listing required fields first, then + * ordering by importance, and finally by name. + */ + private List sortedConfigs() { + // sort first required fields, then by importance, then name + List configs = new ArrayList<>(this.configKeys.values()); + Collections.sort(configs, new Comparator() { + public int compare(ConfigKey k1, ConfigKey k2) { + // first take anything with no default value + if (!k1.hasDefault() && k2.hasDefault()) { + return -1; + } else if (!k2.hasDefault() && k1.hasDefault()) { + return 1; + } + + // then sort by importance + int cmp = k1.importance.compareTo(k2.importance); + if (cmp == 0) { + // then sort in alphabetical order + return k1.name.compareTo(k2.name); + } else { + return cmp; + } + } + }); + return configs; + } } diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigValue.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigValue.java new file mode 100644 index 0000000000000..c9a4a3465a47f --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigValue.java @@ -0,0 +1,113 @@ +/** + * 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 org.apache.kafka.common.config; + +import java.util.LinkedList; +import java.util.List; +import java.util.Objects; + +public class ConfigValue { + + private String name; + private Object value; + private List recommendedValues; + private List errorMessages; + private boolean visible; + + public ConfigValue(String name) { + this(name, null, new LinkedList(), new LinkedList()); + } + + public ConfigValue(String name, Object value, List recommendedValues, List errorMessages) { + this.name = name; + this.value = value; + this.recommendedValues = recommendedValues; + this.errorMessages = errorMessages; + this.visible = true; + } + + public String name() { + return name; + } + + public Object value() { + return value; + } + + public List recommendedValues() { + return recommendedValues; + } + + public List errorMessages() { + return errorMessages; + } + + public boolean visible() { + return visible; + } + + public void value(Object value) { + this.value = value; + } + + public void recommendedValues(List recommendedValues) { + this.recommendedValues = recommendedValues; + } + + public void addErrorMessage(String errorMessage) { + this.errorMessages.add(errorMessage); + } + + public void visible(boolean visible) { + this.visible = visible; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ConfigValue that = (ConfigValue) o; + return Objects.equals(name, that.name) && + Objects.equals(value, that.value) && + Objects.equals(recommendedValues, that.recommendedValues) && + Objects.equals(errorMessages, that.errorMessages) && + Objects.equals(visible, that.visible); + } + + @Override + public int hashCode() { + return Objects.hash(name, value, recommendedValues, errorMessages, visible); + } + + @Override + public String toString() { + StringBuffer sb = new StringBuffer(); + sb.append("[") + .append(name) + .append(",") + .append(value) + .append(",") + .append(recommendedValues) + .append(",") + .append(errorMessages) + .append(",") + .append(visible) + .append("]"); + return sb.toString(); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java index fa0370b9bee81..022fb6bc62397 100644 --- a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java +++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java @@ -12,22 +12,27 @@ */ 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 org.apache.kafka.common.config.ConfigDef.Importance; -import org.apache.kafka.common.config.ConfigDef.Validator; import org.apache.kafka.common.config.ConfigDef.Range; -import org.apache.kafka.common.config.ConfigDef.ValidString; import org.apache.kafka.common.config.ConfigDef.Type; +import org.apache.kafka.common.config.ConfigDef.ValidString; +import org.apache.kafka.common.config.ConfigDef.Validator; +import org.apache.kafka.common.config.ConfigDef.Width; import org.apache.kafka.common.config.types.Password; import org.junit.Test; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; +import java.util.Properties; + +import static java.util.Arrays.asList; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + public class ConfigDefTest { @Test @@ -156,7 +161,8 @@ public void testNullDefaultWithValidator() { final String key = "enum_test"; ConfigDef def = new ConfigDef(); - def.define(key, Type.STRING, ConfigDef.NO_DEFAULT_VALUE, ValidString.in("ONE", "TWO", "THREE"), Importance.HIGH, "docs"); + def.define(key, Type.STRING, ConfigDef.NO_DEFAULT_VALUE, + ValidString.in("ONE", "TWO", "THREE"), Importance.HIGH, "docs"); Properties props = new Properties(); props.put(key, "ONE"); @@ -164,6 +170,180 @@ public void testNullDefaultWithValidator() { assertEquals("ONE", vals.get(key)); } + @Test + public void testGroupInference() { + List expected1 = Arrays.asList("group1", "group2"); + ConfigDef def1 = new ConfigDef() + .define("a", Type.INT, Importance.HIGH, "docs", "group1", 1, Width.SHORT, "a") + .define("b", Type.INT, Importance.HIGH, "docs", "group2", 1, Width.SHORT, "b") + .define("c", Type.INT, Importance.HIGH, "docs", "group1", 2, Width.SHORT, "c"); + + assertEquals(expected1, def1.groups()); + + List expected2 = Arrays.asList("group2", "group1"); + ConfigDef def2 = new ConfigDef() + .define("a", Type.INT, Importance.HIGH, "docs", "group2", 1, Width.SHORT, "a") + .define("b", Type.INT, Importance.HIGH, "docs", "group2", 2, Width.SHORT, "b") + .define("c", Type.INT, Importance.HIGH, "docs", "group1", 2, Width.SHORT, "c"); + + assertEquals(expected2, def2.groups()); + } + + @Test + public void testParseForValidate() { + Map expectedParsed = new HashMap<>(); + expectedParsed.put("a", 1); + expectedParsed.put("b", null); + expectedParsed.put("c", null); + expectedParsed.put("d", 10); + + Map expected = new HashMap<>(); + String errorMessageB = "Missing required configuration \"b\" which has no default value."; + String errorMessageC = "Missing required configuration \"c\" which has no default value."; + ConfigValue configA = new ConfigValue("a", 1, Collections.emptyList(), Collections.emptyList()); + ConfigValue configB = new ConfigValue("b", null, Collections.emptyList(), Arrays.asList(errorMessageB, errorMessageB)); + ConfigValue configC = new ConfigValue("c", null, Collections.emptyList(), Arrays.asList(errorMessageC)); + ConfigValue configD = new ConfigValue("d", 10, Collections.emptyList(), Collections.emptyList()); + expected.put("a", configA); + expected.put("b", configB); + expected.put("c", configC); + expected.put("d", configD); + + ConfigDef def = new ConfigDef() + .define("a", Type.INT, Importance.HIGH, "docs", "group", 1, Width.SHORT, "a", Arrays.asList("b", "c"), new IntegerRecommender(false)) + .define("b", Type.INT, Importance.HIGH, "docs", "group", 2, Width.SHORT, "b", new IntegerRecommender(true)) + .define("c", Type.INT, Importance.HIGH, "docs", "group", 3, Width.SHORT, "c", new IntegerRecommender(true)) + .define("d", Type.INT, Importance.HIGH, "docs", "group", 4, Width.SHORT, "d", Arrays.asList("b"), new IntegerRecommender(false)); + + Map props = new HashMap<>(); + props.put("a", "1"); + props.put("d", "10"); + + Map configValues = new HashMap<>(); + + for (String name: def.configKeys().keySet()) { + configValues.put(name, new ConfigValue(name)); + } + + Map parsed = def.parseForValidate(props, configValues); + + assertEquals(expectedParsed, parsed); + assertEquals(expected, configValues); + } + + @Test + public void testValidate() { + Map expected = new HashMap<>(); + String errorMessageB = "Missing required configuration \"b\" which has no default value."; + String errorMessageC = "Missing required configuration \"c\" which has no default value."; + String errorMessageD = "Invalid value for configuration d"; + + ConfigValue configA = new ConfigValue("a", 1, Arrays.asList(1, 2, 3), Collections.emptyList()); + ConfigValue configB = new ConfigValue("b", null, Arrays.asList(4, 5), Arrays.asList(errorMessageB, errorMessageB)); + ConfigValue configC = new ConfigValue("c", null, Arrays.asList(4, 5), Arrays.asList(errorMessageC)); + ConfigValue configD = new ConfigValue("d", 10, Arrays.asList(1, 2, 3), Arrays.asList(errorMessageD)); + + expected.put("a", configA); + expected.put("b", configB); + expected.put("c", configC); + expected.put("d", configD); + + ConfigDef def = new ConfigDef() + .define("a", Type.INT, Importance.HIGH, "docs", "group", 1, Width.SHORT, "a", Arrays.asList("b", "c"), new IntegerRecommender(false)) + .define("b", Type.INT, Importance.HIGH, "docs", "group", 2, Width.SHORT, "b", new IntegerRecommender(true)) + .define("c", Type.INT, Importance.HIGH, "docs", "group", 3, Width.SHORT, "c", new IntegerRecommender(true)) + .define("d", Type.INT, Importance.HIGH, "docs", "group", 4, Width.SHORT, "d", Arrays.asList("b"), new IntegerRecommender(false)); + + Map props = new HashMap<>(); + props.put("a", "1"); + props.put("d", "10"); + + List configs = def.validate(props); + for (ConfigValue config : configs) { + String name = config.name(); + ConfigValue expectedConfig = expected.get(name); + assertEquals(expectedConfig, config); + } + } + + @Test + public void testValidateMissingConfigKey() { + Map expected = new HashMap<>(); + String errorMessageB = "Missing required configuration \"b\" which has no default value."; + String errorMessageC = "Missing required configuration \"c\" which has no default value."; + String errorMessageD = "d is referred in the dependents, but not defined."; + + ConfigValue configA = new ConfigValue("a", 1, Arrays.asList(1, 2, 3), Collections.emptyList()); + ConfigValue configB = new ConfigValue("b", null, Arrays.asList(4, 5), Arrays.asList(errorMessageB)); + ConfigValue configC = new ConfigValue("c", null, Arrays.asList(4, 5), Arrays.asList(errorMessageC)); + ConfigValue configD = new ConfigValue("d", null, Collections.emptyList(), Arrays.asList(errorMessageD)); + configD.visible(false); + + expected.put("a", configA); + expected.put("b", configB); + expected.put("c", configC); + expected.put("d", configD); + + ConfigDef def = new ConfigDef() + .define("a", Type.INT, Importance.HIGH, "docs", "group", 1, Width.SHORT, "a", Arrays.asList("b", "c", "d"), new IntegerRecommender(false)) + .define("b", Type.INT, Importance.HIGH, "docs", "group", 2, Width.SHORT, "b", new IntegerRecommender(true)) + .define("c", Type.INT, Importance.HIGH, "docs", "group", 3, Width.SHORT, "c", new IntegerRecommender(true)); + + Map props = new HashMap<>(); + props.put("a", "1"); + + List configs = def.validate(props); + for (ConfigValue config: configs) { + String name = config.name(); + ConfigValue expectedConfig = expected.get(name); + assertEquals(expectedConfig, config); + } + } + + @Test + public void testValidateCannotParse() { + Map expected = new HashMap<>(); + String errorMessageB = "Invalid value non_integer for configuration a: Not a number of type INT"; + ConfigValue configA = new ConfigValue("a", null, Collections.emptyList(), Arrays.asList(errorMessageB)); + expected.put("a", configA); + + ConfigDef def = new ConfigDef().define("a", Type.INT, Importance.HIGH, "docs"); + Map props = new HashMap<>(); + props.put("a", "non_integer"); + + List configs = def.validate(props); + for (ConfigValue config: configs) { + String name = config.name(); + ConfigValue expectedConfig = expected.get(name); + assertEquals(expectedConfig, config); + } + } + + private static class IntegerRecommender implements ConfigDef.Recommender { + + private boolean hasParent; + + public IntegerRecommender(boolean hasParent) { + this.hasParent = hasParent; + } + + @Override + public List validValues(String name, Map parsedConfig) { + List values = new LinkedList<>(); + if (!hasParent) { + values.addAll(Arrays.asList(1, 2, 3)); + } else { + values.addAll(Arrays.asList(4, 5)); + } + return values; + } + + @Override + public boolean visible(String name, Map parsedConfig) { + return true; + } + } + private void testValidators(Type type, Validator validator, Object defaultVal, Object[] okValues, Object[] badValues) { ConfigDef def = new ConfigDef().define("name", type, defaultVal, validator, Importance.HIGH, "docs"); diff --git a/connect/api/src/main/java/org/apache/kafka/connect/connector/Connector.java b/connect/api/src/main/java/org/apache/kafka/connect/connector/Connector.java index 934cdbd1da7b9..137015603c59b 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/connector/Connector.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/connector/Connector.java @@ -18,6 +18,9 @@ package org.apache.kafka.connect.connector; import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.common.config.Config; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigValue; import java.util.List; import java.util.Map; @@ -121,4 +124,22 @@ public void reconfigure(Map props) { * Stop this connector. */ public abstract void stop(); + + /** + * Validate the connector configuration values against configuration definitions. + * @param connectorConfigs the provided configuration values + * @return List of Config, each Config contains the updated configuration information given + * the current configuration values. + */ + public Config validate(Map connectorConfigs) { + ConfigDef configDef = config(); + List configValues = configDef.validate(connectorConfigs); + return new Config(configValues); + } + + /** + * Define the configuration for the connector. + * @return The ConfigDef for this connector. + */ + public abstract ConfigDef config(); } diff --git a/connect/api/src/test/java/org/apache/kafka/connect/connector/ConnectorReconfigurationTest.java b/connect/api/src/test/java/org/apache/kafka/connect/connector/ConnectorReconfigurationTest.java index 7ea1de2c211b7..0517b663a166e 100644 --- a/connect/api/src/test/java/org/apache/kafka/connect/connector/ConnectorReconfigurationTest.java +++ b/connect/api/src/test/java/org/apache/kafka/connect/connector/ConnectorReconfigurationTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.connector; +import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.connect.errors.ConnectException; import org.junit.Test; @@ -43,6 +44,7 @@ public void testReconfigureStopException() throws Exception { } private static class TestConnector extends Connector { + private boolean stopException; private int order = 0; public int stopOrder = -1; @@ -78,5 +80,10 @@ public void stop() { if (stopException) throw new ConnectException("error"); } + + @Override + public ConfigDef config() { + return new ConfigDef(); + } } -} \ No newline at end of file +} diff --git a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSinkConnector.java b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSinkConnector.java index a73153f17ffd1..d42331310a252 100644 --- a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSinkConnector.java +++ b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSinkConnector.java @@ -17,6 +17,9 @@ package org.apache.kafka.connect.file; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.connect.connector.Task; import org.apache.kafka.connect.sink.SinkConnector; @@ -31,7 +34,10 @@ * sink modes via its 'mode' setting. */ public class FileStreamSinkConnector extends SinkConnector { + public static final String FILE_CONFIG = "file"; + private static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(FILE_CONFIG, Type.STRING, Importance.HIGH, "Destination filename."); private String filename; @@ -66,4 +72,9 @@ public List> taskConfigs(int maxTasks) { public void stop() { // Nothing to do since FileStreamSinkConnector has no background monitoring. } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } } diff --git a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceConnector.java b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceConnector.java index 843e999e600c6..4fb33b73ceabf 100644 --- a/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceConnector.java +++ b/connect/file/src/main/java/org/apache/kafka/connect/file/FileStreamSourceConnector.java @@ -17,6 +17,9 @@ package org.apache.kafka.connect.file; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.connect.connector.Task; import org.apache.kafka.connect.errors.ConnectException; @@ -35,6 +38,10 @@ public class FileStreamSourceConnector extends SourceConnector { public static final String TOPIC_CONFIG = "topic"; public static final String FILE_CONFIG = "file"; + private static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(FILE_CONFIG, Type.STRING, Importance.HIGH, "Source filename.") + .define(TOPIC_CONFIG, Type.STRING, Importance.HIGH, "The topic to publish data to"); + private String filename; private String topic; @@ -74,4 +81,9 @@ public List> taskConfigs(int maxTasks) { public void stop() { // Nothing to do since FileStreamSourceConnector has no background monitoring. } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java index ca85d87ceeb2d..8d83644d6fbc6 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java @@ -16,7 +16,16 @@ **/ package org.apache.kafka.connect.runtime; +import org.apache.kafka.common.config.Config; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigDef.ConfigKey; +import org.apache.kafka.common.config.ConfigValue; +import org.apache.kafka.connect.connector.Connector; import org.apache.kafka.connect.errors.NotFoundException; +import org.apache.kafka.connect.runtime.rest.entities.ConfigInfo; +import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos; +import org.apache.kafka.connect.runtime.rest.entities.ConfigKeyInfo; +import org.apache.kafka.connect.runtime.rest.entities.ConfigValueInfo; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; import org.apache.kafka.connect.storage.StatusBackingStore; import org.apache.kafka.connect.util.ConnectorTaskId; @@ -27,7 +36,11 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; +import java.util.HashMap; +import java.util.LinkedList; import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; /** * Abstract Herder implementation which handles connector/task lifecycle tracking. Extensions @@ -52,10 +65,14 @@ */ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, ConnectorStatus.Listener { + protected final Worker worker; protected final StatusBackingStore statusBackingStore; private final String workerId; - public AbstractHerder(StatusBackingStore statusBackingStore, String workerId) { + protected Map tempConnectors = new ConcurrentHashMap<>(); + + public AbstractHerder(Worker worker, StatusBackingStore statusBackingStore, String workerId) { + this.worker = worker; this.statusBackingStore = statusBackingStore; this.workerId = workerId; } @@ -143,6 +160,95 @@ public ConnectorStateInfo.TaskState taskStatus(ConnectorTaskId id) { status.workerId(), status.trace()); } + + @Override + public ConfigInfos validateConfigs(String connType, Map connectorConfig) { + ConfigDef connectorConfigDef = ConnectorConfig.configDef(); + List connectorConfigValues = connectorConfigDef.validate(connectorConfig); + ConfigInfos result = generateResult(connType, connectorConfigDef.configKeys(), connectorConfigValues, Collections.emptyList()); + + if (result.errorCount() != 0) { + return result; + } + + Connector connector = getConnector(connType); + + Config config = connector.validate(connectorConfig); + ConfigDef configDef = connector.config(); + Map configKeys = configDef.configKeys(); + List configValues = config.configValues(); + + Map resultConfigKeys = new HashMap<>(configKeys); + resultConfigKeys.putAll(connectorConfigDef.configKeys()); + configValues.addAll(connectorConfigValues); + + List allGroups = new LinkedList<>(connectorConfigDef.groups()); + List groups = configDef.groups(); + allGroups.addAll(groups); + + return generateResult(connType, resultConfigKeys, configValues, allGroups); + } + + // public for testing + public static ConfigInfos generateResult(String connType, Map configKeys, List configValues, List groups) { + int errorCount = 0; + List configInfoList = new LinkedList<>(); + + Map configValueMap = new HashMap<>(); + for (ConfigValue configValue: configValues) { + String configName = configValue.name(); + configValueMap.put(configName, configValue); + if (!configKeys.containsKey(configName)) { + configValue.addErrorMessage("Configuration is not defined: " + configName); + configInfoList.add(new ConfigInfo(null, convertConfigValue(configValue))); + } + } + + for (String configName: configKeys.keySet()) { + ConfigKeyInfo configKeyInfo = convertConfigKey(configKeys.get(configName)); + ConfigValueInfo configValueInfo = null; + if (configValueMap.containsKey(configName)) { + ConfigValue configValue = configValueMap.get(configName); + configValueInfo = convertConfigValue(configValue); + errorCount += configValue.errorMessages().size(); + } + configInfoList.add(new ConfigInfo(configKeyInfo, configValueInfo)); + } + return new ConfigInfos(connType, errorCount, groups, configInfoList); + } + + private static ConfigKeyInfo convertConfigKey(ConfigKey configKey) { + String name = configKey.name; + String type = configKey.type.name(); + Object defaultValue = configKey.defaultValue; + boolean required = false; + if (defaultValue == ConfigDef.NO_DEFAULT_VALUE) { + required = true; + } + String importance = configKey.importance.name(); + String documentation = configKey.documentation; + String group = configKey.group; + int orderInGroup = configKey.orderInGroup; + String width = configKey.width.name(); + String displayName = configKey.displayName; + List dependents = configKey.dependents; + return new ConfigKeyInfo(name, type, required, defaultValue, importance, documentation, group, orderInGroup, width, displayName, dependents); + } + + private static ConfigValueInfo convertConfigValue(ConfigValue configValue) { + return new ConfigValueInfo(configValue.name(), configValue.value(), configValue.recommendedValues(), configValue.errorMessages(), configValue.visible()); + } + + private Connector getConnector(String connType) { + if (tempConnectors.containsKey(connType)) { + return tempConnectors.get(connType); + } else { + Connector connector = worker.getConnector(connType); + tempConnectors.put(connType, connector); + return connector; + } + } + private String trace(Throwable t) { ByteArrayOutputStream output = new ByteArrayOutputStream(); t.printStackTrace(new PrintStream(output)); @@ -152,5 +258,4 @@ private String trace(Throwable t) { return null; } } - } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java index 4824acdc50732..e21faf6cbf161 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; +import org.apache.kafka.common.config.ConfigDef.Width; import java.util.HashMap; import java.util.Map; @@ -37,32 +38,41 @@ *

      */ public class ConnectorConfig extends AbstractConfig { + private static final String COMMON_GROUP = "Common"; public static final String NAME_CONFIG = "name"; private static final String NAME_DOC = "Globally unique name to use for this connector."; + private static final String NAME_DISPLAY = "Connector name"; public static final String CONNECTOR_CLASS_CONFIG = "connector.class"; private static final String CONNECTOR_CLASS_DOC = "Name or alias of the class for this connector. Must be a subclass of org.apache.kafka.connect.connector.Connector. " + "If the connector is org.apache.kafka.connect.file.FileStreamSinkConnector, you can either specify this full name, " + " or use \"FileStreamSink\" or \"FileStreamSinkConnector\" to make the configuration a bit shorter"; + private static final String CONNECTOR_CLASS_DISPLAY = "Connector class"; public static final String TASKS_MAX_CONFIG = "tasks.max"; private static final String TASKS_MAX_DOC = "Maximum number of tasks to use for this connector."; public static final int TASKS_MAX_DEFAULT = 1; + private static final String TASK_MAX_DISPLAY = "Tasks max"; public static final String TOPICS_CONFIG = "topics"; private static final String TOPICS_DOC = ""; public static final String TOPICS_DEFAULT = ""; + private static final String TOPICS_DISPLAY = "Topics"; private static ConfigDef config; static { config = new ConfigDef() - .define(NAME_CONFIG, Type.STRING, Importance.HIGH, NAME_DOC) - .define(CONNECTOR_CLASS_CONFIG, Type.STRING, Importance.HIGH, CONNECTOR_CLASS_DOC) - .define(TASKS_MAX_CONFIG, Type.INT, TASKS_MAX_DEFAULT, Importance.HIGH, TASKS_MAX_DOC) - .define(TOPICS_CONFIG, Type.LIST, TOPICS_DEFAULT, Importance.HIGH, TOPICS_DOC); + .define(NAME_CONFIG, Type.STRING, Importance.HIGH, NAME_DOC, COMMON_GROUP, 1, Width.MEDIUM, NAME_DISPLAY) + .define(CONNECTOR_CLASS_CONFIG, Type.STRING, Importance.HIGH, CONNECTOR_CLASS_DOC, COMMON_GROUP, 2, Width.LONG, CONNECTOR_CLASS_DISPLAY) + .define(TASKS_MAX_CONFIG, Type.INT, TASKS_MAX_DEFAULT, Importance.HIGH, TASKS_MAX_DOC, COMMON_GROUP, 3, Width.SHORT, TASK_MAX_DISPLAY) + .define(TOPICS_CONFIG, Type.LIST, TOPICS_DEFAULT, Importance.HIGH, TOPICS_DOC, COMMON_GROUP, 4, Width.LONG, TOPICS_DISPLAY); + } + + public static ConfigDef configDef() { + return config; } public ConnectorConfig() { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java index 95c7700116602..3ea4a81bb6da5 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.runtime; +import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos; import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; import org.apache.kafka.connect.runtime.rest.entities.TaskInfo; @@ -127,6 +128,12 @@ public interface Herder { */ ConnectorStateInfo.TaskState taskStatus(ConnectorTaskId id); + /** + * Validate the provided connector config values against the configuration definition. + * @param connType the connector class + * @param connectorConfig the provided connector config values + */ + ConfigInfos validateConfigs(String connType, Map connectorConfig); class Created { private final boolean created; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java index aa574935d31af..1a9ff110db58c 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java @@ -189,6 +189,11 @@ public boolean isSinkConnector(String connName) { return SinkConnector.class.isAssignableFrom(workerConnector.delegate.getClass()); } + public Connector getConnector(String connType) { + Class connectorClass = getConnectorClass(connType); + return instantiateConnector(connectorClass); + } + @SuppressWarnings("unchecked") private Class getConnectorClass(String connectorAlias) { // Avoid the classpath scan if the full class name was provided diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java index 16b950b55e036..2fc8297a5f644 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java @@ -84,7 +84,6 @@ public class DistributedHerder extends AbstractHerder implements Runnable { private static final long RECONFIGURE_CONNECTOR_TASKS_BACKOFF_MS = 250; - private final Worker worker; private final KafkaConfigStorage configStorage; private ClusterConfigState configState; private final Time time; @@ -130,9 +129,8 @@ public DistributedHerder(DistributedConfig config, WorkerGroupMember member, String restUrl, Time time) { - super(statusBackingStore, workerId); + super(worker, statusBackingStore, workerId); - this.worker = worker; if (configStorage != null) { // For testing. Assume configuration has already been performed this.configStorage = configStorage; @@ -551,6 +549,7 @@ public int generation() { return generation; } + // Should only be called from work thread, so synchronization should not be needed private boolean isLeader() { return assignment != null && member.memberId().equals(assignment.leader()); @@ -701,7 +700,6 @@ private void startConnector(String connectorName) { String connName = connConfig.getString(ConnectorConfig.NAME_CONFIG); ConnectorContext ctx = new HerderConnectorContext(DistributedHerder.this, connName); worker.startConnector(connConfig, ctx, this); - // Immediately request configuration since this could be a brand new connector. However, also only update those // task configs if they are actually different from the existing ones to avoid unnecessary updates when this is // just restoring an existing connector. diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java index dbac58f230d23..7e4279a83992a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java @@ -20,12 +20,14 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; + import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.runtime.Herder; import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.rest.entities.ErrorMessage; import org.apache.kafka.connect.runtime.rest.errors.ConnectExceptionMapper; import org.apache.kafka.connect.runtime.rest.errors.ConnectRestException; +import org.apache.kafka.connect.runtime.rest.resources.ConnectorPluginsResource; import org.apache.kafka.connect.runtime.rest.resources.ConnectorsResource; import org.apache.kafka.connect.runtime.rest.resources.RootResource; import org.eclipse.jetty.server.Connector; @@ -44,8 +46,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.ws.rs.core.Response; -import javax.ws.rs.core.UriBuilder; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -55,6 +55,9 @@ import java.util.List; import java.util.Map; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.UriBuilder; + /** * Embedded server for the REST API that provides the control plane for Kafka Connect workers. */ @@ -95,6 +98,7 @@ public void start(Herder herder) { resourceConfig.register(RootResource.class); resourceConfig.register(new ConnectorsResource(herder)); + resourceConfig.register(new ConnectorPluginsResource(herder)); resourceConfig.register(ConnectExceptionMapper.class); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigInfo.java new file mode 100644 index 0000000000000..6040563155e4f --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigInfo.java @@ -0,0 +1,66 @@ +/** + * 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 org.apache.kafka.connect.runtime.rest.entities; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +public class ConfigInfo { + + private ConfigKeyInfo configKey; + private ConfigValueInfo configValue; + + @JsonCreator + public ConfigInfo( + @JsonProperty("definition") ConfigKeyInfo configKey, + @JsonProperty("value") ConfigValueInfo configValue) { + this.configKey = configKey; + this.configValue = configValue; + } + + @JsonProperty("definition") + public ConfigKeyInfo configKey() { + return configKey; + } + + @JsonProperty("value") + public ConfigValueInfo configValue() { + return configValue; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ConfigInfo that = (ConfigInfo) o; + return Objects.equals(configKey, that.configKey) && + Objects.equals(configValue, that.configValue); + } + + @Override + public int hashCode() { + return Objects.hash(configKey, configValue); + } + + @Override + public String toString() { + return "[" + configKey.toString() + "," + configValue.toString() + "]"; + } +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigInfos.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigInfos.java new file mode 100644 index 0000000000000..3e73983fe02cb --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigInfos.java @@ -0,0 +1,102 @@ +/** + * 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 org.apache.kafka.connect.runtime.rest.entities; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; +import java.util.Objects; + +public class ConfigInfos { + + @JsonProperty("name") + private final String name; + + @JsonProperty("error_count") + private final int errorCount; + + @JsonProperty("groups") + private final List groups; + + @JsonProperty("configs") + private final List configs; + + @JsonCreator + public ConfigInfos(@JsonProperty("name") String name, + @JsonProperty("error_count") int errorCount, + @JsonProperty("groups") List groups, + @JsonProperty("configs") List configs) { + this.name = name; + this.groups = groups; + this.errorCount = errorCount; + this.configs = configs; + } + + @JsonProperty + public String name() { + return name; + } + + @JsonProperty + public List groups() { + return groups; + } + + @JsonProperty("error_count") + public int errorCount() { + return errorCount; + } + + @JsonProperty("configs") + public List values() { + return configs; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ConfigInfos that = (ConfigInfos) o; + return Objects.equals(name, that.name) && + Objects.equals(errorCount, that.errorCount) && + Objects.equals(groups, that.groups) && + Objects.equals(configs, that.configs); + } + + @Override + public int hashCode() { + return Objects.hash(name, errorCount, groups, configs); + } + + @Override + public String toString() { + StringBuffer sb = new StringBuffer(); + sb.append("[") + .append(name) + .append(",") + .append(errorCount) + .append(",") + .append(groups) + .append(",") + .append(configs) + .append("]"); + return sb.toString(); + } + +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigKeyInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigKeyInfo.java new file mode 100644 index 0000000000000..f8137092efca4 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigKeyInfo.java @@ -0,0 +1,171 @@ +/** + * 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 org.apache.kafka.connect.runtime.rest.entities; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; +import java.util.Objects; + +public class ConfigKeyInfo { + + private final String name; + private final String type; + private final boolean required; + private final Object defaultValue; + private final String importance; + private final String documentation; + private final String group; + private final int orderInGroup; + private final String width; + private final String displayName; + private final List dependents; + + @JsonCreator + public ConfigKeyInfo(@JsonProperty("name") String name, + @JsonProperty("type") String type, + @JsonProperty("required") boolean required, + @JsonProperty("default_value") Object defaultValue, + @JsonProperty("importance") String importance, + @JsonProperty("documentation") String documentation, + @JsonProperty("group") String group, + @JsonProperty("order_in_group") int orderInGroup, + @JsonProperty("width") String width, + @JsonProperty("display_name") String displayName, + @JsonProperty("dependents") List dependents) { + this.name = name; + this.type = type; + this.required = required; + this.defaultValue = defaultValue; + this.importance = importance; + this.documentation = documentation; + this.group = group; + this.orderInGroup = orderInGroup; + this.width = width; + this.displayName = displayName; + this.dependents = dependents; + } + + @JsonProperty + public String name() { + return name; + } + + @JsonProperty + public String type() { + return type; + } + + @JsonProperty + public boolean required() { + return required; + } + + @JsonProperty("default_value") + public Object defaultValue() { + return defaultValue; + } + + @JsonProperty + public String documentation() { + return documentation; + } + + @JsonProperty + public String group() { + return group; + } + + @JsonProperty("order") + public int orderInGroup() { + return orderInGroup; + } + + @JsonProperty + public String width() { + return width; + } + + @JsonProperty + public String importance() { + return importance; + } + + @JsonProperty("display_name") + public String displayName() { + return displayName; + } + + @JsonProperty + public List dependents() { + return dependents; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ConfigKeyInfo that = (ConfigKeyInfo) o; + return Objects.equals(name, that.name) && + Objects.equals(type, that.type) && + Objects.equals(required, that.required) && + Objects.equals(defaultValue, that.defaultValue) && + Objects.equals(importance, that.importance) && + Objects.equals(documentation, that.documentation) && + Objects.equals(group, that.group) && + Objects.equals(orderInGroup, that.orderInGroup) && + Objects.equals(width, that.width) && + Objects.equals(displayName, that.displayName) && + Objects.equals(dependents, that.dependents); + } + + @Override + public int hashCode() { + return Objects.hash(name, type, required, defaultValue, importance, documentation, group, orderInGroup, width, displayName, dependents); + } + + @Override + public String toString() { + StringBuffer sb = new StringBuffer(); + sb.append("[") + .append(name) + .append(",") + .append(type) + .append(",") + .append(required) + .append(",") + .append(defaultValue) + .append(",") + .append(importance) + .append(",") + .append(documentation) + .append(",") + .append(group) + .append(",") + .append(orderInGroup) + .append(",") + .append(width) + .append(",") + .append(displayName) + .append(",") + .append(dependents) + .append("]"); + return sb.toString(); + } +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigValueInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigValueInfo.java new file mode 100644 index 0000000000000..51e7ee5b2f420 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigValueInfo.java @@ -0,0 +1,106 @@ +/** + * 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 org.apache.kafka.connect.runtime.rest.entities; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.List; +import java.util.Objects; + +public class ConfigValueInfo { + private String name; + private Object value; + private List recommendedValues; + private List errors; + private boolean visible; + + @JsonCreator + public ConfigValueInfo( + @JsonProperty("name") String name, + @JsonProperty("value") Object value, + @JsonProperty("recommended_values") List recommendedValues, + @JsonProperty("errors") List errors, + @JsonProperty("visible") boolean visible) { + this.name = name; + this.value = value; + this.recommendedValues = recommendedValues; + this.errors = errors; + this.visible = visible; + } + + @JsonProperty + public String name() { + return name; + } + + @JsonProperty + public Object value() { + return value; + } + + @JsonProperty("recommended_values") + public List recommendedValues() { + return recommendedValues; + } + + @JsonProperty + public List errors() { + return errors; + } + + @JsonProperty + public boolean visible() { + return visible; + } + + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + ConfigValueInfo that = (ConfigValueInfo) o; + return Objects.equals(name, that.name) && + Objects.equals(value, that.value) && + Objects.equals(recommendedValues, that.recommendedValues) && + Objects.equals(errors, that.errors) && + Objects.equals(visible, that.visible); + } + + @Override + public int hashCode() { + return Objects.hash(name, value, recommendedValues, errors, visible); + } + + @Override + public String toString() { + StringBuffer sb = new StringBuffer(); + sb.append("[") + .append(name) + .append(",") + .append(value) + .append(",") + .append(recommendedValues) + .append(",") + .append(errors) + .append(",") + .append(visible) + .append("]"); + return sb.toString(); + } + +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorInfo.java index 8daae05f7c2ef..9567ef925352a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorInfo.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorInfo.java @@ -19,6 +19,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; + import org.apache.kafka.connect.util.ConnectorTaskId; import java.util.ArrayList; @@ -34,13 +35,15 @@ public class ConnectorInfo { private final List tasks; @JsonCreator - public ConnectorInfo(@JsonProperty("name") String name, @JsonProperty("config") Map config, + public ConnectorInfo(@JsonProperty("name") String name, + @JsonProperty("config") Map config, @JsonProperty("tasks") List tasks) { this.name = name; this.config = config; this.tasks = tasks; } + @JsonProperty public String name() { return name; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java new file mode 100644 index 0000000000000..84397078033a0 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java @@ -0,0 +1,49 @@ +/** + * 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 org.apache.kafka.connect.runtime.rest.resources; + +import org.apache.kafka.connect.runtime.Herder; +import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos; + +import java.util.Map; + +import javax.ws.rs.Consumes; +import javax.ws.rs.PUT; +import javax.ws.rs.Path; +import javax.ws.rs.PathParam; +import javax.ws.rs.Produces; +import javax.ws.rs.core.MediaType; + +@Path("/connector-plugins") +@Produces(MediaType.APPLICATION_JSON) +@Consumes(MediaType.APPLICATION_JSON) +public class ConnectorPluginsResource { + + private final Herder herder; + + public ConnectorPluginsResource(Herder herder) { + this.herder = herder; + } + + @PUT + @Path("/{connectorType}/config/validate") + public ConfigInfos validateConfigs(final @PathParam("connectorType") String connType, + final Map connectorConfig) throws Throwable { + return herder.validateConfigs(connType, connectorConfig); + } +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java index d0d940b839281..b6e9f61a75b37 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java @@ -18,6 +18,7 @@ package org.apache.kafka.connect.runtime.rest.resources; import com.fasterxml.jackson.core.type.TypeReference; + import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.runtime.Herder; import org.apache.kafka.connect.runtime.distributed.NotLeaderException; @@ -32,6 +33,14 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.net.URI; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + import javax.servlet.ServletContext; import javax.ws.rs.Consumes; import javax.ws.rs.DELETE; @@ -43,13 +52,6 @@ import javax.ws.rs.Produces; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; -import java.net.URI; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; @Path("/connectors") @Produces(MediaType.APPLICATION_JSON) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java index 707470f9505e0..9c48ed79dd622 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java @@ -49,7 +49,6 @@ public class StandaloneHerder extends AbstractHerder { private static final Logger log = LoggerFactory.getLogger(StandaloneHerder.class); - private final Worker worker; private HashMap connectors = new HashMap<>(); public StandaloneHerder(Worker worker) { @@ -60,8 +59,7 @@ public StandaloneHerder(Worker worker) { StandaloneHerder(String workerId, Worker worker, StatusBackingStore statusBackingStore) { - super(statusBackingStore, workerId); - this.worker = worker; + super(worker, statusBackingStore, workerId); } public synchronized void start() { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSinkConnector.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSinkConnector.java index 0ab64fdfa2602..c2515a07a4f8a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSinkConnector.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSinkConnector.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.tools; +import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.connect.connector.Task; import org.apache.kafka.connect.source.SourceConnector; @@ -61,4 +62,9 @@ public List> taskConfigs(int maxTasks) { @Override public void stop() { } + + @Override + public ConfigDef config() { + return new ConfigDef(); + } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSourceConnector.java b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSourceConnector.java index 5f9afd5de706b..b18db6eeb2d0d 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSourceConnector.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/tools/VerifiableSourceConnector.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.tools; +import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.connect.connector.Task; import org.apache.kafka.connect.source.SourceConnector; @@ -61,4 +62,9 @@ public List> taskConfigs(int maxTasks) { @Override public void stop() { } + + @Override + public ConfigDef config() { + return new ConfigDef(); + } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java index f17023cb53a8b..1dc57846213f5 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java @@ -34,6 +34,7 @@ public class AbstractHerderTest extends EasyMockSupport { @Test public void connectorStatus() { + Worker worker = null; String workerId = "workerId"; String connector = "connector"; int generation = 5; @@ -42,8 +43,8 @@ public void connectorStatus() { StatusBackingStore store = strictMock(StatusBackingStore.class); AbstractHerder herder = partialMockBuilder(AbstractHerder.class) - .withConstructor(StatusBackingStore.class, String.class) - .withArgs(store, workerId) + .withConstructor(Worker.class, StatusBackingStore.class, String.class) + .withArgs(worker, store, workerId) .addMockedMethod("generation") .createMock(); @@ -76,14 +77,15 @@ public void connectorStatus() { @Test public void taskStatus() { + Worker worker = null; ConnectorTaskId taskId = new ConnectorTaskId("connector", 0); String workerId = "workerId"; StatusBackingStore store = strictMock(StatusBackingStore.class); AbstractHerder herder = partialMockBuilder(AbstractHerder.class) - .withConstructor(StatusBackingStore.class, String.class) - .withArgs(store, workerId) + .withConstructor(Worker.class, StatusBackingStore.class, String.class) + .withArgs(worker, store, workerId) .addMockedMethod("generation") .createMock(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java index 67d3fdc4a744b..557d7891610ab 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java @@ -18,6 +18,7 @@ package org.apache.kafka.connect.runtime; import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.connector.Connector; import org.apache.kafka.connect.connector.ConnectorContext; @@ -466,7 +467,11 @@ private void expectStopStorage() { /* Name here needs to be unique as we are testing the aliasing mechanism */ - private static class WorkerTestConnector extends Connector { + public static class WorkerTestConnector extends Connector { + + private static final ConfigDef CONFIG_DEF = new ConfigDef() + .define("configName", ConfigDef.Type.STRING, ConfigDef.Importance.HIGH, "Test configName."); + @Override public String version() { return "1.0"; @@ -491,6 +496,11 @@ public List> taskConfigs(int maxTasks) { public void stop() { } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } } private static class TestSourceTask extends SourceTask { diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java new file mode 100644 index 0000000000000..625c91fa0edd7 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java @@ -0,0 +1,165 @@ +/** + * 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 org.apache.kafka.connect.runtime.rest.resources; + +import com.fasterxml.jackson.core.type.TypeReference; + +import org.apache.kafka.common.config.Config; +import org.apache.kafka.common.config.ConfigDef; +import org.apache.kafka.common.config.ConfigDef.Type; +import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.connect.connector.Connector; +import org.apache.kafka.connect.connector.Task; +import org.apache.kafka.connect.runtime.AbstractHerder; +import org.apache.kafka.connect.runtime.Herder; +import org.apache.kafka.connect.runtime.rest.RestServer; +import org.apache.kafka.connect.runtime.rest.entities.ConfigInfo; +import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos; +import org.apache.kafka.connect.runtime.rest.entities.ConfigKeyInfo; +import org.apache.kafka.connect.runtime.rest.entities.ConfigValueInfo; +import org.easymock.EasyMock; +import org.easymock.IAnswer; +import org.junit.Before; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.api.easymock.annotation.Mock; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.LinkedList; +import java.util.List; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +@RunWith(PowerMockRunner.class) +@PrepareForTest(RestServer.class) +@PowerMockIgnore("javax.management.*") +public class ConnectorPluginsResourceTest { + + private static Map props = new HashMap<>(); + static { + props.put("test.string.config", "testString"); + props.put("test.int.config", "10"); + } + + private static final ConfigInfos CONFIG_INFOS; + static { + List configs = new LinkedList<>(); + + ConfigKeyInfo configKeyInfo = new ConfigKeyInfo("test.string.config", "STRING", true, "", "HIGH", "Test configuration for string type.", null, -1, "NONE", "test.string.config", new LinkedList()); + ConfigValueInfo configValueInfo = new ConfigValueInfo("test.string.config", "testString", Collections.emptyList(), Collections.emptyList(), true); + ConfigInfo configInfo = new ConfigInfo(configKeyInfo, configValueInfo); + configs.add(configInfo); + + configKeyInfo = new ConfigKeyInfo("test.int.config", "INT", true, "", "MEDIUM", "Test configuration for integer type.", null, -1, "NONE", "test.int.config", new LinkedList()); + configValueInfo = new ConfigValueInfo("test.int.config", 10, Collections.emptyList(), Collections.emptyList(), true); + configInfo = new ConfigInfo(configKeyInfo, configValueInfo); + configs.add(configInfo); + + configKeyInfo = new ConfigKeyInfo("test.string.config.default", "STRING", false, "", "LOW", "Test configuration with default value.", null, -1, "NONE", "test.string.config.default", new LinkedList()); + configValueInfo = new ConfigValueInfo("test.string.config.default", "", Collections.emptyList(), Collections.emptyList(), true); + configInfo = new ConfigInfo(configKeyInfo, configValueInfo); + configs.add(configInfo); + + CONFIG_INFOS = new ConfigInfos(ConnectorPluginsResourceTestConnector.class.getName(), 0, Collections.emptyList(), configs); + } + + @Mock + private Herder herder; + private ConnectorPluginsResource connectorPluginsResource; + + @Before + public void setUp() throws NoSuchMethodException { + PowerMock.mockStatic(RestServer.class, + RestServer.class.getMethod("httpRequest", String.class, String.class, Object.class, TypeReference.class)); + connectorPluginsResource = new ConnectorPluginsResource(herder); + } + + @Test + public void testValidateConfig() throws Throwable { + herder.validateConfigs(EasyMock.eq(ConnectorPluginsResourceTestConnector.class.getName()), EasyMock.eq(props)); + + PowerMock.expectLastCall().andAnswer(new IAnswer() { + @Override + public ConfigInfos answer() { + Config config = new ConnectorPluginsResourceTestConnector().validate(props); + Connector connector = new ConnectorPluginsResourceTestConnector(); + ConfigDef configDef = connector.config(); + return AbstractHerder.generateResult(ConnectorPluginsResourceTestConnector.class.getName(), configDef.configKeys(), config.configValues(), configDef.groups()); + } + }); + PowerMock.replayAll(); + + ConfigInfos configInfos = connectorPluginsResource.validateConfigs(ConnectorPluginsResourceTestConnector.class.getName(), props); + assertEquals(CONFIG_INFOS.name(), configInfos.name()); + assertEquals(CONFIG_INFOS.errorCount(), configInfos.errorCount()); + assertEquals(CONFIG_INFOS.groups(), configInfos.groups()); + assertEquals(new HashSet<>(CONFIG_INFOS.values()), new HashSet<>(configInfos.values())); + + PowerMock.verifyAll(); + } + + /* Name here needs to be unique as we are testing the aliasing mechanism */ + public static class ConnectorPluginsResourceTestConnector extends Connector { + + public static final String TEST_STRING_CONFIG = "test.string.config"; + public static final String TEST_INT_CONFIG = "test.int.config"; + public static final String TEST_STRING_CONFIG_DEFAULT = "test.string.config.default"; + + private static final ConfigDef CONFIG_DEF = new ConfigDef() + .define(TEST_STRING_CONFIG, Type.STRING, Importance.HIGH, "Test configuration for string type.") + .define(TEST_INT_CONFIG, Type.INT, Importance.MEDIUM, "Test configuration for integer type.") + .define(TEST_STRING_CONFIG_DEFAULT, Type.STRING, "", Importance.LOW, "Test configuration with default value."); + + @Override + public String version() { + return "1.0"; + } + + @Override + public void start(Map props) { + + } + + @Override + public Class taskClass() { + return null; + } + + @Override + public List> taskConfigs(int maxTasks) { + return null; + } + + @Override + public void stop() { + + } + + @Override + public ConfigDef config() { + return CONFIG_DEF; + } + } +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java index 4659ae8667611..970f56c9522cc 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java @@ -18,6 +18,7 @@ package org.apache.kafka.connect.runtime.rest.resources; import com.fasterxml.jackson.core.type.TypeReference; + import org.apache.kafka.connect.errors.AlreadyExistsException; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.NotFoundException; @@ -83,7 +84,6 @@ public class ConnectorsResourceTest { TASK_INFOS.add(new TaskInfo(new ConnectorTaskId(CONNECTOR_NAME, 1), TASK_CONFIGS.get(1))); } - @Mock private Herder herder; private ConnectorsResource connectorsResource; @@ -172,6 +172,8 @@ public void testCreateConnectorNotLeader() throws Throwable { connectorsResource.createConnector(body); PowerMock.verifyAll(); + + } @Test(expected = AlreadyExistsException.class) From 8ef804dc194bb562b6dbe48855e81965cacd1114 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 17 Mar 2016 13:56:27 -0700 Subject: [PATCH 045/206] KAFKA-3414; Return of MetadataCache.getAliveBrokers should not be mutated by cache updates `Map.values` returns `DefaultValuesIterable` where the default implementation of `toSeq` is (sadly) `toStream`. `Stream` is a lazy collection and it can reflect changes to the underlying map before it's `forced`. I verified that the test failed before my change. Author: Ismael Juma Reviewers: Gwen Shapira Closes #1088 from ijuma/kafka-3414-get-alive-brokers-no-mutation --- .../scala/kafka/server/MetadataCache.scala | 2 +- .../unit/kafka/server/MetadataCacheTest.scala | 28 +++++++++++++++++++ 2 files changed, 29 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index b23ecbe75b9d1..4b68f707320ed 100755 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -132,7 +132,7 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { def getAliveBrokers: Seq[Broker] = { inReadLock(partitionMetadataLock) { - aliveBrokers.values.toSeq + aliveBrokers.values.toBuffer } } diff --git a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala index dcc310fffae3e..017faeabaa617 100644 --- a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala +++ b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala @@ -238,4 +238,32 @@ class MetadataCacheTest { } + @Test + def getAliveBrokersShouldNotBeMutatedByUpdateCache() { + val topic = "topic" + val cache = new MetadataCache(1) + + def updateCache(brokerIds: Set[Int]) { + val brokers = brokerIds.map { brokerId => + new Broker(brokerId, Map(SecurityProtocol.PLAINTEXT -> new EndPoint("foo", 9092)).asJava, "") + } + val controllerEpoch = 1 + val leader = 0 + val leaderEpoch = 0 + val replicas = asSet[Integer](0) + val isr = asList[Integer](0, 1) + val partitionStates = Map( + new TopicPartition(topic, 0) -> new PartitionState(controllerEpoch, leader, leaderEpoch, isr, 3, replicas)) + val updateMetadataRequest = new UpdateMetadataRequest(2, controllerEpoch, partitionStates.asJava, brokers.asJava) + cache.updateCache(15, updateMetadataRequest) + } + + val initialBrokerIds = (0 to 2).toSet + updateCache(initialBrokerIds) + val aliveBrokersFromCache = cache.getAliveBrokers + // This should not change `aliveBrokersFromCache` + updateCache((0 to 3).toSet) + assertEquals(initialBrokerIds, aliveBrokersFromCache.map(_.id).toSet) + } + } From 3f28d77449bd3280f53edab81657e6c2a37f3c3e Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 17 Mar 2016 13:59:56 -0700 Subject: [PATCH 046/206] KAFKA-3373; MINOR: follow-up, a few val renames remaining I also slightly tweaked the wording on a couple of warnings. Author: Ismael Juma Reviewers: Gwen Shapira Closes #1072 from ijuma/kafka-3373-follow-up --- .../scala/kafka/admin/ConfigCommand.scala | 2 +- .../main/scala/kafka/admin/TopicCommand.scala | 2 +- core/src/main/scala/kafka/log/LogConfig.scala | 6 ++--- .../main/scala/kafka/server/KafkaConfig.scala | 24 +++++++++---------- 4 files changed, 17 insertions(+), 17 deletions(-) diff --git a/core/src/main/scala/kafka/admin/ConfigCommand.scala b/core/src/main/scala/kafka/admin/ConfigCommand.scala index 276689a0a736f..614e3fe3b46c1 100644 --- a/core/src/main/scala/kafka/admin/ConfigCommand.scala +++ b/core/src/main/scala/kafka/admin/ConfigCommand.scala @@ -119,7 +119,7 @@ object ConfigCommand { configsToBeAdded.foreach(pair => props.setProperty(pair(0).trim, pair(1).trim)) if (props.containsKey(LogConfig.MessageFormatVersionProp)) { println(s"WARNING: The configuration ${LogConfig.MessageFormatVersionProp}=${props.getProperty(LogConfig.MessageFormatVersionProp)} is specified. " + - s"This configuration will be ignored if the value is on a version newer than the specified inter.broker.protocol.version in the broker.") + s"This configuration will be ignored if the version is newer than the inter.broker.protocol.version specified in the broker.") } props } diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index b3b06351da796..dd4ea88a5aeaf 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -239,7 +239,7 @@ object TopicCommand extends Logging { LogConfig.validate(props) if (props.containsKey(LogConfig.MessageFormatVersionProp)) { println(s"WARNING: The configuration ${LogConfig.MessageFormatVersionProp}=${props.getProperty(LogConfig.MessageFormatVersionProp)} is specified. " + - s"This configuration will be ignored if the value is on a version newer than the specified inter.broker.protocol.version in the broker.") + s"This configuration will be ignored if the version is newer than the inter.broker.protocol.version specified in the broker.") } props } diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index ffec85a95edea..698464ebfcea6 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -137,9 +137,9 @@ object LogConfig { "standard compression codecs ('gzip', 'snappy', lz4). It additionally accepts 'uncompressed' which is equivalent to " + "no compression; and 'producer' which means retain the original compression codec set by the producer." val PreAllocateEnableDoc ="Should pre allocate file when create new segment?" - val MessageFormatVersionDoc = KafkaConfig.MessageFormatVersionDoc - val MessageTimestampTypeDoc = KafkaConfig.MessageTimestampTypeDoc - val MessageTimestampDifferenceMaxMsDoc = KafkaConfig.MessageTimestampDifferenceMaxMsDoc + val MessageFormatVersionDoc = KafkaConfig.LogMessageFormatVersionDoc + val MessageTimestampTypeDoc = KafkaConfig.LogMessageTimestampTypeDoc + val MessageTimestampDifferenceMaxMsDoc = KafkaConfig.LogMessageTimestampDifferenceMaxMsDoc private val configDef = { import org.apache.kafka.common.config.ConfigDef.Importance._ diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 9c2487673e262..7e1013e22a5c6 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -428,18 +428,18 @@ object KafkaConfig { val LogFlushIntervalMsDoc = "The maximum time in ms that a message in any topic is kept in memory before flushed to disk. If not set, the value in " + LogFlushSchedulerIntervalMsProp + " is used" val LogFlushOffsetCheckpointIntervalMsDoc = "The frequency with which we update the persistent record of the last flush which acts as the log recovery point" val LogPreAllocateEnableDoc = "Should pre allocate file when create new segment? If you are using Kafka on Windows, you probably need to set it to true." + val LogMessageFormatVersionDoc = "Specify the message format version the broker will use to append messages to the logs. The value should be a valid ApiVersion. " + + "Some examples are: 0.8.2, 0.9.0.0, 0.10.0, check ApiVersion for more details. By setting a particular message format version, the " + + "user is certifying that all the existing messages on disk are smaller or equal than the specified version. Setting this value incorrectly " + + "will cause consumers with older versions to break as they will receive messages with a format that they don't understand." + val LogMessageTimestampTypeDoc = "Define whether the timestamp in the message is message create time or log append time. The value should be either " + + "`CreateTime` or `LogAppendTime`" + val LogMessageTimestampDifferenceMaxMsDoc = "The maximum difference allowed between the timestamp when a broker receives " + + "a message and the timestamp specified in the message. If message.timestamp.type=CreateTime, a message will be rejected " + + "if the difference in timestamp exceeds this threshold. This configuration is ignored if message.timestamp.type=LogAppendTime." val NumRecoveryThreadsPerDataDirDoc = "The number of threads per data directory to be used for log recovery at startup and flushing at shutdown" val AutoCreateTopicsEnableDoc = "Enable auto creation of topic on the server" val MinInSyncReplicasDoc = "define the minimum number of replicas in ISR needed to satisfy a produce request with acks=all (or -1)" - val MessageFormatVersionDoc = "Specify the message format version the broker will use to append messages to the logs. The value should be a valid ApiVersion. " + - "Some examples are: 0.8.2, 0.9.0.0, 0.10.0, check ApiVersion for more details. By setting a particular message format version, the " + - "user is certifying that all the existing messages on disk are smaller or equal than the specified version. Setting this value incorrectly " + - "will cause consumers with older versions to break as they will receive messages with a format that they don't understand." - val MessageTimestampTypeDoc = "Define whether the timestamp in the message is message create time or log append time. The value should be either " + - "`CreateTime` or `LogAppendTime`" - val MessageTimestampDifferenceMaxMsDoc = "The maximum difference allowed between the timestamp when a broker receives " + - "a message and the timestamp specified in the message. If message.timestamp.type=CreateTime, a message will be rejected " + - "if the difference in timestamp exceeds this threshold. This configuration is ignored if message.timestamp.type=LogAppendTime." /** ********* Replication configuration ***********/ val ControllerSocketTimeoutMsDoc = "The socket timeout for controller-to-broker channels" val ControllerMessageQueueSizeDoc = "The buffer size for controller-to-broker-channels" @@ -617,9 +617,9 @@ object KafkaConfig { .define(NumRecoveryThreadsPerDataDirProp, INT, Defaults.NumRecoveryThreadsPerDataDir, atLeast(1), HIGH, NumRecoveryThreadsPerDataDirDoc) .define(AutoCreateTopicsEnableProp, BOOLEAN, Defaults.AutoCreateTopicsEnable, HIGH, AutoCreateTopicsEnableDoc) .define(MinInSyncReplicasProp, INT, Defaults.MinInSyncReplicas, atLeast(1), HIGH, MinInSyncReplicasDoc) - .define(LogMessageFormatVersionProp, STRING, Defaults.LogMessageFormatVersion, MEDIUM, MessageFormatVersionDoc) - .define(LogMessageTimestampTypeProp, STRING, Defaults.LogMessageTimestampType, in("CreateTime", "LogAppendTime"), MEDIUM, MessageTimestampTypeDoc) - .define(LogMessageTimestampDifferenceMaxMsProp, LONG, Defaults.LogMessageTimestampDifferenceMaxMs, atLeast(0), MEDIUM, MessageTimestampDifferenceMaxMsDoc) + .define(LogMessageFormatVersionProp, STRING, Defaults.LogMessageFormatVersion, MEDIUM, LogMessageFormatVersionDoc) + .define(LogMessageTimestampTypeProp, STRING, Defaults.LogMessageTimestampType, in("CreateTime", "LogAppendTime"), MEDIUM, LogMessageTimestampTypeDoc) + .define(LogMessageTimestampDifferenceMaxMsProp, LONG, Defaults.LogMessageTimestampDifferenceMaxMs, atLeast(0), MEDIUM, LogMessageTimestampDifferenceMaxMsDoc) /** ********* Replication configuration ***********/ .define(ControllerSocketTimeoutMsProp, INT, Defaults.ControllerSocketTimeoutMs, MEDIUM, ControllerSocketTimeoutMsDoc) From 61281f5c53ebe130defa881f32084c581cf16d43 Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Thu, 17 Mar 2016 14:14:12 -0700 Subject: [PATCH 047/206] =?UTF-8?q?KAFKA-3250:=20release=20tarball=20is=20?= =?UTF-8?q?unnecessarily=20large=20due=20to=20duplicate=20l=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …ibraries This ensures duplicates are not copied in the distribution without rewriting all of the tar'ing logic. A larger improvement could be made to the packaging code, but that should be tracked by another jira. Author: Grant Henke Reviewers: Gwen Shapira, Ismael Juma Closes #1075 from granthenke/libs-duplicates --- build.gradle | 11 +++++++++++ 1 file changed, 11 insertions(+) diff --git a/build.gradle b/build.gradle index 4b84ba577947b..81e4af5f542a5 100644 --- a/build.gradle +++ b/build.gradle @@ -365,6 +365,7 @@ project(':core') { exclude('kafka-clients*') } into "$buildDir/dependant-libs-${versions.scala}" + duplicatesStrategy 'exclude' } @@ -417,6 +418,7 @@ project(':core') { compression = Compression.GZIP from project.file("../docs") into 'site-docs' + duplicatesStrategy 'exclude' } tasks.create(name: "releaseTarGz", dependsOn: configurations.archives.artifacts, type: Tar) { @@ -443,6 +445,7 @@ project(':core') { from(project(':streams').configurations.runtime) { into("libs/") } from(project(':streams:examples').jar) { into("libs/") } from(project(':streams:examples').configurations.runtime) { into("libs/") } + duplicatesStrategy 'exclude' } jar { @@ -460,6 +463,7 @@ project(':core') { include('*.jar') } into "$buildDir/dependant-testlibs" + duplicatesStrategy 'exclude' } checkstyle { @@ -573,6 +577,7 @@ project(':tools') { exclude('kafka-clients*') } into "$buildDir/dependant-libs-${versions.scala}" + duplicatesStrategy 'exclude' } jar { @@ -608,6 +613,7 @@ project(':streams') { exclude('kafka-clients*') } into "$buildDir/dependant-libs-${versions.scala}" + duplicatesStrategy 'exclude' } jar { @@ -636,6 +642,7 @@ project(':streams:examples') { exclude('kafka-streams*') } into "$buildDir/dependant-libs-${versions.scala}" + duplicatesStrategy 'exclude' } jar { @@ -684,6 +691,7 @@ project(':connect:api') { exclude('connect-*') } into "$buildDir/dependant-libs" + duplicatesStrategy 'exclude' } jar { @@ -720,6 +728,7 @@ project(':connect:json') { exclude('connect-*') } into "$buildDir/dependant-libs" + duplicatesStrategy 'exclude' } jar { @@ -765,6 +774,7 @@ project(':connect:runtime') { exclude('connect-*') } into "$buildDir/dependant-libs" + duplicatesStrategy 'exclude' } jar { @@ -807,6 +817,7 @@ project(':connect:file') { exclude('connect-*') } into "$buildDir/dependant-libs" + duplicatesStrategy 'exclude' } jar { From 579d473ce9c5ef1a442af734e362dd545e5ab988 Mon Sep 17 00:00:00 2001 From: Dong Lin Date: Thu, 17 Mar 2016 15:21:20 -0700 Subject: [PATCH 048/206] KAFKA-3330; Truncate log cleaner offset checkpoint if the log is truncated becketqin Can you take a look? Author: Dong Lin Reviewers: Ismael Juma , Jun Rao Closes #1009 from lindong28/KAFKA-3330 --- core/src/main/scala/kafka/log/LogCleaner.scala | 7 +++++++ .../src/main/scala/kafka/log/LogCleanerManager.scala | 12 ++++++++++++ core/src/main/scala/kafka/log/LogManager.scala | 8 ++++++-- 3 files changed, 25 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/log/LogCleaner.scala b/core/src/main/scala/kafka/log/LogCleaner.scala index a2e191317d996..e23234bb2fe45 100644 --- a/core/src/main/scala/kafka/log/LogCleaner.scala +++ b/core/src/main/scala/kafka/log/LogCleaner.scala @@ -133,6 +133,13 @@ class LogCleaner(val config: CleanerConfig, cleanerManager.updateCheckpoints(dataDir, update=None) } + /** + * Truncate cleaner offset checkpoint for the given partition if its checkpointed offset is larger than the given offset + */ + def maybeTruncateCheckpoint(dataDir: File, topicAndPartition: TopicAndPartition, offset: Long) { + cleanerManager.maybeTruncateCheckpoint(dataDir, topicAndPartition, offset) + } + /** * Abort the cleaning of a particular partition if it's in progress, and pause any future cleaning of this partition. * This call blocks until the cleaning of the partition is aborted and paused. diff --git a/core/src/main/scala/kafka/log/LogCleanerManager.scala b/core/src/main/scala/kafka/log/LogCleanerManager.scala index f6795d3ffdb85..f92db4ed844fb 100755 --- a/core/src/main/scala/kafka/log/LogCleanerManager.scala +++ b/core/src/main/scala/kafka/log/LogCleanerManager.scala @@ -210,6 +210,18 @@ private[log] class LogCleanerManager(val logDirs: Array[File], val logs: Pool[To } } + def maybeTruncateCheckpoint(dataDir: File, topicAndPartition: TopicAndPartition, offset: Long) { + inLock(lock) { + if (logs.get(topicAndPartition).config.compact) { + val checkpoint = checkpoints(dataDir) + val existing = checkpoint.read() + + if (existing.getOrElse(topicAndPartition, 0L) > offset) + checkpoint.write(existing + (topicAndPartition -> offset)) + } + } + } + /** * Save out the endOffset and remove the given log from the in-progress set, if not aborted. */ diff --git a/core/src/main/scala/kafka/log/LogManager.scala b/core/src/main/scala/kafka/log/LogManager.scala index b64fac6208bfa..749c6229a7d09 100755 --- a/core/src/main/scala/kafka/log/LogManager.scala +++ b/core/src/main/scala/kafka/log/LogManager.scala @@ -286,8 +286,10 @@ class LogManager(val logDirs: Array[File], if (needToStopCleaner && cleaner != null) cleaner.abortAndPauseCleaning(topicAndPartition) log.truncateTo(truncateOffset) - if (needToStopCleaner && cleaner != null) + if (needToStopCleaner && cleaner != null) { + cleaner.maybeTruncateCheckpoint(log.dir.getParentFile, topicAndPartition, log.activeSegment.baseOffset) cleaner.resumeCleaning(topicAndPartition) + } } } checkpointRecoveryPointOffsets() @@ -305,8 +307,10 @@ class LogManager(val logDirs: Array[File], if (cleaner != null) cleaner.abortAndPauseCleaning(topicAndPartition) log.truncateFullyAndStartAt(newOffset) - if (cleaner != null) + if (cleaner != null) { + cleaner.maybeTruncateCheckpoint(log.dir.getParentFile, topicAndPartition, log.activeSegment.baseOffset) cleaner.resumeCleaning(topicAndPartition) + } } checkpointRecoveryPointOffsets() } From f57dabbe56b3db40c06a1946d4cecd0a144b6ac4 Mon Sep 17 00:00:00 2001 From: Eno Thereska Date: Thu, 17 Mar 2016 15:37:37 -0700 Subject: [PATCH 049/206] KAFKA-3202: System test that changes message version on the fly becketqin apovzner please have a look. becketqin the test fails when the producer and consumer are 0.9.x and the message format changes on the fly. Author: Eno Thereska Reviewers: Ewen Cheslack-Postava, Ismael Juma, Gwen Shapira Closes #1070 from enothereska/kafka-3202-format-change-fly --- tests/kafkatest/services/kafka/kafka.py | 13 ++- tests/kafkatest/services/kafka/version.py | 4 + .../kafkatest/tests/message_format_change.py | 92 +++++++++++++++++++ 3 files changed, 107 insertions(+), 2 deletions(-) create mode 100644 tests/kafkatest/tests/message_format_change.py diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index 788d41bb4f12a..33ece3502f8d5 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -242,7 +242,7 @@ def create_topic(self, topic_cfg, node=None): cmd += "--zookeeper %(zk_connect)s --create --topic %(topic)s --partitions %(partitions)d --replication-factor %(replication)d" % { 'zk_connect': self.zk.connect_setting(), 'topic': topic_cfg.get("topic"), - 'partitions': topic_cfg.get('partitions', 1), + 'partitions': topic_cfg.get('partitions', 1), 'replication': topic_cfg.get('replication-factor', 1) } @@ -267,6 +267,15 @@ def describe_topic(self, topic, node=None): for line in node.account.ssh_capture(cmd): output += line return output + + def alter_message_format(self, topic, msg_format_version, node=None): + if node is None: + node = self.nodes[0] + self.logger.info("Altering message format version for topic %s with format %s", topic, msg_format_version) + cmd = "/opt/%s/bin/kafka-configs.sh --zookeeper %s --entity-name %s --entity-type topics --alter --add-config message.format.version=%s" % \ + (kafka_dir(node), self.zk.connect_setting(), topic, msg_format_version) + self.logger.info("Running alter message format command...\n%s" % cmd) + node.account.ssh(cmd) def parse_describe_topic(self, topic_description): """Parse output of kafka-topics.sh --describe (or describe_topic() method above), which is a string of form @@ -508,4 +517,4 @@ def get_offset_shell(self, topic, partitions, max_wait_ms, offsets, time): for line in node.account.ssh_capture(cmd): output += line self.logger.debug(output) - return output \ No newline at end of file + return output diff --git a/tests/kafkatest/services/kafka/version.py b/tests/kafkatest/services/kafka/version.py index 761d91ba2b7e9..dc2582b6a6273 100644 --- a/tests/kafkatest/services/kafka/version.py +++ b/tests/kafkatest/services/kafka/version.py @@ -63,3 +63,7 @@ def __str__(self): V_0_9_0_0 = KafkaVersion("0.9.0.0") V_0_9_0_1 = KafkaVersion("0.9.0.1") LATEST_0_9 = V_0_9_0_1 + +# 0.10.0.X versions +V_0_10_0_0 = KafkaVersion("0.10.0.0") +LATEST_0_10 = V_0_10_0_0 diff --git a/tests/kafkatest/tests/message_format_change.py b/tests/kafkatest/tests/message_format_change.py new file mode 100644 index 0000000000000..357fd17a9e30c --- /dev/null +++ b/tests/kafkatest/tests/message_format_change.py @@ -0,0 +1,92 @@ +# Copyright 2015 Confluent Inc. +# +# Licensed 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. + +from ducktape.tests.test import Test +from ducktape.mark import parametrize +from ducktape.utils.util import wait_until +from kafkatest.services.zookeeper import ZookeeperService +from kafkatest.services.kafka import KafkaService +from kafkatest.services.kafka.version import LATEST_0_9, LATEST_0_10, TRUNK, KafkaVersion +from kafkatest.services.verifiable_producer import VerifiableProducer +from kafkatest.services.console_consumer import ConsoleConsumer +from kafkatest.utils import is_int +from kafkatest.tests.produce_consume_validate import ProduceConsumeValidateTest +from kafkatest.services.kafka import config_property +import time + + +class MessageFormatChangeTest(ProduceConsumeValidateTest): + + def __init__(self, test_context): + super(MessageFormatChangeTest, self).__init__(test_context=test_context) + + def setUp(self): + self.topic = "test_topic" + self.zk = ZookeeperService(self.test_context, num_nodes=1) + + self.zk.start() + + # Producer and consumer + self.producer_throughput = 10000 + self.num_producers = 1 + self.num_consumers = 1 + self.messages_per_producer = 100 + + def produce_and_consume(self, producer_version, consumer_version, group): + self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, + self.topic, + throughput=self.producer_throughput, + message_validator=is_int, + version=KafkaVersion(producer_version)) + self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, + self.topic, consumer_timeout_ms=30000, + message_validator=is_int, version=KafkaVersion(consumer_version)) + self.consumer.group_id = group + self.run_produce_consume_validate(lambda: wait_until( + lambda: self.producer.each_produced_at_least(self.messages_per_producer) == True, + timeout_sec=120, backoff_sec=1, + err_msg="Producer did not produce all messages in reasonable amount of time")) + + @parametrize(producer_version=str(TRUNK), consumer_version=str(TRUNK)) + @parametrize(producer_version=str(LATEST_0_9), consumer_version=str(LATEST_0_9)) + def test_compatibility(self, producer_version, consumer_version): + """ This tests performs the following checks: + The workload is a mix of 0.9.x and 0.10.x producers and consumers + that produce to and consume from a 0.10.x cluster + 1. initially the topic is using message format 0.9.0 + 2. change the message format version for topic to 0.10.0 on the fly. + 3. change the message format version for topic back to 0.9.0 on the fly. + - The producers and consumers should not have any issue. + - Note that for 0.9.x consumers/producers we only do steps 1 and 2 + """ + self.kafka = KafkaService(self.test_context, num_nodes=3, zk=self.zk, version=TRUNK, topics={self.topic: { + "partitions": 3, + "replication-factor": 3, + 'configs': {"min.insync.replicas": 2}}}) + + self.kafka.start() + self.logger.info("First format change to 0.9.0") + self.kafka.alter_message_format(self.topic, str(LATEST_0_9)) + self.produce_and_consume(producer_version, consumer_version, "group1") + + self.logger.info("Second format change to 0.10.0") + self.kafka.alter_message_format(self.topic, str(LATEST_0_10)) + self.produce_and_consume(producer_version, consumer_version, "group2") + + if producer_version == str(TRUNK) and consumer_version == str(TRUNK): + self.logger.info("Third format change back to 0.9.0") + self.kafka.alter_message_format(self.topic, str(LATEST_0_9)) + self.produce_and_consume(producer_version, consumer_version, "group3") + + From dea0719e99211684775780f5da8b93835d7a5dac Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 17 Mar 2016 15:41:59 -0700 Subject: [PATCH 050/206] KAFKA-3336: Unify Serializer and Deserializer into Serialization Author: Guozhang Wang Reviewers: Michael G. Noll, Ismael Juma Closes #1066 from guozhangwang/K3336 --- .../serialization/DoubleDeserializer.java | 46 +++++ .../serialization/DoubleSerializer.java | 46 +++++ .../kafka/common/serialization/Serde.java | 26 +++ .../kafka/common/serialization/Serdes.java | 193 ++++++++++++++++++ .../serialization/SerializationTest.java | 115 ++++++++--- .../examples/pageview/PageViewTypedDemo.java | 39 ++-- .../pageview/PageViewUntypedDemo.java | 20 +- .../kafka/streams/examples/pipe/PipeDemo.java | 9 +- .../examples/wordcount/WordCountDemo.java | 20 +- .../wordcount/WordCountProcessorDemo.java | 9 +- .../apache/kafka/streams/StreamsConfig.java | 75 +++---- .../apache/kafka/streams/kstream/KStream.java | 133 +++++------- .../kafka/streams/kstream/KStreamBuilder.java | 38 ++-- .../apache/kafka/streams/kstream/KTable.java | 51 ++--- .../kstream/internals/KStreamImpl.java | 134 +++++------- .../streams/kstream/internals/KTableImpl.java | 94 ++++----- .../internals/KTableStoreSupplier.java | 13 +- .../streams/processor/ProcessorContext.java | 25 +-- .../streams/processor/TopologyBuilder.java | 24 +-- .../internals/ProcessorContextImpl.java | 35 +--- .../streams/processor/internals/SinkNode.java | 4 +- .../processor/internals/SourceNode.java | 4 +- .../internals/StandbyContextImpl.java | 33 +-- .../processor/internals/StreamThread.java | 2 - .../apache/kafka/streams/state/Serdes.java | 136 ------------ .../kafka/streams/state/StateSerdes.java | 108 ++++++++++ .../apache/kafka/streams/state/Stores.java | 109 ++++++---- .../kafka/streams/state/WindowStoreUtils.java | 6 +- .../InMemoryKeyValueLoggedStore.java | 6 +- .../InMemoryKeyValueStoreSupplier.java | 10 +- .../InMemoryLRUCacheStoreSupplier.java | 6 +- .../state/internals/MemoryLRUCache.java | 6 +- .../RocksDBKeyValueStoreSupplier.java | 6 +- .../streams/state/internals/RocksDBStore.java | 14 +- .../state/internals/RocksDBWindowStore.java | 12 +- .../internals/RocksDBWindowStoreSupplier.java | 6 +- .../state/internals/StoreChangeLogger.java | 10 +- .../kafka/streams/StreamsConfigTest.java | 10 - .../kstream/internals/KStreamBranchTest.java | 8 +- .../kstream/internals/KStreamFilterTest.java | 10 +- .../kstream/internals/KStreamFlatMapTest.java | 8 +- .../internals/KStreamFlatMapValuesTest.java | 8 +- .../kstream/internals/KStreamImplTest.java | 24 +-- .../internals/KStreamKStreamJoinTest.java | 33 ++- .../internals/KStreamKStreamLeftJoinTest.java | 26 +-- .../internals/KStreamKTableLeftJoinTest.java | 20 +- .../kstream/internals/KStreamMapTest.java | 11 +- .../internals/KStreamMapValuesTest.java | 10 +- .../internals/KStreamTransformTest.java | 9 +- .../internals/KStreamTransformValuesTest.java | 8 +- .../internals/KStreamWindowAggregateTest.java | 33 ++- .../internals/KTableAggregateTest.java | 22 +- .../kstream/internals/KTableFilterTest.java | 28 +-- .../kstream/internals/KTableImplTest.java | 37 ++-- .../internals/KTableKTableJoinTest.java | 24 +-- .../internals/KTableKTableLeftJoinTest.java | 30 +-- .../internals/KTableKTableOuterJoinTest.java | 24 +-- .../internals/KTableMapValuesTest.java | 31 +-- .../kstream/internals/KTableSourceTest.java | 26 +-- .../WindowedStreamPartitionerTest.java | 10 +- .../internals/ProcessorTopologyTest.java | 7 +- .../processor/internals/StandbyTaskTest.java | 7 +- .../StreamPartitionAssignorTest.java | 7 +- .../processor/internals/StreamTaskTest.java | 7 +- .../processor/internals/StreamThreadTest.java | 7 +- .../streams/smoketest/SmokeTestClient.java | 69 +++---- .../streams/smoketest/SmokeTestDriver.java | 22 +- .../streams/smoketest/SmokeTestUtil.java | 81 +------- .../state/KeyValueStoreTestDriver.java | 72 +------ .../internals/InMemoryKeyValueStoreTest.java | 8 +- .../internals/InMemoryLRUCacheStoreTest.java | 8 +- .../internals/RocksDBKeyValueStoreTest.java | 8 +- .../internals/RocksDBWindowStoreTest.java | 49 +++-- .../internals/StoreChangeLoggerTest.java | 6 +- .../apache/kafka/test/KStreamTestDriver.java | 13 +- .../kafka/test/MockProcessorContext.java | 55 ++--- 76 files changed, 1184 insertions(+), 1315 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/serialization/DoubleDeserializer.java create mode 100644 clients/src/main/java/org/apache/kafka/common/serialization/DoubleSerializer.java create mode 100644 clients/src/main/java/org/apache/kafka/common/serialization/Serde.java create mode 100644 clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java delete mode 100644 streams/src/main/java/org/apache/kafka/streams/state/Serdes.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/DoubleDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/DoubleDeserializer.java new file mode 100644 index 0000000000000..ed4f323791d82 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/DoubleDeserializer.java @@ -0,0 +1,46 @@ +/** + * 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 org.apache.kafka.common.serialization; + +import org.apache.kafka.common.errors.SerializationException; + +import java.util.Map; + +public class DoubleDeserializer implements Deserializer { + + @Override + public void configure(Map configs, boolean isKey) { + // nothing to do + } + + @Override + public Double deserialize(String topic, byte[] data) { + if (data == null) + return null; + if (data.length != 8) { + throw new SerializationException("Size of data received by Deserializer is not 8"); + } + + long value = 0; + for (byte b : data) { + value <<= 8; + value |= b & 0xFF; + } + return Double.longBitsToDouble(value); + } + + @Override + public void close() { + // nothing to do + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/DoubleSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/DoubleSerializer.java new file mode 100644 index 0000000000000..9d01342c59625 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/DoubleSerializer.java @@ -0,0 +1,46 @@ +/** + * 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 org.apache.kafka.common.serialization; + +import java.util.Map; + +public class DoubleSerializer implements Serializer { + + @Override + public void configure(Map configs, boolean isKey) { + // nothing to do + } + + @Override + public byte[] serialize(String topic, Double data) { + if (data == null) + return null; + + long bits = Double.doubleToLongBits(data); + return new byte[] { + (byte) (bits >>> 56), + (byte) (bits >>> 48), + (byte) (bits >>> 40), + (byte) (bits >>> 32), + (byte) (bits >>> 24), + (byte) (bits >>> 16), + (byte) (bits >>> 8), + (byte) bits + }; + } + + @Override + public void close() { + // nothing to do + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serde.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serde.java new file mode 100644 index 0000000000000..cc7944eb891a5 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serde.java @@ -0,0 +1,26 @@ +/** + * 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 org.apache.kafka.common.serialization; + +/** + * The interface for wrapping a serializer and deserializer for the given data type. + * + * @param + */ +public interface Serde { + + Serializer serializer(); + + Deserializer deserializer(); +} diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java new file mode 100644 index 0000000000000..f27f74fbc84de --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java @@ -0,0 +1,193 @@ +/** + * 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 org.apache.kafka.common.serialization; + +import java.nio.ByteBuffer; + +/** + * Factory for creating serializers / deserializers. + */ +public class Serdes { + + static public final class LongSerde implements Serde { + @Override + public Serializer serializer() { + return new LongSerializer(); + } + + @Override + public Deserializer deserializer() { + return new LongDeserializer(); + } + } + + static public final class IntegerSerde implements Serde { + @Override + public Serializer serializer() { + return new IntegerSerializer(); + } + + @Override + public Deserializer deserializer() { + return new IntegerDeserializer(); + } + } + + static public final class DoubleSerde implements Serde { + @Override + public Serializer serializer() { + return new DoubleSerializer(); + } + + @Override + public Deserializer deserializer() { + return new DoubleDeserializer(); + } + } + + static public final class StringSerde implements Serde { + @Override + public Serializer serializer() { + return new StringSerializer(); + } + + @Override + public Deserializer deserializer() { + return new StringDeserializer(); + } + } + + static public final class ByteBufferSerde implements Serde { + @Override + public Serializer serializer() { + return new ByteBufferSerializer(); + } + + @Override + public Deserializer deserializer() { + return new ByteBufferDeserializer(); + } + } + + static public final class ByteArraySerde implements Serde { + @Override + public Serializer serializer() { + return new ByteArraySerializer(); + } + + @Override + public Deserializer deserializer() { + return new ByteArrayDeserializer(); + } + } + + @SuppressWarnings("unchecked") + static public Serde serdeFrom(Class type) { + if (String.class.isAssignableFrom(type)) { + return (Serde) String(); + } + + if (Integer.class.isAssignableFrom(type)) { + return (Serde) Integer(); + } + + if (Long.class.isAssignableFrom(type)) { + return (Serde) Long(); + } + + if (Double.class.isAssignableFrom(type)) { + return (Serde) Double(); + } + + if (byte[].class.isAssignableFrom(type)) { + return (Serde) ByteArray(); + } + + if (ByteBufferSerde.class.isAssignableFrom(type)) { + return (Serde) ByteBuffer(); + } + + // TODO: we can also serializes objects of type T using generic Java serialization by default + throw new IllegalArgumentException("Unknown class for built-in serializer"); + } + + /** + * Construct a serde object from separate serializer and deserializer + * + * @param serializer must not be null. + * @param deserializer must not be null. + */ + static public Serde serdeFrom(final Serializer serializer, final Deserializer deserializer) { + if (serializer == null) { + throw new IllegalArgumentException("serializer must not be null"); + } + if (deserializer == null) { + throw new IllegalArgumentException("deserializer must not be null"); + } + + return new Serde() { + @Override + public Serializer serializer() { + return serializer; + } + + @Override + public Deserializer deserializer() { + return deserializer; + } + }; + } + + /* + * A serde for nullable long type. + */ + static public Serde Long() { + return new LongSerde(); + } + + /* + * A serde for nullable int type. + */ + static public Serde Integer() { + return new IntegerSerde(); + } + + /* + * A serde for nullable long type. + */ + static public Serde Double() { + return new DoubleSerde(); + } + + /* + * A serde for nullable string type. + */ + static public Serde String() { + return new StringSerde(); + } + + /* + * A serde for nullable byte array type. + */ + static public Serde ByteBuffer() { + return new ByteBufferSerde(); + } + + /* + * A serde for nullable byte array type. + */ + static public Serde ByteArray() { + return new ByteArraySerde(); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java index 87d9e0aa73e1b..e4cd67881734a 100644 --- a/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/serialization/SerializationTest.java @@ -24,34 +24,53 @@ public class SerializationTest { - private static class SerDeser { - final Serializer serializer; - final Deserializer deserializer; + final private String topic = "testTopic"; - public SerDeser(Serializer serializer, Deserializer deserializer) { - this.serializer = serializer; - this.deserializer = deserializer; - } + private class DummyClass { + + } + + @Test + public void testSerdeFrom() { + Serde thisSerde = Serdes.serdeFrom(Long.class); + Serde otherSerde = Serdes.Long(); + + Long value = 423412424L; + + assertEquals("Should get the original long after serialization and deserialization", + value, thisSerde.deserializer().deserialize(topic, otherSerde.serializer().serialize(topic, value))); + assertEquals("Should get the original long after serialization and deserialization", + value, otherSerde.deserializer().deserialize(topic, thisSerde.serializer().serialize(topic, value))); + } + + @Test(expected = IllegalArgumentException.class) + public void testSerdeFromUnknown() { + Serdes.serdeFrom(DummyClass.class); + } + + @Test(expected = IllegalArgumentException.class) + public void testSerdeFromNotNull() { + Serdes.serdeFrom(null, Serdes.Long().deserializer()); } @Test public void testStringSerializer() { String str = "my string"; - String mytopic = "testTopic"; + List encodings = new ArrayList(); encodings.add("UTF8"); encodings.add("UTF-16"); for (String encoding : encodings) { - SerDeser serDeser = getStringSerDeser(encoding); - Serializer serializer = serDeser.serializer; - Deserializer deserializer = serDeser.deserializer; + Serde serDeser = getStringSerde(encoding); + Serializer serializer = serDeser.serializer(); + Deserializer deserializer = serDeser.deserializer(); assertEquals("Should get the original string after serialization and deserialization with encoding " + encoding, - str, deserializer.deserialize(mytopic, serializer.serialize(mytopic, str))); + str, deserializer.deserialize(topic, serializer.serialize(topic, str))); assertEquals("Should support null in serialization and deserialization with encoding " + encoding, - null, deserializer.deserialize(mytopic, serializer.serialize(mytopic, null))); + null, deserializer.deserialize(topic, serializer.serialize(topic, null))); } } @@ -61,18 +80,61 @@ public void testIntegerSerializer() { 423412424, -41243432 }; - String mytopic = "testTopic"; - Serializer serializer = new IntegerSerializer(); - Deserializer deserializer = new IntegerDeserializer(); + Serializer serializer = Serdes.Integer().serializer(); + Deserializer deserializer = Serdes.Integer().deserializer(); for (Integer integer : integers) { assertEquals("Should get the original integer after serialization and deserialization", - integer, deserializer.deserialize(mytopic, serializer.serialize(mytopic, integer))); + integer, deserializer.deserialize(topic, serializer.serialize(topic, integer))); + } + + assertEquals("Should support null in serialization and deserialization", + null, deserializer.deserialize(topic, serializer.serialize(topic, null))); + + serializer.close(); + deserializer.close(); + } + + @Test + public void testLongSerializer() { + Long[] longs = new Long[]{ + 922337203685477580L, + -922337203685477581L + }; + + Serializer serializer = Serdes.Long().serializer(); + Deserializer deserializer = Serdes.Long().deserializer(); + + for (Long value : longs) { + assertEquals("Should get the original long after serialization and deserialization", + value, deserializer.deserialize(topic, serializer.serialize(topic, value))); + } + + assertEquals("Should support null in serialization and deserialization", + null, deserializer.deserialize(topic, serializer.serialize(topic, null))); + + serializer.close(); + deserializer.close(); + } + + @Test + public void testDoubleSerializer() { + Double[] doubles = new Double[]{ + 5678567.12312d, + -5678567.12341d + }; + + Serializer serializer = Serdes.Double().serializer(); + Deserializer deserializer = Serdes.Double().deserializer(); + + for (Double value : doubles) { + assertEquals("Should get the original double after serialization and deserialization", + value, deserializer.deserialize(topic, serializer.serialize(topic, value))); } assertEquals("Should support null in serialization and deserialization", - null, deserializer.deserialize(mytopic, serializer.serialize(mytopic, null))); + null, deserializer.deserialize(topic, serializer.serialize(topic, null))); serializer.close(); deserializer.close(); @@ -80,34 +142,33 @@ public void testIntegerSerializer() { @Test public void testByteBufferSerializer() { - String mytopic = "testTopic"; ByteBuffer buf = ByteBuffer.allocate(10); buf.put("my string".getBytes()); - Serializer serializer = new ByteBufferSerializer(); - Deserializer deserializer = new ByteBufferDeserializer(); + Serializer serializer = Serdes.ByteBuffer().serializer(); + Deserializer deserializer = Serdes.ByteBuffer().deserializer(); assertEquals("Should get the original ByteBuffer after serialization and deserialization", - buf, deserializer.deserialize(mytopic, serializer.serialize(mytopic, buf))); + buf, deserializer.deserialize(topic, serializer.serialize(topic, buf))); assertEquals("Should support null in serialization and deserialization", - null, deserializer.deserialize(mytopic, serializer.serialize(mytopic, null))); + null, deserializer.deserialize(topic, serializer.serialize(topic, null))); serializer.close(); deserializer.close(); } - private SerDeser getStringSerDeser(String encoder) { + private Serde getStringSerde(String encoder) { Map serializerConfigs = new HashMap(); serializerConfigs.put("key.serializer.encoding", encoder); - Serializer serializer = new StringSerializer(); + Serializer serializer = Serdes.String().serializer(); serializer.configure(serializerConfigs, true); Map deserializerConfigs = new HashMap(); deserializerConfigs.put("key.deserializer.encoding", encoder); - Deserializer deserializer = new StringDeserializer(); + Deserializer deserializer = Serdes.String().deserializer(); deserializer.configure(deserializerConfigs, true); - return new SerDeser(serializer, deserializer); + return Serdes.serdeFrom(serializer, deserializer); } } diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java index 4f9de291c3406..15083b23b989e 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java @@ -18,9 +18,8 @@ package org.apache.kafka.streams.examples.pageview; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.HoppingWindows; @@ -83,10 +82,6 @@ public static void main(String[] args) throws Exception { props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pageview-typed"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); - props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - props.put(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, JsonPOJOSerializer.class); - props.put(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, JsonPOJODeserializer.class); props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, JsonTimestampExtractor.class); // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data @@ -94,35 +89,44 @@ public static void main(String[] args) throws Exception { KStreamBuilder builder = new KStreamBuilder(); - final Serializer stringSerializer = new StringSerializer(); - final Deserializer stringDeserializer = new StringDeserializer(); - // TODO: the following can be removed with a serialization factory Map serdeProps = new HashMap<>(); + final Serializer pageViewSerializer = new JsonPOJOSerializer<>(); + serdeProps.put("JsonPOJOClass", PageView.class); + pageViewSerializer.configure(serdeProps, false); + final Deserializer pageViewDeserializer = new JsonPOJODeserializer<>(); serdeProps.put("JsonPOJOClass", PageView.class); pageViewDeserializer.configure(serdeProps, false); - final Deserializer userProfileDeserializer = new JsonPOJODeserializer<>(); - serdeProps.put("JsonPOJOClass", UserProfile.class); - userProfileDeserializer.configure(serdeProps, false); - final Serializer userProfileSerializer = new JsonPOJOSerializer<>(); serdeProps.put("JsonPOJOClass", UserProfile.class); userProfileSerializer.configure(serdeProps, false); + final Deserializer userProfileDeserializer = new JsonPOJODeserializer<>(); + serdeProps.put("JsonPOJOClass", UserProfile.class); + userProfileDeserializer.configure(serdeProps, false); + final Serializer wPageViewByRegionSerializer = new JsonPOJOSerializer<>(); serdeProps.put("JsonPOJOClass", WindowedPageViewByRegion.class); wPageViewByRegionSerializer.configure(serdeProps, false); + final Deserializer wPageViewByRegionDeserializer = new JsonPOJODeserializer<>(); + serdeProps.put("JsonPOJOClass", WindowedPageViewByRegion.class); + wPageViewByRegionDeserializer.configure(serdeProps, false); + final Serializer regionCountSerializer = new JsonPOJOSerializer<>(); serdeProps.put("JsonPOJOClass", RegionCount.class); regionCountSerializer.configure(serdeProps, false); - KStream views = builder.stream(stringDeserializer, pageViewDeserializer, "streams-pageview-input"); + final Deserializer regionCountDeserializer = new JsonPOJODeserializer<>(); + serdeProps.put("JsonPOJOClass", RegionCount.class); + regionCountDeserializer.configure(serdeProps, false); + + KStream views = builder.stream(Serdes.String(), Serdes.serdeFrom(pageViewSerializer, pageViewDeserializer), "streams-pageview-input"); - KTable users = builder.table(stringSerializer, userProfileSerializer, stringDeserializer, userProfileDeserializer, "streams-userprofile-input"); + KTable users = builder.table(Serdes.String(), Serdes.serdeFrom(userProfileSerializer, userProfileDeserializer), "streams-userprofile-input"); KStream regionCount = views .leftJoin(users, new ValueJoiner() { @@ -146,8 +150,7 @@ public KeyValue apply(String user, PageViewByRegion vi return new KeyValue<>(viewRegion.region, viewRegion); } }) - .countByKey(HoppingWindows.of("GeoPageViewsWindow").with(7 * 24 * 60 * 60 * 1000), - stringSerializer, stringDeserializer) + .countByKey(HoppingWindows.of("GeoPageViewsWindow").with(7 * 24 * 60 * 60 * 1000), Serdes.String()) // TODO: we can merge ths toStream().map(...) with a single toStream(...) .toStream() .map(new KeyValueMapper, Long, KeyValue>() { @@ -166,7 +169,7 @@ public KeyValue apply(Windowed ke }); // write to the result topic - regionCount.to("streams-pageviewstats-typed-output", wPageViewByRegionSerializer, regionCountSerializer); + regionCount.to("streams-pageviewstats-typed-output", Serdes.serdeFrom(wPageViewByRegionSerializer, wPageViewByRegionDeserializer), Serdes.serdeFrom(regionCountSerializer, regionCountDeserializer)); KafkaStreams streams = new KafkaStreams(builder, props); streams.start(); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java index 9377095b2128d..5b80f64752d3f 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java @@ -21,9 +21,9 @@ import com.fasterxml.jackson.databind.node.JsonNodeFactory; import com.fasterxml.jackson.databind.node.ObjectNode; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.connect.json.JsonSerializer; import org.apache.kafka.connect.json.JsonDeserializer; import org.apache.kafka.streams.KafkaStreams; @@ -59,10 +59,6 @@ public static void main(String[] args) throws Exception { props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pageview-untyped"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); - props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - props.put(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, JsonSerializer.class); - props.put(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, JsonDeserializer.class); props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, JsonTimestampExtractor.class); // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data @@ -70,14 +66,13 @@ public static void main(String[] args) throws Exception { KStreamBuilder builder = new KStreamBuilder(); - final Serializer stringSerializer = new StringSerializer(); - final Deserializer stringDeserializer = new StringDeserializer(); final Serializer jsonSerializer = new JsonSerializer(); final Deserializer jsonDeserializer = new JsonDeserializer(); + final Serde jsonSerde = Serdes.serdeFrom(jsonSerializer, jsonDeserializer); - KStream views = builder.stream(stringDeserializer, jsonDeserializer, "streams-pageview-input"); + KStream views = builder.stream(Serdes.String(), jsonSerde, "streams-pageview-input"); - KTable users = builder.table(stringSerializer, jsonSerializer, stringDeserializer, jsonDeserializer, "streams-userprofile-input"); + KTable users = builder.table(Serdes.String(), jsonSerde, "streams-userprofile-input"); KTable userRegions = users.mapValues(new ValueMapper() { @Override @@ -103,8 +98,7 @@ public KeyValue apply(String user, JsonNode viewRegion) { return new KeyValue<>(viewRegion.get("region").textValue(), viewRegion); } }) - .countByKey(HoppingWindows.of("GeoPageViewsWindow").with(7 * 24 * 60 * 60 * 1000), - stringSerializer, stringDeserializer) + .countByKey(HoppingWindows.of("GeoPageViewsWindow").with(7 * 24 * 60 * 60 * 1000), Serdes.String()) // TODO: we can merge ths toStream().map(...) with a single toStream(...) .toStream() .map(new KeyValueMapper, Long, KeyValue>() { @@ -122,7 +116,7 @@ public KeyValue apply(Windowed key, Long value) { }); // write to the result topic - regionCount.to("streams-pageviewstats-untyped-output", jsonSerializer, jsonSerializer); + regionCount.to("streams-pageviewstats-untyped-output", jsonSerde, jsonSerde); KafkaStreams streams = new KafkaStreams(builder, props); streams.start(); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java index c37c68ac3f29a..619f33ddd931e 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.examples.pipe; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.StreamsConfig; @@ -41,10 +40,8 @@ public static void main(String[] args) throws Exception { Properties props = new Properties(); props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-pipe"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); - props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - props.put(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - props.put(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data props.put(StreamsConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java index 03d514292ac12..ebd6050c70a4e 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java @@ -17,11 +17,7 @@ package org.apache.kafka.streams.examples.wordcount; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.LongSerializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; @@ -52,21 +48,13 @@ public static void main(String[] args) throws Exception { props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); - props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - props.put(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - props.put(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data props.put(StreamsConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); KStreamBuilder builder = new KStreamBuilder(); - final Serializer stringSerializer = new StringSerializer(); - final Deserializer stringDeserializer = new StringDeserializer(); - final Serializer longSerializer = new LongSerializer(); - - KStream source = builder.stream("streams-file-input"); + KStream source = builder.stream(Serdes.String(), Serdes.String(), "streams-file-input"); KTable counts = source .flatMapValues(new ValueMapper>() { @@ -80,9 +68,9 @@ public KeyValue apply(String key, String value) { return new KeyValue(value, value); } }) - .countByKey(stringSerializer, stringDeserializer, "Counts"); + .countByKey(Serdes.String(), "Counts"); - counts.to("streams-wordcount-output", stringSerializer, longSerializer); + counts.to("streams-wordcount-output", Serdes.String(), Serdes.Long()); KafkaStreams streams = new KafkaStreams(builder, props); streams.start(); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java index b651b3aa40210..8457415f11181 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.examples.wordcount; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.KafkaStreams; @@ -108,10 +107,8 @@ public static void main(String[] args) throws Exception { props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount-processor"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); - props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - props.put(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - props.put(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); + props.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass()); + props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data props.put(StreamsConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 52fdbd4583920..4e989bee5136e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -24,11 +24,12 @@ import org.apache.kafka.common.config.ConfigDef; import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.processor.DefaultPartitionGrouper; import org.apache.kafka.streams.processor.internals.StreamPartitionAssignor; import org.apache.kafka.streams.processor.internals.StreamThread; +import org.apache.kafka.streams.processor.internals.WallclockTimestampExtractor; import java.util.Map; @@ -91,17 +92,13 @@ public class StreamsConfig extends AbstractConfig { public static final String REPLICATION_FACTOR_CONFIG = "replication.factor"; public static final String REPLICATION_FACTOR_DOC = "The replication factor for change log topics and repartition topics created by the stream processing application."; - /** key.serializer */ - public static final String KEY_SERIALIZER_CLASS_CONFIG = ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG; - - /** value.serializer */ - public static final String VALUE_SERIALIZER_CLASS_CONFIG = ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG; - - /** key.deserializer */ - public static final String KEY_DESERIALIZER_CLASS_CONFIG = ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; + /** replication.factor */ + public static final String KEY_SERDE_CLASS_CONFIG = "key.serde"; + public static final String KEY_SERDE_CLASS_DOC = "Serializer / deserializer class for key that implements the Serde interface."; - /** value.deserializer */ - public static final String VALUE_DESERIALIZER_CLASS_CONFIG = ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; + /** replication.factor */ + public static final String VALUE_SERDE_CLASS_CONFIG = "value.serde"; + public static final String VALUE_SERDE_CLASS_DOC = "Serializer / deserializer class for value that implements the Serde interface."; /** metrics.sample.window.ms */ public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = CommonClientConfigs.METRICS_SAMPLE_WINDOW_MS_CONFIG; @@ -121,8 +118,6 @@ public class StreamsConfig extends AbstractConfig { /** auto.offset.reset */ public static final String AUTO_OFFSET_RESET_CONFIG = ConsumerConfig.AUTO_OFFSET_RESET_CONFIG; - private static final String WALLCLOCK_TIMESTAMP_EXTRACTOR = "org.apache.kafka.streams.processor.internals.WallclockTimestampExtractor"; - static { CONFIG = new ConfigDef().define(APPLICATION_ID_CONFIG, // required with no default value Type.STRING, @@ -152,32 +147,26 @@ public class StreamsConfig extends AbstractConfig { 1, Importance.MEDIUM, REPLICATION_FACTOR_DOC) - .define(KEY_SERIALIZER_CLASS_CONFIG, // required with no default value - Type.CLASS, - Importance.HIGH, - ProducerConfig.KEY_SERIALIZER_CLASS_DOC) - .define(VALUE_SERIALIZER_CLASS_CONFIG, // required with no default value - Type.CLASS, - Importance.HIGH, - ProducerConfig.VALUE_SERIALIZER_CLASS_DOC) - .define(KEY_DESERIALIZER_CLASS_CONFIG, // required with no default value - Type.CLASS, - Importance.HIGH, - ConsumerConfig.KEY_DESERIALIZER_CLASS_DOC) - .define(VALUE_DESERIALIZER_CLASS_CONFIG, // required with no default value - Type.CLASS, - Importance.HIGH, - ConsumerConfig.VALUE_DESERIALIZER_CLASS_DOC) .define(TIMESTAMP_EXTRACTOR_CLASS_CONFIG, Type.CLASS, - WALLCLOCK_TIMESTAMP_EXTRACTOR, + WallclockTimestampExtractor.class.getName(), Importance.MEDIUM, TIMESTAMP_EXTRACTOR_CLASS_DOC) .define(PARTITION_GROUPER_CLASS_CONFIG, Type.CLASS, - DefaultPartitionGrouper.class, + DefaultPartitionGrouper.class.getName(), Importance.MEDIUM, PARTITION_GROUPER_CLASS_DOC) + .define(KEY_SERDE_CLASS_CONFIG, + Type.CLASS, + Serdes.ByteArraySerde.class.getName(), + Importance.MEDIUM, + KEY_SERDE_CLASS_DOC) + .define(VALUE_SERDE_CLASS_CONFIG, + Type.CLASS, + Serdes.ByteArraySerde.class.getName(), + Importance.MEDIUM, + VALUE_SERDE_CLASS_DOC) .define(COMMIT_INTERVAL_MS_CONFIG, Type.LONG, 30000, @@ -273,8 +262,6 @@ private Map getBaseConsumerConfigs() { // remove properties that are not required for consumers removeStreamsSpecificConfigs(props); - props.remove(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG); - props.remove(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG); return props; } @@ -287,8 +274,6 @@ public Map getProducerConfigs(String clientId) { // remove properties that are not required for producers removeStreamsSpecificConfigs(props); - props.remove(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG); - props.remove(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG); props.remove(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG); props.put(CommonClientConfigs.CLIENT_ID_CONFIG, clientId + "-producer"); @@ -302,23 +287,17 @@ private void removeStreamsSpecificConfigs(Map props) { props.remove(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); props.remove(StreamsConfig.NUM_STREAM_THREADS_CONFIG); props.remove(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG); + props.remove(StreamsConfig.KEY_SERDE_CLASS_CONFIG); + props.remove(StreamsConfig.VALUE_SERDE_CLASS_CONFIG); props.remove(InternalConfig.STREAM_THREAD_INSTANCE); } - public Serializer keySerializer() { - return getConfiguredInstance(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, Serializer.class); - } - - public Serializer valueSerializer() { - return getConfiguredInstance(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, Serializer.class); - } - - public Deserializer keyDeserializer() { - return getConfiguredInstance(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, Deserializer.class); + public Serde keySerde() { + return getConfiguredInstance(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serde.class); } - public Deserializer valueDeserializer() { - return getConfiguredInstance(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, Deserializer.class); + public Serde valueSerde() { + return getConfiguredInstance(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serde.class); } public static void main(String[] args) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index 1640bdec000c0..1c78652178343 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.kstream; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.ProcessorSupplier; @@ -108,18 +107,14 @@ public interface KStream { * Sends key-value to a topic, also creates a new instance of KStream from the topic. * This is equivalent to calling to(topic) and from(topic). * - * @param topic the topic name - * @param keySerializer key serializer used to send key-value pairs, - * if not specified the default key serializer defined in the configuration will be used - * @param valSerializer value serializer used to send key-value pairs, - * if not specified the default value serializer defined in the configuration will be used - * @param keyDeserializer key deserializer used to create the new KStream, - * if not specified the default key deserializer defined in the configuration will be used - * @param valDeserializer value deserializer used to create the new KStream, - * if not specified the default value deserializer defined in the configuration will be used + * @param topic the topic name + * @param keySerde key serde used to send key-value pairs, + * if not specified the default key serde defined in the configuration will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default value serde defined in the configuration will be used * @return the instance of KStream that consumes the given topic */ - KStream through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer); + KStream through(String topic, Serde keySerde, Serde valSerde); /** * Sends key-value to a topic using default serializers specified in the config. @@ -131,13 +126,13 @@ public interface KStream { /** * Sends key-value to a topic. * - * @param topic the topic name - * @param keySerializer key serializer used to send key-value pairs, - * if not specified the default serializer defined in the configs will be used - * @param valSerializer value serializer used to send key-value pairs, - * if not specified the default serializer defined in the configs will be used + * @param topic the topic name + * @param keySerde key serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * @param keySerde value serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used */ - void to(String topic, Serializer keySerializer, Serializer valSerializer); + void to(String topic, Serde keySerde, Serde valSerde); /** * Applies a stateful transformation to all elements in this stream. @@ -171,18 +166,12 @@ public interface KStream { * @param otherStream the instance of KStream joined with this stream * @param joiner ValueJoiner * @param windows the specification of the join window - * @param keySerializer key serializer, - * if not specified the default serializer defined in the configs will be used - * @param thisValueSerializer value serializer for this stream, - * if not specified the default serializer defined in the configs will be used - * @param otherValueSerializer value serializer for other stream, - * if not specified the default serializer defined in the configs will be used - * @param keyDeserializer key deserializer, - * if not specified the default serializer defined in the configs will be used - * @param thisValueDeserializer value deserializer for this stream, - * if not specified the default serializer defined in the configs will be used - * @param otherValueDeserializer value deserializer for other stream, - * if not specified the default serializer defined in the configs will be used + * @param keySerde key serdes, + * if not specified the default serdes defined in the configs will be used + * @param thisValueSerde value serdes for this stream, + * if not specified the default serdes defined in the configs will be used + * @param otherValueSerde value serdes for other stream, + * if not specified the default serdes defined in the configs will be used * @param the value type of the other stream * @param the value type of the new stream */ @@ -190,12 +179,9 @@ KStream join( KStream otherStream, ValueJoiner joiner, JoinWindows windows, - Serializer keySerializer, - Serializer thisValueSerializer, - Serializer otherValueSerializer, - Deserializer keyDeserializer, - Deserializer thisValueDeserializer, - Deserializer otherValueDeserializer); + Serde keySerde, + Serde thisValueSerde, + Serde otherValueSerde); /** * Combines values of this stream with another KStream using Windowed Outer Join. @@ -203,18 +189,12 @@ KStream join( * @param otherStream the instance of KStream joined with this stream * @param joiner ValueJoiner * @param windows the specification of the join window - * @param keySerializer key serializer, - * if not specified the default serializer defined in the configs will be used - * @param thisValueSerializer value serializer for this stream, - * if not specified the default serializer defined in the configs will be used - * @param otherValueSerializer value serializer for other stream, - * if not specified the default serializer defined in the configs will be used - * @param keyDeserializer key deserializer, - * if not specified the default serializer defined in the configs will be used - * @param thisValueDeserializer value deserializer for this stream, - * if not specified the default serializer defined in the configs will be used - * @param otherValueDeserializer value deserializer for other stream, - * if not specified the default serializer defined in the configs will be used + * @param keySerde key serdes, + * if not specified the default serdes defined in the configs will be used + * @param thisValueSerde value serdes for this stream, + * if not specified the default serdes defined in the configs will be used + * @param otherValueSerde value serdes for other stream, + * if not specified the default serdes defined in the configs will be used * @param the value type of the other stream * @param the value type of the new stream */ @@ -222,12 +202,9 @@ KStream outerJoin( KStream otherStream, ValueJoiner joiner, JoinWindows windows, - Serializer keySerializer, - Serializer thisValueSerializer, - Serializer otherValueSerializer, - Deserializer keyDeserializer, - Deserializer thisValueDeserializer, - Deserializer otherValueDeserializer); + Serde keySerde, + Serde thisValueSerde, + Serde otherValueSerde); /** * Combines values of this stream with another KStream using Windowed Left Join. @@ -235,14 +212,10 @@ KStream outerJoin( * @param otherStream the instance of KStream joined with this stream * @param joiner ValueJoiner * @param windows the specification of the join window - * @param keySerializer key serializer, - * if not specified the default serializer defined in the configs will be used - * @param otherValueSerializer value serializer for other stream, - * if not specified the default serializer defined in the configs will be used - * @param keyDeserializer key deserializer, - * if not specified the default serializer defined in the configs will be used - * @param otherValueDeserializer value deserializer for other stream, - * if not specified the default serializer defined in the configs will be used + * @param keySerde key serdes, + * if not specified the default serdes defined in the configs will be used + * @param otherValueSerde value serdes for other stream, + * if not specified the default serdes defined in the configs will be used * @param the value type of the other stream * @param the value type of the new stream */ @@ -250,10 +223,8 @@ KStream leftJoin( KStream otherStream, ValueJoiner joiner, JoinWindows windows, - Serializer keySerializer, - Serializer otherValueSerializer, - Deserializer keyDeserializer, - Deserializer otherValueDeserializer); + Serde keySerde, + Serde otherValueSerde); /** * Combines values of this stream with KTable using Left Join. @@ -273,10 +244,8 @@ KStream leftJoin( */ KTable, V> reduceByKey(Reducer reducer, Windows windows, - Serializer keySerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer aggValueDeserializer); + Serde keySerde, + Serde aggValueSerde); /** * Aggregate values of this stream by key on a window basis. @@ -284,10 +253,8 @@ KTable, V> reduceByKey(Reducer reducer, * @param reducer the class of Reducer */ KTable reduceByKey(Reducer reducer, - Serializer keySerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer aggValueDeserializer, + Serde keySerde, + Serde aggValueSerde, String name); /** @@ -301,10 +268,8 @@ KTable reduceByKey(Reducer reducer, KTable, T> aggregateByKey(Initializer initializer, Aggregator aggregator, Windows windows, - Serializer keySerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer aggValueDeserializer); + Serde keySerde, + Serde aggValueSerde); /** * Aggregate values of this stream by key without a window basis, and hence @@ -316,10 +281,8 @@ KTable, T> aggregateByKey(Initializer initi */ KTable aggregateByKey(Initializer initializer, Aggregator aggregator, - Serializer keySerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer aggValueDeserializer, + Serde keySerde, + Serde aggValueSerde, String name); /** @@ -328,14 +291,12 @@ KTable aggregateByKey(Initializer initializer, * @param windows the specification of the aggregation window */ KTable, Long> countByKey(Windows windows, - Serializer keySerializer, - Deserializer keyDeserializer); + Serde keySerde); /** * Count number of messages of this stream by key without a window basis, and hence * return a ever updating counting table. */ - KTable countByKey(Serializer keySerializer, - Deserializer keyDeserializer, + KTable countByKey(Serde keySerde, String name); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java index 3cf198c37d393..dfd9281e711ee 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.kstream; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.kstream.internals.KStreamImpl; import org.apache.kafka.streams.kstream.internals.KTableImpl; import org.apache.kafka.streams.kstream.internals.KTableSource; @@ -40,7 +39,6 @@ public KStreamBuilder() { super(); } - // TODO: needs updated /** * Creates a KStream instance for the specified topic. * The default deserializers specified in the config are used. @@ -55,17 +53,17 @@ public KStream stream(String... topics) { /** * Creates a KStream instance for the specified topic. * - * @param keyDeserializer key deserializer used to read this source KStream, - * if not specified the default deserializer defined in the configs will be used - * @param valDeserializer value deserializer used to read this source KStream, - * if not specified the default deserializer defined in the configs will be used - * @param topics the topic names, if empty default to all the topics in the config + * @param keySerde key serde used to read this source KStream, + * if not specified the default serde defined in the configs will be used + * @param valSerde value serde used to read this source KStream, + * if not specified the default serde defined in the configs will be used + * @param topics the topic names, if empty default to all the topics in the config * @return KStream */ - public KStream stream(Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { + public KStream stream(Serde keySerde, Serde valSerde, String... topics) { String name = newName(KStreamImpl.SOURCE_NAME); - addSource(name, keyDeserializer, valDeserializer, topics); + addSource(name, keySerde == null ? null : keySerde.deserializer(), valSerde == null ? null : valSerde.deserializer(), topics); return new KStreamImpl<>(this, name, Collections.singleton(name)); } @@ -78,33 +76,29 @@ public KStream stream(Deserializer keyDeserializer, Deserializer * @return KTable */ public KTable table(String topic) { - return table(null, null, null, null, topic); + return table(null, null, topic); } /** * Creates a KTable instance for the specified topic. * - * @param keySerializer key serializer used to send key-value pairs, - * if not specified the default key serializer defined in the configuration will be used - * @param valSerializer value serializer used to send key-value pairs, - * if not specified the default value serializer defined in the configuration will be used - * @param keyDeserializer key deserializer used to read this source KStream, - * if not specified the default deserializer defined in the configs will be used - * @param valDeserializer value deserializer used to read this source KStream, - * if not specified the default deserializer defined in the configs will be used + * @param keySerde key serde used to send key-value pairs, + * if not specified the default key serde defined in the configuration will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default value serde defined in the configuration will be used * @param topic the topic name * @return KStream */ - public KTable table(Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer, String topic) { + public KTable table(Serde keySerde, Serde valSerde, String topic) { String source = newName(KStreamImpl.SOURCE_NAME); String name = newName(KTableImpl.SOURCE_NAME); - addSource(source, keyDeserializer, valDeserializer, topic); + addSource(source, keySerde == null ? null : keySerde.deserializer(), valSerde == null ? null : valSerde.deserializer(), topic); ProcessorSupplier processorSupplier = new KTableSource<>(topic); addProcessor(name, processorSupplier, source); - return new KTableImpl<>(this, name, processorSupplier, Collections.singleton(source), keySerializer, valSerializer, keyDeserializer, valDeserializer); + return new KTableImpl<>(this, name, processorSupplier, Collections.singleton(source), keySerde, valSerde); } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index b44ed21cf8a84..0ae515055b70d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.kstream; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.KeyValue; /** @@ -68,17 +67,13 @@ public interface KTable { * This is equivalent to calling to(topic) and table(topic). * * @param topic the topic name - * @param keySerializer key serializer used to send key-value pairs, - * if not specified the default key serializer defined in the configuration will be used - * @param valSerializer value serializer used to send key-value pairs, - * if not specified the default value serializer defined in the configuration will be used - * @param keyDeserializer key deserializer used to create the new KStream, - * if not specified the default key deserializer defined in the configuration will be used - * @param valDeserializer value deserializer used to create the new KStream, - * if not specified the default value deserializer defined in the configuration will be used + * @param keySerde key serde used to send key-value pairs, + * if not specified the default key serde defined in the configuration will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default value serde defined in the configuration will be used * @return the new stream that consumes the given topic */ - KTable through(String topic, Serializer keySerializer, Serializer valSerializer, Deserializer keyDeserializer, Deserializer valDeserializer); + KTable through(String topic, Serde keySerde, Serde valSerde); /** * Sends key-value to a topic using default serializers specified in the config. @@ -90,13 +85,13 @@ public interface KTable { /** * Sends key-value to a topic. * - * @param topic the topic name - * @param keySerializer key serializer used to send key-value pairs, - * if not specified the default serializer defined in the configs will be used - * @param valSerializer value serializer used to send key-value pairs, - * if not specified the default serializer defined in the configs will be used + * @param topic the topic name + * @param keySerde key serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used */ - void to(String topic, Serializer keySerializer, Serializer valSerializer); + void to(String topic, Serde keySerde, Serde valSerde); /** * Creates a new instance of KStream from this KTable @@ -152,10 +147,8 @@ public interface KTable { KTable reduce(Reducer addReducer, Reducer removeReducer, KeyValueMapper> selector, - Serializer keySerializer, - Serializer valueSerializer, - Deserializer keyDeserializer, - Deserializer valueDeserializer, + Serde keySerde, + Serde valueSerde, String name); /** @@ -174,12 +167,9 @@ KTable aggregate(Initializer initializer, Aggregator add, Aggregator remove, KeyValueMapper> selector, - Serializer keySerializer, - Serializer valueSerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer valueDeserializer, - Deserializer aggValueDeserializer, + Serde keySerde, + Serde valueSerde, + Serde aggValueSerde, String name); /** @@ -191,10 +181,7 @@ KTable aggregate(Initializer initializer, * @return the instance of KTable */ KTable count(KeyValueMapper selector, - Serializer keySerializer, - Serializer valueSerializer, - Deserializer keyDeserializer, - Deserializer valueDeserializer, + Serde keySerde, + Serde valueSerde, String name); - } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 884933b259b6b..b29349684833a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -17,9 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.LongDeserializer; -import org.apache.kafka.common.serialization.LongSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Initializer; @@ -93,9 +92,6 @@ public class KStreamImpl extends AbstractStream implements KStream sourceNodes) { super(topology, name, sourceNodes); } @@ -199,18 +195,16 @@ public static KStream merge(KStreamBuilder topology, KStream[ @Override public KStream through(String topic, - Serializer keySerializer, - Serializer valSerializer, - Deserializer keyDeserializer, - Deserializer valDeserializer) { - to(topic, keySerializer, valSerializer); + Serde keySerde, + Serde valSerde) { + to(topic, keySerde, valSerde); - return topology.stream(keyDeserializer, valDeserializer, topic); + return topology.stream(keySerde, valSerde); } @Override public KStream through(String topic) { - return through(topic, null, null, null, null); + return through(topic, null, null); } @Override @@ -220,10 +214,13 @@ public void to(String topic) { @SuppressWarnings("unchecked") @Override - public void to(String topic, Serializer keySerializer, Serializer valSerializer) { + public void to(String topic, Serde keySerde, Serde valSerde) { String name = topology.newName(SINK_NAME); StreamPartitioner streamPartitioner = null; + Serializer keySerializer = keySerde == null ? null : keySerde.serializer(); + Serializer valSerializer = keySerde == null ? null : valSerde.serializer(); + if (keySerializer != null && keySerializer instanceof WindowedSerializer) { WindowedSerializer windowedSerializer = (WindowedSerializer) keySerializer; streamPartitioner = (StreamPartitioner) new WindowedStreamPartitioner(windowedSerializer); @@ -265,16 +262,11 @@ public KStream join( KStream other, ValueJoiner joiner, JoinWindows windows, - Serializer keySerializer, - Serializer thisValueSerializer, - Serializer otherValueSerializer, - Deserializer keyDeserializer, - Deserializer thisValueDeserializer, - Deserializer otherValueDeserializer) { - - return join(other, joiner, windows, - keySerializer, thisValueSerializer, otherValueSerializer, - keyDeserializer, thisValueDeserializer, otherValueDeserializer, false); + Serde keySerde, + Serde thisValueSerde, + Serde otherValueSerde) { + + return join(other, joiner, windows, keySerde, thisValueSerde, otherValueSerde, false); } @Override @@ -282,16 +274,11 @@ public KStream outerJoin( KStream other, ValueJoiner joiner, JoinWindows windows, - Serializer keySerializer, - Serializer thisValueSerializer, - Serializer otherValueSerializer, - Deserializer keyDeserializer, - Deserializer thisValueDeserializer, - Deserializer otherValueDeserializer) { - - return join(other, joiner, windows, - keySerializer, thisValueSerializer, otherValueSerializer, - keyDeserializer, thisValueDeserializer, otherValueDeserializer, true); + Serde keySerde, + Serde thisValueSerde, + Serde otherValueSerde) { + + return join(other, joiner, windows, keySerde, thisValueSerde, otherValueSerde, true); } @SuppressWarnings("unchecked") @@ -299,26 +286,23 @@ private KStream join( KStream other, ValueJoiner joiner, JoinWindows windows, - Serializer keySerializer, - Serializer thisValueSerializer, - Serializer otherValueSerializer, - Deserializer keyDeserializer, - Deserializer thisValueDeserializer, - Deserializer otherValueDeserializer, + Serde keySerde, + Serde thisValueSerde, + Serde otherValueSerde, boolean outer) { Set allSourceNodes = ensureJoinableWith((AbstractStream) other); StateStoreSupplier thisWindow = Stores.create(windows.name() + "-this") - .withKeys(keySerializer, keyDeserializer) - .withValues(thisValueSerializer, thisValueDeserializer) + .withKeys(keySerde) + .withValues(thisValueSerde) .persistent() .windowed(windows.maintainMs(), windows.segments, true) .build(); StateStoreSupplier otherWindow = Stores.create(windows.name() + "-other") - .withKeys(keySerializer, keyDeserializer) - .withValues(otherValueSerializer, otherValueDeserializer) + .withKeys(keySerde) + .withValues(otherValueSerde) .persistent() .windowed(windows.maintainMs(), windows.segments, true) .build(); @@ -354,16 +338,14 @@ public KStream leftJoin( KStream other, ValueJoiner joiner, JoinWindows windows, - Serializer keySerializer, - Serializer otherValueSerializer, - Deserializer keyDeserializer, - Deserializer otherValueDeserializer) { + Serde keySerde, + Serde otherValueSerde) { Set allSourceNodes = ensureJoinableWith((AbstractStream) other); StateStoreSupplier otherWindow = Stores.create(windows.name() + "-other") - .withKeys(keySerializer, keyDeserializer) - .withValues(otherValueSerializer, otherValueDeserializer) + .withKeys(keySerde) + .withValues(otherValueSerde) .persistent() .windowed(windows.maintainMs(), windows.segments, true) .build(); @@ -397,18 +379,16 @@ public KStream leftJoin(KTable other, ValueJoiner @Override public KTable, V> reduceByKey(Reducer reducer, Windows windows, - Serializer keySerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer aggValueDeserializer) { + Serde keySerde, + Serde aggValueSerde) { String reduceName = topology.newName(REDUCE_NAME); KStreamWindowReduce reduceSupplier = new KStreamWindowReduce<>(windows, windows.name(), reducer); StateStoreSupplier reduceStore = Stores.create(windows.name()) - .withKeys(keySerializer, keyDeserializer) - .withValues(aggValueSerializer, aggValueDeserializer) + .withKeys(keySerde) + .withValues(aggValueSerde) .persistent() .windowed(windows.maintainMs(), windows.segments, false) .build(); @@ -423,10 +403,8 @@ public KTable, V> reduceByKey(Reducer reducer, @Override public KTable reduceByKey(Reducer reducer, - Serializer keySerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer aggValueDeserializer, + Serde keySerde, + Serde aggValueSerde, String name) { String reduceName = topology.newName(REDUCE_NAME); @@ -434,8 +412,8 @@ public KTable reduceByKey(Reducer reducer, KStreamReduce reduceSupplier = new KStreamReduce<>(name, reducer); StateStoreSupplier reduceStore = Stores.create(name) - .withKeys(keySerializer, keyDeserializer) - .withValues(aggValueSerializer, aggValueDeserializer) + .withKeys(keySerde) + .withValues(aggValueSerde) .persistent() .build(); @@ -451,18 +429,16 @@ public KTable reduceByKey(Reducer reducer, public KTable, T> aggregateByKey(Initializer initializer, Aggregator aggregator, Windows windows, - Serializer keySerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer aggValueDeserializer) { + Serde keySerde, + Serde aggValueSerde) { String aggregateName = topology.newName(AGGREGATE_NAME); KStreamAggProcessorSupplier, V, T> aggregateSupplier = new KStreamWindowAggregate<>(windows, windows.name(), initializer, aggregator); StateStoreSupplier aggregateStore = Stores.create(windows.name()) - .withKeys(keySerializer, keyDeserializer) - .withValues(aggValueSerializer, aggValueDeserializer) + .withKeys(keySerde) + .withValues(aggValueSerde) .persistent() .windowed(windows.maintainMs(), windows.segments, false) .build(); @@ -478,10 +454,8 @@ public KTable, T> aggregateByKey(Initializer KTable aggregateByKey(Initializer initializer, Aggregator aggregator, - Serializer keySerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer aggValueDeserializer, + Serde keySerde, + Serde aggValueSerde, String name) { String aggregateName = topology.newName(AGGREGATE_NAME); @@ -489,8 +463,8 @@ public KTable aggregateByKey(Initializer initializer, KStreamAggProcessorSupplier aggregateSupplier = new KStreamAggregate<>(name, initializer, aggregator); StateStoreSupplier aggregateStore = Stores.create(name) - .withKeys(keySerializer, keyDeserializer) - .withValues(aggValueSerializer, aggValueDeserializer) + .withKeys(keySerde) + .withValues(aggValueSerde) .persistent() .build(); @@ -504,8 +478,7 @@ public KTable aggregateByKey(Initializer initializer, @Override public KTable, Long> countByKey(Windows windows, - Serializer keySerializer, - Deserializer keyDeserializer) { + Serde keySerde) { return this.aggregateByKey( new Initializer() { @Override @@ -518,13 +491,12 @@ public Long apply() { public Long apply(K aggKey, V value, Long aggregate) { return aggregate + 1L; } - }, windows, keySerializer, LONG_SERIALIZER, keyDeserializer, LONG_DESERIALIZER); + }, windows, keySerde, Serdes.Long()); } @Override - public KTable countByKey(Serializer keySerializer, - Deserializer keyDeserializer, - String name) { + public KTable countByKey(Serde keySerde, + String name) { return this.aggregateByKey( new Initializer() { @Override @@ -537,6 +509,6 @@ public Long apply() { public Long apply(K aggKey, V value, Long aggregate) { return aggregate + 1L; } - }, keySerializer, LONG_SERIALIZER, keyDeserializer, LONG_DESERIALIZER, name); + }, keySerde, Serdes.Long(), name); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index d63fcc82bde5f..496a476afaef1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -17,10 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.LongDeserializer; -import org.apache.kafka.common.serialization.LongSerializer; -import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Initializer; import org.apache.kafka.streams.kstream.KStream; @@ -77,15 +75,10 @@ public class KTableImpl extends AbstractStream implements KTable processorSupplier; - private final Serializer keySerializer; - private final Serializer valSerializer; - private final Deserializer keyDeserializer; - private final Deserializer valDeserializer; + private final Serde keySerde; + private final Serde valSerde; private boolean sendOldValues = false; @@ -93,23 +86,19 @@ public KTableImpl(KStreamBuilder topology, String name, ProcessorSupplier processorSupplier, Set sourceNodes) { - this(topology, name, processorSupplier, sourceNodes, null, null, null, null); + this(topology, name, processorSupplier, sourceNodes, null, null); } public KTableImpl(KStreamBuilder topology, String name, ProcessorSupplier processorSupplier, Set sourceNodes, - Serializer keySerializer, - Serializer valSerializer, - Deserializer keyDeserializer, - Deserializer valDeserializer) { + Serde keySerde, + Serde valSerde) { super(topology, name, sourceNodes); this.processorSupplier = processorSupplier; - this.keySerializer = keySerializer; - this.valSerializer = valSerializer; - this.keyDeserializer = keyDeserializer; - this.valDeserializer = valDeserializer; + this.keySerde = keySerde; + this.valSerde = valSerde; } @Override @@ -143,18 +132,16 @@ public KTable mapValues(ValueMapper mapper) { @Override public KTable through(String topic, - Serializer keySerializer, - Serializer valSerializer, - Deserializer keyDeserializer, - Deserializer valDeserializer) { - to(topic, keySerializer, valSerializer); + Serde keySerde, + Serde valSerde) { + to(topic, keySerde, valSerde); - return topology.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic); + return topology.table(keySerde, valSerde, topic); } @Override public KTable through(String topic) { - return through(topic, null, null, null, null); + return through(topic, null, null); } @Override @@ -163,8 +150,8 @@ public void to(String topic) { } @Override - public void to(String topic, Serializer keySerializer, Serializer valSerializer) { - this.toStream().to(topic, keySerializer, valSerializer); + public void to(String topic, Serde keySerde, Serde valSerde) { + this.toStream().to(topic, keySerde, valSerde); } @Override @@ -255,12 +242,9 @@ public KTable aggregate(Initializer initializer, Aggregator add, Aggregator remove, KeyValueMapper> selector, - Serializer keySerializer, - Serializer valueSerializer, - Serializer aggValueSerializer, - Deserializer keyDeserializer, - Deserializer valueDeserializer, - Deserializer aggValueDeserializer, + Serde keySerde, + Serde valueSerde, + Serde aggValueSerde, String name) { String selectName = topology.newName(SELECT_NAME); @@ -270,16 +254,16 @@ public KTable aggregate(Initializer initializer, String topic = name + REPARTITION_TOPIC_SUFFIX; - ChangedSerializer changedValueSerializer = new ChangedSerializer<>(valueSerializer); - ChangedDeserializer changedValueDeserializer = new ChangedDeserializer<>(valueDeserializer); + ChangedSerializer changedValueSerializer = new ChangedSerializer<>(valueSerde.serializer()); + ChangedDeserializer changedValueDeserializer = new ChangedDeserializer<>(valueSerde.deserializer()); KTableProcessorSupplier> selectSupplier = new KTableRepartitionMap<>(this, selector); ProcessorSupplier> aggregateSupplier = new KTableAggregate<>(name, initializer, add, remove); StateStoreSupplier aggregateStore = Stores.create(name) - .withKeys(keySerializer, keyDeserializer) - .withValues(aggValueSerializer, aggValueDeserializer) + .withKeys(keySerde) + .withValues(aggValueSerde) .persistent() .build(); @@ -289,10 +273,10 @@ public KTable aggregate(Initializer initializer, // send the aggregate key-value pairs to the intermediate topic for partitioning topology.addInternalTopic(topic); - topology.addSink(sinkName, topic, keySerializer, changedValueSerializer, selectName); + topology.addSink(sinkName, topic, keySerde.serializer(), changedValueSerializer, selectName); // read the intermediate topic - topology.addSource(sourceName, keyDeserializer, changedValueDeserializer, topic); + topology.addSource(sourceName, keySerde.deserializer(), changedValueDeserializer, topic); // aggregate the values with the aggregator and local store topology.addProcessor(aggregateName, aggregateSupplier, sourceName); @@ -304,10 +288,8 @@ public KTable aggregate(Initializer initializer, @Override public KTable count(final KeyValueMapper selector, - Serializer keySerializer, - Serializer valueSerializer, - Deserializer keyDeserializer, - Deserializer valueDeserializer, + Serde keySerde, + Serde valueSerde, String name) { return this.aggregate( new Initializer() { @@ -332,17 +314,15 @@ public KeyValue apply(K key, V value) { return new KeyValue<>(selector.apply(key, value), value); } }, - keySerializer, valueSerializer, LONG_SERIALIZER, keyDeserializer, valueDeserializer, LONG_DESERIALIZER, name); + keySerde, valueSerde, Serdes.Long(), name); } @Override public KTable reduce(Reducer addReducer, Reducer removeReducer, KeyValueMapper> selector, - Serializer keySerializer, - Serializer valueSerializer, - Deserializer keyDeserializer, - Deserializer valueDeserializer, + Serde keySerde, + Serde valueSerde, String name) { String selectName = topology.newName(SELECT_NAME); @@ -352,16 +332,16 @@ public KTable reduce(Reducer addReducer, String topic = name + REPARTITION_TOPIC_SUFFIX; - ChangedSerializer changedValueSerializer = new ChangedSerializer<>(valueSerializer); - ChangedDeserializer changedValueDeserializer = new ChangedDeserializer<>(valueDeserializer); + ChangedSerializer changedValueSerializer = new ChangedSerializer<>(valueSerde.serializer()); + ChangedDeserializer changedValueDeserializer = new ChangedDeserializer<>(valueSerde.deserializer()); KTableProcessorSupplier> selectSupplier = new KTableRepartitionMap<>(this, selector); ProcessorSupplier> aggregateSupplier = new KTableReduce<>(name, addReducer, removeReducer); StateStoreSupplier aggregateStore = Stores.create(name) - .withKeys(keySerializer, keyDeserializer) - .withValues(valueSerializer, valueDeserializer) + .withKeys(keySerde) + .withValues(valueSerde) .persistent() .build(); @@ -371,10 +351,10 @@ public KTable reduce(Reducer addReducer, // send the aggregate key-value pairs to the intermediate topic for partitioning topology.addInternalTopic(topic); - topology.addSink(sinkName, topic, keySerializer, changedValueSerializer, selectName); + topology.addSink(sinkName, topic, keySerde.serializer(), changedValueSerializer, selectName); // read the intermediate topic - topology.addSource(sourceName, keyDeserializer, changedValueDeserializer, topic); + topology.addSource(sourceName, keySerde.deserializer(), changedValueDeserializer, topic); // aggregate the values with the aggregator and local store topology.addProcessor(reduceName, aggregateSupplier, sourceName); @@ -421,7 +401,7 @@ private void materialize(KTableSource source) { synchronized (source) { if (!source.isMaterialized()) { StateStoreSupplier storeSupplier = - new KTableStoreSupplier<>(source.topic, keySerializer, keyDeserializer, valSerializer, valDeserializer, null); + new KTableStoreSupplier<>(source.topic, keySerde, valSerde, null); // mark this state as non internal hence it is read directly from a user topic topology.addStateStore(storeSupplier, false, name); source.materialize(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableStoreSupplier.java index ffd5cf05c3789..af3c0d71b5297 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableStoreSupplier.java @@ -17,14 +17,13 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.state.internals.MeteredKeyValueStore; import org.apache.kafka.streams.state.internals.RocksDBStore; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; /** * A KTable storage. It stores all entries in a local RocksDB database. @@ -35,15 +34,15 @@ public class KTableStoreSupplier implements StateStoreSupplier { private final String name; - private final Serdes serdes; + private final StateSerdes serdes; private final Time time; protected KTableStoreSupplier(String name, - Serializer keySerializer, Deserializer keyDeserializer, - Serializer valSerializer, Deserializer valDeserializer, + Serde keySerde, + Serde valSerde, Time time) { this.name = name; - this.serdes = new Serdes<>(name, keySerializer, keyDeserializer, valSerializer, valDeserializer); + this.serdes = new StateSerdes<>(name, keySerde, valSerde); this.time = time; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java index e9d5252b1e31d..fdcff19e01bc2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.processor; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.StreamsMetrics; import java.io.File; @@ -43,32 +42,18 @@ public interface ProcessorContext { TaskId taskId(); /** - * Returns the key serializer + * Returns the default key serde * * @return the key serializer */ - Serializer keySerializer(); + Serde keySerde(); /** - * Returns the value serializer + * Returns the default value serde * * @return the value serializer */ - Serializer valueSerializer(); - - /** - * Returns the key deserializer - * - * @return the key deserializer - */ - Deserializer keyDeserializer(); - - /** - * Returns the value deserializer - * - * @return the value deserializer - */ - Deserializer valueDeserializer(); + Serde valueSerde(); /** * Returns the state directory for the partition. diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java index ab7122ba1faaf..7f5d6454d55c2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java @@ -194,8 +194,8 @@ public TopologyBuilder() {} /** * Add a new source that consumes the named topics and forwards the messages to child processor and/or sink nodes. - * The source will use the {@link org.apache.kafka.streams.StreamsConfig#KEY_DESERIALIZER_CLASS_CONFIG default key deserializer} and - * {@link org.apache.kafka.streams.StreamsConfig#VALUE_DESERIALIZER_CLASS_CONFIG default value deserializer} specified in the + * The source will use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key deserializer} and + * {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value deserializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration}. * * @param name the unique name of the source used to reference this node when @@ -214,10 +214,10 @@ public final TopologyBuilder addSource(String name, String... topics) { * @param name the unique name of the source used to reference this node when * {@link #addProcessor(String, ProcessorSupplier, String...) adding processor children}. * @param keyDeserializer the {@link Deserializer key deserializer} used when consuming messages; may be null if the source - * should use the {@link org.apache.kafka.streams.StreamsConfig#KEY_DESERIALIZER_CLASS_CONFIG default key deserializer} specified in the + * should use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key deserializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration} * @param valDeserializer the {@link Deserializer value deserializer} used when consuming messages; may be null if the source - * should use the {@link org.apache.kafka.streams.StreamsConfig#VALUE_DESERIALIZER_CLASS_CONFIG default value deserializer} specified in the + * should use the {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value deserializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration} * @param topics the name of one or more Kafka topics that this source is to consume * @return this builder instance so methods can be chained together; never null @@ -242,8 +242,8 @@ public final TopologyBuilder addSource(String name, Deserializer keyDeserializer /** * Add a new sink that forwards messages from upstream parent processor and/or source nodes to the named Kafka topic. - * The sink will use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERIALIZER_CLASS_CONFIG default key serializer} and - * {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERIALIZER_CLASS_CONFIG default value serializer} specified in the + * The sink will use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key serializer} and + * {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value serializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration}. * * @param name the unique name of the sink @@ -262,8 +262,8 @@ public final TopologyBuilder addSink(String name, String topic, String... parent /** * Add a new sink that forwards messages from upstream parent processor and/or source nodes to the named Kafka topic, using * the supplied partitioner. - * The sink will use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERIALIZER_CLASS_CONFIG default key serializer} and - * {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERIALIZER_CLASS_CONFIG default value serializer} specified in the + * The sink will use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key serializer} and + * {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value serializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration}. *

      * The sink will also use the specified {@link StreamPartitioner} to determine how messages are distributed among @@ -293,10 +293,10 @@ public final TopologyBuilder addSink(String name, String topic, StreamPartitione * @param name the unique name of the sink * @param topic the name of the Kafka topic to which this sink should write its messages * @param keySerializer the {@link Serializer key serializer} used when consuming messages; may be null if the sink - * should use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERIALIZER_CLASS_CONFIG default key serializer} specified in the + * should use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key serializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration} * @param valSerializer the {@link Serializer value serializer} used when consuming messages; may be null if the sink - * should use the {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERIALIZER_CLASS_CONFIG default value serializer} specified in the + * should use the {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value serializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration} * @param parentNames the name of one or more source or processor nodes whose output message this sink should consume * and write to its topic @@ -316,10 +316,10 @@ public final TopologyBuilder addSink(String name, String topic, Serializer keySe * @param name the unique name of the sink * @param topic the name of the Kafka topic to which this sink should write its messages * @param keySerializer the {@link Serializer key serializer} used when consuming messages; may be null if the sink - * should use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERIALIZER_CLASS_CONFIG default key serializer} specified in the + * should use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key serializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration} * @param valSerializer the {@link Serializer value serializer} used when consuming messages; may be null if the sink - * should use the {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERIALIZER_CLASS_CONFIG default value serializer} specified in the + * should use the {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value serializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration} * @param partitioner the function that should be used to determine the partition for each message processed by the sink * @param parentNames the name of one or more source or processor nodes whose output message this sink should consume diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index f6e43d0f18a90..888b89e2deade 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -17,11 +17,10 @@ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.streams.errors.TopologyBuilderException; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsMetrics; -import org.apache.kafka.streams.errors.TopologyBuilderException; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateRestoreCallback; @@ -37,10 +36,8 @@ public class ProcessorContextImpl implements ProcessorContext, RecordCollector.S private final RecordCollector collector; private final ProcessorStateManager stateMgr; - private final Serializer keySerializer; - private final Serializer valSerializer; - private final Deserializer keyDeserializer; - private final Deserializer valDeserializer; + private final Serde keySerde; + private final Serde valSerde; private boolean initialized; @@ -57,10 +54,8 @@ public ProcessorContextImpl(TaskId id, this.collector = collector; this.stateMgr = stateMgr; - this.keySerializer = config.keySerializer(); - this.valSerializer = config.valueSerializer(); - this.keyDeserializer = config.keyDeserializer(); - this.valDeserializer = config.valueDeserializer(); + this.keySerde = config.keySerde(); + this.valSerde = config.valueSerde(); this.initialized = false; } @@ -89,23 +84,13 @@ public RecordCollector recordCollector() { } @Override - public Serializer keySerializer() { - return this.keySerializer; - } - - @Override - public Serializer valueSerializer() { - return this.valSerializer; - } - - @Override - public Deserializer keyDeserializer() { - return this.keyDeserializer; + public Serde keySerde() { + return this.keySerde; } @Override - public Deserializer valueDeserializer() { - return this.valDeserializer; + public Serde valueSerde() { + return this.valSerde; } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java index 7ab59ee6ca3f3..ffc72fd928753 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java @@ -49,8 +49,8 @@ public void addChild(ProcessorNode child) { @Override public void init(ProcessorContext context) { this.context = context; - if (this.keySerializer == null) this.keySerializer = (Serializer) context.keySerializer(); - if (this.valSerializer == null) this.valSerializer = (Serializer) context.valueSerializer(); + if (this.keySerializer == null) this.keySerializer = (Serializer) context.keySerde().serializer(); + if (this.valSerializer == null) this.valSerializer = (Serializer) context.valueSerde().serializer(); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java index fa4afafbbdf27..1868c1b20b797 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java @@ -47,8 +47,8 @@ public void init(ProcessorContext context) { this.context = context; // if serializers are null, get the default ones from the context - if (this.keyDeserializer == null) this.keyDeserializer = (Deserializer) context.keyDeserializer(); - if (this.valDeserializer == null) this.valDeserializer = (Deserializer) context.valueDeserializer(); + if (this.keyDeserializer == null) this.keyDeserializer = (Deserializer) context.keySerde().deserializer(); + if (this.valDeserializer == null) this.valDeserializer = (Deserializer) context.valueSerde().deserializer(); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java index 0bcae18270e21..3ad06e2b1d8eb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.processor.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.StreamsMetrics; import org.apache.kafka.streams.processor.ProcessorContext; @@ -35,10 +34,8 @@ public class StandbyContextImpl implements ProcessorContext, RecordCollector.Sup private final StreamsMetrics metrics; private final ProcessorStateManager stateMgr; - private final Serializer keySerializer; - private final Serializer valSerializer; - private final Deserializer keyDeserializer; - private final Deserializer valDeserializer; + private final Serde keySerde; + private final Serde valSerde; private boolean initialized; @@ -52,10 +49,8 @@ public StandbyContextImpl(TaskId id, this.metrics = metrics; this.stateMgr = stateMgr; - this.keySerializer = config.keySerializer(); - this.valSerializer = config.valueSerializer(); - this.keyDeserializer = config.keyDeserializer(); - this.valDeserializer = config.valueDeserializer(); + this.keySerde = config.keySerde(); + this.valSerde = config.valueSerde(); this.initialized = false; } @@ -84,23 +79,13 @@ public RecordCollector recordCollector() { } @Override - public Serializer keySerializer() { - return this.keySerializer; + public Serde keySerde() { + return this.keySerde; } @Override - public Serializer valueSerializer() { - return this.valSerializer; - } - - @Override - public Deserializer keyDeserializer() { - return this.keyDeserializer; - } - - @Override - public Deserializer valueDeserializer() { - return this.valDeserializer; + public Serde valueSerde() { + return this.valSerde; } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 491c8126fcc4c..e1a518d4b99d4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -100,7 +100,6 @@ public class StreamThread extends Thread { private long lastClean; private long lastCommit; - private long recordsProcessed; private Throwable rebalanceException = null; private Map>> standbyRecords; @@ -201,7 +200,6 @@ public StreamThread(TopologyBuilder builder, this.lastClean = Long.MAX_VALUE; // the cleaning cycle won't start until partition assignment this.lastCommit = time.milliseconds(); - this.recordsProcessed = 0; this.time = time; this.sensors = new StreamsMetricsImpl(metrics); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/Serdes.java b/streams/src/main/java/org/apache/kafka/streams/state/Serdes.java deleted file mode 100644 index e92531210b1fb..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/state/Serdes.java +++ /dev/null @@ -1,136 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.streams.state; - -import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.LongDeserializer; -import org.apache.kafka.common.serialization.LongSerializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; - -/** - * Factory for creating serializers / deserializers for state stores in Kafka Streams. - * - * @param key type of serdes - * @param value type of serdes - */ -public final class Serdes { - - public static Serdes withBuiltinTypes(String topic, Class keyClass, Class valueClass) { - Serializer keySerializer = serializer(keyClass); - Deserializer keyDeserializer = deserializer(keyClass); - Serializer valueSerializer = serializer(valueClass); - Deserializer valueDeserializer = deserializer(valueClass); - return new Serdes<>(topic, keySerializer, keyDeserializer, valueSerializer, valueDeserializer); - } - - @SuppressWarnings("unchecked") - static Serializer serializer(Class type) { - if (String.class.isAssignableFrom(type)) return (Serializer) new StringSerializer(); - if (Integer.class.isAssignableFrom(type)) return (Serializer) new IntegerSerializer(); - if (Long.class.isAssignableFrom(type)) return (Serializer) new LongSerializer(); - if (byte[].class.isAssignableFrom(type)) return (Serializer) new ByteArraySerializer(); - throw new IllegalArgumentException("Unknown class for built-in serializer"); - } - - @SuppressWarnings("unchecked") - static Deserializer deserializer(Class type) { - if (String.class.isAssignableFrom(type)) return (Deserializer) new StringDeserializer(); - if (Integer.class.isAssignableFrom(type)) return (Deserializer) new IntegerDeserializer(); - if (Long.class.isAssignableFrom(type)) return (Deserializer) new LongDeserializer(); - if (byte[].class.isAssignableFrom(type)) return (Deserializer) new ByteArrayDeserializer(); - throw new IllegalArgumentException("Unknown class for built-in serializer"); - } - - private final String topic; - private Serializer keySerializer; - private Serializer valueSerializer; - private Deserializer keyDeserializer; - private Deserializer valueDeserializer; - - /** - * Create a context for serialization using the specified serializers and deserializers which - * must match the key and value types used as parameters for this object. - * - * @param topic the name of the topic - * @param keySerializer the serializer for keys; may be null - * @param keyDeserializer the deserializer for keys; may be null - * @param valueSerializer the serializer for values; may be null - * @param valueDeserializer the deserializer for values; may be null - */ - @SuppressWarnings("unchecked") - public Serdes(String topic, - Serializer keySerializer, Deserializer keyDeserializer, - Serializer valueSerializer, Deserializer valueDeserializer) { - this.topic = topic; - - if (keySerializer == null) - throw new NullPointerException(); - if (keyDeserializer == null) - throw new NullPointerException(); - if (valueSerializer == null) - throw new NullPointerException(); - if (valueDeserializer == null) - throw new NullPointerException(); - - this.keySerializer = keySerializer; - this.keyDeserializer = keyDeserializer; - this.valueSerializer = valueSerializer; - this.valueDeserializer = valueDeserializer; - } - - public Deserializer keyDeserializer() { - return keyDeserializer; - } - - public Serializer keySerializer() { - return keySerializer; - } - - public Deserializer valueDeserializer() { - return valueDeserializer; - } - - public Serializer valueSerializer() { - return valueSerializer; - } - - public String topic() { - return topic; - } - - public K keyFrom(byte[] rawKey) { - return keyDeserializer.deserialize(topic, rawKey); - } - - public V valueFrom(byte[] rawValue) { - return valueDeserializer.deserialize(topic, rawValue); - } - - public byte[] rawKey(K key) { - return keySerializer.serialize(topic, key); - } - - public byte[] rawValue(V value) { - return valueSerializer.serialize(topic, value); - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java b/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java new file mode 100644 index 0000000000000..1a41a1660fa35 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java @@ -0,0 +1,108 @@ +/** + * 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 org.apache.kafka.streams.state; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.Serializer; + +/** + * Factory for creating serializers / deserializers for state stores in Kafka Streams. + * + * @param key type of serdes + * @param value type of serdes + */ +public final class StateSerdes { + + public static StateSerdes withBuiltinTypes(String topic, Class keyClass, Class valueClass) { + return new StateSerdes<>(topic, Serdes.serdeFrom(keyClass), Serdes.serdeFrom(valueClass)); + } + + private final String topic; + private final Serde keySerde; + private final Serde valueSerde; + + /** + * Create a context for serialization using the specified serializers and deserializers which + * must match the key and value types used as parameters for this object; the state changelog topic + * is provided to bind this serde factory to, so that future calls for serialize / deserialize do not + * need to provide the topic name any more. + * + * @param topic the name of the topic + * @param keySerde the serde for keys; cannot be null + * @param valueSerde the serde for values; cannot be null + */ + @SuppressWarnings("unchecked") + public StateSerdes(String topic, + Serde keySerde, + Serde valueSerde) { + this.topic = topic; + + if (keySerde == null) + throw new IllegalArgumentException("key serde cannot be null"); + if (valueSerde == null) + throw new IllegalArgumentException("value serde cannot be null"); + + this.keySerde = keySerde; + this.valueSerde = valueSerde; + } + + public Serde keySerde() { + return keySerde; + } + + public Serde valueSerde() { + return valueSerde; + } + + public Deserializer keyDeserializer() { + return keySerde.deserializer(); + } + + public Serializer keySerializer() { + return keySerde.serializer(); + } + + public Deserializer valueDeserializer() { + return valueSerde.deserializer(); + } + + public Serializer valueSerializer() { + return valueSerde.serializer(); + } + + public String topic() { + return topic; + } + + public K keyFrom(byte[] rawKey) { + return keySerde.deserializer().deserialize(topic, rawKey); + } + + public V valueFrom(byte[] rawValue) { + return valueSerde.deserializer().deserialize(topic, rawValue); + } + + public byte[] rawKey(K key) { + return keySerde.serializer().serialize(topic, key); + } + + public byte[] rawValue(V value) { + return valueSerde.serializer().serialize(topic, value); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java index e803832ba8c4b..33df13f79aa00 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java @@ -16,22 +16,16 @@ */ package org.apache.kafka.streams.state; -import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import org.apache.kafka.common.serialization.ByteArraySerializer; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.LongDeserializer; -import org.apache.kafka.common.serialization.LongSerializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.state.internals.InMemoryKeyValueStoreSupplier; import org.apache.kafka.streams.state.internals.InMemoryLRUCacheStoreSupplier; import org.apache.kafka.streams.state.internals.RocksDBKeyValueStoreSupplier; import org.apache.kafka.streams.state.internals.RocksDBWindowStoreSupplier; +import java.nio.ByteBuffer; + /** * Factory for creating state stores in Kafka Streams. */ @@ -46,13 +40,12 @@ public class Stores { public static StoreFactory create(final String name) { return new StoreFactory() { @Override - public ValueFactory withKeys(final Serializer keySerializer, final Deserializer keyDeserializer) { + public ValueFactory withKeys(final Serde keySerde) { return new ValueFactory() { @Override - public KeyValueFactory withValues(final Serializer valueSerializer, - final Deserializer valueDeserializer) { - final Serdes serdes = - new Serdes<>(name, keySerializer, keyDeserializer, valueSerializer, valueDeserializer); + public KeyValueFactory withValues(final Serde valueSerde) { + final StateSerdes serdes = + new StateSerdes<>(name, keySerde, valueSerde); return new KeyValueFactory() { @Override public InMemoryKeyValueFactory inMemory() { @@ -116,7 +109,7 @@ public static abstract class StoreFactory { * @return the interface used to specify the type of values; never null */ public ValueFactory withStringKeys() { - return withKeys(new StringSerializer(), new StringDeserializer()); + return withKeys(Serdes.String()); } /** @@ -125,7 +118,7 @@ public ValueFactory withStringKeys() { * @return the interface used to specify the type of values; never null */ public ValueFactory withIntegerKeys() { - return withKeys(new IntegerSerializer(), new IntegerDeserializer()); + return withKeys(Serdes.Integer()); } /** @@ -134,7 +127,25 @@ public ValueFactory withIntegerKeys() { * @return the interface used to specify the type of values; never null */ public ValueFactory withLongKeys() { - return withKeys(new LongSerializer(), new LongDeserializer()); + return withKeys(Serdes.Long()); + } + + /** + * Begin to create a {@link KeyValueStore} by specifying the keys will be {@link Double}s. + * + * @return the interface used to specify the type of values; never null + */ + public ValueFactory withDoubleKeys() { + return withKeys(Serdes.Double()); + } + + /** + * Begin to create a {@link KeyValueStore} by specifying the keys will be {@link ByteBuffer}. + * + * @return the interface used to specify the type of values; never null + */ + public ValueFactory withByteBufferKeys() { + return withKeys(Serdes.ByteBuffer()); } /** @@ -143,30 +154,26 @@ public ValueFactory withLongKeys() { * @return the interface used to specify the type of values; never null */ public ValueFactory withByteArrayKeys() { - return withKeys(new ByteArraySerializer(), new ByteArrayDeserializer()); + return withKeys(Serdes.ByteArray()); } /** - * Begin to create a {@link KeyValueStore} by specifying the keys will be either {@link String}, {@link Integer}, - * {@link Long}, or {@code byte[]}. + * Begin to create a {@link KeyValueStore} by specifying the keys. * - * @param keyClass the class for the keys, which must be one of the types for which Kafka has built-in serializers and - * deserializers (e.g., {@code String.class}, {@code Integer.class}, {@code Long.class}, or - * {@code byte[].class}) + * @param keyClass the class for the keys, which must be one of the types for which Kafka has built-in serdes * @return the interface used to specify the type of values; never null */ public ValueFactory withKeys(Class keyClass) { - return withKeys(Serdes.serializer(keyClass), Serdes.deserializer(keyClass)); + return withKeys(Serdes.serdeFrom(keyClass)); } /** * Begin to create a {@link KeyValueStore} by specifying the serializer and deserializer for the keys. * - * @param keySerializer the serializer for keys; may not be null - * @param keyDeserializer the deserializer for keys; may not be null + * @param keySerde the serialization factory for keys; may not be null * @return the interface used to specify the type of values; never null */ - public abstract ValueFactory withKeys(Serializer keySerializer, Deserializer keyDeserializer); + public abstract ValueFactory withKeys(Serde keySerde); } /** @@ -181,7 +188,7 @@ public static abstract class ValueFactory { * @return the interface used to specify the remaining key-value store options; never null */ public KeyValueFactory withStringValues() { - return withValues(new StringSerializer(), new StringDeserializer()); + return withValues(Serdes.String()); } /** @@ -190,7 +197,7 @@ public KeyValueFactory withStringValues() { * @return the interface used to specify the remaining key-value store options; never null */ public KeyValueFactory withIntegerValues() { - return withValues(new IntegerSerializer(), new IntegerDeserializer()); + return withValues(Serdes.Integer()); } /** @@ -199,7 +206,25 @@ public KeyValueFactory withIntegerValues() { * @return the interface used to specify the remaining key-value store options; never null */ public KeyValueFactory withLongValues() { - return withValues(new LongSerializer(), new LongDeserializer()); + return withValues(Serdes.Long()); + } + + /** + * Use {@link Double} values. + * + * @return the interface used to specify the remaining key-value store options; never null + */ + public KeyValueFactory withDoubleValues() { + return withValues(Serdes.Double()); + } + + /** + * Use {@link ByteBuffer} for values. + * + * @return the interface used to specify the remaining key-value store options; never null + */ + public KeyValueFactory withByteBufferValues() { + return withValues(Serdes.ByteBuffer()); } /** @@ -208,30 +233,26 @@ public KeyValueFactory withLongValues() { * @return the interface used to specify the remaining key-value store options; never null */ public KeyValueFactory withByteArrayValues() { - return withValues(new ByteArraySerializer(), new ByteArrayDeserializer()); + return withValues(Serdes.ByteArray()); } /** - * Use values of the specified type, which must be either {@link String}, {@link Integer}, {@link Long}, or {@code byte[]} - * . + * Use values of the specified type. * - * @param valueClass the class for the values, which must be one of the types for which Kafka has built-in serializers and - * deserializers (e.g., {@code String.class}, {@code Integer.class}, {@code Long.class}, or - * {@code byte[].class}) + * @param valueClass the class for the values, which must be one of the types for which Kafka has built-in serdes * @return the interface used to specify the remaining key-value store options; never null */ public KeyValueFactory withValues(Class valueClass) { - return withValues(Serdes.serializer(valueClass), Serdes.deserializer(valueClass)); + return withValues(Serdes.serdeFrom(valueClass)); } /** * Use the specified serializer and deserializer for the values. * - * @param valueSerializer the serializer for value; may not be null - * @param valueDeserializer the deserializer for values; may not be null + * @param valueSerde the serialization factory for values; may not be null * @return the interface used to specify the remaining key-value store options; never null */ - public abstract KeyValueFactory withValues(Serializer valueSerializer, Deserializer valueDeserializer); + public abstract KeyValueFactory withValues(Serde valueSerde); } /** @@ -240,7 +261,7 @@ public KeyValueFactory withValues(Class valueClass) { * @param the type of keys * @param the type of values */ - public static interface KeyValueFactory { + public interface KeyValueFactory { /** * Keep all key-value entries in-memory, although for durability all entries are recorded in a Kafka topic that can be * read to restore the entries if they are lost. @@ -264,7 +285,7 @@ public static interface KeyValueFactory { * @param the type of keys * @param the type of values */ - public static interface InMemoryKeyValueFactory { + public interface InMemoryKeyValueFactory { /** * Limits the in-memory key-value store to hold a maximum number of entries. The default is {@link Integer#MAX_VALUE}, which is * equivalent to not placing a limit on the number of entries. @@ -288,7 +309,7 @@ public static interface InMemoryKeyValueFactory { * @param the type of keys * @param the type of values */ - public static interface PersistentKeyValueFactory { + public interface PersistentKeyValueFactory { /** * Set the persistent store as a windowed key-value store diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java index c6bbb232c5f58..66e13387eb153 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java @@ -25,11 +25,11 @@ public class WindowStoreUtils { public static final int TIMESTAMP_SIZE = 8; public static final int SEQNUM_SIZE = 4; - public static final Serdes INNER_SERDES = Serdes.withBuiltinTypes("", byte[].class, byte[].class); + public static final StateSerdes INNER_SERDES = StateSerdes.withBuiltinTypes("", byte[].class, byte[].class); @SuppressWarnings("unchecked") public static final KeyValueIterator[] NO_ITERATORS = (KeyValueIterator[]) new KeyValueIterator[0]; - public static byte[] toBinaryKey(K key, final long timestamp, final int seqnum, Serdes serdes) { + public static byte[] toBinaryKey(K key, final long timestamp, final int seqnum, StateSerdes serdes) { byte[] serializedKey = serdes.rawKey(key); ByteBuffer buf = ByteBuffer.allocate(serializedKey.length + TIMESTAMP_SIZE + SEQNUM_SIZE); @@ -40,7 +40,7 @@ public static byte[] toBinaryKey(K key, final long timestamp, final int seqn return buf.array(); } - public static K keyFromBinaryKey(byte[] binaryKey, Serdes serdes) { + public static K keyFromBinaryKey(byte[] binaryKey, StateSerdes serdes) { byte[] bytes = new byte[binaryKey.length - TIMESTAMP_SIZE - SEQNUM_SIZE]; System.arraycopy(binaryKey, 0, bytes, 0, bytes.length); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java index d25faa83600ad..32116ddf725f4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java @@ -22,20 +22,20 @@ import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; import java.util.List; public class InMemoryKeyValueLoggedStore implements KeyValueStore { private final KeyValueStore inner; - private final Serdes serdes; + private final StateSerdes serdes; private final String storeName; private StoreChangeLogger changeLogger; private StoreChangeLogger.ValueGetter getter; - public InMemoryKeyValueLoggedStore(final String storeName, final KeyValueStore inner, final Serdes serdes) { + public InMemoryKeyValueLoggedStore(final String storeName, final KeyValueStore inner, final StateSerdes serdes) { this.storeName = storeName; this.inner = inner; this.serdes = serdes; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java index b96a1034bbb04..4054d683f381a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java @@ -25,7 +25,7 @@ import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; import java.util.Iterator; import java.util.List; @@ -45,9 +45,9 @@ public class InMemoryKeyValueStoreSupplier implements StateStoreSupplier { private final String name; private final Time time; - private final Serdes serdes; + private final StateSerdes serdes; - public InMemoryKeyValueStoreSupplier(String name, Serdes serdes, Time time) { + public InMemoryKeyValueStoreSupplier(String name, StateSerdes serdes, Time time) { this.name = name; this.time = time; this.serdes = serdes; @@ -67,7 +67,7 @@ private static class MemoryStore implements KeyValueStore { private final NavigableMap map; private boolean loggingEnabled = false; - private Serdes serdes = null; + private StateSerdes serdes = null; public MemoryStore(String name) { super(); @@ -75,7 +75,7 @@ public MemoryStore(String name) { this.map = new TreeMap<>(); } - public KeyValueStore enableLogging(Serdes serdes) { + public KeyValueStore enableLogging(StateSerdes serdes) { this.loggingEnabled = true; this.serdes = serdes; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreSupplier.java index 9b7936a3bcd2f..1c2241fdc1a60 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreSupplier.java @@ -19,7 +19,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreSupplier; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; /** * An in-memory key-value store that is limited in size and retains a maximum number of most recently used entries. @@ -32,10 +32,10 @@ public class InMemoryLRUCacheStoreSupplier implements StateStoreSupplier { private final String name; private final int capacity; - private final Serdes serdes; + private final StateSerdes serdes; private final Time time; - public InMemoryLRUCacheStoreSupplier(String name, int capacity, Serdes serdes, Time time) { + public InMemoryLRUCacheStoreSupplier(String name, int capacity, StateSerdes serdes, Time time) { this.name = name; this.capacity = capacity; this.serdes = serdes; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java index bd03f03b515b7..a5aaa0666afa7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java @@ -22,7 +22,7 @@ import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; import java.util.HashSet; import java.util.LinkedHashMap; @@ -43,7 +43,7 @@ public interface EldestEntryRemovalListener { protected EldestEntryRemovalListener listener; private boolean loggingEnabled = false; - private Serdes serdes = null; + private StateSerdes serdes = null; // this is used for extended MemoryNavigableLRUCache only public MemoryLRUCache() {} @@ -69,7 +69,7 @@ protected boolean removeEldestEntry(Map.Entry eldest) { }; } - public KeyValueStore enableLogging(Serdes serdes) { + public KeyValueStore enableLogging(StateSerdes serdes) { this.loggingEnabled = true; this.serdes = serdes; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java index 3a4c351dcaeb4..ec10c3fdfa6cd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java @@ -20,7 +20,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreSupplier; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; /** * A {@link org.apache.kafka.streams.state.KeyValueStore} that stores all entries in a local RocksDB database. @@ -33,10 +33,10 @@ public class RocksDBKeyValueStoreSupplier implements StateStoreSupplier { private final String name; - private final Serdes serdes; + private final StateSerdes serdes; private final Time time; - public RocksDBKeyValueStoreSupplier(String name, Serdes serdes, Time time) { + public RocksDBKeyValueStoreSupplier(String name, StateSerdes serdes, Time time) { this.name = name; this.serdes = serdes; this.time = time; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index c295aea6e8ed1..3045856451469 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -24,7 +24,7 @@ import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; import org.rocksdb.BlockBasedTableConfig; import org.rocksdb.CompactionStyle; @@ -68,7 +68,7 @@ public class RocksDBStore implements KeyValueStore { private final FlushOptions fOptions; private ProcessorContext context; - private Serdes serdes; + private StateSerdes serdes; protected File dbDir; private RocksDB db; @@ -92,11 +92,11 @@ public RocksDBStore withCacheSize(int cacheSize) { return this; } - public RocksDBStore(String name, Serdes serdes) { + public RocksDBStore(String name, StateSerdes serdes) { this(name, DB_FILE_DIR, serdes); } - public RocksDBStore(String name, String parentDir, Serdes serdes) { + public RocksDBStore(String name, String parentDir, StateSerdes serdes) { this.name = name; this.parentDir = parentDir; this.serdes = serdes; @@ -399,9 +399,9 @@ public void close() { private static class RocksDbIterator implements KeyValueIterator { private final RocksIterator iter; - private final Serdes serdes; + private final StateSerdes serdes; - public RocksDbIterator(RocksIterator iter, Serdes serdes) { + public RocksDbIterator(RocksIterator iter, StateSerdes serdes) { this.iter = iter; this.serdes = serdes; } @@ -463,7 +463,7 @@ private static class RocksDBRangeIterator extends RocksDbIterator { private final Comparator comparator = new LexicographicComparator(); byte[] rawToKey; - public RocksDBRangeIterator(RocksIterator iter, Serdes serdes, + public RocksDBRangeIterator(RocksIterator iter, StateSerdes serdes, K from, K to) { super(iter, serdes); iter.seek(serdes.rawKey(from)); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java index b1605a3b2518d..61c2e5eec8733 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java @@ -25,7 +25,7 @@ import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueIterator; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; import org.apache.kafka.streams.state.WindowStoreUtils; @@ -61,15 +61,15 @@ public void destroy() { } private static class RocksDBWindowStoreIterator implements WindowStoreIterator { - private final Serdes serdes; + private final StateSerdes serdes; private final KeyValueIterator[] iterators; private int index = 0; - RocksDBWindowStoreIterator(Serdes serdes) { + RocksDBWindowStoreIterator(StateSerdes serdes) { this(serdes, WindowStoreUtils.NO_ITERATORS); } - RocksDBWindowStoreIterator(Serdes serdes, KeyValueIterator[] iterators) { + RocksDBWindowStoreIterator(StateSerdes serdes, KeyValueIterator[] iterators) { this.serdes = serdes; this.iterators = iterators; } @@ -114,7 +114,7 @@ public void close() { private final long segmentInterval; private final boolean retainDuplicates; private final Segment[] segments; - private final Serdes serdes; + private final StateSerdes serdes; private final SimpleDateFormat formatter; private final StoreChangeLogger.ValueGetter getter; @@ -125,7 +125,7 @@ public void close() { private boolean loggingEnabled = false; private StoreChangeLogger changeLogger = null; - public RocksDBWindowStore(String name, long retentionPeriod, int numSegments, boolean retainDuplicates, Serdes serdes) { + public RocksDBWindowStore(String name, long retentionPeriod, int numSegments, boolean retainDuplicates, StateSerdes serdes) { this.name = name; // The segment interval must be greater than MIN_SEGMENT_INTERVAL diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java index 6823e6dbf9c31..00089abebb0e3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java @@ -20,7 +20,7 @@ import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreSupplier; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; /** * A {@link org.apache.kafka.streams.state.KeyValueStore} that stores all entries in a local RocksDB database. @@ -36,10 +36,10 @@ public class RocksDBWindowStoreSupplier implements StateStoreSupplier { private final long retentionPeriod; private final boolean retainDuplicates; private final int numSegments; - private final Serdes serdes; + private final StateSerdes serdes; private final Time time; - public RocksDBWindowStoreSupplier(String name, long retentionPeriod, int numSegments, boolean retainDuplicates, Serdes serdes, Time time) { + public RocksDBWindowStoreSupplier(String name, long retentionPeriod, int numSegments, boolean retainDuplicates, StateSerdes serdes, Time time) { this.name = name; this.retentionPeriod = retentionPeriod; this.retainDuplicates = retainDuplicates; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java index 4229f940a00c6..a4391179c1696 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java @@ -22,7 +22,7 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.internals.ProcessorStateManager; import org.apache.kafka.streams.processor.internals.RecordCollector; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; import java.util.HashSet; import java.util.Set; @@ -36,7 +36,7 @@ public interface ValueGetter { // TODO: these values should be configurable protected static final int DEFAULT_WRITE_BATCH_SIZE = 100; - protected final Serdes serialization; + protected final StateSerdes serialization; private final String topic; private final int partition; @@ -47,16 +47,16 @@ public interface ValueGetter { protected Set dirty; protected Set removed; - public StoreChangeLogger(String storeName, ProcessorContext context, Serdes serialization) { + public StoreChangeLogger(String storeName, ProcessorContext context, StateSerdes serialization) { this(storeName, context, serialization, DEFAULT_WRITE_BATCH_SIZE, DEFAULT_WRITE_BATCH_SIZE); } - public StoreChangeLogger(String storeName, ProcessorContext context, Serdes serialization, int maxDirty, int maxRemoved) { + public StoreChangeLogger(String storeName, ProcessorContext context, StateSerdes serialization, int maxDirty, int maxRemoved) { this(storeName, context, context.taskId().partition, serialization, maxDirty, maxRemoved); init(); } - protected StoreChangeLogger(String storeName, ProcessorContext context, int partition, Serdes serialization, int maxDirty, int maxRemoved) { + protected StoreChangeLogger(String storeName, ProcessorContext context, int partition, StateSerdes serialization, int maxDirty, int maxRemoved) { this.topic = ProcessorStateManager.storeChangelogTopic(context.applicationId(), storeName); this.context = context; this.partition = partition; diff --git a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java index 83ebe48f1c1e3..0dacde79f50a8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/StreamsConfigTest.java @@ -18,11 +18,6 @@ package org.apache.kafka.streams; import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; -import org.apache.kafka.streams.processor.internals.WallclockTimestampExtractor; import org.apache.kafka.streams.processor.internals.StreamThread; import org.junit.Before; import org.junit.Test; @@ -43,11 +38,6 @@ public class StreamsConfigTest { public void setUp() { props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-config-test"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); - props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); - props.put(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, IntegerSerializer.class); - props.put(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class); - props.put(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, IntegerDeserializer.class); - props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, WallclockTimestampExtractor.class); streamsConfig = new StreamsConfig(props); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java index 88366fac3dd24..e04a2738d9fb4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.Predicate; @@ -34,9 +33,6 @@ public class KStreamBranchTest { private String topicName = "topic"; - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); - @SuppressWarnings("unchecked") @Test public void testKStreamBranch() { @@ -67,7 +63,7 @@ public boolean test(Integer key, String value) { KStream[] branches; MockProcessorSupplier[] processors; - stream = builder.stream(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName); branches = stream.branch(isEven, isMultipleOfThree, isOdd); assertEquals(3, branches.length); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java index 3bad0416152b1..ecf11153dd01d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.Predicate; @@ -33,9 +32,6 @@ public class KStreamFilterTest { private String topicName = "topic"; - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); - private Predicate isMultipleOfThree = new Predicate() { @Override public boolean test(Integer key, String value) { @@ -52,7 +48,7 @@ public void testFilter() { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream = builder.stream(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName); stream.filter(isMultipleOfThree).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); @@ -72,7 +68,7 @@ public void testFilterOut() { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream = builder.stream(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName); stream.filterOut(isMultipleOfThree).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java index 693f58e443f1c..bc85757ef450a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.KeyValue; @@ -35,9 +34,6 @@ public class KStreamFlatMapTest { private String topicName = "topic"; - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); - @Test public void testFlatMap() { KStreamBuilder builder = new KStreamBuilder(); @@ -60,7 +56,7 @@ public Iterable> apply(Integer key, String value) { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream = builder.stream(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName); stream.flatMap(mapper).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java index eef7933c627f8..a904cb15c6266 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java @@ -17,8 +17,7 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.ValueMapper; @@ -34,9 +33,6 @@ public class KStreamFlatMapValuesTest { private String topicName = "topic"; - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); - @Test public void testFlatMapValues() { KStreamBuilder builder = new KStreamBuilder(); @@ -58,7 +54,7 @@ public Iterable apply(String value) { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream = builder.stream(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName); stream.flatMapValues(mapper).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java index 3d3a9e3d6c4d2..38182bc355fa7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java @@ -17,12 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.JoinWindows; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; @@ -39,18 +35,16 @@ public class KStreamImplTest { + final private Serde stringSerde = Serdes.String(); + final private Serde intSerde = Serdes.Integer(); + @Test public void testNumProcesses() { - final Serializer stringSerializer = new StringSerializer(); - final Deserializer stringDeserializer = new StringDeserializer(); - final Serializer integerSerializer = new IntegerSerializer(); - final Deserializer integerDeserializer = new IntegerDeserializer(); - final KStreamBuilder builder = new KStreamBuilder(); - KStream source1 = builder.stream(stringDeserializer, stringDeserializer, "topic-1", "topic-2"); + KStream source1 = builder.stream(stringSerde, stringSerde, "topic-1", "topic-2"); - KStream source2 = builder.stream(stringDeserializer, stringDeserializer, "topic-3", "topic-4"); + KStream source2 = builder.stream(stringSerde, stringSerde, "topic-3", "topic-4"); KStream stream1 = source1.filter(new Predicate() { @@ -114,14 +108,14 @@ public boolean test(String key, Integer value) { public Integer apply(Integer value1, Integer value2) { return value1 + value2; } - }, JoinWindows.of("join-0"), stringSerializer, integerSerializer, integerSerializer, stringDeserializer, integerDeserializer, integerDeserializer); + }, JoinWindows.of("join-0"), stringSerde, intSerde, intSerde); KStream stream5 = streams2[1].join(streams3[1], new ValueJoiner() { @Override public Integer apply(Integer value1, Integer value2) { return value1 + value2; } - }, JoinWindows.of("join-1"), stringSerializer, integerSerializer, integerSerializer, stringDeserializer, integerDeserializer, integerDeserializer); + }, JoinWindows.of("join-1"), stringSerde, intSerde, intSerde); stream4.to("topic-5"); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java index e763fd25a41d9..d24ab15461c44 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java @@ -17,10 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.JoinWindows; import org.apache.kafka.streams.kstream.KStream; @@ -44,10 +42,8 @@ public class KStreamKStreamJoinTest { private String topic1 = "topic1"; private String topic2 = "topic2"; - private IntegerSerializer keySerializer = new IntegerSerializer(); - private StringSerializer valSerializer = new StringSerializer(); - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); + final private Serde intSerde = Serdes.Integer(); + final private Serde stringSerde = Serdes.String(); private ValueJoiner joiner = new ValueJoiner() { @Override @@ -71,10 +67,9 @@ public void testJoin() throws Exception { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream1 = builder.stream(keyDeserializer, valDeserializer, topic1); - stream2 = builder.stream(keyDeserializer, valDeserializer, topic2); - joined = stream1.join(stream2, joiner, JoinWindows.of("test").within(100), - keySerializer, valSerializer, valSerializer, keyDeserializer, valDeserializer, valDeserializer); + stream1 = builder.stream(intSerde, stringSerde, topic1); + stream2 = builder.stream(intSerde, stringSerde, topic2); + joined = stream1.join(stream2, joiner, JoinWindows.of("test").within(100), intSerde, stringSerde, stringSerde); joined.process(processor); Collection> copartitionGroups = builder.copartitionGroups(); @@ -177,10 +172,9 @@ public void testOuterJoin() throws Exception { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream1 = builder.stream(keyDeserializer, valDeserializer, topic1); - stream2 = builder.stream(keyDeserializer, valDeserializer, topic2); - joined = stream1.outerJoin(stream2, joiner, JoinWindows.of("test").within(100), - keySerializer, valSerializer, valSerializer, keyDeserializer, valDeserializer, valDeserializer); + stream1 = builder.stream(intSerde, stringSerde, topic1); + stream2 = builder.stream(intSerde, stringSerde, topic2); + joined = stream1.outerJoin(stream2, joiner, JoinWindows.of("test").within(100), intSerde, stringSerde, stringSerde); joined.process(processor); Collection> copartitionGroups = builder.copartitionGroups(); @@ -285,10 +279,9 @@ public void testWindowing() throws Exception { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream1 = builder.stream(keyDeserializer, valDeserializer, topic1); - stream2 = builder.stream(keyDeserializer, valDeserializer, topic2); - joined = stream1.join(stream2, joiner, JoinWindows.of("test").within(100), - keySerializer, valSerializer, valSerializer, keyDeserializer, valDeserializer, valDeserializer); + stream1 = builder.stream(intSerde, stringSerde, topic1); + stream2 = builder.stream(intSerde, stringSerde, topic2); + joined = stream1.join(stream2, joiner, JoinWindows.of("test").within(100), intSerde, stringSerde, stringSerde); joined.process(processor); Collection> copartitionGroups = builder.copartitionGroups(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java index 8c6e43b2eae07..166e8ba038571 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java @@ -17,10 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.JoinWindows; import org.apache.kafka.streams.kstream.KStream; @@ -44,10 +42,8 @@ public class KStreamKStreamLeftJoinTest { private String topic1 = "topic1"; private String topic2 = "topic2"; - private IntegerSerializer keySerializer = new IntegerSerializer(); - private StringSerializer valSerializer = new StringSerializer(); - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); + final private Serde intSerde = Serdes.Integer(); + final private Serde stringSerde = Serdes.String(); private ValueJoiner joiner = new ValueJoiner() { @Override @@ -71,10 +67,9 @@ public void testLeftJoin() throws Exception { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream1 = builder.stream(keyDeserializer, valDeserializer, topic1); - stream2 = builder.stream(keyDeserializer, valDeserializer, topic2); - joined = stream1.leftJoin(stream2, joiner, JoinWindows.of("test").within(100), - keySerializer, valSerializer, keyDeserializer, valDeserializer); + stream1 = builder.stream(intSerde, stringSerde, topic1); + stream2 = builder.stream(intSerde, stringSerde, topic2); + joined = stream1.leftJoin(stream2, joiner, JoinWindows.of("test").within(100), intSerde, stringSerde); joined.process(processor); Collection> copartitionGroups = builder.copartitionGroups(); @@ -157,10 +152,9 @@ public void testWindowing() throws Exception { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream1 = builder.stream(keyDeserializer, valDeserializer, topic1); - stream2 = builder.stream(keyDeserializer, valDeserializer, topic2); - joined = stream1.leftJoin(stream2, joiner, JoinWindows.of("test").within(100), - keySerializer, valSerializer, keyDeserializer, valDeserializer); + stream1 = builder.stream(intSerde, stringSerde, topic1); + stream2 = builder.stream(intSerde, stringSerde, topic2); + joined = stream1.leftJoin(stream2, joiner, JoinWindows.of("test").within(100), intSerde, stringSerde); joined.process(processor); Collection> copartitionGroups = builder.copartitionGroups(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java index f226ceee2855e..8e672a2311d30 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java @@ -18,10 +18,8 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; @@ -47,10 +45,8 @@ public class KStreamKTableLeftJoinTest { private String topic1 = "topic1"; private String topic2 = "topic2"; - private IntegerSerializer keySerializer = new IntegerSerializer(); - private StringSerializer valSerializer = new StringSerializer(); - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); + final private Serde intSerde = new Serdes.IntegerSerde(); + final private Serde stringSerde = new Serdes.StringSerde(); private ValueJoiner joiner = new ValueJoiner() { @Override @@ -81,8 +77,8 @@ public void testJoin() throws Exception { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream = builder.stream(keyDeserializer, valDeserializer, topic1); - table = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); + stream = builder.stream(intSerde, stringSerde, topic1); + table = builder.table(intSerde, stringSerde, topic2); stream.leftJoin(table, joiner).process(processor); Collection> copartitionGroups = builder.copartitionGroups(); @@ -162,8 +158,8 @@ public void testNotJoinable() { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream = builder.stream(keyDeserializer, valDeserializer, topic1).map(keyValueMapper); - table = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); + stream = builder.stream(intSerde, stringSerde, topic1).map(keyValueMapper); + table = builder.table(intSerde, stringSerde, topic2); stream.leftJoin(table, joiner).process(processor); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java index 73c517bc65818..68fa6565d50dc 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java @@ -17,8 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.KeyValue; @@ -33,8 +33,8 @@ public class KStreamMapTest { private String topicName = "topic"; - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); + final private Serde intSerde = Serdes.Integer(); + final private Serde stringSerde = Serdes.String(); @Test public void testMap() { @@ -50,11 +50,10 @@ public KeyValue apply(Integer key, String value) { final int[] expectedKeys = new int[]{0, 1, 2, 3}; - KStream stream; + KStream stream = builder.stream(intSerde, stringSerde, topicName); MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream = builder.stream(keyDeserializer, valDeserializer, topicName); stream.map(mapper).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java index 68fd285870f71..e671aabce392b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java @@ -17,8 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.StringDeserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.ValueMapper; @@ -32,8 +32,8 @@ public class KStreamMapValuesTest { private String topicName = "topic"; - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); + final private Serde intSerde = Serdes.Integer(); + final private Serde stringSerde = Serdes.String(); @Test public void testFlatMapValues() { @@ -51,7 +51,7 @@ public Integer apply(String value) { KStream stream; MockProcessorSupplier processor = new MockProcessorSupplier<>(); - stream = builder.stream(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(intSerde, stringSerde, topicName); stream.mapValues(mapper).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java index 426259f689dbe..4244de541640c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java @@ -17,7 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.KeyValue; @@ -34,8 +35,7 @@ public class KStreamTransformTest { private String topicName = "topic"; - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private IntegerDeserializer valDeserializer = new IntegerDeserializer(); + final private Serde intSerde = Serdes.Integer(); @Test public void testTransform() { @@ -71,9 +71,8 @@ public void close() { final int[] expectedKeys = {1, 10, 100, 1000}; - KStream stream; MockProcessorSupplier processor = new MockProcessorSupplier<>(); - stream = builder.stream(keyDeserializer, valDeserializer, topicName); + KStream stream = builder.stream(intSerde, intSerde, topicName); stream.transform(transformerSupplier).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java index 7def9db45bca0..52abdf728c93e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java @@ -17,7 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.ValueTransformer; @@ -33,8 +34,7 @@ public class KStreamTransformValuesTest { private String topicName = "topic"; - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private IntegerDeserializer valDeserializer = new IntegerDeserializer(); + final private Serde intSerde = Serdes.Integer(); @Test public void testTransform() { @@ -72,7 +72,7 @@ public void close() { KStream stream; MockProcessorSupplier processor = new MockProcessorSupplier<>(); - stream = builder.stream(keyDeserializer, valDeserializer, topicName); + stream = builder.stream(intSerde, intSerde, topicName); stream.transformValues(valueTransformerSupplier).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java index 9e0745a374c4a..e19510f679307 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java @@ -17,10 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.HoppingWindows; @@ -41,8 +39,7 @@ public class KStreamWindowAggregateTest { - private final Serializer strSerializer = new StringSerializer(); - private final Deserializer strDeserializer = new StringDeserializer(); + final private Serde strSerde = new Serdes.StringSerde(); private class StringAdd implements Aggregator { @@ -68,13 +65,11 @@ public void testAggBasic() throws Exception { final KStreamBuilder builder = new KStreamBuilder(); String topic1 = "topic1"; - KStream stream1 = builder.stream(strDeserializer, strDeserializer, topic1); + KStream stream1 = builder.stream(strSerde, strSerde, topic1); KTable, String> table2 = stream1.aggregateByKey(new StringInit(), new StringAdd(), HoppingWindows.of("topic1-Canonized").with(10L).every(5L), - strSerializer, - strSerializer, - strDeserializer, - strDeserializer); + strSerde, + strSerde); MockProcessorSupplier, String> proc2 = new MockProcessorSupplier<>(); table2.toStream().process(proc2); @@ -147,24 +142,20 @@ public void testJoin() throws Exception { String topic1 = "topic1"; String topic2 = "topic2"; - KStream stream1 = builder.stream(strDeserializer, strDeserializer, topic1); + KStream stream1 = builder.stream(strSerde, strSerde, topic1); KTable, String> table1 = stream1.aggregateByKey(new StringInit(), new StringAdd(), HoppingWindows.of("topic1-Canonized").with(10L).every(5L), - strSerializer, - strSerializer, - strDeserializer, - strDeserializer); + strSerde, + strSerde); MockProcessorSupplier, String> proc1 = new MockProcessorSupplier<>(); table1.toStream().process(proc1); - KStream stream2 = builder.stream(strDeserializer, strDeserializer, topic2); + KStream stream2 = builder.stream(strSerde, strSerde, topic2); KTable, String> table2 = stream2.aggregateByKey(new StringInit(), new StringAdd(), HoppingWindows.of("topic2-Canonized").with(10L).every(5L), - strSerializer, - strSerializer, - strDeserializer, - strDeserializer); + strSerde, + strSerde); MockProcessorSupplier, String> proc2 = new MockProcessorSupplier<>(); table2.toStream().process(proc2); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java index ec85ed7546265..fc01e5e7f6004 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java @@ -17,10 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Initializer; @@ -38,8 +36,7 @@ public class KTableAggregateTest { - private final Serializer strSerializer = new StringSerializer(); - private final Deserializer strDeserializer = new StringDeserializer(); + final private Serde stringSerde = new Serdes.StringSerde(); private class StringAdd implements Aggregator { @@ -74,15 +71,12 @@ public void testAggBasic() throws Exception { final KStreamBuilder builder = new KStreamBuilder(); String topic1 = "topic1"; - KTable table1 = builder.table(strSerializer, strSerializer, strDeserializer, strDeserializer, topic1); - KTable table2 = table1.aggregate(new StringInit(), new StringAdd(), new StringRemove(), + KTable table1 = builder.table(stringSerde, stringSerde, topic1); + KTable table2 = table1.aggregate(new StringInit(), new StringAdd(), new StringRemove(), new NoOpKeyValueMapper(), - strSerializer, - strSerializer, - strSerializer, - strDeserializer, - strDeserializer, - strDeserializer, + stringSerde, + stringSerde, + stringSerde, "topic1-Canonized"); MockProcessorSupplier proc2 = new MockProcessorSupplier<>(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java index c43bea0c1860a..5491ea316e8ca 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java @@ -17,12 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; @@ -40,10 +36,8 @@ public class KTableFilterTest { - private final Serializer strSerializer = new StringSerializer(); - private final Deserializer strDeserializer = new StringDeserializer(); - private final Serializer intSerializer = new IntegerSerializer(); - private final Deserializer intDeserializer = new IntegerDeserializer(); + final private Serde intSerde = new Serdes.IntegerSerde(); + final private Serde stringSerde = new Serdes.StringSerde(); @Test public void testKTable() { @@ -51,7 +45,7 @@ public void testKTable() { String topic1 = "topic1"; - KTable table1 = builder.table(strSerializer, intSerializer, strDeserializer, intDeserializer, topic1); + KTable table1 = builder.table(stringSerde, intSerde, topic1); KTable table2 = table1.filter(new Predicate() { @Override @@ -93,7 +87,7 @@ public void testValueGetter() throws IOException { String topic1 = "topic1"; KTableImpl table1 = - (KTableImpl) builder.table(strSerializer, intSerializer, strDeserializer, intDeserializer, topic1); + (KTableImpl) builder.table(stringSerde, intSerde, topic1); KTableImpl table2 = (KTableImpl) table1.filter( new Predicate() { @Override @@ -112,7 +106,7 @@ public boolean test(String key, Integer value) { KTableValueGetterSupplier getterSupplier2 = table2.valueGetterSupplier(); KTableValueGetterSupplier getterSupplier3 = table3.valueGetterSupplier(); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); KTableValueGetter getter2 = getterSupplier2.get(); KTableValueGetter getter3 = getterSupplier3.get(); @@ -178,7 +172,7 @@ public void testNotSendingOldValue() throws IOException { String topic1 = "topic1"; KTableImpl table1 = - (KTableImpl) builder.table(strSerializer, intSerializer, strDeserializer, intDeserializer, topic1); + (KTableImpl) builder.table(stringSerde, intSerde, topic1); KTableImpl table2 = (KTableImpl) table1.filter( new Predicate() { @Override @@ -193,7 +187,7 @@ public boolean test(String key, Integer value) { builder.addProcessor("proc1", proc1, table1.name); builder.addProcessor("proc2", proc2, table2.name); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); driver.process(topic1, "A", 1); driver.process(topic1, "B", 1); @@ -233,7 +227,7 @@ public void testSendingOldValue() throws IOException { String topic1 = "topic1"; KTableImpl table1 = - (KTableImpl) builder.table(strSerializer, intSerializer, strDeserializer, intDeserializer, topic1); + (KTableImpl) builder.table(stringSerde, intSerde, topic1); KTableImpl table2 = (KTableImpl) table1.filter( new Predicate() { @Override @@ -250,7 +244,7 @@ public boolean test(String key, Integer value) { builder.addProcessor("proc1", proc1, table1.name); builder.addProcessor("proc2", proc2, table2.name); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); driver.process(topic1, "A", 1); driver.process(topic1, "B", 1); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java index 2317c977cf4c2..20c3a287cab14 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java @@ -17,10 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; @@ -40,16 +38,16 @@ public class KTableImplTest { + final private Serde stringSerde = new Serdes.StringSerde(); + @Test public void testKTable() { - final Serializer serializer = new StringSerializer(); - final Deserializer deserializer = new StringDeserializer(); final KStreamBuilder builder = new KStreamBuilder(); String topic1 = "topic1"; String topic2 = "topic2"; - KTable table1 = builder.table(serializer, serializer, deserializer, deserializer, topic1); + KTable table1 = builder.table(stringSerde, stringSerde, topic1); MockProcessorSupplier proc1 = new MockProcessorSupplier<>(); table1.toStream().process(proc1); @@ -74,7 +72,7 @@ public boolean test(String key, Integer value) { MockProcessorSupplier proc3 = new MockProcessorSupplier<>(); table3.toStream().process(proc3); - KTable table4 = table1.through(topic2, serializer, serializer, deserializer, deserializer); + KTable table4 = table1.through(topic2, stringSerde, stringSerde); MockProcessorSupplier proc4 = new MockProcessorSupplier<>(); table4.toStream().process(proc4); @@ -96,15 +94,13 @@ public boolean test(String key, Integer value) { public void testValueGetter() throws IOException { File stateDir = Files.createTempDirectory("test").toFile(); try { - final Serializer serializer = new StringSerializer(); - final Deserializer deserializer = new StringDeserializer(); final KStreamBuilder builder = new KStreamBuilder(); String topic1 = "topic1"; String topic2 = "topic2"; KTableImpl table1 = - (KTableImpl) builder.table(serializer, serializer, deserializer, deserializer, topic1); + (KTableImpl) builder.table(stringSerde, stringSerde, topic1); KTableImpl table2 = (KTableImpl) table1.mapValues( new ValueMapper() { @Override @@ -120,14 +116,14 @@ public boolean test(String key, Integer value) { } }); KTableImpl table4 = (KTableImpl) - table1.through(topic2, serializer, serializer, deserializer, deserializer); + table1.through(topic2, stringSerde, stringSerde); KTableValueGetterSupplier getterSupplier1 = table1.valueGetterSupplier(); KTableValueGetterSupplier getterSupplier2 = table2.valueGetterSupplier(); KTableValueGetterSupplier getterSupplier3 = table3.valueGetterSupplier(); KTableValueGetterSupplier getterSupplier4 = table4.valueGetterSupplier(); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); // two state store should be created assertEquals(2, driver.allStateStores().size()); @@ -223,9 +219,6 @@ public boolean test(String key, Integer value) { @Test public void testStateStore() throws IOException { - final Serializer serializer = new StringSerializer(); - final Deserializer deserializer = new StringDeserializer(); - String topic1 = "topic1"; String topic2 = "topic2"; @@ -234,9 +227,9 @@ public void testStateStore() throws IOException { KStreamBuilder builder = new KStreamBuilder(); KTableImpl table1 = - (KTableImpl) builder.table(serializer, serializer, deserializer, deserializer, topic1); + (KTableImpl) builder.table(stringSerde, stringSerde, topic1); KTableImpl table2 = - (KTableImpl) builder.table(serializer, serializer, deserializer, deserializer, topic2); + (KTableImpl) builder.table(stringSerde, stringSerde, topic2); KTableImpl table1Mapped = (KTableImpl) table1.mapValues( new ValueMapper() { @@ -253,7 +246,7 @@ public boolean test(String key, Integer value) { } }); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); driver.setTime(0L); // no state store should be created @@ -267,9 +260,9 @@ public boolean test(String key, Integer value) { KStreamBuilder builder = new KStreamBuilder(); KTableImpl table1 = - (KTableImpl) builder.table(serializer, serializer, deserializer, deserializer, topic1); + (KTableImpl) builder.table(stringSerde, stringSerde, topic1); KTableImpl table2 = - (KTableImpl) builder.table(serializer, serializer, deserializer, deserializer, topic2); + (KTableImpl) builder.table(stringSerde, stringSerde, topic2); KTableImpl table1Mapped = (KTableImpl) table1.mapValues( new ValueMapper() { @@ -293,7 +286,7 @@ public String apply(String v1, Integer v2) { } }); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); driver.setTime(0L); // two state store should be created diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java index 12bfb9cd7691d..5f30574cd5a10 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java @@ -17,10 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; @@ -47,10 +45,8 @@ public class KTableKTableJoinTest { private String topic1 = "topic1"; private String topic2 = "topic2"; - private IntegerSerializer keySerializer = new IntegerSerializer(); - private StringSerializer valSerializer = new StringSerializer(); - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); + final private Serde intSerde = new Serdes.IntegerSerde(); + final private Serde stringSerde = new Serdes.StringSerde(); private ValueJoiner joiner = new ValueJoiner() { @Override @@ -80,8 +76,8 @@ public void testJoin() throws Exception { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - table1 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic1); - table2 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); joined = table1.join(table2, joiner); joined.toStream().process(processor); @@ -179,8 +175,8 @@ public void testNotSendingOldValues() throws Exception { KTable joined; MockProcessorSupplier proc; - table1 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic1); - table2 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); joined = table1.join(table2, joiner); proc = new MockProcessorSupplier<>(); @@ -267,8 +263,8 @@ public void testSendingOldValues() throws Exception { KTable joined; MockProcessorSupplier proc; - table1 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic1); - table2 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); joined = table1.join(table2, joiner); ((KTableImpl) joined).enableSendingOldValues(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java index e3cf22b97a575..f92c5ca81414f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java @@ -17,10 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; @@ -48,10 +46,8 @@ public class KTableKTableLeftJoinTest { private String topic1 = "topic1"; private String topic2 = "topic2"; - private IntegerSerializer keySerializer = new IntegerSerializer(); - private StringSerializer valSerializer = new StringSerializer(); - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); + final private Serde intSerde = new Serdes.IntegerSerde(); + final private Serde stringSerde = new Serdes.StringSerde(); private ValueJoiner joiner = new ValueJoiner() { @Override @@ -83,15 +79,11 @@ public void testJoin() throws Exception { final int[] expectedKeys = new int[]{0, 1, 2, 3}; - KTable table1; - KTable table2; - KTable joined; + KTable table1 = builder.table(intSerde, stringSerde, topic1); + KTable table2 = builder.table(intSerde, stringSerde, topic2); + KTable joined = table1.leftJoin(table2, joiner); MockProcessorSupplier processor; - processor = new MockProcessorSupplier<>(); - table1 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic1); - table2 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); - joined = table1.leftJoin(table2, joiner); joined.toStream().process(processor); Collection> copartitionGroups = builder.copartitionGroups(); @@ -188,8 +180,8 @@ public void testNotSendingOldValue() throws Exception { KTable joined; MockProcessorSupplier proc; - table1 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic1); - table2 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); joined = table1.leftJoin(table2, joiner); proc = new MockProcessorSupplier<>(); @@ -276,8 +268,8 @@ public void testSendingOldValue() throws Exception { KTable joined; MockProcessorSupplier proc; - table1 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic1); - table2 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); joined = table1.leftJoin(table2, joiner); ((KTableImpl) joined).enableSendingOldValues(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java index feabc08612b81..6cc77e0b1b0c6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java @@ -17,10 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; @@ -47,10 +45,8 @@ public class KTableKTableOuterJoinTest { private String topic1 = "topic1"; private String topic2 = "topic2"; - private IntegerSerializer keySerializer = new IntegerSerializer(); - private StringSerializer valSerializer = new StringSerializer(); - private IntegerDeserializer keyDeserializer = new IntegerDeserializer(); - private StringDeserializer valDeserializer = new StringDeserializer(); + final private Serde intSerde = new Serdes.IntegerSerde(); + final private Serde stringSerde = new Serdes.StringSerde(); private ValueJoiner joiner = new ValueJoiner() { @Override @@ -80,8 +76,8 @@ public void testJoin() throws Exception { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - table1 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic1); - table2 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); joined = table1.outerJoin(table2, joiner); joined.toStream().process(processor); @@ -188,8 +184,8 @@ public void testNotSendingOldValue() throws Exception { KTable joined; MockProcessorSupplier proc; - table1 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic1); - table2 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); joined = table1.outerJoin(table2, joiner); proc = new MockProcessorSupplier<>(); @@ -284,8 +280,8 @@ public void testSendingOldValue() throws Exception { KTable joined; MockProcessorSupplier proc; - table1 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic1); - table2 = builder.table(keySerializer, valSerializer, keyDeserializer, valDeserializer, topic2); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); joined = table1.outerJoin(table2, joiner); ((KTableImpl) joined).enableSendingOldValues(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java index 58f1c2a127d0e..aa3daeb8de574 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java @@ -17,10 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; @@ -41,8 +39,7 @@ public class KTableMapValuesTest { - private final Serializer strSerializer = new StringSerializer(); - private final Deserializer strDeserializer = new StringDeserializer(); + final private Serde stringSerde = new Serdes.StringSerde(); @Test public void testKTable() { @@ -50,7 +47,7 @@ public void testKTable() { String topic1 = "topic1"; - KTable table1 = builder.table(strSerializer, strSerializer, strDeserializer, strDeserializer, topic1); + KTable table1 = builder.table(stringSerde, stringSerde, topic1); KTable table2 = table1.mapValues(new ValueMapper() { @Override public Integer apply(String value) { @@ -75,15 +72,13 @@ public Integer apply(String value) { public void testValueGetter() throws IOException { File stateDir = Files.createTempDirectory("test").toFile(); try { - final Serializer serializer = new StringSerializer(); - final Deserializer deserializer = new StringDeserializer(); final KStreamBuilder builder = new KStreamBuilder(); String topic1 = "topic1"; String topic2 = "topic2"; KTableImpl table1 = - (KTableImpl) builder.table(serializer, serializer, deserializer, deserializer, topic1); + (KTableImpl) builder.table(stringSerde, stringSerde, topic1); KTableImpl table2 = (KTableImpl) table1.mapValues( new ValueMapper() { @Override @@ -99,14 +94,14 @@ public boolean test(String key, Integer value) { } }); KTableImpl table4 = (KTableImpl) - table1.through(topic2, serializer, serializer, deserializer, deserializer); + table1.through(topic2, stringSerde, stringSerde); KTableValueGetterSupplier getterSupplier1 = table1.valueGetterSupplier(); KTableValueGetterSupplier getterSupplier2 = table2.valueGetterSupplier(); KTableValueGetterSupplier getterSupplier3 = table3.valueGetterSupplier(); KTableValueGetterSupplier getterSupplier4 = table4.valueGetterSupplier(); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); KTableValueGetter getter1 = getterSupplier1.get(); getter1.init(driver.context()); @@ -201,14 +196,12 @@ public boolean test(String key, Integer value) { public void testNotSendingOldValue() throws IOException { File stateDir = Files.createTempDirectory("test").toFile(); try { - final Serializer serializer = new StringSerializer(); - final Deserializer deserializer = new StringDeserializer(); final KStreamBuilder builder = new KStreamBuilder(); String topic1 = "topic1"; KTableImpl table1 = - (KTableImpl) builder.table(serializer, serializer, deserializer, deserializer, topic1); + (KTableImpl) builder.table(stringSerde, stringSerde, topic1); KTableImpl table2 = (KTableImpl) table1.mapValues( new ValueMapper() { @Override @@ -221,7 +214,7 @@ public Integer apply(String value) { builder.addProcessor("proc", proc, table2.name); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); assertFalse(table1.sendingOldValueEnabled()); assertFalse(table2.sendingOldValueEnabled()); @@ -254,14 +247,12 @@ public Integer apply(String value) { public void testSendingOldValue() throws IOException { File stateDir = Files.createTempDirectory("test").toFile(); try { - final Serializer serializer = new StringSerializer(); - final Deserializer deserializer = new StringDeserializer(); final KStreamBuilder builder = new KStreamBuilder(); String topic1 = "topic1"; KTableImpl table1 = - (KTableImpl) builder.table(serializer, serializer, deserializer, deserializer, topic1); + (KTableImpl) builder.table(stringSerde, stringSerde, topic1); KTableImpl table2 = (KTableImpl) table1.mapValues( new ValueMapper() { @Override @@ -276,7 +267,7 @@ public Integer apply(String value) { builder.addProcessor("proc", proc, table2.name); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); assertTrue(table1.sendingOldValueEnabled()); assertTrue(table2.sendingOldValueEnabled()); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java index 187a6f2fe2e50..51276f3228271 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java @@ -17,10 +17,8 @@ package org.apache.kafka.streams.kstream.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; @@ -38,8 +36,7 @@ public class KTableSourceTest { - private final Serializer strSerializer = new StringSerializer(); - private final Deserializer strDeserializer = new StringDeserializer(); + final private Serde stringSerde = new Serdes.StringSerde(); @Test public void testKTable() { @@ -47,7 +44,7 @@ public void testKTable() { String topic1 = "topic1"; - KTable table1 = builder.table(strSerializer, strSerializer, strDeserializer, strDeserializer, topic1); + KTable table1 = builder.table(stringSerde, stringSerde, topic1); MockProcessorSupplier proc1 = new MockProcessorSupplier<>(); table1.toStream().process(proc1); @@ -72,12 +69,11 @@ public void testValueGetter() throws IOException { String topic1 = "topic1"; - KTableImpl table1 = (KTableImpl) - builder.table(strSerializer, strSerializer, strDeserializer, strDeserializer, topic1); + KTableImpl table1 = (KTableImpl) builder.table(stringSerde, stringSerde, topic1); KTableValueGetterSupplier getterSupplier1 = table1.valueGetterSupplier(); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); KTableValueGetter getter1 = getterSupplier1.get(); getter1.init(driver.context()); @@ -123,14 +119,13 @@ public void testNotSedingOldValue() throws IOException { String topic1 = "topic1"; - KTableImpl table1 = (KTableImpl) - builder.table(strSerializer, strSerializer, strDeserializer, strDeserializer, topic1); + KTableImpl table1 = (KTableImpl) builder.table(stringSerde, stringSerde, topic1); MockProcessorSupplier proc1 = new MockProcessorSupplier<>(); builder.addProcessor("proc1", proc1, table1.name); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); driver.process(topic1, "A", "01"); driver.process(topic1, "B", "01"); @@ -165,8 +160,7 @@ public void testSedingOldValue() throws IOException { String topic1 = "topic1"; - KTableImpl table1 = (KTableImpl) - builder.table(strSerializer, strSerializer, strDeserializer, strDeserializer, topic1); + KTableImpl table1 = (KTableImpl) builder.table(stringSerde, stringSerde, topic1); table1.enableSendingOldValues(); @@ -176,7 +170,7 @@ public void testSedingOldValue() throws IOException { builder.addProcessor("proc1", proc1, table1.name); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null, null, null); + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); driver.process(topic1, "A", "01"); driver.process(topic1, "B", "01"); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java index 1b8cbb8a449c4..7c6d5ec0fd2a2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java @@ -37,8 +37,8 @@ public class WindowedStreamPartitionerTest { private String topicName = "topic"; - private IntegerSerializer keySerializer = new IntegerSerializer(); - private StringSerializer valSerializer = new StringSerializer(); + private IntegerSerializer intSerializer = new IntegerSerializer(); + private StringSerializer stringSerializer = new StringSerializer(); private List infos = Arrays.asList( new PartitionInfo(topicName, 0, Node.noNode(), new Node[0], new Node[0]), @@ -58,15 +58,15 @@ public void testCopartitioning() { DefaultPartitioner defaultPartitioner = new DefaultPartitioner(); - WindowedSerializer windowedSerializer = new WindowedSerializer<>(keySerializer); + WindowedSerializer windowedSerializer = new WindowedSerializer<>(intSerializer); WindowedStreamPartitioner streamPartitioner = new WindowedStreamPartitioner<>(windowedSerializer); for (int k = 0; k < 10; k++) { Integer key = rand.nextInt(); - byte[] keyBytes = keySerializer.serialize(topicName, key); + byte[] keyBytes = intSerializer.serialize(topicName, key); String value = key.toString(); - byte[] valueBytes = valSerializer.serialize(topicName, value); + byte[] valueBytes = stringSerializer.serialize(topicName, value); Integer expected = defaultPartitioner.partition("topic", key, keyBytes, value, valueBytes, cluster); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java index 12210cca8673a..ef081761eb4e4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java @@ -24,6 +24,7 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.serialization.StringDeserializer; import org.apache.kafka.common.serialization.StringSerializer; @@ -70,11 +71,9 @@ public void setup() { props.setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "processor-topology-test"); props.setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9091"); props.setProperty(StreamsConfig.STATE_DIR_CONFIG, localState.getAbsolutePath()); + props.setProperty(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); + props.setProperty(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); props.setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, CustomTimestampExtractor.class.getName()); - props.setProperty(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); - props.setProperty(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); - props.setProperty(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class.getName()); - props.setProperty(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); this.config = new StreamsConfig(props); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java index 21bdaffbd8edf..ea24441643e07 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java @@ -33,6 +33,7 @@ import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.state.internals.OffsetCheckpoint; import org.apache.kafka.test.MockStateStoreSupplier; +import org.apache.kafka.test.MockTimestampExtractor; import org.junit.Before; import org.junit.Test; @@ -89,15 +90,11 @@ public class StandbyTaskTest { private StreamsConfig createConfig(final File baseDir) throws Exception { return new StreamsConfig(new Properties() { { - setProperty(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - setProperty(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); - setProperty(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - setProperty(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); - setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, "org.apache.kafka.test.MockTimestampExtractor"); setProperty(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"); setProperty(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"); setProperty(StreamsConfig.STATE_DIR_CONFIG, baseDir.getCanonicalPath()); + setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()); } }); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java index a5990bdcf692d..be851bff67b9a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java @@ -36,6 +36,7 @@ import org.apache.kafka.streams.processor.internals.assignment.SubscriptionInfo; import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.MockStateStoreSupplier; +import org.apache.kafka.test.MockTimestampExtractor; import org.junit.Test; import java.util.ArrayList; @@ -89,14 +90,10 @@ public class StreamPartitionAssignorTest { private Properties configProps() { return new Properties() { { - setProperty(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - setProperty(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); - setProperty(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - setProperty(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); - setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, "org.apache.kafka.test.MockTimestampExtractor"); setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "stream-partition-assignor-test"); setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"); setProperty(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"); + setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()); } }; } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index f2ade6ba8c15b..33fa5c4a091a7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -33,6 +33,7 @@ import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.test.MockSourceNode; +import org.apache.kafka.test.MockTimestampExtractor; import org.junit.Test; import org.junit.Before; @@ -73,15 +74,11 @@ public class StreamTaskTest { private StreamsConfig createConfig(final File baseDir) throws Exception { return new StreamsConfig(new Properties() { { - setProperty(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - setProperty(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); - setProperty(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - setProperty(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); - setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, "org.apache.kafka.test.MockTimestampExtractor"); setProperty(StreamsConfig.APPLICATION_ID_CONFIG, "stream-task-test"); setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"); setProperty(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"); setProperty(StreamsConfig.STATE_DIR_CONFIG, baseDir.getCanonicalPath()); + setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()); } }); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java index b201c078dec57..e387a59d68e7a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamThreadTest.java @@ -41,6 +41,7 @@ import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.TopologyBuilder; import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockTimestampExtractor; import org.junit.Test; import java.io.File; @@ -113,14 +114,10 @@ private ByteBuffer subscriptionUserData() { private Properties configProps() { return new Properties() { { - setProperty(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - setProperty(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); - setProperty(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArraySerializer"); - setProperty(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.ByteArrayDeserializer"); - setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, "org.apache.kafka.test.MockTimestampExtractor"); setProperty(StreamsConfig.APPLICATION_ID_CONFIG, applicationId); setProperty(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:2171"); setProperty(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, "3"); + setProperty(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class.getName()); } }; } diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java index 063eafeb04f5d..ce4956c1453a4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java +++ b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java @@ -18,8 +18,6 @@ package org.apache.kafka.streams.smoketest; import org.apache.kafka.clients.consumer.ConsumerConfig; -import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; @@ -85,11 +83,7 @@ private static KafkaStreams createKafkaStreams(File stateDir, String kafka, Stri props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString()); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, zookeeper); - props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TestTimestampExtractor.class); - props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); - props.put(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); - props.put(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class); - props.put(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class); + props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, TestTimestampExtractor.class.getName()); props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 3); props.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, 2); props.put(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG, 100); @@ -98,9 +92,9 @@ private static KafkaStreams createKafkaStreams(File stateDir, String kafka, Stri KStreamBuilder builder = new KStreamBuilder(); - KStream source = builder.stream(stringDeserializer, integerDeserializer, "data"); + KStream source = builder.stream(stringSerde, intSerde, "data"); - source.to("echo", stringSerializer, integerSerializer); + source.to("echo", stringSerde, intSerde); KStream data = source.filter(new Predicate() { @Override @@ -125,15 +119,13 @@ public Integer apply(String aggKey, Integer value, Integer aggregate) { } }, UnlimitedWindows.of("uwin-min"), - stringSerializer, - integerSerializer, - stringDeserializer, - integerDeserializer + stringSerde, + intSerde ).toStream().map( new Unwindow() - ).to("min", stringSerializer, integerSerializer); + ).to("min", stringSerde, intSerde); - KTable minTable = builder.table(stringSerializer, integerSerializer, stringDeserializer, integerDeserializer, "min"); + KTable minTable = builder.table(stringSerde, intSerde, "min"); minTable.toStream().process(SmokeTestUtil.printProcessorSupplier("min")); // max @@ -150,15 +142,13 @@ public Integer apply(String aggKey, Integer value, Integer aggregate) { } }, UnlimitedWindows.of("uwin-max"), - stringSerializer, - integerSerializer, - stringDeserializer, - integerDeserializer + stringSerde, + intSerde ).toStream().map( new Unwindow() - ).to("max", stringSerializer, integerSerializer); + ).to("max", stringSerde, intSerde); - KTable maxTable = builder.table(stringSerializer, integerSerializer, stringDeserializer, integerDeserializer, "max"); + KTable maxTable = builder.table(stringSerde, intSerde, "max"); maxTable.toStream().process(SmokeTestUtil.printProcessorSupplier("max")); // sum @@ -175,28 +165,25 @@ public Long apply(String aggKey, Integer value, Long aggregate) { } }, UnlimitedWindows.of("win-sum"), - stringSerializer, - longSerializer, - stringDeserializer, - longDeserializer + stringSerde, + longSerde ).toStream().map( new Unwindow() - ).to("sum", stringSerializer, longSerializer); + ).to("sum", stringSerde, longSerde); - KTable sumTable = builder.table(stringSerializer, longSerializer, stringDeserializer, longDeserializer, "sum"); + KTable sumTable = builder.table(stringSerde, longSerde, "sum"); sumTable.toStream().process(SmokeTestUtil.printProcessorSupplier("sum")); // cnt data.countByKey( UnlimitedWindows.of("uwin-cnt"), - stringSerializer, - stringDeserializer + stringSerde ).toStream().map( new Unwindow() - ).to("cnt", stringSerializer, longSerializer); + ).to("cnt", stringSerde, longSerde); - KTable cntTable = builder.table(stringSerializer, longSerializer, stringDeserializer, longDeserializer, "cnt"); + KTable cntTable = builder.table(stringSerde, longSerde, "cnt"); cntTable.toStream().process(SmokeTestUtil.printProcessorSupplier("cnt")); // dif @@ -206,7 +193,7 @@ public Integer apply(Integer value1, Integer value2) { return value1 - value2; } } - ).to("dif", stringSerializer, integerSerializer); + ).to("dif", stringSerde, intSerde); // avg sumTable.join( @@ -216,13 +203,12 @@ public Double apply(Long value1, Long value2) { return (double) value1 / (double) value2; } } - ).to("avg", stringSerializer, doubleSerializer); + ).to("avg", stringSerde, doubleSerde); // windowed count data.countByKey( TumblingWindows.of("tumbling-win-cnt").with(WINDOW_SIZE), - stringSerializer, - stringDeserializer + stringSerde ).toStream().map( new KeyValueMapper, Long, KeyValue>() { @Override @@ -230,7 +216,7 @@ public KeyValue apply(Windowed key, Long value) { return new KeyValue<>(key.value() + "@" + key.window().start(), value); } } - ).to("wcnt", stringSerializer, longSerializer); + ).to("wcnt", stringSerde, longSerde); // test repartition Agg agg = new Agg(); @@ -239,14 +225,11 @@ public KeyValue apply(Windowed key, Long value) { agg.adder(), agg.remover(), agg.selector(), - stringSerializer, - longSerializer, - longSerializer, - stringDeserializer, - longDeserializer, - longDeserializer, + stringSerde, + longSerde, + longSerde, "cntByCnt" - ).to("tagg", stringSerializer, longSerializer); + ).to("tagg", stringSerde, longSerde); return new KafkaStreams(builder, props); } diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java index c0a6f4674875d..1abf88d17ebf2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java @@ -157,7 +157,7 @@ public static Map> generate(String kafka, final int numKeys } ProducerRecord record = - new ProducerRecord<>("data", stringSerializer.serialize("", key), integerSerializer.serialize("", value)); + new ProducerRecord<>("data", stringSerde.serializer().serialize("", key), intSerde.serializer().serialize("", value)); producer.send(record); @@ -233,10 +233,10 @@ public static void verify(String kafka, Map> allData, int m retryCount = 0; for (ConsumerRecord record : records) { - String key = stringDeserializer.deserialize("", record.key()); + String key = stringSerde.deserializer().deserialize("", record.key()); switch (record.topic()) { case "echo": - Integer value = integerDeserializer.deserialize("", record.value()); + Integer value = intSerde.deserializer().deserialize("", record.value()); if (value != null && value == END) { keys.remove(key); if (keys.isEmpty()) { @@ -249,28 +249,28 @@ public static void verify(String kafka, Map> allData, int m } break; case "min": - min.put(key, integerDeserializer.deserialize("", record.value())); + min.put(key, intSerde.deserializer().deserialize("", record.value())); break; case "max": - max.put(key, integerDeserializer.deserialize("", record.value())); + max.put(key, intSerde.deserializer().deserialize("", record.value())); break; case "dif": - dif.put(key, integerDeserializer.deserialize("", record.value())); + dif.put(key, intSerde.deserializer().deserialize("", record.value())); break; case "sum": - sum.put(key, longDeserializer.deserialize("", record.value())); + sum.put(key, longSerde.deserializer().deserialize("", record.value())); break; case "cnt": - cnt.put(key, longDeserializer.deserialize("", record.value())); + cnt.put(key, longSerde.deserializer().deserialize("", record.value())); break; case "avg": - avg.put(key, doubleDeserializer.deserialize("", record.value())); + avg.put(key, doubleSerde.deserializer().deserialize("", record.value())); break; case "wcnt": - wcnt.put(key, longDeserializer.deserialize("", record.value())); + wcnt.put(key, longSerde.deserializer().deserialize("", record.value())); break; case "tagg": - tagg.put(key, longDeserializer.deserialize("", record.value())); + tagg.put(key, longSerde.deserializer().deserialize("", record.value())); break; default: System.out.println("unknown topic: " + record.topic()); diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestUtil.java b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestUtil.java index 3f5503f948100..c5ded5eb53b2d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestUtil.java +++ b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestUtil.java @@ -17,15 +17,8 @@ package org.apache.kafka.streams.smoketest; -import org.apache.kafka.common.errors.SerializationException; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; -import org.apache.kafka.common.serialization.LongDeserializer; -import org.apache.kafka.common.serialization.LongSerializer; -import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringDeserializer; -import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Initializer; @@ -36,7 +29,6 @@ import org.apache.kafka.streams.processor.ProcessorSupplier; import java.io.File; -import java.util.Map; public class SmokeTestUtil { @@ -128,74 +120,13 @@ public Long apply(String aggKey, Long value, Long aggregate) { } } - public static Serializer stringSerializer = new StringSerializer(); + public static Serde stringSerde = Serdes.String(); - public static Deserializer stringDeserializer = new StringDeserializer(); + public static Serde intSerde = Serdes.Integer(); - public static Serializer integerSerializer = new IntegerSerializer(); + public static Serde longSerde = Serdes.Long(); - public static Deserializer integerDeserializer = new IntegerDeserializer(); - - public static Serializer longSerializer = new LongSerializer(); - - public static Deserializer longDeserializer = new LongDeserializer(); - - public static Serializer doubleSerializer = new Serializer() { - - @Override - public void configure(Map configs, boolean isKey) { - } - - @Override - public byte[] serialize(String topic, Double data) { - if (data == null) - return null; - - long bits = Double.doubleToLongBits(data); - return new byte[] { - (byte) (bits >>> 56), - (byte) (bits >>> 48), - (byte) (bits >>> 40), - (byte) (bits >>> 32), - (byte) (bits >>> 24), - (byte) (bits >>> 16), - (byte) (bits >>> 8), - (byte) bits - }; - } - - @Override - public void close() { - } - }; - - public static Deserializer doubleDeserializer = new Deserializer() { - - @Override - public void configure(Map configs, boolean isKey) { - } - - @Override - public Double deserialize(String topic, byte[] data) { - if (data == null) - return null; - if (data.length != 8) { - throw new SerializationException("Size of data received by Deserializer is " + - "not 8"); - } - - long value = 0; - for (byte b : data) { - value <<= 8; - value |= b & 0xFF; - } - return Double.longBitsToDouble(value); - } - - @Override - public void close() { - } - }; + public static Serde doubleSerde = Serdes.Double(); public static File createDir(String path) throws Exception { File dir = new File(path); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java index d8b034f87818f..0468f491f9d61 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java @@ -22,6 +22,7 @@ import org.apache.kafka.common.metrics.Sensor; import org.apache.kafka.common.serialization.ByteArraySerializer; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; @@ -129,58 +130,6 @@ */ public class KeyValueStoreTestDriver { - private static Serializer unusableSerializer() { - return new Serializer() { - @Override - public void configure(Map configs, boolean isKey) { - } - - @Override - public byte[] serialize(String topic, T data) { - throw new UnsupportedOperationException("This serializer should not be used"); - } - - @Override - public void close() { - } - }; - }; - - private static Deserializer unusableDeserializer() { - return new Deserializer() { - @Override - public void configure(Map configs, boolean isKey) { - } - - @Override - public T deserialize(String topic, byte[] data) { - throw new UnsupportedOperationException("This deserializer should not be used"); - } - - @Override - public void close() { - } - }; - }; - - /** - * Create a driver object that will have a {@link #context()} that records messages - * {@link ProcessorContext#forward(Object, Object) forwarded} by the store and that provides unusable default key and - * value serializers and deserializers. This can be used when the actual serializers and deserializers are supplied to the - * store during creation, which should eliminate the need for a store to depend on the ProcessorContext's default key and - * value serializers and deserializers. - * - * @return the test driver; never null - */ - public static KeyValueStoreTestDriver create() { - Serializer keySerializer = unusableSerializer(); - Deserializer keyDeserializer = unusableDeserializer(); - Serializer valueSerializer = unusableSerializer(); - Deserializer valueDeserializer = unusableDeserializer(); - Serdes serdes = new Serdes("unexpected", keySerializer, keyDeserializer, valueSerializer, valueDeserializer); - return new KeyValueStoreTestDriver(serdes); - } - /** * Create a driver object that will have a {@link #context()} that records messages * {@link ProcessorContext#forward(Object, Object) forwarded} by the store and that provides default serializers and @@ -195,7 +144,7 @@ public static KeyValueStoreTestDriver create() { * @return the test driver; never null */ public static KeyValueStoreTestDriver create(Class keyClass, Class valueClass) { - Serdes serdes = Serdes.withBuiltinTypes("unexpected", keyClass, valueClass); + StateSerdes serdes = StateSerdes.withBuiltinTypes("unexpected", keyClass, valueClass); return new KeyValueStoreTestDriver(serdes); } @@ -215,7 +164,9 @@ public static KeyValueStoreTestDriver create(Serializer keySeria Deserializer keyDeserializer, Serializer valueSerializer, Deserializer valueDeserializer) { - Serdes serdes = new Serdes("unexpected", keySerializer, keyDeserializer, valueSerializer, valueDeserializer); + StateSerdes serdes = new StateSerdes("unexpected", + Serdes.serdeFrom(keySerializer, keyDeserializer), + Serdes.serdeFrom(valueSerializer, valueDeserializer)); return new KeyValueStoreTestDriver(serdes); } @@ -237,7 +188,7 @@ public void recordLatency(Sensor sensor, long startNs, long endNs) { private final RecordCollector recordCollector; private File stateDir = null; - protected KeyValueStoreTestDriver(final Serdes serdes) { + protected KeyValueStoreTestDriver(final StateSerdes serdes) { ByteArraySerializer rawSerializer = new ByteArraySerializer(); Producer producer = new MockProducer<>(true, rawSerializer, rawSerializer); @@ -276,13 +227,10 @@ public void send(ProducerRecord record, Serializer keySeria Properties props = new Properties(); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, MockTimestampExtractor.class); - props.put(StreamsConfig.KEY_SERIALIZER_CLASS_CONFIG, serdes.keySerializer().getClass()); - props.put(StreamsConfig.KEY_DESERIALIZER_CLASS_CONFIG, serdes.keyDeserializer().getClass()); - props.put(StreamsConfig.VALUE_SERIALIZER_CLASS_CONFIG, serdes.valueSerializer().getClass()); - props.put(StreamsConfig.VALUE_DESERIALIZER_CLASS_CONFIG, serdes.valueDeserializer().getClass()); + props.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, serdes.keySerde().getClass()); + props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, serdes.valueSerde().getClass()); - this.context = new MockProcessorContext(null, this.stateDir, serdes.keySerializer(), serdes.keyDeserializer(), serdes.valueSerializer(), - serdes.valueDeserializer(), recordCollector) { + this.context = new MockProcessorContext(null, this.stateDir, serdes.keySerde(), serdes.valueSerde(), recordCollector) { @Override public TaskId taskId() { return new TaskId(0, 1); @@ -328,7 +276,7 @@ protected void recordFlushed(K key, V value) { } } - private void restoreEntries(StateRestoreCallback func, Serdes serdes) { + private void restoreEntries(StateRestoreCallback func, StateSerdes serdes) { for (KeyValue entry : restorableEntries) { if (entry != null) { byte[] rawKey = serdes.rawKey(entry.key); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreTest.java index 46948bd1e98b6..b44583d8c5cad 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreTest.java @@ -16,8 +16,6 @@ */ package org.apache.kafka.streams.state.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.state.KeyValueStore; @@ -34,11 +32,7 @@ protected KeyValueStore createKeyValueStore( StateStoreSupplier supplier; if (useContextSerdes) { - Serializer keySer = (Serializer) context.keySerializer(); - Deserializer keyDeser = (Deserializer) context.keyDeserializer(); - Serializer valSer = (Serializer) context.valueSerializer(); - Deserializer valDeser = (Deserializer) context.valueDeserializer(); - supplier = Stores.create("my-store").withKeys(keySer, keyDeser).withValues(valSer, valDeser).inMemory().build(); + supplier = Stores.create("my-store").withKeys(context.keySerde()).withValues(context.valueSerde()).inMemory().build(); } else { supplier = Stores.create("my-store").withKeys(keyClass).withValues(valueClass).inMemory().build(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreTest.java index a2b79e5f92f91..c301223a93520 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreTest.java @@ -16,8 +16,6 @@ */ package org.apache.kafka.streams.state.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.state.KeyValueStore; @@ -40,11 +38,7 @@ protected KeyValueStore createKeyValueStore( StateStoreSupplier supplier; if (useContextSerdes) { - Serializer keySer = (Serializer) context.keySerializer(); - Deserializer keyDeser = (Deserializer) context.keyDeserializer(); - Serializer valSer = (Serializer) context.valueSerializer(); - Deserializer valDeser = (Deserializer) context.valueDeserializer(); - supplier = Stores.create("my-store").withKeys(keySer, keyDeser).withValues(valSer, valDeser).inMemory().maxEntries(10).build(); + supplier = Stores.create("my-store").withKeys(context.keySerde()).withValues(context.valueSerde()).inMemory().maxEntries(10).build(); } else { supplier = Stores.create("my-store").withKeys(keyClass).withValues(valueClass).inMemory().maxEntries(10).build(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreTest.java index 8e8f69c4265eb..280255a58dab0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreTest.java @@ -16,8 +16,6 @@ */ package org.apache.kafka.streams.state.internals; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.state.KeyValueStore; @@ -35,11 +33,7 @@ protected KeyValueStore createKeyValueStore( StateStoreSupplier supplier; if (useContextSerdes) { - Serializer keySer = (Serializer) context.keySerializer(); - Deserializer keyDeser = (Deserializer) context.keyDeserializer(); - Serializer valSer = (Serializer) context.valueSerializer(); - Deserializer valDeser = (Deserializer) context.valueDeserializer(); - supplier = Stores.create("my-store").withKeys(keySer, keyDeser).withValues(valSer, valDeser).persistent().build(); + supplier = Stores.create("my-store").withKeys(context.keySerde()).withValues(context.valueSerde()).persistent().build(); } else { supplier = Stores.create("my-store").withKeys(keyClass).withValues(valueClass).persistent().build(); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java index 5a196ece84d94..ffc97c3e7a4d7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java @@ -20,15 +20,15 @@ import org.apache.kafka.clients.producer.MockProducer; import org.apache.kafka.clients.producer.Producer; import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.serialization.ByteArrayDeserializer; -import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.processor.internals.RecordCollector; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; import org.apache.kafka.streams.state.WindowStoreUtils; @@ -51,17 +51,16 @@ public class RocksDBWindowStoreTest { - private final ByteArraySerializer byteArraySerializer = new ByteArraySerializer(); - private final ByteArrayDeserializer byteArrayDeserializer = new ByteArrayDeserializer(); + private final Serde byteArraySerde = Serdes.ByteArray(); private final String windowName = "window"; private final int numSegments = 3; private final long segmentSize = RocksDBWindowStore.MIN_SEGMENT_INTERVAL; private final long retentionPeriod = segmentSize * (numSegments - 1); private final long windowSize = 3; - private final Serdes serdes = Serdes.withBuiltinTypes("", Integer.class, String.class); + private final StateSerdes serdes = StateSerdes.withBuiltinTypes("", Integer.class, String.class); @SuppressWarnings("unchecked") - protected WindowStore createWindowStore(ProcessorContext context, Serdes serdes) { + protected WindowStore createWindowStore(ProcessorContext context, StateSerdes serdes) { StateStoreSupplier supplier = new RocksDBWindowStoreSupplier<>(windowName, retentionPeriod, numSegments, true, serdes, null); WindowStore store = (WindowStore) supplier.get(); @@ -74,7 +73,7 @@ public void testPutAndFetch() throws IOException { File baseDir = Files.createTempDirectory("test").toFile(); try { final List> changeLog = new ArrayList<>(); - Producer producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer); + Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollector(producer) { @Override public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { @@ -87,7 +86,7 @@ public void send(ProducerRecord record, Serializer keySeria MockProcessorContext context = new MockProcessorContext( null, baseDir, - byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer, + byteArraySerde, byteArraySerde, recordCollector); WindowStore store = createWindowStore(context, serdes); @@ -170,7 +169,7 @@ public void testPutAndFetchBefore() throws IOException { File baseDir = Files.createTempDirectory("test").toFile(); try { final List> changeLog = new ArrayList<>(); - Producer producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer); + Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollector(producer) { @Override public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { @@ -183,7 +182,7 @@ public void send(ProducerRecord record, Serializer keySeria MockProcessorContext context = new MockProcessorContext( null, baseDir, - byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer, + byteArraySerde, byteArraySerde, recordCollector); WindowStore store = createWindowStore(context, serdes); @@ -266,7 +265,7 @@ public void testPutAndFetchAfter() throws IOException { File baseDir = Files.createTempDirectory("test").toFile(); try { final List> changeLog = new ArrayList<>(); - Producer producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer); + Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollector(producer) { @Override public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { @@ -279,7 +278,7 @@ public void send(ProducerRecord record, Serializer keySeria MockProcessorContext context = new MockProcessorContext( null, baseDir, - byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer, + byteArraySerde, byteArraySerde, recordCollector); WindowStore store = createWindowStore(context, serdes); @@ -362,7 +361,7 @@ public void testPutSameKeyTimestamp() throws IOException { File baseDir = Files.createTempDirectory("test").toFile(); try { final List> changeLog = new ArrayList<>(); - Producer producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer); + Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollector(producer) { @Override public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { @@ -375,7 +374,7 @@ public void send(ProducerRecord record, Serializer keySeria MockProcessorContext context = new MockProcessorContext( null, baseDir, - byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer, + byteArraySerde, byteArraySerde, recordCollector); WindowStore store = createWindowStore(context, serdes); @@ -421,7 +420,7 @@ public void testRolling() throws IOException { File baseDir = Files.createTempDirectory("test").toFile(); try { final List> changeLog = new ArrayList<>(); - Producer producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer); + Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollector(producer) { @Override public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { @@ -434,7 +433,7 @@ public void send(ProducerRecord record, Serializer keySeria MockProcessorContext context = new MockProcessorContext( null, baseDir, - byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer, + byteArraySerde, byteArraySerde, recordCollector); WindowStore store = createWindowStore(context, serdes); @@ -538,7 +537,7 @@ public void testRestore() throws IOException { File baseDir = Files.createTempDirectory("test").toFile(); try { - Producer producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer); + Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollector(producer) { @Override public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { @@ -551,7 +550,7 @@ public void send(ProducerRecord record, Serializer keySeria MockProcessorContext context = new MockProcessorContext( null, baseDir, - byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer, + byteArraySerde, byteArraySerde, recordCollector); WindowStore store = createWindowStore(context, serdes); @@ -587,7 +586,7 @@ public void send(ProducerRecord record, Serializer keySeria File baseDir2 = Files.createTempDirectory("test").toFile(); try { - Producer producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer); + Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollector(producer) { @Override public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { @@ -600,7 +599,7 @@ public void send(ProducerRecord record, Serializer keySeria MockProcessorContext context = new MockProcessorContext( null, baseDir, - byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer, + byteArraySerde, byteArraySerde, recordCollector); WindowStore store = createWindowStore(context, serdes); @@ -642,7 +641,7 @@ public void send(ProducerRecord record, Serializer keySeria public void testSegmentMaintenance() throws IOException { File baseDir = Files.createTempDirectory("test").toFile(); try { - Producer producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer); + Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollector(producer) { @Override public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { @@ -652,7 +651,7 @@ public void send(ProducerRecord record, Serializer keySeria MockProcessorContext context = new MockProcessorContext( null, baseDir, - byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer, + byteArraySerde, byteArraySerde, recordCollector); WindowStore store = createWindowStore(context, serdes); @@ -745,7 +744,7 @@ public void send(ProducerRecord record, Serializer keySeria public void testInitialLoading() throws IOException { File baseDir = Files.createTempDirectory("test").toFile(); try { - Producer producer = new MockProducer<>(true, byteArraySerializer, byteArraySerializer); + Producer producer = new MockProducer<>(true, byteArraySerde.serializer(), byteArraySerde.serializer()); RecordCollector recordCollector = new RecordCollector(producer) { @Override public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { @@ -755,7 +754,7 @@ public void send(ProducerRecord record, Serializer keySeria MockProcessorContext context = new MockProcessorContext( null, baseDir, - byteArraySerializer, byteArrayDeserializer, byteArraySerializer, byteArrayDeserializer, + byteArraySerde, byteArraySerde, recordCollector); File storeDir = new File(baseDir, windowName); diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java index 5f014ef513999..9a477dfc10a69 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java @@ -31,7 +31,7 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.processor.internals.RecordCollector; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; import org.apache.kafka.test.MockProcessorContext; import org.junit.Test; @@ -44,7 +44,7 @@ public class StoreChangeLoggerTest { private final Map logged = new HashMap<>(); private final Map written = new HashMap<>(); - private final ProcessorContext context = new MockProcessorContext(Serdes.withBuiltinTypes(topic, Integer.class, String.class), + private final ProcessorContext context = new MockProcessorContext(StateSerdes.withBuiltinTypes(topic, Integer.class, String.class), new RecordCollector(null) { @SuppressWarnings("unchecked") @Override @@ -61,7 +61,7 @@ public void send(ProducerRecord record, Serializer keySeria } ); - private final StoreChangeLogger changeLogger = new StoreChangeLogger<>(topic, context, Serdes.withBuiltinTypes(topic, Integer.class, String.class), 3, 3); + private final StoreChangeLogger changeLogger = new StoreChangeLogger<>(topic, context, StateSerdes.withBuiltinTypes(topic, Integer.class, String.class), 3, 3); private final StoreChangeLogger rawChangeLogger = new RawStoreChangeLogger(topic, context, 3, 3); diff --git a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java index edbcb4a9faaf4..05713c19c3ec9 100644 --- a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java @@ -18,7 +18,8 @@ package org.apache.kafka.test; import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.processor.ProcessorContext; @@ -42,20 +43,20 @@ public class KStreamTestDriver { private ProcessorNode currNode; public KStreamTestDriver(KStreamBuilder builder) { - this(builder, null, null, null, null, null); + this(builder, null, Serdes.ByteArray(), Serdes.ByteArray()); } public KStreamTestDriver(KStreamBuilder builder, File stateDir) { - this(builder, stateDir, null, null, null, null); + this(builder, stateDir, Serdes.ByteArray(), Serdes.ByteArray()); } public KStreamTestDriver(KStreamBuilder builder, File stateDir, - Serializer keySerializer, Deserializer keyDeserializer, - Serializer valSerializer, Deserializer valDeserializer) { + Serde keySerde, + Serde valSerde) { this.topology = builder.build("X", null); this.stateDir = stateDir; - this.context = new MockProcessorContext(this, stateDir, keySerializer, keyDeserializer, valSerializer, valDeserializer, new MockRecordCollector()); + this.context = new MockProcessorContext(this, stateDir, keySerde, valSerde, new MockRecordCollector()); for (StateStoreSupplier stateStoreSupplier : topology.stateStoreSuppliers()) { StateStore store = stateStoreSupplier.get(); diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java index b463669595bd7..e57e1c7fabc43 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -18,16 +18,15 @@ package org.apache.kafka.test; import org.apache.kafka.common.metrics.Sensor; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsMetrics; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; -import org.apache.kafka.common.serialization.Deserializer; -import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.processor.TaskId; import org.apache.kafka.streams.processor.internals.RecordCollector; -import org.apache.kafka.streams.state.Serdes; +import org.apache.kafka.streams.state.StateSerdes; import java.io.File; import java.util.Collections; @@ -38,10 +37,8 @@ public class MockProcessorContext implements ProcessorContext, RecordCollector.Supplier { private final KStreamTestDriver driver; - private final Serializer keySerializer; - private final Serializer valueSerializer; - private final Deserializer keyDeserializer; - private final Deserializer valueDeserializer; + private final Serde keySerde; + private final Serde valSerde; private final RecordCollector.Supplier recordCollectorSupplier; private final File stateDir; @@ -50,21 +47,15 @@ public class MockProcessorContext implements ProcessorContext, RecordCollector.S long timestamp = -1L; - public MockProcessorContext(Serdes serdes, RecordCollector collector) { - this(null, null, serdes.keySerializer(), serdes.keyDeserializer(), serdes.valueSerializer(), serdes.valueDeserializer(), collector); - } - - public MockProcessorContext(Serializer keySerializer, Deserializer keyDeserializer, - Serializer valueSerializer, Deserializer valueDeserializer, - RecordCollector collector) { - this(null, null, keySerializer, keyDeserializer, valueSerializer, valueDeserializer, collector); + public MockProcessorContext(StateSerdes serdes, RecordCollector collector) { + this(null, null, serdes.keySerde(), serdes.valueSerde(), collector); } public MockProcessorContext(KStreamTestDriver driver, File stateDir, - Serializer keySerializer, Deserializer keyDeserializer, - Serializer valueSerializer, Deserializer valueDeserializer, + Serde keySerde, + Serde valSerde, final RecordCollector collector) { - this(driver, stateDir, keySerializer, keyDeserializer, valueSerializer, valueDeserializer, + this(driver, stateDir, keySerde, valSerde, new RecordCollector.Supplier() { @Override public RecordCollector recordCollector() { @@ -74,15 +65,13 @@ public RecordCollector recordCollector() { } public MockProcessorContext(KStreamTestDriver driver, File stateDir, - Serializer keySerializer, Deserializer keyDeserializer, - Serializer valueSerializer, Deserializer valueDeserializer, + Serde keySerde, + Serde valSerde, RecordCollector.Supplier collectorSupplier) { this.driver = driver; this.stateDir = stateDir; - this.keySerializer = keySerializer; - this.valueSerializer = valueSerializer; - this.keyDeserializer = keyDeserializer; - this.valueDeserializer = valueDeserializer; + this.keySerde = keySerde; + this.valSerde = valSerde; this.recordCollectorSupplier = collectorSupplier; } @@ -111,23 +100,13 @@ public String applicationId() { } @Override - public Serializer keySerializer() { - return keySerializer; - } - - @Override - public Serializer valueSerializer() { - return valueSerializer; - } - - @Override - public Deserializer keyDeserializer() { - return keyDeserializer; + public Serde keySerde() { + return this.keySerde; } @Override - public Deserializer valueDeserializer() { - return valueDeserializer; + public Serde valueSerde() { + return this.valSerde; } @Override From e89a9ce1a4383af32435c7f4ee04361b1b65797d Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Thu, 17 Mar 2016 18:12:40 -0700 Subject: [PATCH 051/206] KAFKA-2982; Mark the old Scala producer and related classes as deprecated Also update server tests to always use new producer. Author: Ismael Juma Reviewers: Gwen Shapira Closes #1092 from ijuma/kafka-2982-deprecate-old-producers --- .../main/scala/kafka/client/ClientUtils.scala | 1 + .../kafka/javaapi/producer/Producer.scala | 2 + .../kafka/metrics/KafkaMetricsGroup.scala | 1 + .../scala/kafka/producer/BaseProducer.scala | 6 ++ .../kafka/producer/BrokerPartitionInfo.scala | 3 +- .../kafka/producer/ByteArrayPartitioner.scala | 2 + .../kafka/producer/DefaultPartitioner.scala | 2 + .../scala/kafka/producer/KeyedMessage.scala | 2 + .../scala/kafka/producer/Partitioner.scala | 2 + .../main/scala/kafka/producer/Producer.scala | 3 +- .../producer/ProducerClosedException.scala | 1 + .../scala/kafka/producer/ProducerConfig.scala | 4 + .../scala/kafka/producer/ProducerPool.scala | 3 +- .../kafka/producer/ProducerRequestStats.scala | 3 + .../scala/kafka/producer/ProducerStats.scala | 2 + .../kafka/producer/ProducerTopicStats.scala | 4 +- .../scala/kafka/producer/SyncProducer.scala | 4 + .../kafka/producer/SyncProducerConfig.scala | 6 ++ .../producer/async/AsyncProducerConfig.scala | 2 + .../producer/async/DefaultEventHandler.scala | 1 + .../kafka/producer/async/EventHandler.scala | 1 + .../async/IllegalQueueStateException.scala | 1 + .../async/MissingConfigException.scala | 1 + .../producer/async/ProducerSendThread.scala | 1 + .../scala/kafka/tools/KafkaMigrationTool.java | 2 +- .../kafka/api/BaseProducerSendTest.scala | 22 +---- .../kafka/api/PlaintextProducerSendTest.scala | 24 ++++- .../api/ProducerFailureHandlingTest.scala | 9 +- .../kafka/tools/ConsoleProducerTest.scala | 3 +- .../scala/unit/kafka/common/ConfigTest.scala | 1 + .../ZookeeperConsumerConnectorTest.scala | 1 + .../integration/AutoOffsetResetTest.scala | 1 + .../unit/kafka/integration/FetcherTest.scala | 4 +- .../kafka/integration/PrimitiveApiTest.scala | 1 + .../ProducerConsumerTestHarness.scala | 1 + .../UncleanLeaderElectionTest.scala | 23 +++-- .../ZookeeperConsumerConnectorTest.scala | 2 +- .../unit/kafka/metrics/MetricsTest.scala | 6 +- .../unit/kafka/network/SocketServerTest.scala | 14 ++- .../kafka/producer/AsyncProducerTest.scala | 21 ++-- .../unit/kafka/producer/ProducerTest.scala | 1 + .../kafka/producer/SyncProducerTest.scala | 34 ++++--- .../kafka/server/BaseReplicaFetchTest.scala | 22 +++-- .../unit/kafka/server/LogRecoveryTest.scala | 25 ++--- .../kafka/server/ServerShutdownTest.scala | 36 ++++--- .../scala/unit/kafka/utils/TestUtils.scala | 99 +++++++++++-------- 46 files changed, 263 insertions(+), 147 deletions(-) diff --git a/core/src/main/scala/kafka/client/ClientUtils.scala b/core/src/main/scala/kafka/client/ClientUtils.scala index 2093749f3ee6b..fd1fc26d1b95a 100755 --- a/core/src/main/scala/kafka/client/ClientUtils.scala +++ b/core/src/main/scala/kafka/client/ClientUtils.scala @@ -42,6 +42,7 @@ object ClientUtils extends Logging{ * @param producerConfig The producer's config * @return topic metadata response */ + @deprecated("This method has been deprecated and will be removed in a future release.", "0.10.0.0") def fetchTopicMetadata(topics: Set[String], brokers: Seq[BrokerEndPoint], producerConfig: ProducerConfig, correlationId: Int): TopicMetadataResponse = { var fetchMetaDataSucceeded: Boolean = false var i: Int = 0 diff --git a/core/src/main/scala/kafka/javaapi/producer/Producer.scala b/core/src/main/scala/kafka/javaapi/producer/Producer.scala index c465da58019f8..44f924535ec6f 100644 --- a/core/src/main/scala/kafka/javaapi/producer/Producer.scala +++ b/core/src/main/scala/kafka/javaapi/producer/Producer.scala @@ -21,6 +21,8 @@ import kafka.producer.ProducerConfig import kafka.producer.KeyedMessage import scala.collection.mutable +@deprecated("This class has been deprecated and will be removed in a future release. " + + "Please use org.apache.kafka.clients.producer.KafkaProducer instead.", "0.10.0.0") class Producer[K,V](private val underlying: kafka.producer.Producer[K,V]) // for testing only { def this(config: ProducerConfig) = this(new kafka.producer.Producer[K,V](config)) diff --git a/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala b/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala index 72ecae1f3484f..12dfeb16c9f79 100644 --- a/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala +++ b/core/src/main/scala/kafka/metrics/KafkaMetricsGroup.scala @@ -192,6 +192,7 @@ object KafkaMetricsGroup extends KafkaMetricsGroup with Logging { removeAllMetricsInList(KafkaMetricsGroup.consumerMetricNameList, clientId) } + @deprecated("This method has been deprecated and will be removed in a future release.", "0.10.0.0") def removeAllProducerMetrics(clientId: String) { ProducerRequestStatsRegistry.removeProducerRequestStats(clientId) ProducerTopicStatsRegistry.removeProducerTopicStats(clientId) diff --git a/core/src/main/scala/kafka/producer/BaseProducer.scala b/core/src/main/scala/kafka/producer/BaseProducer.scala index 9d0976f49c3d0..83d9aa75133b2 100644 --- a/core/src/main/scala/kafka/producer/BaseProducer.scala +++ b/core/src/main/scala/kafka/producer/BaseProducer.scala @@ -21,11 +21,15 @@ import java.util.Properties // A base producer used whenever we need to have options for both old and new producers; // this class will be removed once we fully rolled out 0.9 +@deprecated("This trait has been deprecated and will be removed in a future release. " + + "Please use org.apache.kafka.clients.producer.KafkaProducer instead.", "0.10.0.0") trait BaseProducer { def send(topic: String, key: Array[Byte], value: Array[Byte]) def close() } +@deprecated("This class has been deprecated and will be removed in a future release. " + + "Please use org.apache.kafka.clients.producer.KafkaProducer instead.", "0.10.0.0") class NewShinyProducer(producerProps: Properties) extends BaseProducer { import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback @@ -50,6 +54,8 @@ class NewShinyProducer(producerProps: Properties) extends BaseProducer { } } +@deprecated("This class has been deprecated and will be removed in a future release. " + + "Please use org.apache.kafka.clients.producer.KafkaProducer instead.", "0.10.0.0") class OldProducer(producerProps: Properties) extends BaseProducer { // default to byte array partitioner diff --git a/core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala b/core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala index 6fa00dd0174ee..4616c7e623c06 100644 --- a/core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala +++ b/core/src/main/scala/kafka/producer/BrokerPartitionInfo.scala @@ -24,7 +24,7 @@ import kafka.common.KafkaException import kafka.utils.Logging import kafka.client.ClientUtils - +@deprecated("This class has been deprecated and will be removed in a future release.", "0.10.0.0") class BrokerPartitionInfo(producerConfig: ProducerConfig, producerPool: ProducerPool, topicPartitionInfo: HashMap[String, TopicMetadata]) @@ -101,4 +101,5 @@ class BrokerPartitionInfo(producerConfig: ProducerConfig, } +@deprecated("This class has been deprecated and will be removed in a future release.", "0.10.0.0") case class PartitionAndLeader(topic: String, partitionId: Int, leaderBrokerIdOpt: Option[Int]) diff --git a/core/src/main/scala/kafka/producer/ByteArrayPartitioner.scala b/core/src/main/scala/kafka/producer/ByteArrayPartitioner.scala index e6b100eb0f5de..7848456288f56 100755 --- a/core/src/main/scala/kafka/producer/ByteArrayPartitioner.scala +++ b/core/src/main/scala/kafka/producer/ByteArrayPartitioner.scala @@ -21,6 +21,8 @@ package kafka.producer import kafka.utils._ import org.apache.kafka.common.utils.Utils +@deprecated("This class has been deprecated and will be removed in a future release. " + + "It has been replaced by org.apache.kafka.clients.producer.internals.DefaultPartitioner.", "0.10.0.0") class ByteArrayPartitioner(props: VerifiableProperties = null) extends Partitioner { def partition(key: Any, numPartitions: Int): Int = { Utils.abs(java.util.Arrays.hashCode(key.asInstanceOf[Array[Byte]])) % numPartitions diff --git a/core/src/main/scala/kafka/producer/DefaultPartitioner.scala b/core/src/main/scala/kafka/producer/DefaultPartitioner.scala index 1141ed16769b8..6b10e513f5dbd 100755 --- a/core/src/main/scala/kafka/producer/DefaultPartitioner.scala +++ b/core/src/main/scala/kafka/producer/DefaultPartitioner.scala @@ -21,6 +21,8 @@ package kafka.producer import kafka.utils._ import org.apache.kafka.common.utils.Utils +@deprecated("This class has been deprecated and will be removed in a future release. " + + "It has been replaced by org.apache.kafka.clients.producer.internals.DefaultPartitioner.", "0.10.0.0") class DefaultPartitioner(props: VerifiableProperties = null) extends Partitioner { private val random = new java.util.Random diff --git a/core/src/main/scala/kafka/producer/KeyedMessage.scala b/core/src/main/scala/kafka/producer/KeyedMessage.scala index dbcf29515bb86..84ea2328de553 100644 --- a/core/src/main/scala/kafka/producer/KeyedMessage.scala +++ b/core/src/main/scala/kafka/producer/KeyedMessage.scala @@ -21,6 +21,8 @@ package kafka.producer * A topic, key, and value. * If a partition key is provided it will override the key for the purpose of partitioning but will not be stored. */ +@deprecated("This class has been deprecated and will be removed in a future release. " + + "Please use org.apache.kafka.clients.producer.ProducerRecord instead.", "0.10.0.0") case class KeyedMessage[K, V](topic: String, key: K, partKey: Any, message: V) { if(topic == null) throw new IllegalArgumentException("Topic cannot be null.") diff --git a/core/src/main/scala/kafka/producer/Partitioner.scala b/core/src/main/scala/kafka/producer/Partitioner.scala index efe6d6df87b0e..5d24692db5c56 100644 --- a/core/src/main/scala/kafka/producer/Partitioner.scala +++ b/core/src/main/scala/kafka/producer/Partitioner.scala @@ -23,6 +23,8 @@ package kafka.producer * Implementations will be constructed via reflection and are required to have a constructor that takes a single * VerifiableProperties instance--this allows passing configuration properties into the partitioner implementation. */ +@deprecated("This trait has been deprecated and will be removed in a future release. " + + "Please use org.apache.kafka.clients.producer.Partitioner instead.", "0.10.0.0") trait Partitioner { /** * Uses the key to calculate a partition bucket id for routing diff --git a/core/src/main/scala/kafka/producer/Producer.scala b/core/src/main/scala/kafka/producer/Producer.scala index 4be06c8d3fc7f..c11ad21ded9e0 100755 --- a/core/src/main/scala/kafka/producer/Producer.scala +++ b/core/src/main/scala/kafka/producer/Producer.scala @@ -25,7 +25,8 @@ import kafka.producer.async.{DefaultEventHandler, EventHandler, ProducerSendThre import kafka.serializer.Encoder import kafka.utils._ - +@deprecated("This class has been deprecated and will be removed in a future release. " + + "Please use org.apache.kafka.clients.producer.KafkaProducer instead.", "0.10.0.0") class Producer[K,V](val config: ProducerConfig, private val eventHandler: EventHandler[K,V]) // only for unit testing extends Logging { diff --git a/core/src/main/scala/kafka/producer/ProducerClosedException.scala b/core/src/main/scala/kafka/producer/ProducerClosedException.scala index 27a529331106f..4f2f7316dabf6 100644 --- a/core/src/main/scala/kafka/producer/ProducerClosedException.scala +++ b/core/src/main/scala/kafka/producer/ProducerClosedException.scala @@ -17,5 +17,6 @@ package kafka.producer +@deprecated("This class has been deprecated and will be removed in a future release.", "0.10.0.0") class ProducerClosedException() extends RuntimeException("producer already closed") { } diff --git a/core/src/main/scala/kafka/producer/ProducerConfig.scala b/core/src/main/scala/kafka/producer/ProducerConfig.scala index 08a4e5146678d..c2715d06fc380 100755 --- a/core/src/main/scala/kafka/producer/ProducerConfig.scala +++ b/core/src/main/scala/kafka/producer/ProducerConfig.scala @@ -23,6 +23,8 @@ import kafka.utils.{CoreUtils, VerifiableProperties} import kafka.message.NoCompressionCodec import kafka.common.{InvalidConfigException, Config} +@deprecated("This object has been deprecated and will be removed in a future release. " + + "Please use org.apache.kafka.clients.producer.ProducerConfig instead.", "0.10.0.0") object ProducerConfig extends Config { def validate(config: ProducerConfig) { validateClientId(config.clientId) @@ -48,6 +50,8 @@ object ProducerConfig extends Config { } } +@deprecated("This class has been deprecated and will be removed in a future release. " + + "Please use org.apache.kafka.clients.producer.ProducerConfig instead.", "0.10.0.0") class ProducerConfig private (val props: VerifiableProperties) extends AsyncProducerConfig with SyncProducerConfigShared { import ProducerConfig._ diff --git a/core/src/main/scala/kafka/producer/ProducerPool.scala b/core/src/main/scala/kafka/producer/ProducerPool.scala index 5ad68129403dd..60cef6397f7b4 100644 --- a/core/src/main/scala/kafka/producer/ProducerPool.scala +++ b/core/src/main/scala/kafka/producer/ProducerPool.scala @@ -26,7 +26,7 @@ import kafka.utils.Logging import scala.collection.mutable.HashMap - +@deprecated("This object has been deprecated and will be removed in a future release.", "0.10.0.0") object ProducerPool { /** * Used in ProducerPool to initiate a SyncProducer connection with a broker. @@ -40,6 +40,7 @@ object ProducerPool { } } +@deprecated("This class has been deprecated and will be removed in a future release.", "0.10.0.0") class ProducerPool(val config: ProducerConfig) extends Logging { private val syncProducers = new HashMap[Int, SyncProducer] private val lock = new Object() diff --git a/core/src/main/scala/kafka/producer/ProducerRequestStats.scala b/core/src/main/scala/kafka/producer/ProducerRequestStats.scala index b453f639b9082..8ab948a368047 100644 --- a/core/src/main/scala/kafka/producer/ProducerRequestStats.scala +++ b/core/src/main/scala/kafka/producer/ProducerRequestStats.scala @@ -21,6 +21,7 @@ import java.util.concurrent.TimeUnit import kafka.utils.Pool import kafka.common.{ClientIdAllBrokers, ClientIdBroker, ClientIdAndBroker} +@deprecated("This class has been deprecated and will be removed in a future release.", "0.10.0.0") class ProducerRequestMetrics(metricId: ClientIdBroker) extends KafkaMetricsGroup { val tags = metricId match { case ClientIdAndBroker(clientId, brokerHost, brokerPort) => Map("clientId" -> clientId, "brokerHost" -> brokerHost, "brokerPort" -> brokerPort.toString) @@ -36,6 +37,7 @@ class ProducerRequestMetrics(metricId: ClientIdBroker) extends KafkaMetricsGroup * Tracks metrics of requests made by a given producer client to all brokers. * @param clientId ClientId of the given producer */ +@deprecated("This class has been deprecated and will be removed in a future release.", "0.10.0.0") class ProducerRequestStats(clientId: String) { private val valueFactory = (k: ClientIdBroker) => new ProducerRequestMetrics(k) private val stats = new Pool[ClientIdBroker, ProducerRequestMetrics](Some(valueFactory)) @@ -51,6 +53,7 @@ class ProducerRequestStats(clientId: String) { /** * Stores the request stats information of each producer client in a (clientId -> ProducerRequestStats) map. */ +@deprecated("This object has been deprecated and will be removed in a future release.", "0.10.0.0") object ProducerRequestStatsRegistry { private val valueFactory = (k: String) => new ProducerRequestStats(k) private val globalStats = new Pool[String, ProducerRequestStats](Some(valueFactory)) diff --git a/core/src/main/scala/kafka/producer/ProducerStats.scala b/core/src/main/scala/kafka/producer/ProducerStats.scala index 1d0fa888c99a5..9466f26d13a2e 100644 --- a/core/src/main/scala/kafka/producer/ProducerStats.scala +++ b/core/src/main/scala/kafka/producer/ProducerStats.scala @@ -20,6 +20,7 @@ import kafka.metrics.KafkaMetricsGroup import java.util.concurrent.TimeUnit import kafka.utils.Pool +@deprecated("This class has been deprecated and will be removed in a future release.", "0.10.0.0") class ProducerStats(clientId: String) extends KafkaMetricsGroup { val tags: Map[String, String] = Map("clientId" -> clientId) val serializationErrorRate = newMeter("SerializationErrorsPerSec", "errors", TimeUnit.SECONDS, tags) @@ -30,6 +31,7 @@ class ProducerStats(clientId: String) extends KafkaMetricsGroup { /** * Stores metrics of serialization and message sending activity of each producer client in a (clientId -> ProducerStats) map. */ +@deprecated("This object has been deprecated and will be removed in a future release.", "0.10.0.0") object ProducerStatsRegistry { private val valueFactory = (k: String) => new ProducerStats(k) private val statsRegistry = new Pool[String, ProducerStats](Some(valueFactory)) diff --git a/core/src/main/scala/kafka/producer/ProducerTopicStats.scala b/core/src/main/scala/kafka/producer/ProducerTopicStats.scala index 97594c8313672..7bb9610c2c408 100644 --- a/core/src/main/scala/kafka/producer/ProducerTopicStats.scala +++ b/core/src/main/scala/kafka/producer/ProducerTopicStats.scala @@ -21,7 +21,7 @@ import kafka.common.{ClientIdTopic, ClientIdAllTopics, ClientIdAndTopic} import kafka.utils.{Pool, threadsafe} import java.util.concurrent.TimeUnit - +@deprecated("This class has been deprecated and will be removed in a future release.", "0.10.0.0") @threadsafe class ProducerTopicMetrics(metricId: ClientIdTopic) extends KafkaMetricsGroup { val tags = metricId match { @@ -38,6 +38,7 @@ class ProducerTopicMetrics(metricId: ClientIdTopic) extends KafkaMetricsGroup { * Tracks metrics for each topic the given producer client has produced data to. * @param clientId The clientId of the given producer client. */ +@deprecated("This class has been deprecated and will be removed in a future release.", "0.10.0.0") class ProducerTopicStats(clientId: String) { private val valueFactory = (k: ClientIdTopic) => new ProducerTopicMetrics(k) private val stats = new Pool[ClientIdTopic, ProducerTopicMetrics](Some(valueFactory)) @@ -53,6 +54,7 @@ class ProducerTopicStats(clientId: String) { /** * Stores the topic stats information of each producer client in a (clientId -> ProducerTopicStats) map. */ +@deprecated("This object has been deprecated and will be removed in a future release.", "0.10.0.0") object ProducerTopicStatsRegistry { private val valueFactory = (k: String) => new ProducerTopicStats(k) private val globalStats = new Pool[String, ProducerTopicStats](Some(valueFactory)) diff --git a/core/src/main/scala/kafka/producer/SyncProducer.scala b/core/src/main/scala/kafka/producer/SyncProducer.scala index ec3c4ab342bb9..de4f4ad5316f6 100644 --- a/core/src/main/scala/kafka/producer/SyncProducer.scala +++ b/core/src/main/scala/kafka/producer/SyncProducer.scala @@ -27,6 +27,8 @@ import org.apache.kafka.common.network.NetworkReceive import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.utils.Utils._ +@deprecated("This object has been deprecated and will be removed in a future release. " + + "Please use org.apache.kafka.clients.producer.KafkaProducer instead.", "0.10.0.0") object SyncProducer { val RequestKey: Short = 0 val randomGenerator = new Random @@ -36,6 +38,8 @@ object SyncProducer { * Send a message set. */ @threadsafe +@deprecated("This class has been deprecated and will be removed in a future release. " + + "Please use org.apache.kafka.clients.producer.KafkaProducer instead.", "0.10.0.0") class SyncProducer(val config: SyncProducerConfig) extends Logging { private val lock = new Object() diff --git a/core/src/main/scala/kafka/producer/SyncProducerConfig.scala b/core/src/main/scala/kafka/producer/SyncProducerConfig.scala index a08ce00a0aae7..207779c558b3a 100644 --- a/core/src/main/scala/kafka/producer/SyncProducerConfig.scala +++ b/core/src/main/scala/kafka/producer/SyncProducerConfig.scala @@ -20,6 +20,8 @@ package kafka.producer import java.util.Properties import kafka.utils.VerifiableProperties +@deprecated("This class has been deprecated and will be removed in a future release. " + + "Please use org.apache.kafka.clients.producer.ProducerConfig instead.", "0.10.0.0") class SyncProducerConfig private (val props: VerifiableProperties) extends SyncProducerConfigShared { def this(originalProps: Properties) { this(new VerifiableProperties(originalProps)) @@ -33,6 +35,8 @@ class SyncProducerConfig private (val props: VerifiableProperties) extends SyncP val port = props.getInt("port") } +@deprecated("This trait has been deprecated and will be removed in a future release. " + + "Please use org.apache.kafka.clients.producer.ProducerConfig instead.", "0.10.0.0") trait SyncProducerConfigShared { val props: VerifiableProperties @@ -59,6 +63,8 @@ trait SyncProducerConfigShared { (1, Integer.MAX_VALUE)) } +@deprecated("This object has been deprecated and will be removed in a future release. " + + "Please use org.apache.kafka.clients.producer.ProducerConfig instead.", "0.10.0.0") object SyncProducerConfig { val DefaultClientId = "" val DefaultRequiredAcks : Short = 0 diff --git a/core/src/main/scala/kafka/producer/async/AsyncProducerConfig.scala b/core/src/main/scala/kafka/producer/async/AsyncProducerConfig.scala index dd39de57b039a..cc3a79d44c1ee 100644 --- a/core/src/main/scala/kafka/producer/async/AsyncProducerConfig.scala +++ b/core/src/main/scala/kafka/producer/async/AsyncProducerConfig.scala @@ -18,6 +18,8 @@ package kafka.producer.async import kafka.utils.VerifiableProperties +@deprecated("This trait has been deprecated and will be removed in a future release. " + + "Please use org.apache.kafka.clients.producer.ProducerConfig instead.", "0.10.0.0") trait AsyncProducerConfig { val props: VerifiableProperties diff --git a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala index 7abe48a36fc38..b79e64b9a57ef 100755 --- a/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala +++ b/core/src/main/scala/kafka/producer/async/DefaultEventHandler.scala @@ -31,6 +31,7 @@ import java.util.concurrent.atomic._ import kafka.api.{TopicMetadata, ProducerRequest} import org.apache.kafka.common.utils.Utils +@deprecated("This class has been deprecated and will be removed in a future release.", "0.10.0.0") class DefaultEventHandler[K,V](config: ProducerConfig, private val partitioner: Partitioner, private val encoder: Encoder[V], diff --git a/core/src/main/scala/kafka/producer/async/EventHandler.scala b/core/src/main/scala/kafka/producer/async/EventHandler.scala index e72400077dc69..3a17bfb37ebc2 100644 --- a/core/src/main/scala/kafka/producer/async/EventHandler.scala +++ b/core/src/main/scala/kafka/producer/async/EventHandler.scala @@ -21,6 +21,7 @@ import kafka.producer.KeyedMessage /** * Handler that dispatches the batched data from the queue. */ +@deprecated("This trait has been deprecated and will be removed in a future release.", "0.10.0.0") trait EventHandler[K,V] { /** diff --git a/core/src/main/scala/kafka/producer/async/IllegalQueueStateException.scala b/core/src/main/scala/kafka/producer/async/IllegalQueueStateException.scala index 9ecdf7682b860..7779715a67472 100644 --- a/core/src/main/scala/kafka/producer/async/IllegalQueueStateException.scala +++ b/core/src/main/scala/kafka/producer/async/IllegalQueueStateException.scala @@ -20,6 +20,7 @@ package kafka.producer.async /** * Indicates that the given config parameter has invalid value */ +@deprecated("This class has been deprecated and will be removed in a future release.", "0.10.0.0") class IllegalQueueStateException(message: String) extends RuntimeException(message) { def this() = this(null) } diff --git a/core/src/main/scala/kafka/producer/async/MissingConfigException.scala b/core/src/main/scala/kafka/producer/async/MissingConfigException.scala index 304e0b24ef25c..a42678b2eb204 100644 --- a/core/src/main/scala/kafka/producer/async/MissingConfigException.scala +++ b/core/src/main/scala/kafka/producer/async/MissingConfigException.scala @@ -18,6 +18,7 @@ package kafka.producer.async /* Indicates any missing configuration parameter */ +@deprecated("This class has been deprecated and will be removed in a future release.", "0.10.0.0") class MissingConfigException(message: String) extends RuntimeException(message) { def this() = this(null) } diff --git a/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala b/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala index 8a903f32c7fc3..d4237571fb839 100644 --- a/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala +++ b/core/src/main/scala/kafka/producer/async/ProducerSendThread.scala @@ -24,6 +24,7 @@ import kafka.producer.KeyedMessage import kafka.metrics.KafkaMetricsGroup import com.yammer.metrics.core.Gauge +@deprecated("This class has been deprecated and will be removed in a future release.", "0.10.0.0") class ProducerSendThread[K,V](val threadName: String, val queue: BlockingQueue[KeyedMessage[K,V]], val handler: EventHandler[K,V], diff --git a/core/src/main/scala/kafka/tools/KafkaMigrationTool.java b/core/src/main/scala/kafka/tools/KafkaMigrationTool.java index b1ab64974128f..0b94902b8806b 100755 --- a/core/src/main/scala/kafka/tools/KafkaMigrationTool.java +++ b/core/src/main/scala/kafka/tools/KafkaMigrationTool.java @@ -60,7 +60,7 @@ * The user need to provide the configuration file for 0.7 consumer and 0.8 producer. For 0.8 producer, * the "serializer.class" config is set to "kafka.serializer.DefaultEncoder" by the code. */ -@SuppressWarnings({"unchecked", "rawtypes"}) +@SuppressWarnings({"unchecked", "rawtypes", "deprecation"}) public class KafkaMigrationTool { private static final org.apache.log4j.Logger log = org.apache.log4j.Logger.getLogger(KafkaMigrationTool.class.getName()); private static final String KAFKA_07_STATIC_CONSUMER_CLASS_NAME = "kafka.consumer.Consumer"; diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index 2d89bf893ba72..49ce748bac1d1 100644 --- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -48,7 +48,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { private var consumer2: SimpleConsumer = null private val producers = Buffer[KafkaProducer[Array[Byte], Array[Byte]]]() - private val topic = "topic" + protected val topic = "topic" private val numRecords = 100 @Before @@ -227,26 +227,6 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { } } - @Test - def testWrongSerializer() { - // send a record with a wrong type should receive a serialization exception - try { - val producer = createProducerWithWrongSerializer(brokerList) - val record5 = new ProducerRecord[Array[Byte], Array[Byte]](topic, new Integer(0), "key".getBytes, "value".getBytes) - producer.send(record5) - fail("Should have gotten a SerializationException") - } catch { - case se: SerializationException => // this is ok - } - } - - private def createProducerWithWrongSerializer(brokerList: String): KafkaProducer[Array[Byte], Array[Byte]] = { - val producerProps = new Properties() - producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer") - producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer") - createProducer(brokerList, props = Some(producerProps)) - } - /** * testClose checks the closing behavior * diff --git a/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala index d017d13d2cc43..111bc15d0bc03 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextProducerSendTest.scala @@ -19,8 +19,9 @@ package kafka.api import java.util.Properties -import org.apache.kafka.clients.producer.{ProducerConfig, KafkaProducer} +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} import org.apache.kafka.common.config.ConfigException +import org.apache.kafka.common.errors.SerializationException import org.apache.kafka.common.serialization.ByteArraySerializer import org.junit.Test @@ -51,4 +52,25 @@ class PlaintextProducerSendTest extends BaseProducerSendTest { return new KafkaProducer[Array[Byte],Array[Byte]](producerProps, new ByteArraySerializer, new ByteArraySerializer) } + @Test + def testWrongSerializer() { + // send a record with a wrong type should receive a serialization exception + try { + val producer = createProducerWithWrongSerializer(brokerList) + val record5 = new ProducerRecord[Array[Byte], Array[Byte]](topic, new Integer(0), "key".getBytes, "value".getBytes) + producer.send(record5) + fail("Should have gotten a SerializationException") + } catch { + case se: SerializationException => // this is ok + } + } + + private def createProducerWithWrongSerializer(brokerList: String): KafkaProducer[Array[Byte], Array[Byte]] = { + val producerProps = new Properties() + producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) + producerProps.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer") + producerProps.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, "org.apache.kafka.common.serialization.StringSerializer") + return new KafkaProducer[Array[Byte],Array[Byte]](producerProps) + } + } diff --git a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala index 2bb203db4d91b..7a22c730c28db 100644 --- a/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerFailureHandlingTest.scala @@ -63,9 +63,12 @@ class ProducerFailureHandlingTest extends KafkaServerTestHarness { override def setUp() { super.setUp() - producer1 = TestUtils.createNewProducer(brokerList, acks = 0, maxBlockMs = 10000L, bufferSize = producerBufferSize) - producer2 = TestUtils.createNewProducer(brokerList, acks = 1, maxBlockMs = 10000L, bufferSize = producerBufferSize) - producer3 = TestUtils.createNewProducer(brokerList, acks = -1, maxBlockMs = 10000L, bufferSize = producerBufferSize) + producer1 = TestUtils.createNewProducer(brokerList, acks = 0, requestTimeoutMs = 30000L, maxBlockMs = 10000L, + bufferSize = producerBufferSize) + producer2 = TestUtils.createNewProducer(brokerList, acks = 1, requestTimeoutMs = 30000L, maxBlockMs = 10000L, + bufferSize = producerBufferSize) + producer3 = TestUtils.createNewProducer(brokerList, acks = -1, requestTimeoutMs = 30000L, maxBlockMs = 10000L, + bufferSize = producerBufferSize) } @After diff --git a/core/src/test/scala/kafka/tools/ConsoleProducerTest.scala b/core/src/test/scala/kafka/tools/ConsoleProducerTest.scala index 7e211b72596d7..4ddc7feb186ce 100644 --- a/core/src/test/scala/kafka/tools/ConsoleProducerTest.scala +++ b/core/src/test/scala/kafka/tools/ConsoleProducerTest.scala @@ -51,9 +51,10 @@ class ConsoleProducerTest { } @Test + @deprecated("This test has been deprecated and it will be removed in a future release.", "0.10.0.0") def testValidConfigsOldProducer() { val config = new ConsoleProducer.ProducerConfig(validArgs) - new producer.ProducerConfig(ConsoleProducer.getOldProducerProps(config)); + new producer.ProducerConfig(ConsoleProducer.getOldProducerProps(config)) } @Test diff --git a/core/src/test/scala/unit/kafka/common/ConfigTest.scala b/core/src/test/scala/unit/kafka/common/ConfigTest.scala index a42836c6fdf79..26154f232d58e 100644 --- a/core/src/test/scala/unit/kafka/common/ConfigTest.scala +++ b/core/src/test/scala/unit/kafka/common/ConfigTest.scala @@ -26,6 +26,7 @@ import kafka.consumer.ConsumerConfig class ConfigTest { @Test + @deprecated("This test is deprecated and it will be removed in a future release.", "0.10.0.0") def testInvalidClientIds() { val invalidClientIds = new ArrayBuffer[String]() val badChars = Array('/', '\\', ',', '\u0000', ':', "\"", '\'', ';', '*', '?', ' ', '\t', '\r', '\n', '=') diff --git a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala index 28b1dd56c46e7..a69fba17a73f4 100644 --- a/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/consumer/ZookeeperConsumerConnectorTest.scala @@ -34,6 +34,7 @@ import org.junit.{Test, After, Before} import scala.collection._ +@deprecated("This test has been deprecated and it will be removed in a future release", "0.10.0.0") class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with Logging { val RebalanceBackoffMs = 5000 diff --git a/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala b/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala index a71ddf1f2a03b..4515b94a084f2 100644 --- a/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala +++ b/core/src/test/scala/unit/kafka/integration/AutoOffsetResetTest.scala @@ -28,6 +28,7 @@ import org.junit.{After, Before, Test} import org.apache.log4j.{Level, Logger} import org.junit.Assert._ +@deprecated("This test has been deprecated and it will be removed in a future release", "0.10.0.0") class AutoOffsetResetTest extends KafkaServerTestHarness with Logging { def generateConfigs() = List(KafkaConfig.fromProps(TestUtils.createBrokerConfig(0, zkConnect))) diff --git a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala index 5af5d1ad3e88d..3dd04541d28ab 100644 --- a/core/src/test/scala/unit/kafka/integration/FetcherTest.scala +++ b/core/src/test/scala/unit/kafka/integration/FetcherTest.scala @@ -68,11 +68,11 @@ class FetcherTest extends KafkaServerTestHarness { @Test def testFetcher() { val perNode = 2 - var count = TestUtils.sendMessages(servers, topic, perNode).size + var count = TestUtils.produceMessages(servers, topic, perNode).size fetch(count) assertQueueEmpty() - count = TestUtils.sendMessages(servers, topic, perNode).size + count = TestUtils.produceMessages(servers, topic, perNode).size fetch(count) assertQueueEmpty() } diff --git a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala index df752db573c13..beb5d0eafdb11 100755 --- a/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala +++ b/core/src/test/scala/unit/kafka/integration/PrimitiveApiTest.scala @@ -34,6 +34,7 @@ import java.util.Properties /** * End to end tests of the primitive apis against a local server */ +@deprecated("This test has been deprecated and it will be removed in a future release", "0.10.0.0") class PrimitiveApiTest extends ProducerConsumerTestHarness with ZooKeeperTestHarness { val requestHandlerLogger = Logger.getLogger(classOf[KafkaRequestHandler]) diff --git a/core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala index cc5954dfc5e55..2fdfc48b2fdb0 100644 --- a/core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/ProducerConsumerTestHarness.scala @@ -23,6 +23,7 @@ import kafka.producer.Producer import kafka.utils.{StaticPartitioner, TestUtils} import kafka.serializer.StringEncoder +@deprecated("This test has been deprecated and it will be removed in a future release", "0.10.0.0") trait ProducerConsumerTestHarness extends KafkaServerTestHarness { val host = "localhost" var producer: Producer[String, String] = null diff --git a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala index 8e72ad34e6fe3..b725d8b59ee2b 100755 --- a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala @@ -23,6 +23,8 @@ import org.junit.{Test, After, Before} import scala.util.Random import org.apache.log4j.{Level, Logger} import java.util.Properties +import java.util.concurrent.ExecutionException + import kafka.admin.AdminUtils import kafka.common.FailedToSendMessageException import kafka.consumer.{Consumer, ConsumerConfig} @@ -31,6 +33,7 @@ import kafka.server.{KafkaConfig, KafkaServer} import kafka.utils.CoreUtils import kafka.utils.TestUtils._ import kafka.zk.ZooKeeperTestHarness +import org.apache.kafka.common.errors.TimeoutException import org.junit.Assert._ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { @@ -180,14 +183,14 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { val followerId = if (leaderId == brokerId1) brokerId2 else brokerId1 debug("Follower for " + topic + " is: %s".format(followerId)) - sendMessage(servers, topic, "first") + produceMessage(servers, topic, "first") waitUntilMetadataIsPropagated(servers, topic, partitionId) assertEquals(List("first"), consumeAllMessages(topic)) // shutdown follower server servers.filter(server => server.config.brokerId == followerId).map(server => shutdownServer(server)) - sendMessage(servers, topic, "second") + produceMessage(servers, topic, "second") assertEquals(List("first", "second"), consumeAllMessages(topic)) // shutdown leader and then restart follower @@ -197,7 +200,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { // wait until new leader is (uncleanly) elected waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partitionId, newLeaderOpt = Some(followerId)) - sendMessage(servers, topic, "third") + produceMessage(servers, topic, "third") // second message was lost due to unclean election assertEquals(List("first", "third"), consumeAllMessages(topic)) @@ -215,14 +218,14 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { val followerId = if (leaderId == brokerId1) brokerId2 else brokerId1 debug("Follower for " + topic + " is: %s".format(followerId)) - sendMessage(servers, topic, "first") + produceMessage(servers, topic, "first") waitUntilMetadataIsPropagated(servers, topic, partitionId) assertEquals(List("first"), consumeAllMessages(topic)) // shutdown follower server servers.filter(server => server.config.brokerId == followerId).map(server => shutdownServer(server)) - sendMessage(servers, topic, "second") + produceMessage(servers, topic, "second") assertEquals(List("first", "second"), consumeAllMessages(topic)) // shutdown leader and then restart follower @@ -233,16 +236,20 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partitionId, newLeaderOpt = Some(-1)) // message production and consumption should both fail while leader is down - intercept[FailedToSendMessageException] { - sendMessage(servers, topic, "third") + try { + produceMessage(servers, topic, "third") + fail("Message produced while leader is down should fail, but it succeeded") + } catch { + case e: ExecutionException if e.getCause.isInstanceOf[TimeoutException] => // expected } + assertEquals(List.empty[String], consumeAllMessages(topic)) // restart leader temporarily to send a successfully replicated message servers.filter(server => server.config.brokerId == leaderId).map(server => server.startup()) waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partitionId, newLeaderOpt = Some(leaderId)) - sendMessage(servers, topic, "third") + produceMessage(servers, topic, "third") waitUntilMetadataIsPropagated(servers, topic, partitionId) servers.filter(server => server.config.brokerId == leaderId).map(server => shutdownServer(server)) diff --git a/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala b/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala index 88d95e8f6731d..e4c46973b5d76 100644 --- a/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala +++ b/core/src/test/scala/unit/kafka/javaapi/consumer/ZookeeperConsumerConnectorTest.scala @@ -36,7 +36,7 @@ import scala.collection.JavaConversions import org.apache.log4j.{Level, Logger} import org.junit.Assert._ - +@deprecated("This test has been deprecated and it will be removed in a future release", "0.10.0.0") class ZookeeperConsumerConnectorTest extends KafkaServerTestHarness with ZooKeeperTestHarness with Logging { val numNodes = 2 val numParts = 2 diff --git a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala index ee41fd7e78feb..3707deb3a2dfd 100644 --- a/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala +++ b/core/src/test/scala/unit/kafka/metrics/MetricsTest.scala @@ -52,6 +52,7 @@ class MetricsTest extends KafkaServerTestHarness with Logging { } @Test + @deprecated("This test has been deprecated and it will be removed in a future release", "0.10.0.0") def testMetricsLeak() { // create topic topic1 with 1 partition on broker 0 createTopic(zkUtils, topic, numPartitions = 1, replicationFactor = 1, servers = servers) @@ -78,13 +79,14 @@ class MetricsTest extends KafkaServerTestHarness with Logging { assertFalse("Topic metrics exists after deleteTopic", checkTopicMetricsExists(topic)) } + @deprecated("This test has been deprecated and it will be removed in a future release", "0.10.0.0") def createAndShutdownStep(group: String, consumerId: String, producerId: String): Unit = { - val sentMessages1 = sendMessages(servers, topic, nMessages) + sendMessages(servers, topic, nMessages) // create a consumer val consumerConfig1 = new ConsumerConfig(TestUtils.createConsumerProperties(zkConnect, group, consumerId)) val zkConsumerConnector1 = new ZookeeperConsumerConnector(consumerConfig1, true) val topicMessageStreams1 = zkConsumerConnector1.createMessageStreams(Map(topic -> 1), new StringDecoder(), new StringDecoder()) - val receivedMessages1 = getMessages(topicMessageStreams1, nMessages) + getMessages(topicMessageStreams1, nMessages) zkConsumerConnector1.shutdown() } diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index d94c3143202a1..5d28894e32bff 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -17,7 +17,6 @@ package kafka.network; - import java.net._ import javax.net.ssl._ import java.io._ @@ -33,7 +32,6 @@ import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.requests.{ProduceRequest, RequestHeader} import org.apache.kafka.common.utils.SystemTime -import kafka.producer.SyncProducerConfig import kafka.server.KafkaConfig import kafka.utils.TestUtils @@ -103,9 +101,9 @@ class SocketServerTest extends JUnitSuite { private def producerRequestBytes: Array[Byte] = { val apiKey: Short = 0 val correlationId = -1 - val clientId = SyncProducerConfig.DefaultClientId - val ackTimeoutMs = SyncProducerConfig.DefaultAckTimeoutMs - val ack = SyncProducerConfig.DefaultRequiredAcks + val clientId = "" + val ackTimeoutMs = 10000 + val ack = 0: Short val emptyHeader = new RequestHeader(apiKey, clientId, correlationId) val emptyRequest = new ProduceRequest(ack, ackTimeoutMs, new HashMap[TopicPartition, ByteBuffer]()) @@ -249,9 +247,9 @@ class SocketServerTest extends JUnitSuite { val apiKey = ApiKeys.PRODUCE.id val correlationId = -1 - val clientId = SyncProducerConfig.DefaultClientId - val ackTimeoutMs = SyncProducerConfig.DefaultAckTimeoutMs - val ack = SyncProducerConfig.DefaultRequiredAcks + val clientId = "" + val ackTimeoutMs = 10000 + val ack = 0: Short val emptyHeader = new RequestHeader(apiKey, clientId, correlationId) val emptyRequest = new ProduceRequest(ack, ackTimeoutMs, new HashMap[TopicPartition, ByteBuffer]()) diff --git a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala index f711ca4e5bf7b..3088199ceef1a 100755 --- a/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/AsyncProducerTest.scala @@ -35,7 +35,13 @@ import scala.collection.Map import scala.collection.mutable.ArrayBuffer import kafka.utils._ +@deprecated("This test has been deprecated and it will be removed in a future release.", "0.10.0.0") class AsyncProducerTest { + + class NegativePartitioner(props: VerifiableProperties = null) extends Partitioner { + def partition(data: Any, numPartitions: Int): Int = -1 + } + // One of the few cases we can just set a fixed port because the producer is mocked out here since this uses mocks val props = Seq(createBrokerConfig(1, "127.0.0.1:1", port=65534)) val configs = props.map(KafkaConfig.fromProps) @@ -373,15 +379,20 @@ class AsyncProducerTest { val msgs = TestUtils.getMsgStrings(2) + import SyncProducerConfig.{DefaultAckTimeoutMs, DefaultClientId} + // produce request for topic1 and partitions 0 and 1. Let the first request fail // entirely. The second request will succeed for partition 1 but fail for partition 0. // On the third try for partition 0, let it succeed. - val request1 = TestUtils.produceRequestWithAcks(List(topic1), List(0, 1), messagesToSet(msgs), acks = 1, correlationId = 11) - val request2 = TestUtils.produceRequestWithAcks(List(topic1), List(0, 1), messagesToSet(msgs), acks = 1, correlationId = 17) + val request1 = TestUtils.produceRequestWithAcks(List(topic1), List(0, 1), messagesToSet(msgs), acks = 1, + correlationId = 11, timeout = DefaultAckTimeoutMs, clientId = DefaultClientId) + val request2 = TestUtils.produceRequestWithAcks(List(topic1), List(0, 1), messagesToSet(msgs), acks = 1, + correlationId = 17, timeout = DefaultAckTimeoutMs, clientId = DefaultClientId) val response1 = ProducerResponse(0, Map((TopicAndPartition("topic1", 0), ProducerResponseStatus(Errors.NOT_LEADER_FOR_PARTITION.code, 0L)), (TopicAndPartition("topic1", 1), ProducerResponseStatus(Errors.NONE.code, 0L)))) - val request3 = TestUtils.produceRequest(topic1, 0, messagesToSet(msgs), acks = 1, correlationId = 21) + val request3 = TestUtils.produceRequest(topic1, 0, messagesToSet(msgs), acks = 1, correlationId = 21, + timeout = DefaultAckTimeoutMs, clientId = DefaultClientId) val response2 = ProducerResponse(0, Map((TopicAndPartition("topic1", 0), ProducerResponseStatus(Errors.NONE.code, 0L)))) val mockSyncProducer = EasyMock.createMock(classOf[SyncProducer]) @@ -480,7 +491,3 @@ class AsyncProducerTest { messages.map(m => new Message(key = key, bytes = m, timestamp = 0L, magicValue = Message.MagicValue_V1)): _*) } } - -class NegativePartitioner(props: VerifiableProperties = null) extends Partitioner { - def partition(data: Any, numPartitions: Int): Int = -1 -} diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala index de19f6f1b87e7..4a1ad5abcfdc5 100755 --- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -37,6 +37,7 @@ import org.junit.Assert._ import org.junit.{After, Before, Test} import org.scalatest.exceptions.TestFailedException +@deprecated("This test has been deprecated and it will be removed in a future release.", "0.10.0.0") class ProducerTest extends ZooKeeperTestHarness with Logging{ private val brokerId1 = 0 private val brokerId2 = 1 diff --git a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala index c1034febde779..8e234d264bb3a 100644 --- a/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/SyncProducerTest.scala @@ -21,7 +21,7 @@ import java.net.SocketTimeoutException import java.util.Properties import kafka.admin.AdminUtils -import kafka.api.ProducerResponseStatus +import kafka.api.{ProducerRequest, ProducerResponseStatus} import kafka.common.TopicAndPartition import kafka.integration.KafkaServerTestHarness import kafka.message._ @@ -31,11 +31,22 @@ import org.apache.kafka.common.protocol.{Errors, SecurityProtocol} import org.junit.Test import org.junit.Assert._ +@deprecated("This test has been deprecated and it will be removed in a future release", "0.10.0.0") class SyncProducerTest extends KafkaServerTestHarness { private val messageBytes = new Array[Byte](2) // turning off controlled shutdown since testProducerCanTimeout() explicitly shuts down request handler pool. def generateConfigs() = List(KafkaConfig.fromProps(TestUtils.createBrokerConfigs(1, zkConnect, false).head)) + private def produceRequest(topic: String, + partition: Int, + message: ByteBufferMessageSet, + acks: Int, + timeout: Int = SyncProducerConfig.DefaultAckTimeoutMs, + correlationId: Int = 0, + clientId: String = SyncProducerConfig.DefaultClientId): ProducerRequest = { + TestUtils.produceRequest(topic, partition, message, acks, timeout, correlationId, clientId) + } + @Test def testReachableServer() { val server = servers.head @@ -46,7 +57,7 @@ class SyncProducerTest extends KafkaServerTestHarness { val producer = new SyncProducer(new SyncProducerConfig(props)) val firstStart = SystemTime.milliseconds try { - val response = producer.send(TestUtils.produceRequest("test", 0, + val response = producer.send(produceRequest("test", 0, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = new Message(messageBytes)), acks = 1)) assertNotNull(response) } catch { @@ -56,7 +67,7 @@ class SyncProducerTest extends KafkaServerTestHarness { assertTrue((firstEnd-firstStart) < 500) val secondStart = SystemTime.milliseconds try { - val response = producer.send(TestUtils.produceRequest("test", 0, + val response = producer.send(produceRequest("test", 0, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = new Message(messageBytes)), acks = 1)) assertNotNull(response) } catch { @@ -65,7 +76,7 @@ class SyncProducerTest extends KafkaServerTestHarness { val secondEnd = SystemTime.milliseconds assertTrue((secondEnd-secondStart) < 500) try { - val response = producer.send(TestUtils.produceRequest("test", 0, + val response = producer.send(produceRequest("test", 0, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = new Message(messageBytes)), acks = 1)) assertNotNull(response) } catch { @@ -101,7 +112,7 @@ class SyncProducerTest extends KafkaServerTestHarness { val message1 = new Message(new Array[Byte](configs(0).messageMaxBytes + 1)) val messageSet1 = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = message1) - val response1 = producer.send(TestUtils.produceRequest("test", 0, messageSet1, acks = 1)) + val response1 = producer.send(produceRequest("test", 0, messageSet1, acks = 1)) assertEquals(1, response1.status.count(_._2.error != Errors.NONE.code)) assertEquals(Errors.MESSAGE_TOO_LARGE.code, response1.status(TopicAndPartition("test", 0)).error) @@ -110,7 +121,7 @@ class SyncProducerTest extends KafkaServerTestHarness { val safeSize = configs(0).messageMaxBytes - Message.MinMessageOverhead - Message.TimestampLength - MessageSet.LogOverhead - 1 val message2 = new Message(new Array[Byte](safeSize)) val messageSet2 = new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = message2) - val response2 = producer.send(TestUtils.produceRequest("test", 0, messageSet2, acks = 1)) + val response2 = producer.send(produceRequest("test", 0, messageSet2, acks = 1)) assertEquals(1, response1.status.count(_._2.error != Errors.NONE.code)) assertEquals(Errors.NONE.code, response2.status(TopicAndPartition("test", 0)).error) @@ -130,14 +141,14 @@ class SyncProducerTest extends KafkaServerTestHarness { TestUtils.waitUntilLeaderIsElectedOrChanged(zkUtils, "test", 0) // This message will be dropped silently since message size too large. - producer.send(TestUtils.produceRequest("test", 0, + producer.send(produceRequest("test", 0, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = new Message(new Array[Byte](configs(0).messageMaxBytes + 1))), acks = 0)) // Send another message whose size is large enough to exceed the buffer size so // the socket buffer will be flushed immediately; // this send should fail since the socket has been closed try { - producer.send(TestUtils.produceRequest("test", 0, + producer.send(produceRequest("test", 0, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = new Message(new Array[Byte](configs(0).messageMaxBytes + 1))), acks = 0)) } catch { case e : java.io.IOException => // success @@ -154,7 +165,8 @@ class SyncProducerTest extends KafkaServerTestHarness { val messages = new ByteBufferMessageSet(NoCompressionCodec, new Message(messageBytes)) // #1 - test that we get an error when partition does not belong to broker in response - val request = TestUtils.produceRequestWithAcks(Array("topic1", "topic2", "topic3"), Array(0), messages, 1) + val request = TestUtils.produceRequestWithAcks(Array("topic1", "topic2", "topic3"), Array(0), messages, 1, + timeout = SyncProducerConfig.DefaultAckTimeoutMs, clientId = SyncProducerConfig.DefaultClientId) val response = producer.send(request) assertNotNull(response) @@ -199,7 +211,7 @@ class SyncProducerTest extends KafkaServerTestHarness { val producer = new SyncProducer(new SyncProducerConfig(props)) val messages = new ByteBufferMessageSet(NoCompressionCodec, new Message(messageBytes)) - val request = TestUtils.produceRequest("topic1", 0, messages, acks = 1) + val request = produceRequest("topic1", 0, messages, acks = 1) // stop IO threads and request handling, but leave networking operational // any requests should be accepted and queue up, but not handled @@ -248,7 +260,7 @@ class SyncProducerTest extends KafkaServerTestHarness { AdminUtils.createTopic(zkUtils, topicName, 1, 1,topicProps) TestUtils.waitUntilLeaderIsElectedOrChanged(zkUtils, topicName, 0) - val response = producer.send(TestUtils.produceRequest(topicName, 0, + val response = producer.send(produceRequest(topicName, 0, new ByteBufferMessageSet(compressionCodec = NoCompressionCodec, messages = new Message(messageBytes)),-1)) assertEquals(Errors.NOT_ENOUGH_REPLICAS.code, response.status(TopicAndPartition(topicName, 0)).error) diff --git a/core/src/test/scala/unit/kafka/server/BaseReplicaFetchTest.scala b/core/src/test/scala/unit/kafka/server/BaseReplicaFetchTest.scala index 5ecc2c0f91dc9..c5b61ded051c1 100644 --- a/core/src/test/scala/unit/kafka/server/BaseReplicaFetchTest.scala +++ b/core/src/test/scala/unit/kafka/server/BaseReplicaFetchTest.scala @@ -20,13 +20,13 @@ package kafka.server import java.io.File import org.apache.kafka.common.protocol.SecurityProtocol -import org.junit.{Test, After, Before} +import org.junit.{After, Before, Test} import kafka.zk.ZooKeeperTestHarness -import kafka.utils.TestUtils._ -import kafka.producer.KeyedMessage -import kafka.serializer.StringEncoder -import kafka.utils.{TestUtils} +import kafka.utils.TestUtils +import TestUtils._ import kafka.common._ +import org.apache.kafka.clients.producer.ProducerRecord +import org.apache.kafka.common.serialization.StringSerializer abstract class BaseReplicaFetchTest extends ZooKeeperTestHarness { var brokers: Seq[KafkaServer] = null @@ -63,11 +63,13 @@ abstract class BaseReplicaFetchTest extends ZooKeeperTestHarness { } // send test messages to leader - val producer = TestUtils.createProducer[String, String](TestUtils.getBrokerListStrFromServers(brokers), - encoder = classOf[StringEncoder].getName, - keyEncoder = classOf[StringEncoder].getName) - val messages = testMessageList1.map(m => new KeyedMessage(topic1, m, m)) ++ testMessageList2.map(m => new KeyedMessage(topic2, m, m)) - producer.send(messages:_*) + val producer = TestUtils.createNewProducer(TestUtils.getBrokerListStrFromServers(brokers), + retries = 5, + keySerializer = new StringSerializer, + valueSerializer = new StringSerializer) + val records = testMessageList1.map(m => new ProducerRecord(topic1, m, m)) ++ + testMessageList2.map(m => new ProducerRecord(topic2, m, m)) + records.map(producer.send).foreach(_.get) producer.close() def logsMatch(): Boolean = { diff --git a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala index d11c40f3443d2..e13bfd96342dd 100755 --- a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala @@ -19,15 +19,14 @@ package kafka.server import java.util.Properties import kafka.utils.TestUtils._ -import kafka.utils.{IntEncoder, CoreUtils, TestUtils} +import kafka.utils.{CoreUtils, TestUtils} import kafka.zk.ZooKeeperTestHarness import kafka.common._ -import kafka.producer.{KeyedMessage, Producer} -import kafka.serializer.StringEncoder - import java.io.File -import org.junit.{Test, After, Before} +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} +import org.apache.kafka.common.serialization.{IntegerSerializer, StringSerializer} +import org.junit.{After, Before, Test} import org.junit.Assert._ class LogRecoveryTest extends ZooKeeperTestHarness { @@ -54,7 +53,7 @@ class LogRecoveryTest extends ZooKeeperTestHarness { val message = "hello" - var producer: Producer[Int, String] = null + var producer: KafkaProducer[Integer, String] = null def hwFile1 = new OffsetCheckpoint(new File(configProps1.logDirs(0), ReplicaManager.HighWatermarkFilename)) def hwFile2 = new OffsetCheckpoint(new File(configProps2.logDirs(0), ReplicaManager.HighWatermarkFilename)) var servers = Seq.empty[KafkaServer] @@ -64,16 +63,19 @@ class LogRecoveryTest extends ZooKeeperTestHarness { def updateProducer() = { if (producer != null) producer.close() - producer = TestUtils.createProducer[Int, String](TestUtils.getBrokerListStrFromServers(servers), - encoder = classOf[StringEncoder].getName, - keyEncoder = classOf[IntEncoder].getName) + producer = TestUtils.createNewProducer( + TestUtils.getBrokerListStrFromServers(servers), + retries = 5, + keySerializer = new IntegerSerializer, + valueSerializer = new StringSerializer + ) } @Before override def setUp() { super.setUp() - configs = TestUtils.createBrokerConfigs(2, zkConnect, false).map(KafkaConfig.fromProps(_, overridingProps)) + configs = TestUtils.createBrokerConfigs(2, zkConnect, enableControlledShutdown = false).map(KafkaConfig.fromProps(_, overridingProps)) // start both servers server1 = TestUtils.createServer(configProps1) @@ -230,7 +232,6 @@ class LogRecoveryTest extends ZooKeeperTestHarness { } private def sendMessages(n: Int = 1) { - for(i <- 0 until n) - producer.send(new KeyedMessage[Int, String](topic, 0, message)) + (0 until n).map(_ => producer.send(new ProducerRecord(topic, 0, message))).foreach(_.get) } } diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index 8f081b9628b4a..67f62d9cf7569 100755 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -18,15 +18,14 @@ package kafka.server import kafka.zk.ZooKeeperTestHarness import kafka.consumer.SimpleConsumer -import kafka.producer._ -import kafka.utils.{IntEncoder, TestUtils, CoreUtils} +import kafka.utils.{CoreUtils, TestUtils} import kafka.utils.TestUtils._ import kafka.api.FetchRequestBuilder import kafka.message.ByteBufferMessageSet -import kafka.serializer.StringEncoder - import java.io.File +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} +import org.apache.kafka.common.serialization.{IntegerSerializer, StringSerializer} import org.junit.{Before, Test} import org.junit.Assert._ @@ -46,27 +45,34 @@ class ServerShutdownTest extends ZooKeeperTestHarness { @Test def testCleanShutdown() { + + def createProducer(server: KafkaServer): KafkaProducer[Integer, String] = + TestUtils.createNewProducer( + TestUtils.getBrokerListStrFromServers(Seq(server)), + retries = 5, + keySerializer = new IntegerSerializer, + valueSerializer = new StringSerializer + ) + var server = new KafkaServer(config, threadNamePrefix = Option(this.getClass.getName)) server.startup() - var producer = TestUtils.createProducer[Int, String](TestUtils.getBrokerListStrFromServers(Seq(server)), - encoder = classOf[StringEncoder].getName, - keyEncoder = classOf[IntEncoder].getName) + var producer = createProducer(server) // create topic createTopic(zkUtils, topic, numPartitions = 1, replicationFactor = 1, servers = Seq(server)) // send some messages - producer.send(sent1.map(m => new KeyedMessage[Int, String](topic, 0, m)):_*) + sent1.map(value => producer.send(new ProducerRecord(topic, 0, value))).foreach(_.get) // do a clean shutdown and check that offset checkpoint file exists server.shutdown() - for(logDir <- config.logDirs) { + for (logDir <- config.logDirs) { val OffsetCheckpointFile = new File(logDir, server.logManager.RecoveryPointCheckpointFile) assertTrue(OffsetCheckpointFile.exists) assertTrue(OffsetCheckpointFile.length() > 0) } producer.close() - + /* now restart the server and check that the written data is still readable and everything still works */ server = new KafkaServer(config) server.startup() @@ -74,13 +80,11 @@ class ServerShutdownTest extends ZooKeeperTestHarness { // wait for the broker to receive the update metadata request after startup TestUtils.waitUntilMetadataIsPropagated(Seq(server), topic, 0) - producer = TestUtils.createProducer[Int, String](TestUtils.getBrokerListStrFromServers(Seq(server)), - encoder = classOf[StringEncoder].getName, - keyEncoder = classOf[IntEncoder].getName) + producer = createProducer(server) val consumer = new SimpleConsumer(host, server.boundPort(), 1000000, 64*1024, "") var fetchedMessage: ByteBufferMessageSet = null - while(fetchedMessage == null || fetchedMessage.validBytes == 0) { + while (fetchedMessage == null || fetchedMessage.validBytes == 0) { val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, 0, 10000).maxWait(0).build()) fetchedMessage = fetched.messageSet(topic, 0) } @@ -88,10 +92,10 @@ class ServerShutdownTest extends ZooKeeperTestHarness { val newOffset = fetchedMessage.last.nextOffset // send some more messages - producer.send(sent2.map(m => new KeyedMessage[Int, String](topic, 0, m)):_*) + sent2.map(value => producer.send(new ProducerRecord(topic, 0, value))).foreach(_.get) fetchedMessage = null - while(fetchedMessage == null || fetchedMessage.validBytes == 0) { + while (fetchedMessage == null || fetchedMessage.validBytes == 0) { val fetched = consumer.fetch(new FetchRequestBuilder().addFetch(topic, 0, newOffset, 10000).build()) fetchedMessage = fetched.messageSet(topic, 0) } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 49fb85fb11c33..7b3e95590ce77 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -42,14 +42,15 @@ import kafka.consumer.{ConsumerConfig, ConsumerTimeoutException, KafkaStream} import kafka.serializer.{DefaultEncoder, Encoder, StringEncoder} import kafka.common.TopicAndPartition import kafka.admin.AdminUtils -import kafka.producer.ProducerConfig import kafka.log._ import kafka.utils.ZkUtils._ import org.junit.Assert._ -import org.apache.kafka.clients.producer.KafkaProducer -import org.apache.kafka.clients.consumer.{KafkaConsumer, RangeAssignor} +import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord} +import org.apache.kafka.clients.consumer.{Consumer, ConsumerRecord, KafkaConsumer, RangeAssignor} import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.common.network.Mode +import org.apache.kafka.common.record.CompressionType +import org.apache.kafka.common.serialization.{ByteArraySerializer, Serializer} import scala.collection.Map import scala.collection.JavaConversions._ @@ -342,7 +343,7 @@ object TestUtils extends Logging { // check if the actual iterator was longer if (actual.hasNext) { - var length2 = length; + var length2 = length while (actual.hasNext) { actual.next length2 += 1 @@ -419,6 +420,7 @@ object TestUtils extends Logging { * Create a producer with a few pre-configured properties. * If certain properties need to be overridden, they can be provided in producerProps. */ + @deprecated("This method has been deprecated and it will be removed in a future release.", "0.10.0.0") def createProducer[K, V](brokerList: String, encoder: String = classOf[DefaultEncoder].getName, keyEncoder: String = classOf[DefaultEncoder].getName, @@ -433,7 +435,7 @@ object TestUtils extends Logging { props.put("serializer.class", encoder) props.put("key.serializer.class", keyEncoder) props.put("partitioner.class", partitioner) - new Producer[K, V](new ProducerConfig(props)) + new Producer[K, V](new kafka.producer.ProducerConfig(props)) } private def securityConfigs(mode: Mode, @@ -453,16 +455,18 @@ object TestUtils extends Logging { /** * Create a (new) producer with a few pre-configured properties. */ - def createNewProducer(brokerList: String, + def createNewProducer[K, V](brokerList: String, acks: Int = -1, maxBlockMs: Long = 60 * 1000L, bufferSize: Long = 1024L * 1024L, retries: Int = 0, lingerMs: Long = 0, + requestTimeoutMs: Long = 10 * 1024L, securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT, trustStoreFile: Option[File] = None, - props: Option[Properties] = None): KafkaProducer[Array[Byte], Array[Byte]] = { - import org.apache.kafka.clients.producer.ProducerConfig + keySerializer: Serializer[K] = new ByteArraySerializer, + valueSerializer: Serializer[V] = new ByteArraySerializer, + props: Option[Properties] = None): KafkaProducer[K, V] = { val producerProps = props.getOrElse(new Properties) producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) @@ -470,15 +474,15 @@ object TestUtils extends Logging { producerProps.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, maxBlockMs.toString) producerProps.put(ProducerConfig.BUFFER_MEMORY_CONFIG, bufferSize.toString) producerProps.put(ProducerConfig.RETRIES_CONFIG, retries.toString) + producerProps.put(ProducerConfig.REQUEST_TIMEOUT_MS_CONFIG, requestTimeoutMs.toString) /* Only use these if not already set */ val defaultProps = Map( ProducerConfig.RETRY_BACKOFF_MS_CONFIG -> "100", ProducerConfig.RECONNECT_BACKOFF_MS_CONFIG -> "200", - ProducerConfig.LINGER_MS_CONFIG -> lingerMs.toString, - ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer", - ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG -> "org.apache.kafka.common.serialization.ByteArraySerializer" + ProducerConfig.LINGER_MS_CONFIG -> lingerMs.toString ) + defaultProps.foreach { case (key, value) => if (!producerProps.containsKey(key)) producerProps.put(key, value) } @@ -489,10 +493,10 @@ object TestUtils extends Logging { * invoke it before this call in IntegrationTestHarness, otherwise the * SSL client auth fails. */ - if(!producerProps.containsKey(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)) + if (!producerProps.containsKey(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)) producerProps.putAll(producerSecurityConfigs(securityProtocol, trustStoreFile)) - new KafkaProducer[Array[Byte],Array[Byte]](producerProps) + new KafkaProducer[K, V](producerProps, keySerializer, valueSerializer) } private def usesSslTransportLayer(securityProtocol: SecurityProtocol): Boolean = securityProtocol match { @@ -558,8 +562,6 @@ object TestUtils extends Logging { props.put("request.timeout.ms", "2000") props.put("request.required.acks", "-1") props.put("send.buffer.bytes", "65536") - props.put("connect.timeout.ms", "100000") - props.put("reconnect.interval", "10000") props } @@ -620,23 +622,25 @@ object TestUtils extends Logging { /** * Create a wired format request based on simple basic information */ + @deprecated("This method has been deprecated and it will be removed in a future release", "0.10.0.0") def produceRequest(topic: String, partition: Int, message: ByteBufferMessageSet, - acks: Int = SyncProducerConfig.DefaultRequiredAcks, - timeout: Int = SyncProducerConfig.DefaultAckTimeoutMs, + acks: Int, + timeout: Int, correlationId: Int = 0, - clientId: String = SyncProducerConfig.DefaultClientId): ProducerRequest = { + clientId: String): ProducerRequest = { produceRequestWithAcks(Seq(topic), Seq(partition), message, acks, timeout, correlationId, clientId) } + @deprecated("This method has been deprecated and it will be removed in a future release", "0.10.0.0") def produceRequestWithAcks(topics: Seq[String], partitions: Seq[Int], message: ByteBufferMessageSet, - acks: Int = SyncProducerConfig.DefaultRequiredAcks, - timeout: Int = SyncProducerConfig.DefaultAckTimeoutMs, + acks: Int, + timeout: Int, correlationId: Int = 0, - clientId: String = SyncProducerConfig.DefaultClientId): ProducerRequest = { + clientId: String): ProducerRequest = { val data = topics.flatMap(topic => partitions.map(partition => (TopicAndPartition(topic, partition), message)) ) @@ -889,6 +893,8 @@ object TestUtils extends Logging { time = time, brokerState = new BrokerState()) } + + @deprecated("This method has been deprecated and it will be removed in a future release.", "0.10.0.0") def sendMessages(servers: Seq[KafkaServer], topic: String, numMessages: Int, @@ -908,7 +914,7 @@ object TestUtils extends Logging { partitioner = classOf[FixedValuePartitioner].getName, producerProps = props) - producer.send(ms.map(m => new KeyedMessage[Int, String](topic, partition, m)):_*) + producer.send(ms.map(m => new KeyedMessage[Int, String](topic, partition, m)): _*) debug("Sent %d messages for partition [%s,%d]".format(ms.size, topic, partition)) producer.close() ms.toList @@ -920,24 +926,43 @@ object TestUtils extends Logging { keyEncoder = classOf[StringEncoder].getName, partitioner = classOf[DefaultPartitioner].getName, producerProps = props) - producer.send(ms.map(m => new KeyedMessage[String, String](topic, topic, m)):_*) + producer.send(ms.map(m => new KeyedMessage[String, String](topic, topic, m)): _*) producer.close() debug("Sent %d messages for topic [%s]".format(ms.size, topic)) ms.toList } - } - def sendMessage(servers: Seq[KafkaServer], - topic: String, - message: String) = { + def produceMessages(servers: Seq[KafkaServer], + topic: String, + numMessages: Int): Seq[String] = { + + val producer = createNewProducer( + TestUtils.getBrokerListStrFromServers(servers), + retries = 5, + requestTimeoutMs = 2000 + ) + + val values = (0 until numMessages).map(x => s"test-$x") + + val futures = values.map { value => + producer.send(new ProducerRecord(topic, null, null, value.getBytes)) + } + futures.foreach(_.get) + producer.close() + + debug(s"Sent ${values.size} messages for topic [$topic]") - val producer: Producer[String, String] = - createProducer(TestUtils.getBrokerListStrFromServers(servers), - encoder = classOf[StringEncoder].getName(), - keyEncoder = classOf[StringEncoder].getName()) + values + } - producer.send(new KeyedMessage[String, String](topic, topic, message)) + def produceMessage(servers: Seq[KafkaServer], topic: String, message: String) { + val producer = createNewProducer( + TestUtils.getBrokerListStrFromServers(servers), + retries = 5, + requestTimeoutMs = 2000 + ) + producer.send(new ProducerRecord(topic, topic.getBytes, message.getBytes)).get producer.close() } @@ -1056,18 +1081,14 @@ class IntEncoder(props: VerifiableProperties = null) extends Encoder[Int] { override def toBytes(n: Int) = n.toString.getBytes } -class StaticPartitioner(props: VerifiableProperties = null) extends Partitioner{ +@deprecated("This class is deprecated and it will be removed in a future release.", "0.10.0.0") +class StaticPartitioner(props: VerifiableProperties = null) extends Partitioner { def partition(data: Any, numPartitions: Int): Int = { (data.asInstanceOf[String].length % numPartitions) } } -class HashPartitioner(props: VerifiableProperties = null) extends Partitioner { - def partition(data: Any, numPartitions: Int): Int = { - (data.hashCode % numPartitions) - } -} - +@deprecated("This class has been deprecated and it will be removed in a future release.", "0.10.0.0") class FixedValuePartitioner(props: VerifiableProperties = null) extends Partitioner { def partition(data: Any, numPartitions: Int): Int = data.asInstanceOf[Int] } From 0d8cbbcb208ccaf1cb84df0440331d4cef064391 Mon Sep 17 00:00:00 2001 From: Eno Thereska Date: Fri, 18 Mar 2016 12:01:56 -0700 Subject: [PATCH 052/206] HOTFIX: Renamed tests to match expected suffix ewencp gwenshap granders could you have a look please? Thanks. Author: Eno Thereska Reviewers: Ewen Cheslack-Postava Closes #1096 from enothereska/systest-hotfix-name --- ...y_test_new_broker.py => compatibility_test_new_broker_test.py} | 0 ...nsumer_rolling_upgrade.py => consumer_rolling_upgrade_test.py} | 0 .../{message_format_change.py => message_format_change_test.py} | 0 3 files changed, 0 insertions(+), 0 deletions(-) rename tests/kafkatest/tests/{compatibility_test_new_broker.py => compatibility_test_new_broker_test.py} (100%) rename tests/kafkatest/tests/{consumer_rolling_upgrade.py => consumer_rolling_upgrade_test.py} (100%) rename tests/kafkatest/tests/{message_format_change.py => message_format_change_test.py} (100%) diff --git a/tests/kafkatest/tests/compatibility_test_new_broker.py b/tests/kafkatest/tests/compatibility_test_new_broker_test.py similarity index 100% rename from tests/kafkatest/tests/compatibility_test_new_broker.py rename to tests/kafkatest/tests/compatibility_test_new_broker_test.py diff --git a/tests/kafkatest/tests/consumer_rolling_upgrade.py b/tests/kafkatest/tests/consumer_rolling_upgrade_test.py similarity index 100% rename from tests/kafkatest/tests/consumer_rolling_upgrade.py rename to tests/kafkatest/tests/consumer_rolling_upgrade_test.py diff --git a/tests/kafkatest/tests/message_format_change.py b/tests/kafkatest/tests/message_format_change_test.py similarity index 100% rename from tests/kafkatest/tests/message_format_change.py rename to tests/kafkatest/tests/message_format_change_test.py From 5d0cd7667f7e584f05ab4e76ed139fbafa81e042 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 18 Mar 2016 12:39:41 -0700 Subject: [PATCH 053/206] KAFKA-3422: Add overloading functions without serdes in Streams DSL Also include: 1) remove streams specific configs before passing to producer and consumer to avoid warning message; 2) add `ConsumerRecord` timestamp extractor and set as the default extractor. Author: Guozhang Wang Reviewers: Michael G. Noll, Ewen Cheslack-Postava Closes #1093 from guozhangwang/KConfigWarn --- .../examples/pageview/PageViewTypedDemo.java | 15 +- .../pageview/PageViewUntypedDemo.java | 2 +- .../examples/wordcount/WordCountDemo.java | 9 +- .../apache/kafka/streams/StreamsConfig.java | 49 ++++-- .../apache/kafka/streams/kstream/KStream.java | 165 ++++++++++++++---- .../apache/kafka/streams/kstream/KTable.java | 76 ++++++-- .../kstream/internals/KStreamImpl.java | 83 +++++++-- .../streams/kstream/internals/KTableImpl.java | 52 ++++-- .../internals/KTableStoreSupplier.java | 11 +- .../ConsumerRecordTimestampExtractor.java | 39 +++++ .../WallclockTimestampExtractor.java | 11 +- .../kafka/streams/state/StateSerdes.java | 22 +-- .../apache/kafka/streams/state/Stores.java | 18 +- .../kafka/streams/state/WindowStoreUtils.java | 15 +- .../InMemoryKeyValueLoggedStore.java | 26 ++- .../InMemoryKeyValueStoreSupplier.java | 47 +++-- .../InMemoryLRUCacheStoreSupplier.java | 16 +- .../state/internals/MemoryLRUCache.java | 25 +-- .../RocksDBKeyValueStoreSupplier.java | 16 +- .../streams/state/internals/RocksDBStore.java | 27 ++- .../state/internals/RocksDBWindowStore.java | 24 ++- .../internals/RocksDBWindowStoreSupplier.java | 16 +- .../kstream/internals/KTableImplTest.java | 4 +- .../internals/KTableMapValuesTest.java | 2 +- .../streams/smoketest/SmokeTestClient.java | 18 +- .../internals/RocksDBWindowStoreTest.java | 28 +-- 26 files changed, 584 insertions(+), 232 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/processor/ConsumerRecordTimestampExtractor.java rename streams/src/main/java/org/apache/kafka/streams/processor/{internals => }/WallclockTimestampExtractor.java (72%) diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java index 15083b23b989e..0385bdeaceb1f 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java @@ -18,6 +18,7 @@ package org.apache.kafka.streams.examples.pageview; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.KafkaStreams; @@ -100,6 +101,8 @@ public static void main(String[] args) throws Exception { serdeProps.put("JsonPOJOClass", PageView.class); pageViewDeserializer.configure(serdeProps, false); + final Serde pageViewSerde = Serdes.serdeFrom(pageViewSerializer, pageViewDeserializer); + final Serializer userProfileSerializer = new JsonPOJOSerializer<>(); serdeProps.put("JsonPOJOClass", UserProfile.class); userProfileSerializer.configure(serdeProps, false); @@ -108,6 +111,8 @@ public static void main(String[] args) throws Exception { serdeProps.put("JsonPOJOClass", UserProfile.class); userProfileDeserializer.configure(serdeProps, false); + final Serde userProfileSerde = Serdes.serdeFrom(userProfileSerializer, userProfileDeserializer); + final Serializer wPageViewByRegionSerializer = new JsonPOJOSerializer<>(); serdeProps.put("JsonPOJOClass", WindowedPageViewByRegion.class); wPageViewByRegionSerializer.configure(serdeProps, false); @@ -116,6 +121,8 @@ public static void main(String[] args) throws Exception { serdeProps.put("JsonPOJOClass", WindowedPageViewByRegion.class); wPageViewByRegionDeserializer.configure(serdeProps, false); + final Serde wPageViewByRegionSerde = Serdes.serdeFrom(wPageViewByRegionSerializer, wPageViewByRegionDeserializer); + final Serializer regionCountSerializer = new JsonPOJOSerializer<>(); serdeProps.put("JsonPOJOClass", RegionCount.class); regionCountSerializer.configure(serdeProps, false); @@ -124,9 +131,11 @@ public static void main(String[] args) throws Exception { serdeProps.put("JsonPOJOClass", RegionCount.class); regionCountDeserializer.configure(serdeProps, false); - KStream views = builder.stream(Serdes.String(), Serdes.serdeFrom(pageViewSerializer, pageViewDeserializer), "streams-pageview-input"); + final Serde regionCountSerde = Serdes.serdeFrom(regionCountSerializer, regionCountDeserializer); + + KStream views = builder.stream(Serdes.String(), pageViewSerde, "streams-pageview-input"); - KTable users = builder.table(Serdes.String(), Serdes.serdeFrom(userProfileSerializer, userProfileDeserializer), "streams-userprofile-input"); + KTable users = builder.table(Serdes.String(), userProfileSerde, "streams-userprofile-input"); KStream regionCount = views .leftJoin(users, new ValueJoiner() { @@ -169,7 +178,7 @@ public KeyValue apply(Windowed ke }); // write to the result topic - regionCount.to("streams-pageviewstats-typed-output", Serdes.serdeFrom(wPageViewByRegionSerializer, wPageViewByRegionDeserializer), Serdes.serdeFrom(regionCountSerializer, regionCountDeserializer)); + regionCount.to(wPageViewByRegionSerde, regionCountSerde, "streams-pageviewstats-typed-output"); KafkaStreams streams = new KafkaStreams(builder, props); streams.start(); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java index 5b80f64752d3f..6f5cdf29701c3 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java @@ -116,7 +116,7 @@ public KeyValue apply(Windowed key, Long value) { }); // write to the result topic - regionCount.to("streams-pageviewstats-untyped-output", jsonSerde, jsonSerde); + regionCount.to(jsonSerde, jsonSerde, "streams-pageviewstats-untyped-output"); KafkaStreams streams = new KafkaStreams(builder, props); streams.start(); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java index ebd6050c70a4e..e892abb0798c3 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java @@ -48,13 +48,15 @@ public static void main(String[] args) throws Exception { props.put(StreamsConfig.APPLICATION_ID_CONFIG, "streams-wordcount"); props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9092"); props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, "localhost:2181"); + props.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); + props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data props.put(StreamsConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); KStreamBuilder builder = new KStreamBuilder(); - KStream source = builder.stream(Serdes.String(), Serdes.String(), "streams-file-input"); + KStream source = builder.stream("streams-file-input"); KTable counts = source .flatMapValues(new ValueMapper>() { @@ -68,9 +70,10 @@ public KeyValue apply(String key, String value) { return new KeyValue(value, value); } }) - .countByKey(Serdes.String(), "Counts"); + .countByKey("Counts"); - counts.to("streams-wordcount-output", Serdes.String(), Serdes.Long()); + // need to override value serde to Long type + counts.to(Serdes.String(), Serdes.Long(), "streams-wordcount-output"); KafkaStreams streams = new KafkaStreams(builder, props); streams.start(); diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 4e989bee5136e..d4efbee89105f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -26,10 +26,10 @@ import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.processor.ConsumerRecordTimestampExtractor; import org.apache.kafka.streams.processor.DefaultPartitionGrouper; import org.apache.kafka.streams.processor.internals.StreamPartitionAssignor; import org.apache.kafka.streams.processor.internals.StreamThread; -import org.apache.kafka.streams.processor.internals.WallclockTimestampExtractor; import java.util.Map; @@ -149,7 +149,7 @@ public class StreamsConfig extends AbstractConfig { REPLICATION_FACTOR_DOC) .define(TIMESTAMP_EXTRACTOR_CLASS_CONFIG, Type.CLASS, - WallclockTimestampExtractor.class.getName(), + ConsumerRecordTimestampExtractor.class.getName(), Importance.MEDIUM, TIMESTAMP_EXTRACTOR_CLASS_DOC) .define(PARTITION_GROUPER_CLASS_CONFIG, @@ -233,12 +233,18 @@ public StreamsConfig(Map props) { public Map getConsumerConfigs(StreamThread streamThread, String groupId, String clientId) { Map props = getBaseConsumerConfigs(); + // add client id with stream client id prefix, and group id props.put(ConsumerConfig.GROUP_ID_CONFIG, groupId); props.put(CommonClientConfigs.CLIENT_ID_CONFIG, clientId + "-consumer"); - props.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, getInt(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG)); - props.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, StreamPartitionAssignor.class.getName()); + // add configs required for stream partition assignor props.put(StreamsConfig.InternalConfig.STREAM_THREAD_INSTANCE, streamThread); + props.put(StreamsConfig.REPLICATION_FACTOR_CONFIG, getInt(REPLICATION_FACTOR_CONFIG)); + props.put(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG, getInt(NUM_STANDBY_REPLICAS_CONFIG)); + props.put(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, StreamPartitionAssignor.class.getName()); + + if (!getString(ZOOKEEPER_CONNECT_CONFIG).equals("")) + props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, getString(ZOOKEEPER_CONNECT_CONFIG)); return props; } @@ -249,6 +255,7 @@ public Map getRestoreConsumerConfigs(String clientId) { // no need to set group id for a restore consumer props.remove(ConsumerConfig.GROUP_ID_CONFIG); + // add client id with stream client id prefix props.put(CommonClientConfigs.CLIENT_ID_CONFIG, clientId + "-restore-consumer"); return props; @@ -257,39 +264,49 @@ public Map getRestoreConsumerConfigs(String clientId) { private Map getBaseConsumerConfigs() { Map props = this.originals(); + // remove streams properties + removeStreamsSpecificConfigs(props); + // set consumer default property values props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); - // remove properties that are not required for consumers - removeStreamsSpecificConfigs(props); - return props; } public Map getProducerConfigs(String clientId) { Map props = this.originals(); - // set producer default property values - props.put(ProducerConfig.LINGER_MS_CONFIG, "100"); + // remove consumer properties that are not required for producers + props.remove(StreamsConfig.AUTO_OFFSET_RESET_CONFIG); - // remove properties that are not required for producers + // remove streams properties removeStreamsSpecificConfigs(props); - props.remove(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG); + // set producer default property values + props.put(ProducerConfig.LINGER_MS_CONFIG, "100"); + + // add client id with stream client id prefix props.put(CommonClientConfigs.CLIENT_ID_CONFIG, clientId + "-producer"); return props; } private void removeStreamsSpecificConfigs(Map props) { - props.remove(StreamsConfig.APPLICATION_ID_CONFIG); + props.remove(StreamsConfig.POLL_MS_CONFIG); props.remove(StreamsConfig.STATE_DIR_CONFIG); - props.remove(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); - props.remove(StreamsConfig.NUM_STREAM_THREADS_CONFIG); - props.remove(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG); + props.remove(StreamsConfig.APPLICATION_ID_CONFIG); props.remove(StreamsConfig.KEY_SERDE_CLASS_CONFIG); props.remove(StreamsConfig.VALUE_SERDE_CLASS_CONFIG); - props.remove(InternalConfig.STREAM_THREAD_INSTANCE); + props.remove(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG); + props.remove(StreamsConfig.REPLICATION_FACTOR_CONFIG); + props.remove(StreamsConfig.COMMIT_INTERVAL_MS_CONFIG); + props.remove(StreamsConfig.NUM_STREAM_THREADS_CONFIG); + props.remove(StreamsConfig.NUM_STANDBY_REPLICAS_CONFIG); + props.remove(StreamsConfig.STATE_CLEANUP_DELAY_MS_CONFIG); + props.remove(StreamsConfig.PARTITION_GROUPER_CLASS_CONFIG); + props.remove(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG); + props.remove(StreamsConfig.BUFFERED_RECORDS_PER_PARTITION_CONFIG); + props.remove(StreamsConfig.InternalConfig.STREAM_THREAD_INSTANCE); } public Serde keySerde() { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index 1c78652178343..6f05c3b7213d6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -98,8 +98,9 @@ public interface KStream { * Sends key-value to a topic, also creates a new instance of KStream from the topic. * This is equivalent to calling to(topic) and from(topic). * - * @param topic the topic name - * @return the instance of KStream that consumes the given topic + * @param topic the topic name + * + * @return the instance of {@link KStream} that consumes the given topic */ KStream through(String topic); @@ -107,32 +108,33 @@ public interface KStream { * Sends key-value to a topic, also creates a new instance of KStream from the topic. * This is equivalent to calling to(topic) and from(topic). * - * @param topic the topic name - * @param keySerde key serde used to send key-value pairs, - * if not specified the default key serde defined in the configuration will be used - * @param valSerde value serde used to send key-value pairs, - * if not specified the default value serde defined in the configuration will be used - * @return the instance of KStream that consumes the given topic + * @param keySerde key serde used to send key-value pairs, + * if not specified the default key serde defined in the configuration will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default value serde defined in the configuration will be used + * @param topic the topic name + * + * @return the instance of {@link KStream} that consumes the given topic */ - KStream through(String topic, Serde keySerde, Serde valSerde); + KStream through(Serde keySerde, Serde valSerde, String topic); /** * Sends key-value to a topic using default serializers specified in the config. * - * @param topic the topic name + * @param topic the topic name */ void to(String topic); /** * Sends key-value to a topic. * - * @param topic the topic name - * @param keySerde key serde used to send key-value pairs, - * if not specified the default serde defined in the configs will be used - * @param keySerde value serde used to send key-value pairs, - * if not specified the default serde defined in the configs will be used + * @param keySerde key serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * @param topic the topic name */ - void to(String topic, Serde keySerde, Serde valSerde); + void to(Serde keySerde, Serde valSerde, String topic); /** * Applies a stateful transformation to all elements in this stream. @@ -183,6 +185,20 @@ KStream join( Serde thisValueSerde, Serde otherValueSerde); + /** + * Combines values of this stream with another KStream using Windowed Inner Join. + * + * @param otherStream the instance of {@link KStream} joined with this stream + * @param joiner the instance of {@link ValueJoiner} + * @param windows the specification of the {@link JoinWindows} + * @param the value type of the other stream + * @param the value type of the new stream + */ + KStream join( + KStream otherStream, + ValueJoiner joiner, + JoinWindows windows); + /** * Combines values of this stream with another KStream using Windowed Outer Join. * @@ -206,6 +222,20 @@ KStream outerJoin( Serde thisValueSerde, Serde otherValueSerde); + /** + * Combines values of this stream with another KStream using Windowed Outer Join. + * + * @param otherStream the instance of {@link KStream} joined with this stream + * @param joiner the instance of {@link ValueJoiner} + * @param windows the specification of the {@link JoinWindows} + * @param the value type of the other stream + * @param the value type of the new stream + */ + KStream outerJoin( + KStream otherStream, + ValueJoiner joiner, + JoinWindows windows); + /** * Combines values of this stream with another KStream using Windowed Left Join. * @@ -226,27 +256,49 @@ KStream leftJoin( Serde keySerde, Serde otherValueSerde); + /** + * Combines values of this stream with another KStream using Windowed Left Join. + * + * @param otherStream the instance of {@link KStream} joined with this stream + * @param joiner the instance of {@link ValueJoiner} + * @param windows the specification of the {@link JoinWindows} + * @param the value type of the other stream + * @param the value type of the new stream + */ + KStream leftJoin( + KStream otherStream, + ValueJoiner joiner, + JoinWindows windows); + /** * Combines values of this stream with KTable using Left Join. * - * @param ktable the instance of KTable joined with this stream - * @param joiner ValueJoiner - * @param the value type of the table - * @param the value type of the new stream + * @param table the instance of {@link KTable} joined with this stream + * @param joiner the instance of {@link ValueJoiner} + * @param the value type of the table + * @param the value type of the new stream */ - KStream leftJoin(KTable ktable, ValueJoiner joiner); + KStream leftJoin(KTable table, ValueJoiner joiner); /** * Aggregate values of this stream by key on a window basis. * - * @param reducer the class of Reducer - * @param windows the specification of the aggregation window + * @param reducer the class of {@link Reducer} + * @param windows the specification of the aggregation {@link Windows} */ KTable, V> reduceByKey(Reducer reducer, Windows windows, Serde keySerde, Serde aggValueSerde); + /** + * Aggregate values of this stream by key on a window basis. + * + * @param reducer the class of {@link Reducer} + * @param windows the specification of the aggregation {@link Windows} + */ + KTable, V> reduceByKey(Reducer reducer, Windows windows); + /** * Aggregate values of this stream by key on a window basis. * @@ -257,12 +309,19 @@ KTable reduceByKey(Reducer reducer, Serde aggValueSerde, String name); + /** + * Aggregate values of this stream by key on a window basis. + * + * @param reducer the class of {@link Reducer} + */ + KTable reduceByKey(Reducer reducer, String name); + /** * Aggregate values of this stream by key on a window basis. * * @param initializer the class of Initializer * @param aggregator the class of Aggregator - * @param windows the specification of the aggregation window + * @param windows the specification of the aggregation {@link Windows} * @param the value type of the aggregated table */ KTable, T> aggregateByKey(Initializer initializer, @@ -271,13 +330,26 @@ KTable, T> aggregateByKey(Initializer initi Serde keySerde, Serde aggValueSerde); + /** + * Aggregate values of this stream by key on a window basis. + * + * @param initializer the class of {@link Initializer} + * @param aggregator the class of {@link Aggregator} + * @param windows the specification of the aggregation {@link Windows} + * @param the value type of the aggregated table + */ + KTable, T> aggregateByKey(Initializer initializer, + Aggregator aggregator, + Windows windows); + /** * Aggregate values of this stream by key without a window basis, and hence * return an ever updating table * - * @param initializer the class of Initializer - * @param aggregator the class of Aggregator - * @param the value type of the aggregated table + * @param initializer the class of {@link Initializer} + * @param aggregator the class of {@link Aggregator} + * @param name the name of the aggregated table + * @param the value type of the aggregated table */ KTable aggregateByKey(Initializer initializer, Aggregator aggregator, @@ -285,18 +357,47 @@ KTable aggregateByKey(Initializer initializer, Serde aggValueSerde, String name); + /** + * Aggregate values of this stream by key without a window basis, and hence + * return an ever updating table + * + * @param initializer the class of {@link Initializer} + * @param aggregator the class of {@link Aggregator} + * @param name the name of the aggregated table + * @param the value type of the aggregated table + */ + KTable aggregateByKey(Initializer initializer, + Aggregator aggregator, + String name); + + /** + * Count number of messages of this stream by key on a window basis. + * + * @param windows the specification of the aggregation {@link Windows} + */ + KTable, Long> countByKey(Windows windows, Serde keySerde); + /** * Count number of messages of this stream by key on a window basis. * - * @param windows the specification of the aggregation window + * @param windows the specification of the aggregation {@link Windows} */ - KTable, Long> countByKey(Windows windows, - Serde keySerde); + KTable, Long> countByKey(Windows windows); /** * Count number of messages of this stream by key without a window basis, and hence * return a ever updating counting table. + * + * @param name the name of the aggregated table + */ + KTable countByKey(Serde keySerde, String name); + + /** + * Count number of messages of this stream by key without a window basis, and hence + * return a ever updating counting table. + * + * @param name the name of the aggregated table */ - KTable countByKey(Serde keySerde, - String name); + KTable countByKey(String name); + } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index 0ae515055b70d..997cb4dc9f105 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -66,14 +66,14 @@ public interface KTable { * Sends key-value to a topic, also creates a new instance of KTable from the topic. * This is equivalent to calling to(topic) and table(topic). * - * @param topic the topic name - * @param keySerde key serde used to send key-value pairs, - * if not specified the default key serde defined in the configuration will be used - * @param valSerde value serde used to send key-value pairs, - * if not specified the default value serde defined in the configuration will be used + * @param keySerde key serde used to send key-value pairs, + * if not specified the default key serde defined in the configuration will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default value serde defined in the configuration will be used + * @param topic the topic name * @return the new stream that consumes the given topic */ - KTable through(String topic, Serde keySerde, Serde valSerde); + KTable through(Serde keySerde, Serde valSerde, String topic); /** * Sends key-value to a topic using default serializers specified in the config. @@ -85,13 +85,13 @@ public interface KTable { /** * Sends key-value to a topic. * - * @param topic the topic name - * @param keySerde key serde used to send key-value pairs, - * if not specified the default serde defined in the configs will be used - * @param valSerde value serde used to send key-value pairs, - * if not specified the default serde defined in the configs will be used + * @param keySerde key serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * @param topic the topic name */ - void to(String topic, Serde keySerde, Serde valSerde); + void to(Serde keySerde, Serde valSerde, String topic); /** * Creates a new instance of KStream from this KTable @@ -136,21 +136,37 @@ public interface KTable { /** * Reduce values of this table by the selected key. * - * @param addReducer the class of Reducer - * @param removeReducer the class of Reducer + * @param adder the class of Reducer + * @param subtractor the class of Reducer * @param selector the KeyValue mapper that select the aggregate key * @param name the name of the resulted table * @param the key type of the aggregated table * @param the value type of the aggregated table * @return the instance of KTable */ - KTable reduce(Reducer addReducer, - Reducer removeReducer, + KTable reduce(Reducer adder, + Reducer subtractor, KeyValueMapper> selector, Serde keySerde, Serde valueSerde, String name); + /** + * Reduce values of this table by the selected key. + * + * @param adder the instance of {@link Reducer} for addition + * @param subtractor the instance of {@link Reducer} for subtraction + * @param selector the instance of {@link KeyValueMapper} that select the aggregate key + * @param name the name of the resulted table + * @param the key type of the aggregated table + * @param the value type of the aggregated table + * @return the instance of KTable + */ + KTable reduce(Reducer adder, + Reducer subtractor, + KeyValueMapper> selector, + String name); + /** * Aggregate values of this table by the selected key. * @@ -172,6 +188,24 @@ KTable aggregate(Initializer initializer, Serde aggValueSerde, String name); + /** + * Aggregate values of this table by the selected key. + * + * @param initializer the instance of {@link Initializer} + * @param adder the instance of {@link Aggregator} for addition + * @param substractor the instance of {@link Aggregator} for subtraction + * @param selector the instance of {@link KeyValueMapper} that select the aggregate key + * @param name the name of the resulted table + * @param the key type of the aggregated table + * @param the value type of the aggregated table + * @return the instance of aggregated {@link KTable} + */ + KTable aggregate(Initializer initializer, + Aggregator adder, + Aggregator substractor, + KeyValueMapper> selector, + String name); + /** * Count number of records of this table by the selected key. * @@ -184,4 +218,14 @@ KTable count(KeyValueMapper selector, Serde keySerde, Serde valueSerde, String name); + + /** + * Count number of records of this table by the selected key. + * + * @param selector the instance of {@link KeyValueMapper} that select the aggregate key + * @param name the name of the resulted table + * @param the key type of the aggregated table + * @return the instance of aggregated {@link KTable} + */ + KTable count(KeyValueMapper selector, String name); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index b29349684833a..567b06c61ddc0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -194,27 +194,25 @@ public static KStream merge(KStreamBuilder topology, KStream[ } @Override - public KStream through(String topic, - Serde keySerde, - Serde valSerde) { - to(topic, keySerde, valSerde); + public KStream through(Serde keySerde, Serde valSerde, String topic) { + to(keySerde, valSerde, topic); - return topology.stream(keySerde, valSerde); + return topology.stream(keySerde, valSerde, topic); } @Override public KStream through(String topic) { - return through(topic, null, null); + return through(null, null, topic); } @Override public void to(String topic) { - to(topic, null, null); + to(null, null, topic); } @SuppressWarnings("unchecked") @Override - public void to(String topic, Serde keySerde, Serde valSerde) { + public void to(Serde keySerde, Serde valSerde, String topic) { String name = topology.newName(SINK_NAME); StreamPartitioner streamPartitioner = null; @@ -269,6 +267,15 @@ public KStream join( return join(other, joiner, windows, keySerde, thisValueSerde, otherValueSerde, false); } + @Override + public KStream join( + KStream other, + ValueJoiner joiner, + JoinWindows windows) { + + return join(other, joiner, windows, null, null, null, false); + } + @Override public KStream outerJoin( KStream other, @@ -281,6 +288,15 @@ public KStream outerJoin( return join(other, joiner, windows, keySerde, thisValueSerde, otherValueSerde, true); } + @Override + public KStream outerJoin( + KStream other, + ValueJoiner joiner, + JoinWindows windows) { + + return join(other, joiner, windows, null, null, null, true); + } + @SuppressWarnings("unchecked") private KStream join( KStream other, @@ -363,6 +379,15 @@ public KStream leftJoin( return new KStreamImpl<>(topology, joinThisName, allSourceNodes); } + @Override + public KStream leftJoin( + KStream other, + ValueJoiner joiner, + JoinWindows windows) { + + return leftJoin(other, joiner, windows, null, null); + } + @SuppressWarnings("unchecked") @Override public KStream leftJoin(KTable other, ValueJoiner joiner) { @@ -401,6 +426,13 @@ public KTable, V> reduceByKey(Reducer reducer, return new KTableImpl<>(topology, reduceName, reduceSupplier, sourceNodes); } + @Override + public KTable, V> reduceByKey(Reducer reducer, + Windows windows) { + + return reduceByKey(reducer, windows, null, null); + } + @Override public KTable reduceByKey(Reducer reducer, Serde keySerde, @@ -425,6 +457,12 @@ public KTable reduceByKey(Reducer reducer, return new KTableImpl<>(topology, reduceName, reduceSupplier, sourceNodes); } + @Override + public KTable reduceByKey(Reducer reducer, String name) { + + return reduceByKey(reducer, null, null, name); + } + @Override public KTable, T> aggregateByKey(Initializer initializer, Aggregator aggregator, @@ -451,6 +489,14 @@ public KTable, T> aggregateByKey(Initializer, T, T>(topology, aggregateName, aggregateSupplier, sourceNodes); } + @Override + public KTable, T> aggregateByKey(Initializer initializer, + Aggregator aggregator, + Windows windows) { + + return aggregateByKey(initializer, aggregator, windows, null, null); + } + @Override public KTable aggregateByKey(Initializer initializer, Aggregator aggregator, @@ -476,6 +522,14 @@ public KTable aggregateByKey(Initializer initializer, return new KTableImpl<>(topology, aggregateName, aggregateSupplier, sourceNodes); } + @Override + public KTable aggregateByKey(Initializer initializer, + Aggregator aggregator, + String name) { + + return aggregateByKey(initializer, aggregator, null, null, name); + } + @Override public KTable, Long> countByKey(Windows windows, Serde keySerde) { @@ -495,8 +549,12 @@ public Long apply(K aggKey, V value, Long aggregate) { } @Override - public KTable countByKey(Serde keySerde, - String name) { + public KTable, Long> countByKey(Windows windows) { + return countByKey(windows, null); + } + + @Override + public KTable countByKey(Serde keySerde, String name) { return this.aggregateByKey( new Initializer() { @Override @@ -511,4 +569,9 @@ public Long apply(K aggKey, V value, Long aggregate) { } }, keySerde, Serdes.Long(), name); } + + @Override + public KTable countByKey(String name) { + return countByKey(null, name); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index 496a476afaef1..ca1e659114658 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -131,27 +131,27 @@ public KTable mapValues(ValueMapper mapper) { } @Override - public KTable through(String topic, - Serde keySerde, - Serde valSerde) { - to(topic, keySerde, valSerde); + public KTable through(Serde keySerde, + Serde valSerde, + String topic) { + to(keySerde, valSerde, topic); return topology.table(keySerde, valSerde, topic); } @Override public KTable through(String topic) { - return through(topic, null, null); + return through(null, null, topic); } @Override public void to(String topic) { - to(topic, null, null); + to(null, null, topic); } @Override - public void to(String topic, Serde keySerde, Serde valSerde) { - this.toStream().to(topic, keySerde, valSerde); + public void to(Serde keySerde, Serde valSerde, String topic) { + this.toStream().to(keySerde, valSerde, topic); } @Override @@ -239,8 +239,8 @@ public KTable leftJoin(KTable other, ValueJoiner @Override public KTable aggregate(Initializer initializer, - Aggregator add, - Aggregator remove, + Aggregator adder, + Aggregator subtractor, KeyValueMapper> selector, Serde keySerde, Serde valueSerde, @@ -259,7 +259,7 @@ public KTable aggregate(Initializer initializer, KTableProcessorSupplier> selectSupplier = new KTableRepartitionMap<>(this, selector); - ProcessorSupplier> aggregateSupplier = new KTableAggregate<>(name, initializer, add, remove); + ProcessorSupplier> aggregateSupplier = new KTableAggregate<>(name, initializer, adder, subtractor); StateStoreSupplier aggregateStore = Stores.create(name) .withKeys(keySerde) @@ -286,6 +286,16 @@ public KTable aggregate(Initializer initializer, return new KTableImpl<>(topology, aggregateName, aggregateSupplier, Collections.singleton(sourceName)); } + @Override + public KTable aggregate(Initializer initializer, + Aggregator adder, + Aggregator substractor, + KeyValueMapper> selector, + String name) { + + return aggregate(initializer, adder, substractor, selector, null, null, null, name); + } + @Override public KTable count(final KeyValueMapper selector, Serde keySerde, @@ -318,8 +328,13 @@ public KeyValue apply(K key, V value) { } @Override - public KTable reduce(Reducer addReducer, - Reducer removeReducer, + public KTable count(final KeyValueMapper selector, String name) { + return count(selector, null, null, name); + } + + @Override + public KTable reduce(Reducer adder, + Reducer subtractor, KeyValueMapper> selector, Serde keySerde, Serde valueSerde, @@ -337,7 +352,7 @@ public KTable reduce(Reducer addReducer, KTableProcessorSupplier> selectSupplier = new KTableRepartitionMap<>(this, selector); - ProcessorSupplier> aggregateSupplier = new KTableReduce<>(name, addReducer, removeReducer); + ProcessorSupplier> aggregateSupplier = new KTableReduce<>(name, adder, subtractor); StateStoreSupplier aggregateStore = Stores.create(name) .withKeys(keySerde) @@ -364,6 +379,15 @@ public KTable reduce(Reducer addReducer, return new KTableImpl<>(topology, reduceName, aggregateSupplier, Collections.singleton(sourceName)); } + @Override + public KTable reduce(Reducer adder, + Reducer subtractor, + KeyValueMapper> selector, + String name) { + + return reduce(adder, subtractor, selector, null, null, name); + } + @SuppressWarnings("unchecked") KTableValueGetterSupplier valueGetterSupplier() { if (processorSupplier instanceof KTableSource) { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableStoreSupplier.java index af3c0d71b5297..ff118da02db70 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableStoreSupplier.java @@ -23,7 +23,6 @@ import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.state.internals.MeteredKeyValueStore; import org.apache.kafka.streams.state.internals.RocksDBStore; -import org.apache.kafka.streams.state.StateSerdes; /** * A KTable storage. It stores all entries in a local RocksDB database. @@ -34,15 +33,17 @@ public class KTableStoreSupplier implements StateStoreSupplier { private final String name; - private final StateSerdes serdes; + private final Serde keySerde; + private final Serde valueSerde; private final Time time; protected KTableStoreSupplier(String name, Serde keySerde, - Serde valSerde, + Serde valueSerde, Time time) { this.name = name; - this.serdes = new StateSerdes<>(name, keySerde, valSerde); + this.keySerde = keySerde; + this.valueSerde = valueSerde; this.time = time; } @@ -51,7 +52,7 @@ public String name() { } public StateStore get() { - return new MeteredKeyValueStore<>(new RocksDBStore<>(name, serdes), "rocksdb-state", time); + return new MeteredKeyValueStore<>(new RocksDBStore<>(name, keySerde, valueSerde), "rocksdb-state", time); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ConsumerRecordTimestampExtractor.java b/streams/src/main/java/org/apache/kafka/streams/processor/ConsumerRecordTimestampExtractor.java new file mode 100644 index 0000000000000..61b1c9817a211 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ConsumerRecordTimestampExtractor.java @@ -0,0 +1,39 @@ +/** + * 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 org.apache.kafka.streams.processor; + +import org.apache.kafka.clients.consumer.ConsumerRecord; + +/** + * Retrieves built-in timestamps from Kafka messages (introduced in KIP-32: Add timestamps to Kafka message). + * + * Here, "built-in" refers to the fact that compatible Kafka producer clients automatically and + * transparently embed such timestamps into messages they sent to Kafka, which can then be retrieved + * via this timestamp extractor. + * + * If CreateTime is used to define the built-in timestamps, using this extractor effectively provide + * event-time semantics. + * + * If you need processing-time semantics, use {@link WallclockTimestampExtractor}. + */ +public class ConsumerRecordTimestampExtractor implements TimestampExtractor { + @Override + public long extract(ConsumerRecord record) { + return record.timestamp(); + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/WallclockTimestampExtractor.java b/streams/src/main/java/org/apache/kafka/streams/processor/WallclockTimestampExtractor.java similarity index 72% rename from streams/src/main/java/org/apache/kafka/streams/processor/internals/WallclockTimestampExtractor.java rename to streams/src/main/java/org/apache/kafka/streams/processor/WallclockTimestampExtractor.java index 60b3b96dd8995..81821cefd4a20 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/WallclockTimestampExtractor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/WallclockTimestampExtractor.java @@ -15,11 +15,18 @@ * limitations under the License. */ -package org.apache.kafka.streams.processor.internals; +package org.apache.kafka.streams.processor; import org.apache.kafka.clients.consumer.ConsumerRecord; -import org.apache.kafka.streams.processor.TimestampExtractor; +/** + * Retrieves current wall clock timestamps as {@link System#currentTimeMillis()}. + * + * Using this extractor effectively provides processing-time semantics. + * + * If you need event-time semantics, use {@link ConsumerRecordTimestampExtractor} with + * built-in CreateTime timestamp (see KIP-32: Add timestamps to Kafka message for details). + */ public class WallclockTimestampExtractor implements TimestampExtractor { @Override public long extract(ConsumerRecord record) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java b/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java index 1a41a1660fa35..9daac98136f01 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java @@ -33,7 +33,7 @@ public static StateSerdes withBuiltinTypes(String topic, Class k return new StateSerdes<>(topic, Serdes.serdeFrom(keyClass), Serdes.serdeFrom(valueClass)); } - private final String topic; + private final String stateName; private final Serde keySerde; private final Serde valueSerde; @@ -43,15 +43,15 @@ public static StateSerdes withBuiltinTypes(String topic, Class k * is provided to bind this serde factory to, so that future calls for serialize / deserialize do not * need to provide the topic name any more. * - * @param topic the name of the topic - * @param keySerde the serde for keys; cannot be null - * @param valueSerde the serde for values; cannot be null + * @param stateName the name of the state + * @param keySerde the serde for keys; cannot be null + * @param valueSerde the serde for values; cannot be null */ @SuppressWarnings("unchecked") - public StateSerdes(String topic, + public StateSerdes(String stateName, Serde keySerde, Serde valueSerde) { - this.topic = topic; + this.stateName = stateName; if (keySerde == null) throw new IllegalArgumentException("key serde cannot be null"); @@ -87,22 +87,22 @@ public Serializer valueSerializer() { } public String topic() { - return topic; + return stateName; } public K keyFrom(byte[] rawKey) { - return keySerde.deserializer().deserialize(topic, rawKey); + return keySerde.deserializer().deserialize(stateName, rawKey); } public V valueFrom(byte[] rawValue) { - return valueSerde.deserializer().deserialize(topic, rawValue); + return valueSerde.deserializer().deserialize(stateName, rawValue); } public byte[] rawKey(K key) { - return keySerde.serializer().serialize(topic, key); + return keySerde.serializer().serialize(stateName, key); } public byte[] rawValue(V value) { - return valueSerde.serializer().serialize(topic, value); + return valueSerde.serializer().serialize(stateName, value); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java index 33df13f79aa00..4e281876b0767 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java @@ -44,8 +44,6 @@ public ValueFactory withKeys(final Serde keySerde) { return new ValueFactory() { @Override public KeyValueFactory withValues(final Serde valueSerde) { - final StateSerdes serdes = - new StateSerdes<>(name, keySerde, valueSerde); return new KeyValueFactory() { @Override public InMemoryKeyValueFactory inMemory() { @@ -62,9 +60,9 @@ public InMemoryKeyValueFactory maxEntries(int capacity) { @Override public StateStoreSupplier build() { if (capacity < Integer.MAX_VALUE) { - return new InMemoryLRUCacheStoreSupplier<>(name, capacity, serdes, null); + return new InMemoryLRUCacheStoreSupplier<>(name, capacity, keySerde, valueSerde); } - return new InMemoryKeyValueStoreSupplier<>(name, serdes, null); + return new InMemoryKeyValueStoreSupplier<>(name, keySerde, valueSerde); } }; } @@ -88,10 +86,10 @@ public PersistentKeyValueFactory windowed(long retentionPeriod, int numSeg @Override public StateStoreSupplier build() { if (numSegments > 0) { - return new RocksDBWindowStoreSupplier<>(name, retentionPeriod, numSegments, retainDuplicates, serdes, null); + return new RocksDBWindowStoreSupplier<>(name, retentionPeriod, numSegments, retainDuplicates, keySerde, valueSerde); } - return new RocksDBKeyValueStoreSupplier<>(name, serdes, null); + return new RocksDBKeyValueStoreSupplier<>(name, keySerde, valueSerde); } }; } @@ -170,8 +168,8 @@ public ValueFactory withKeys(Class keyClass) { /** * Begin to create a {@link KeyValueStore} by specifying the serializer and deserializer for the keys. * - * @param keySerde the serialization factory for keys; may not be null - * @return the interface used to specify the type of values; never null + * @param keySerde the serialization factory for keys; may be null + * @return the interface used to specify the type of values; never null */ public abstract ValueFactory withKeys(Serde keySerde); } @@ -249,8 +247,8 @@ public KeyValueFactory withValues(Class valueClass) { /** * Use the specified serializer and deserializer for the values. * - * @param valueSerde the serialization factory for values; may not be null - * @return the interface used to specify the remaining key-value store options; never null + * @param valueSerde the serialization factory for values; may be null + * @return the interface used to specify the remaining key-value store options; never null */ public abstract KeyValueFactory withValues(Serde valueSerde); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java index 66e13387eb153..fdf3269356127 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java @@ -19,13 +19,22 @@ package org.apache.kafka.streams.state; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; + import java.nio.ByteBuffer; public class WindowStoreUtils { - public static final int TIMESTAMP_SIZE = 8; - public static final int SEQNUM_SIZE = 4; - public static final StateSerdes INNER_SERDES = StateSerdes.withBuiltinTypes("", byte[].class, byte[].class); + private static final int SEQNUM_SIZE = 4; + private static final int TIMESTAMP_SIZE = 8; + + /** Inner byte array serde used for segments */ + public static final Serde INNER_SERDE = Serdes.ByteArray(); + + /** Inner byte array state serde used for segments */ + public static final StateSerdes INNER_SERDES = new StateSerdes<>("", INNER_SERDE, INNER_SERDE); + @SuppressWarnings("unchecked") public static final KeyValueIterator[] NO_ITERATORS = (KeyValueIterator[]) new KeyValueIterator[0]; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java index 32116ddf725f4..efcdac72947db 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueLoggedStore.java @@ -17,8 +17,10 @@ package org.apache.kafka.streams.state.internals; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; @@ -29,16 +31,19 @@ public class InMemoryKeyValueLoggedStore implements KeyValueStore { private final KeyValueStore inner; - private final StateSerdes serdes; + private final Serde keySerde; + private final Serde valueSerde; private final String storeName; + private StateSerdes serdes; private StoreChangeLogger changeLogger; private StoreChangeLogger.ValueGetter getter; - public InMemoryKeyValueLoggedStore(final String storeName, final KeyValueStore inner, final StateSerdes serdes) { + public InMemoryKeyValueLoggedStore(final String storeName, final KeyValueStore inner, Serde keySerde, Serde valueSerde) { this.storeName = storeName; this.inner = inner; - this.serdes = serdes; + this.keySerde = keySerde; + this.valueSerde = valueSerde; } @Override @@ -47,9 +52,24 @@ public String name() { } @Override + @SuppressWarnings("unchecked") public void init(ProcessorContext context, StateStore root) { + // construct the serde + this.serdes = new StateSerdes<>(storeName, + keySerde == null ? (Serde) context.keySerde() : keySerde, + valueSerde == null ? (Serde) context.valueSerde() : valueSerde); + this.changeLogger = new StoreChangeLogger<>(storeName, context, serdes); + context.register(root, true, new StateRestoreCallback() { + @Override + public void restore(byte[] key, byte[] value) { + + // directly call inner functions so that the operation is not logged + inner.put(serdes.keyFrom(key), serdes.valueFrom(value)); + } + }); + inner.init(context, root); this.getter = new StoreChangeLogger.ValueGetter() { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java index 4054d683f381a..3a5819c108f53 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java @@ -17,15 +17,14 @@ package org.apache.kafka.streams.state.internals; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; -import org.apache.kafka.streams.state.StateSerdes; import java.util.Iterator; import java.util.List; @@ -45,12 +44,18 @@ public class InMemoryKeyValueStoreSupplier implements StateStoreSupplier { private final String name; private final Time time; - private final StateSerdes serdes; + private final Serde keySerde; + private final Serde valueSerde; - public InMemoryKeyValueStoreSupplier(String name, StateSerdes serdes, Time time) { + public InMemoryKeyValueStoreSupplier(String name, Serde keySerde, Serde valueSerde) { + this(name, keySerde, valueSerde, null); + } + + public InMemoryKeyValueStoreSupplier(String name, Serde keySerde, Serde valueSerde, Time time) { this.name = name; this.time = time; - this.serdes = serdes; + this.keySerde = keySerde; + this.valueSerde = valueSerde; } public String name() { @@ -58,28 +63,24 @@ public String name() { } public StateStore get() { - return new MeteredKeyValueStore<>(new MemoryStore(name).enableLogging(serdes), "in-memory-state", time); + return new MeteredKeyValueStore<>(new MemoryStore(name, keySerde, valueSerde).enableLogging(), "in-memory-state", time); } private static class MemoryStore implements KeyValueStore { - private final String name; + private final Serde keySerde; + private final Serde valueSerde; private final NavigableMap map; - private boolean loggingEnabled = false; - private StateSerdes serdes = null; - - public MemoryStore(String name) { - super(); + public MemoryStore(String name, Serde keySerde, Serde valueSerde) { this.name = name; + this.keySerde = keySerde; + this.valueSerde = valueSerde; this.map = new TreeMap<>(); } - public KeyValueStore enableLogging(StateSerdes serdes) { - this.loggingEnabled = true; - this.serdes = serdes; - - return new InMemoryKeyValueLoggedStore<>(this.name, this, serdes); + public KeyValueStore enableLogging() { + return new InMemoryKeyValueLoggedStore<>(this.name, this, keySerde, valueSerde); } @Override @@ -88,17 +89,9 @@ public String name() { } @Override + @SuppressWarnings("unchecked") public void init(ProcessorContext context, StateStore root) { - if (loggingEnabled) { - context.register(root, true, new StateRestoreCallback() { - - @Override - public void restore(byte[] key, byte[] value) { - put(serdes.keyFrom(key), serdes.valueFrom(value)); - } - }); - - } + // do nothing } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreSupplier.java index 1c2241fdc1a60..4a4fa5f96af16 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryLRUCacheStoreSupplier.java @@ -16,10 +16,10 @@ */ package org.apache.kafka.streams.state.internals; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreSupplier; -import org.apache.kafka.streams.state.StateSerdes; /** * An in-memory key-value store that is limited in size and retains a maximum number of most recently used entries. @@ -32,13 +32,19 @@ public class InMemoryLRUCacheStoreSupplier implements StateStoreSupplier { private final String name; private final int capacity; - private final StateSerdes serdes; + private final Serde keySerde; + private final Serde valueSerde; private final Time time; - public InMemoryLRUCacheStoreSupplier(String name, int capacity, StateSerdes serdes, Time time) { + public InMemoryLRUCacheStoreSupplier(String name, int capacity, Serde keySerde, Serde valueSerde) { + this(name, capacity, keySerde, valueSerde, null); + } + + public InMemoryLRUCacheStoreSupplier(String name, int capacity, Serde keySerde, Serde valueSerde, Time time) { this.name = name; this.capacity = capacity; - this.serdes = serdes; + this.keySerde = keySerde; + this.valueSerde = valueSerde; this.time = time; } @@ -49,7 +55,7 @@ public String name() { @SuppressWarnings("unchecked") public StateStore get() { final MemoryNavigableLRUCache cache = new MemoryNavigableLRUCache(name, capacity); - final InMemoryKeyValueLoggedStore loggedCache = (InMemoryKeyValueLoggedStore) cache.enableLogging(serdes); + final InMemoryKeyValueLoggedStore loggedCache = (InMemoryKeyValueLoggedStore) cache.enableLogging(keySerde, valueSerde); final MeteredKeyValueStore store = new MeteredKeyValueStore<>(loggedCache, "in-memory-lru-state", time); cache.whenEldestRemoved(new MemoryNavigableLRUCache.EldestEntryRemovalListener() { @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java index a5aaa0666afa7..a859bd2cbbe1c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java @@ -16,13 +16,12 @@ */ package org.apache.kafka.streams.state.internals; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.processor.StateRestoreCallback; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.state.KeyValueIterator; import org.apache.kafka.streams.state.KeyValueStore; -import org.apache.kafka.streams.state.StateSerdes; import java.util.HashSet; import java.util.LinkedHashMap; @@ -42,9 +41,6 @@ public interface EldestEntryRemovalListener { protected EldestEntryRemovalListener listener; - private boolean loggingEnabled = false; - private StateSerdes serdes = null; - // this is used for extended MemoryNavigableLRUCache only public MemoryLRUCache() {} @@ -69,11 +65,8 @@ protected boolean removeEldestEntry(Map.Entry eldest) { }; } - public KeyValueStore enableLogging(StateSerdes serdes) { - this.loggingEnabled = true; - this.serdes = serdes; - - return new InMemoryKeyValueLoggedStore<>(this.name, this, serdes); + public KeyValueStore enableLogging(Serde keySerde, Serde valueSerde) { + return new InMemoryKeyValueLoggedStore<>(this.name, this, keySerde, valueSerde); } public MemoryLRUCache whenEldestRemoved(EldestEntryRemovalListener listener) { @@ -88,17 +81,9 @@ public String name() { } @Override + @SuppressWarnings("unchecked") public void init(ProcessorContext context, StateStore root) { - if (loggingEnabled) { - context.register(root, true, new StateRestoreCallback() { - - @Override - public void restore(byte[] key, byte[] value) { - put(serdes.keyFrom(key), serdes.valueFrom(value)); - } - }); - - } + // do nothing } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java index ec10c3fdfa6cd..af9873366b18b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBKeyValueStoreSupplier.java @@ -17,10 +17,10 @@ package org.apache.kafka.streams.state.internals; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreSupplier; -import org.apache.kafka.streams.state.StateSerdes; /** * A {@link org.apache.kafka.streams.state.KeyValueStore} that stores all entries in a local RocksDB database. @@ -33,12 +33,18 @@ public class RocksDBKeyValueStoreSupplier implements StateStoreSupplier { private final String name; - private final StateSerdes serdes; + private final Serde keySerde; + private final Serde valueSerde; private final Time time; - public RocksDBKeyValueStoreSupplier(String name, StateSerdes serdes, Time time) { + public RocksDBKeyValueStoreSupplier(String name, Serde keySerde, Serde valueSerde) { + this(name, keySerde, valueSerde, null); + } + + public RocksDBKeyValueStoreSupplier(String name, Serde keySerde, Serde valueSerde, Time time) { this.name = name; - this.serdes = serdes; + this.keySerde = keySerde; + this.valueSerde = valueSerde; this.time = time; } @@ -47,6 +53,6 @@ public String name() { } public StateStore get() { - return new MeteredKeyValueStore<>(new RocksDBStore<>(name, serdes).enableLogging(), "rocksdb-state", time); + return new MeteredKeyValueStore<>(new RocksDBStore<>(name, keySerde, valueSerde).enableLogging(), "rocksdb-state", time); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index 3045856451469..b206f37962afd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.state.internals; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.processor.ProcessorContext; @@ -67,7 +68,9 @@ public class RocksDBStore implements KeyValueStore { private final WriteOptions wOptions; private final FlushOptions fOptions; - private ProcessorContext context; + private final Serde keySerde; + private final Serde valueSerde; + private StateSerdes serdes; protected File dbDir; private RocksDB db; @@ -92,14 +95,15 @@ public RocksDBStore withCacheSize(int cacheSize) { return this; } - public RocksDBStore(String name, StateSerdes serdes) { - this(name, DB_FILE_DIR, serdes); + public RocksDBStore(String name, Serde keySerde, Serde valueSerde) { + this(name, DB_FILE_DIR, keySerde, valueSerde); } - public RocksDBStore(String name, String parentDir, StateSerdes serdes) { + public RocksDBStore(String name, String parentDir, Serde keySerde, Serde valueSerde) { this.name = name; this.parentDir = parentDir; - this.serdes = serdes; + this.keySerde = keySerde; + this.valueSerde = valueSerde; // initialize the rocksdb options BlockBasedTableConfig tableConfig = new BlockBasedTableConfig(); @@ -136,15 +140,20 @@ public RocksDBCacheEntry(V value, boolean isDirty) { } } + @SuppressWarnings("unchecked") public void openDB(ProcessorContext context) { - this.context = context; - this.dbDir = new File(new File(this.context.stateDir(), parentDir), this.name); + // we need to construct the serde while opening DB since + // it is also triggered by windowed DB segments without initialization + this.serdes = new StateSerdes<>(name, + keySerde == null ? (Serde) context.keySerde() : keySerde, + valueSerde == null ? (Serde) context.valueSerde() : valueSerde); + + this.dbDir = new File(new File(context.stateDir(), parentDir), this.name); this.db = openDB(this.dbDir, this.options, TTL_SECONDS); } - @SuppressWarnings("unchecked") public void init(ProcessorContext context, StateStore root) { - // first open the DB dir + // open the DB dir openDB(context); this.changeLogger = this.loggingEnabled ? new RawStoreChangeLogger(name, context) : null; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java index 61c2e5eec8733..4c6a2296b6325 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java @@ -19,6 +19,7 @@ package org.apache.kafka.streams.state.internals; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.ProcessorContext; @@ -51,7 +52,7 @@ private static class Segment extends RocksDBStore { public final long id; Segment(String segmentName, String windowName, long id) { - super(segmentName, windowName, WindowStoreUtils.INNER_SERDES); + super(segmentName, windowName, WindowStoreUtils.INNER_SERDE, WindowStoreUtils.INNER_SERDE); this.id = id; } @@ -114,7 +115,8 @@ public void close() { private final long segmentInterval; private final boolean retainDuplicates; private final Segment[] segments; - private final StateSerdes serdes; + private final Serde keySerde; + private final Serde valueSerde; private final SimpleDateFormat formatter; private final StoreChangeLogger.ValueGetter getter; @@ -122,17 +124,20 @@ public void close() { private int seqnum = 0; private long currentSegmentId = -1L; + private StateSerdes serdes; + private boolean loggingEnabled = false; private StoreChangeLogger changeLogger = null; - public RocksDBWindowStore(String name, long retentionPeriod, int numSegments, boolean retainDuplicates, StateSerdes serdes) { + public RocksDBWindowStore(String name, long retentionPeriod, int numSegments, boolean retainDuplicates, Serde keySerde, Serde valueSerde) { this.name = name; // The segment interval must be greater than MIN_SEGMENT_INTERVAL this.segmentInterval = Math.max(retentionPeriod / (numSegments - 1), MIN_SEGMENT_INTERVAL); this.segments = new Segment[numSegments]; - this.serdes = serdes; + this.keySerde = keySerde; + this.valueSerde = valueSerde; this.retainDuplicates = retainDuplicates; @@ -159,13 +164,18 @@ public String name() { } @Override + @SuppressWarnings("unchecked") public void init(ProcessorContext context, StateStore root) { this.context = context; + // construct the serde + this.serdes = new StateSerdes<>(name, + keySerde == null ? (Serde) context.keySerde() : keySerde, + valueSerde == null ? (Serde) context.valueSerde() : valueSerde); + openExistingSegments(); - this.changeLogger = this.loggingEnabled ? - new RawStoreChangeLogger(name, context) : null; + this.changeLogger = this.loggingEnabled ? new RawStoreChangeLogger(name, context) : null; // register and possibly restore the state from the logs context.register(root, loggingEnabled, new StateRestoreCallback() { @@ -202,7 +212,7 @@ private void openExistingSegments() { dir.mkdir(); } } catch (Exception ex) { - + // ignore } } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java index 00089abebb0e3..0407299562a83 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreSupplier.java @@ -17,10 +17,10 @@ package org.apache.kafka.streams.state.internals; +import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.utils.Time; import org.apache.kafka.streams.processor.StateStore; import org.apache.kafka.streams.processor.StateStoreSupplier; -import org.apache.kafka.streams.state.StateSerdes; /** * A {@link org.apache.kafka.streams.state.KeyValueStore} that stores all entries in a local RocksDB database. @@ -36,15 +36,21 @@ public class RocksDBWindowStoreSupplier implements StateStoreSupplier { private final long retentionPeriod; private final boolean retainDuplicates; private final int numSegments; - private final StateSerdes serdes; + private final Serde keySerde; + private final Serde valueSerde; private final Time time; - public RocksDBWindowStoreSupplier(String name, long retentionPeriod, int numSegments, boolean retainDuplicates, StateSerdes serdes, Time time) { + public RocksDBWindowStoreSupplier(String name, long retentionPeriod, int numSegments, boolean retainDuplicates, Serde keySerde, Serde valueSerde) { + this(name, retentionPeriod, numSegments, retainDuplicates, keySerde, valueSerde, null); + } + + public RocksDBWindowStoreSupplier(String name, long retentionPeriod, int numSegments, boolean retainDuplicates, Serde keySerde, Serde valueSerde, Time time) { this.name = name; this.retentionPeriod = retentionPeriod; this.retainDuplicates = retainDuplicates; this.numSegments = numSegments; - this.serdes = serdes; + this.keySerde = keySerde; + this.valueSerde = valueSerde; this.time = time; } @@ -53,7 +59,7 @@ public String name() { } public StateStore get() { - return new MeteredWindowStore<>(new RocksDBWindowStore<>(name, retentionPeriod, numSegments, retainDuplicates, serdes).enableLogging(), "rocksdb-window", time); + return new MeteredWindowStore<>(new RocksDBWindowStore<>(name, retentionPeriod, numSegments, retainDuplicates, keySerde, valueSerde).enableLogging(), "rocksdb-window", time); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java index 20c3a287cab14..6f49b6a870577 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java @@ -72,7 +72,7 @@ public boolean test(String key, Integer value) { MockProcessorSupplier proc3 = new MockProcessorSupplier<>(); table3.toStream().process(proc3); - KTable table4 = table1.through(topic2, stringSerde, stringSerde); + KTable table4 = table1.through(stringSerde, stringSerde, topic2); MockProcessorSupplier proc4 = new MockProcessorSupplier<>(); table4.toStream().process(proc4); @@ -116,7 +116,7 @@ public boolean test(String key, Integer value) { } }); KTableImpl table4 = (KTableImpl) - table1.through(topic2, stringSerde, stringSerde); + table1.through(stringSerde, stringSerde, topic2); KTableValueGetterSupplier getterSupplier1 = table1.valueGetterSupplier(); KTableValueGetterSupplier getterSupplier2 = table2.valueGetterSupplier(); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java index aa3daeb8de574..9ec12583b5bbd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java @@ -94,7 +94,7 @@ public boolean test(String key, Integer value) { } }); KTableImpl table4 = (KTableImpl) - table1.through(topic2, stringSerde, stringSerde); + table1.through(stringSerde, stringSerde, topic2); KTableValueGetterSupplier getterSupplier1 = table1.valueGetterSupplier(); KTableValueGetterSupplier getterSupplier2 = table2.valueGetterSupplier(); diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java index ce4956c1453a4..0a02824d1fadc 100644 --- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java +++ b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java @@ -94,7 +94,7 @@ private static KafkaStreams createKafkaStreams(File stateDir, String kafka, Stri KStream source = builder.stream(stringSerde, intSerde, "data"); - source.to("echo", stringSerde, intSerde); + source.to(stringSerde, intSerde, "echo"); KStream data = source.filter(new Predicate() { @Override @@ -123,7 +123,7 @@ public Integer apply(String aggKey, Integer value, Integer aggregate) { intSerde ).toStream().map( new Unwindow() - ).to("min", stringSerde, intSerde); + ).to(stringSerde, intSerde, "min"); KTable minTable = builder.table(stringSerde, intSerde, "min"); minTable.toStream().process(SmokeTestUtil.printProcessorSupplier("min")); @@ -146,7 +146,7 @@ public Integer apply(String aggKey, Integer value, Integer aggregate) { intSerde ).toStream().map( new Unwindow() - ).to("max", stringSerde, intSerde); + ).to(stringSerde, intSerde, "max"); KTable maxTable = builder.table(stringSerde, intSerde, "max"); maxTable.toStream().process(SmokeTestUtil.printProcessorSupplier("max")); @@ -169,7 +169,7 @@ public Long apply(String aggKey, Integer value, Long aggregate) { longSerde ).toStream().map( new Unwindow() - ).to("sum", stringSerde, longSerde); + ).to(stringSerde, longSerde, "sum"); KTable sumTable = builder.table(stringSerde, longSerde, "sum"); @@ -181,7 +181,7 @@ public Long apply(String aggKey, Integer value, Long aggregate) { stringSerde ).toStream().map( new Unwindow() - ).to("cnt", stringSerde, longSerde); + ).to(stringSerde, longSerde, "cnt"); KTable cntTable = builder.table(stringSerde, longSerde, "cnt"); cntTable.toStream().process(SmokeTestUtil.printProcessorSupplier("cnt")); @@ -193,7 +193,7 @@ public Integer apply(Integer value1, Integer value2) { return value1 - value2; } } - ).to("dif", stringSerde, intSerde); + ).to(stringSerde, intSerde, "dif"); // avg sumTable.join( @@ -203,7 +203,7 @@ public Double apply(Long value1, Long value2) { return (double) value1 / (double) value2; } } - ).to("avg", stringSerde, doubleSerde); + ).to(stringSerde, doubleSerde, "avg"); // windowed count data.countByKey( @@ -216,7 +216,7 @@ public KeyValue apply(Windowed key, Long value) { return new KeyValue<>(key.value() + "@" + key.window().start(), value); } } - ).to("wcnt", stringSerde, longSerde); + ).to(stringSerde, longSerde, "wcnt"); // test repartition Agg agg = new Agg(); @@ -229,7 +229,7 @@ public KeyValue apply(Windowed key, Long value) { longSerde, longSerde, "cntByCnt" - ).to("tagg", stringSerde, longSerde); + ).to(stringSerde, longSerde, "tagg"); return new KafkaStreams(builder, props); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java index ffc97c3e7a4d7..502870b2a5448 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java @@ -57,11 +57,13 @@ public class RocksDBWindowStoreTest { private final long segmentSize = RocksDBWindowStore.MIN_SEGMENT_INTERVAL; private final long retentionPeriod = segmentSize * (numSegments - 1); private final long windowSize = 3; - private final StateSerdes serdes = StateSerdes.withBuiltinTypes("", Integer.class, String.class); + private final Serde intSerde = Serdes.Integer(); + private final Serde stringSerde = Serdes.String(); + private final StateSerdes serdes = new StateSerdes<>("", intSerde, stringSerde); @SuppressWarnings("unchecked") - protected WindowStore createWindowStore(ProcessorContext context, StateSerdes serdes) { - StateStoreSupplier supplier = new RocksDBWindowStoreSupplier<>(windowName, retentionPeriod, numSegments, true, serdes, null); + protected WindowStore createWindowStore(ProcessorContext context) { + StateStoreSupplier supplier = new RocksDBWindowStoreSupplier<>(windowName, retentionPeriod, numSegments, true, intSerde, stringSerde); WindowStore store = (WindowStore) supplier.get(); store.init(context, store); @@ -89,7 +91,7 @@ public void send(ProducerRecord record, Serializer keySeria byteArraySerde, byteArraySerde, recordCollector); - WindowStore store = createWindowStore(context, serdes); + WindowStore store = createWindowStore(context); try { long startTime = segmentSize - 4L; @@ -185,7 +187,7 @@ public void send(ProducerRecord record, Serializer keySeria byteArraySerde, byteArraySerde, recordCollector); - WindowStore store = createWindowStore(context, serdes); + WindowStore store = createWindowStore(context); try { long startTime = segmentSize - 4L; @@ -281,7 +283,7 @@ public void send(ProducerRecord record, Serializer keySeria byteArraySerde, byteArraySerde, recordCollector); - WindowStore store = createWindowStore(context, serdes); + WindowStore store = createWindowStore(context); try { long startTime = segmentSize - 4L; @@ -377,7 +379,7 @@ public void send(ProducerRecord record, Serializer keySeria byteArraySerde, byteArraySerde, recordCollector); - WindowStore store = createWindowStore(context, serdes); + WindowStore store = createWindowStore(context); try { long startTime = segmentSize - 4L; @@ -436,7 +438,7 @@ public void send(ProducerRecord record, Serializer keySeria byteArraySerde, byteArraySerde, recordCollector); - WindowStore store = createWindowStore(context, serdes); + WindowStore store = createWindowStore(context); RocksDBWindowStore inner = (RocksDBWindowStore) ((MeteredWindowStore) store).inner(); try { @@ -553,7 +555,7 @@ public void send(ProducerRecord record, Serializer keySeria byteArraySerde, byteArraySerde, recordCollector); - WindowStore store = createWindowStore(context, serdes); + WindowStore store = createWindowStore(context); try { context.setTime(startTime); store.put(0, "zero"); @@ -602,7 +604,7 @@ public void send(ProducerRecord record, Serializer keySeria byteArraySerde, byteArraySerde, recordCollector); - WindowStore store = createWindowStore(context, serdes); + WindowStore store = createWindowStore(context); RocksDBWindowStore inner = (RocksDBWindowStore) ((MeteredWindowStore) store).inner(); @@ -654,7 +656,7 @@ public void send(ProducerRecord record, Serializer keySeria byteArraySerde, byteArraySerde, recordCollector); - WindowStore store = createWindowStore(context, serdes); + WindowStore store = createWindowStore(context); RocksDBWindowStore inner = (RocksDBWindowStore) ((MeteredWindowStore) store).inner(); @@ -759,7 +761,7 @@ public void send(ProducerRecord record, Serializer keySeria File storeDir = new File(baseDir, windowName); - WindowStore store = createWindowStore(context, serdes); + WindowStore store = createWindowStore(context); RocksDBWindowStore inner = (RocksDBWindowStore) ((MeteredWindowStore) store).inner(); @@ -775,7 +777,7 @@ public void send(ProducerRecord record, Serializer keySeria store.close(); } - store = createWindowStore(context, serdes); + store = createWindowStore(context); inner = (RocksDBWindowStore) ((MeteredWindowStore) store).inner(); try { From d9bf55171ba41962edc05cdf106f540cf94a2c3e Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Fri, 18 Mar 2016 13:37:33 -0700 Subject: [PATCH 054/206] KAFKA-3394; allow null offset metadata in commit API Author: Jason Gustafson Reviewers: Ismael Juma , Grant Henke , Ewen Cheslack-Postava , Jun Rao Closes #1064 from hachikuji/KAFKA-3394 --- .../clients/consumer/OffsetAndMetadata.java | 3 --- .../kafka/common/protocol/Protocol.java | 8 +++---- .../common/requests/OffsetCommitResponse.java | 5 +++++ .../common/requests/RequestResponseTest.java | 21 +++++++++++++++---- .../main/scala/kafka/server/KafkaApis.scala | 3 ++- .../kafka/api/PlaintextConsumerTest.scala | 6 +++++- 6 files changed, 33 insertions(+), 13 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndMetadata.java b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndMetadata.java index 66b257dba3099..df8bf37adbada 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/OffsetAndMetadata.java @@ -29,9 +29,6 @@ public class OffsetAndMetadata implements Serializable { * @param metadata Non-null metadata */ public OffsetAndMetadata(long offset, String metadata) { - if (metadata == null) - throw new IllegalArgumentException("Metadata cannot be null"); - this.offset = offset; this.metadata = metadata; } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index e32d0b6cd4353..43110b5152899 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -178,7 +178,7 @@ public class Protocol { INT64, "Message offset to be committed."), new Field("metadata", - STRING, + NULLABLE_STRING, "Any associated metadata the client wants to keep.")); public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V1 = new Schema(new Field("partition", @@ -191,7 +191,7 @@ public class Protocol { INT64, "Timestamp of the commit"), new Field("metadata", - STRING, + NULLABLE_STRING, "Any associated metadata the client wants to keep.")); public static final Schema OFFSET_COMMIT_REQUEST_PARTITION_V2 = new Schema(new Field("partition", @@ -201,7 +201,7 @@ public class Protocol { INT64, "Message offset to be committed."), new Field("metadata", - STRING, + NULLABLE_STRING, "Any associated metadata the client wants to keep.")); public static final Schema OFFSET_COMMIT_REQUEST_TOPIC_V0 = new Schema(new Field("topic", @@ -314,7 +314,7 @@ public class Protocol { INT64, "Last committed message offset."), new Field("metadata", - STRING, + NULLABLE_STRING, "Any associated metadata the client wants to keep."), new Field("error_code", INT16)); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java index 9b53fb4b94134..71dd490363fe2 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/OffsetCommitResponse.java @@ -97,6 +97,11 @@ public Map responseData() { return responseData; } + public static OffsetCommitResponse parse(ByteBuffer buffer, int version) { + Schema schema = ProtoUtils.responseSchema(ApiKeys.OFFSET_COMMIT.id, version); + return new OffsetCommitResponse(schema.read(buffer)); + } + public static OffsetCommitResponse parse(ByteBuffer buffer) { return new OffsetCommitResponse(CURRENT_SCHEMA.read(buffer)); } diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index b556b4647468e..b7f0caf192595 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -74,8 +74,8 @@ public void testSerialization() throws Exception { createMetadataRequest(), createMetadataRequest().getErrorResponse(0, new UnknownServerException()), createMetadataResponse(), - createOffsetCommitRequest(), - createOffsetCommitRequest().getErrorResponse(0, new UnknownServerException()), + createOffsetCommitRequest(2), + createOffsetCommitRequest(2).getErrorResponse(2, new UnknownServerException()), createOffsetCommitResponse(), createOffsetFetchRequest(), createOffsetFetchRequest().getErrorResponse(0, new UnknownServerException()), @@ -98,6 +98,10 @@ public void testSerialization() throws Exception { for (AbstractRequestResponse req : requestResponseList) checkSerialization(req, null); + checkSerialization(createOffsetCommitRequest(0), 0); + checkSerialization(createOffsetCommitRequest(0).getErrorResponse(0, new UnknownServerException()), 0); + checkSerialization(createOffsetCommitRequest(1), 1); + checkSerialization(createOffsetCommitRequest(1).getErrorResponse(1, new UnknownServerException()), 1); checkSerialization(createUpdateMetadataRequest(0, null), 0); checkSerialization(createUpdateMetadataRequest(0, null).getErrorResponse(0, new UnknownServerException()), 0); checkSerialization(createUpdateMetadataRequest(1, null), 1); @@ -292,10 +296,18 @@ private AbstractRequestResponse createMetadataResponse() { return new MetadataResponse(Arrays.asList(node), allTopicMetadata); } - private AbstractRequest createOffsetCommitRequest() { + private AbstractRequest createOffsetCommitRequest(int version) { Map commitData = new HashMap<>(); commitData.put(new TopicPartition("test", 0), new OffsetCommitRequest.PartitionData(100, "")); - return new OffsetCommitRequest("group1", 100, "consumer1", 1000000, commitData); + commitData.put(new TopicPartition("test", 1), new OffsetCommitRequest.PartitionData(200, null)); + if (version == 0) { + return new OffsetCommitRequest("group1", commitData); + } else if (version == 1) { + return new OffsetCommitRequest("group1", 100, "consumer1", commitData); + } else if (version == 2) { + return new OffsetCommitRequest("group1", 100, "consumer1", 1000000, commitData); + } + throw new IllegalArgumentException("Unknown offset commit request version " + version); } private AbstractRequestResponse createOffsetCommitResponse() { @@ -311,6 +323,7 @@ private AbstractRequest createOffsetFetchRequest() { private AbstractRequestResponse createOffsetFetchResponse() { Map responseData = new HashMap<>(); responseData.put(new TopicPartition("test", 0), new OffsetFetchResponse.PartitionData(100L, "", Errors.NONE.code())); + responseData.put(new TopicPartition("test", 1), new OffsetFetchResponse.PartitionData(100L, null, Errors.NONE.code())); return new OffsetFetchResponse(responseData); } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 0fb4d74cab576..4f77d30273427 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -294,8 +294,9 @@ class KafkaApis(val requestChannel: RequestChannel, val currentTimestamp = SystemTime.milliseconds val defaultExpireTimestamp = offsetRetention + currentTimestamp val partitionData = authorizedRequestInfo.mapValues { partitionData => + val metadata = if (partitionData.metadata == null) OffsetMetadata.NoMetadata else partitionData.metadata; new OffsetAndMetadata( - offsetMetadata = OffsetMetadata(partitionData.offset, partitionData.metadata), + offsetMetadata = OffsetMetadata(partitionData.offset, metadata), commitTimestamp = currentTimestamp, expireTimestamp = { if (partitionData.timestamp == OffsetCommitRequest.DEFAULT_TIMESTAMP) diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 801447956c725..ca0497b0c6927 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -224,8 +224,12 @@ class PlaintextConsumerTest extends BaseConsumerTest { val callback = new CountConsumerCommitCallback this.consumers(0).commitAsync(Map((tp, asyncMetadata)).asJava, callback) awaitCommitCallback(this.consumers(0), callback) - assertEquals(asyncMetadata, this.consumers(0).committed(tp)) + + // handle null metadata + val nullMetadata = new OffsetAndMetadata(5, null) + this.consumers(0).commitSync(Map((tp, nullMetadata)).asJava) + assertEquals(nullMetadata, this.consumers(0).committed(tp)) } @Test From 07360cb0d24b18f6f7c8a35d437c7cb2b0142506 Mon Sep 17 00:00:00 2001 From: Xin Wang Date: Fri, 18 Mar 2016 13:43:13 -0700 Subject: [PATCH 055/206] KAFKA-3397: use -1(latest) as time default value for tools.GetOffsetShell minor fix for prompt the user, or we will get a error message: >Missing required argument "[time]" Author: Xin Wang Reviewers: Ashish Singh, Guozhang Wang Closes #1068 from vesense/patch-4 --- core/src/main/scala/kafka/tools/GetOffsetShell.scala | 1 + 1 file changed, 1 insertion(+) diff --git a/core/src/main/scala/kafka/tools/GetOffsetShell.scala b/core/src/main/scala/kafka/tools/GetOffsetShell.scala index 3d9293e4abbe3..30c7afe7edca8 100644 --- a/core/src/main/scala/kafka/tools/GetOffsetShell.scala +++ b/core/src/main/scala/kafka/tools/GetOffsetShell.scala @@ -47,6 +47,7 @@ object GetOffsetShell { .withRequiredArg .describedAs("timestamp/-1(latest)/-2(earliest)") .ofType(classOf[java.lang.Long]) + .defaultsTo(-1) val nOffsetsOpt = parser.accepts("offsets", "number of offsets returned") .withRequiredArg .describedAs("count") From 4332175c11dda5deb491f27a6ecf66661676ca47 Mon Sep 17 00:00:00 2001 From: Pierre-Yves Ritschard Date: Fri, 18 Mar 2016 16:07:20 -0700 Subject: [PATCH 056/206] KAFKA-3006: standardize KafkaConsumer API to use Collection Author: Pierre-Yves Ritschard Reviewers: Jason Gustafson, Gwen Shapira Closes #1098 from hachikuji/KAFKA-3006 --- .../kafka/clients/consumer/Consumer.java | 35 ++++---- .../kafka/clients/consumer/KafkaConsumer.java | 85 +++++++++---------- .../kafka/clients/consumer/MockConsumer.java | 16 ++-- .../consumer/internals/SubscriptionState.java | 7 +- .../clients/consumer/KafkaConsumerTest.java | 29 ++++--- .../kafka/connect/runtime/WorkerSinkTask.java | 9 +- .../runtime/WorkerSinkTaskContext.java | 5 +- .../kafka/connect/util/KafkaBasedLog.java | 6 +- .../connect/runtime/WorkerSinkTaskTest.java | 14 +-- .../runtime/WorkerSinkTaskThreadedTest.java | 8 +- .../kafka/admin/ConsumerGroupCommand.scala | 2 +- .../kafka/tools/ConsumerPerformance.scala | 3 +- .../scala/kafka/tools/EndToEndLatency.scala | 5 +- .../kafka/api/BaseConsumerTest.scala | 6 +- .../kafka/api/ConsumerBounceTest.scala | 4 +- .../kafka/api/PlaintextConsumerTest.scala | 17 ++-- .../internals/ProcessorStateManager.java | 6 +- .../processor/internals/StreamTask.java | 6 +- .../processor/internals/StreamThread.java | 8 +- .../internals/ProcessorStateManagerTest.java | 14 +-- .../processor/internals/StandbyTaskTest.java | 5 +- .../streams/smoketest/SmokeTestDriver.java | 2 +- .../test/ProcessorTopologyTestDriver.java | 5 +- 23 files changed, 156 insertions(+), 141 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java index c0f3030a46bd3..0862c326c1ade 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -19,6 +19,7 @@ import org.apache.kafka.common.annotation.InterfaceStability; import java.io.Closeable; +import java.util.Collection; import java.util.List; import java.util.Map; import java.util.Set; @@ -30,7 +31,7 @@ */ @InterfaceStability.Unstable public interface Consumer extends Closeable { - + /** * @see KafkaConsumer#assignment() */ @@ -42,19 +43,19 @@ public interface Consumer extends Closeable { public Set subscription(); /** - * @see KafkaConsumer#subscribe(List) + * @see KafkaConsumer#subscribe(Collection) */ - public void subscribe(List topics); + public void subscribe(Collection topics); /** - * @see KafkaConsumer#subscribe(List, ConsumerRebalanceListener) + * @see KafkaConsumer#subscribe(Collection, ConsumerRebalanceListener) */ - public void subscribe(List topics, ConsumerRebalanceListener callback); + public void subscribe(Collection topics, ConsumerRebalanceListener callback); /** - * @see KafkaConsumer#assign(List) + * @see KafkaConsumer#assign(Collection) */ - public void assign(List partitions); + public void assign(Collection partitions); /** * @see KafkaConsumer#subscribe(Pattern, ConsumerRebalanceListener) @@ -102,14 +103,14 @@ public interface Consumer extends Closeable { public void seek(TopicPartition partition, long offset); /** - * @see KafkaConsumer#seekToBeginning(TopicPartition...) + * @see KafkaConsumer#seekToBeginning(Collection) */ - public void seekToBeginning(TopicPartition... partitions); + public void seekToBeginning(Collection partitions); /** - * @see KafkaConsumer#seekToEnd(TopicPartition...) + * @see KafkaConsumer#seekToEnd(Collection) */ - public void seekToEnd(TopicPartition... partitions); + public void seekToEnd(Collection partitions); /** * @see KafkaConsumer#position(TopicPartition) @@ -137,19 +138,19 @@ public interface Consumer extends Closeable { public Map> listTopics(); /** - * @see KafkaConsumer#pause(TopicPartition...) + * @see KafkaConsumer#paused() */ - public void pause(TopicPartition... partitions); + public Set paused(); /** - * @see KafkaConsumer#paused() + * @see KafkaConsumer#pause(Collection) */ - public Set paused(); + public void pause(Collection partitions); /** - * @see KafkaConsumer#resume(TopicPartition...) + * @see KafkaConsumer#resume(Collection) */ - public void resume(TopicPartition... partitions); + public void resume(Collection partitions); /** * @see KafkaConsumer#close() diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 804a16036101e..b7eafbe346f45 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -16,12 +16,12 @@ import org.apache.kafka.clients.Metadata; import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.clients.consumer.internals.ConsumerCoordinator; +import org.apache.kafka.clients.consumer.internals.ConsumerInterceptors; import org.apache.kafka.clients.consumer.internals.ConsumerNetworkClient; import org.apache.kafka.clients.consumer.internals.Fetcher; import org.apache.kafka.clients.consumer.internals.NoOpConsumerRebalanceListener; import org.apache.kafka.clients.consumer.internals.PartitionAssignor; import org.apache.kafka.clients.consumer.internals.SubscriptionState; -import org.apache.kafka.clients.consumer.internals.ConsumerInterceptors; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.Metric; @@ -45,7 +45,6 @@ import org.slf4j.LoggerFactory; import java.net.InetSocketAddress; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.ConcurrentModificationException; @@ -100,7 +99,7 @@ * distributed over many machines to provide additional scalability and fault tolerance for processing. *

      * Each Kafka consumer is able to configure a consumer group that it belongs to, and can dynamically set the - * list of topics it wants to subscribe to through {@link #subscribe(List, ConsumerRebalanceListener)}, + * list of topics it wants to subscribe to through {@link #subscribe(Collection, ConsumerRebalanceListener)}, * or subscribe to all topics matching certain pattern through {@link #subscribe(Pattern, ConsumerRebalanceListener)}. * Kafka will deliver each message in the * subscribed topics to one process in each consumer group. This is achieved by balancing the partitions in the topic @@ -127,7 +126,7 @@ * commits (note that offsets are always committed for a given consumer group), etc. * See Storing Offsets Outside Kafka for more details *

      - * It is also possible for the consumer to manually specify the partitions that are assigned to it through {@link #assign(List)}, + * It is also possible for the consumer to manually specify the partitions that are assigned to it through {@link #assign(Collection)}, * which disables this dynamic partition assignment. * *

      Usage Examples

      @@ -313,7 +312,7 @@ * This type of usage is simplest when the partition assignment is also done manually (this would be likely in the * search index use case described above). If the partition assignment is done automatically special care is * needed to handle the case where partition assignments change. This can be done by providing a - * {@link ConsumerRebalanceListener} instance in the call to {@link #subscribe(List, ConsumerRebalanceListener)} + * {@link ConsumerRebalanceListener} instance in the call to {@link #subscribe(Collection, ConsumerRebalanceListener)} * and {@link #subscribe(Pattern, ConsumerRebalanceListener)}. * For example, when partitions are taken from a consumer the consumer will want to commit its offset for those partitions by * implementing {@link ConsumerRebalanceListener#onPartitionsRevoked(Collection)}. When partitions are assigned to a @@ -342,7 +341,7 @@ *

      * Kafka allows specifying the position using {@link #seek(TopicPartition, long)} to specify the new position. Special * methods for seeking to the earliest and latest offset the server maintains are also available ( - * {@link #seekToBeginning(TopicPartition...)} and {@link #seekToEnd(TopicPartition...)} respectively). + * {@link #seekToBeginning(Collection)} and {@link #seekToEnd(Collection)} respectively). * *

      Consumption Flow Control

      * @@ -359,7 +358,7 @@ * fetching other topics. * *

      - * Kafka supports dynamic controlling of consumption flows by using {@link #pause(TopicPartition...)} and {@link #resume(TopicPartition...)} + * Kafka supports dynamic controlling of consumption flows by using {@link #pause(Collection)} and {@link #resume(Collection)} * to pause the consumption on the specified assigned partitions and resume the consumption * on the specified paused partitions respectively in the future {@link #poll(long)} calls. * @@ -660,7 +659,7 @@ private KafkaConsumer(ConsumerConfig config, /** * Get the set of partitions currently assigned to this consumer. If subscription happened by directly assigning - * partitions using {@link #assign(List)} then this will simply return the same partitions that + * partitions using {@link #assign(Collection)} then this will simply return the same partitions that * were assigned. If topic subscription was used, then this will give the set of topic partitions currently assigned * to the consumer (which may be none if the assignment hasn't happened yet, or the partitions are in the * process of getting reassigned). @@ -677,7 +676,7 @@ public Set assignment() { /** * Get the current subscription. Will return the same topics used in the most recent call to - * {@link #subscribe(List, ConsumerRebalanceListener)}, or an empty set if no such call has been made. + * {@link #subscribe(Collection, ConsumerRebalanceListener)}, or an empty set if no such call has been made. * @return The set of topics currently subscribed to */ public Set subscription() { @@ -693,7 +692,7 @@ public Set subscription() { * Subscribe to the given list of topics to get dynamically * assigned partitions. Topic subscriptions are not incremental. This list will replace the current * assignment (if there is one). Note that it is not possible to combine topic subscription with group management - * with manual partition assignment through {@link #assign(List)}. + * with manual partition assignment through {@link #assign(Collection)}. * * If the given list of topics is empty, it is treated the same as {@link #unsubscribe()}. * @@ -718,7 +717,7 @@ public Set subscription() { * subscribed topics */ @Override - public void subscribe(List topics, ConsumerRebalanceListener listener) { + public void subscribe(Collection topics, ConsumerRebalanceListener listener) { acquire(); try { if (topics.isEmpty()) { @@ -738,21 +737,21 @@ public void subscribe(List topics, ConsumerRebalanceListener listener) { * Subscribe to the given list of topics to get dynamically assigned partitions. * Topic subscriptions are not incremental. This list will replace the current * assignment (if there is one). It is not possible to combine topic subscription with group management - * with manual partition assignment through {@link #assign(List)}. + * with manual partition assignment through {@link #assign(Collection)}. * * If the given list of topics is empty, it is treated the same as {@link #unsubscribe()}. * *

      - * This is a short-hand for {@link #subscribe(List, ConsumerRebalanceListener)}, which + * This is a short-hand for {@link #subscribe(Collection, ConsumerRebalanceListener)}, which * uses a noop listener. If you need the ability to either seek to particular offsets, you should prefer - * {@link #subscribe(List, ConsumerRebalanceListener)}, since group rebalances will cause partition offsets + * {@link #subscribe(Collection, ConsumerRebalanceListener)}, since group rebalances will cause partition offsets * to be reset. You should also prefer to provide your own listener if you are doing your own offset * management since the listener gives you an opportunity to commit offsets before a rebalance finishes. * * @param topics The list of topics to subscribe to */ @Override - public void subscribe(List topics) { + public void subscribe(Collection topics) { subscribe(topics, new NoOpConsumerRebalanceListener()); } @@ -785,8 +784,8 @@ public void subscribe(Pattern pattern, ConsumerRebalanceListener listener) { } /** - * Unsubscribe from topics currently subscribed with {@link #subscribe(List)}. This - * also clears any partitions directly assigned through {@link #assign(List)}. + * Unsubscribe from topics currently subscribed with {@link #subscribe(Collection)}. This + * also clears any partitions directly assigned through {@link #assign(Collection)}. */ public void unsubscribe() { acquire(); @@ -806,13 +805,13 @@ public void unsubscribe() { *

      * Manual topic assignment through this method does not use the consumer's group management * functionality. As such, there will be no rebalance operation triggered when group membership or cluster and topic - * metadata change. Note that it is not possible to use both manual partition assignment with {@link #assign(List)} - * and group assignment with {@link #subscribe(List, ConsumerRebalanceListener)}. + * metadata change. Note that it is not possible to use both manual partition assignment with {@link #assign(Collection)} + * and group assignment with {@link #subscribe(Collection, ConsumerRebalanceListener)}. * * @param partitions The list of partitions to assign this consumer */ @Override - public void assign(List partitions) { + public void assign(Collection partitions) { acquire(); try { log.debug("Subscribed to partition(s): {}", Utils.join(partitions, ", ")); @@ -931,7 +930,7 @@ private Map>> pollOnce(long timeout) { * encountered (in which case it is thrown to the caller). * * @throws org.apache.kafka.clients.consumer.CommitFailedException if the commit failed and cannot be retried. - * This can only occur if you are using automatic group management with {@link #subscribe(List)}, + * This can only occur if you are using automatic group management with {@link #subscribe(Collection)}, * or if there is an active group with the same groupId which is using group management. * @throws org.apache.kafka.common.errors.WakeupException if {@link #wakeup()} is called before or while this * function is called @@ -963,7 +962,7 @@ public void commitSync() { * * @param offsets A map of offsets by partition with associated metadata * @throws org.apache.kafka.clients.consumer.CommitFailedException if the commit failed and cannot be retried. - * This can only occur if you are using automatic group management with {@link #subscribe(List)}, + * This can only occur if you are using automatic group management with {@link #subscribe(Collection)}, * or if there is an active group with the same groupId which is using group management. * @throws org.apache.kafka.common.errors.WakeupException if {@link #wakeup()} is called before or while this * function is called @@ -1063,11 +1062,10 @@ public void seek(TopicPartition partition, long offset) { * first offset in all partitions only when {@link #poll(long)} or {@link #position(TopicPartition)} are called. * If no partition is provided, seek to the first offset for all of the currently assigned partitions. */ - public void seekToBeginning(TopicPartition... partitions) { + public void seekToBeginning(Collection partitions) { acquire(); try { - Collection parts = partitions.length == 0 ? this.subscriptions.assignedPartitions() - : Arrays.asList(partitions); + Collection parts = partitions.size() == 0 ? this.subscriptions.assignedPartitions() : partitions; for (TopicPartition tp : parts) { log.debug("Seeking to beginning of partition {}", tp); subscriptions.needOffsetReset(tp, OffsetResetStrategy.EARLIEST); @@ -1082,11 +1080,10 @@ public void seekToBeginning(TopicPartition... partitions) { * final offset in all partitions only when {@link #poll(long)} or {@link #position(TopicPartition)} are called. * If no partition is provided, seek to the final offset for all of the currently assigned partitions. */ - public void seekToEnd(TopicPartition... partitions) { + public void seekToEnd(Collection partitions) { acquire(); try { - Collection parts = partitions.length == 0 ? this.subscriptions.assignedPartitions() - : Arrays.asList(partitions); + Collection parts = partitions.size() == 0 ? this.subscriptions.assignedPartitions() : partitions; for (TopicPartition tp : parts) { log.debug("Seeking to end of partition {}", tp); subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST); @@ -1222,13 +1219,13 @@ public Map> listTopics() { /** * Suspend fetching from the requested partitions. Future calls to {@link #poll(long)} will not return - * any records from these partitions until they have been resumed using {@link #resume(TopicPartition...)}. + * any records from these partitions until they have been resumed using {@link #resume(Collection)}. * Note that this method does not affect partition subscription. In particular, it does not cause a group * rebalance when automatic assignment is used. * @param partitions The partitions which should be paused */ @Override - public void pause(TopicPartition... partitions) { + public void pause(Collection partitions) { acquire(); try { for (TopicPartition partition: partitions) { @@ -1241,34 +1238,34 @@ public void pause(TopicPartition... partitions) { } /** - * Get the set of partitions that were previously paused by a call to {@link #pause(TopicPartition...)}. - * - * @return The set of paused partitions + * Resume specified partitions which have been paused with {@link #pause(Collection)}. New calls to + * {@link #poll(long)} will return records from these partitions if there are any to be fetched. + * If the partitions were not previously paused, this method is a no-op. + * @param partitions The partitions which should be resumed */ @Override - public Set paused() { + public void resume(Collection partitions) { acquire(); try { - return Collections.unmodifiableSet(subscriptions.pausedPartitions()); + for (TopicPartition partition: partitions) { + log.debug("Resuming partition {}", partition); + subscriptions.resume(partition); + } } finally { release(); } } /** - * Resume specified partitions which have been paused with {@link #pause(TopicPartition...)}. New calls to - * {@link #poll(long)} will return records from these partitions if there are any to be fetched. - * If the partitions were not previously paused, this method is a no-op. - * @param partitions The partitions which should be resumed + * Get the set of partitions that were previously paused by a call to {@link #pause(Collection)}. + * + * @return The set of paused partitions */ @Override - public void resume(TopicPartition... partitions) { + public Set paused() { acquire(); try { - for (TopicPartition partition: partitions) { - log.debug("Resuming partition {}", partition); - subscriptions.resume(partition); - } + return Collections.unmodifiableSet(subscriptions.pausedPartitions()); } finally { release(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java index c7f0a46754b35..8dce1f1f9411c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/MockConsumer.java @@ -67,7 +67,7 @@ public MockConsumer(OffsetResetStrategy offsetResetStrategy) { this.exception = null; this.wakeup = new AtomicBoolean(false); } - + @Override public Set assignment() { return this.subscriptions.assignedPartitions(); @@ -86,7 +86,7 @@ public Set subscription() { } @Override - public void subscribe(List topics) { + public void subscribe(Collection topics) { subscribe(topics, new NoOpConsumerRebalanceListener()); } @@ -105,13 +105,13 @@ public void subscribe(Pattern pattern, final ConsumerRebalanceListener listener) } @Override - public void subscribe(List topics, final ConsumerRebalanceListener listener) { + public void subscribe(Collection topics, final ConsumerRebalanceListener listener) { ensureNotClosed(); this.subscriptions.subscribe(topics, listener); } @Override - public void assign(List partitions) { + public void assign(Collection partitions) { ensureNotClosed(); this.subscriptions.assignFromUser(partitions); } @@ -238,7 +238,7 @@ public long position(TopicPartition partition) { } @Override - public void seekToBeginning(TopicPartition... partitions) { + public void seekToBeginning(Collection partitions) { ensureNotClosed(); for (TopicPartition tp : partitions) subscriptions.needOffsetReset(tp, OffsetResetStrategy.EARLIEST); @@ -249,7 +249,7 @@ public void updateBeginningOffsets(Map newOffsets) { } @Override - public void seekToEnd(TopicPartition... partitions) { + public void seekToEnd(Collection partitions) { ensureNotClosed(); for (TopicPartition tp : partitions) subscriptions.needOffsetReset(tp, OffsetResetStrategy.LATEST); @@ -287,7 +287,7 @@ public void updatePartitions(String topic, List partitions) { } @Override - public void pause(TopicPartition... partitions) { + public void pause(Collection partitions) { for (TopicPartition partition : partitions) { subscriptions.pause(partition); paused.add(partition); @@ -295,7 +295,7 @@ public void pause(TopicPartition... partitions) { } @Override - public void resume(TopicPartition... partitions) { + public void resume(Collection partitions) { for (TopicPartition partition : partitions) { subscriptions.resume(partition); paused.remove(partition); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java index af263577ac279..e72a476ee5c52 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SubscriptionState.java @@ -21,7 +21,6 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Iterator; -import java.util.List; import java.util.Map; import java.util.Set; import java.util.regex.Pattern; @@ -89,7 +88,7 @@ public SubscriptionState(OffsetResetStrategy defaultResetStrategy) { this.subscribedPattern = null; } - public void subscribe(List topics, ConsumerRebalanceListener listener) { + public void subscribe(Collection topics, ConsumerRebalanceListener listener) { if (listener == null) throw new IllegalArgumentException("RebalanceListener cannot be null"); @@ -101,7 +100,7 @@ public void subscribe(List topics, ConsumerRebalanceListener listener) { changeSubscription(topics); } - public void changeSubscription(List topicsToSubscribe) { + public void changeSubscription(Collection topicsToSubscribe) { if (!this.subscription.equals(new HashSet<>(topicsToSubscribe))) { this.subscription.clear(); this.subscription.addAll(topicsToSubscribe); @@ -415,4 +414,4 @@ private boolean isFetchable() { } -} \ No newline at end of file +} diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java index 2ac024f02365e..ff07461ac7845 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java @@ -29,6 +29,9 @@ import java.util.Collections; import java.util.Properties; +import static java.util.Collections.singleton; +import static org.junit.Assert.assertEquals; + public class KafkaConsumerTest { private final String topic = "test"; @@ -47,9 +50,9 @@ public void testConstructorClose() throws Exception { KafkaConsumer consumer = new KafkaConsumer( props, new ByteArrayDeserializer(), new ByteArrayDeserializer()); } catch (KafkaException e) { - Assert.assertEquals(oldInitCount + 1, MockMetricsReporter.INIT_COUNT.get()); - Assert.assertEquals(oldCloseCount + 1, MockMetricsReporter.CLOSE_COUNT.get()); - Assert.assertEquals("Failed to construct kafka consumer", e.getMessage()); + assertEquals(oldInitCount + 1, MockMetricsReporter.INIT_COUNT.get()); + assertEquals(oldCloseCount + 1, MockMetricsReporter.CLOSE_COUNT.get()); + assertEquals("Failed to construct kafka consumer", e.getMessage()); return; } Assert.fail("should have caught an exception and returned"); @@ -60,7 +63,7 @@ public void testSubscription() { KafkaConsumer consumer = newConsumer(); consumer.subscribe(Collections.singletonList(topic)); - Assert.assertEquals(Collections.singleton(topic), consumer.subscription()); + assertEquals(singleton(topic), consumer.subscription()); Assert.assertTrue(consumer.assignment().isEmpty()); consumer.subscribe(Collections.emptyList()); @@ -69,7 +72,7 @@ public void testSubscription() { consumer.assign(Collections.singletonList(tp0)); Assert.assertTrue(consumer.subscription().isEmpty()); - Assert.assertEquals(Collections.singleton(tp0), consumer.assignment()); + assertEquals(singleton(tp0), consumer.assignment()); consumer.unsubscribe(); Assert.assertTrue(consumer.subscription().isEmpty()); @@ -98,12 +101,12 @@ public void testInterceptorConstructorClose() throws Exception { KafkaConsumer consumer = new KafkaConsumer( props, new StringDeserializer(), new StringDeserializer()); - Assert.assertEquals(1, MockConsumerInterceptor.INIT_COUNT.get()); - Assert.assertEquals(0, MockConsumerInterceptor.CLOSE_COUNT.get()); + assertEquals(1, MockConsumerInterceptor.INIT_COUNT.get()); + assertEquals(0, MockConsumerInterceptor.CLOSE_COUNT.get()); consumer.close(); - Assert.assertEquals(1, MockConsumerInterceptor.INIT_COUNT.get()); - Assert.assertEquals(1, MockConsumerInterceptor.CLOSE_COUNT.get()); + assertEquals(1, MockConsumerInterceptor.INIT_COUNT.get()); + assertEquals(1, MockConsumerInterceptor.CLOSE_COUNT.get()); } finally { // cleanup since we are using mutable static variables in MockConsumerInterceptor MockConsumerInterceptor.resetCounters(); @@ -115,13 +118,13 @@ public void testPause() { KafkaConsumer consumer = newConsumer(); consumer.assign(Collections.singletonList(tp0)); - Assert.assertEquals(Collections.singleton(tp0), consumer.assignment()); + assertEquals(singleton(tp0), consumer.assignment()); Assert.assertTrue(consumer.paused().isEmpty()); - consumer.pause(tp0); - Assert.assertEquals(Collections.singleton(tp0), consumer.paused()); + consumer.pause(singleton(tp0)); + assertEquals(singleton(tp0), consumer.paused()); - consumer.resume(tp0); + consumer.resume(singleton(tp0)); Assert.assertTrue(consumer.paused().isEmpty()); consumer.unsubscribe(); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java index eb643556bd9b9..62934553959af 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java @@ -49,6 +49,8 @@ import java.util.Map; import java.util.Set; +import static java.util.Collections.singleton; + /** * WorkerTask that uses a SinkTask to export data from Kafka. */ @@ -350,7 +352,7 @@ private void deliverMessages() { if (pausedForRedelivery) { for (TopicPartition tp : consumer.assignment()) if (!context.pausedPartitions().contains(tp)) - consumer.resume(tp); + consumer.resume(singleton(tp)); pausedForRedelivery = false; } } catch (RetriableException e) { @@ -358,8 +360,7 @@ private void deliverMessages() { // If we're retrying a previous batch, make sure we've paused all topic partitions so we don't get new data, // but will still be able to poll in order to handle user-requested timeouts, keep group membership, etc. pausedForRedelivery = true; - for (TopicPartition tp : consumer.assignment()) - consumer.pause(tp); + consumer.pause(consumer.assignment()); // Let this exit normally, the batch will be reprocessed on the next loop. } catch (Throwable t) { log.error("Task {} threw an uncaught and unrecoverable exception", id, t); @@ -419,7 +420,7 @@ public void onPartitionsAssigned(Collection partitions) { for (TopicPartition tp : partitions) { if (!taskPaused.contains(tp)) - consumer.resume(tp); + consumer.resume(singleton(tp)); } Iterator tpIter = taskPaused.iterator(); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTaskContext.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTaskContext.java index 06f483844f673..c762bddb340f4 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTaskContext.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTaskContext.java @@ -16,6 +16,7 @@ import org.apache.kafka.connect.errors.IllegalWorkerStateException; import org.apache.kafka.connect.sink.SinkTaskContext; +import java.util.Arrays; import java.util.HashMap; import java.util.HashSet; import java.util.Map; @@ -85,7 +86,7 @@ public void pause(TopicPartition... partitions) { try { for (TopicPartition partition : partitions) pausedPartitions.add(partition); - consumer.pause(partitions); + consumer.pause(Arrays.asList(partitions)); } catch (IllegalStateException e) { throw new IllegalWorkerStateException("SinkTasks may not pause partitions that are not currently assigned to them.", e); } @@ -99,7 +100,7 @@ public void resume(TopicPartition... partitions) { try { for (TopicPartition partition : partitions) pausedPartitions.remove(partition); - consumer.resume(partitions); + consumer.resume(Arrays.asList(partitions)); } catch (IllegalStateException e) { throw new IllegalWorkerStateException("SinkTasks may not resume partitions that are not currently assigned to them.", e); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java index 5ab60cd35393c..290f8a2b6369f 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/KafkaBasedLog.java @@ -21,7 +21,6 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.producer.KafkaProducer; import org.apache.kafka.clients.producer.Producer; @@ -30,6 +29,7 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.connect.errors.ConnectException; @@ -46,6 +46,8 @@ import java.util.Set; import java.util.concurrent.Future; +import static java.util.Collections.singleton; + /** *

      * KafkaBasedLog provides a generic implementation of a shared, compacted log of records stored in Kafka that all @@ -267,7 +269,7 @@ private void readToLogEnd() { for (TopicPartition tp : assignment) { long offset = consumer.position(tp); offsets.put(tp, offset); - consumer.seekToEnd(tp); + consumer.seekToEnd(singleton(tp)); } Map endOffsets = new HashMap<>(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java index f419a7b7880a3..7bc83de2deaf8 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java @@ -57,6 +57,7 @@ import java.util.List; import java.util.Map; +import static java.util.Collections.singleton; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -136,10 +137,9 @@ public void testPollRedelivery() throws Exception { sinkTask.put(EasyMock.capture(records)); EasyMock.expectLastCall().andThrow(new RetriableException("retry")); // Pause - EasyMock.expect(consumer.assignment()).andReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2))); - consumer.pause(TOPIC_PARTITION); - PowerMock.expectLastCall(); - consumer.pause(TOPIC_PARTITION2); + HashSet partitions = new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2)); + EasyMock.expect(consumer.assignment()).andReturn(partitions); + consumer.pause(partitions); PowerMock.expectLastCall(); // Retry delivery should succeed @@ -147,10 +147,10 @@ public void testPollRedelivery() throws Exception { sinkTask.put(EasyMock.capture(records)); EasyMock.expectLastCall(); // And unpause - EasyMock.expect(consumer.assignment()).andReturn(new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2))); - consumer.resume(TOPIC_PARTITION); + EasyMock.expect(consumer.assignment()).andReturn(partitions); + consumer.resume(singleton(TOPIC_PARTITION)); PowerMock.expectLastCall(); - consumer.resume(TOPIC_PARTITION2); + consumer.resume(singleton(TOPIC_PARTITION2)); PowerMock.expectLastCall(); PowerMock.replayAll(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java index 1099d7a2740b9..25f0bf4e66da7 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java @@ -377,9 +377,9 @@ public Object answer() throws Throwable { return null; } }); - consumer.pause(UNASSIGNED_TOPIC_PARTITION); + consumer.pause(Arrays.asList(UNASSIGNED_TOPIC_PARTITION)); PowerMock.expectLastCall().andThrow(new IllegalStateException("unassigned topic partition")); - consumer.pause(TOPIC_PARTITION, TOPIC_PARTITION2); + consumer.pause(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2)); PowerMock.expectLastCall(); expectOnePoll().andAnswer(new IAnswer() { @@ -396,9 +396,9 @@ public Object answer() throws Throwable { return null; } }); - consumer.resume(UNASSIGNED_TOPIC_PARTITION); + consumer.resume(Arrays.asList(UNASSIGNED_TOPIC_PARTITION)); PowerMock.expectLastCall().andThrow(new IllegalStateException("unassigned topic partition")); - consumer.resume(TOPIC_PARTITION, TOPIC_PARTITION2); + consumer.resume(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2)); PowerMock.expectLastCall(); expectStopTask(0); diff --git a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala index 4f9aca373b861..03864048c1f88 100755 --- a/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala +++ b/core/src/main/scala/kafka/admin/ConsumerGroupCommand.scala @@ -335,7 +335,7 @@ object ConsumerGroupCommand { val consumer = getConsumer() val topicPartition = new TopicPartition(topic, partition) consumer.assign(List(topicPartition).asJava) - consumer.seekToEnd(topicPartition) + consumer.seekToEnd(List(topicPartition).asJava) val logEndOffset = consumer.position(topicPartition) LogEndOffsetResult.LogEndOffset(logEndOffset) } diff --git a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala index c7f907238b152..a38c04b4ff322 100644 --- a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala +++ b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala @@ -28,6 +28,7 @@ import org.apache.log4j.Logger import org.apache.kafka.clients.consumer.{ConsumerRebalanceListener, KafkaConsumer} import org.apache.kafka.common.serialization.ByteArrayDeserializer import org.apache.kafka.common.utils.Utils +import org.apache.kafka.common.TopicPartition import kafka.utils.CommandLineUtils import java.util.{ Random, Properties } import kafka.consumer.Consumer @@ -121,7 +122,7 @@ object ConsumerPerformance { } consumer.poll(100) } - consumer.seekToBeginning() + consumer.seekToBeginning(List[TopicPartition]()) // Now start the benchmark val startMs = System.currentTimeMillis diff --git a/core/src/main/scala/kafka/tools/EndToEndLatency.scala b/core/src/main/scala/kafka/tools/EndToEndLatency.scala index e670d82a2de1d..584d4fb7ee7e5 100755 --- a/core/src/main/scala/kafka/tools/EndToEndLatency.scala +++ b/core/src/main/scala/kafka/tools/EndToEndLatency.scala @@ -22,6 +22,7 @@ import java.util.{Arrays, Properties} import org.apache.kafka.clients.consumer.{ConsumerConfig, KafkaConsumer} import org.apache.kafka.clients.producer._ import org.apache.kafka.common.utils.Utils +import org.apache.kafka.common.TopicPartition import scala.collection.JavaConversions._ @@ -85,7 +86,7 @@ object EndToEndLatency { //Ensure we are at latest offset. seekToEnd evaluates lazily, that is to say actually performs the seek only when //a poll() or position() request is issued. Hence we need to poll after we seek to ensure we see our first write. - consumer.seekToEnd() + consumer.seekToEnd(List[TopicPartition]()) consumer.poll(0) var totalTime = 0.0 @@ -143,4 +144,4 @@ object EndToEndLatency { def randomBytesOfLen(len: Int): Array[Byte] = { Array.fill(len)((scala.util.Random.nextInt(26) + 65).toByte) } -} \ No newline at end of file +} diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index f576be575c377..9939309dc41b5 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -99,7 +99,7 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { val rebalanceListener = new ConsumerRebalanceListener { override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]) = { // keep partitions paused in this test so that we can verify the commits based on specific seeks - partitions.asScala.foreach(consumer0.pause(_)) + consumer0.pause(partitions) } override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]) = {} @@ -145,7 +145,7 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { assertEquals(3, this.consumers(0).committed(tp).offset) assertNull(this.consumers(0).committed(tp2)) - // positions should not change + // Positions should not change assertEquals(pos1, this.consumers(0).position(tp)) assertEquals(pos2, this.consumers(0).position(tp2)) this.consumers(0).commitSync(Map[TopicPartition, OffsetAndMetadata]((tp2, new OffsetAndMetadata(5L))).asJava) @@ -244,7 +244,7 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { sendRecords(5) consumer0.subscribe(List(topic).asJava) consumeAndVerifyRecords(consumer = consumer0, numRecords = 5, startingOffset = 0) - consumer0.pause(tp) + consumer0.pause(List(tp).asJava) // subscribe to a new topic to trigger a rebalance consumer0.subscribe(List("topic2").asJava) diff --git a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala index 029eaf41c4fda..84243403cb617 100644 --- a/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala +++ b/core/src/test/scala/integration/kafka/api/ConsumerBounceTest.scala @@ -104,7 +104,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { assertEquals(consumer.position(tp), consumer.committed(tp).offset) if (consumer.position(tp) == numRecords) { - consumer.seekToBeginning() + consumer.seekToBeginning(List[TopicPartition]()) consumed = 0 } } catch { @@ -140,7 +140,7 @@ class ConsumerBounceTest extends IntegrationTestHarness with Logging { val coin = TestUtils.random.nextInt(3) if (coin == 0) { info("Seeking to end of log") - consumer.seekToEnd() + consumer.seekToEnd(List[TopicPartition]()) assertEquals(numRecords.toLong, consumer.position(tp)) } else if (coin == 1) { val pos = TestUtils.random.nextInt(numRecords).toLong diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index ca0497b0c6927..9c560107e4111 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -300,11 +300,11 @@ class PlaintextConsumerTest extends BaseConsumerTest { sendRecords(totalRecords.toInt, tp) consumer.assign(List(tp).asJava) - consumer.seekToEnd(tp) + consumer.seekToEnd(List(tp).asJava) assertEquals(totalRecords, consumer.position(tp)) assertFalse(consumer.poll(totalRecords).iterator().hasNext) - consumer.seekToBeginning(tp) + consumer.seekToBeginning(List(tp).asJava) assertEquals(0, consumer.position(tp), 0) consumeAndVerifyRecords(consumer, numRecords = 1, startingOffset = 0) @@ -318,11 +318,11 @@ class PlaintextConsumerTest extends BaseConsumerTest { sendCompressedMessages(totalRecords.toInt, tp2) consumer.assign(List(tp2).asJava) - consumer.seekToEnd(tp2) + consumer.seekToEnd(List(tp2).asJava) assertEquals(totalRecords, consumer.position(tp2)) assertFalse(consumer.poll(totalRecords).iterator().hasNext) - consumer.seekToBeginning(tp2) + consumer.seekToBeginning(List(tp2).asJava) assertEquals(0, consumer.position(tp2), 0) consumeAndVerifyRecords(consumer, numRecords = 1, startingOffset = 0, tp = tp2) @@ -375,13 +375,14 @@ class PlaintextConsumerTest extends BaseConsumerTest { @Test def testPartitionPauseAndResume() { + val partitions = List(tp).asJava sendRecords(5) - this.consumers(0).assign(List(tp).asJava) + this.consumers(0).assign(partitions) consumeAndVerifyRecords(consumer = this.consumers(0), numRecords = 5, startingOffset = 0) - this.consumers(0).pause(tp) + this.consumers(0).pause(partitions) sendRecords(5) assertTrue(this.consumers(0).poll(0).isEmpty) - this.consumers(0).resume(tp) + this.consumers(0).resume(partitions) consumeAndVerifyRecords(consumer = this.consumers(0), numRecords = 5, startingOffset = 5) } @@ -632,7 +633,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { val rebalanceListener = new ConsumerRebalanceListener { override def onPartitionsAssigned(partitions: util.Collection[TopicPartition]) = { // keep partitions paused in this test so that we can verify the commits based on specific seeks - partitions.asScala.foreach(testConsumer.pause(_)) + testConsumer.pause(partitions) } override def onPartitionsRevoked(partitions: util.Collection[TopicPartition]) = {} diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index df8516ceb8fd8..665d39f8264e5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -43,6 +43,8 @@ import java.util.Map; import java.util.Set; +import static java.util.Collections.singleton; + public class ProcessorStateManager { private static final Logger log = LoggerFactory.getLogger(ProcessorStateManager.class); @@ -204,7 +206,7 @@ private void restoreActiveState(String topicName, StateRestoreCallback stateRest try { // calculate the end offset of the partition // TODO: this is a bit hacky to first seek then position to get the end offset - restoreConsumer.seekToEnd(storePartition); + restoreConsumer.seekToEnd(singleton(storePartition)); long endOffset = restoreConsumer.position(storePartition); // restore from the checkpointed offset of the change log if it is persistent and the offset exists; @@ -212,7 +214,7 @@ private void restoreActiveState(String topicName, StateRestoreCallback stateRest if (checkpointedOffsets.containsKey(storePartition)) { restoreConsumer.seek(storePartition, checkpointedOffsets.get(storePartition)); } else { - restoreConsumer.seekToBeginning(storePartition); + restoreConsumer.seekToBeginning(singleton(storePartition)); } // restore its state from changelog records diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 54a25c1aabfe1..c4cc2baf0e026 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -34,6 +34,8 @@ import java.util.List; import java.util.Map; +import static java.util.Collections.singleton; + /** * A StreamTask is associated with a {@link PartitionGroup}, and is assigned to a StreamThread for processing. */ @@ -136,7 +138,7 @@ public void addRecords(TopicPartition partition, Iterable this.maxBufferedSize) { - consumer.pause(partition); + consumer.pause(singleton(partition)); } } @@ -178,7 +180,7 @@ public int process() { // after processing this record, if its partition queue's buffered size has been // decreased to the threshold, we can then resume the consumption on this partition if (partitionGroup.numBuffered(partition) == this.maxBufferedSize) { - consumer.resume(partition); + consumer.resume(singleton(partition)); requiresPoll = true; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index e1a518d4b99d4..7d6b98f9b8680 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -66,6 +66,8 @@ import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; +import static java.util.Collections.singleton; + public class StreamThread extends Thread { private static final Logger log = LoggerFactory.getLogger(StreamThread.class); @@ -382,7 +384,7 @@ private void maybeUpdateStandbyTasks() { if (remaining != null) { remainingStandbyRecords.put(partition, remaining); } else { - restoreConsumer.resume(partition); + restoreConsumer.resume(singleton(partition)); } } } @@ -405,7 +407,7 @@ private void maybeUpdateStandbyTasks() { List> remaining = task.update(partition, records.records(partition)); if (remaining != null) { - restoreConsumer.pause(partition); + restoreConsumer.pause(singleton(partition)); standbyRecords.put(partition, remaining); } } @@ -690,7 +692,7 @@ private void addStandbyTasks() { if (offset >= 0) { restoreConsumer.seek(partition, offset); } else { - restoreConsumer.seekToBeginning(partition); + restoreConsumer.seekToBeginning(singleton(partition)); } } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java index 1d0a969865606..84b59e639f79d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java @@ -38,9 +38,9 @@ import java.nio.channels.FileLock; import java.nio.file.Files; 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.Set; @@ -94,7 +94,7 @@ public void bufferRecord(ConsumerRecord record) { } @Override - public synchronized void assign(List partitions) { + public synchronized void assign(Collection partitions) { int numPartitions = partitions.size(); if (numPartitions > 1) throw new IllegalArgumentException("RestoreConsumer: more than one partition specified"); @@ -102,7 +102,7 @@ public synchronized void assign(List partitions) { if (numPartitions == 1) { if (assignedPartition != null) throw new IllegalStateException("RestoreConsumer: partition already assigned"); - assignedPartition = partitions.get(0); + assignedPartition = partitions.iterator().next(); // set the beginning offset to 0 // NOTE: this is users responsible to set the initial lEO. @@ -154,8 +154,8 @@ public synchronized void seek(TopicPartition partition, long offset) { } @Override - public synchronized void seekToBeginning(TopicPartition... partitions) { - if (partitions.length != 1) + public synchronized void seekToBeginning(Collection partitions) { + if (partitions.size() != 1) throw new IllegalStateException("RestoreConsumer: other than one partition specified"); for (TopicPartition partition : partitions) { @@ -168,8 +168,8 @@ public synchronized void seekToBeginning(TopicPartition... partitions) { } @Override - public synchronized void seekToEnd(TopicPartition... partitions) { - if (partitions.length != 1) + public synchronized void seekToEnd(Collection partitions) { + if (partitions.size() != 1) throw new IllegalStateException("RestoreConsumer: other than one partition specified"); for (TopicPartition partition : partitions) { diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java index ea24441643e07..e7fb9a4e41f77 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StandbyTaskTest.java @@ -49,6 +49,7 @@ import java.util.Properties; import java.util.Set; +import static java.util.Collections.singleton; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; @@ -181,7 +182,7 @@ public void testUpdate() throws Exception { if (offset >= 0) { restoreStateConsumer.seek(partition, offset); } else { - restoreStateConsumer.seekToBeginning(partition); + restoreStateConsumer.seekToBeginning(singleton(partition)); } } @@ -246,7 +247,7 @@ public void testUpdateKTable() throws Exception { if (offset >= 0) { restoreStateConsumer.seek(partition, offset); } else { - restoreStateConsumer.seekToBeginning(partition); + restoreStateConsumer.seekToBeginning(singleton(partition)); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java index 1abf88d17ebf2..d7b0139618fd0 100644 --- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java @@ -200,7 +200,7 @@ public static void verify(String kafka, Map> allData, int m KafkaConsumer consumer = new KafkaConsumer<>(props); List partitions = getAllPartitions(consumer, "echo", "max", "min", "dif", "sum", "cnt", "avg", "wcnt", "tagg"); consumer.assign(partitions); - consumer.seekToBeginning(partitions.toArray(new TopicPartition[partitions.size()])); + consumer.seekToBeginning(partitions); final int recordsGenerated = allData.size() * maxRecordsPerKey; int recordsProcessed = 0; diff --git a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java index a2948a260706a..4ddbc2a6bdcdf 100644 --- a/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/ProcessorTopologyTestDriver.java @@ -40,6 +40,7 @@ import org.apache.kafka.streams.state.KeyValueStore; import java.util.ArrayList; +import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.LinkedList; @@ -317,12 +318,12 @@ public void close() { protected MockConsumer createRestoreConsumer(TaskId id, String... storeNames) { MockConsumer consumer = new MockConsumer(OffsetResetStrategy.LATEST) { @Override - public synchronized void seekToEnd(TopicPartition... partitions) { + public synchronized void seekToEnd(Collection partitions) { // do nothing ... } @Override - public synchronized void seekToBeginning(TopicPartition... partitions) { + public synchronized void seekToBeginning(Collection partitions) { // do nothing ... } From b6c6291e1f42b2ca1fbcdaf662a2a868c8881921 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 18 Mar 2016 16:13:33 -0700 Subject: [PATCH 057/206] KAFKA-3378; Client blocks forever if SocketChannel connects instantly This is a different implementation to the one in #1085 by Larkin Lowrey (llowrey). The hard part here was actually finding the problem and all credit goes to llowrey. This PR also fixes our handling of `finishConnect` (we now check the return value). Author: Ismael Juma Reviewers: Jun Rao Closes #1094 from ijuma/KAFKA-3378-instantly-connecting-socket-channels --- .../kafka/common/network/KafkaChannel.java | 4 +- .../network/PlaintextTransportLayer.java | 8 +- .../apache/kafka/common/network/Selector.java | 161 ++++++++++-------- .../common/network/SslTransportLayer.java | 8 +- .../kafka/common/network/TransportLayer.java | 2 +- 5 files changed, 101 insertions(+), 82 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/network/KafkaChannel.java b/clients/src/main/java/org/apache/kafka/common/network/KafkaChannel.java index defcc2484d6be..f72f91b8f005a 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/KafkaChannel.java +++ b/clients/src/main/java/org/apache/kafka/common/network/KafkaChannel.java @@ -68,8 +68,8 @@ public void disconnect() { } - public void finishConnect() throws IOException { - transportLayer.finishConnect(); + public boolean finishConnect() throws IOException { + return transportLayer.finishConnect(); } public boolean isConnected() { diff --git a/clients/src/main/java/org/apache/kafka/common/network/PlaintextTransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/PlaintextTransportLayer.java index 8949e5efbb25d..3db4345c14f01 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/PlaintextTransportLayer.java +++ b/clients/src/main/java/org/apache/kafka/common/network/PlaintextTransportLayer.java @@ -50,9 +50,11 @@ public boolean ready() { } @Override - public void finishConnect() throws IOException { - socketChannel.finishConnect(); - key.interestOps(key.interestOps() & ~SelectionKey.OP_CONNECT | SelectionKey.OP_READ); + public boolean finishConnect() throws IOException { + boolean connected = socketChannel.finishConnect(); + if (connected) + key.interestOps(key.interestOps() & ~SelectionKey.OP_CONNECT | SelectionKey.OP_READ); + return connected; } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java index 8bb33488cb2eb..f9e232dcc5f82 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java @@ -24,6 +24,7 @@ import java.util.ArrayList; import java.util.Deque; import java.util.HashMap; +import java.util.HashSet; import java.util.Iterator; import java.util.LinkedHashMap; import java.util.List; @@ -41,7 +42,6 @@ import org.apache.kafka.common.metrics.stats.Count; import org.apache.kafka.common.metrics.stats.Max; import org.apache.kafka.common.metrics.stats.Rate; -import org.apache.kafka.common.utils.SystemTime; import org.apache.kafka.common.utils.Time; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -84,6 +84,7 @@ public class Selector implements Selectable { private final List completedSends; private final List completedReceives; private final Map> stagedReceives; + private final Set immediatelyConnectedKeys; private final List disconnected; private final List connected; private final List failedSends; @@ -114,18 +115,19 @@ public Selector(int maxReceiveSize, long connectionMaxIdleMs, Metrics metrics, T this.time = time; this.metricGrpPrefix = metricGrpPrefix; this.metricTags = metricTags; - this.channels = new HashMap(); - this.completedSends = new ArrayList(); - this.completedReceives = new ArrayList(); - this.stagedReceives = new HashMap>(); - this.connected = new ArrayList(); - this.disconnected = new ArrayList(); - this.failedSends = new ArrayList(); + this.channels = new HashMap<>(); + this.completedSends = new ArrayList<>(); + this.completedReceives = new ArrayList<>(); + this.stagedReceives = new HashMap<>(); + this.immediatelyConnectedKeys = new HashSet<>(); + this.connected = new ArrayList<>(); + this.disconnected = new ArrayList<>(); + this.failedSends = new ArrayList<>(); this.sensors = new SelectorMetrics(metrics); this.channelBuilder = channelBuilder; // initial capacity and load factor are default, we set them explicitly because we want to set accessOrder = true - this.lruConnections = new LinkedHashMap(16, .75F, true); - currentTimeNanos = new SystemTime().nanoseconds(); + this.lruConnections = new LinkedHashMap<>(16, .75F, true); + currentTimeNanos = time.nanoseconds(); nextIdleCloseCheckTime = currentTimeNanos + connectionsMaxIdleNanos; this.metricsPerConnection = metricsPerConnection; } @@ -161,8 +163,9 @@ public void connect(String id, InetSocketAddress address, int sendBufferSize, in if (receiveBufferSize != Selectable.USE_DEFAULT_BUFFER_SIZE) socket.setReceiveBufferSize(receiveBufferSize); socket.setTcpNoDelay(true); + boolean connected; try { - socketChannel.connect(address); + connected = socketChannel.connect(address); } catch (UnresolvedAddressException e) { socketChannel.close(); throw new IOException("Can't resolve address: " + address, e); @@ -174,6 +177,13 @@ public void connect(String id, InetSocketAddress address, int sendBufferSize, in KafkaChannel channel = channelBuilder.buildChannel(id, key, maxReceiveSize); key.attach(channel); this.channels.put(id, channel); + + if (connected) { + // OP_CONNECT won't trigger for immediately connected channels + log.debug("Immediately connected to node {}", channel.id()); + immediatelyConnectedKeys.add(key); + key.interestOps(0); + } } /** @@ -206,17 +216,15 @@ public void close() { close(id); try { this.nioSelector.close(); - } catch (IOException e) { + } catch (IOException | SecurityException e) { log.error("Exception closing nioSelector:", e); - } catch (SecurityException se) { - log.error("Exception closing nioSelector:", se); } sensors.close(); channelBuilder.close(); } /** - * Queue the given request for sending in the subsequent {@poll(long)} calls + * Queue the given request for sending in the subsequent {@link #poll(long)} calls * @param send The request to send */ public void send(Send send) { @@ -235,7 +243,7 @@ public void send(Send send) { * * 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)} call and repopulated by the call if there is + * lists will be cleared at the beginning of each `poll` call and repopulated by the call if there is * any completed I/O. * * In the "Plaintext" setting, we are using socketChannel to read & write to the network. But for the "SSL" setting, @@ -258,9 +266,12 @@ public void send(Send send) { public void poll(long timeout) throws IOException { if (timeout < 0) throw new IllegalArgumentException("timeout should be >= 0"); + clear(); - if (hasStagedReceives()) + + if (hasStagedReceives() || !immediatelyConnectedKeys.isEmpty()) timeout = 0; + /* check ready keys */ long startSelect = time.nanoseconds(); int readyKeys = select(timeout); @@ -268,72 +279,78 @@ public void poll(long timeout) throws IOException { currentTimeNanos = endSelect; this.sensors.selectTime.record(endSelect - startSelect, time.milliseconds()); - if (readyKeys > 0) { - Set keys = this.nioSelector.selectedKeys(); - Iterator iter = keys.iterator(); - while (iter.hasNext()) { - SelectionKey key = iter.next(); - iter.remove(); - KafkaChannel channel = channel(key); - - // register all per-connection metrics at once - sensors.maybeRegisterConnectionMetrics(channel.id()); - lruConnections.put(channel.id(), currentTimeNanos); - - try { - /* complete any connections that have finished their handshake */ - if (key.isConnectable()) { - channel.finishConnect(); + if (readyKeys > 0 || !immediatelyConnectedKeys.isEmpty()) { + pollSelectionKeys(this.nioSelector.selectedKeys()); + pollSelectionKeys(immediatelyConnectedKeys); + } + + addToCompletedReceives(); + + long endIo = time.nanoseconds(); + this.sensors.ioTime.record(endIo - endSelect, time.milliseconds()); + maybeCloseOldestConnection(); + } + + private void pollSelectionKeys(Iterable selectionKeys) { + Iterator iterator = selectionKeys.iterator(); + while (iterator.hasNext()) { + SelectionKey key = iterator.next(); + iterator.remove(); + KafkaChannel channel = channel(key); + + // register all per-connection metrics at once + sensors.maybeRegisterConnectionMetrics(channel.id()); + lruConnections.put(channel.id(), currentTimeNanos); + + try { + + /* complete any connections that have finished their handshake */ + if (key.isConnectable()) { + if (channel.finishConnect()) { this.connected.add(channel.id()); this.sensors.connectionCreated.record(); - } + } else + continue; + } - /* if channel is not ready finish prepare */ - if (channel.isConnected() && !channel.ready()) - channel.prepare(); + /* if channel is not ready finish prepare */ + if (channel.isConnected() && !channel.ready()) + channel.prepare(); - /* if channel is ready read from any connections that have readable data */ - if (channel.ready() && key.isReadable() && !hasStagedReceive(channel)) { - NetworkReceive networkReceive; - while ((networkReceive = channel.read()) != null) - addToStagedReceives(channel, networkReceive); - } + /* if channel is ready read from any connections that have readable data */ + if (channel.ready() && key.isReadable() && !hasStagedReceive(channel)) { + NetworkReceive networkReceive; + while ((networkReceive = channel.read()) != null) + addToStagedReceives(channel, networkReceive); + } - /* if channel is ready write to any sockets that have space in their buffer and for which we have data */ - if (channel.ready() && key.isWritable()) { - Send send = channel.write(); - if (send != null) { - this.completedSends.add(send); - this.sensors.recordBytesSent(channel.id(), send.size()); - } + /* if channel is ready write to any sockets that have space in their buffer and for which we have data */ + if (channel.ready() && key.isWritable()) { + Send send = channel.write(); + if (send != null) { + this.completedSends.add(send); + this.sensors.recordBytesSent(channel.id(), send.size()); } + } - /* cancel any defunct sockets */ - if (!key.isValid()) { - close(channel); - this.disconnected.add(channel.id()); - } - } catch (Exception e) { - String desc = channel.socketDescription(); - if (e instanceof IOException) - log.debug("Connection with {} disconnected", desc, e); - else - log.warn("Unexpected error from {}; closing connection", desc, e); + /* cancel any defunct sockets */ + if (!key.isValid()) { close(channel); this.disconnected.add(channel.id()); } + + } catch (Exception e) { + String desc = channel.socketDescription(); + if (e instanceof IOException) + log.debug("Connection with {} disconnected", desc, e); + else + log.warn("Unexpected error from {}; closing connection", desc, e); + close(channel); + this.disconnected.add(channel.id()); } } - - addToCompletedReceives(); - - long endIo = time.nanoseconds(); - this.sensors.ioTime.record(endIo - endSelect, time.milliseconds()); - maybeCloseOldestConnection(); } - - @Override public List completedSends() { return this.completedSends; @@ -468,9 +485,7 @@ private void close(KafkaChannel channel) { @Override public boolean isChannelReady(String id) { KafkaChannel channel = this.channels.get(id); - if (channel == null) - return false; - return channel.ready(); + return channel != null && channel.ready(); } private KafkaChannel channelOrFail(String id) { @@ -645,7 +660,7 @@ public void maybeRegisterConnectionMetrics(String connectionId) { if (nodeRequest == null) { String metricGrpName = metricGrpPrefix + "-node-metrics"; - Map tags = new LinkedHashMap(metricTags); + Map tags = new LinkedHashMap<>(metricTags); tags.put("node-id", "node-" + connectionId); nodeRequest = sensor(nodeRequestName); diff --git a/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java index 27e2ea9e2c40f..d18d6b77a1092 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SslTransportLayer.java @@ -106,9 +106,11 @@ public boolean ready() { * does socketChannel.finishConnect() */ @Override - public void finishConnect() throws IOException { - socketChannel.finishConnect(); - key.interestOps(key.interestOps() & ~SelectionKey.OP_CONNECT | SelectionKey.OP_READ); + public boolean finishConnect() throws IOException { + boolean connected = socketChannel.finishConnect(); + if (connected) + key.interestOps(key.interestOps() & ~SelectionKey.OP_CONNECT | SelectionKey.OP_READ); + return connected; } /** diff --git a/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java b/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java index 258d89d4547cf..092df4dbb0bd2 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java +++ b/clients/src/main/java/org/apache/kafka/common/network/TransportLayer.java @@ -43,7 +43,7 @@ public interface TransportLayer extends ScatteringByteChannel, GatheringByteChan /** * Finishes the process of connecting a socket channel. */ - void finishConnect() throws IOException; + boolean finishConnect() throws IOException; /** * disconnect socketChannel From 655367971875a8d6a079ff7d186c05a6b76a9c53 Mon Sep 17 00:00:00 2001 From: Jun Rao Date: Sat, 19 Mar 2016 18:04:56 -0700 Subject: [PATCH 058/206] KAFKA-3427: broker can return incorrect version of fetch response when the broker hits an unknown exception Author: Jun Rao Reviewers: Ismael Juma, Becket Qin Closes #1101 from junrao/kafka-3427 --- core/src/main/scala/kafka/api/FetchRequest.scala | 3 ++- 1 file changed, 2 insertions(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/api/FetchRequest.scala b/core/src/main/scala/kafka/api/FetchRequest.scala index f47942ce1f6f9..83e139a65e45f 100644 --- a/core/src/main/scala/kafka/api/FetchRequest.scala +++ b/core/src/main/scala/kafka/api/FetchRequest.scala @@ -150,7 +150,8 @@ case class FetchRequest(versionId: Short = FetchRequest.CurrentVersion, case (topicAndPartition, data) => (topicAndPartition, FetchResponsePartitionData(Errors.forException(e).code, -1, MessageSet.Empty)) } - val errorResponse = FetchResponse(correlationId, fetchResponsePartitionData) + val fetchRequest = request.requestObj.asInstanceOf[FetchRequest] + val errorResponse = FetchResponse(correlationId, fetchResponsePartitionData, fetchRequest.versionId) // Magic value does not matter here because the message set is empty requestChannel.sendResponse(new RequestChannel.Response(request, new FetchResponseSend(request.connectionId, errorResponse))) } From eb823281a52f3b27c3a889e7412bc07b3024e688 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Sat, 19 Mar 2016 18:39:52 -0700 Subject: [PATCH 059/206] KAFKA-3424: Add CORS support to Connect REST API Author: Ewen Cheslack-Postava Reviewers: Gwen Shapira Closes #1099 from ewencp/cors-rest-support --- build.gradle | 1 + .../kafka/connect/runtime/WorkerConfig.java | 14 +- .../connect/runtime/rest/RestServer.java | 12 ++ .../connect/runtime/rest/RestServerTest.java | 150 ++++++++++++++++++ gradle/dependencies.gradle | 1 + 5 files changed, 177 insertions(+), 1 deletion(-) create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java diff --git a/build.gradle b/build.gradle index 81e4af5f542a5..c29ad5a8eb83b 100644 --- a/build.gradle +++ b/build.gradle @@ -749,6 +749,7 @@ project(':connect:runtime') { compile libs.jerseyContainerServlet compile libs.jettyServer compile libs.jettyServlet + compile libs.jettyServlets compile libs.reflections testCompile project(':clients').sourceSets.test.output diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java index 4ecacbb4f1467..471e4a57975a5 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java @@ -101,6 +101,15 @@ public class WorkerConfig extends AbstractConfig { private static final String REST_ADVERTISED_PORT_DOC = "If this is set, this is the port that will be given out to other workers to connect to."; + public static final String ACCESS_CONTROL_ALLOW_ORIGIN_CONFIG = "access.control.allow.origin"; + protected static final String ACCESS_CONTROL_ALLOW_ORIGIN_DOC = + "Value to set the Access-Control-Allow-Origin header to for REST API requests." + + "To enable cross origin access, set this to the domain of the application that should be permitted" + + " to access the API, or '*' to allow access from any domain. The default value only allows access" + + " from the domain of the REST API."; + protected static final String ACCESS_CONTROL_ALLOW_ORIGIN_DEFAULT = ""; + + /** * Get a basic ConfigDef for a WorkerConfig. This includes all the common settings. Subclasses can use this to * bootstrap their own ConfigDef. @@ -129,7 +138,10 @@ protected static ConfigDef baseConfigDef() { .define(REST_HOST_NAME_CONFIG, Type.STRING, null, Importance.LOW, REST_HOST_NAME_DOC) .define(REST_PORT_CONFIG, Type.INT, REST_PORT_DEFAULT, Importance.LOW, REST_PORT_DOC) .define(REST_ADVERTISED_HOST_NAME_CONFIG, Type.STRING, null, Importance.LOW, REST_ADVERTISED_HOST_NAME_DOC) - .define(REST_ADVERTISED_PORT_CONFIG, Type.INT, null, Importance.LOW, REST_ADVERTISED_PORT_DOC); + .define(REST_ADVERTISED_PORT_CONFIG, Type.INT, null, Importance.LOW, REST_ADVERTISED_PORT_DOC) + .define(ACCESS_CONTROL_ALLOW_ORIGIN_CONFIG, Type.STRING, + ACCESS_CONTROL_ALLOW_ORIGIN_DEFAULT, Importance.LOW, + ACCESS_CONTROL_ALLOW_ORIGIN_DOC); } public WorkerConfig(ConfigDef definition, Map props) { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java index 7e4279a83992a..1505a01332d33 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java @@ -39,8 +39,10 @@ import org.eclipse.jetty.server.handler.HandlerCollection; import org.eclipse.jetty.server.handler.RequestLogHandler; import org.eclipse.jetty.server.handler.StatisticsHandler; +import org.eclipse.jetty.servlet.FilterHolder; import org.eclipse.jetty.servlet.ServletContextHandler; import org.eclipse.jetty.servlet.ServletHolder; +import org.eclipse.jetty.servlets.CrossOriginFilter; import org.glassfish.jersey.server.ResourceConfig; import org.glassfish.jersey.servlet.ServletContainer; import org.slf4j.Logger; @@ -52,9 +54,11 @@ import java.net.HttpURLConnection; import java.net.URI; import java.net.URL; +import java.util.EnumSet; import java.util.List; import java.util.Map; +import javax.servlet.DispatcherType; import javax.ws.rs.core.Response; import javax.ws.rs.core.UriBuilder; @@ -109,6 +113,14 @@ public void start(Herder herder) { context.setContextPath("/"); context.addServlet(servletHolder, "/*"); + String allowedOrigins = config.getString(WorkerConfig.ACCESS_CONTROL_ALLOW_ORIGIN_CONFIG); + if (allowedOrigins != null && !allowedOrigins.trim().isEmpty()) { + FilterHolder filterHolder = new FilterHolder(new CrossOriginFilter()); + filterHolder.setName("cross-origin"); + filterHolder.setInitParameter("allowedOrigins", allowedOrigins); + context.addFilter(filterHolder, "/*", EnumSet.of(DispatcherType.REQUEST)); + } + RequestLogHandler requestLogHandler = new RequestLogHandler(); Slf4jRequestLog requestLog = new Slf4jRequestLog(); requestLog.setLoggerName(RestServer.class.getCanonicalName()); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java new file mode 100644 index 0000000000000..8e9d52b4ca431 --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java @@ -0,0 +1,150 @@ +/** + * 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 org.apache.kafka.connect.runtime.rest; + +import org.apache.kafka.connect.runtime.Herder; +import org.apache.kafka.connect.runtime.WorkerConfig; +import org.apache.kafka.connect.runtime.standalone.StandaloneConfig; +import org.apache.kafka.connect.util.Callback; +import org.easymock.Capture; +import org.easymock.EasyMock; +import org.easymock.IAnswer; +import org.junit.After; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.powermock.api.easymock.PowerMock; +import org.powermock.api.easymock.annotation.MockStrict; +import org.powermock.modules.junit4.PowerMockRunner; + +import javax.ws.rs.client.Client; +import javax.ws.rs.client.ClientBuilder; +import javax.ws.rs.client.Invocation; +import javax.ws.rs.client.WebTarget; +import javax.ws.rs.core.Response; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.Arrays; +import java.util.Collection; +import java.util.HashMap; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +@RunWith(PowerMockRunner.class) +public class RestServerTest { + + @MockStrict + private Herder herder; + private RestServer server; + + @After + public void tearDown() { + server.stop(); + } + + private Map baseWorkerProps() { + Map workerProps = new HashMap<>(); + workerProps.put("key.converter", "org.apache.kafka.connect.json.JsonConverter"); + workerProps.put("value.converter", "org.apache.kafka.connect.json.JsonConverter"); + workerProps.put("internal.key.converter", "org.apache.kafka.connect.json.JsonConverter"); + workerProps.put("internal.value.converter", "org.apache.kafka.connect.json.JsonConverter"); + workerProps.put("internal.key.converter.schemas.enable", "false"); + workerProps.put("internal.value.converter.schemas.enable", "false"); + workerProps.put("offset.storage.file.filename", "/tmp/connect.offsets"); + return workerProps; + } + + @Test + public void testCORSEnabled() { + checkCORSRequest("*", "http://bar.com", "http://bar.com"); + } + + @Test + public void testCORSDisabled() { + checkCORSRequest("", "http://bar.com", null); + } + + public void checkCORSRequest(String corsDomain, String origin, String expectedHeader) { + // To be able to set the Origin, we need to toggle this flag + System.setProperty("sun.net.http.allowRestrictedHeaders", "true"); + + final Capture>> connectorsCallback = EasyMock.newCapture(); + herder.connectors(EasyMock.capture(connectorsCallback)); + PowerMock.expectLastCall().andAnswer(new IAnswer() { + @Override + public Object answer() throws Throwable { + connectorsCallback.getValue().onCompletion(null, Arrays.asList("a", "b")); + return null; + } + }); + PowerMock.replayAll(); + + Map workerProps = baseWorkerProps(); + workerProps.put(WorkerConfig.ACCESS_CONTROL_ALLOW_ORIGIN_CONFIG, corsDomain); + WorkerConfig workerConfig = new StandaloneConfig(workerProps); + server = new RestServer(workerConfig); + server.start(herder); + + Response response = request("/connectors") + .header("Referer", origin + "/page") + .header("Origin", origin) + .get(); + assertEquals(200, response.getStatus()); + + assertEquals(expectedHeader, response.getHeaderString("Access-Control-Allow-Origin")); + PowerMock.verifyAll(); + } + + protected Invocation.Builder request(String path) { + return request(path, null, null, null); + } + + protected Invocation.Builder request(String path, Map queryParams) { + return request(path, null, null, queryParams); + } + + protected Invocation.Builder request(String path, String templateName, Object templateValue) { + return request(path, templateName, templateValue, null); + } + + protected Invocation.Builder request(String path, String templateName, Object templateValue, + Map queryParams) { + Client client = ClientBuilder.newClient(); + WebTarget target; + URI pathUri = null; + try { + pathUri = new URI(path); + } catch (URISyntaxException e) { + // Ignore, use restConnect and assume this is a valid path part + } + if (pathUri != null && pathUri.isAbsolute()) { + target = client.target(path); + } else { + target = client.target(server.advertisedUrl()).path(path); + } + if (templateName != null && templateValue != null) { + target = target.resolveTemplate(templateName, templateValue); + } + if (queryParams != null) { + for (Map.Entry queryParam : queryParams.entrySet()) { + target = target.queryParam(queryParam.getKey(), queryParam.getValue()); + } + } + return target.request(); + } +} diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index aa1d3f9bdf8f4..47158d64bf910 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -72,6 +72,7 @@ libs += [ jacksonJaxrsJsonProvider: "com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:$versions.jackson", jettyServer: "org.eclipse.jetty:jetty-server:$versions.jetty", jettyServlet: "org.eclipse.jetty:jetty-servlet:$versions.jetty", + jettyServlets: "org.eclipse.jetty:jetty-servlets:$versions.jetty", jerseyContainerServlet: "org.glassfish.jersey.containers:jersey-container-servlet:$versions.jersey", junit: "junit:junit:$versions.junit", joptSimple: "net.sf.jopt-simple:jopt-simple:$versions.jopt", From bfac36ad0e378b5f39e3889e40a75c5c1fc48fa7 Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Sun, 20 Mar 2016 00:46:12 -0700 Subject: [PATCH 060/206] =?UTF-8?q?KAFKA-3328:=20SimpleAclAuthorizer=20can?= =?UTF-8?q?=20lose=20ACLs=20with=20frequent=20add/remov=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …e calls Changes the SimpleAclAuthorizer to: - Track and utilize the zookeeper version when updating zookeeper to prevent data loss in the case of stale reads and race conditions - Update local cache when modifying ACLs - Add debug logging Author: Grant Henke Author: Grant Henke Author: Ismael Juma Reviewers: Flavio Junqueira, Jun Rao, Ismael Juma, Gwen Shapira Closes #1006 from granthenke/simple-authorizer-fix --- .../security/auth/SimpleAclAuthorizer.scala | 219 ++++++++++++------ core/src/main/scala/kafka/utils/ZkUtils.scala | 39 ++-- .../auth/SimpleAclAuthorizerTest.scala | 86 ++++++- .../scala/unit/kafka/utils/TestUtils.scala | 60 ++++- 4 files changed, 318 insertions(+), 86 deletions(-) diff --git a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala index 77e23f81cfe4e..1a06af2ad8fd2 100644 --- a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala +++ b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala @@ -19,19 +19,20 @@ package kafka.security.auth import java.util import java.util.concurrent.locks.ReentrantReadWriteLock import kafka.common.{NotificationHandler, ZkNodeChangeNotificationListener} -import org.apache.zookeeper.Watcher.Event.KeeperState - import kafka.network.RequestChannel.Session +import kafka.security.auth.SimpleAclAuthorizer.VersionedAcls import kafka.server.KafkaConfig import kafka.utils.CoreUtils.{inReadLock, inWriteLock} import kafka.utils._ -import org.I0Itec.zkclient.IZkStateListener +import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException} import org.apache.kafka.common.security.JaasUtils import org.apache.kafka.common.security.auth.KafkaPrincipal import scala.collection.JavaConverters._ import org.apache.log4j.Logger +import scala.util.Random + object SimpleAclAuthorizer { //optional override zookeeper cluster configuration where acls will be stored, if not specified acls will be stored in //same zookeeper where all other kafka broker info is stored. @@ -62,6 +63,8 @@ object SimpleAclAuthorizer { //prefix of all the change notification sequence node. val AclChangedPrefix = "acl_changes_" + + private case class VersionedAcls(acls: Set[Acl], zkVersion: Int) } class SimpleAclAuthorizer extends Authorizer with Logging { @@ -71,9 +74,16 @@ class SimpleAclAuthorizer extends Authorizer with Logging { private var zkUtils: ZkUtils = null private var aclChangeListener: ZkNodeChangeNotificationListener = null - private val aclCache = new scala.collection.mutable.HashMap[Resource, Set[Acl]] + private val aclCache = new scala.collection.mutable.HashMap[Resource, VersionedAcls] private val lock = new ReentrantReadWriteLock() + // The maximum number of times we should try to update the resource acls in zookeeper before failing; + // This should never occur, but is a safeguard just in case. + private val maxUpdateRetries = 10 + + private val retryBackoffMs = 100 + private val retryBackoffJitterMs = 50 + /** * Guaranteed to be called before any authorize call is made. */ @@ -164,67 +174,51 @@ class SimpleAclAuthorizer extends Authorizer with Logging { override def addAcls(acls: Set[Acl], resource: Resource) { if (acls != null && acls.nonEmpty) { - val updatedAcls = getAcls(resource) ++ acls - val path = toResourcePath(resource) - - if (zkUtils.pathExists(path)) - zkUtils.updatePersistentPath(path, Json.encode(Acl.toJsonCompatibleMap(updatedAcls))) - else - zkUtils.createPersistentPath(path, Json.encode(Acl.toJsonCompatibleMap(updatedAcls))) - - updateAclChangedFlag(resource) + inWriteLock(lock) { + updateResourceAcls(resource) { currentAcls => + currentAcls ++ acls + } + } } } override def removeAcls(aclsTobeRemoved: Set[Acl], resource: Resource): Boolean = { - if (zkUtils.pathExists(toResourcePath(resource))) { - val existingAcls = getAcls(resource) - val filteredAcls = existingAcls.filter((acl: Acl) => !aclsTobeRemoved.contains(acl)) - - val aclNeedsRemoval = (existingAcls != filteredAcls) - if (aclNeedsRemoval) { - val path: String = toResourcePath(resource) - if (filteredAcls.nonEmpty) - zkUtils.updatePersistentPath(path, Json.encode(Acl.toJsonCompatibleMap(filteredAcls))) - else - zkUtils.deletePath(toResourcePath(resource)) - - updateAclChangedFlag(resource) + inWriteLock(lock) { + updateResourceAcls(resource) { currentAcls => + currentAcls -- aclsTobeRemoved } - - aclNeedsRemoval - } else false + } } override def removeAcls(resource: Resource): Boolean = { - if (zkUtils.pathExists(toResourcePath(resource))) { - zkUtils.deletePath(toResourcePath(resource)) + inWriteLock(lock) { + val result = zkUtils.deletePath(toResourcePath(resource)) + updateCache(resource, VersionedAcls(Set(), 0)) updateAclChangedFlag(resource) - true - } else false + result + } } override def getAcls(resource: Resource): Set[Acl] = { inReadLock(lock) { - aclCache.get(resource).getOrElse(Set.empty[Acl]) + aclCache.get(resource).map(_.acls).getOrElse(Set.empty[Acl]) } } - private def getAclsFromZk(resource: Resource): Set[Acl] = { - val aclJson = zkUtils.readDataMaybeNull(toResourcePath(resource))._1 - aclJson.map(Acl.fromJson).getOrElse(Set.empty) - } - override def getAcls(principal: KafkaPrincipal): Map[Resource, Set[Acl]] = { - aclCache.mapValues { acls => - acls.filter(_.principal == principal) - }.filter { case (_, acls) => - acls.nonEmpty - }.toMap + inReadLock(lock) { + aclCache.mapValues { versionedAcls => + versionedAcls.acls.filter(_.principal == principal) + }.filter { case (_, acls) => + acls.nonEmpty + }.toMap + } } override def getAcls(): Map[Resource, Set[Acl]] = { - aclCache.toMap + inReadLock(lock) { + aclCache.mapValues(_.acls).toMap + } } def close() { @@ -233,25 +227,17 @@ class SimpleAclAuthorizer extends Authorizer with Logging { } private def loadCache() { - var acls = Set.empty[Acl] - val resourceTypes = zkUtils.getChildren(SimpleAclAuthorizer.AclZkPath) - for (rType <- resourceTypes) { - val resourceType = ResourceType.fromString(rType) - val resourceTypePath = SimpleAclAuthorizer.AclZkPath + "/" + resourceType.name - val resourceNames = zkUtils.getChildren(resourceTypePath) - for (resourceName <- resourceNames) { - acls = getAclsFromZk(Resource(resourceType, resourceName.toString)) - updateCache(new Resource(resourceType, resourceName), acls) - } - } - } - - private def updateCache(resource: Resource, acls: Set[Acl]) { inWriteLock(lock) { - if (acls.nonEmpty) - aclCache.put(resource, acls) - else - aclCache.remove(resource) + val resourceTypes = zkUtils.getChildren(SimpleAclAuthorizer.AclZkPath) + for (rType <- resourceTypes) { + val resourceType = ResourceType.fromString(rType) + val resourceTypePath = SimpleAclAuthorizer.AclZkPath + "/" + resourceType.name + val resourceNames = zkUtils.getChildren(resourceTypePath) + for (resourceName <- resourceNames) { + val versionedAcls = getAclsFromZk(Resource(resourceType, resourceName.toString)) + updateCache(new Resource(resourceType, resourceName), versionedAcls) + } + } } } @@ -264,16 +250,117 @@ class SimpleAclAuthorizer extends Authorizer with Logging { authorizerLogger.debug(s"Principal = $principal is $permissionType Operation = $operation from host = $host on resource = $resource") } + /** + * Safely updates the resources ACLs by ensuring reads and writes respect the expected zookeeper version. + * Continues to retry until it succesfully updates zookeeper. + * + * Returns a boolean indicating if the content of the ACLs was actually changed. + * + * @param resource the resource to change ACLs for + * @param getNewAcls function to transform existing acls to new ACLs + * @return boolean indicating if a change was made + */ + private def updateResourceAcls(resource: Resource)(getNewAcls: Set[Acl] => Set[Acl]): Boolean = { + val path = toResourcePath(resource) + + var currentVersionedAcls = + if (aclCache.contains(resource)) + getAclsFromCache(resource) + else + getAclsFromZk(resource) + var newVersionedAcls: VersionedAcls = null + var writeComplete = false + var retries = 0 + while (!writeComplete && retries <= maxUpdateRetries) { + val newAcls = getNewAcls(currentVersionedAcls.acls) + val data = Json.encode(Acl.toJsonCompatibleMap(newAcls)) + val (updateSucceeded, updateVersion) = + if (!newAcls.isEmpty) { + updatePath(path, data, currentVersionedAcls.zkVersion) + } else { + trace(s"Deleting path for $resource because it had no ACLs remaining") + (zkUtils.conditionalDeletePath(path, currentVersionedAcls.zkVersion), 0) + } + + if (!updateSucceeded) { + trace(s"Failed to update ACLs for $resource. Used version ${currentVersionedAcls.zkVersion}. Reading data and retrying update.") + Thread.sleep(backoffTime) + currentVersionedAcls = getAclsFromZk(resource); + retries += 1 + } else { + newVersionedAcls = VersionedAcls(newAcls, updateVersion) + writeComplete = updateSucceeded + } + } + + if(!writeComplete) + throw new IllegalStateException(s"Failed to update ACLs for $resource after trying a maximum of $maxUpdateRetries times") + + if (newVersionedAcls.acls != currentVersionedAcls.acls) { + debug(s"Updated ACLs for $resource to ${newVersionedAcls.acls} with version ${newVersionedAcls.zkVersion}") + updateCache(resource, newVersionedAcls) + updateAclChangedFlag(resource) + true + } else { + debug(s"Updated ACLs for $resource, no change was made") + updateCache(resource, newVersionedAcls) // Even if no change, update the version + false + } + } + + /** + * Updates a zookeeper path with an expected version. If the topic does not exist, it will create it. + * Returns if the update was successful and the new version. + */ + private def updatePath(path: String, data: String, expectedVersion: Int): (Boolean, Int) = { + try { + zkUtils.conditionalUpdatePersistentPathIfExists(path, data, expectedVersion) + } catch { + case e: ZkNoNodeException => + try { + debug(s"Node $path does not exist, attempting to create it.") + zkUtils.createPersistentPath(path, data) + (true, 0) + } catch { + case e: ZkNodeExistsException => + debug(s"Failed to create node for $path because it already exists.") + (false, 0) + } + } + } + + private def getAclsFromCache(resource: Resource): VersionedAcls = { + aclCache.getOrElse(resource, throw new IllegalArgumentException(s"ACLs do not exist in the cache for resource $resource")) + } + + private def getAclsFromZk(resource: Resource): VersionedAcls = { + val (aclJson, stat) = zkUtils.readDataMaybeNull(toResourcePath(resource)) + VersionedAcls(aclJson.map(Acl.fromJson).getOrElse(Set()), stat.getVersion) + } + + private def updateCache(resource: Resource, versionedAcls: VersionedAcls) { + if (versionedAcls.acls.nonEmpty) { + aclCache.put(resource, versionedAcls) + } else { + aclCache.remove(resource) + } + } + private def updateAclChangedFlag(resource: Resource) { zkUtils.createSequentialPersistentPath(SimpleAclAuthorizer.AclChangedZkPath + "/" + SimpleAclAuthorizer.AclChangedPrefix, resource.toString) } - object AclChangedNotificationHandler extends NotificationHandler { + private def backoffTime = { + retryBackoffMs + Random.nextInt(retryBackoffJitterMs) + } + object AclChangedNotificationHandler extends NotificationHandler { override def processNotification(notificationMessage: String) { val resource: Resource = Resource.fromString(notificationMessage) - val acls = getAclsFromZk(resource) - updateCache(resource, acls) + inWriteLock(lock) { + val versionedAcls = getAclsFromZk(resource) + updateCache(resource, versionedAcls) + } } } } diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 99c8196a03897..49d3cfaaf8bfe 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -52,12 +52,12 @@ object ZkUtils { val IsrChangeNotificationPath = "/isr_change_notification" val EntityConfigPath = "/config" val EntityConfigChangesPath = "/config/changes" - + def apply(zkUrl: String, sessionTimeout: Int, connectionTimeout: Int, isZkSecurityEnabled: Boolean): ZkUtils = { val (zkClient, zkConnection) = createZkClientAndConnection(zkUrl, sessionTimeout, connectionTimeout) new ZkUtils(zkClient, zkConnection, isZkSecurityEnabled) } - + /* * Used in tests */ @@ -75,7 +75,7 @@ object ZkUtils { val zkClient = new ZkClient(zkConnection, connectionTimeout, ZKStringSerializer) (zkClient, zkConnection) } - + def DefaultAcls(isSecure: Boolean): java.util.List[ACL] = if (isSecure) { val list = new java.util.ArrayList[ACL] list.addAll(ZooDefs.Ids.CREATOR_ALL_ACL) @@ -84,7 +84,7 @@ object ZkUtils { } else { ZooDefs.Ids.OPEN_ACL_UNSAFE } - + def maybeDeletePath(zkUrl: String, dir: String) { try { val zk = createZkClient(zkUrl, 30*1000, 30*1000) @@ -94,7 +94,7 @@ object ZkUtils { case _: Throwable => // swallow } } - + /* * Get calls that only depend on static paths */ @@ -111,7 +111,7 @@ object ZkUtils { def getTopicPartitionLeaderAndIsrPath(topic: String, partitionId: Int): String = getTopicPartitionPath(topic, partitionId) + "/" + "state" - + def getEntityConfigRootPath(entityType: String): String = ZkUtils.EntityConfigPath + "/" + entityType @@ -122,7 +122,7 @@ object ZkUtils { DeleteTopicsPath + "/" + topic } -class ZkUtils(val zkClient: ZkClient, +class ZkUtils(val zkClient: ZkClient, val zkConnection: ZkConnection, val isSecure: Boolean) extends Logging { // These are persistent ZK paths that should exist on kafka broker startup. @@ -146,7 +146,7 @@ class ZkUtils(val zkClient: ZkClient, IsrChangeNotificationPath) val DefaultAcls: java.util.List[ACL] = ZkUtils.DefaultAcls(isSecure) - + def getController(): Int = { readDataMaybeNull(ControllerPath)._1 match { case Some(controller) => KafkaController.parseControllerId(controller) @@ -512,6 +512,19 @@ class ZkUtils(val zkClient: ZkClient, } } + /** + * Conditional delete the persistent path data, return true if it succeeds, + * otherwise (the current version is not the expected version) + */ + def conditionalDeletePath(path: String, expectedVersion: Int): Boolean = { + try { + zkClient.delete(path, expectedVersion) + true + } catch { + case e: KeeperException.BadVersionException => false + } + } + def deletePathRecursive(path: String) { try { zkClient.deleteRecursive(path) @@ -847,7 +860,7 @@ class ZkUtils(val zkClient: ZkClient, } } } - + def close() { if(zkClient != null) { zkClient.close() @@ -941,7 +954,7 @@ object ZkPath { * znode is created and the create call returns OK. If * the call receives a node exists event, then it checks * if the session matches. If it does, then it returns OK, - * and otherwise it fails the operation. + * and otherwise it fails the operation. */ class ZKCheckedEphemeral(path: String, @@ -952,7 +965,7 @@ class ZKCheckedEphemeral(path: String, private val getDataCallback = new GetDataCallback val latch: CountDownLatch = new CountDownLatch(1) var result: Code = Code.OK - + private class CreateCallback extends StringCallback { def processResult(rc: Int, path: String, @@ -1009,7 +1022,7 @@ class ZKCheckedEphemeral(path: String, } } } - + private def createEphemeral() { zkHandle.create(path, ZKStringSerializer.serialize(data), @@ -1018,7 +1031,7 @@ class ZKCheckedEphemeral(path: String, createCallback, null) } - + private def createRecursive(prefix: String, suffix: String) { debug("Path: %s, Prefix: %s, Suffix: %s".format(path, prefix, suffix)) if(suffix.isEmpty()) { diff --git a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala index efcf930ac6461..bdadb15f430d9 100644 --- a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala +++ b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala @@ -17,7 +17,7 @@ package kafka.security.auth import java.net.InetAddress -import java.util.UUID +import java.util.{UUID} import kafka.network.RequestChannel.Session import kafka.security.auth.Acl.WildCardHost @@ -31,6 +31,7 @@ import org.junit.{After, Before, Test} class SimpleAclAuthorizerTest extends ZooKeeperTestHarness { val simpleAclAuthorizer = new SimpleAclAuthorizer + val simpleAclAuthorizer2 = new SimpleAclAuthorizer val testPrincipal = Acl.WildCardPrincipal val testHostName = InetAddress.getByName("192.168.0.1") val session = new Session(testPrincipal, testHostName) @@ -48,12 +49,14 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness { config = KafkaConfig.fromProps(props) simpleAclAuthorizer.configure(config.originals) + simpleAclAuthorizer2.configure(config.originals) resource = new Resource(Topic, UUID.randomUUID().toString) } @After override def tearDown(): Unit = { simpleAclAuthorizer.close() + simpleAclAuthorizer2.close() } @Test @@ -254,6 +257,87 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness { assertEquals(acls1, authorizer.getAcls(resource1)) } + @Test + def testLocalConcurrentModificationOfResourceAcls() { + val commonResource = new Resource(Topic, "test") + + val user1 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username) + val acl1 = new Acl(user1, Allow, WildCardHost, Read) + + val user2 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob") + val acl2 = new Acl(user2, Deny, WildCardHost, Read) + + simpleAclAuthorizer.addAcls(Set(acl1), commonResource) + simpleAclAuthorizer.addAcls(Set(acl2), commonResource) + + TestUtils.waitAndVerifyAcls(Set(acl1, acl2), simpleAclAuthorizer, commonResource) + } + + @Test + def testDistributedConcurrentModificationOfResourceAcls() { + val commonResource = new Resource(Topic, "test") + + val user1 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, username) + val acl1 = new Acl(user1, Allow, WildCardHost, Read) + + val user2 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "bob") + val acl2 = new Acl(user2, Deny, WildCardHost, Read) + + // Add on each instance + simpleAclAuthorizer.addAcls(Set(acl1), commonResource) + simpleAclAuthorizer2.addAcls(Set(acl2), commonResource) + + TestUtils.waitAndVerifyAcls(Set(acl1, acl2), simpleAclAuthorizer, commonResource) + TestUtils.waitAndVerifyAcls(Set(acl1, acl2), simpleAclAuthorizer2, commonResource) + + val user3 = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, "joe") + val acl3 = new Acl(user3, Deny, WildCardHost, Read) + + // Add on one instance and delete on another + simpleAclAuthorizer.addAcls(Set(acl3), commonResource) + val deleted = simpleAclAuthorizer2.removeAcls(Set(acl3), commonResource) + + assertTrue("The authorizer should see a value that needs to be deleted", deleted) + + TestUtils.waitAndVerifyAcls(Set(acl1, acl2), simpleAclAuthorizer, commonResource) + TestUtils.waitAndVerifyAcls(Set(acl1, acl2), simpleAclAuthorizer2, commonResource) + } + + @Test + def testHighConcurrencyModificationOfResourceAcls() { + val commonResource = new Resource(Topic, "test") + + val acls = (0 to 100).map { i => + val useri = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, i.toString) + new Acl(useri, Allow, WildCardHost, Read) + } + + // Alternate authorizer, Remove all acls that end in 0 + val concurrentFuctions = acls.map { acl => + () => { + val aclId = acl.principal.getName.toInt + if (aclId % 2 == 0) { + simpleAclAuthorizer.addAcls(Set(acl), commonResource) + } else { + simpleAclAuthorizer2.addAcls(Set(acl), commonResource) + } + if (aclId % 10 == 0) { + simpleAclAuthorizer2.removeAcls(Set(acl), commonResource) + } + } + } + + val expectedAcls = acls.filter { acl => + val aclId = acl.principal.getName.toInt + aclId % 10 != 0 + }.toSet + + TestUtils.assertConcurrent("Should support many concurrent calls", concurrentFuctions, 15000) + + TestUtils.waitAndVerifyAcls(expectedAcls, simpleAclAuthorizer, commonResource) + TestUtils.waitAndVerifyAcls(expectedAcls, simpleAclAuthorizer2, commonResource) + } + private def changeAclAndVerify(originalAcls: Set[Acl], addedAcls: Set[Acl], removedAcls: Set[Acl], resource: Resource = resource): Set[Acl] = { var acls = originalAcls diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 7b3e95590ce77..0730468a496b5 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -21,8 +21,9 @@ import java.io._ import java.nio._ import java.nio.file.Files import java.nio.channels._ -import java.util.Random -import java.util.Properties +import java.util +import java.util.concurrent.{Callable, TimeUnit, Executors} +import java.util.{Collections, Random, Properties} import java.security.cert.X509Certificate import javax.net.ssl.X509TrustManager import charset.Charset @@ -54,6 +55,7 @@ import org.apache.kafka.common.serialization.{ByteArraySerializer, Serializer} import scala.collection.Map import scala.collection.JavaConversions._ +import scala.collection.JavaConverters._ /** * Utility functions to help with testing @@ -131,6 +133,7 @@ object TestUtils extends Logging { /** * Create a kafka server instance with appropriate test settings * USING THIS IS A SIGN YOU ARE NOT WRITING A REAL UNIT TEST + * * @param config The configuration of the server */ def createServer(config: KafkaConfig, time: Time = SystemTime): KafkaServer = { @@ -141,7 +144,7 @@ object TestUtils extends Logging { /** * Create a test config for the provided parameters. - * + * * Note that if `interBrokerSecurityProtocol` is defined, the listener for the `SecurityProtocol` will be enabled. */ def createBrokerConfigs(numConfigs: Int, @@ -281,6 +284,7 @@ object TestUtils extends Logging { /** * Wrap the message in a message set + * * @param payload The bytes of the message */ def singleMessageSet(payload: Array[Byte], @@ -291,6 +295,7 @@ object TestUtils extends Logging { /** * Generate an array of random bytes + * * @param numBytes The size of the array */ def randomBytes(numBytes: Int): Array[Byte] = { @@ -301,6 +306,7 @@ object TestUtils extends Logging { /** * Generate a random string of letters and digits of the given length + * * @param len The length of the string * @return The random string */ @@ -679,6 +685,7 @@ object TestUtils extends Logging { * If neither oldLeaderOpt nor newLeaderOpt is defined, wait until the leader of a partition is elected. * If oldLeaderOpt is defined, it waits until the new leader is different from the old leader. * If newLeaderOpt is defined, it waits until the new leader becomes the expected new leader. + * * @return The new leader or assertion failure if timeout is reached. */ def waitUntilLeaderIsElectedOrChanged(zkUtils: ZkUtils, topic: String, partition: Int, timeoutMs: Long = 5000L, @@ -786,6 +793,7 @@ object TestUtils extends Logging { /** * Wait until a valid leader is propagated to the metadata cache in each broker. * It assumes that the leader propagated to each broker is the same. + * * @param servers The list of servers that the metadata should reach to * @param topic The topic name * @param partition The partition Id @@ -812,7 +820,7 @@ object TestUtils extends Logging { } def waitUntilLeaderIsKnown(servers: Seq[KafkaServer], topic: String, partition: Int, timeout: Long = 5000L): Unit = { - TestUtils.waitUntilTrue(() => + TestUtils.waitUntilTrue(() => servers.exists { server => server.replicaManager.getPartition(topic, partition).exists(_.leaderReplicaIfLocal().isDefined) }, @@ -968,12 +976,11 @@ object TestUtils extends Logging { /** * Consume all messages (or a specific number of messages) + * * @param topicMessageStreams the Topic Message Streams * @param nMessagesPerThread an optional field to specify the exact number of messages to be returned. * ConsumerTimeoutException will be thrown if there are no messages to be consumed. * If not specified, then all available messages will be consumed, and no exception is thrown. - * - * * @return the list of messages consumed. */ def getMessages(topicMessageStreams: Map[String, List[KafkaStream[String, String]]], @@ -1033,6 +1040,7 @@ object TestUtils extends Logging { /** * Translate the given buffer into a string + * * @param buffer The buffer to translate * @param encoding The encoding to use in translating bytes to characters */ @@ -1075,6 +1083,46 @@ object TestUtils extends Logging { s"expected acls $expected but got ${authorizer.getAcls(resource)}", waitTime = 10000) } + /** + * To use this you pass in a sequence of functions that are your arrange/act/assert test on the SUT. + * They all run at the same time in the assertConcurrent method; the chances of triggering a multithreading code error, + * and thereby failing some assertion are greatly increased. + */ + def assertConcurrent(message: String, functions: Seq[() => Any], timeoutMs: Int) { + + def failWithTimeout() { + fail(s"$message. Timed out, the concurrent functions took more than $timeoutMs milliseconds") + } + + val numThreads = functions.size + val threadPool = Executors.newFixedThreadPool(numThreads) + val exceptions = ArrayBuffer[Throwable]() + try { + val runnables = functions.map { function => + new Callable[Unit] { + override def call(): Unit = function() + } + }.asJava + val futures = threadPool.invokeAll(runnables, timeoutMs, TimeUnit.MILLISECONDS).asScala + futures.foreach { future => + if (future.isCancelled) + failWithTimeout() + else + try future.get() + catch { case e: Exception => + exceptions += e + } + } + } catch { + case ie: InterruptedException => failWithTimeout() + case e => exceptions += e + } finally { + threadPool.shutdownNow() + } + assertTrue(s"$message failed with exception(s) $exceptions", exceptions.isEmpty) + + } + } class IntEncoder(props: VerifiableProperties = null) extends Encoder[Int] { From c188a68e2b487191f1f3004e22b68c21e26c3f2e Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Sun, 20 Mar 2016 16:46:42 -0700 Subject: [PATCH 061/206] MINOR: Add vagrant up wrapper for simple parallel bringup on aws The main impediment to bringing up aws machines in parallel using vagrant was the interaction between `vagrant-hostmanager` and `vagrant-aws`. If you disable hostmanager during the `up` phase, and run it after the cluster is up, parallel bringup is possible. The only caveat is that machines must be brought up in small-ish batches to prevent rate limit errors from AWS since `vagrant-aws` doesn't seem to have mechanisms to This PR: - disables `vagrant-hostmanager` during bringup - adds a wrapper script to make it convenient to bring machines up in batches on aws Author: Geoff Anderson Reviewers: Ewen Cheslack-Postava Closes #982 from granders/vagrant-disable-hostmanager --- Vagrantfile | 4 +- tests/README.md | 9 +- vagrant/README.md | 17 +- vagrant/aws/aws-example-Vagrantfile.local | 1 + vagrant/vagrant-up.sh | 237 ++++++++++++++++++++++ 5 files changed, 258 insertions(+), 10 deletions(-) create mode 100755 vagrant/vagrant-up.sh diff --git a/Vagrantfile b/Vagrantfile index 51066ff5bde53..0471a7e78df2f 100644 --- a/Vagrantfile +++ b/Vagrantfile @@ -22,6 +22,8 @@ VAGRANTFILE_API_VERSION = "2" # General config enable_dns = false +# Override to false when bringing up a cluster on AWS +enable_hostmanager = true enable_jmx = false num_zookeepers = 1 num_brokers = 3 @@ -55,7 +57,7 @@ end # TODO(ksweeney): RAM requirements are not empirical and can probably be significantly lowered. Vagrant.configure(VAGRANTFILE_API_VERSION) do |config| - config.hostmanager.enabled = true + config.hostmanager.enabled = enable_hostmanager config.hostmanager.manage_host = enable_dns config.hostmanager.include_offline = false diff --git a/tests/README.md b/tests/README.md index 045732f49f5e1..143711d8b4927 100644 --- a/tests/README.md +++ b/tests/README.md @@ -26,7 +26,8 @@ https://cwiki.apache.org/confluence/display/KAFKA/tutorial+-+set+up+and+run+Kafk * Bring up the test cluster - $ vagrant up + $ vagrant/vagrant-up.sh + $ # When using Virtualbox, it also works to run: vagrant up * Build the desired branch of Kafka @@ -111,6 +112,7 @@ the test driver machine. ec2_instance_type = "..." # Pick something appropriate for your # test. Note that the default m3.medium has # a small disk. + enable_hostmanager = false num_zookeepers = 0 num_kafka = 0 num_workers = 9 @@ -120,9 +122,10 @@ the test driver machine. ec2_region = 'us-west-2' ec2_ami = "ami-29ebb519" -* Start up the instances (note we have found bringing up machines in parallel can cause errors on aws): +* Start up the instances: - $ vagrant up --provider=aws --no-provision --no-parallel && vagrant provision + # This will brink up worker machines in small parallel batches + $ vagrant/vagrant-up.sh --aws * Now you should be able to run tests: diff --git a/vagrant/README.md b/vagrant/README.md index 47c78767ddfd0..7021010bdc4cc 100644 --- a/vagrant/README.md +++ b/vagrant/README.md @@ -27,11 +27,15 @@ clusters concurrently. Now bring up the cluster: - $ vagrant up --no-provision && vagrant provision + $ vagrant/vagrant-up.sh + $ # If on aws, run: vagrant/vagrant-up.sh --aws -We separate out the two steps (bringing up the base VMs and configuring them) +(This essentially runs vagrant up --no-provision && vagrant hostmanager && vagrant provision) + +We separate out the steps (bringing up the base VMs, mapping hostnames, and configuring the VMs) due to current limitations in ZooKeeper (ZOOKEEPER-1506) that require us to -collect IPs for all nodes before starting ZooKeeper nodes. +collect IPs for all nodes before starting ZooKeeper nodes. Breaking into multiple steps +also allows us to bring machies up in parallel on AWS. Once this completes: @@ -66,7 +70,7 @@ the cluster to your most recent development version. Finally, you can clean up the cluster by destroying all the VMs: - vagrant destroy + vagrant destroy -f ## Configuration ## @@ -75,6 +79,7 @@ You can override some default settings by specifying the values in only ever need to change a few simple configuration variables. Some values you might want to override: +* `enable_hostmanager` - true by default; override to false if on AWS to allow parallel cluster bringup. * `enable_dns` - Register each VM with a hostname in /etc/hosts on the hosts. Hostnames are always set in the /etc/hosts in the VMs, so this is only necessary if you want to address them conveniently from the host for tasks @@ -96,7 +101,7 @@ Install the `vagrant-aws` plugin to provide EC2 support: $ vagrant plugin install vagrant-aws Next, configure parameters in `Vagrantfile.local`. A few are *required*: -`enable_dns`, `ec2_access_key`, `ec2_secret_key`, `ec2_keypair_name`, `ec2_keypair_file`, and +`enable_hostmanager`, `enable_dns`, `ec2_access_key`, `ec2_secret_key`, `ec2_keypair_name`, `ec2_keypair_file`, and `ec2_security_groups`. A couple of important notes: 1. You definitely want to use `enable_dns` if you plan to run clients outside of @@ -122,7 +127,7 @@ Next, configure parameters in `Vagrantfile.local`. A few are *required*: Now start things up, but specify the aws provider: - $ vagrant up --provider=aws --no-parallel --no-provision && vagrant provision + $ vagrant/vagrant-up.sh Your instances should get tagged with a name including your hostname to make them identifiable and make it easier to track instances in the AWS management diff --git a/vagrant/aws/aws-example-Vagrantfile.local b/vagrant/aws/aws-example-Vagrantfile.local index 00b3d6164ee71..853671eb6f3ab 100644 --- a/vagrant/aws/aws-example-Vagrantfile.local +++ b/vagrant/aws/aws-example-Vagrantfile.local @@ -18,6 +18,7 @@ # To use it, move it to the base kafka directory and rename # it to Vagrantfile.local, and adjust variables as needed. ec2_instance_type = "m3.xlarge" +enable_hostmanager = false num_zookeepers = 0 num_brokers = 0 num_workers = 9 diff --git a/vagrant/vagrant-up.sh b/vagrant/vagrant-up.sh new file mode 100755 index 0000000000000..ad5d5be0dce73 --- /dev/null +++ b/vagrant/vagrant-up.sh @@ -0,0 +1,237 @@ +# 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. + +#! /usr/bin/env bash + +set -o nounset +set -o errexit # exit script if any command exits with nonzero value + +readonly PROG_NAME=$(basename $0) +readonly PROG_DIR=$(dirname $(realpath $0)) +readonly INVOKE_DIR=$(pwd) +readonly ARGS="$@" + +# overrideable defaults +AWS=false +PARALLEL=true +MAX_PARALLEL=5 + +readonly USAGE="Usage: $PROG_NAME [-h | --help] [--aws [--no-parallel] [--max-parallel MAX]]" +readonly HELP="$(cat < 0 ]]; do + key="$1" + case $key in + -h | --help) + help + ;; + --aws) + AWS=true + ;; + --no-parallel) + PARALLEL=false + ;; + --max-parallel) + MAX_PARALLEL="$2" + shift + ;; + *) + # unknown option + echo "Unknown option $1" + exit 1 + ;; +esac +shift # past argument or value +done + +# Get a list of vagrant machines (in any state) +function read_vagrant_machines { + local ignore_state="ignore" + local reading_state="reading" + local tmp_file="tmp-$RANDOM" + + local state="$ignore_state" + local machines="" + + while read -r line; do + # Lines before the first empty line are ignored + # The first empty line triggers change from ignore state to reading state + # When in reading state, we parse in machine names until we hit the next empty line, + # which signals that we're done parsing + if [[ -z "$line" ]]; then + if [[ "$state" == "$ignore_state" ]]; then + state="$reading_state" + else + # all done + echo "$machines" + return + fi + continue + fi + + # Parse machine name while in reading state + if [[ "$state" == "$reading_state" ]]; then + line=$(echo "$line" | cut -d ' ' -f 1) + if [[ -z "$machines" ]]; then + machines="$line" + else + machines="${machines} ${line}" + fi + fi + done < <(vagrant status) +} + +# Filter "list", returning a list of strings containing pattern as a substring +function filter { + local list="$1" + local pattern="$2" + + local result="" + for item in $list; do + if [[ ! -z "$(echo $item | grep "$pattern")" ]]; then + result="$result $item" + fi + done + echo "$result" +} + +# Given a list of machine names, return only test worker machines +function worker { + local machines="$1" + local workers=$(filter "$machines" "worker") + workers=$(echo "$workers" | xargs) # trim leading/trailing whitespace + echo "$workers" +} + +# Given a list of machine names, return only zookeeper and broker machines +function zk_broker { + local machines="$1" + local zk_broker_list=$(filter "$machines" "zk") + zk_broker_list="$zk_broker_list $(filter "$machines" "broker")" + zk_broker_list=$(echo "$zk_broker_list" | xargs) # trim leading/trailing whitespace + echo "$zk_broker_list" +} + +# Run a vagrant command on batches of machines of size $group_size +# This is annoying but necessary on aws to avoid errors due to AWS request rate +# throttling +# +# Example +# $ vagrant_batch_command "vagrant up" "m1 m2 m3 m4 m5" "2" +# +# This is equivalent to running "vagrant up" on groups of machines of size 2 or less, i.e.: +# $ vagrant up m1 m2 +# $ vagrant up m3 m4 +# $ vagrant up m5 +function vagrant_batch_command { + local vagrant_cmd="$1" + local machines="$2" + local group_size="$3" + + local count=1 + local m_group="" + # Using --provision flag makes this command useable both when bringing up a cluster from scratch, + # and when bringing up a halted cluster. Permissions on certain directores set during provisioning + # seem to revert when machines are halted, so --provision ensures permissions are set correctly in all cases + for machine in $machines; do + m_group="$m_group $machine" + + if [[ $(expr $count % $group_size) == 0 ]]; then + # We've reached a full group + # Bring up this part of the cluster + $vagrant_cmd $m_group + m_group="" + fi + ((count++)) + done + + # Take care of any leftover partially complete group + if [[ ! -z "$m_group" ]]; then + $vagrant_cmd $m_group + fi +} + +# We assume vagrant-hostmanager is installed, but may or may not be disabled during vagrant up +# In this fashion, we ensure we run hostmanager after machines are up, and before provisioning. +# This sequence of commands is necessary for example for bringing up a multi-node zookeeper cluster +function bring_up_local { + vagrant up --no-provision + vagrant hostmanager + vagrant provision +} + +function bring_up_aws { + local parallel="$1" + local max_parallel="$2" + local machines="$(read_vagrant_machines)" + + zk_broker_machines=$(zk_broker "$machines") + worker_machines=$(worker "$machines") + + if [[ "$parallel" == "true" ]]; then + if [[ ! -z "$zk_broker_machines" ]]; then + # We still have to bring up zookeeper/broker nodes serially + echo "Bringing up zookeeper/broker machines serially" + vagrant up --provider=aws --no-parallel --no-provision $zk_broker_machines + vagrant hostmanager + vagrant provision + fi + + if [[ ! -z "$worker_machines" ]]; then + echo "Bringing up test worker machines in parallel" + vagrant_batch_command "vagrant up --provider=aws" "$worker_machines" "$max_parallel" + vagrant hostmanager + fi + else + vagrant up --provider=aws --no-parallel --no-provision + vagrant hostmanager + vagrant provision + fi +} + +function main { + if [[ "$AWS" == "true" ]]; then + bring_up_aws "$PARALLEL" "$MAX_PARALLEL" + else + bring_up_local + fi +} + +main From 95eabc8c8b383af84466d4c2cfafd0920e5a52ee Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Sun, 20 Mar 2016 19:58:31 -0700 Subject: [PATCH 062/206] KAFKA-3378; Follow-up to ensure we `finishConnect` for immediately connected keys Author: Ismael Juma Reviewers: Larkin Lowrey , Jun Rao Closes #1103 from ijuma/kafka-3378-follow-up --- .../java/org/apache/kafka/common/network/Selector.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java index f9e232dcc5f82..698b99c3b8384 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java @@ -280,8 +280,8 @@ public void poll(long timeout) throws IOException { this.sensors.selectTime.record(endSelect - startSelect, time.milliseconds()); if (readyKeys > 0 || !immediatelyConnectedKeys.isEmpty()) { - pollSelectionKeys(this.nioSelector.selectedKeys()); - pollSelectionKeys(immediatelyConnectedKeys); + pollSelectionKeys(this.nioSelector.selectedKeys(), false); + pollSelectionKeys(immediatelyConnectedKeys, true); } addToCompletedReceives(); @@ -291,7 +291,7 @@ public void poll(long timeout) throws IOException { maybeCloseOldestConnection(); } - private void pollSelectionKeys(Iterable selectionKeys) { + private void pollSelectionKeys(Iterable selectionKeys, boolean isImmediatelyConnected) { Iterator iterator = selectionKeys.iterator(); while (iterator.hasNext()) { SelectionKey key = iterator.next(); @@ -304,8 +304,8 @@ private void pollSelectionKeys(Iterable selectionKeys) { try { - /* complete any connections that have finished their handshake */ - if (key.isConnectable()) { + /* complete any connections that have finished their handshake (either normally or immediately) */ + if (isImmediatelyConnected || key.isConnectable()) { if (channel.finishConnect()) { this.connected.add(channel.id()); this.sensors.connectionCreated.record(); From 4f048c4f194a90ded5f0df35e4e23379272d5bc6 Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Sun, 20 Mar 2016 20:19:54 -0700 Subject: [PATCH 063/206] MINOR: Fix FetchRequest.getErrorResponse for version 1 Author: Grant Henke Reviewers: Ismael Juma , Jun Rao Closes #1091 from granthenke/fetch-error --- .../org/apache/kafka/common/requests/FetchRequest.java | 8 +++++--- .../apache/kafka/common/requests/RequestResponseTest.java | 3 ++- 2 files changed, 7 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java index a7d8349281e1c..f8b7fe3a6da78 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/FetchRequest.java @@ -3,9 +3,9 @@ * 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. @@ -27,7 +27,7 @@ import org.apache.kafka.common.utils.CollectionUtils; public class FetchRequest extends AbstractRequest { - + public static final int CONSUMER_REPLICA_ID = -1; private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.FETCH.id); private static final String REPLICA_ID_KEY_NAME = "replica_id"; @@ -132,6 +132,8 @@ public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { switch (versionId) { case 0: + return new FetchResponse(responseData); + case 1: return new FetchResponse(responseData, 0); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index b7f0caf192595..a4c5238bcf02a 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -51,7 +51,7 @@ public void testSerialization() throws Exception { createControlledShutdownResponse(), createControlledShutdownRequest().getErrorResponse(1, new UnknownServerException()), createFetchRequest(), - createFetchRequest().getErrorResponse(0, new UnknownServerException()), + createFetchRequest().getErrorResponse(1, new UnknownServerException()), createFetchResponse(), createHeartBeatRequest(), createHeartBeatRequest().getErrorResponse(0, new UnknownServerException()), @@ -98,6 +98,7 @@ public void testSerialization() throws Exception { for (AbstractRequestResponse req : requestResponseList) checkSerialization(req, null); + checkSerialization(createFetchRequest().getErrorResponse(0, new UnknownServerException()), 0); checkSerialization(createOffsetCommitRequest(0), 0); checkSerialization(createOffsetCommitRequest(0).getErrorResponse(0, new UnknownServerException()), 0); checkSerialization(createOffsetCommitRequest(1), 1); From 702d560c555bf7121eca02010adedd4986e36b87 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Mon, 21 Mar 2016 09:53:17 -0700 Subject: [PATCH 064/206] Changing version to 0.10.1.0-SNAPSHOT --- gradle.properties | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/gradle.properties b/gradle.properties index b058e58cbe301..0a612f6c87108 100644 --- a/gradle.properties +++ b/gradle.properties @@ -16,7 +16,7 @@ group=org.apache.kafka # NOTE: When you change this version number, you should also make sure to update # the version numbers in tests/kafkatest/__init__.py and kafka-merge-pr.py. -version=0.10.0.0-SNAPSHOT +version=0.10.1.0-SNAPSHOT scalaVersion=2.10.6 task=build org.gradle.jvmargs=-XX:MaxPermSize=512m -Xmx1024m -Xss2m From b6c29e3810bd59f39fa93c429817396cf8c324b7 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 21 Mar 2016 12:06:07 -0700 Subject: [PATCH 065/206] MINOR: Add InterfaceStability.Unstable annotations to some Kafka Streams public APIs Also improves Java docs for the Streams high-level DSL. Author: Guozhang Wang Reviewers: Ismael Juma, Michael G. Noll Closes #1097 from guozhangwang/KNewJavaDoc --- .../kafka/streams/kstream/Aggregator.java | 6 +- .../kafka/streams/kstream/Initializer.java | 4 +- .../kafka/streams/kstream/JoinWindows.java | 15 +- .../apache/kafka/streams/kstream/KStream.java | 270 ++++++++++-------- .../kafka/streams/kstream/KStreamBuilder.java | 44 ++- .../apache/kafka/streams/kstream/KTable.java | 165 ++++++----- .../kafka/streams/kstream/KeyValueMapper.java | 6 +- .../kafka/streams/kstream/Predicate.java | 4 +- .../apache/kafka/streams/kstream/Reducer.java | 2 +- .../kafka/streams/kstream/Transformer.java | 6 +- .../kafka/streams/kstream/ValueJoiner.java | 6 +- .../kafka/streams/kstream/ValueMapper.java | 4 +- .../streams/kstream/ValueTransformer.java | 4 +- .../kafka/streams/kstream/Windowed.java | 5 +- .../apache/kafka/streams/kstream/Windows.java | 33 +-- .../processor/DefaultPartitionGrouper.java | 8 +- .../streams/processor/ProcessorContext.java | 2 + .../kafka/streams/state/KeyValueStore.java | 2 + .../kafka/streams/state/WindowStore.java | 2 + 19 files changed, 303 insertions(+), 285 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java index 0d29409bc1164..9ec9f966442a9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java @@ -20,9 +20,9 @@ /** * The Aggregator interface for aggregating values of the given key. * - * @param Key type. - * @param Receiving value type. - * @param Aggregate value type. + * @param key type + * @param original value type + * @param aggregate value type */ public interface Aggregator { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Initializer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Initializer.java index fdd522030a7ad..67c1c21638408 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Initializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Initializer.java @@ -18,9 +18,9 @@ package org.apache.kafka.streams.kstream; /** - * The Initializer interface for creating an initial value for aggregations. + * The Initializer interface for creating an initial value in aggregations. * - * @param Aggregate value type. + * @param aggregate value type */ public interface Initializer { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java index 70294a80d3275..24dbdd33b2547 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java @@ -45,8 +45,7 @@ public static JoinWindows of(String name) { * Specifies that records of the same key are joinable if their timestamp stamps are within * timeDifference. * - * @param timeDifference - * @return + * @param timeDifference join window interval in milliseconds */ public JoinWindows within(long timeDifference) { return new JoinWindows(this.name, timeDifference, timeDifference); @@ -54,11 +53,10 @@ public JoinWindows within(long timeDifference) { /** * Specifies that records of the same key are joinable if their timestamp stamps are within - * timeDifference, and if the timestamp of a record from the secondary stream is - * is earlier than or equal to the timestamp of a record from the first stream. + * the join window interval, and if the timestamp of a record from the secondary stream is + * earlier than or equal to the timestamp of a record from the first stream. * - * @param timeDifference - * @return + * @param timeDifference join window interval in milliseconds */ public JoinWindows before(long timeDifference) { return new JoinWindows(this.name, timeDifference, this.after); @@ -66,11 +64,10 @@ public JoinWindows before(long timeDifference) { /** * Specifies that records of the same key are joinable if their timestamp stamps are within - * timeDifference, and if the timestamp of a record from the secondary stream is + * the join window interval, and if the timestamp of a record from the secondary stream * is later than or equal to the timestamp of a record from the first stream. * - * @param timeDifference - * @return + * @param timeDifference join window interval in milliseconds */ public JoinWindows after(long timeDifference) { return new JoinWindows(this.name, this.before, timeDifference); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index 6f05c3b7213d6..c4188de44c508 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -17,116 +17,110 @@ package org.apache.kafka.streams.kstream; +import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.ProcessorSupplier; /** - * KStream is an abstraction of an event stream in key-value pairs. + * KStream is an abstraction of a record stream of key-value pairs. * * @param Type of keys * @param Type of values */ +@InterfaceStability.Unstable public interface KStream { /** - * Creates a new instance of KStream consists of all elements of this stream which satisfy a predicate + * Create a new instance of {@link KStream} that consists of all elements of this stream which satisfy a predicate. * - * @param predicate the instance of Predicate - * @return the instance of KStream with only those elements that satisfy the predicate + * @param predicate the instance of {@link Predicate} */ KStream filter(Predicate predicate); /** - * Creates a new instance of KStream consists all elements of this stream which do not satisfy a predicate + * Create a new instance of {@link KStream} that consists all elements of this stream which do not satisfy a predicate. * - * @param predicate the instance of Predicate - * @return the instance of KStream with only those elements that do not satisfy the predicate + * @param predicate the instance of {@link Predicate} */ KStream filterOut(Predicate predicate); /** - * Creates a new instance of KStream by applying transforming each element in this stream into a different element in the new stream. + * Create a new instance of {@link KStream} by transforming each element in this stream into a different element in the new stream. * - * @param mapper the instance of KeyValueMapper - * @param the key type of the new stream - * @param the value type of the new stream - * @return the instance of KStream + * @param mapper the instance of {@link KeyValueMapper} + * @param the key type of the new stream + * @param the value type of the new stream */ KStream map(KeyValueMapper> mapper); /** - * Creates a new instance of KStream by transforming each value in this stream into a different value in the new stream. + * Create a new instance of {@link KStream} by transforming the value of each element in this stream into a new value in the new stream. * - * @param mapper the instance of ValueMapper - * @param the value type of the new stream - * @return the instance of KStream + * @param mapper the instance of {@link ValueMapper} + * @param the value type of the new stream */ KStream mapValues(ValueMapper mapper); /** - * Creates a new instance of KStream by transforming each element in this stream into zero or more elements in the new stream. + * Create a new instance of {@link KStream} by transforming each element in this stream into zero or more elements in the new stream. * - * @param mapper the instance of KeyValueMapper - * @param the key type of the new stream - * @param the value type of the new stream - * @return the instance of KStream + * @param mapper the instance of {@link KeyValueMapper} + * @param the key type of the new stream + * @param the value type of the new stream */ KStream flatMap(KeyValueMapper>> mapper); /** - * Creates a new stream by transforming each value in this stream into zero or more values in the new stream. + * Create a new instance of {@link KStream} by transforming the value of each element in this stream into zero or more values with the same key in the new stream. * - * @param processor the instance of Processor - * @param the value type of the new stream - * @return the instance of KStream + * @param processor the instance of {@link ValueMapper} + * @param the value type of the new stream */ KStream flatMapValues(ValueMapper> processor); /** - * Creates an array of streams from this stream. Each stream in the array corresponds to a predicate in - * supplied predicates in the same order. Predicates are evaluated in order. An element is streamed to - * a corresponding stream for the first predicate is evaluated true. - * An element will be dropped if none of the predicates evaluate true. + * Creates an array of {@link KStream} from this stream by branching the elements in the original stream based on the supplied predicates. + * Each element is evaluated against the supplied predicates, and predicates are evaluated in order. Each stream in the result array + * corresponds position-wise (index) to the predicate in the supplied predicates. The branching happens on first-match: An element + * in the original stream is assigned to the corresponding result stream for the first predicate that evaluates to true, and + * assigned to this stream only. An element will be dropped if none of the predicates evaluate to true. * - * @param predicates the ordered list of Predicate instances - * @return the instances of KStream that each contain those elements for which their Predicate evaluated to true. + * @param predicates the ordered list of {@link Predicate} instances */ KStream[] branch(Predicate... predicates); /** - * Sends key-value to a topic, also creates a new instance of KStream from the topic. - * This is equivalent to calling to(topic) and from(topic). + * Materialize this stream to a topic, also creates a new instance of {@link KStream} from the topic + * using default serializers and deserializers. + * This is equivalent to calling {@link #to(String)} and {@link org.apache.kafka.streams.kstream.KStreamBuilder#stream(String...)}. * * @param topic the topic name - * - * @return the instance of {@link KStream} that consumes the given topic */ KStream through(String topic); /** - * Sends key-value to a topic, also creates a new instance of KStream from the topic. - * This is equivalent to calling to(topic) and from(topic). + * Materialize this stream to a topic, also creates a new instance of {@link KStream} from the topic. + * This is equivalent to calling {@link #to(Serde, Serde, String)} and + * {@link org.apache.kafka.streams.kstream.KStreamBuilder#stream(Serde, Serde, String...)}. * * @param keySerde key serde used to send key-value pairs, * if not specified the default key serde defined in the configuration will be used * @param valSerde value serde used to send key-value pairs, * if not specified the default value serde defined in the configuration will be used * @param topic the topic name - * - * @return the instance of {@link KStream} that consumes the given topic */ KStream through(Serde keySerde, Serde valSerde, String topic); /** - * Sends key-value to a topic using default serializers specified in the config. + * Materialize this stream to a topic using default serializers specified in the config. * * @param topic the topic name */ void to(String topic); /** - * Sends key-value to a topic. + * Materialize this stream to a topic. * * @param keySerde key serde used to send key-value pairs, * if not specified the default serde defined in the configs will be used @@ -137,45 +131,43 @@ public interface KStream { void to(Serde keySerde, Serde valSerde, String topic); /** - * Applies a stateful transformation to all elements in this stream. + * Create a new {@link KStream} instance by applying a {@link org.apache.kafka.streams.kstream.Transformer} to all elements in this stream, one element at a time. * - * @param transformerSupplier the class of valueTransformerSupplier - * @param stateStoreNames the names of the state store used by the processor - * @return the instance of KStream that contains transformed keys and values + * @param transformerSupplier the instance of {@link TransformerSupplier} that generates {@link org.apache.kafka.streams.kstream.Transformer} + * @param stateStoreNames the names of the state store used by the processor */ KStream transform(TransformerSupplier> transformerSupplier, String... stateStoreNames); /** - * Applies a stateful transformation to all values in this stream. + * Create a new {@link KStream} instance by applying a {@link org.apache.kafka.streams.kstream.ValueTransformer} to all values in this stream, one element at a time. * - * @param valueTransformerSupplier the class of valueTransformerSupplier - * @param stateStoreNames the names of the state store used by the processor - * @return the instance of KStream that contains the keys and transformed values + * @param valueTransformerSupplier the instance of {@link ValueTransformerSupplier} that generates {@link org.apache.kafka.streams.kstream.ValueTransformer} + * @param stateStoreNames the names of the state store used by the processor */ KStream transformValues(ValueTransformerSupplier valueTransformerSupplier, String... stateStoreNames); /** - * Processes all elements in this stream by applying a processor. + * Process all elements in this stream, one element at a time, by applying a {@link org.apache.kafka.streams.processor.Processor}. * - * @param processorSupplier the supplier of the Processor to use - * @param stateStoreNames the names of the state store used by the processor + * @param processorSupplier the supplier of {@link ProcessorSupplier} that generates {@link org.apache.kafka.streams.processor.Processor} + * @param stateStoreNames the names of the state store used by the processor */ void process(ProcessorSupplier processorSupplier, String... stateStoreNames); /** - * Combines values of this stream with another KStream using Windowed Inner Join. + * Combine element values of this stream with another {@link KStream}'s elements of the same key using windowed Inner Join. * - * @param otherStream the instance of KStream joined with this stream - * @param joiner ValueJoiner - * @param windows the specification of the join window - * @param keySerde key serdes, - * if not specified the default serdes defined in the configs will be used - * @param thisValueSerde value serdes for this stream, - * if not specified the default serdes defined in the configs will be used - * @param otherValueSerde value serdes for other stream, - * if not specified the default serdes defined in the configs will be used - * @param the value type of the other stream - * @param the value type of the new stream + * @param otherStream the instance of {@link KStream} joined with this stream + * @param joiner the instance of {@link ValueJoiner} + * @param windows the specification of the {@link JoinWindows} + * @param keySerde key serdes for materializing both streams, + * if not specified the default serdes defined in the configs will be used + * @param thisValueSerde value serdes for materializing this stream, + * if not specified the default serdes defined in the configs will be used + * @param otherValueSerde value serdes for materializing the other stream, + * if not specified the default serdes defined in the configs will be used + * @param the value type of the other stream + * @param the value type of the new stream */ KStream join( KStream otherStream, @@ -186,7 +178,8 @@ KStream join( Serde otherValueSerde); /** - * Combines values of this stream with another KStream using Windowed Inner Join. + * Combine element values of this stream with another {@link KStream}'s elements of the same key using windowed Inner Join + * with default serializers and deserializers. * * @param otherStream the instance of {@link KStream} joined with this stream * @param joiner the instance of {@link ValueJoiner} @@ -200,19 +193,19 @@ KStream join( JoinWindows windows); /** - * Combines values of this stream with another KStream using Windowed Outer Join. + * Combine values of this stream with another {@link KStream}'s elements of the same key using windowed Outer Join. * - * @param otherStream the instance of KStream joined with this stream - * @param joiner ValueJoiner - * @param windows the specification of the join window - * @param keySerde key serdes, - * if not specified the default serdes defined in the configs will be used - * @param thisValueSerde value serdes for this stream, - * if not specified the default serdes defined in the configs will be used - * @param otherValueSerde value serdes for other stream, - * if not specified the default serdes defined in the configs will be used - * @param the value type of the other stream - * @param the value type of the new stream + * @param otherStream the instance of {@link KStream} joined with this stream + * @param joiner the instance of {@link ValueJoiner} + * @param windows the specification of the {@link JoinWindows} + * @param keySerde key serdes for materializing both streams, + * if not specified the default serdes defined in the configs will be used + * @param thisValueSerde value serdes for materializing this stream, + * if not specified the default serdes defined in the configs will be used + * @param otherValueSerde value serdes for materializing the other stream, + * if not specified the default serdes defined in the configs will be used + * @param the value type of the other stream + * @param the value type of the new stream */ KStream outerJoin( KStream otherStream, @@ -223,7 +216,8 @@ KStream outerJoin( Serde otherValueSerde); /** - * Combines values of this stream with another KStream using Windowed Outer Join. + * Combine values of this stream with another {@link KStream}'s elements of the same key using windowed Outer Join + * with default serializers and deserializers. * * @param otherStream the instance of {@link KStream} joined with this stream * @param joiner the instance of {@link ValueJoiner} @@ -237,17 +231,17 @@ KStream outerJoin( JoinWindows windows); /** - * Combines values of this stream with another KStream using Windowed Left Join. + * Combine values of this stream with another {@link KStream}'s elements of the same key using windowed Left Join. * - * @param otherStream the instance of KStream joined with this stream - * @param joiner ValueJoiner - * @param windows the specification of the join window - * @param keySerde key serdes, - * if not specified the default serdes defined in the configs will be used - * @param otherValueSerde value serdes for other stream, - * if not specified the default serdes defined in the configs will be used - * @param the value type of the other stream - * @param the value type of the new stream + * @param otherStream the instance of {@link KStream} joined with this stream + * @param joiner the instance of {@link ValueJoiner} + * @param windows the specification of the {@link JoinWindows} + * @param keySerde key serdes for materializing the other stream, + * if not specified the default serdes defined in the configs will be used + * @param otherValueSerde value serdes for materializing the other stream, + * if not specified the default serdes defined in the configs will be used + * @param the value type of the other stream + * @param the value type of the new stream */ KStream leftJoin( KStream otherStream, @@ -257,7 +251,8 @@ KStream leftJoin( Serde otherValueSerde); /** - * Combines values of this stream with another KStream using Windowed Left Join. + * Combine values of this stream with another {@link KStream}'s elements of the same key using windowed Left Join + * with default serializers and deserializers. * * @param otherStream the instance of {@link KStream} joined with this stream * @param joiner the instance of {@link ValueJoiner} @@ -271,7 +266,7 @@ KStream leftJoin( JoinWindows windows); /** - * Combines values of this stream with KTable using Left Join. + * Combine values of this stream with {@link KTable}'s elements of the same key using non-windowed Left Join. * * @param table the instance of {@link KTable} joined with this stream * @param joiner the instance of {@link ValueJoiner} @@ -281,48 +276,63 @@ KStream leftJoin( KStream leftJoin(KTable table, ValueJoiner joiner); /** - * Aggregate values of this stream by key on a window basis. + * Combine values of this stream by key on a window basis into a new instance of windowed {@link KTable}. * - * @param reducer the class of {@link Reducer} - * @param windows the specification of the aggregation {@link Windows} + * @param reducer the instance of {@link Reducer} + * @param windows the specification of the aggregation {@link Windows} + * @param keySerde key serdes for materializing the aggregated table, + * if not specified the default serdes defined in the configs will be used + * @param valueSerde value serdes for materializing the aggregated table, + * if not specified the default serdes defined in the configs will be used */ KTable, V> reduceByKey(Reducer reducer, Windows windows, Serde keySerde, - Serde aggValueSerde); + Serde valueSerde); /** - * Aggregate values of this stream by key on a window basis. + * Combine values of this stream by key on a window basis into a new instance of windowed {@link KTable} + * with default serializers and deserializers. * - * @param reducer the class of {@link Reducer} + * @param reducer the instance of {@link Reducer} * @param windows the specification of the aggregation {@link Windows} */ KTable, V> reduceByKey(Reducer reducer, Windows windows); /** - * Aggregate values of this stream by key on a window basis. + * Combine values of this stream by key into a new instance of ever-updating {@link KTable}. * - * @param reducer the class of Reducer + * @param reducer the instance of {@link Reducer} + * @param keySerde key serdes for materializing the aggregated table, + * if not specified the default serdes defined in the configs will be used + * @param valueSerde value serdes for materializing the aggregated table, + * if not specified the default serdes defined in the configs will be used + * @param name the name of the resulted {@link KTable} */ KTable reduceByKey(Reducer reducer, Serde keySerde, - Serde aggValueSerde, + Serde valueSerde, String name); /** - * Aggregate values of this stream by key on a window basis. + * Combine values of this stream by key into a new instance of ever-updating {@link KTable} with default serializers and deserializers. * - * @param reducer the class of {@link Reducer} + * @param reducer the instance of {@link Reducer} + * @param name the name of the resulted {@link KTable} */ KTable reduceByKey(Reducer reducer, String name); /** - * Aggregate values of this stream by key on a window basis. + * Aggregate values of this stream by key on a window basis into a new instance of windowed {@link KTable}. * - * @param initializer the class of Initializer - * @param aggregator the class of Aggregator - * @param windows the specification of the aggregation {@link Windows} - * @param the value type of the aggregated table + * @param initializer the instance of {@link Initializer} + * @param aggregator the instance of {@link Aggregator} + * @param windows the specification of the aggregation {@link Windows} + * @param keySerde key serdes for materializing the aggregated table, + * if not specified the default serdes defined in the configs will be used + * @param aggValueSerde aggregate value serdes for materializing the aggregated table, + * if not specified the default serdes defined in the configs will be used + * @param the value type of the resulted {@link KTable} */ KTable, T> aggregateByKey(Initializer initializer, Aggregator aggregator, @@ -331,25 +341,29 @@ KTable, T> aggregateByKey(Initializer initi Serde aggValueSerde); /** - * Aggregate values of this stream by key on a window basis. + * Aggregate values of this stream by key on a window basis into a new instance of windowed {@link KTable} + * with default serializers and deserializers. * - * @param initializer the class of {@link Initializer} - * @param aggregator the class of {@link Aggregator} + * @param initializer the instance of {@link Initializer} + * @param aggregator the instance of {@link Aggregator} * @param windows the specification of the aggregation {@link Windows} - * @param the value type of the aggregated table + * @param the value type of the resulted {@link KTable} */ KTable, T> aggregateByKey(Initializer initializer, Aggregator aggregator, Windows windows); /** - * Aggregate values of this stream by key without a window basis, and hence - * return an ever updating table + * Aggregate values of this stream by key into a new instance of ever-updating {@link KTable}. * * @param initializer the class of {@link Initializer} * @param aggregator the class of {@link Aggregator} - * @param name the name of the aggregated table - * @param the value type of the aggregated table + * @param keySerde key serdes for materializing the aggregated table, + * if not specified the default serdes defined in the configs will be used + * @param aggValueSerde aggregate value serdes for materializing the aggregated table, + * if not specified the default serdes defined in the configs will be used + * @param name the name of the resulted {@link KTable} + * @param the value type of the resulted {@link KTable} */ KTable aggregateByKey(Initializer initializer, Aggregator aggregator, @@ -358,45 +372,49 @@ KTable aggregateByKey(Initializer initializer, String name); /** - * Aggregate values of this stream by key without a window basis, and hence - * return an ever updating table + * Aggregate values of this stream by key into a new instance of ever-updating {@link KTable} + * with default serializers and deserializers. * * @param initializer the class of {@link Initializer} * @param aggregator the class of {@link Aggregator} - * @param name the name of the aggregated table - * @param the value type of the aggregated table + * @param name the name of the resulted {@link KTable} + * @param the value type of the resulted {@link KTable} */ KTable aggregateByKey(Initializer initializer, Aggregator aggregator, String name); /** - * Count number of messages of this stream by key on a window basis. + * Count number of messages of this stream by key on a window basis into a new instance of windowed {@link KTable}. * * @param windows the specification of the aggregation {@link Windows} + * @param keySerde key serdes for materializing the counting table, + * if not specified the default serdes defined in the configs will be used */ KTable, Long> countByKey(Windows windows, Serde keySerde); /** - * Count number of messages of this stream by key on a window basis. + * Count number of messages of this stream by key on a window basis into a new instance of windowed {@link KTable} + * with default serializers and deserializers. * * @param windows the specification of the aggregation {@link Windows} */ KTable, Long> countByKey(Windows windows); /** - * Count number of messages of this stream by key without a window basis, and hence - * return a ever updating counting table. + * Count number of messages of this stream by key into a new instance of ever-updating {@link KTable}. * - * @param name the name of the aggregated table + * @param keySerde key serdes for materializing the counting table, + * if not specified the default serdes defined in the configs will be used + * @param name the name of the resulted {@link KTable} */ KTable countByKey(Serde keySerde, String name); /** - * Count number of messages of this stream by key without a window basis, and hence - * return a ever updating counting table. + * Count number of messages of this stream by key into a new instance of ever-updating {@link KTable} + * with default serializers and deserializers. * - * @param name the name of the aggregated table + * @param name the name of the resulted {@link KTable} */ KTable countByKey(String name); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java index dfd9281e711ee..6b770b4006e4f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java @@ -28,8 +28,8 @@ import java.util.concurrent.atomic.AtomicInteger; /** - * KStreamBuilder is a subclass of {@link TopologyBuilder} that provides the {@link KStream} DSL - * for users to specify computational logic and translates the given logic to a processor topology. + * KStreamBuilder is a subclass of {@link TopologyBuilder} that provides the Kafka Streams DSL + * for users to specify computational logic and translates the given logic to a {@link org.apache.kafka.streams.processor.internals.ProcessorTopology}. */ public class KStreamBuilder extends TopologyBuilder { @@ -40,25 +40,23 @@ public KStreamBuilder() { } /** - * Creates a KStream instance for the specified topic. + * Creates a {@link KStream} instance from the specified topics. * The default deserializers specified in the config are used. * - * @param topics the topic names, if empty default to all the topics in the config - * @return KStream + * @param topics the topic names; must contain at least one topic name */ public KStream stream(String... topics) { return stream(null, null, topics); } /** - * Creates a KStream instance for the specified topic. + * Creates a {@link KStream} instance for the specified topics. * - * @param keySerde key serde used to read this source KStream, - * if not specified the default serde defined in the configs will be used - * @param valSerde value serde used to read this source KStream, - * if not specified the default serde defined in the configs will be used - * @param topics the topic names, if empty default to all the topics in the config - * @return KStream + * @param keySerde key serde used to read this source {@link KStream}, + * if not specified the default serde defined in the configs will be used + * @param valSerde value serde used to read this source {@link KStream}, + * if not specified the default serde defined in the configs will be used + * @param topics the topic names; must contain at least one topic name */ public KStream stream(Serde keySerde, Serde valSerde, String... topics) { String name = newName(KStreamImpl.SOURCE_NAME); @@ -69,25 +67,23 @@ public KStream stream(Serde keySerde, Serde valSerde, String. } /** - * Creates a KTable instance for the specified topic. + * Creates a {@link KTable} instance for the specified topic. * The default deserializers specified in the config are used. * - * @param topic the topic name - * @return KTable + * @param topic the topic name; cannot be null */ public KTable table(String topic) { return table(null, null, topic); } /** - * Creates a KTable instance for the specified topic. + * Creates a {@link KTable} instance for the specified topic. * * @param keySerde key serde used to send key-value pairs, - * if not specified the default key serde defined in the configuration will be used + * if not specified the default key serde defined in the configuration will be used * @param valSerde value serde used to send key-value pairs, - * if not specified the default value serde defined in the configuration will be used - * @param topic the topic name - * @return KStream + * if not specified the default value serde defined in the configuration will be used + * @param topic the topic name; cannot be null */ public KTable table(Serde keySerde, Serde valSerde, String topic) { String source = newName(KStreamImpl.SOURCE_NAME); @@ -102,10 +98,9 @@ public KTable table(Serde keySerde, Serde valSerde, String to } /** - * Creates a new stream by merging the given streams + * Creates a new instance of {@link KStream} by merging the given streams * - * @param streams the streams to be merged - * @return KStream + * @param streams the instances of {@link KStream} to be merged */ public KStream merge(KStream... streams) { return KStreamImpl.merge(this, streams); @@ -115,8 +110,7 @@ public KStream merge(KStream... streams) { * Create a unique processor name used for translation into the processor topology. * This function is only for internal usage. * - * @param prefix Processor name prefix. - * @return The unique processor name. + * @param prefix processor name prefix */ public String newName(String prefix) { return prefix + String.format("%010d", index.getAndIncrement()); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index 997cb4dc9f105..9a2a8a8b7e8a7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -17,73 +17,72 @@ package org.apache.kafka.streams.kstream; +import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.KeyValue; /** - * KTable is an abstraction of a change log stream from a primary-keyed table. + * KTable is an abstraction of a changelog stream from a primary-keyed table. * * @param Type of primary keys * @param Type of value changes */ +@InterfaceStability.Unstable public interface KTable { /** - * Creates a new instance of KTable consists of all elements of this stream which satisfy a predicate + * Create a new instance of {@link KTable} that consists of all elements of this stream which satisfy a predicate. * - * @param predicate the instance of Predicate - * @return the instance of KTable with only those elements that satisfy the predicate + * @param predicate the instance of {@link Predicate} */ KTable filter(Predicate predicate); /** - * Creates a new instance of KTable consists all elements of this stream which do not satisfy a predicate + * Create a new instance of {@link KTable} that consists all elements of this stream which do not satisfy a predicate * - * @param predicate the instance of Predicate - * @return the instance of KTable with only those elements that do not satisfy the predicate + * @param predicate the instance of {@link Predicate} */ KTable filterOut(Predicate predicate); /** - * Creates a new instance of KTable by transforming each value in this stream into a different value in the new stream. + * Create a new instance of {@link KTable} by transforming the value of each element in this stream into a new value in the new stream. * - * @param mapper the instance of ValueMapper - * @param the value type of the new stream - * @return the instance of KTable + * @param mapper the instance of {@link ValueMapper} + * @param the value type of the new stream */ KTable mapValues(ValueMapper mapper); /** - * Sends key-value to a topic, also creates a new instance of KTable from the topic. - * This is equivalent to calling to(topic) and table(topic). + * Materialize this stream to a topic, also creates a new instance of {@link KTable} from the topic + * using default serializers and deserializers. + * This is equivalent to calling {@link #to(String)} and {@link org.apache.kafka.streams.kstream.KStreamBuilder#table(String)}. * - * @param topic the topic name - * @return the instance of KTable that consumes the given topic + * @param topic the topic name */ KTable through(String topic); /** - * Sends key-value to a topic, also creates a new instance of KTable from the topic. - * This is equivalent to calling to(topic) and table(topic). + * Materialize this stream to a topic, also creates a new instance of {@link KTable} from the topic. + * This is equivalent to calling {@link #to(Serde, Serde, String)} and + * {@link org.apache.kafka.streams.kstream.KStreamBuilder#table(Serde, Serde, String)}. * * @param keySerde key serde used to send key-value pairs, * if not specified the default key serde defined in the configuration will be used * @param valSerde value serde used to send key-value pairs, * if not specified the default value serde defined in the configuration will be used * @param topic the topic name - * @return the new stream that consumes the given topic */ KTable through(Serde keySerde, Serde valSerde, String topic); /** - * Sends key-value to a topic using default serializers specified in the config. + * Materialize this stream to a topic using default serializers specified in the config. * * @param topic the topic name */ void to(String topic); /** - * Sends key-value to a topic. + * Materialize this stream to a topic. * * @param keySerde key serde used to send key-value pairs, * if not specified the default serde defined in the configs will be used @@ -94,55 +93,53 @@ public interface KTable { void to(Serde keySerde, Serde valSerde, String topic); /** - * Creates a new instance of KStream from this KTable - * - * @return the instance of KStream + * Convert this stream to a new instance of {@link KStream}. */ KStream toStream(); /** - * Combines values of this KTable with another KTable using Inner Join. + * Combine values of this stream with another {@link KTable} stream's elements of the same key using Inner Join. * - * @param other the instance of KTable joined with this stream - * @param joiner ValueJoiner - * @param the value type of the other stream - * @param the value type of the new stream - * @return the instance of KTable + * @param other the instance of {@link KTable} joined with this stream + * @param joiner the instance of {@link ValueJoiner} + * @param the value type of the other stream + * @param the value type of the new stream */ KTable join(KTable other, ValueJoiner joiner); /** - * Combines values of this KTable with another KTable using Outer Join. + * Combine values of this stream with another {@link KTable} stream's elements of the same key using Outer Join. * - * @param other the instance of KTable joined with this stream - * @param joiner ValueJoiner - * @param the value type of the other stream - * @param the value type of the new stream - * @return the instance of KTable + * @param other the instance of {@link KTable} joined with this stream + * @param joiner the instance of {@link ValueJoiner} + * @param the value type of the other stream + * @param the value type of the new stream */ KTable outerJoin(KTable other, ValueJoiner joiner); /** - * Combines values of this KTable with another KTable using Left Join. + * Combine values of this stream with another {@link KTable} stream's elements of the same key using Left Join. * - * @param other the instance of KTable joined with this stream - * @param joiner ValueJoiner - * @param the value type of the other stream - * @param the value type of the new stream - * @return the instance of KTable + * @param other the instance of {@link KTable} joined with this stream + * @param joiner the instance of {@link ValueJoiner} + * @param the value type of the other stream + * @param the value type of the new stream */ KTable leftJoin(KTable other, ValueJoiner joiner); /** - * Reduce values of this table by the selected key. + * Combine updating values of this stream by the selected key into a new instance of {@link KTable}. * - * @param adder the class of Reducer - * @param subtractor the class of Reducer - * @param selector the KeyValue mapper that select the aggregate key - * @param name the name of the resulted table - * @param the key type of the aggregated table - * @param the value type of the aggregated table - * @return the instance of KTable + * @param adder the instance of {@link Reducer} for addition + * @param subtractor the instance of {@link Reducer} for subtraction + * @param selector the instance of {@link KeyValueMapper} that select the aggregate key + * @param keySerde key serdes for materializing the aggregated table, + * if not specified the default serdes defined in the configs will be used + * @param valueSerde value serdes for materializing the aggregated table, + * if not specified the default serdes defined in the configs will be used + * @param name the name of the resulted {@link KTable} + * @param the key type of the aggregated {@link KTable} + * @param the value type of the aggregated {@link KTable} */ KTable reduce(Reducer adder, Reducer subtractor, @@ -152,15 +149,15 @@ KTable reduce(Reducer adder, String name); /** - * Reduce values of this table by the selected key. + * Combine updating values of this stream by the selected key into a new instance of {@link KTable} + * using default serializers and deserializers. * * @param adder the instance of {@link Reducer} for addition * @param subtractor the instance of {@link Reducer} for subtraction * @param selector the instance of {@link KeyValueMapper} that select the aggregate key - * @param name the name of the resulted table - * @param the key type of the aggregated table - * @param the value type of the aggregated table - * @return the instance of KTable + * @param name the name of the resulted {@link KTable} + * @param the key type of the aggregated {@link KTable} + * @param the value type of the aggregated {@link KTable} */ KTable reduce(Reducer adder, Reducer subtractor, @@ -168,20 +165,26 @@ KTable reduce(Reducer adder, String name); /** - * Aggregate values of this table by the selected key. + * Aggregate updating values of this stream by the selected key into a new instance of {@link KTable}. * - * @param initializer the class of Initializer - * @param add the class of Aggregator - * @param remove the class of Aggregator - * @param selector the KeyValue mapper that select the aggregate key - * @param name the name of the resulted table - * @param the key type of the aggregated table - * @param the value type of the aggregated table - * @return the instance of KTable + * @param initializer the instance of {@link Initializer} + * @param adder the instance of {@link Aggregator} for addition + * @param substractor the instance of {@link Aggregator} for subtraction + * @param selector the instance of {@link KeyValueMapper} that select the aggregate key + * @param keySerde key serdes for materializing this stream and the aggregated table, + * if not specified the default serdes defined in the configs will be used + * @param valueSerde value serdes for materializing this stream, + * if not specified the default serdes defined in the configs will be used + * @param aggValueSerde value serdes for materializing the aggregated table, + * if not specified the default serdes defined in the configs will be used + * @param name the name of the resulted table + * @param the key type of this {@link KTable} + * @param the value type of this {@link KTable} + * @param the value type of the aggregated {@link KTable} */ KTable aggregate(Initializer initializer, - Aggregator add, - Aggregator remove, + Aggregator adder, + Aggregator substractor, KeyValueMapper> selector, Serde keySerde, Serde valueSerde, @@ -189,16 +192,17 @@ KTable aggregate(Initializer initializer, String name); /** - * Aggregate values of this table by the selected key. + * Aggregate updating values of this stream by the selected key into a new instance of {@link KTable} + * using default serializers and deserializers. * * @param initializer the instance of {@link Initializer} * @param adder the instance of {@link Aggregator} for addition * @param substractor the instance of {@link Aggregator} for subtraction * @param selector the instance of {@link KeyValueMapper} that select the aggregate key - * @param name the name of the resulted table - * @param the key type of the aggregated table - * @param the value type of the aggregated table - * @return the instance of aggregated {@link KTable} + * @param name the name of the resulted {@link KTable} + * @param the key type of the aggregated {@link KTable} + * @param the value type of the aggregated {@link KTable} + * @param the value type of the aggregated {@link KTable} */ KTable aggregate(Initializer initializer, Aggregator adder, @@ -207,12 +211,15 @@ KTable aggregate(Initializer initializer, String name); /** - * Count number of records of this table by the selected key. + * Count number of records of this stream by the selected key into a new instance of {@link KTable}. * - * @param selector the KeyValue mapper that select the aggregate key - * @param name the name of the resulted table - * @param the key type of the aggregated table - * @return the instance of KTable + * @param selector the instance of {@link KeyValueMapper} that select the aggregate key + * @param keySerde key serdes for materializing this stream, + * if not specified the default serdes defined in the configs will be used + * @param valueSerde value serdes for materializing this stream, + * if not specified the default serdes defined in the configs will be used + * @param name the name of the resulted table + * @param the key type of the aggregated {@link KTable} */ KTable count(KeyValueMapper selector, Serde keySerde, @@ -220,12 +227,12 @@ KTable count(KeyValueMapper selector, String name); /** - * Count number of records of this table by the selected key. + * Count number of records of this stream by the selected key into a new instance of {@link KTable} + * using default serializers and deserializers. * * @param selector the instance of {@link KeyValueMapper} that select the aggregate key - * @param name the name of the resulted table - * @param the key type of the aggregated table - * @return the instance of aggregated {@link KTable} + * @param name the name of the resulted {@link KTable} + * @param the key type of the aggregated {@link KTable} */ KTable count(KeyValueMapper selector, String name); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java index 9c04ef5aa7c8c..a4aed91f92c4c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java @@ -20,9 +20,9 @@ /** * The KeyValueMapper interface for mapping a key-value pair to a new value (could be another key-value pair). * - * @param Original key type. - * @param Original value type. - * @param Mapped value type. + * @param original key type + * @param original value type + * @param mapped value type */ public interface KeyValueMapper { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java index 784f5b1a0a085..c90554b192cc6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java @@ -20,8 +20,8 @@ /** * The Predicate interface represents a predicate (boolean-valued function) of a key-value pair. * - * @param Key type. - * @param Value type. + * @param key type + * @param value type */ public interface Predicate { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Reducer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Reducer.java index bf25f7360d11c..551a6720e66c4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Reducer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Reducer.java @@ -20,7 +20,7 @@ /** * The Reducer interface for combining two values of the same type into a new value. * - * @param Value type. + * @param value type */ public interface Reducer { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java index 47198e4991d99..8069dca60602e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java @@ -22,9 +22,9 @@ /** * A stateful Transformer interface for transform a key-value pair into a new value. * - * @param Key type. - * @param Value type. - * @param Return type. + * @param key type + * @param value type + * @param return type */ public interface Transformer { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java index 41005b3997a5c..5f00a1a7ff203 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java @@ -20,9 +20,9 @@ /** * The ValueJoiner interface for joining two values and return a the joined new value. * - * @param First value type. - * @param Second value type. - * @param Joined value type. + * @param first value type + * @param second value type + * @param joined value type */ public interface ValueJoiner { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java index d507c870c1ecd..6e62a55774f40 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java @@ -20,8 +20,8 @@ /** * The KeyValueMapper interface for mapping an original value to a new value (could be another key-value pair). * - * @param Original value type. - * @param Mapped value type. + * @param original value type + * @param mapped value type */ public interface ValueMapper { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java index b4d2b38213cb8..1a0679d681595 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java @@ -22,8 +22,8 @@ /** * A stateful Value Transformer interface for transform a value into a new value. * - * @param Value type. - * @param Return type. + * @param value type + * @param return type */ public interface ValueTransformer { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java index eed5fe1f40222..369128277733b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java @@ -19,9 +19,8 @@ /** * The windowed key interface used in {@link KTable}, used for representing a windowed table result from windowed stream aggregations, - * i.e. {@link KStream#aggregateByKey(Initializer, Aggregator, Windows, org.apache.kafka.common.serialization.Serializer, - * org.apache.kafka.common.serialization.Serializer, org.apache.kafka.common.serialization.Deserializer, - * org.apache.kafka.common.serialization.Deserializer)} + * i.e. {@link KStream#aggregateByKey(Initializer, Aggregator, Windows, org.apache.kafka.common.serialization.Serde, + * org.apache.kafka.common.serialization.Serde)} * * @param Type of the key */ diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java index 678e3516ad365..e7dc23ec9a739 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java @@ -24,7 +24,7 @@ /** * The window specification interface that can be extended for windowing operation in joins and aggregations. * - * @param Type of the window instance + * @param type of the window instance */ public abstract class Windows { @@ -38,17 +38,17 @@ public abstract class Windows { protected String name; - private long emitDuration; + private long emitDurationMs; - private long maintainDuration; + private long maintainDurationMs; public int segments; protected Windows(String name) { this.name = name; this.segments = DEFAULT_NUM_SEGMENTS; - this.emitDuration = DEFAULT_EMIT_DURATION; - this.maintainDuration = DEFAULT_MAINTAIN_DURATION; + this.emitDurationMs = DEFAULT_EMIT_DURATION; + this.maintainDurationMs = DEFAULT_MAINTAIN_DURATION; } public String name() { @@ -56,29 +56,26 @@ public String name() { } /** - * Set the window emit duration in milliseconds of system time + * Set the window emit duration in milliseconds of system time. */ - public Windows emit(long duration) { - this.emitDuration = duration; + public Windows emit(long durationMs) { + this.emitDurationMs = durationMs; return this; } /** - * Set the window maintain duration in milliseconds of system time + * Set the window maintain duration in milliseconds of system time. */ - public Windows until(long duration) { - this.maintainDuration = duration; + public Windows until(long durationMs) { + this.maintainDurationMs = durationMs; return this; } /** - * Specifies the number of segments to be used for rolling the window store, - * this function is not exposed to users but can be called by developers that extend this JoinWindows specs - * - * @param segments - * @return + * Specify the number of segments to be used for rolling the window store, + * this function is not exposed to users but can be called by developers that extend this JoinWindows specs. */ protected Windows segments(int segments) { this.segments = segments; @@ -87,11 +84,11 @@ protected Windows segments(int segments) { } public long emitEveryMs() { - return this.emitDuration; + return this.emitDurationMs; } public long maintainMs() { - return this.maintainDuration; + return this.maintainDurationMs; } protected String newName(String prefix) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java b/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java index 06681ac51e2ee..999f6a94f3bf5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java @@ -39,11 +39,11 @@ public class DefaultPartitionGrouper implements PartitionGrouper { /** - * Generate tasks with the assigned topic partitions + * Generate tasks with the assigned topic partitions. * - * @param topicGroups {@link TopologyBuilder#topicGroups()} where topics of the same group need to be joined together - * @param metadata Metadata of the consuming cluster - * @return The map from generated task ids to the assigned partitions. + * @param topicGroups group of topics that need to be joined together + * @param metadata metadata of the consuming cluster + * @return The map from generated task ids to the assigned partitions */ public Map> partitionGroups(Map> topicGroups, Cluster metadata) { Map> groups = new HashMap<>(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java index fdcff19e01bc2..434996e2fd563 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.processor; +import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.StreamsMetrics; @@ -25,6 +26,7 @@ /** * Processor context interface. */ +@InterfaceStability.Unstable public interface ProcessorContext { /** diff --git a/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java index 3e7f6fbc8c245..908e116e9fb20 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/KeyValueStore.java @@ -19,6 +19,7 @@ package org.apache.kafka.streams.state; +import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.StateStore; @@ -30,6 +31,7 @@ * @param The key type * @param The value type */ +@InterfaceStability.Unstable public interface KeyValueStore extends StateStore { /** diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java index cbd373cf8bc3b..c7a882fc3ac0e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java @@ -19,6 +19,7 @@ package org.apache.kafka.streams.state; +import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.streams.processor.StateStore; /** @@ -27,6 +28,7 @@ * @param Type of keys * @param Type of values */ +@InterfaceStability.Unstable public interface WindowStore extends StateStore { /** From 69df734470f1db9ba1732cb4bedaa1541d721103 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Mon, 21 Mar 2016 13:15:53 -0700 Subject: [PATCH 066/206] MINOR: fix documentation version This will need to be double-committed. Author: Gwen Shapira Reviewers: Ismael Juma , Ewen Cheslack-Postava Closes #1107 from gwenshap/fix-doc-version --- docs/documentation.html | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/documentation.html b/docs/documentation.html index 4ce759988bc16..67a2954acbf11 100644 --- a/docs/documentation.html +++ b/docs/documentation.html @@ -17,8 +17,8 @@ -

      Kafka 0.9.0 Documentation

      -Prior releases: 0.7.x, 0.8.0, 0.8.1.X, 0.8.2.X. +

      Kafka 0.10.0 Documentation

      +Prior releases: 0.7.x, 0.8.0, 0.8.1.X, 0.8.2.X, 0.9.0.X.
        From 4f0417931a71a974241e685031f2a5f1680e9b51 Mon Sep 17 00:00:00 2001 From: Gwen Shapira Date: Mon, 21 Mar 2016 13:40:59 -0700 Subject: [PATCH 067/206] MINOR: update new version in additional places Note: This goes only to trunk. 0.10.0 branch will need a separate PR with different versions. Author: Gwen Shapira Reviewers: Ismael Juma , Ewen Cheslack-Postava Closes #1109 from gwenshap/minor-fix-version-trunk --- kafka-merge-pr.py | 2 +- tests/kafkatest/__init__.py | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/kafka-merge-pr.py b/kafka-merge-pr.py index e12410511b65b..f26a0a9c8197e 100644 --- a/kafka-merge-pr.py +++ b/kafka-merge-pr.py @@ -72,7 +72,7 @@ DEV_BRANCH_NAME = "trunk" -DEFAULT_FIX_VERSION = os.environ.get("DEFAULT_FIX_VERSION", "0.10.0.0") +DEFAULT_FIX_VERSION = os.environ.get("DEFAULT_FIX_VERSION", "0.10.1.0") def get_json(url): try: diff --git a/tests/kafkatest/__init__.py b/tests/kafkatest/__init__.py index df1a6129dbc8e..10163a02b5a83 100644 --- a/tests/kafkatest/__init__.py +++ b/tests/kafkatest/__init__.py @@ -23,4 +23,4 @@ # Instead, in trunk, the version should have a suffix of the form ".devN" # # For example, when Kafka is at version 0.9.0.0-SNAPSHOT, this should be something like "0.9.0.0.dev0" -__version__ = '0.10.0.0.dev0' +__version__ = '0.10.1.0.dev0' From 8d8e3aaa6172d314230a8d61e6892e9c09dc45b6 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Mon, 21 Mar 2016 20:47:25 -0700 Subject: [PATCH 068/206] KAFKA-3412: multiple asynchronous commits causes send failures Author: Jason Gustafson Reviewers: Ismael Juma , Ewen Cheslack-Postava Closes #1108 from hachikuji/KAFKA-3412 --- .../kafka/clients/consumer/KafkaConsumer.java | 2 +- .../internals/ConsumerCoordinator.java | 4 ++++ .../internals/ConsumerNetworkClient.java | 5 ++-- .../internals/ConsumerCoordinatorTest.java | 8 ------- .../kafka/api/BaseConsumerTest.scala | 24 ++++++++++++------- .../kafka/api/PlaintextConsumerTest.scala | 15 ++++++++++++ 6 files changed, 39 insertions(+), 19 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index b7eafbe346f45..c36b7f1aa5de0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -870,7 +870,7 @@ public ConsumerRecords poll(long timeout) { // must return these records to users to process before being interrupted or // auto-committing offsets fetcher.sendFetches(metadata.fetch()); - client.quickPoll(); + client.quickPoll(false); return this.interceptors == null ? new ConsumerRecords<>(records) : this.interceptors.onConsume(new ConsumerRecords<>(records)); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index cf935309cd383..e582ce3fc71b7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -345,6 +345,10 @@ public void onFailure(RuntimeException e) { cb.onComplete(offsets, e); } }); + + // ensure commit has a chance to be transmitted (without blocking on its completion) + // note that we allow delayed tasks to be executed in case heartbeats need to be sent + client.quickPoll(true); } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java index 4492306fd3a8e..b70994d87fc44 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java @@ -196,10 +196,11 @@ public void poll(long timeout) { /** * Poll for network IO and return immediately. This will not trigger wakeups, * nor will it execute any delayed tasks. + * @param executeDelayedTasks Whether to allow delayed task execution (true allows) */ - public void quickPoll() { + public void quickPoll(boolean executeDelayedTasks) { disableWakeups(); - poll(0, time.milliseconds(), false); + poll(0, time.milliseconds(), executeDelayedTasks); enableWakeups(); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index 260ee7a081ed1..8844adc80b964 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -627,7 +627,6 @@ public void testCommitOffsetOnly() { AtomicBoolean success = new AtomicBoolean(false); coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), callback(success)); - consumerClient.poll(0); assertTrue(success.get()); assertEquals(100L, subscriptions.committed(tp).offset()); @@ -644,7 +643,6 @@ public void testCommitOffsetMetadata() { AtomicBoolean success = new AtomicBoolean(false); coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L, "hello")), callback(success)); - consumerClient.poll(0); assertTrue(success.get()); assertEquals(100L, subscriptions.committed(tp).offset()); @@ -658,7 +656,6 @@ public void testCommitOffsetAsyncWithDefaultCallback() { coordinator.ensureCoordinatorKnown(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), null); - consumerClient.poll(0); assertEquals(invokedBeforeTest + 1, defaultOffsetCommitCallback.invoked); assertNull(defaultOffsetCommitCallback.exception); } @@ -693,7 +690,6 @@ public boolean matches(ClientRequest request) { AtomicBoolean success = new AtomicBoolean(false); coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), callback(success)); - consumerClient.poll(0); assertTrue(success.get()); } @@ -704,7 +700,6 @@ public void testCommitOffsetAsyncFailedWithDefaultCallback() { coordinator.ensureCoordinatorKnown(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code()))); coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), null); - consumerClient.poll(0); assertEquals(invokedBeforeTest + 1, defaultOffsetCommitCallback.invoked); assertEquals(Errors.GROUP_COORDINATOR_NOT_AVAILABLE.exception(), defaultOffsetCommitCallback.exception); } @@ -718,7 +713,6 @@ public void testCommitOffsetAsyncCoordinatorNotAvailable() { MockCommitCallback cb = new MockCommitCallback(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.GROUP_COORDINATOR_NOT_AVAILABLE.code()))); coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), cb); - consumerClient.poll(0); assertTrue(coordinator.coordinatorUnknown()); assertEquals(1, cb.invoked); @@ -734,7 +728,6 @@ public void testCommitOffsetAsyncNotCoordinator() { MockCommitCallback cb = new MockCommitCallback(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NOT_COORDINATOR_FOR_GROUP.code()))); coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), cb); - consumerClient.poll(0); assertTrue(coordinator.coordinatorUnknown()); assertEquals(1, cb.invoked); @@ -750,7 +743,6 @@ public void testCommitOffsetAsyncDisconnected() { MockCommitCallback cb = new MockCommitCallback(); client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code())), true); coordinator.commitOffsetsAsync(Collections.singletonMap(tp, new OffsetAndMetadata(100L)), cb); - consumerClient.poll(0); assertTrue(coordinator.coordinatorUnknown()); assertEquals(1, cb.invoked); diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index 9939309dc41b5..1408cd96a1657 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -81,7 +81,7 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { // shouldn't make progress until poll is invoked Thread.sleep(10) - assertEquals(0, commitCallback.count) + assertEquals(0, commitCallback.successCount) awaitCommitCallback(this.consumers(0), commitCallback) } @@ -330,18 +330,26 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { records } - protected def awaitCommitCallback[K, V](consumer: Consumer[K, V], commitCallback: CountConsumerCommitCallback): Unit = { - val startCount = commitCallback.count + protected def awaitCommitCallback[K, V](consumer: Consumer[K, V], + commitCallback: CountConsumerCommitCallback, + count: Int = 1): Unit = { + val startCount = commitCallback.successCount val started = System.currentTimeMillis() - while (commitCallback.count == startCount && System.currentTimeMillis() - started < 10000) + while (commitCallback.successCount < startCount + count && System.currentTimeMillis() - started < 10000) consumer.poll(50) - assertEquals(startCount + 1, commitCallback.count) + assertEquals(startCount + count, commitCallback.successCount) } protected class CountConsumerCommitCallback extends OffsetCommitCallback { - var count = 0 - - override def onComplete(offsets: util.Map[TopicPartition, OffsetAndMetadata], exception: Exception): Unit = count += 1 + var successCount = 0 + var failCount = 0 + + override def onComplete(offsets: util.Map[TopicPartition, OffsetAndMetadata], exception: Exception): Unit = { + if (exception == null) + successCount += 1 + else + failCount += 1 + } } protected class ConsumerAssignmentPoller(consumer: Consumer[Array[Byte], Array[Byte]], diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 9c560107e4111..ff2e63d366aa9 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -232,6 +232,21 @@ class PlaintextConsumerTest extends BaseConsumerTest { assertEquals(nullMetadata, this.consumers(0).committed(tp)) } + @Test + def testAsyncCommit() { + val consumer = this.consumers(0) + consumer.assign(List(tp).asJava) + consumer.poll(0) + + val callback = new CountConsumerCommitCallback + val count = 5 + for (i <- 1 to count) + consumer.commitAsync(Map(tp -> new OffsetAndMetadata(i)).asJava, callback) + + awaitCommitCallback(consumer, callback, count=count) + assertEquals(new OffsetAndMetadata(count), consumer.committed(tp)) + } + @Test def testExpandingTopicSubscriptions() { val otherTopic = "other" From 99cfb99aa161007f2ceae33945d11b968d932175 Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Tue, 22 Mar 2016 11:53:12 -0700 Subject: [PATCH 069/206] KAFKA-3435: Remove `Unstable` annotation from new Java Consumer Author: Grant Henke Reviewers: Gwen Shapira Closes #1113 from granthenke/remove-unstable --- .../org/apache/kafka/clients/consumer/Consumer.java | 5 ++--- .../apache/kafka/clients/consumer/KafkaConsumer.java | 11 +++++------ 2 files changed, 7 insertions(+), 9 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java index 0862c326c1ade..ef80606232b1d 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -3,9 +3,9 @@ * 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. @@ -29,7 +29,6 @@ * @see KafkaConsumer * @see MockConsumer */ -@InterfaceStability.Unstable public interface Consumer extends Closeable { /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index c36b7f1aa5de0..103125298b8a7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -165,11 +165,11 @@ * In this example the client is subscribing to the topics foo and bar as part of a group of consumers * called test as described above. *

        - * The broker will automatically detect failed processes in the test group by using a heartbeat mechanism. The - * consumer will automatically ping the cluster periodically, which lets the cluster know that it is alive. Note that - * the consumer is single-threaded, so periodic heartbeats can only be sent when {@link #poll(long)} is called. As long as - * the consumer is able to do this it is considered alive and retains the right to consume from the partitions assigned - * to it. If it stops heartbeating by failing to call {@link #poll(long)} for a period of time longer than session.timeout.ms + * The broker will automatically detect failed processes in the test group by using a heartbeat mechanism. The + * consumer will automatically ping the cluster periodically, which lets the cluster know that it is alive. Note that + * the consumer is single-threaded, so periodic heartbeats can only be sent when {@link #poll(long)} is called. As long as + * the consumer is able to do this it is considered alive and retains the right to consume from the partitions assigned + * to it. If it stops heartbeating by failing to call {@link #poll(long)} for a period of time longer than session.timeout.ms * then it will be considered dead and its partitions will be assigned to another process. *

        * The deserializer settings specify how to turn bytes into objects. For example, by specifying string deserializers, we @@ -452,7 +452,6 @@ * commit. * */ -@InterfaceStability.Unstable public class KafkaConsumer implements Consumer { private static final Logger log = LoggerFactory.getLogger(KafkaConsumer.class); From 73470b028cd659c1c405e6b6ba52bf483113b90b Mon Sep 17 00:00:00 2001 From: gaob13 Date: Tue, 22 Mar 2016 11:58:01 -0700 Subject: [PATCH 070/206] MINOR: Remove the very misleading comment lines It is not true in practice. Maybe the implied feature is not yet implemented or removed. These lines can be super misleading. Please merge. Thank you. Author: gaob13 Reviewers: Ismael Juma, Ewen Cheslack-Postava Closes #793 from gaob13/trunk --- config/server.properties | 30 +++++++++---------- .../main/scala/kafka/server/KafkaConfig.scala | 20 +++++++++---- 2 files changed, 28 insertions(+), 22 deletions(-) diff --git a/config/server.properties b/config/server.properties index ddb695aff7a1d..aebcb87ca9e46 100644 --- a/config/server.properties +++ b/config/server.properties @@ -21,22 +21,18 @@ broker.id=0 ############################# Socket Server Settings ############################# -listeners=PLAINTEXT://:9092 - -# The port the socket server listens on -#port=9092 - -# Hostname the broker will bind to. If not set, the server will bind to all interfaces -#host.name=localhost - -# Hostname the broker will advertise to producers and consumers. If not set, it uses the -# value for "host.name" if configured. Otherwise, it will use the value returned from -# java.net.InetAddress.getCanonicalHostName(). -#advertised.host.name= - -# The port to publish to ZooKeeper for clients to use. If this is not set, -# it will publish the same port that the broker binds to. -#advertised.port= +# The address the socket server listens on. It will get the value returned from +# java.net.InetAddress.getCanonicalHostName() if not configured. +# FORMAT: +# listeners = security_protocol://host_name:port +# EXAMPLE: +# listeners = PLAINTEXT://your.host.name:9092 +#listeners=PLAINTEXT://:9092 + +# Hostname and port the broker will advertise to producers and consumers. If not set, +# it uses the value for "listeners" if configured. Otherwise, it will use the value +# returned from java.net.InetAddress.getCanonicalHostName(). +#advertised.listeners=PLAINTEXT://your.host.name:9092 # The number of threads handling network requests num.network.threads=3 @@ -117,3 +113,5 @@ zookeeper.connect=localhost:2181 # Timeout in ms for connecting to zookeeper zookeeper.connection.timeout.ms=6000 + + diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 7e1013e22a5c6..a6018ad45b8d0 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -366,24 +366,32 @@ object KafkaConfig { /************* Authorizer Configuration ***********/ val AuthorizerClassNameDoc = "The authorizer class that should be used for authorization" /** ********* Socket Server Configuration ***********/ - val PortDoc = "the port to listen and accept connections on" - val HostNameDoc = "hostname of broker. If this is set, it will only bind to this address. If this is not set, it will bind to all interfaces" + val PortDoc = "DEPRECATED: only used when `listeners` is not set. " + + "Use `listeners` instead. \n" + + "the port to listen and accept connections on" + val HostNameDoc = "DEPRECATED: only used when `listeners` is not set. " + + "Use `listeners` instead. \n" + + "hostname of broker. If this is set, it will only bind to this address. If this is not set, it will bind to all interfaces" val ListenersDoc = "Listener List - Comma-separated list of URIs we will listen on and their protocols.\n" + " Specify hostname as 0.0.0.0 to bind to all interfaces.\n" + " Leave hostname empty to bind to default interface.\n" + " Examples of legal listener lists:\n" + " PLAINTEXT://myhost:9092,TRACE://:9091\n" + " PLAINTEXT://0.0.0.0:9092, TRACE://localhost:9093\n" - val AdvertisedHostNameDoc = "Hostname to publish to ZooKeeper for clients to use. In IaaS environments, this may " + + val AdvertisedHostNameDoc = "DEPRECATED: only used when `advertised.listeners` or `listeners` are not set. " + + "Use `advertised.listeners` instead. \n" + + "Hostname to publish to ZooKeeper for clients to use. In IaaS environments, this may " + "need to be different from the interface to which the broker binds. If this is not set, " + - "it will use the value for \"host.name\" if configured. Otherwise " + + "it will use the value for `host.name` if configured. Otherwise " + "it will use the value returned from java.net.InetAddress.getCanonicalHostName()." - val AdvertisedPortDoc = "The port to publish to ZooKeeper for clients to use. In IaaS environments, this may " + + val AdvertisedPortDoc = "DEPRECATED: only used when `advertised.listeners` or `listeners` are not set. " + + "Use `advertised.listeners` instead. \n" + + "The port to publish to ZooKeeper for clients to use. In IaaS environments, this may " + "need to be different from the port to which the broker binds. If this is not set, " + "it will publish the same port that the broker binds to." val AdvertisedListenersDoc = "Listeners to publish to ZooKeeper for clients to use, if different than the listeners above." + " In IaaS environments, this may need to be different from the interface to which the broker binds." + - " If this is not set, the value for \"listeners\" will be used." + " If this is not set, the value for `listeners` will be used." val SocketSendBufferBytesDoc = "The SO_SNDBUF buffer of the socket sever sockets" val SocketReceiveBufferBytesDoc = "The SO_RCVBUF buffer of the socket sever sockets" val SocketRequestMaxBytesDoc = "The maximum number of bytes in a socket request" From 73c79000edddd929cd0af25f4a29fcc682a1c9c0 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 22 Mar 2016 12:39:04 -0700 Subject: [PATCH 071/206] KAFKA-3426; Improve protocol type errors when invalid sizes are received Author: Ismael Juma Reviewers: Grant Henke, Gwen Shapira Closes #1100 from ijuma/kafka-3426-invalid-protocol-type-errors-invalid-sizes --- .../kafka/common/protocol/types/ArrayOf.java | 2 + .../kafka/common/protocol/types/Type.java | 22 ++++- .../types/ProtocolSerializationTest.java | 93 ++++++++++++++++++- 3 files changed, 112 insertions(+), 5 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java index 4a36cb71d2c79..a08f876abdb36 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java @@ -41,6 +41,8 @@ public void write(ByteBuffer buffer, Object o) { @Override public Object read(ByteBuffer buffer) { int size = buffer.getInt(); + if (size < 0) + throw new SchemaException("Array size " + size + " cannot be negative"); if (size > buffer.remaining()) throw new SchemaException("Error reading array of size " + size + ", only " + buffer.remaining() + " bytes available"); Object[] objs = new Object[size]; diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java index c4bcb1e492018..92c1f7c67b17e 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java @@ -184,14 +184,19 @@ public Long validate(Object item) { 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."); + throw new SchemaException("String length " + bytes.length + " is larger than the maximum string length."); buffer.putShort((short) bytes.length); buffer.put(bytes); } @Override public Object read(ByteBuffer buffer) { - int length = buffer.getShort(); + short length = buffer.getShort(); + if (length < 0) + throw new SchemaException("String length " + length + " cannot be negative"); + if (length > buffer.remaining()) + throw new SchemaException("Error reading string of length " + length + ", only " + buffer.remaining() + " bytes available"); + byte[] bytes = new byte[length]; buffer.get(bytes); return Utils.utf8(bytes); @@ -231,16 +236,18 @@ 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."); + throw new SchemaException("String length " + bytes.length + " is larger than the maximum string length."); buffer.putShort((short) bytes.length); buffer.put(bytes); } @Override public Object read(ByteBuffer buffer) { - int length = buffer.getShort(); + short length = buffer.getShort(); if (length < 0) return null; + if (length > buffer.remaining()) + throw new SchemaException("Error reading string of length " + length + ", only " + buffer.remaining() + " bytes available"); byte[] bytes = new byte[length]; buffer.get(bytes); @@ -285,6 +292,11 @@ public void write(ByteBuffer buffer, Object o) { @Override public Object read(ByteBuffer buffer) { int size = buffer.getInt(); + if (size < 0) + throw new SchemaException("Bytes size " + size + " cannot be negative"); + if (size > buffer.remaining()) + throw new SchemaException("Error reading bytes of size " + size + ", only " + buffer.remaining() + " bytes available"); + ByteBuffer val = buffer.slice(); val.limit(size); buffer.position(buffer.position() + size); @@ -336,6 +348,8 @@ public Object read(ByteBuffer buffer) { int size = buffer.getInt(); if (size < 0) return null; + if (size > buffer.remaining()) + throw new SchemaException("Error reading bytes of size " + size + ", only " + buffer.remaining() + " bytes available"); ByteBuffer val = buffer.slice(); val.limit(size); diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java index e20aa1093258d..5c342776b48f6 100644 --- a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java @@ -117,7 +117,7 @@ private void checkNullableDefault(Type type, Object defaultValue) { } @Test - public void testArray() { + public void testReadArraySizeTooLarge() { Type type = new ArrayOf(Type.INT8); int size = 10; ByteBuffer invalidBuffer = ByteBuffer.allocate(4 + size); @@ -133,6 +133,97 @@ public void testArray() { } } + @Test + public void testReadNegativeArraySize() { + Type type = new ArrayOf(Type.INT8); + int size = 10; + ByteBuffer invalidBuffer = ByteBuffer.allocate(4 + size); + invalidBuffer.putInt(-1); + for (int i = 0; i < size; i++) + invalidBuffer.put((byte) i); + invalidBuffer.rewind(); + try { + type.read(invalidBuffer); + fail("Array size not validated"); + } catch (SchemaException e) { + // Expected exception + } + } + + @Test + public void testReadStringSizeTooLarge() { + byte[] stringBytes = "foo".getBytes(); + ByteBuffer invalidBuffer = ByteBuffer.allocate(2 + stringBytes.length); + invalidBuffer.putShort((short) (stringBytes.length * 5)); + invalidBuffer.put(stringBytes); + invalidBuffer.rewind(); + try { + Type.STRING.read(invalidBuffer); + fail("String size not validated"); + } catch (SchemaException e) { + // Expected exception + } + invalidBuffer.rewind(); + try { + Type.NULLABLE_STRING.read(invalidBuffer); + fail("String size not validated"); + } catch (SchemaException e) { + // Expected exception + } + } + + @Test + public void testReadNegativeStringSize() { + byte[] stringBytes = "foo".getBytes(); + ByteBuffer invalidBuffer = ByteBuffer.allocate(2 + stringBytes.length); + invalidBuffer.putShort((short) -1); + invalidBuffer.put(stringBytes); + invalidBuffer.rewind(); + try { + Type.STRING.read(invalidBuffer); + fail("String size not validated"); + } catch (SchemaException e) { + // Expected exception + } + } + + @Test + public void testReadBytesSizeTooLarge() { + byte[] stringBytes = "foo".getBytes(); + ByteBuffer invalidBuffer = ByteBuffer.allocate(4 + stringBytes.length); + invalidBuffer.putInt(stringBytes.length * 5); + invalidBuffer.put(stringBytes); + invalidBuffer.rewind(); + try { + Type.BYTES.read(invalidBuffer); + fail("Bytes size not validated"); + } catch (SchemaException e) { + // Expected exception + } + invalidBuffer.rewind(); + try { + Type.NULLABLE_BYTES.read(invalidBuffer); + fail("Bytes size not validated"); + } catch (SchemaException e) { + // Expected exception + } + } + + @Test + public void testReadNegativeBytesSize() { + byte[] stringBytes = "foo".getBytes(); + ByteBuffer invalidBuffer = ByteBuffer.allocate(4 + stringBytes.length); + invalidBuffer.putInt(-20); + invalidBuffer.put(stringBytes); + invalidBuffer.rewind(); + try { + Type.BYTES.read(invalidBuffer); + fail("Bytes size not validated"); + } catch (SchemaException e) { + // Expected exception + } + } + private Object roundtrip(Type type, Object obj) { ByteBuffer buffer = ByteBuffer.allocate(type.sizeOf(obj)); type.write(buffer, obj); From ca77d67058726fc9df9bdd7cc0217ee62ccc5106 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Tue, 22 Mar 2016 13:09:13 -0700 Subject: [PATCH 072/206] KAFKA-3319: improve session timeout broker/client config documentation Author: Jason Gustafson Reviewers: Grant Henke, Ismael Juma, Guozhang Wang Closes #1106 from hachikuji/KAFKA-3319 --- .../kafka/clients/consumer/ConsumerConfig.java | 16 ++++++++++++---- .../org/apache/kafka/common/protocol/Errors.java | 3 ++- .../main/scala/kafka/server/KafkaConfig.scala | 4 ++-- 3 files changed, 16 insertions(+), 7 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index 9101307cc0c5a..c97c8fb470242 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -43,11 +43,22 @@ public class ConsumerConfig extends AbstractConfig { public static final String GROUP_ID_CONFIG = "group.id"; private static final String GROUP_ID_DOC = "A unique string that identifies the consumer group this consumer belongs to. This property is required if the consumer uses either the group management functionality by using subscribe(topic) or the Kafka-based offset management strategy."; + /** max.poll.records */ + public static final String MAX_POLL_RECORDS_CONFIG = "max.poll.records"; + private static final String MAX_POLL_RECORDS_DOC = "The maximum number of records returned in a single call to poll()."; + /** * session.timeout.ms */ public static final String SESSION_TIMEOUT_MS_CONFIG = "session.timeout.ms"; - private static final String SESSION_TIMEOUT_MS_DOC = "The timeout used to detect failures when using Kafka's group management facilities."; + private static final String SESSION_TIMEOUT_MS_DOC = "The timeout used to detect failures when using Kafka's " + + "group management facilities. When a consumer's heartbeat is not received within the session timeout, " + + "the broker will mark the consumer as failed and rebalance the group. Since heartbeats are sent only " + + "when poll() is invoked, a higher session timeout allows more time for message processing in the consumer's " + + "poll loop at the cost of a longer time to detect hard failures. See also " + MAX_POLL_RECORDS_CONFIG + " for " + + "another option to control the processing time in the poll loop. Note that the value must be in the " + + "allowable range as configured in the broker configuration by group.min.session.timeout.ms " + + "and group.max.session.timeout.ms."; /** * heartbeat.interval.ms @@ -168,9 +179,6 @@ public class ConsumerConfig extends AbstractConfig { + "Implementing the ConsumerInterceptor interface allows you to intercept (and possibly mutate) records " + "received by the consumer. By default, there are no interceptors."; - /** max.poll.records */ - public static final String MAX_POLL_RECORDS_CONFIG = "max.poll.records"; - private static final String MAX_POLL_RECORDS_DOC = "The maximum number of records returned in a single call to poll()."; /** exclude.internal.topics */ public static final String EXCLUDE_INTERNAL_TOPICS_CONFIG = "exclude.internal.topics"; diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index 90be0144109af..0f33516b934c2 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -117,7 +117,8 @@ public enum Errors { UNKNOWN_MEMBER_ID(25, new UnknownMemberIdException("The coordinator is not aware of this member.")), INVALID_SESSION_TIMEOUT(26, - new InvalidSessionTimeoutException("The session timeout is not within an acceptable range.")), + new InvalidSessionTimeoutException("The session timeout is not within the range allowed by the broker " + + "(as configured by group.min.session.timeout.ms and group.max.session.timeout.ms).")), REBALANCE_IN_PROGRESS(27, new RebalanceInProgressException("The group is rebalancing, so a rejoin is needed.")), INVALID_COMMIT_OFFSET_SIZE(28, diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index a6018ad45b8d0..dc2a0a0814bf5 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -480,8 +480,8 @@ object KafkaConfig { val ControlledShutdownRetryBackoffMsDoc = "Before each retry, the system needs time to recover from the state that caused the previous failure (Controller fail over, replica lag etc). This config determines the amount of time to wait before retrying." val ControlledShutdownEnableDoc = "Enable controlled shutdown of the server" /** ********* Consumer coordinator configuration ***********/ - val ConsumerMinSessionTimeoutMsDoc = "The minimum allowed session timeout for registered consumers" - val ConsumerMaxSessionTimeoutMsDoc = "The maximum allowed session timeout for registered consumers" + val ConsumerMinSessionTimeoutMsDoc = "The minimum allowed session timeout for registered consumers. Shorter timeouts leader to quicker failure detection at the cost of more frequent consumer heartbeating, which can overwhelm broker resources." + val ConsumerMaxSessionTimeoutMsDoc = "The maximum allowed session timeout for registered consumers. Longer timeouts give consumers more time to process messages in between heartbeats at the cost of a longer time to detect failures." /** ********* Offset management configuration ***********/ val OffsetMetadataMaxSizeDoc = "The maximum size for a metadata entry associated with an offset commit" val OffsetsLoadBufferSizeDoc = "Batch size for reading from the offsets segments when loading offsets into the cache." From ad3dfc6ab25c3f80d2425e24e72ae732b850dc60 Mon Sep 17 00:00:00 2001 From: Vahid Hashemian Date: Tue, 22 Mar 2016 13:10:07 -0700 Subject: [PATCH 073/206] KAFKA-3219: Fix long topic name validation This fixes an issue with long topic names by considering, during topic validation, the '-' and the partition id that is appended to the log folder created for each topic partition. Author: Vahid Hashemian Reviewers: Gwen Shapira, Grant Henke Closes #898 from vahidhashemian/KAFKA-3219 --- core/src/main/scala/kafka/common/Topic.scala | 2 +- core/src/test/scala/unit/kafka/common/TopicTest.scala | 3 ++- docs/ops.html | 2 ++ 3 files changed, 5 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/common/Topic.scala b/core/src/main/scala/kafka/common/Topic.scala index 930d0e441568c..6067712762afe 100644 --- a/core/src/main/scala/kafka/common/Topic.scala +++ b/core/src/main/scala/kafka/common/Topic.scala @@ -22,7 +22,7 @@ import kafka.coordinator.GroupCoordinator object Topic { val legalChars = "[a-zA-Z0-9\\._\\-]" - private val maxNameLength = 255 + private val maxNameLength = 249 private val rgx = new Regex(legalChars + "+") def validate(topic: String) { diff --git a/core/src/test/scala/unit/kafka/common/TopicTest.scala b/core/src/test/scala/unit/kafka/common/TopicTest.scala index da6083a2dd9ff..66549afe4b3d8 100644 --- a/core/src/test/scala/unit/kafka/common/TopicTest.scala +++ b/core/src/test/scala/unit/kafka/common/TopicTest.scala @@ -31,6 +31,7 @@ class TopicTest { for (i <- 1 to 6) longName += longName invalidTopicNames += longName + invalidTopicNames += longName.drop(6) val badChars = Array('/', '\\', ',', '\u0000', ':', "\"", '\'', ';', '*', '?', ' ', '\t', '\r', '\n', '=') for (weirdChar <- badChars) { invalidTopicNames += "Is" + weirdChar + "illegal" @@ -47,7 +48,7 @@ class TopicTest { } val validTopicNames = new ArrayBuffer[String]() - validTopicNames += ("valid", "TOPIC", "nAmEs", "ar6", "VaL1d", "_0-9_.") + validTopicNames += ("valid", "TOPIC", "nAmEs", "ar6", "VaL1d", "_0-9_.", longName.drop(7)) for (i <- 0 until validTopicNames.size) { try { Topic.validate(validTopicNames(i)) diff --git a/docs/ops.html b/docs/ops.html index 4cfe17b1899ac..541a01ddcaf86 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -34,6 +34,8 @@

        Adding and removing

        The partition count controls how many logs the topic will be sharded into. There are several impacts of the partition count. First each partition must fit entirely on a single server. So if you have 20 partitions the full data set (and read and write load) will be handled by no more than 20 servers (no counting replicas). Finally the partition count impacts the maximum parallelism of your consumers. This is discussed in greater detail in the concepts section.

        +Each sharded partition log is placed into its own folder under the Kafka log directory. The name of such folders consists of the topic name, appended by a dash (-) and the partition id. Since a typical folder name can not be over 255 characters long, there will be a limitation on the length of topic names. We assume the number of partitions will not ever be above 100,000. Therefore, topic names cannot be longer than 249 characters. This leaves just enough room in the folder name for a dash and a potentially 5 digit long partition id. +

        The configurations added on the command line override the default settings the server has for things like the length of time data should be retained. The complete set of per-topic configurations is documented here.

        Modifying topics

        From d6b450ced57bb5ab8a7164358e6a360f97a7b5cc Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Tue, 22 Mar 2016 14:09:10 -0700 Subject: [PATCH 074/206] KAFKA-3435: Follow up to fix checkstyle Author: Ewen Cheslack-Postava Reviewers: Gwen Shapira Closes #1116 from ewencp/kafka-3435-follow-up --- .../main/java/org/apache/kafka/clients/consumer/Consumer.java | 1 - .../java/org/apache/kafka/clients/consumer/KafkaConsumer.java | 1 - 2 files changed, 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java index ef80606232b1d..6f5a6b64c9bd0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/Consumer.java @@ -16,7 +16,6 @@ import org.apache.kafka.common.MetricName; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.annotation.InterfaceStability; import java.io.Closeable; import java.util.Collection; diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 103125298b8a7..b15d07f80969e 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -28,7 +28,6 @@ import org.apache.kafka.common.MetricName; import org.apache.kafka.common.PartitionInfo; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.config.ConfigException; import org.apache.kafka.common.metrics.JmxReporter; import org.apache.kafka.common.metrics.MetricConfig; From 983e9486ef139f8c9797679732d09feb3f7bfb87 Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Tue, 22 Mar 2016 15:01:05 -0700 Subject: [PATCH 075/206] =?UTF-8?q?KAFKA-3301;=20CommonClientConfigs.METRI?= =?UTF-8?q?CS=5FSAMPLE=5FWINDOW=5FMS=5FDOC=20is=20incor=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …rect Author: Grant Henke Reviewers: Jun Rao Closes #1114 from granthenke/window-docs --- .../main/java/org/apache/kafka/clients/CommonClientConfigs.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java index 2c5e67c5d78b2..298e1d8c541be 100644 --- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -53,7 +53,7 @@ public class CommonClientConfigs { public static final String RETRY_BACKOFF_MS_DOC = "The amount of time to wait before attempting to retry a failed fetch request to a given topic partition. This avoids repeated fetching-and-failing in a tight loop."; public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = "metrics.sample.window.ms"; - public static final String METRICS_SAMPLE_WINDOW_MS_DOC = "The number of samples maintained to compute metrics."; + public static final String METRICS_SAMPLE_WINDOW_MS_DOC = "The window of time a metrics sample is computed over."; public static final String METRICS_NUM_SAMPLES_CONFIG = "metrics.num.samples"; public static final String METRICS_NUM_SAMPLES_DOC = "The number of samples maintained to compute metrics."; From adca4d7df3155ac29ae510c237adf06c9521d221 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 22 Mar 2016 18:20:07 -0700 Subject: [PATCH 076/206] KAFKA-3447; partitionState in UpdateMetadataRequest not logged properly state-change log Author: Ismael Juma Reviewers: Jun Rao Closes #1117 from ijuma/kafka-3447-metadata-cache-logging --- .../scala/kafka/server/MetadataCache.scala | 26 +++++++++---------- 1 file changed, 12 insertions(+), 14 deletions(-) diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 4b68f707320ed..06fae42116e54 100755 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -44,7 +44,7 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { private val aliveNodes = mutable.Map[Int, collection.Map[SecurityProtocol, Node]]() private val partitionMetadataLock = new ReentrantReadWriteLock() - this.logIdent = "[Kafka Metadata Cache on broker %d] ".format(brokerId) + this.logIdent = s"[Kafka Metadata Cache on broker $brokerId] " private def getAliveEndpoints(brokers: Iterable[Int], protocol: SecurityProtocol): Seq[Node] = { val result = new mutable.ArrayBuffer[Node](math.min(aliveBrokers.size, brokers.size)) @@ -73,7 +73,7 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { maybeLeader match { case None => - debug("Error while fetching metadata for %s: leader not available".format(topicPartition)) + debug(s"Error while fetching metadata for $topicPartition: leader not available") new MetadataResponse.PartitionMetadata(Errors.LEADER_NOT_AVAILABLE, partitionId, Node.noNode(), replicaInfo.asJava, java.util.Collections.emptyList()) @@ -82,14 +82,14 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { val isrInfo = getAliveEndpoints(isr, protocol) if (replicaInfo.size < replicas.size) { - debug("Error while fetching metadata for %s: replica information not available for following brokers %s" - .format(topicPartition, replicas.filterNot(replicaInfo.map(_.id).contains).mkString(","))) + debug(s"Error while fetching metadata for $topicPartition: replica information not available for " + + s"following brokers ${replicas.filterNot(replicaInfo.map(_.id).contains).mkString(",")}") new MetadataResponse.PartitionMetadata(Errors.REPLICA_NOT_AVAILABLE, partitionId, leader, replicaInfo.asJava, isrInfo.asJava) } else if (isrInfo.size < isr.size) { - debug("Error while fetching metadata for %s: in sync replica information not available for following brokers %s" - .format(topicPartition, isr.filterNot(isrInfo.map(_.id).contains).mkString(","))) + debug(s"Error while fetching metadata for $topicPartition: in sync replica information not available for " + + s"following brokers ${isr.filterNot(isrInfo.map(_.id).contains).mkString(",")}") new MetadataResponse.PartitionMetadata(Errors.REPLICA_NOT_AVAILABLE, partitionId, leader, replicaInfo.asJava, isrInfo.asJava) } else { @@ -167,19 +167,17 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { } updateMetadataRequest.partitionStates.asScala.foreach { case (tp, info) => + val controllerId = updateMetadataRequest.controllerId + val controllerEpoch = updateMetadataRequest.controllerEpoch if (info.leader == LeaderAndIsr.LeaderDuringDelete) { removePartitionInfo(tp.topic, tp.partition) - stateChangeLogger.trace(("Broker %d deleted partition %s from metadata cache in response to UpdateMetadata request " + - "sent by controller %d epoch %d with correlation id %d") - .format(brokerId, tp, updateMetadataRequest.controllerId, - updateMetadataRequest.controllerEpoch, correlationId)) + stateChangeLogger.trace(s"Broker $brokerId deleted partition $tp from metadata cache in response to UpdateMetadata " + + s"request sent by controller $controllerId epoch $controllerEpoch with correlation id $correlationId") } else { val partitionInfo = partitionStateToPartitionStateInfo(info) addOrUpdatePartitionInfo(tp.topic, tp.partition, partitionInfo) - stateChangeLogger.trace(("Broker %d cached leader info %s for partition %s in response to UpdateMetadata request " + - "sent by controller %d epoch %d with correlation id %d") - .format(brokerId, info, tp, updateMetadataRequest.controllerId, - updateMetadataRequest.controllerEpoch, correlationId)) + stateChangeLogger.trace(s"Broker $brokerId cached leader info $partitionInfo for partition $tp in response to " + + s"UpdateMetadata request sent by controller $controllerId epoch $controllerEpoch with correlation id $correlationId") } } } From 255b5e13863a95cfc327236856db2df188f04d49 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 22 Mar 2016 19:13:26 -0700 Subject: [PATCH 077/206] KAFKA-3431: Remove `o.a.k.common.BrokerEndPoint` in favour of `Node` Also included a minor efficiency improvement in kafka.cluster.EndPoint. Author: Ismael Juma Reviewers: Gwen Shapira Closes #1105 from ijuma/kafka-3431-replace-broker-end-point-with-node --- .../apache/kafka/common/BrokerEndPoint.java | 88 ------------------- .../common/requests/LeaderAndIsrRequest.java | 14 +-- .../requests/UpdateMetadataRequest.java | 8 +- .../common/requests/RequestResponseTest.java | 13 ++- .../main/scala/kafka/cluster/EndPoint.scala | 3 +- .../controller/ControllerChannelManager.scala | 12 +-- .../kafka/api/AuthorizerIntegrationTest.scala | 8 +- .../kafka/server/LeaderElectionTest.scala | 17 ++-- .../kafka/server/ReplicaManagerTest.scala | 6 +- 9 files changed, 38 insertions(+), 131 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/common/BrokerEndPoint.java diff --git a/clients/src/main/java/org/apache/kafka/common/BrokerEndPoint.java b/clients/src/main/java/org/apache/kafka/common/BrokerEndPoint.java deleted file mode 100644 index d5275c419a669..0000000000000 --- a/clients/src/main/java/org/apache/kafka/common/BrokerEndPoint.java +++ /dev/null @@ -1,88 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ -package org.apache.kafka.common; - -import java.io.Serializable; - -/** - * Broker id, host and port - */ -public final class BrokerEndPoint implements Serializable { - - private int hash = 0; - private final int id; - private final String host; - private final int port; - - public BrokerEndPoint(int id, String host, int port) { - this.id = id; - this.host = host; - this.port = port; - } - - public int id() { - return id; - } - - public String host() { - return host; - } - - public int port() { - return port; - } - - @Override - public int hashCode() { - if (hash != 0) - return hash; - final int prime = 31; - int result = 1; - result = prime * result + id; - result = prime * result + ((host == null) ? 0 : host.hashCode()); - result = prime * result + port; - 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; - BrokerEndPoint other = (BrokerEndPoint) obj; - if (id != other.id) - return false; - if (port != other.port) - return false; - if (host == null) { - if (other.host != null) - return false; - } else if (!host.equals(other.host)) - return false; - return true; - } - - @Override - public String toString() { - return "[" + id + ", " + host + ":" + port + "]"; - } - -} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java index 264af903b7ead..fee3c2111410d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/LeaderAndIsrRequest.java @@ -17,7 +17,7 @@ package org.apache.kafka.common.requests; -import org.apache.kafka.common.BrokerEndPoint; +import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; @@ -78,10 +78,10 @@ public PartitionState(int controllerEpoch, int leader, int leaderEpoch, List partitionStates; - private final Set liveLeaders; + private final Set liveLeaders; public LeaderAndIsrRequest(int controllerId, int controllerEpoch, Map partitionStates, - Set liveLeaders) { + Set liveLeaders) { super(new Struct(CURRENT_SCHEMA)); struct.set(CONTROLLER_ID_KEY_NAME, controllerId); struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch); @@ -104,7 +104,7 @@ public LeaderAndIsrRequest(int controllerId, int controllerEpoch, Map leadersData = new ArrayList<>(liveLeaders.size()); - for (BrokerEndPoint leader : liveLeaders) { + for (Node leader : liveLeaders) { Struct leaderData = struct.instance(LIVE_LEADERS_KEY_NAME); leaderData.set(END_POINT_ID_KEY_NAME, leader.id()); leaderData.set(HOST_KEY_NAME, leader.host()); @@ -148,13 +148,13 @@ public LeaderAndIsrRequest(Struct struct) { } - Set leaders = new HashSet<>(); + Set leaders = new HashSet<>(); for (Object leadersDataObj : struct.getArray(LIVE_LEADERS_KEY_NAME)) { Struct leadersData = (Struct) leadersDataObj; int id = leadersData.getInt(END_POINT_ID_KEY_NAME); String host = leadersData.getString(HOST_KEY_NAME); int port = leadersData.getInt(PORT_KEY_NAME); - leaders.add(new BrokerEndPoint(id, host, port)); + leaders.add(new Node(id, host, port)); } controllerId = struct.getInt(CONTROLLER_ID_KEY_NAME); @@ -191,7 +191,7 @@ public Map partitionStates() { return partitionStates; } - public Set liveLeaders() { + public Set liveLeaders() { return liveLeaders; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java index 4c3d0a74740d9..27f89fa5796f0 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/UpdateMetadataRequest.java @@ -13,7 +13,7 @@ package org.apache.kafka.common.requests; -import org.apache.kafka.common.BrokerEndPoint; +import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.protocol.Errors; @@ -113,15 +113,15 @@ public EndPoint(String host, int port) { * Constructor for version 0. */ @Deprecated - public UpdateMetadataRequest(int controllerId, int controllerEpoch, Set liveBrokers, + public UpdateMetadataRequest(int controllerId, int controllerEpoch, Set liveBrokers, Map partitionStates) { this(0, controllerId, controllerEpoch, partitionStates, brokerEndPointsToBrokers(liveBrokers)); } - private static Set brokerEndPointsToBrokers(Set brokerEndPoints) { + private static Set brokerEndPointsToBrokers(Set brokerEndPoints) { Set brokers = new HashSet<>(brokerEndPoints.size()); - for (BrokerEndPoint brokerEndPoint : brokerEndPoints) { + for (Node brokerEndPoint : brokerEndPoints) { Map endPoints = Collections.singletonMap(SecurityProtocol.PLAINTEXT, new EndPoint(brokerEndPoint.host(), brokerEndPoint.port())); brokers.add(new Broker(brokerEndPoint.id(), endPoints, null)); diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index a4c5238bcf02a..9def5577a5bc3 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -13,7 +13,6 @@ package org.apache.kafka.common.requests; -import org.apache.kafka.common.BrokerEndPoint; import org.apache.kafka.common.Node; import org.apache.kafka.common.TopicPartition; import org.apache.kafka.common.errors.UnknownServerException; @@ -374,9 +373,9 @@ private AbstractRequest createLeaderAndIsrRequest() { partitionStates.put(new TopicPartition("topic20", 1), new LeaderAndIsrRequest.PartitionState(1, 0, 1, new ArrayList<>(isr), 2, new HashSet<>(replicas))); - Set leaders = new HashSet<>(Arrays.asList( - new BrokerEndPoint(0, "test0", 1223), - new BrokerEndPoint(1, "test1", 1223) + Set leaders = new HashSet<>(Arrays.asList( + new Node(0, "test0", 1223), + new Node(1, "test1", 1223) )); return new LeaderAndIsrRequest(1, 10, partitionStates, leaders); @@ -401,9 +400,9 @@ private AbstractRequest createUpdateMetadataRequest(int version, String rack) { new UpdateMetadataRequest.PartitionState(1, 0, 1, new ArrayList<>(isr), 2, new HashSet<>(replicas))); if (version == 0) { - Set liveBrokers = new HashSet<>(Arrays.asList( - new BrokerEndPoint(0, "host1", 1223), - new BrokerEndPoint(1, "host2", 1234) + Set liveBrokers = new HashSet<>(Arrays.asList( + new Node(0, "host1", 1223), + new Node(1, "host2", 1234) )); return new UpdateMetadataRequest(1, 10, liveBrokers, partitionStates); diff --git a/core/src/main/scala/kafka/cluster/EndPoint.scala b/core/src/main/scala/kafka/cluster/EndPoint.scala index 32c27ed37717b..3d248628ada44 100644 --- a/core/src/main/scala/kafka/cluster/EndPoint.scala +++ b/core/src/main/scala/kafka/cluster/EndPoint.scala @@ -26,6 +26,8 @@ import org.apache.kafka.common.utils.Utils object EndPoint { + private val uriParseExp = """^(.*)://\[?([0-9a-zA-Z\-.:]*)\]?:(-?[0-9]+)""".r + def readFrom(buffer: ByteBuffer): EndPoint = { val port = buffer.getInt() val host = readShortString(buffer) @@ -42,7 +44,6 @@ object EndPoint { * @return */ def createEndPoint(connectionString: String): EndPoint = { - val uriParseExp = """^(.*)://\[?([0-9a-zA-Z\-.:]*)\]?:(-?[0-9]+)""".r connectionString match { case uriParseExp(protocol, "", port) => new EndPoint(null, port.toInt, SecurityProtocol.forName(protocol)) case uriParseExp(protocol, host, port) => new EndPoint(host, port.toInt, SecurityProtocol.forName(protocol)) diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index ea156fa66fd24..b376d15e4eb1f 100755 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -30,7 +30,7 @@ import org.apache.kafka.common.network.{ChannelBuilders, LoginType, Mode, Networ import org.apache.kafka.common.protocol.{ApiKeys, SecurityProtocol} import org.apache.kafka.common.requests.{UpdateMetadataRequest, _} import org.apache.kafka.common.utils.Time -import org.apache.kafka.common.{BrokerEndPoint, Node, TopicPartition} +import org.apache.kafka.common.{Node, TopicPartition} import scala.collection.JavaConverters._ import scala.collection.{Set, mutable} @@ -351,9 +351,8 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging topicPartition.topic, topicPartition.partition)) } val leaderIds = partitionStateInfos.map(_._2.leaderIsrAndControllerEpoch.leaderAndIsr.leader).toSet - val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)).map { b => - val brokerEndPoint = b.getBrokerEndPoint(controller.config.interBrokerSecurityProtocol) - new BrokerEndPoint(brokerEndPoint.id, brokerEndPoint.host, brokerEndPoint.port) + val leaders = controllerContext.liveOrShuttingDownBrokers.filter(b => leaderIds.contains(b.id)).map { + _.getNode(controller.config.interBrokerSecurityProtocol) } val partitionStates = partitionStateInfos.map { case (topicPartition, partitionStateInfo) => val LeaderIsrAndControllerEpoch(leaderIsr, controllerEpoch) = partitionStateInfo.leaderIsrAndControllerEpoch @@ -387,10 +386,7 @@ class ControllerBrokerRequestBatch(controller: KafkaController) extends Logging val updateMetadataRequest = if (version == 0) { - val liveBrokers = controllerContext.liveOrShuttingDownBrokers.map { broker => - val brokerEndPoint = broker.getBrokerEndPoint(SecurityProtocol.PLAINTEXT) - new BrokerEndPoint(brokerEndPoint.id, brokerEndPoint.host, brokerEndPoint.port) - } + val liveBrokers = controllerContext.liveOrShuttingDownBrokers.map(_.getNode(SecurityProtocol.PLAINTEXT)) new UpdateMetadataRequest(controllerId, controllerEpoch, liveBrokers.asJava, partitionStates.asJava) } else { diff --git a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala index fad76575939d6..bc705f13f10d3 100644 --- a/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala +++ b/core/src/test/scala/integration/kafka/api/AuthorizerIntegrationTest.scala @@ -17,6 +17,7 @@ import java.net.Socket import java.nio.ByteBuffer import java.util.concurrent.ExecutionException import java.util.{ArrayList, Collections, Properties} + import kafka.cluster.EndPoint import kafka.common.TopicAndPartition import kafka.coordinator.GroupCoordinator @@ -24,15 +25,16 @@ import kafka.integration.KafkaServerTestHarness import kafka.security.auth._ import kafka.server.KafkaConfig import kafka.utils.TestUtils -import org.apache.kafka.clients.consumer.{OffsetAndMetadata, Consumer, ConsumerRecord, KafkaConsumer} +import org.apache.kafka.clients.consumer.{Consumer, ConsumerRecord, KafkaConsumer, OffsetAndMetadata} import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.errors._ import org.apache.kafka.common.protocol.{ApiKeys, Errors, SecurityProtocol} import org.apache.kafka.common.requests._ import org.apache.kafka.common.security.auth.KafkaPrincipal -import org.apache.kafka.common.{BrokerEndPoint, TopicPartition, requests} +import org.apache.kafka.common.{Node, TopicPartition, requests} import org.junit.Assert._ import org.junit.{After, Assert, Before, Test} + import scala.collection.JavaConverters._ import scala.collection.mutable import scala.collection.mutable.Buffer @@ -214,7 +216,7 @@ class AuthorizerIntegrationTest extends KafkaServerTestHarness { private def createLeaderAndIsrRequest = { new requests.LeaderAndIsrRequest(brokerId, Int.MaxValue, Map(tp -> new requests.LeaderAndIsrRequest.PartitionState(Int.MaxValue, brokerId, Int.MaxValue, List(brokerId).asJava, 2, Set(brokerId).asJava)).asJava, - Set(new BrokerEndPoint(brokerId,"localhost", 0)).asJava) + Set(new Node(brokerId, "localhost", 0)).asJava) } private def createStopReplicaRequest = { diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index 94013bcb225d2..e84780aaa2a1a 100755 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -17,22 +17,22 @@ package kafka.server -import org.apache.kafka.common.{BrokerEndPoint, TopicPartition} +import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.requests.LeaderAndIsrRequest.PartitionState import scala.collection.JavaConverters._ import kafka.api.LeaderAndIsr -import org.apache.kafka.common.requests.{LeaderAndIsrResponse, LeaderAndIsrRequest, AbstractRequestResponse} +import org.apache.kafka.common.requests.{AbstractRequestResponse, LeaderAndIsrRequest, LeaderAndIsrResponse} import org.junit.Assert._ -import kafka.utils.{TestUtils, CoreUtils} +import kafka.utils.{CoreUtils, TestUtils} import kafka.cluster.Broker import kafka.controller.{ControllerChannelManager, ControllerContext} import kafka.utils.TestUtils._ import kafka.zk.ZooKeeperTestHarness import org.apache.kafka.common.metrics.Metrics -import org.apache.kafka.common.protocol.{Errors, ApiKeys, SecurityProtocol} +import org.apache.kafka.common.protocol.{ApiKeys, Errors, SecurityProtocol} import org.apache.kafka.common.utils.SystemTime -import org.junit.{Test, After, Before} +import org.junit.{After, Before, Test} class LeaderElectionTest extends ZooKeeperTestHarness { val brokerId1 = 0 @@ -130,10 +130,7 @@ class LeaderElectionTest extends ZooKeeperTestHarness { val controllerConfig = KafkaConfig.fromProps(TestUtils.createBrokerConfig(controllerId, zkConnect)) val brokers = servers.map(s => new Broker(s.config.brokerId, "localhost", s.boundPort())) - val brokerEndPoints = brokers.map { b => - val brokerEndPoint = b.getBrokerEndPoint(SecurityProtocol.PLAINTEXT) - new BrokerEndPoint(brokerEndPoint.id, brokerEndPoint.host, brokerEndPoint.port) - } + val nodes = brokers.map(_.getNode(SecurityProtocol.PLAINTEXT)) val controllerContext = new ControllerContext(zkUtils, 6000) controllerContext.liveBrokers = brokers.toSet @@ -148,7 +145,7 @@ class LeaderElectionTest extends ZooKeeperTestHarness { Set(0, 1).map(Integer.valueOf).asJava) ) val leaderAndIsrRequest = new LeaderAndIsrRequest(controllerId, staleControllerEpoch, partitionStates.asJava, - brokerEndPoints.toSet.asJava) + nodes.toSet.asJava) controllerChannelManager.sendRequest(brokerId2, ApiKeys.LEADER_AND_ISR, None, leaderAndIsrRequest, staleControllerEpochCallback) diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index a5a8df1e32aaf..ee14af4af518d 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -33,7 +33,7 @@ import org.apache.kafka.common.requests.LeaderAndIsrRequest import org.apache.kafka.common.requests.LeaderAndIsrRequest.PartitionState import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.utils.{MockTime => JMockTime} -import org.apache.kafka.common.{BrokerEndPoint, TopicPartition} +import org.apache.kafka.common.{Node, TopicPartition} import org.easymock.EasyMock import org.junit.Assert.{assertEquals, assertTrue} import org.junit.Test @@ -162,7 +162,7 @@ class ReplicaManagerTest { // Make this replica the leader. val leaderAndIsrRequest1 = new LeaderAndIsrRequest(0, 0, collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerSet)).asJava, - Set(new BrokerEndPoint(0, "host1", 0), new BrokerEndPoint(1, "host2", 1)).asJava) + Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava) rm.becomeLeaderOrFollower(0, leaderAndIsrRequest1, metadataCache, (_, _) => {}) rm.getLeaderReplicaIfLocal(topic, 0) @@ -185,7 +185,7 @@ class ReplicaManagerTest { // Make this replica the follower val leaderAndIsrRequest2 = new LeaderAndIsrRequest(0, 0, collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 1, 1, brokerList, 0, brokerSet)).asJava, - Set(new BrokerEndPoint(0, "host1", 0), new BrokerEndPoint(1, "host2", 1)).asJava) + Set(new Node(0, "host1", 0), new Node(1, "host2", 1)).asJava) rm.becomeLeaderOrFollower(1, leaderAndIsrRequest2, metadataCache, (_, _) => {}) assertTrue(produceCallbackFired) From 7af67ce22aa02121d6b82dc54dad42358282e524 Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Wed, 23 Mar 2016 07:15:59 -0700 Subject: [PATCH 078/206] KAFKA-3442; Fix FileMessageSet iterator. Author: Jiangjie Qin Reviewers: Ismael Juma , Jun Rao Closes #1112 from becketqin/KAFKA-3442 --- .../main/scala/kafka/log/FileMessageSet.scala | 28 +++++++++++-------- .../unit/kafka/log/FileMessageSetTest.scala | 17 +++++++++-- 2 files changed, 32 insertions(+), 13 deletions(-) diff --git a/core/src/main/scala/kafka/log/FileMessageSet.scala b/core/src/main/scala/kafka/log/FileMessageSet.scala index 45b3df9970e13..a164b4b96730c 100755 --- a/core/src/main/scala/kafka/log/FileMessageSet.scala +++ b/core/src/main/scala/kafka/log/FileMessageSet.scala @@ -206,7 +206,7 @@ class FileMessageSet private[kafka](@volatile var file: File, /** * Convert this message set to use the specified message format. */ - def toMessageFormat(toMagicValue: Byte): ByteBufferMessageSet = { + def toMessageFormat(toMagicValue: Byte): MessageSet = { val offsets = new ArrayBuffer[Long] val newMessages = new ArrayBuffer[Message] this.foreach { messageAndOffset => @@ -224,11 +224,16 @@ class FileMessageSet private[kafka](@volatile var file: File, } } - // We use the offset seq to assign offsets so the offset of the messages does not change. - new ByteBufferMessageSet( - compressionCodec = this.headOption.map(_.message.compressionCodec).getOrElse(NoCompressionCodec), - offsetSeq = offsets, - newMessages: _*) + if (sizeInBytes > 0 && newMessages.size == 0) { + // This indicates that the message is too large. We just return all the bytes in the file message set. + this + } else { + // We use the offset seq to assign offsets so the offset of the messages does not change. + new ByteBufferMessageSet( + compressionCodec = this.headOption.map(_.message.compressionCodec).getOrElse(NoCompressionCodec), + offsetSeq = offsets, + newMessages: _*) + } } /** @@ -245,10 +250,11 @@ class FileMessageSet private[kafka](@volatile var file: File, def iterator(maxMessageSize: Int): Iterator[MessageAndOffset] = { new IteratorTemplate[MessageAndOffset] { var location = start - val sizeOffsetBuffer = ByteBuffer.allocate(12) + val sizeOffsetLength = 12 + val sizeOffsetBuffer = ByteBuffer.allocate(sizeOffsetLength) override def makeNext(): MessageAndOffset = { - if(location >= end) + if(location + sizeOffsetLength >= end) return allDone() // read the size of the item @@ -260,20 +266,20 @@ class FileMessageSet private[kafka](@volatile var file: File, sizeOffsetBuffer.rewind() val offset = sizeOffsetBuffer.getLong() val size = sizeOffsetBuffer.getInt() - if(size < Message.MinMessageOverhead) + if(size < Message.MinMessageOverhead || location + sizeOffsetLength + size > end) return allDone() if(size > maxMessageSize) throw new CorruptRecordException("Message size exceeds the largest allowable message size (%d).".format(maxMessageSize)) // read the item itself val buffer = ByteBuffer.allocate(size) - channel.read(buffer, location + 12) + channel.read(buffer, location + sizeOffsetLength) if(buffer.hasRemaining) return allDone() buffer.rewind() // increment the location and return the item - location += size + 12 + location += size + sizeOffsetLength new MessageAndOffset(new Message(buffer), offset) } } diff --git a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala index a3e5b2d4f01e0..534443ce3203e 100644 --- a/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala +++ b/core/src/test/scala/unit/kafka/log/FileMessageSetTest.scala @@ -133,11 +133,13 @@ class FileMessageSetTest extends BaseMessageSetTestCases { def testIteratorWithLimits() { val message = messageSet.toList(1) val start = messageSet.searchFor(1, 0).position - val size = message.message.size + val size = message.message.size + 12 val slice = messageSet.read(start, size) assertEquals(List(message), slice.toList) + val slice2 = messageSet.read(start, size - 1) + assertEquals(List(), slice2.toList) } - + /** * Test the truncateTo method lops off messages and appropriately updates the size */ @@ -202,6 +204,17 @@ class FileMessageSetTest extends BaseMessageSetTestCases { assertEquals(oldposition, tempReopen.length) } + @Test + def testFormatConversionWithPartialMessage() { + val message = messageSet.toList(1) + val start = messageSet.searchFor(1, 0).position + val size = message.message.size + 12 + val slice = messageSet.read(start, size - 1) + val messageV0 = slice.toMessageFormat(Message.MagicValue_V0) + assertEquals("No message should be there", 0, messageV0.size) + assertEquals(s"There should be ${size - 1} bytes", size - 1, messageV0.sizeInBytes) + } + @Test def testMessageFormatConversion() { From 20c313526a0518a51142d3abc5ee2a4d2ef3cb34 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Wed, 23 Mar 2016 09:47:48 -0700 Subject: [PATCH 079/206] KAFKA-3409: handle CommitFailedException in MirrorMaker Author: Jason Gustafson Reviewers: Ismael Juma, Ashish Singh, Guozhang Wang Closes #1115 from hachikuji/KAFKA-3409 --- .../main/scala/kafka/tools/MirrorMaker.scala | 24 +++++++++++++------ 1 file changed, 17 insertions(+), 7 deletions(-) diff --git a/core/src/main/scala/kafka/tools/MirrorMaker.scala b/core/src/main/scala/kafka/tools/MirrorMaker.scala index 26f4826982e60..87f3cc53bafb1 100755 --- a/core/src/main/scala/kafka/tools/MirrorMaker.scala +++ b/core/src/main/scala/kafka/tools/MirrorMaker.scala @@ -25,14 +25,13 @@ import java.util.{Collections, Properties} import com.yammer.metrics.core.Gauge import joptsimple.OptionParser -import kafka.client.ClientUtils import kafka.consumer.{BaseConsumerRecord, ConsumerIterator, BaseConsumer, Blacklist, ConsumerConfig, ConsumerThreadId, ConsumerTimeoutException, TopicFilter, Whitelist, ZookeeperConsumerConnector} import kafka.javaapi.consumer.ConsumerRebalanceListener -import kafka.message.MessageAndMetadata import kafka.metrics.KafkaMetricsGroup import kafka.serializer.DefaultDecoder import kafka.utils.{CommandLineUtils, CoreUtils, Logging} -import org.apache.kafka.clients.consumer.{OffsetAndMetadata, Consumer, ConsumerRecord, KafkaConsumer} +import org.apache.kafka.clients.consumer +import org.apache.kafka.clients.consumer.{OffsetAndMetadata, Consumer, ConsumerRecord, KafkaConsumer, CommitFailedException} import org.apache.kafka.clients.producer.internals.ErrorLoggingCallback import org.apache.kafka.clients.producer.{KafkaProducer, ProducerConfig, ProducerRecord, RecordMetadata} import org.apache.kafka.common.TopicPartition @@ -356,6 +355,12 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { // and re-throw to break the loop mirrorMakerConsumer.commit() throw e + + case e: CommitFailedException => + warn("Failed to commit offsets because the consumer group has rebalanced and assigned partitions to " + + "another instance. If you see this regularly, it could indicate that you need to either increase " + + s"the consumer's ${consumer.ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG} or reduce the number of records " + + s"handled on each iteration with ${consumer.ConsumerConfig.MAX_POLL_RECORDS_CONFIG}") } } else { info("Exiting on send failure, skip committing offsets.") @@ -422,10 +427,15 @@ object MirrorMaker extends Logging with KafkaMetricsGroup { case t: Throwable => fatal("Mirror maker thread failure due to ", t) } finally { - info("Flushing producer.") - producer.flush() - info("Committing consumer offsets.") - CoreUtils.swallow(commitOffsets(mirrorMakerConsumer)) + CoreUtils.swallow { + info("Flushing producer.") + producer.flush() + + // note that this commit is skipped if flush() fails which ensures that we don't lose messages + info("Committing consumer offsets.") + commitOffsets(mirrorMakerConsumer) + } + info("Shutting down consumer connectors.") CoreUtils.swallow(mirrorMakerConsumer.stop()) CoreUtils.swallow(mirrorMakerConsumer.cleanup()) From d57847641037823a7306f53251b063869f7affb6 Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Wed, 23 Mar 2016 12:54:26 -0700 Subject: [PATCH 080/206] KAFKA-3441: 0.10.0 documentation still says "0.9.0" Author: Grant Henke Reviewers: Gwen Shapira Closes #1122 from granthenke/docs-10 --- docs/api.html | 8 ++++---- docs/quickstart.html | 6 +++--- 2 files changed, 7 insertions(+), 7 deletions(-) diff --git a/docs/api.html b/docs/api.html index 254155392027b..d303244427d73 100644 --- a/docs/api.html +++ b/docs/api.html @@ -24,12 +24,12 @@

        2.1 Producer API

        <dependency> <groupId>org.apache.kafka</groupId> <artifactId>kafka-clients</artifactId> - <version>0.9.0.0</version> + <version>0.10.0.0</version> </dependency> Examples showing how to use the producer are given in the -javadocs. +javadocs.

        For those interested in the legacy Scala producer api, information can be found @@ -159,9 +159,9 @@

        2.2.3 New Consumer API

        Examples showing how to use the consumer are given in the -javadocs. +javadocs. diff --git a/docs/quickstart.html b/docs/quickstart.html index 1238316dbee69..1e7b62ce3c0a8 100644 --- a/docs/quickstart.html +++ b/docs/quickstart.html @@ -21,11 +21,11 @@

        1.3 Quick Start

        Step 1: Download the code

        -Download the 0.9.0.0 release and un-tar it. +Download the 0.10.0.0 release and un-tar it.
        -> tar -xzf kafka_2.11-0.9.0.0.tgz
        -> cd kafka_2.11-0.9.0.0
        +> tar -xzf kafka_2.11-0.10.0.0.tgz
        +> cd kafka_2.11-0.10.0.0
         

        Step 2: Start the server

        From d4d5920ed40736d21f056188efa8a86c93e22506 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 23 Mar 2016 13:53:37 -0700 Subject: [PATCH 081/206] KAFKA-3432; Cluster.update() thread-safety Replace `update` with `withPartitions`, which returns a copy instead of mutating the instance. Author: Ismael Juma Reviewers: Guozhang Wang Closes #1118 from ijuma/kafka-3432-cluster-update-thread-safety --- .../java/org/apache/kafka/common/Cluster.java | 40 ++++--------------- .../internals/StreamPartitionAssignor.java | 12 +++++- 2 files changed, 17 insertions(+), 35 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/Cluster.java b/clients/src/main/java/org/apache/kafka/common/Cluster.java index 4f3735850fc11..8e85df8f0903e 100644 --- a/clients/src/main/java/org/apache/kafka/common/Cluster.java +++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java @@ -19,6 +19,7 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Set; @@ -50,7 +51,7 @@ public Cluster(Collection nodes, this.nodes = Collections.unmodifiableList(copy); this.nodesById = new HashMap<>(); - for (Node node: nodes) + for (Node node : nodes) this.nodesById.put(node.id(), node); // index the partitions by topic/partition for quick lookup @@ -118,39 +119,12 @@ public static Cluster bootstrap(List addresses) { } /** - * Update the cluster information for specific topic with new partition information + * Return a copy of this cluster combined with `partitions`. */ - public Cluster update(String topic, Collection partitions) { - - // re-index the partitions by topic/partition for quick lookup - for (PartitionInfo p : partitions) - this.partitionsByTopicPartition.put(new TopicPartition(p.topic(), p.partition()), p); - - // re-index the partitions by topic and node respectively - this.partitionsByTopic.put(topic, Collections.unmodifiableList(new ArrayList<>(partitions))); - - List availablePartitions = new ArrayList<>(); - for (PartitionInfo part : partitions) { - if (part.leader() != null) - availablePartitions.add(part); - } - this.availablePartitionsByTopic.put(topic, Collections.unmodifiableList(availablePartitions)); - - HashMap> partsForNode = new HashMap<>(); - for (Node n : this.nodes) { - partsForNode.put(n.id(), new ArrayList()); - } - for (PartitionInfo p : partitions) { - if (p.leader() != null) { - List psNode = Utils.notNull(partsForNode.get(p.leader().id())); - psNode.add(p); - } - } - - for (Map.Entry> entry : partsForNode.entrySet()) - this.partitionsByNode.put(entry.getKey(), Collections.unmodifiableList(entry.getValue())); - - return this; + public Cluster withPartitions(Map partitions) { + Map combinedPartitions = new HashMap<>(this.partitionsByTopicPartition); + combinedPartitions.putAll(partitions); + return new Cluster(this.nodes, combinedPartitions.values(), new HashSet<>(this.unauthorizedTopics)); } /** diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java index a6b82af02e28c..1dd082d8cf5c1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java @@ -226,6 +226,8 @@ public Map assign(Cluster metadata, Map internalPartitionInfos = new HashMap<>(); + // if ZK is specified, prepare the internal source topic before calling partition grouper if (internalTopicManager != null) { log.debug("Starting to validate internal source topics in partition assignor."); @@ -247,15 +249,21 @@ public Map assign(Cluster metadata, Map> partitionsForTask = streamThread.partitionGrouper.partitionGroups(sourceTopicGroups, metadata); + Map> partitionsForTask = streamThread.partitionGrouper.partitionGroups( + sourceTopicGroups, metadataWithInternalTopics); // add tasks to state change log topic subscribers stateChangelogTopicToTaskIds = new HashMap<>(); From 80d78f81470f109dc6d221f755b039c7332bb93b Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 23 Mar 2016 14:25:08 -0700 Subject: [PATCH 082/206] HOTFIX: fix NPE in changelogger Fix NPE in StoreChangeLogger caused by a record out of window retention period. guozhangwang Author: Yasuhiro Matsuda Reviewers: Guozhang Wang Closes #1124 from ymatsuda/logger_npe --- .../kafka/streams/state/internals/RocksDBWindowStore.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java index 4c6a2296b6325..9851c0489b886 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java @@ -245,7 +245,7 @@ public void close() { public void put(K key, V value) { byte[] rawKey = putAndReturnInternalKey(key, value, USE_CURRENT_TIMESTAMP); - if (loggingEnabled) { + if (rawKey != null && loggingEnabled) { changeLogger.add(rawKey); changeLogger.maybeLogChange(this.getter); } @@ -255,7 +255,7 @@ public void put(K key, V value) { public void put(K key, V value, long timestamp) { byte[] rawKey = putAndReturnInternalKey(key, value, timestamp); - if (loggingEnabled) { + if (rawKey != null && loggingEnabled) { changeLogger.add(rawKey); changeLogger.maybeLogChange(this.getter); } From de062443381df84ee0d65acc20e44ffca2b2552b Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 23 Mar 2016 14:57:03 -0700 Subject: [PATCH 083/206] MINOR: remove streams-smoke-test.sh guozhangwang Author: Yasuhiro Matsuda Reviewers: Guozhang Wang Closes #1125 from ymatsuda/remove_smoketest_shell_script --- bin/streams-smoke-test.sh | 23 ----------------------- tests/kafkatest/services/streams.py | 3 ++- 2 files changed, 2 insertions(+), 24 deletions(-) delete mode 100755 bin/streams-smoke-test.sh diff --git a/bin/streams-smoke-test.sh b/bin/streams-smoke-test.sh deleted file mode 100755 index 196990ef41c65..0000000000000 --- a/bin/streams-smoke-test.sh +++ /dev/null @@ -1,23 +0,0 @@ -#!/bin/sh -# 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. - -base_dir=$(dirname $0) - -if [ "x$KAFKA_LOG4J_OPTS" = "x" ]; then - export KAFKA_LOG4J_OPTS="-Dlog4j.configuration=file:$base_dir/../config/tools-log4j.properties" -fi - -exec $(dirname $0)/kafka-run-class.sh org.apache.kafka.streams.smoketest.StreamsSmokeTest "$@" diff --git a/tests/kafkatest/services/streams.py b/tests/kafkatest/services/streams.py index 192a8d9fcfa86..dcbcc696b8b3f 100644 --- a/tests/kafkatest/services/streams.py +++ b/tests/kafkatest/services/streams.py @@ -107,7 +107,8 @@ def start_cmd(self, node): args['kafka_dir'] = kafka_dir(node) cmd = "( export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%(log4j)s\"; " \ - "/opt/%(kafka_dir)s/bin/streams-smoke-test.sh %(command)s %(kafka)s %(zk)s %(state_dir)s " \ + "/opt/%(kafka_dir)s/bin/kafka-run-class.sh org.apache.kafka.streams.smoketest.StreamsSmokeTest " \ + " %(command)s %(kafka)s %(zk)s %(state_dir)s " \ " & echo $! >&3 ) 1>> %(stdout)s 2>> %(stderr)s 3> %(pidfile)s" % args return cmd From cb78223bf90aca4f75699f36c1a82db7661a62f3 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Wed, 23 Mar 2016 22:36:19 -0700 Subject: [PATCH 084/206] KAFKA-3434; add old constructor to ConsumerRecord Author: Jason Gustafson Reviewers: Grant Henke , Ismael Juma , Ewen Cheslack-Postava Closes #1123 from hachikuji/KAFKA-3434 --- .../clients/consumer/ConsumerRecord.java | 29 +++++++++++ .../clients/consumer/internals/Fetcher.java | 4 +- .../clients/consumer/ConsumerRecordTest.java | 48 +++++++++++++++++++ 3 files changed, 79 insertions(+), 2 deletions(-) create mode 100644 clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordTest.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java index 4165534456de6..586156e074613 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerRecord.java @@ -12,6 +12,7 @@ */ package org.apache.kafka.clients.consumer; +import org.apache.kafka.common.record.Record; import org.apache.kafka.common.record.TimestampType; /** @@ -19,6 +20,10 @@ * record is being received and an offset that points to the record in a Kafka partition. */ public final class ConsumerRecord { + public static final long NO_TIMESTAMP = Record.NO_TIMESTAMP; + public static final int NULL_SIZE = -1; + public static final int NULL_CHECKSUM = -1; + private final String topic; private final int partition; private final long offset; @@ -30,6 +35,27 @@ public final class ConsumerRecord { private final K key; private final V value; + /** + * Creates a record to be received from a specified topic and partition (provided for + * compatibility with Kafka 0.9 before the message format supported timestamps and before + * serialized metadata were exposed). + * + * @param topic The topic this record is received from + * @param partition The partition of the topic this record is received from + * @param offset The offset of this record in the corresponding Kafka partition + * @param key The key of the record, if one exists (null is allowed) + * @param value The record contents + */ + public ConsumerRecord(String topic, + int partition, + long offset, + K key, + V value) { + this(topic, partition, offset, NO_TIMESTAMP, TimestampType.NO_TIMESTAMP_TYPE, + NULL_CHECKSUM, NULL_SIZE, NULL_SIZE, key, value); + } + + /** * Creates a record to be received from a specified topic and partition * @@ -38,6 +64,9 @@ public final class ConsumerRecord { * @param offset The offset of this record in the corresponding Kafka partition * @param timestamp The timestamp of the record. * @param timestampType The timestamp type + * @param checksum The checksum (CRC32) of the full record + * @param serializedKeySize The length of the serialized key + * @param serializedValueSize The length of the serialized value * @param key The key of the record, if one exists (null is allowed) * @param value The record contents */ diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 802a2f0b63e7b..9a26551a1e7ab 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -653,8 +653,8 @@ private ConsumerRecord parseRecord(TopicPartition partition, LogEntry logE return new ConsumerRecord<>(partition.topic(), partition.partition(), offset, timestamp, timestampType, logEntry.record().checksum(), - keyByteArray == null ? -1 : keyByteArray.length, - valueByteArray == null ? -1 : valueByteArray.length, + keyByteArray == null ? ConsumerRecord.NULL_SIZE : keyByteArray.length, + valueByteArray == null ? ConsumerRecord.NULL_SIZE : valueByteArray.length, key, value); } catch (KafkaException e) { throw e; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordTest.java new file mode 100644 index 0000000000000..d1d3b24afeff8 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordTest.java @@ -0,0 +1,48 @@ +/** + * 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 org.apache.kafka.clients.consumer; + +import org.apache.kafka.common.record.TimestampType; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class ConsumerRecordTest { + + @Test + public void testOldConstructor() { + String topic = "topic"; + int partition = 0; + long offset = 23; + String key = "key"; + String value = "value"; + + ConsumerRecord record = new ConsumerRecord(topic, partition, offset, key, value); + assertEquals(topic, record.topic()); + assertEquals(partition, record.partition()); + assertEquals(offset, record.offset()); + assertEquals(key, record.key()); + assertEquals(value, record.value()); + assertEquals(TimestampType.NO_TIMESTAMP_TYPE, record.timestampType()); + assertEquals(ConsumerRecord.NO_TIMESTAMP, record.timestamp()); + assertEquals(ConsumerRecord.NULL_CHECKSUM, record.checksum()); + assertEquals(ConsumerRecord.NULL_SIZE, record.serializedKeySize()); + assertEquals(ConsumerRecord.NULL_SIZE, record.serializedValueSize()); + } + + +} From 419e268d6fe9695e31e9be299c99573522a81dd0 Mon Sep 17 00:00:00 2001 From: Ryan P Date: Thu, 24 Mar 2016 10:12:19 -0700 Subject: [PATCH 085/206] KAFKA-3445: Validate TASKS_MAX_CONFIG's lower bound Currently the property TASKS_MAX_CONFIG is not validated against nonsensical values such as 0. This patch leverages the Range.atLeast() method to ensure value is at least 1. Author: Ryan P Reviewers: Ewen Cheslack-Postava Closes #1132 from rnpridgeon/KAFKA-3445 --- .../org/apache/kafka/connect/runtime/ConnectorConfig.java | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java index e21faf6cbf161..e4395523263f4 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorConfig.java @@ -22,6 +22,8 @@ import org.apache.kafka.common.config.ConfigDef.Importance; import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.config.ConfigDef.Width; +import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; + import java.util.HashMap; import java.util.Map; @@ -54,6 +56,8 @@ public class ConnectorConfig extends AbstractConfig { public static final String TASKS_MAX_CONFIG = "tasks.max"; private static final String TASKS_MAX_DOC = "Maximum number of tasks to use for this connector."; public static final int TASKS_MAX_DEFAULT = 1; + private static final int TASKS_MIN_CONFIG = 1; + private static final String TASK_MAX_DISPLAY = "Tasks max"; public static final String TOPICS_CONFIG = "topics"; @@ -67,7 +71,7 @@ public class ConnectorConfig extends AbstractConfig { config = new ConfigDef() .define(NAME_CONFIG, Type.STRING, Importance.HIGH, NAME_DOC, COMMON_GROUP, 1, Width.MEDIUM, NAME_DISPLAY) .define(CONNECTOR_CLASS_CONFIG, Type.STRING, Importance.HIGH, CONNECTOR_CLASS_DOC, COMMON_GROUP, 2, Width.LONG, CONNECTOR_CLASS_DISPLAY) - .define(TASKS_MAX_CONFIG, Type.INT, TASKS_MAX_DEFAULT, Importance.HIGH, TASKS_MAX_DOC, COMMON_GROUP, 3, Width.SHORT, TASK_MAX_DISPLAY) + .define(TASKS_MAX_CONFIG, Type.INT, TASKS_MAX_DEFAULT, atLeast(TASKS_MIN_CONFIG), Importance.HIGH, TASKS_MAX_DOC, COMMON_GROUP, 3, Width.SHORT, TASK_MAX_DISPLAY) .define(TOPICS_CONFIG, Type.LIST, TOPICS_DEFAULT, Importance.HIGH, TOPICS_DOC, COMMON_GROUP, 4, Width.LONG, TOPICS_DISPLAY); } From 2dc17230f90be60c6c08061600fdf79c8e5f5b52 Mon Sep 17 00:00:00 2001 From: Jeremy Custenborder Date: Thu, 24 Mar 2016 11:03:30 -0700 Subject: [PATCH 086/206] KAFKA-3407 - ErrorLoggingCallback trims helpful diagnostic information. This should help when diagnosing issues with the console producer. This allows the logger to use `exception` rather than `exception.getMessage()`. Author: Jeremy Custenborder Reviewers: Ewen Cheslack-Postava Closes #1079 from jcustenborder/KAFKA-3407 --- .../clients/producer/internals/ErrorLoggingCallback.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ErrorLoggingCallback.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ErrorLoggingCallback.java index 747e29fadbcf6..18088c1c04291 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/ErrorLoggingCallback.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/ErrorLoggingCallback.java @@ -44,8 +44,8 @@ public void onCompletion(RecordMetadata metadata, Exception e) { logAsString ? new String(key) : key.length + " bytes"; String valueString = (valueLength == -1) ? "null" : logAsString ? new String(value) : valueLength + " bytes"; - log.error("Error when sending message to topic {} with key: {}, value: {} with error: {}", - topic, keyString, valueString, e.getMessage()); + log.error("Error when sending message to topic {} with key: {}, value: {} with error:", + topic, keyString, valueString, e); } } } From d3a66a65365579e9320347663969b30c1148e497 Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Fri, 25 Mar 2016 10:07:05 -0700 Subject: [PATCH 087/206] KAFKA-3460: Remove old 0.7 KafkaMigrationTool Author: Grant Henke Reviewers: Gwen Shapira Closes #1136 from granthenke/remove-07-migration --- .../scala/kafka/tools/KafkaMigrationTool.java | 487 ------------------ docs/upgrade.html | 1 + 2 files changed, 1 insertion(+), 487 deletions(-) delete mode 100755 core/src/main/scala/kafka/tools/KafkaMigrationTool.java diff --git a/core/src/main/scala/kafka/tools/KafkaMigrationTool.java b/core/src/main/scala/kafka/tools/KafkaMigrationTool.java deleted file mode 100755 index 0b94902b8806b..0000000000000 --- a/core/src/main/scala/kafka/tools/KafkaMigrationTool.java +++ /dev/null @@ -1,487 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.tools; - -import joptsimple.ArgumentAcceptingOptionSpec; -import joptsimple.OptionParser; -import joptsimple.OptionSet; -import joptsimple.OptionSpec; -import joptsimple.OptionSpecBuilder; -import kafka.javaapi.producer.Producer; -import kafka.producer.KeyedMessage; -import kafka.producer.ProducerConfig; -import org.apache.kafka.common.utils.Utils; - -import java.io.File; -import java.io.FileInputStream; -import java.io.IOException; -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; -import java.net.URL; -import java.net.URLClassLoader; -import java.nio.ByteBuffer; -import java.util.ArrayList; -import java.util.List; -import java.util.Properties; -import java.util.concurrent.ArrayBlockingQueue; -import java.util.concurrent.BlockingQueue; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.atomic.AtomicBoolean; - -/** - * This is a kafka 0.7 to 0.8 online migration tool used for migrating data from 0.7 to 0.8 cluster. Internally, - * it's composed of a kafka 0.7 consumer and kafka 0.8 producer. The kafka 0.7 consumer consumes data from the - * 0.7 cluster, and the kafka 0.8 producer produces data to the 0.8 cluster. - * - * The 0.7 consumer is loaded from kafka 0.7 jar using a "parent last, child first" java class loader. - * Ordinary class loader is "parent first, child last", and kafka 0.8 and 0.7 both have classes for a lot of - * class names like "kafka.consumer.Consumer", etc., so ordinary java URLClassLoader with kafka 0.7 jar will - * will still load the 0.8 version class. - * - * As kafka 0.7 and kafka 0.8 used different version of zkClient, the zkClient jar used by kafka 0.7 should - * also be used by the class loader. - * - * The user need to provide the configuration file for 0.7 consumer and 0.8 producer. For 0.8 producer, - * the "serializer.class" config is set to "kafka.serializer.DefaultEncoder" by the code. - */ -@SuppressWarnings({"unchecked", "rawtypes", "deprecation"}) -public class KafkaMigrationTool { - private static final org.apache.log4j.Logger log = org.apache.log4j.Logger.getLogger(KafkaMigrationTool.class.getName()); - private static final String KAFKA_07_STATIC_CONSUMER_CLASS_NAME = "kafka.consumer.Consumer"; - private static final String KAFKA_07_CONSUMER_CONFIG_CLASS_NAME = "kafka.consumer.ConsumerConfig"; - private static final String KAFKA_07_CONSUMER_STREAM_CLASS_NAME = "kafka.consumer.KafkaStream"; - private static final String KAFKA_07_CONSUMER_ITERATOR_CLASS_NAME = "kafka.consumer.ConsumerIterator"; - private static final String KAFKA_07_CONSUMER_CONNECTOR_CLASS_NAME = "kafka.javaapi.consumer.ConsumerConnector"; - private static final String KAFKA_07_MESSAGE_AND_METADATA_CLASS_NAME = "kafka.message.MessageAndMetadata"; - private static final String KAFKA_07_MESSAGE_CLASS_NAME = "kafka.message.Message"; - private static final String KAFKA_07_WHITE_LIST_CLASS_NAME = "kafka.consumer.Whitelist"; - private static final String KAFKA_07_TOPIC_FILTER_CLASS_NAME = "kafka.consumer.TopicFilter"; - private static final String KAFKA_07_BLACK_LIST_CLASS_NAME = "kafka.consumer.Blacklist"; - - private static Class kafkaStaticConsumer07 = null; - private static Class consumerConfig07 = null; - private static Class consumerConnector07 = null; - private static Class kafkaStream07 = null; - private static Class topicFilter07 = null; - private static Class whiteList07 = null; - private static Class blackList07 = null; - private static Class kafkaConsumerIteratorClass07 = null; - private static Class kafkaMessageAndMetaDataClass07 = null; - private static Class kafkaMessageClass07 = null; - - public static void main(String[] args) throws InterruptedException, IOException { - OptionParser parser = new OptionParser(); - ArgumentAcceptingOptionSpec consumerConfigOpt - = parser.accepts("consumer.config", "Kafka 0.7 consumer config to consume from the source 0.7 cluster. " + "You man specify multiple of these.") - .withRequiredArg() - .describedAs("config file") - .ofType(String.class); - - ArgumentAcceptingOptionSpec producerConfigOpt - = parser.accepts("producer.config", "Producer config.") - .withRequiredArg() - .describedAs("config file") - .ofType(String.class); - - ArgumentAcceptingOptionSpec numProducersOpt - = parser.accepts("num.producers", "Number of producer instances") - .withRequiredArg() - .describedAs("Number of producers") - .ofType(Integer.class) - .defaultsTo(1); - - ArgumentAcceptingOptionSpec zkClient01JarOpt - = parser.accepts("zkclient.01.jar", "zkClient 0.1 jar file") - .withRequiredArg() - .describedAs("zkClient 0.1 jar file required by Kafka 0.7") - .ofType(String.class); - - ArgumentAcceptingOptionSpec kafka07JarOpt - = parser.accepts("kafka.07.jar", "Kafka 0.7 jar file") - .withRequiredArg() - .describedAs("kafka 0.7 jar") - .ofType(String.class); - - ArgumentAcceptingOptionSpec numStreamsOpt - = parser.accepts("num.streams", "Number of consumer streams") - .withRequiredArg() - .describedAs("Number of consumer threads") - .ofType(Integer.class) - .defaultsTo(1); - - ArgumentAcceptingOptionSpec whitelistOpt - = parser.accepts("whitelist", "Whitelist of topics to migrate from the 0.7 cluster") - .withRequiredArg() - .describedAs("Java regex (String)") - .ofType(String.class); - - ArgumentAcceptingOptionSpec blacklistOpt - = parser.accepts("blacklist", "Blacklist of topics to migrate from the 0.7 cluster") - .withRequiredArg() - .describedAs("Java regex (String)") - .ofType(String.class); - - ArgumentAcceptingOptionSpec queueSizeOpt - = parser.accepts("queue.size", "Number of messages that are buffered between the 0.7 consumer and 0.8 producer") - .withRequiredArg() - .describedAs("Queue size in terms of number of messages") - .ofType(Integer.class) - .defaultsTo(10000); - - OptionSpecBuilder helpOpt - = parser.accepts("help", "Print this message."); - - OptionSet options = parser.parse(args); - - if (options.has(helpOpt)) { - parser.printHelpOn(System.out); - System.exit(0); - } - - checkRequiredArgs(parser, options, new OptionSpec[]{consumerConfigOpt, producerConfigOpt, zkClient01JarOpt, kafka07JarOpt}); - int whiteListCount = options.has(whitelistOpt) ? 1 : 0; - int blackListCount = options.has(blacklistOpt) ? 1 : 0; - if (whiteListCount + blackListCount != 1) { - System.err.println("Exactly one of whitelist or blacklist is required."); - System.exit(1); - } - - String kafkaJarFile07 = options.valueOf(kafka07JarOpt); - String zkClientJarFile = options.valueOf(zkClient01JarOpt); - String consumerConfigFile07 = options.valueOf(consumerConfigOpt); - int numConsumers = options.valueOf(numStreamsOpt); - String producerConfigFile08 = options.valueOf(producerConfigOpt); - int numProducers = options.valueOf(numProducersOpt); - final List migrationThreads = new ArrayList(numConsumers); - final List producerThreads = new ArrayList(numProducers); - - try { - File kafkaJar07 = new File(kafkaJarFile07); - File zkClientJar = new File(zkClientJarFile); - ParentLastURLClassLoader c1 = new ParentLastURLClassLoader(new URL[]{ - kafkaJar07.toURI().toURL(), - zkClientJar.toURI().toURL() - }); - - /** Construct the 07 consumer config **/ - consumerConfig07 = c1.loadClass(KAFKA_07_CONSUMER_CONFIG_CLASS_NAME); - kafkaStaticConsumer07 = c1.loadClass(KAFKA_07_STATIC_CONSUMER_CLASS_NAME); - consumerConnector07 = c1.loadClass(KAFKA_07_CONSUMER_CONNECTOR_CLASS_NAME); - kafkaStream07 = c1.loadClass(KAFKA_07_CONSUMER_STREAM_CLASS_NAME); - topicFilter07 = c1.loadClass(KAFKA_07_TOPIC_FILTER_CLASS_NAME); - whiteList07 = c1.loadClass(KAFKA_07_WHITE_LIST_CLASS_NAME); - blackList07 = c1.loadClass(KAFKA_07_BLACK_LIST_CLASS_NAME); - kafkaMessageClass07 = c1.loadClass(KAFKA_07_MESSAGE_CLASS_NAME); - kafkaConsumerIteratorClass07 = c1.loadClass(KAFKA_07_CONSUMER_ITERATOR_CLASS_NAME); - kafkaMessageAndMetaDataClass07 = c1.loadClass(KAFKA_07_MESSAGE_AND_METADATA_CLASS_NAME); - - Constructor consumerConfigConstructor07 = consumerConfig07.getConstructor(Properties.class); - Properties kafkaConsumerProperties07 = new Properties(); - kafkaConsumerProperties07.load(new FileInputStream(consumerConfigFile07)); - /** Disable shallow iteration because the message format is different between 07 and 08, we have to get each individual message **/ - if (kafkaConsumerProperties07.getProperty("shallow.iterator.enable", "").equals("true")) { - log.warn("Shallow iterator should not be used in the migration tool"); - kafkaConsumerProperties07.setProperty("shallow.iterator.enable", "false"); - } - Object consumerConfig07 = consumerConfigConstructor07.newInstance(kafkaConsumerProperties07); - - /** Construct the 07 consumer connector **/ - Method consumerConnectorCreationMethod07 = kafkaStaticConsumer07.getMethod("createJavaConsumerConnector", KafkaMigrationTool.consumerConfig07); - final Object consumerConnector07 = consumerConnectorCreationMethod07.invoke(null, consumerConfig07); - Method consumerConnectorCreateMessageStreamsMethod07 = KafkaMigrationTool.consumerConnector07.getMethod( - "createMessageStreamsByFilter", - topicFilter07, int.class); - final Method consumerConnectorShutdownMethod07 = KafkaMigrationTool.consumerConnector07.getMethod("shutdown"); - Constructor whiteListConstructor07 = whiteList07.getConstructor(String.class); - Constructor blackListConstructor07 = blackList07.getConstructor(String.class); - Object filterSpec = null; - if (options.has(whitelistOpt)) - filterSpec = whiteListConstructor07.newInstance(options.valueOf(whitelistOpt)); - else - filterSpec = blackListConstructor07.newInstance(options.valueOf(blacklistOpt)); - - Object retKafkaStreams = consumerConnectorCreateMessageStreamsMethod07.invoke(consumerConnector07, filterSpec, numConsumers); - - Properties kafkaProducerProperties08 = new Properties(); - kafkaProducerProperties08.load(new FileInputStream(producerConfigFile08)); - kafkaProducerProperties08.setProperty("serializer.class", "kafka.serializer.DefaultEncoder"); - // create a producer channel instead - int queueSize = options.valueOf(queueSizeOpt); - ProducerDataChannel> producerDataChannel = new ProducerDataChannel>(queueSize); - int threadId = 0; - - Runtime.getRuntime().addShutdownHook(new Thread() { - @Override - public void run() { - try { - consumerConnectorShutdownMethod07.invoke(consumerConnector07); - } catch (Exception e) { - log.error("Error while shutting down Kafka consumer", e); - } - for (MigrationThread migrationThread : migrationThreads) { - migrationThread.shutdown(); - } - for (ProducerThread producerThread : producerThreads) { - producerThread.shutdown(); - } - for (ProducerThread producerThread : producerThreads) { - producerThread.awaitShutdown(); - } - log.info("Kafka migration tool shutdown successfully"); - } - }); - - // start consumer threads - for (Object stream : (List) retKafkaStreams) { - MigrationThread thread = new MigrationThread(stream, producerDataChannel, threadId); - threadId++; - thread.start(); - migrationThreads.add(thread); - } - - String clientId = kafkaProducerProperties08.getProperty("client.id"); - // start producer threads - for (int i = 0; i < numProducers; i++) { - kafkaProducerProperties08.put("client.id", clientId + "-" + i); - ProducerConfig producerConfig08 = new ProducerConfig(kafkaProducerProperties08); - Producer producer = new Producer(producerConfig08); - ProducerThread producerThread = new ProducerThread(producerDataChannel, producer, i); - producerThread.start(); - producerThreads.add(producerThread); - } - } catch (Throwable e) { - System.out.println("Kafka migration tool failed due to: " + Utils.stackTrace(e)); - log.error("Kafka migration tool failed: ", e); - } - } - - private static void checkRequiredArgs(OptionParser parser, OptionSet options, OptionSpec[] required) throws IOException { - for (OptionSpec arg : required) { - if (!options.has(arg)) { - System.err.println("Missing required argument \"" + arg + "\""); - parser.printHelpOn(System.err); - System.exit(1); - } - } - } - - static class ProducerDataChannel { - private final int producerQueueSize; - private final BlockingQueue producerRequestQueue; - - public ProducerDataChannel(int queueSize) { - producerQueueSize = queueSize; - producerRequestQueue = new ArrayBlockingQueue(producerQueueSize); - } - - public void sendRequest(T data) throws InterruptedException { - producerRequestQueue.put(data); - } - - public T receiveRequest() throws InterruptedException { - return producerRequestQueue.take(); - } - } - - private static class MigrationThread extends Thread { - private final Object stream; - private final ProducerDataChannel> producerDataChannel; - private final int threadId; - private final String threadName; - private final org.apache.log4j.Logger logger; - private CountDownLatch shutdownComplete = new CountDownLatch(1); - private final AtomicBoolean isRunning = new AtomicBoolean(true); - - MigrationThread(Object stream, ProducerDataChannel> producerDataChannel, int threadId) { - this.stream = stream; - this.producerDataChannel = producerDataChannel; - this.threadId = threadId; - threadName = "MigrationThread-" + threadId; - logger = org.apache.log4j.Logger.getLogger(MigrationThread.class.getName()); - this.setName(threadName); - } - - public void run() { - try { - Method messageGetPayloadMethod07 = kafkaMessageClass07.getMethod("payload"); - Method kafkaGetMessageMethod07 = kafkaMessageAndMetaDataClass07.getMethod("message"); - Method kafkaGetTopicMethod07 = kafkaMessageAndMetaDataClass07.getMethod("topic"); - Method consumerIteratorMethod = kafkaStream07.getMethod("iterator"); - Method kafkaStreamHasNextMethod07 = kafkaConsumerIteratorClass07.getMethod("hasNext"); - Method kafkaStreamNextMethod07 = kafkaConsumerIteratorClass07.getMethod("next"); - Object iterator = consumerIteratorMethod.invoke(stream); - - while (((Boolean) kafkaStreamHasNextMethod07.invoke(iterator)).booleanValue()) { - Object messageAndMetaData07 = kafkaStreamNextMethod07.invoke(iterator); - Object message07 = kafkaGetMessageMethod07.invoke(messageAndMetaData07); - Object topic = kafkaGetTopicMethod07.invoke(messageAndMetaData07); - Object payload07 = messageGetPayloadMethod07.invoke(message07); - int size = ((ByteBuffer) payload07).remaining(); - byte[] bytes = new byte[size]; - ((ByteBuffer) payload07).get(bytes); - if (logger.isDebugEnabled()) - logger.debug("Migration thread " + threadId + " sending message of size " + bytes.length + " to topic " + topic); - KeyedMessage producerData = new KeyedMessage((String) topic, null, bytes); - producerDataChannel.sendRequest(producerData); - } - logger.info("Migration thread " + threadName + " finished running"); - } catch (InvocationTargetException t) { - logger.fatal("Migration thread failure due to root cause ", t.getCause()); - } catch (Throwable t) { - logger.fatal("Migration thread failure due to ", t); - } finally { - shutdownComplete.countDown(); - } - } - - public void shutdown() { - logger.info("Migration thread " + threadName + " shutting down"); - isRunning.set(false); - interrupt(); - try { - shutdownComplete.await(); - } catch (InterruptedException ie) { - logger.warn("Interrupt during shutdown of MigrationThread", ie); - } - logger.info("Migration thread " + threadName + " shutdown complete"); - } - } - - static class ProducerThread extends Thread { - private final ProducerDataChannel> producerDataChannel; - private final Producer producer; - private final int threadId; - private String threadName; - private org.apache.log4j.Logger logger; - private CountDownLatch shutdownComplete = new CountDownLatch(1); - private KeyedMessage shutdownMessage = new KeyedMessage("shutdown", null, null); - - public ProducerThread(ProducerDataChannel> producerDataChannel, - Producer producer, - int threadId) { - this.producerDataChannel = producerDataChannel; - this.producer = producer; - this.threadId = threadId; - threadName = "ProducerThread-" + threadId; - logger = org.apache.log4j.Logger.getLogger(ProducerThread.class.getName()); - this.setName(threadName); - } - - public void run() { - try { - while (true) { - KeyedMessage data = producerDataChannel.receiveRequest(); - if (!data.equals(shutdownMessage)) { - producer.send(data); - if (logger.isDebugEnabled()) - logger.debug(String.format("Sending message %s", new String(data.message()))); - } else - break; - } - logger.info("Producer thread " + threadName + " finished running"); - } catch (Throwable t) { - logger.fatal("Producer thread failure due to ", t); - } finally { - shutdownComplete.countDown(); - } - } - - public void shutdown() { - try { - logger.info("Producer thread " + threadName + " shutting down"); - producerDataChannel.sendRequest(shutdownMessage); - } catch (InterruptedException ie) { - logger.warn("Interrupt during shutdown of ProducerThread", ie); - } - } - - public void awaitShutdown() { - try { - shutdownComplete.await(); - producer.close(); - logger.info("Producer thread " + threadName + " shutdown complete"); - } catch (InterruptedException ie) { - logger.warn("Interrupt during shutdown of ProducerThread", ie); - } - } - } - - /** - * A parent-last class loader that will try the child class loader first and then the parent. - * This takes a fair bit of doing because java really prefers parent-first. - */ - private static class ParentLastURLClassLoader extends ClassLoader { - private ChildURLClassLoader childClassLoader; - - /** - * This class allows me to call findClass on a class loader - */ - private static class FindClassClassLoader extends ClassLoader { - public FindClassClassLoader(ClassLoader parent) { - super(parent); - } - - @Override - public Class findClass(String name) throws ClassNotFoundException { - return super.findClass(name); - } - } - - /** - * This class delegates (child then parent) for the findClass method for a URLClassLoader. - * We need this because findClass is protected in URLClassLoader - */ - private static class ChildURLClassLoader extends URLClassLoader { - private FindClassClassLoader realParent; - - public ChildURLClassLoader(URL[] urls, FindClassClassLoader realParent) { - super(urls, null); - this.realParent = realParent; - } - - @Override - public Class findClass(String name) throws ClassNotFoundException { - try { - // first try to use the URLClassLoader findClass - return super.findClass(name); - } catch (ClassNotFoundException e) { - // if that fails, we ask our real parent class loader to load the class (we give up) - return realParent.loadClass(name); - } - } - } - - public ParentLastURLClassLoader(URL[] urls) { - super(Thread.currentThread().getContextClassLoader()); - childClassLoader = new ChildURLClassLoader(urls, new FindClassClassLoader(this.getParent())); - } - - @Override - protected synchronized Class loadClass(String name, boolean resolve) throws ClassNotFoundException { - try { - // first we try to find a class inside the child class loader - return childClassLoader.findClass(name); - } catch (ClassNotFoundException e) { - // didn't find it, try the parent - return super.loadClass(name, resolve); - } - } - } -} - diff --git a/docs/upgrade.html b/docs/upgrade.html index ba3d0248718ae..f1e1e40e05e60 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -78,6 +78,7 @@

        potential breaking c
      • MessageFormatter's package was changed from kafka.tools to kafka.common
      • MessageReader's package was changed from kafka.tools to kafka.common
      • MirrorMakerMessageHandler no longer exposes the handle(record: MessageAndMetadata[Array[Byte], Array[Byte]]) method as it was never called.
      • +
      • The 0.7 KafkaMigrationTool is no longer packaged with Kafka. If you need to migrate from 0.7 to 0.10.0, please migrate to 0.8 first and then follow the documented upgrade process to upgrade from 0.8 to 0.10.0.

      Upgrading from 0.8.0, 0.8.1.X or 0.8.2.X to 0.9.0.0

      From d691faf98cb573c4e92748d95d5c8afc492db806 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Fri, 25 Mar 2016 12:51:54 -0700 Subject: [PATCH 088/206] KAFKA-3463: change default receive buffer size for consumer to 64K Author: Jason Gustafson Reviewers: Gwen Shapira Closes #1140 from hachikuji/KAFKA-3463 --- .../org/apache/kafka/clients/consumer/ConsumerConfig.java | 2 +- docs/upgrade.html | 8 +++++++- 2 files changed, 8 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java index c97c8fb470242..69c4a3620925a 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/ConsumerConfig.java @@ -243,7 +243,7 @@ public class ConsumerConfig extends AbstractConfig { CommonClientConfigs.SEND_BUFFER_DOC) .define(RECEIVE_BUFFER_CONFIG, Type.INT, - 32 * 1024, + 64 * 1024, atLeast(0), Importance.MEDIUM, CommonClientConfigs.RECEIVE_BUFFER_DOC) diff --git a/docs/upgrade.html b/docs/upgrade.html index f1e1e40e05e60..060c3deb33dd1 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -64,7 +64,7 @@
      is set to 0.10.0, one should not change it back to an earlier format as it may break consumers on versions before 0.10.0.0.

      -
      potential breaking changes in 0.10.0.0
      +
      Potential breaking changes in 0.10.0.0
      +
      Notable changes in 0.10.0.0
      + +
        +
      • The default value of the configuration parameter receive.buffer.bytes is now 64K for the new consumer
      • +
      +

      Upgrading from 0.8.0, 0.8.1.X or 0.8.2.X to 0.9.0.0

      0.9.0.0 has potential breaking changes (please review before upgrading) and an inter-broker protocol change from previous versions. This means that upgraded brokers and clients may not be compatible with older versions. It is important that you upgrade your Kafka cluster before upgrading your clients. If you are using MirrorMaker downstream clusters should be upgraded first as well. From c1d8c38345e0a1e04ced143ed07e63fe02ceb8b0 Mon Sep 17 00:00:00 2001 From: Andrea Cosentino Date: Fri, 25 Mar 2016 15:00:45 -0700 Subject: [PATCH 089/206] KAFKA-3449: Rename filterOut() to filterNot() to achieve better terminology MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …nology Hi all, This is my first contribution and I hope it will be good. The PR is related to this issue: https://issues.apache.org/jira/browse/KAFKA-3449 Thanks a lot, Andrea Author: Andrea Cosentino Reviewers: Yasuhiro Matsuda, Guozhang Wang Closes #1134 from oscerd/KAFKA-3449 --- .../java/org/apache/kafka/streams/kstream/KStream.java | 2 +- .../java/org/apache/kafka/streams/kstream/KTable.java | 2 +- .../kafka/streams/kstream/internals/KStreamFilter.java | 8 ++++---- .../kafka/streams/kstream/internals/KStreamImpl.java | 2 +- .../kafka/streams/kstream/internals/KTableFilter.java | 8 ++++---- .../kafka/streams/kstream/internals/KTableImpl.java | 2 +- .../streams/kstream/internals/KStreamFilterTest.java | 4 ++-- .../kafka/streams/kstream/internals/KStreamImplTest.java | 2 +- .../kafka/streams/kstream/internals/KTableFilterTest.java | 4 ++-- 9 files changed, 17 insertions(+), 17 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index c4188de44c508..2313b8bf749c9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -43,7 +43,7 @@ public interface KStream { * * @param predicate the instance of {@link Predicate} */ - KStream filterOut(Predicate predicate); + KStream filterNot(Predicate predicate); /** * Create a new instance of {@link KStream} by transforming each element in this stream into a different element in the new stream. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index 9a2a8a8b7e8a7..30ea882563791 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -42,7 +42,7 @@ public interface KTable { * * @param predicate the instance of {@link Predicate} */ - KTable filterOut(Predicate predicate); + KTable filterNot(Predicate predicate); /** * Create a new instance of {@link KTable} by transforming the value of each element in this stream into a new value in the new stream. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java index 0b1f1e05a4a47..f5c2fbc5c369d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamFilter.java @@ -25,11 +25,11 @@ class KStreamFilter implements ProcessorSupplier { private final Predicate predicate; - private final boolean filterOut; + private final boolean filterNot; - public KStreamFilter(Predicate predicate, boolean filterOut) { + public KStreamFilter(Predicate predicate, boolean filterNot) { this.predicate = predicate; - this.filterOut = filterOut; + this.filterNot = filterNot; } @Override @@ -40,7 +40,7 @@ public Processor get() { private class KStreamFilterProcessor extends AbstractProcessor { @Override public void process(K key, V value) { - if (filterOut ^ predicate.test(key, value)) { + if (filterNot ^ predicate.test(key, value)) { context().forward(key, value); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 567b06c61ddc0..5889e078c3306 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -106,7 +106,7 @@ public KStream filter(Predicate predicate) { } @Override - public KStream filterOut(final Predicate predicate) { + public KStream filterNot(final Predicate predicate) { String name = topology.newName(FILTER_NAME); topology.addProcessor(name, new KStreamFilter<>(predicate, true), this.name); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java index 72f1d88e5ce92..080fd9d52a111 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableFilter.java @@ -26,14 +26,14 @@ class KTableFilter implements KTableProcessorSupplier { private final KTableImpl parent; private final Predicate predicate; - private final boolean filterOut; + private final boolean filterNot; private boolean sendOldValues = false; - public KTableFilter(KTableImpl parent, Predicate predicate, boolean filterOut) { + public KTableFilter(KTableImpl parent, Predicate predicate, boolean filterNot) { this.parent = parent; this.predicate = predicate; - this.filterOut = filterOut; + this.filterNot = filterNot; } @Override @@ -64,7 +64,7 @@ public void enableSendingOldValues() { private V computeValue(K key, V value) { V newValue = null; - if (value != null && (filterOut ^ predicate.test(key, value))) + if (value != null && (filterNot ^ predicate.test(key, value))) newValue = value; return newValue; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index ca1e659114658..fd464a08d7e92 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -111,7 +111,7 @@ public KTable filter(Predicate predicate) { } @Override - public KTable filterOut(final Predicate predicate) { + public KTable filterNot(final Predicate predicate) { String name = topology.newName(FILTER_NAME); KTableProcessorSupplier processorSupplier = new KTableFilter<>(this, predicate, true); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java index ecf11153dd01d..75465c85bb8c2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java @@ -60,7 +60,7 @@ public void testFilter() { } @Test - public void testFilterOut() { + public void testFilterNot() { KStreamBuilder builder = new KStreamBuilder(); final int[] expectedKeys = new int[]{1, 2, 3, 4, 5, 6, 7}; @@ -69,7 +69,7 @@ public void testFilterOut() { processor = new MockProcessorSupplier<>(); stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName); - stream.filterOut(isMultipleOfThree).process(processor); + stream.filterNot(isMultipleOfThree).process(processor); KStreamTestDriver driver = new KStreamTestDriver(builder); for (int i = 0; i < expectedKeys.length; i++) { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java index 38182bc355fa7..b5c3d47a80b10 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java @@ -52,7 +52,7 @@ public void testNumProcesses() { public boolean test(String key, String value) { return true; } - }).filterOut(new Predicate() { + }).filterNot(new Predicate() { @Override public boolean test(String key, String value) { return false; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java index 5491ea316e8ca..78d274eb695bd 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java @@ -53,7 +53,7 @@ public boolean test(String key, Integer value) { return (value % 2) == 0; } }); - KTable table3 = table1.filterOut(new Predicate() { + KTable table3 = table1.filterNot(new Predicate() { @Override public boolean test(String key, Integer value) { return (value % 2) == 0; @@ -95,7 +95,7 @@ public boolean test(String key, Integer value) { return (value % 2) == 0; } }); - KTableImpl table3 = (KTableImpl) table1.filterOut( + KTableImpl table3 = (KTableImpl) table1.filterNot( new Predicate() { @Override public boolean test(String key, Integer value) { From 23b50093f4100ce7fbff325cdc92ee6cf3c54102 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 25 Mar 2016 16:04:58 -0700 Subject: [PATCH 090/206] KAFKA-3454: add Kafka Streams web docs Author: Guozhang Wang Reviewers: Gwen Shapira Closes #1127 from guozhangwang/KStreamsDocs --- build.gradle | 123 ++++--- docs/configuration.html | 5 + docs/documentation.html | 15 + docs/quickstart.html | 109 ++++++ docs/streams.html | 341 ++++++++++++++++++ .../examples/pageview/PageViewTypedDemo.java | 3 +- .../pageview/PageViewUntypedDemo.java | 3 +- .../kafka/streams/examples/pipe/PipeDemo.java | 3 +- .../examples/wordcount/WordCountDemo.java | 3 +- .../wordcount/WordCountProcessorDemo.java | 3 +- .../apache/kafka/streams/StreamsConfig.java | 12 +- 11 files changed, 546 insertions(+), 74 deletions(-) create mode 100644 docs/streams.html diff --git a/build.gradle b/build.gradle index c29ad5a8eb83b..13a8b4e5ac18c 100644 --- a/build.gradle +++ b/build.gradle @@ -413,7 +413,7 @@ project(':core') { task siteDocsTar(dependsOn: ['genProtocolErrorDocs', 'genProtocolApiKeyDocs', 'genProtocolMessageDocs', 'genProducerConfigDocs', 'genConsumerConfigDocs', 'genKafkaConfigDocs', - ':connect:runtime:genConnectConfigDocs'], type: Tar) { + ':connect:runtime:genConnectConfigDocs', ':streams:genStreamsConfigDocs'], type: Tar) { classifier = 'site-docs' compression = Compression.GZIP from project.file("../docs") @@ -552,77 +552,84 @@ project(':clients') { } project(':tools') { - archivesBaseName = "kafka-tools" + archivesBaseName = "kafka-tools" - dependencies { - compile project(':clients') - compile project(':log4j-appender') - compile libs.argparse4j - compile libs.jacksonDatabind - compile libs.slf4jlog4j + dependencies { + compile project(':clients') + compile project(':log4j-appender') + compile libs.argparse4j + compile libs.jacksonDatabind + compile libs.slf4jlog4j - testCompile project(':clients') - testCompile libs.junit - } + testCompile project(':clients') + testCompile libs.junit + } - javadoc { - include "**/org/apache/kafka/tools/*" - } + javadoc { + include "**/org/apache/kafka/tools/*" + } - tasks.create(name: "copyDependantLibs", type: Copy) { - from (configurations.testRuntime) { - include('slf4j-log4j12*') - } - from (configurations.runtime) { - exclude('kafka-clients*') - } - into "$buildDir/dependant-libs-${versions.scala}" - duplicatesStrategy 'exclude' + tasks.create(name: "copyDependantLibs", type: Copy) { + from (configurations.testRuntime) { + include('slf4j-log4j12*') } - - jar { - dependsOn 'copyDependantLibs' + from (configurations.runtime) { + exclude('kafka-clients*') } + into "$buildDir/dependant-libs-${versions.scala}" + duplicatesStrategy 'exclude' + } + + jar { + dependsOn 'copyDependantLibs' + } } project(':streams') { - archivesBaseName = "kafka-streams" - - dependencies { - compile project(':clients') - compile project(':connect:json') // this dependency should be removed after we unify data API - compile libs.slf4jlog4j - compile libs.rocksDBJni - compile libs.zkclient // this dependency should be removed after KIP-4 - compile libs.jacksonDatabind // this dependency should be removed after KIP-4 - - testCompile project(':clients').sourceSets.test.output - testCompile libs.junit - } + archivesBaseName = "kafka-streams" - javadoc { - include "**/org/apache/kafka/streams/**" - exclude "**/internals/**" - } + dependencies { + compile project(':clients') + compile project(':connect:json') // this dependency should be removed after we unify data API + compile libs.slf4jlog4j + compile libs.rocksDBJni + compile libs.zkclient // this dependency should be removed after KIP-4 + compile libs.jacksonDatabind // this dependency should be removed after KIP-4 - tasks.create(name: "copyDependantLibs", type: Copy) { - from (configurations.testRuntime) { - include('slf4j-log4j12*') - } - from (configurations.runtime) { - exclude('kafka-clients*') - } - into "$buildDir/dependant-libs-${versions.scala}" - duplicatesStrategy 'exclude' - } + testCompile project(':clients').sourceSets.test.output + testCompile libs.junit + } - jar { - dependsOn 'copyDependantLibs' - } + javadoc { + include "**/org/apache/kafka/streams/**" + exclude "**/internals/**" + } - systemTestLibs { - dependsOn testJar + tasks.create(name: "copyDependantLibs", type: Copy) { + from (configurations.testRuntime) { + include('slf4j-log4j12*') } + from (configurations.runtime) { + exclude('kafka-clients*') + } + into "$buildDir/dependant-libs-${versions.scala}" + duplicatesStrategy 'exclude' + } + + jar { + dependsOn 'copyDependantLibs' + } + + systemTestLibs { + dependsOn testJar + } + + task genStreamsConfigDocs(type: JavaExec) { + classpath = sourceSets.main.runtimeClasspath + main = 'org.apache.kafka.streams.StreamsConfig' + if( !generatedDocsDir.exists() ) { generatedDocsDir.mkdirs() } + standardOutput = new File(generatedDocsDir, "streams_config.html").newOutputStream() + } } project(':streams:examples') { diff --git a/docs/configuration.html b/docs/configuration.html index a89778d3270fa..e5280a5744c9a 100644 --- a/docs/configuration.html +++ b/docs/configuration.html @@ -333,4 +333,9 @@

      3.3.2 New Consumer Con

      3.4 Kafka Connect Configs

      +Below is the configuration of the Kafka Connect framework. + +

      3.5 Kafka Streams Configs

      +Below is the configuration of the Kafka Streams client library. + diff --git a/docs/documentation.html b/docs/documentation.html index 67a2954acbf11..70002ab8ec4e0 100644 --- a/docs/documentation.html +++ b/docs/documentation.html @@ -52,6 +52,7 @@

      Kafka 0.10.0 Documentation

    4. 3.3.2 New Consumer Configs
    5. 3.4 Kafka Connect Configs +
    6. 3.5 Kafka Streams Configs
    7. 4. Design @@ -136,6 +137,17 @@

      Kafka 0.10.0 Documentation

    8. 8.3 Connector Development Guide
    9. +
    10. 9. Kafka Streams + +
    11. 1. Getting Started

      @@ -171,4 +183,7 @@

      7. Security

      8. Kafka Connect

      +

      9. Kafka Streams

      + + diff --git a/docs/quickstart.html b/docs/quickstart.html index 1e7b62ce3c0a8..7a923c69fc008 100644 --- a/docs/quickstart.html +++ b/docs/quickstart.html @@ -249,3 +249,112 @@

      Step 7: Use You should see the line appear in the console consumer output and in the sink file. + +

      Step 8: Use Kafka Streams to process data

      + +

      +Kafka Streams is a client library of Kafka for real-time stream processing and analyzing data stored in Kafka brokers. +This quickstart example will demonstrate how to run a streaming application coded in this library. Here is the gist +of the WordCountDemo example code (converted to use Java 8 lambda expressions for easy reading). +

      +
      +KStream wordCounts = textLines
      +// Split each text line, by whitespace, into words.
      +.flatMapValues(value -> Arrays.asList(value.toLowerCase().split("\\W+")))
      +// Ensure the words are available as message keys for the next aggregate operation.
      +.map((key, value) -> new KeyValue<>(value, value))
      +// Count the occurrences of each word (message key).
      +.countByKey(stringSerializer, longSerializer, stringDeserializer, longDeserializer, "Counts")
      +// Convert the resulted aggregate table into another stream.
      +.toStream();
      +
      + +

      +It implements the WordCount +algorithm, which computes a word occurrence histogram from the input text. However, unlike other WordCount examples +you might have seen before that operate on bounded data, the WordCount demo application behaves slightly differently because it is +designed to operate on an infinite, unbounded stream of data. Similar to the bounded variant, it is a stateful algorithm that +tracks and updates the counts of words. However, since it must assume potentially +unbounded input data, it will periodically output its current state and results while continuing to process more data +because it cannot know when it has processed "all" the input data. +

      +

      +We will now prepare input data to a Kafka topic, which will subsequently processed by a Kafka Streams application. +

      + + + +
      +> echo -e "all streams lead to kafka\nhello kafka streams\njoin kafka summit" > file-input.txt
      +
      + +

      +Next, we send this input data to the input topic named streams-file-input using the console producer (in practice, +stream data will likely be flowing continuously into Kafka where the application will be up and running): +

      + +
      +> cat /tmp/file-input.txt | ./bin/kafka-console-producer --broker-list localhost:9092 --topic streams-file-input
      +
      + +

      +We can now run the WordCount demo application to process the input data: +

      + +
      +> ./bin/kafka-run-class org.apache.kafka.streams.examples.wordcount.WordCountDemo
      +
      + +

      +There won't be any STDOUT output except log entries as the results are continuously written back into another topic named streams-wordcount-output in Kafka. +The demo will run for a few seconds and then, unlike typical stream processing applications, terminate automatically. +

      +

      +We can now inspect the output of the WordCount demo application by reading from its output topic: +

      + +
      +> ./bin/kafka-console-consumer --zookeeper localhost:2181 \
      +            --topic streams-wordcount-output \
      +            --from-beginning \
      +            --formatter kafka.tools.DefaultMessageFormatter \
      +            --property print.key=true \
      +            --property print.key=true \
      +            --property key.deserializer=org.apache.kafka.common.serialization.StringDeserializer \
      +            --property value.deserializer=org.apache.kafka.common.serialization.LongDeserializer
      +
      + +

      +with the following output data being printed to the console (You can stop the console consumer via Ctrl-C): +

      + +
      +all     1
      +streams 1
      +lead    1
      +to      1
      +kafka   1
      +hello   1
      +kafka   2
      +streams 2
      +join    1
      +kafka   3
      +summit  1
      +^C
      +
      + +

      +Here, the first column is the Kafka message key, and the second column is the message value, both in in java.lang.String format. +Note that the output is actually a continuous stream of updates, where each data record (i.e. each line in the original output above) is +an updated count of a single word, aka record key such as "kafka". For multiple records with the same key, each later record is an update of the previous one. +

      \ No newline at end of file diff --git a/docs/streams.html b/docs/streams.html new file mode 100644 index 0000000000000..9b94bb32c06df --- /dev/null +++ b/docs/streams.html @@ -0,0 +1,341 @@ + + +

      9.1 Overview

      + +

      +Kafka Streams is a client library for processing and analyzing data stored in Kafka and either write the resulting data back to Kafka or send the final output to an external system. It builds upon important stream processing concepts such as properly distinguishing between event time and processing time, windowing support, and simple yet efficient management of application state. +Kafka Streams has a low barrier to entry: You can quickly write and run a small-scale proof-of-concept on a single machine; and you only need to run additional instances of your application on multiple machines to scale up to high-volume production workloads. Kafka Streams transparently handles the load balancing of multiple instances of the same application by leveraging Kafka's parallelism model. +

      +

      +Some highlights of Kafka Streams: +

      + +
        +
      • Designed as a simple and lightweight client library, which can be easily embedded in any Java application and integrated with any existing packaging, deployment and operational tools that users have for their streaming applications.
      • +
      • Has no external dependencies on systems other than Apache Kafka itself as the internal messaging layer; notably, it uses Kafka's partitioning model to horizontally scale processing while maintaining strong ordering guarantees.
      • +
      • Supports fault-tolerant local state, which enables very fast and efficient stateful operations like joins and windowed aggregations.
      • +
      • Employs one-record-at-a-time processing to achieve low processing latency, and supports event-time based windowing operations.
      • +
      • Offers necessary stream processing primitives, along with a high-level Streams DSL and a low-level Processor API.
      • + +
      + +

      9.2 Developer Guide

      + +

      +There is a quickstart example that provides how to run a stream processing program coded in the Kafka Streams library. +This section focuses on how to write, configure, and execute a Kafka Streams application. +

      + +

      Core Concepts

      + +

      +We first summarize the key concepts of Kafka Streams. +

      + +
      Stream Processing Topology
      + +
        +
      • A streamis the most important abstraction provided by Kafka Streams: it represents an unbounded, continuously updating data set. A stream is an ordered, replayable, and fault-tolerant sequence of immutable data records, where a data record is defined as a key-value pair.
      • +
      • A stream processing application written in Kafka Streams defines its computational logic through one or more processor topologies, where a processor topology is a graph of stream processors (nodes) that are connected by streams (edges).
      • +
      • A stream processor is a node in the processor topology; it represents a processing step to transform data in streams by receiving one input record at a time from its upstream processors in the topology, applying its operation to it, and may subsequently producing one or more output records to its downstream processors.
      • +
      + +

      +Kafka Streams offers two ways to define the stream processing topology: the Kafka Streams DSL provides +the most common data transformation operations such as map and filter; the lower-level Processor API allows +developers define and connect custom processors as well as to interact with state stores. +

      + +
      Time
      + +

      +A critical aspect in stream processing is the the notion of time, and how it is modeled and integrated. +For example, some operations such as windowing are defined based on time boundaries. +

      +

      +Common notions of time in streams are: +

      + +
        +
      • Event time - The point in time when an event or data record occurred, i.e. was originally created "at the source".
      • +
      • Processing time - The point in time when the event or data record happens to be processed by the stream processing application, i.e. when the record is being consumed. The processing time may be milliseconds, hours, or days etc. later than the original event time.
      • +
      + +

      +Kafka Streams assigns a timestamp to every data record +via the TimestampExtractor interface. +Concrete implementations of this interface may retrieve or compute timestamps based on the actual contents of data records such as an embedded timestamp field +to provide event-time semantics, or use any other approach such as returning the current wall-clock time at the time of processing, +thereby yielding processing-time semantics to stream processing applications. +Developers can thus enforce different notions of time depending on their business needs. For example, +per-record timestamps describe the progress of a stream with regards to time (although records may be out-of-order within the stream) and +are leveraged by time-dependent operations such as joins. +

      + +
      States
      + +

      +Some stream processing applications don't require state, which means the processing of a message is independent from +the processing of all other messages. +However, being able to maintain state opens up many possibilities for sophisticated stream processing applications: you +can join input streams, or group and aggregate data records. Many such stateful operators are provided by the Kafka Streams DSL. +

      +

      +Kafka Streams provides so-called state stores, which can be used by stream processing applications to store and query data. +This is an important capability when implementing stateful operations. +Every task in Kafka Streams embeds one or more state stores that can be accessed via APIs to store and query data required for processing. +These state stores can either be a persistent key-value store, an in-memory hashmap, or another convenient data structure. +Kafka Streams offers fault-tolerance and automatic recovery for local state stores. +

      +
      +

      +As we have mentioned above, the computational logic of a Kafka Streams application is defined as a processor topology. +Currently Kafka Streams provides two sets of APIs to define the processor topology, which will be described in the subsequent sections. +

      + +

      Low-Level Processor API

      + +
      Processor
      + +

      +Developers can define their customized processing logic by implementing the Processor interface, which +provides process and punctuate methods. The process method is performed on each +of the received record; and the punctuate method is performed periodically based on elapsed time. +In addition, the processor can maintain the current ProcessorContext instance variable initialized in the +init method, and use the context to schedule the punctuation period (context().schedule), to +forward the modified / new key-value pair to downstream processors (context().forward), to commit the current +processing progress (context().commit), etc. +

      + +
      +    public class MyProcessor extends Processor {
      +        private ProcessorContext context;
      +        private KeyValueStore kvStore;
      +
      +        @Override
      +        @SuppressWarnings("unchecked")
      +        public void init(ProcessorContext context) {
      +            this.context = context;
      +            this.context.schedule(1000);
      +            this.kvStore = (KeyValueStore) context.getStateStore("Counts");
      +        }
      +
      +        @Override
      +        public void process(String dummy, String line) {
      +            String[] words = line.toLowerCase().split(" ");
      +
      +            for (String word : words) {
      +                Integer oldValue = this.kvStore.get(word);
      +
      +                if (oldValue == null) {
      +                    this.kvStore.put(word, 1);
      +                } else {
      +                    this.kvStore.put(word, oldValue + 1);
      +                }
      +            }
      +        }
      +
      +        @Override
      +        public void punctuate(long timestamp) {
      +            KeyValueIterator iter = this.kvStore.all();
      +
      +            while (iter.hasNext()) {
      +                KeyValue entry = iter.next();
      +                context.forward(entry.key, entry.value.toString());
      +            }
      +
      +            iter.close();
      +            context.commit();
      +        }
      +
      +        @Override
      +        public void close() {
      +            this.kvStore.close();
      +        }
      +    };
      +
      + +

      +In the above implementation, the following actions are performed: + +

        +
      • In the init method, schedule the punctuation every 1 second and retrieve the local state store by its name "Counts".
      • +
      • In the process method, upon each received record, split the value string into words, and update their counts into the state store (we will talk about this feature later in the section).
      • +
      • In the punctuate method, iterate the local state store and send the aggregated counts to the downstream processor, and commit the current stream state.
      • +
      +

      + +
      Processor Topology
      + +

      +With the customized processors defined in the Processor API, developers can use the TopologyBuilder to build a processor topology +by connecting these processors together: + +

      +    TopologyBuilder builder = new TopologyBuilder();
      +
      +    builder.addSource("SOURCE", "src-topic")
      +
      +        .addProcessor("PROCESS1", MyProcessor1::new /* the ProcessorSupplier that can generate MyProcessor1 */, "SOURCE")
      +        .addProcessor("PROCESS2", MyProcessor2::new /* the ProcessorSupplier that can generate MyProcessor2 */, "PROCESS1")
      +        .addProcessor("PROCESS3", MyProcessor3::new /* the ProcessorSupplier that can generate MyProcessor3 */, "PROCESS1")
      +
      +        .addSink("SINK1", "sink-topic1", "PROCESS1")
      +        .addSink("SINK2", "sink-topic2", "PROCESS2")
      +        .addSink("SINK3", "sink-topic3", "PROCESS3");
      +
      + +There are several steps in the above code to build the topology, and here is a quick walk through: + +
        +
      • First of all a source node named "SOURCE" is added to the topology using the addSource method, with one Kafka topic "src-topic" fed to it.
      • +
      • Three processor nodes are then added using the addProcessor method; here the first processor is a child of the "SOURCE" node, but is the parent of the other two processors.
      • +
      • Finally three sink nodes are added to complete the topology using the addSink method, each piping from a different parent processor node and writing to a separate topic.
      • +
      +

      + +
      Local State Store
      + +

      +Note that the Processor API is not limited to only accessing the current records as they arrive, but can also maintain local state stores +that keep recently arrived records to use in stateful processing operations such as aggregation or windowed joins. +To take advantage of this local states, developers can use the TopologyBuilder.addStateStore method when building the +processor topology to create the local state and associate it with the processor nodes that needs to access it; or they can connect a created +local state store with the existing processor nodes through TopologyBuilder.connectProcessorAndStateStores. + +

      +    TopologyBuilder builder = new TopologyBuilder();
      +
      +    builder.addSource("SOURCE", "src-topic")
      +
      +        .addProcessor("PROCESS1", MyProcessor1::new, "SOURCE")
      +        // create the in-memory state store "COUNTS" associated with processor "PROCESS1"
      +        .addStateStore(Stores.create("COUNTS").withStringKeys().withStringValues().inMemory().build(), "PROCESS1")
      +        .addProcessor("PROCESS2", MyProcessor3::new /* the ProcessorSupplier that can generate MyProcessor3 */, "PROCESS1")
      +        .addProcessor("PROCESS3", MyProcessor3::new /* the ProcessorSupplier that can generate MyProcessor3 */, "PROCESS1")
      +
      +        // connect the state store "COUNTS" with processor "PROCESS2"
      +        .connectProcessorAndStateStores("PROCESS2", "COUNTS");
      +
      +        .addSink("SINK1", "sink-topic1", "PROCESS1")
      +        .addSink("SINK2", "sink-topic2", "PROCESS2")
      +        .addSink("SINK3", "sink-topic3", "PROCESS3");
      +
      + +

      + +In the next section we present another way to build the processor topology: the Kafka Streams DSL. + +

      High-Level Streams DSL

      + +To build a processor topology using the Streams DSL, developers can apply the KStreamBuilder class, which is extended from the TopologyBuilder. +A simple example is included with the source code for Kafka in the streams/examples package. The rest of this section will walk +through some code to demonstrate the key steps in creating a topology using the Streams DSL, but we recommend developers to read the full example source +codes for details. + +
      Create Source Streams from Kafka
      + +

      +Either a record stream (defined as KStream) or a changelog stream (defined as KTable) +can be created as a source stream from one or more Kafka topics (for KTable you can only create the source stream +from a single topic). +

      + +
      +    KStreamBuilder builder = new KStreamBuilder();
      +
      +    KStream source1 = builder.stream("topic1", "topic2");
      +    KTable source2 = builder.table("topic3");
      +
      + +
      Transform a stream
      + +

      +There is a list of transformation operations provided for KStream and KTable respectively. +Each of these operations may generate either one or more KStream and KTable objects and +can be translated into one or more connected processors into the underlying processor topology. +All these transformation methods can be chained together to compose a complex processor topology. +Since KStream and KTable are strongly typed, all these transformation operations are defined as +generics functions where users could specify the input and output data types. +

      + +

      +Among these transformations, filter, map, mapValues, etc, are stateless +transformation operations and can be applied to both KStream and KTable, +where users can usually pass a customized function to these functions as a parameter, such as Predicate for filter, +KeyValueMapper for map, etc: + +

      + +
      +    // written in Java 8+, using lambda expressions
      +    KStream mapped = source1.mapValue(record -> record.get("category"));
      +
      + +

      +Stateless transformations, by definition, do not depend on any state for processing, and hence implementation-wise +they do not require a state store associated with the stream processor; Stateful transformations, on the other hand, +require accessing an associated state for processing and producing outputs. +For example, in join and aggregate operations, a windowing state is usually used to store all the received records +within the defined window boundary so far. The operators can then access these accumulated records in the store and compute +based on them. +

      + +
      +    // written in Java 8+, using lambda expressions
      +    KTable, Long> counts = source1.aggregateByKey(
      +        () -> 0L,  // initial value
      +        (aggKey, value, aggregate) -> aggregate + 1L,   // aggregating value
      +        HoppingWindows.of("counts").with(5000L).every(1000L), // intervals in milliseconds
      +    );
      +
      +    KStream joined = source1.leftJoin(source2,
      +        (record1, record2) -> record1.get("user") + "-" + record2.get("region");
      +    );
      +
      + +
      Write streams back to Kafka
      + +

      +At the end of the processing, users can choose to (continuously) write the final resulted streams back to a Kafka topic through +KStream.to and KTable.to. +

      + +
      +    joined.to("topic4");
      +
      + +If your application needs to continue reading and processing the records after they have been materialized +to a topic via to above, one option is to construct a new stream that reads from the output topic; +Kafka Streams provides a convenience method called through: + +
      +    // equivalent to
      +    //
      +    // joined.to("topic4");
      +    // materialized = builder.stream("topic4");
      +    KStream materialized = joined.through("topic4");
      +
      + + +
      +

      +Besides defining the topology, developers will also need to configure their applications +in StreamsConfig before running it. A complete list of +Kafka Streams configs can be found here. +

      \ No newline at end of file diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java index 0385bdeaceb1f..4124b32c301d8 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.examples.pageview; +import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; @@ -86,7 +87,7 @@ public static void main(String[] args) throws Exception { props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, JsonTimestampExtractor.class); // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data - props.put(StreamsConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); KStreamBuilder builder = new KStreamBuilder(); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java index 6f5cdf29701c3..e61842ffe3273 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java @@ -20,6 +20,7 @@ import com.fasterxml.jackson.databind.JsonNode; import com.fasterxml.jackson.databind.node.JsonNodeFactory; import com.fasterxml.jackson.databind.node.ObjectNode; +import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; @@ -62,7 +63,7 @@ public static void main(String[] args) throws Exception { props.put(StreamsConfig.TIMESTAMP_EXTRACTOR_CLASS_CONFIG, JsonTimestampExtractor.class); // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data - props.put(StreamsConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); KStreamBuilder builder = new KStreamBuilder(); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java index 619f33ddd931e..3c1bd8c049a67 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pipe/PipeDemo.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.examples.pipe; +import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.KafkaStreams; @@ -44,7 +45,7 @@ public static void main(String[] args) throws Exception { props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data - props.put(StreamsConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); KStreamBuilder builder = new KStreamBuilder(); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java index e892abb0798c3..c12977f8d43f1 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.examples.wordcount; +import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; @@ -52,7 +53,7 @@ public static void main(String[] args) throws Exception { props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data - props.put(StreamsConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); KStreamBuilder builder = new KStreamBuilder(); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java index 8457415f11181..a5cddfd005e72 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.examples.wordcount; +import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; @@ -111,7 +112,7 @@ public static void main(String[] args) throws Exception { props.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass()); // setting offset reset to earliest so that we can re-run the demo code with the same pre-loaded data - props.put(StreamsConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); TopologyBuilder builder = new TopologyBuilder(); diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index d4efbee89105f..3e0f9550a5465 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -34,7 +34,6 @@ import java.util.Map; import static org.apache.kafka.common.config.ConfigDef.Range.atLeast; -import static org.apache.kafka.common.config.ConfigDef.ValidString.in; /** * Configuration for Kafka Streams. Documentation for these configurations can be found in the client.id */ public static final String CLIENT_ID_CONFIG = CommonClientConfigs.CLIENT_ID_CONFIG; - /** auto.offset.reset */ - public static final String AUTO_OFFSET_RESET_CONFIG = ConsumerConfig.AUTO_OFFSET_RESET_CONFIG; - static { CONFIG = new ConfigDef().define(APPLICATION_ID_CONFIG, // required with no default value Type.STRING, @@ -197,12 +193,6 @@ public class StreamsConfig extends AbstractConfig { 60000, Importance.LOW, STATE_CLEANUP_DELAY_MS_DOC) - .define(AUTO_OFFSET_RESET_CONFIG, - Type.STRING, - "latest", - in("latest", "earliest", "none"), - Importance.MEDIUM, - ConsumerConfig.AUTO_OFFSET_RESET_DOC) .define(METRIC_REPORTER_CLASSES_CONFIG, Type.LIST, "", @@ -277,7 +267,7 @@ public Map getProducerConfigs(String clientId) { Map props = this.originals(); // remove consumer properties that are not required for producers - props.remove(StreamsConfig.AUTO_OFFSET_RESET_CONFIG); + props.remove(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG); // remove streams properties removeStreamsSpecificConfigs(props); From 78fa20eb58a948abd9ad4e44acfed606400a47f3 Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Fri, 25 Mar 2016 16:46:53 -0700 Subject: [PATCH 091/206] KAFKA-3316: Add REST API for listing connector plugins Author: Liquan Pei Reviewers: Ewen Cheslack-Postava Closes #1090 from Ishiihara/kafka-3316 --- .../kafka/connect/runtime/AbstractHerder.java | 32 ++++++++++- .../rest/entities/ConnectorPluginInfo.java | 54 +++++++++++++++++++ .../resources/ConnectorPluginsResource.java | 10 ++++ .../connect/runtime/AbstractHerderTest.java | 1 - .../ConnectorPluginsResourceTest.java | 22 +++++++- 5 files changed, 116 insertions(+), 3 deletions(-) create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorPluginInfo.java diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java index 8d83644d6fbc6..a97c4db8faff7 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java @@ -26,20 +26,29 @@ import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos; import org.apache.kafka.connect.runtime.rest.entities.ConfigKeyInfo; import org.apache.kafka.connect.runtime.rest.entities.ConfigValueInfo; +import org.apache.kafka.connect.runtime.rest.entities.ConnectorPluginInfo; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; import org.apache.kafka.connect.storage.StatusBackingStore; +import org.apache.kafka.connect.tools.VerifiableSinkConnector; +import org.apache.kafka.connect.tools.VerifiableSourceConnector; import org.apache.kafka.connect.util.ConnectorTaskId; +import org.reflections.Reflections; +import org.reflections.util.ClasspathHelper; +import org.reflections.util.ConfigurationBuilder; import java.io.ByteArrayOutputStream; import java.io.PrintStream; import java.io.UnsupportedEncodingException; +import java.lang.reflect.Modifier; import java.util.ArrayList; +import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.concurrent.ConcurrentHashMap; /** @@ -69,7 +78,9 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con protected final StatusBackingStore statusBackingStore; private final String workerId; - protected Map tempConnectors = new ConcurrentHashMap<>(); + private Map tempConnectors = new ConcurrentHashMap<>(); + private static final List> SKIPPED_CONNECTORS = Arrays.>asList(VerifiableSourceConnector.class, VerifiableSinkConnector.class); + private static List validConnectorPlugins; public AbstractHerder(Worker worker, StatusBackingStore statusBackingStore, String workerId) { this.worker = worker; @@ -189,6 +200,25 @@ public ConfigInfos validateConfigs(String connType, Map connecto return generateResult(connType, resultConfigKeys, configValues, allGroups); } + public static List connectorPlugins() { + if (validConnectorPlugins != null) { + return validConnectorPlugins; + } + + Reflections reflections = new Reflections(new ConfigurationBuilder().setUrls(ClasspathHelper.forJavaClassPath())); + Set> connectorClasses = reflections.getSubTypesOf(Connector.class); + connectorClasses.removeAll(SKIPPED_CONNECTORS); + List connectorPlugins = new LinkedList<>(); + for (Class connectorClass: connectorClasses) { + int mod = connectorClass.getModifiers(); + if (!Modifier.isAbstract(mod) && !Modifier.isInterface(mod)) { + connectorPlugins.add(new ConnectorPluginInfo(connectorClass.getCanonicalName())); + } + } + validConnectorPlugins = connectorPlugins; + return connectorPlugins; + } + // public for testing public static ConfigInfos generateResult(String connType, Map configKeys, List configValues, List groups) { int errorCount = 0; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorPluginInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorPluginInfo.java new file mode 100644 index 0000000000000..097142e945f00 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConnectorPluginInfo.java @@ -0,0 +1,54 @@ +/** + * 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 org.apache.kafka.connect.runtime.rest.entities; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import java.util.Objects; + +public class ConnectorPluginInfo { + + private String clazz; + + @JsonCreator + public ConnectorPluginInfo(@JsonProperty("class") String clazz) { + this.clazz = clazz; + } + + @JsonProperty("class") + public String clazz() { + return clazz; + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ConnectorPluginInfo that = (ConnectorPluginInfo) o; + return Objects.equals(clazz, that.clazz); + } + + @Override + public int hashCode() { + return Objects.hash(clazz); + } + + @Override + public String toString() { + return clazz; + } +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java index 84397078033a0..9e87d0c4aa2e0 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResource.java @@ -17,12 +17,16 @@ package org.apache.kafka.connect.runtime.rest.resources; +import org.apache.kafka.connect.runtime.AbstractHerder; import org.apache.kafka.connect.runtime.Herder; import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos; +import org.apache.kafka.connect.runtime.rest.entities.ConnectorPluginInfo; +import java.util.List; import java.util.Map; import javax.ws.rs.Consumes; +import javax.ws.rs.GET; import javax.ws.rs.PUT; import javax.ws.rs.Path; import javax.ws.rs.PathParam; @@ -46,4 +50,10 @@ public ConfigInfos validateConfigs(final @PathParam("connectorType") String conn final Map connectorConfig) throws Throwable { return herder.validateConfigs(connType, connectorConfig); } + + @GET + @Path("/") + public List listConnectorPlugins() { + return AbstractHerder.connectorPlugins(); + } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java index 1dc57846213f5..e4084a809ae55 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java @@ -114,5 +114,4 @@ public TaskStatus answer() throws Throwable { verifyAll(); } - } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java index 625c91fa0edd7..1049e7e0dd78f 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java @@ -21,8 +21,8 @@ import org.apache.kafka.common.config.Config; import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.connect.connector.Connector; import org.apache.kafka.connect.connector.Task; import org.apache.kafka.connect.runtime.AbstractHerder; @@ -32,6 +32,11 @@ import org.apache.kafka.connect.runtime.rest.entities.ConfigInfos; import org.apache.kafka.connect.runtime.rest.entities.ConfigKeyInfo; import org.apache.kafka.connect.runtime.rest.entities.ConfigValueInfo; +import org.apache.kafka.connect.runtime.rest.entities.ConnectorPluginInfo; +import org.apache.kafka.connect.sink.SinkConnector; +import org.apache.kafka.connect.source.SourceConnector; +import org.apache.kafka.connect.tools.VerifiableSinkConnector; +import org.apache.kafka.connect.tools.VerifiableSourceConnector; import org.easymock.EasyMock; import org.easymock.IAnswer; import org.junit.Before; @@ -49,8 +54,11 @@ import java.util.LinkedList; import java.util.List; import java.util.Map; +import java.util.Set; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; @RunWith(PowerMockRunner.class) @PrepareForTest(RestServer.class) @@ -64,6 +72,7 @@ public class ConnectorPluginsResourceTest { } private static final ConfigInfos CONFIG_INFOS; + static { List configs = new LinkedList<>(); @@ -120,6 +129,17 @@ public ConfigInfos answer() { PowerMock.verifyAll(); } + @Test + public void testListConnectorPlugins() { + Set connectorPlugins = new HashSet<>(connectorPluginsResource.listConnectorPlugins()); + assertFalse(connectorPlugins.contains(new ConnectorPluginInfo(Connector.class.getCanonicalName()))); + assertFalse(connectorPlugins.contains(new ConnectorPluginInfo(SourceConnector.class.getCanonicalName()))); + assertFalse(connectorPlugins.contains(new ConnectorPluginInfo(SinkConnector.class.getCanonicalName()))); + assertFalse(connectorPlugins.contains(new ConnectorPluginInfo(VerifiableSourceConnector.class.getCanonicalName()))); + assertFalse(connectorPlugins.contains(new ConnectorPluginInfo(VerifiableSinkConnector.class.getCanonicalName()))); + assertTrue(connectorPlugins.contains(new ConnectorPluginInfo(ConnectorPluginsResourceTestConnector.class.getCanonicalName()))); + } + /* Name here needs to be unique as we are testing the aliasing mechanism */ public static class ConnectorPluginsResourceTestConnector extends Connector { From 1fbe445dde71df0023a978c5e54dd229d3d23e1b Mon Sep 17 00:00:00 2001 From: Jiangjie Qin Date: Sat, 26 Mar 2016 09:22:59 -0700 Subject: [PATCH 092/206] KAFKA-3388; Fix expiration of batches sitting in the accumulator Author: Jiangjie Qin Reviewers: Ismael Juma , Jun Rao Closes #1056 from becketqin/KAFKA-3388 --- .../producer/internals/RecordAccumulator.java | 35 ++++++---- .../producer/internals/RecordBatch.java | 19 ++++-- .../internals/RecordAccumulatorTest.java | 67 ++++++++++++++++--- 3 files changed, 94 insertions(+), 27 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index beaa832d3889e..915c4d3e9d930 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -13,6 +13,7 @@ package org.apache.kafka.clients.producer.internals; import java.util.Iterator; + import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.common.Cluster; import org.apache.kafka.common.MetricName; @@ -217,19 +218,27 @@ public List abortExpiredBatches(int requestTimeout, Cluster cluster int count = 0; for (Map.Entry> entry : this.batches.entrySet()) { Deque dq = entry.getValue(); - synchronized (dq) { - // iterate over the batches and expire them if they have stayed in accumulator for more than requestTimeOut - Iterator batchIterator = dq.iterator(); - while (batchIterator.hasNext()) { - RecordBatch batch = batchIterator.next(); - // check if the batch is expired - if (batch.maybeExpire(requestTimeout, now, this.lingerMs)) { - expiredBatches.add(batch); - count++; - batchIterator.remove(); - deallocate(batch); - } else { - if (!batch.inRetry()) { + // We only check if the batch should be expired if the partition does not have a batch in flight. + // This is to avoid the later batches get expired when an earlier batch is still in progress. + // This protection only takes effect when user sets max.in.flight.request.per.connection=1. + // Otherwise the expiration order is not guranteed. + TopicPartition tp = entry.getKey(); + if (!muted.contains(tp)) { + synchronized (dq) { + // iterate over the batches and expire them if they have stayed in accumulator for more than requestTimeOut + RecordBatch lastBatch = dq.peekLast(); + Iterator batchIterator = dq.iterator(); + while (batchIterator.hasNext()) { + RecordBatch batch = batchIterator.next(); + boolean isFull = batch != lastBatch || batch.records.isFull(); + // check if the batch is expired + if (batch.maybeExpire(requestTimeout, retryBackoffMs, now, this.lingerMs, isFull)) { + expiredBatches.add(batch); + count++; + batchIterator.remove(); + deallocate(batch); + } else { + // Stop at the first batch that has not expired. break; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java index eb7bbb3cefbaa..e6cd68fd065e7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordBatch.java @@ -134,14 +134,23 @@ public String toString() { } /** - * Expire the batch that is ready but is sitting in accumulator for more than requestTimeout due to metadata being unavailable. - * We need to explicitly check if the record is full or linger time is met because the accumulator's partition may not be ready - * if the leader is unavailable. + * A batch whose metadata is not available should be expired if one of the following is true: + *

        + *
      1. the batch is not in retry AND request timeout has elapsed after it is ready (full or linger.ms has reached). + *
      2. the batch is in retry AND request timeout has elapsed after the backoff period ended. + *
      */ - public boolean maybeExpire(int requestTimeout, long now, long lingerMs) { + public boolean maybeExpire(int requestTimeoutMs, long retryBackoffMs, long now, long lingerMs, boolean isFull) { boolean expire = false; - if ((this.records.isFull() && requestTimeout < (now - this.lastAppendTime)) || requestTimeout < (now - (this.lastAttemptMs + lingerMs))) { + + if (!this.inRetry() && isFull && requestTimeoutMs < (now - this.lastAppendTime)) + expire = true; + else if (!this.inRetry() && requestTimeoutMs < (now - (this.createdMs + lingerMs))) expire = true; + else if (this.inRetry() && requestTimeoutMs < (now - (this.lastAttemptMs + retryBackoffMs))) + expire = true; + + if (expire) { this.records.close(); this.done(-1L, Record.NO_TIMESTAMP, new TimeoutException("Batch containing " + recordCount + " record(s) expired due to timeout while requesting metadata from brokers for " + topicPartition)); } diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java index 366027286181d..904aa73bd33cd 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java @@ -297,22 +297,71 @@ public void onCompletion(RecordMetadata metadata, Exception exception) { @Test public void testExpiredBatches() throws InterruptedException { - long now = time.milliseconds(); - RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, 10, 100L, metrics, time); + long retryBackoffMs = 100L; + long lingerMs = 3000L; + int requestTimeout = 60; + + RecordAccumulator accum = new RecordAccumulator(1024, 10 * 1024, CompressionType.NONE, lingerMs, retryBackoffMs, metrics, time); int appends = 1024 / msgSize; + + // Test batches not in retry for (int i = 0; i < appends; i++) { accum.append(tp1, 0L, key, value, null, maxBlockTimeMs); - assertEquals("No partitions should be ready.", 0, accum.ready(cluster, now).readyNodes.size()); + assertEquals("No partitions should be ready.", 0, accum.ready(cluster, time.milliseconds()).readyNodes.size()); } - time.sleep(2000); - accum.ready(cluster, now); + // Make the batches ready due to batch full accum.append(tp1, 0L, key, value, null, 0); Set readyNodes = accum.ready(cluster, time.milliseconds()).readyNodes; assertEquals("Our partition's leader should be ready", Collections.singleton(node1), readyNodes); - Cluster cluster = new Cluster(new ArrayList(), new ArrayList(), Collections.emptySet()); - now = time.milliseconds(); - List expiredBatches = accum.abortExpiredBatches(60, cluster, now); - assertEquals(1, expiredBatches.size()); + // Advance the clock to expire the batch. + time.sleep(requestTimeout + 1); + accum.mutePartition(tp1); + List expiredBatches = accum.abortExpiredBatches(requestTimeout, cluster, time.milliseconds()); + assertEquals("The batch should not be expired when the partition is muted", 0, expiredBatches.size()); + + accum.unmutePartition(tp1); + expiredBatches = accum.abortExpiredBatches(requestTimeout, cluster, time.milliseconds()); + assertEquals("The batch should be expired", 1, expiredBatches.size()); + assertEquals("No partitions should be ready.", 0, accum.ready(cluster, time.milliseconds()).readyNodes.size()); + + // Advance the clock to make the next batch ready due to linger.ms + time.sleep(lingerMs); + assertEquals("Our partition's leader should be ready", Collections.singleton(node1), readyNodes); + time.sleep(requestTimeout + 1); + + accum.mutePartition(tp1); + expiredBatches = accum.abortExpiredBatches(requestTimeout, cluster, time.milliseconds()); + assertEquals("The batch should not be expired when metadata is still available and partition is muted", 0, expiredBatches.size()); + + accum.unmutePartition(tp1); + expiredBatches = accum.abortExpiredBatches(requestTimeout, cluster, time.milliseconds()); + assertEquals("The batch should be expired when the partition is not muted", 1, expiredBatches.size()); + assertEquals("No partitions should be ready.", 0, accum.ready(cluster, time.milliseconds()).readyNodes.size()); + + // Test batches in retry. + // Create a retried batch + accum.append(tp1, 0L, key, value, null, 0); + time.sleep(lingerMs); + readyNodes = accum.ready(cluster, time.milliseconds()).readyNodes; + assertEquals("Our partition's leader should be ready", Collections.singleton(node1), readyNodes); + Map> drained = accum.drain(cluster, readyNodes, Integer.MAX_VALUE, time.milliseconds()); + assertEquals("There should be only one batch.", drained.get(node1.id()).size(), 1); + time.sleep(1000L); + accum.reenqueue(drained.get(node1.id()).get(0), time.milliseconds()); + + // test expiration. + time.sleep(requestTimeout + retryBackoffMs); + expiredBatches = accum.abortExpiredBatches(requestTimeout, cluster, time.milliseconds()); + assertEquals("The batch should not be expired.", 0, expiredBatches.size()); + time.sleep(1L); + + accum.mutePartition(tp1); + expiredBatches = accum.abortExpiredBatches(requestTimeout, cluster, time.milliseconds()); + assertEquals("The batch should not be expired when the partition is muted", 0, expiredBatches.size()); + + accum.unmutePartition(tp1); + expiredBatches = accum.abortExpiredBatches(requestTimeout, cluster, time.milliseconds()); + assertEquals("The batch should be expired when the partition is not muted.", 1, expiredBatches.size()); } @Test From 4c0660bf3da9879cb405a0f85cf1524511e091e8 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Mon, 28 Mar 2016 09:00:03 -0700 Subject: [PATCH 093/206] MINOR: Fix typo and tweak wording in `RecordAccumulator` comments This was recently introduced in: https://github.com/apache/kafka/commit/1fbe445dde71df0023a978c5e54dd229d3d23e1b Author: Ismael Juma Reviewers: Jun Rao Closes #1152 from ijuma/fix-typos-in-record-accumulator --- .../clients/producer/internals/RecordAccumulator.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index 915c4d3e9d930..7f5b16f244eca 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -218,14 +218,14 @@ public List abortExpiredBatches(int requestTimeout, Cluster cluster int count = 0; for (Map.Entry> entry : this.batches.entrySet()) { Deque dq = entry.getValue(); - // We only check if the batch should be expired if the partition does not have a batch in flight. - // This is to avoid the later batches get expired when an earlier batch is still in progress. - // This protection only takes effect when user sets max.in.flight.request.per.connection=1. - // Otherwise the expiration order is not guranteed. TopicPartition tp = entry.getKey(); + // We only check if the batch should be expired if the partition does not have a batch in flight. + // This is to prevent later batches from being expired while an earlier batch is still in progress. + // Note that `muted` is only ever populated if `max.in.flight.request.per.connection=1` so this protection + // is only active in this case. Otherwise the expiration order is not guaranteed. if (!muted.contains(tp)) { synchronized (dq) { - // iterate over the batches and expire them if they have stayed in accumulator for more than requestTimeOut + // iterate over the batches and expire them if they have been in the accumulator for more than requestTimeOut RecordBatch lastBatch = dq.peekLast(); Iterator batchIterator = dq.iterator(); while (batchIterator.hasNext()) { From 43d5078e981bbb25fd81cdc8ba4c339cd2d3f3d2 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Mon, 28 Mar 2016 14:35:31 -0700 Subject: [PATCH 094/206] MINOR: Remove a couple of redundant `CoreUtils.rm` methods Also: * Rename remaining `CoreUtils.rm` to `delete` for consistency * Use `try with resources` in `Utils` to simplify code * Silence compiler warning due to exception catch clause in `TestUtils` Author: Ismael Juma Reviewers: Guozhang Wang Closes #1153 from ijuma/remove-redundant-core-utils-rm --- .../org/apache/kafka/common/utils/Utils.java | 18 ++++--------- core/src/main/scala/kafka/log/Log.scala | 3 ++- .../metrics/KafkaCSVMetricsReporter.scala | 7 ++--- .../main/scala/kafka/utils/CoreUtils.scala | 27 +------------------ .../kafka/api/ProducerCompressionTest.scala | 2 +- .../scala/other/kafka/StressTestLog.scala | 5 ++-- .../other/kafka/TestLinearWriteSpeed.scala | 7 +++-- .../unit/kafka/admin/AddPartitionsTest.scala | 2 +- .../scala/unit/kafka/admin/AdminTest.scala | 12 +++++---- .../integration/KafkaServerTestHarness.scala | 2 +- .../kafka/integration/RollingBounceTest.scala | 2 +- .../UncleanLeaderElectionTest.scala | 2 +- .../kafka/log/BrokerCompressionTest.scala | 9 +++---- .../scala/unit/kafka/log/CleanerTest.scala | 2 +- .../kafka/log/LogCleanerIntegrationTest.scala | 5 ++-- .../scala/unit/kafka/log/LogManagerTest.scala | 5 ++-- .../test/scala/unit/kafka/log/LogTest.scala | 6 ++--- .../unit/kafka/producer/ProducerTest.scala | 4 +-- .../kafka/server/AdvertiseBrokerTest.scala | 4 +-- .../server/HighwatermarkPersistenceTest.scala | 7 +++-- .../kafka/server/LeaderElectionTest.scala | 2 +- .../unit/kafka/server/LogOffsetTest.scala | 3 ++- .../unit/kafka/server/LogRecoveryTest.scala | 3 ++- .../unit/kafka/server/OffsetCommitTest.scala | 9 +++---- .../server/ServerGenerateBrokerIdTest.scala | 18 ++++++------- .../kafka/server/ServerShutdownTest.scala | 6 ++--- .../unit/kafka/server/ServerStartupTest.scala | 6 ++--- .../scala/unit/kafka/utils/TestUtils.scala | 10 +++---- .../unit/kafka/zk/EmbeddedZookeeper.scala | 9 +++---- 29 files changed, 86 insertions(+), 111 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index 4c4225bdcf89a..01675487ff99f 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -442,13 +442,8 @@ public static String join(Collection list, String seperator) { */ public static Properties loadProps(String filename) throws IOException, FileNotFoundException { Properties props = new Properties(); - InputStream propStream = null; - try { - propStream = new FileInputStream(filename); + try (InputStream propStream = new FileInputStream(filename)) { props.load(propStream); - } finally { - if (propStream != null) - propStream.close(); } return props; } @@ -540,16 +535,13 @@ public static byte[] readBytes(ByteBuffer buffer) { */ public static String readFileAsString(String path, Charset charset) throws IOException { if (charset == null) charset = Charset.defaultCharset(); - FileInputStream stream = new FileInputStream(new File(path)); - String result = new String(); - try { + + try (FileInputStream stream = new FileInputStream(new File(path))) { FileChannel fc = stream.getChannel(); MappedByteBuffer bb = fc.map(FileChannel.MapMode.READ_ONLY, 0, fc.size()); - result = charset.decode(bb).toString(); - } finally { - stream.close(); + return charset.decode(bb).toString(); } - return result; + } public static String readFileAsString(String path) throws IOException { diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 8c956f7378f58..81c19fae93185 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -32,6 +32,7 @@ import org.apache.kafka.common.record.TimestampType import scala.collection.JavaConversions import com.yammer.metrics.core.Gauge +import org.apache.kafka.common.utils.Utils object LogAppendInfo { val UnknownLogAppendInfo = LogAppendInfo(-1, -1, Message.NoTimestamp, NoCompressionCodec, NoCompressionCodec, -1, -1, false) @@ -714,7 +715,7 @@ class Log(val dir: File, removeLogMetrics() logSegments.foreach(_.delete()) segments.clear() - CoreUtils.rm(dir) + Utils.delete(dir) } } diff --git a/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala b/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala index cc0da9f7b89c0..686f692548d9d 100755 --- a/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala +++ b/core/src/main/scala/kafka/metrics/KafkaCSVMetricsReporter.scala @@ -22,14 +22,15 @@ package kafka.metrics import com.yammer.metrics.Metrics import java.io.File + import com.yammer.metrics.reporting.CsvReporter import java.util.concurrent.TimeUnit -import kafka.utils.{CoreUtils, VerifiableProperties, Logging} +import kafka.utils.{Logging, VerifiableProperties} +import org.apache.kafka.common.utils.Utils private trait KafkaCSVMetricsReporterMBean extends KafkaMetricsReporterMBean - private class KafkaCSVMetricsReporter extends KafkaMetricsReporter with KafkaCSVMetricsReporterMBean with Logging { @@ -48,7 +49,7 @@ private class KafkaCSVMetricsReporter extends KafkaMetricsReporter if (!initialized) { val metricsConfig = new KafkaMetricsConfig(props) csvDir = new File(props.getString("kafka.csv.metrics.dir", "kafka_metrics")) - CoreUtils.rm(csvDir) + Utils.delete(csvDir) csvDir.mkdirs() underlying = new CsvReporter(Metrics.defaultRegistry(), csvDir) if (props.getBoolean("kafka.csv.metrics.reporter.enabled", default = false)) { diff --git a/core/src/main/scala/kafka/utils/CoreUtils.scala b/core/src/main/scala/kafka/utils/CoreUtils.scala index b01f5cc413abc..fe2bebf195cf2 100755 --- a/core/src/main/scala/kafka/utils/CoreUtils.scala +++ b/core/src/main/scala/kafka/utils/CoreUtils.scala @@ -78,36 +78,11 @@ object CoreUtils extends Logging { } } - /** - * Recursively delete the given file/directory and any subfiles (if any exist) - * @param file The root file at which to begin deleting - */ - def rm(file: String): Unit = rm(new File(file)) - /** * Recursively delete the list of files/directories and any subfiles (if any exist) * @param files sequence of files to be deleted */ - def rm(files: Seq[String]): Unit = files.foreach(f => rm(new File(f))) - - /** - * Recursively delete the given file/directory and any subfiles (if any exist) - * @param file The root file at which to begin deleting - */ - def rm(file: File) { - if(file == null) { - return - } else if(file.isDirectory) { - val files = file.listFiles() - if(files != null) { - for(f <- files) - rm(f) - } - file.delete() - } else { - file.delete() - } - } + def delete(files: Seq[String]): Unit = files.foreach(f => Utils.delete(new File(f))) /** * Register the given mbean with the platform mbean server, diff --git a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala index c4a2bd788956d..fc1ceec731c0a 100755 --- a/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala +++ b/core/src/test/scala/integration/kafka/api/ProducerCompressionTest.scala @@ -55,7 +55,7 @@ class ProducerCompressionTest(compression: String) extends ZooKeeperTestHarness @After override def tearDown() { server.shutdown - CoreUtils.rm(server.config.logDirs) + CoreUtils.delete(server.config.logDirs) super.tearDown() } diff --git a/core/src/test/scala/other/kafka/StressTestLog.scala b/core/src/test/scala/other/kafka/StressTestLog.scala index dead0ebb743f0..8adc7e2b88249 100755 --- a/core/src/test/scala/other/kafka/StressTestLog.scala +++ b/core/src/test/scala/other/kafka/StressTestLog.scala @@ -19,11 +19,12 @@ package kafka import java.util.Properties import java.util.concurrent.atomic._ -import kafka.common._ + import kafka.message._ import kafka.log._ import kafka.utils._ import org.apache.kafka.clients.consumer.OffsetOutOfRangeException +import org.apache.kafka.common.utils.Utils /** * A stress test that instantiates a log and then runs continual appends against it from one thread and continual reads against it @@ -55,7 +56,7 @@ object StressTestLog { running.set(false) writer.join() reader.join() - CoreUtils.rm(dir) + Utils.delete(dir) } }) diff --git a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala index 236d8579d9bce..db281bfa543ac 100755 --- a/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala +++ b/core/src/test/scala/other/kafka/TestLinearWriteSpeed.scala @@ -21,11 +21,14 @@ import java.io._ import java.nio._ import java.nio.channels._ import java.util.{Properties, Random} + import kafka.log._ import kafka.utils._ import kafka.message._ + import scala.math._ import joptsimple._ +import org.apache.kafka.common.utils.Utils /** * This test does linear writes using either a kafka log or a file and measures throughput and latency. @@ -196,7 +199,7 @@ object TestLinearWriteSpeed { } class LogWritable(val dir: File, config: LogConfig, scheduler: Scheduler, val messages: ByteBufferMessageSet) extends Writable { - CoreUtils.rm(dir) + Utils.delete(dir) val log = new Log(dir, config, 0L, scheduler, SystemTime) def write(): Int = { log.append(messages, true) @@ -204,7 +207,7 @@ object TestLinearWriteSpeed { } def close() { log.close() - CoreUtils.rm(log.dir) + Utils.delete(log.dir) } } diff --git a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala index b9bbaceb15576..ab8d36314477f 100755 --- a/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AddPartitionsTest.scala @@ -59,7 +59,7 @@ class AddPartitionsTest extends ZooKeeperTestHarness { @After override def tearDown() { servers.foreach(_.shutdown()) - servers.foreach(server => CoreUtils.rm(server.config.logDirs)) + servers.foreach(server => CoreUtils.delete(server.config.logDirs)) super.tearDown() } diff --git a/core/src/test/scala/unit/kafka/admin/AdminTest.scala b/core/src/test/scala/unit/kafka/admin/AdminTest.scala index 8910e096d8497..21bb6ab4631b3 100755 --- a/core/src/test/scala/unit/kafka/admin/AdminTest.scala +++ b/core/src/test/scala/unit/kafka/admin/AdminTest.scala @@ -22,13 +22,15 @@ import org.apache.kafka.common.protocol.ApiKeys import org.junit.Assert._ import org.junit.Test import java.util.Properties + import kafka.utils._ import kafka.log._ import kafka.zk.ZooKeeperTestHarness -import kafka.utils.{Logging, ZkUtils, TestUtils} -import kafka.common.{TopicExistsException, TopicAndPartition} -import kafka.server.{ConfigType, KafkaServer, KafkaConfig} +import kafka.utils.{Logging, TestUtils, ZkUtils} +import kafka.common.{TopicAndPartition, TopicExistsException} +import kafka.server.{ConfigType, KafkaConfig, KafkaServer} import java.io.File + import TestUtils._ import scala.collection.{Map, immutable} @@ -418,7 +420,7 @@ class AdminTest extends ZooKeeperTestHarness with Logging with RackAwareTest { assertEquals(newConfig, configInZk) } finally { server.shutdown() - server.config.logDirs.foreach(CoreUtils.rm(_)) + CoreUtils.delete(server.config.logDirs) } } @@ -449,7 +451,7 @@ class AdminTest extends ZooKeeperTestHarness with Logging with RackAwareTest { assertEquals(new Quota(2000, true), server.apis.quotaManagers(ApiKeys.FETCH.id).quota(clientId)) } finally { server.shutdown() - server.config.logDirs.foreach(CoreUtils.rm(_)) + CoreUtils.delete(server.config.logDirs) } } diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index 870b9ad5423d8..676772f331188 100755 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -87,7 +87,7 @@ trait KafkaServerTestHarness extends ZooKeeperTestHarness { @After override def tearDown() { servers.foreach(_.shutdown()) - servers.foreach(_.config.logDirs.foreach(CoreUtils.rm(_))) + servers.foreach(server => CoreUtils.delete(server.config.logDirs)) super.tearDown } diff --git a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala index b9315684481bf..5221855603a3a 100755 --- a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala +++ b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala @@ -43,7 +43,7 @@ class RollingBounceTest extends ZooKeeperTestHarness { @After override def tearDown() { servers.foreach(_.shutdown()) - servers.foreach(server => CoreUtils.rm(server.config.logDirs)) + servers.foreach(server => CoreUtils.delete(server.config.logDirs)) super.tearDown() } diff --git a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala index b725d8b59ee2b..a8ba283d9d195 100755 --- a/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/integration/UncleanLeaderElectionTest.scala @@ -82,7 +82,7 @@ class UncleanLeaderElectionTest extends ZooKeeperTestHarness { @After override def tearDown() { servers.foreach(server => shutdownServer(server)) - servers.foreach(server => CoreUtils.rm(server.config.logDirs)) + servers.foreach(server => CoreUtils.delete(server.config.logDirs)) // restore log levels kafkaApisLogger.setLevel(Level.ERROR) diff --git a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala index d0cb4a1dc3190..7487bc57e8fc7 100755 --- a/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala +++ b/core/src/test/scala/unit/kafka/log/BrokerCompressionTest.scala @@ -17,7 +17,6 @@ package kafka.log -import java.io.File import kafka.utils._ import kafka.message._ import org.scalatest.junit.JUnitSuite @@ -26,9 +25,9 @@ import org.junit.Assert._ import org.junit.runner.RunWith import org.junit.runners.Parameterized import org.junit.runners.Parameterized.Parameters -import java.util.{Properties, Collection, ArrayList} -import kafka.server.KafkaConfig import org.apache.kafka.common.record.CompressionType +import org.apache.kafka.common.utils.Utils +import java.util.{Collection, Properties} import scala.collection.JavaConversions._ @RunWith(value = classOf[Parameterized]) @@ -41,7 +40,7 @@ class BrokerCompressionTest(messageCompression: String, brokerCompression: Strin @After def tearDown() { - CoreUtils.rm(tmpDir) + Utils.delete(tmpDir) } /** @@ -78,4 +77,4 @@ object BrokerCompressionTest { messageCompression <- CompressionType.values ) yield Array(messageCompression.name, brokerCompression) } -} \ No newline at end of file +} diff --git a/core/src/test/scala/unit/kafka/log/CleanerTest.scala b/core/src/test/scala/unit/kafka/log/CleanerTest.scala index 377323318aaed..b6849f0dcdc7d 100755 --- a/core/src/test/scala/unit/kafka/log/CleanerTest.scala +++ b/core/src/test/scala/unit/kafka/log/CleanerTest.scala @@ -49,7 +49,7 @@ class CleanerTest extends JUnitSuite { @After def teardown() { - CoreUtils.rm(tmpdir) + Utils.delete(tmpdir) } /** diff --git a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala index 6b91611f1e9cb..cc9873c862a60 100755 --- a/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogCleanerIntegrationTest.scala @@ -25,6 +25,7 @@ import kafka.message._ import kafka.server.OffsetCheckpoint import kafka.utils._ import org.apache.kafka.common.record.CompressionType +import org.apache.kafka.common.utils.Utils import org.junit.Assert._ import org.junit._ import org.junit.runner.RunWith @@ -119,7 +120,7 @@ class LogCleanerIntegrationTest(compressionCodec: String) { @After def teardown() { time.scheduler.shutdown() - CoreUtils.rm(logDir) + Utils.delete(logDir) } /* create a cleaner instance and logs with the given parameters */ @@ -165,4 +166,4 @@ object LogCleanerIntegrationTest { list.add(Array(codec.name)) list } -} \ No newline at end of file +} diff --git a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala index 46bfbed476e49..f290d54702f96 100755 --- a/core/src/test/scala/unit/kafka/log/LogManagerTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogManagerTest.scala @@ -24,6 +24,7 @@ import kafka.common._ import kafka.server.OffsetCheckpoint import kafka.utils._ import org.apache.kafka.common.errors.OffsetOutOfRangeException +import org.apache.kafka.common.utils.Utils import org.junit.Assert._ import org.junit.{After, Before, Test} @@ -54,8 +55,8 @@ class LogManagerTest { def tearDown() { if(logManager != null) logManager.shutdown() - CoreUtils.rm(logDir) - logManager.logDirs.foreach(CoreUtils.rm(_)) + Utils.delete(logDir) + logManager.logDirs.foreach(Utils.delete) } /** diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index c2eb8179fbe20..4d75d53c254a2 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -19,7 +19,6 @@ package kafka.log import java.io._ import java.util.Properties -import java.util.concurrent.atomic._ import org.apache.kafka.common.errors.{CorruptRecordException, OffsetOutOfRangeException, RecordBatchTooLargeException, RecordTooLargeException} import kafka.api.ApiVersion @@ -30,6 +29,7 @@ import org.junit.{After, Before, Test} import kafka.message._ import kafka.utils._ import kafka.server.KafkaConfig +import org.apache.kafka.common.utils.Utils class LogTest extends JUnitSuite { @@ -47,7 +47,7 @@ class LogTest extends JUnitSuite { @After def tearDown() { - CoreUtils.rm(tmpDir) + Utils.delete(tmpDir) } def createEmptyLogs(dir: File, offsets: Int*) { @@ -810,7 +810,7 @@ class LogTest extends JUnitSuite { log = new Log(logDir, config, recoveryPoint, time.scheduler, time) assertEquals(numMessages, log.logEndOffset) assertEquals("Messages in the log after recovery should be the same.", messages, log.logSegments.flatMap(_.log.iterator.toList)) - CoreUtils.rm(logDir) + Utils.delete(logDir) } } diff --git a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala index 4a1ad5abcfdc5..cf25cdbf1c635 100755 --- a/core/src/test/scala/unit/kafka/producer/ProducerTest.scala +++ b/core/src/test/scala/unit/kafka/producer/ProducerTest.scala @@ -96,8 +96,8 @@ class ProducerTest extends ZooKeeperTestHarness with Logging{ server1.shutdown server2.shutdown - CoreUtils.rm(server1.config.logDirs) - CoreUtils.rm(server2.config.logDirs) + CoreUtils.delete(server1.config.logDirs) + CoreUtils.delete(server2.config.logDirs) super.tearDown() } diff --git a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala index 75fa664725e72..dc17aa434f982 100755 --- a/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala +++ b/core/src/test/scala/unit/kafka/server/AdvertiseBrokerTest.scala @@ -43,7 +43,7 @@ class AdvertiseBrokerTest extends ZooKeeperTestHarness { @After override def tearDown() { server.shutdown() - CoreUtils.rm(server.config.logDirs) + CoreUtils.delete(server.config.logDirs) super.tearDown() } @@ -55,4 +55,4 @@ class AdvertiseBrokerTest extends ZooKeeperTestHarness { assertEquals(advertisedPort, endpoint.port) } -} \ No newline at end of file +} diff --git a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala index 2e6660195ada7..26e28171a409f 100755 --- a/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala +++ b/core/src/test/scala/unit/kafka/server/HighwatermarkPersistenceTest.scala @@ -18,16 +18,15 @@ package kafka.server import kafka.log._ import java.io.File -import org.I0Itec.zkclient.ZkClient import org.apache.kafka.common.metrics.Metrics +import org.apache.kafka.common.utils.{Utils, MockTime => JMockTime} import org.easymock.EasyMock import org.junit._ import org.junit.Assert._ import kafka.common._ import kafka.cluster.Replica -import kafka.utils.{ZkUtils, SystemTime, KafkaScheduler, TestUtils, MockTime, CoreUtils} +import kafka.utils.{KafkaScheduler, MockTime, SystemTime, TestUtils, ZkUtils} import java.util.concurrent.atomic.AtomicBoolean -import org.apache.kafka.common.utils.{MockTime => JMockTime} class HighwatermarkPersistenceTest { @@ -42,7 +41,7 @@ class HighwatermarkPersistenceTest { @After def teardown() { for(manager <- logManagers; dir <- manager.logDirs) - CoreUtils.rm(dir) + Utils.delete(dir) } @Test diff --git a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala index e84780aaa2a1a..72589800e111c 100755 --- a/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala +++ b/core/src/test/scala/unit/kafka/server/LeaderElectionTest.scala @@ -58,7 +58,7 @@ class LeaderElectionTest extends ZooKeeperTestHarness { @After override def tearDown() { servers.foreach(_.shutdown()) - servers.foreach(server => CoreUtils.rm(server.config.logDirs)) + servers.foreach(server => CoreUtils.delete(server.config.logDirs)) super.tearDown() } diff --git a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala index 8c86a7b5956af..d5c696ee9ff17 100755 --- a/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogOffsetTest.scala @@ -30,6 +30,7 @@ import kafka.utils._ import kafka.zk.ZooKeeperTestHarness import org.apache.kafka.common.TopicPartition import org.apache.kafka.common.protocol.Errors +import org.apache.kafka.common.utils.Utils import org.junit.Assert._ import org.junit.{After, Before, Test} @@ -57,7 +58,7 @@ class LogOffsetTest extends ZooKeeperTestHarness { override def tearDown() { simpleConsumer.close server.shutdown - CoreUtils.rm(logDir) + Utils.delete(logDir) super.tearDown() } diff --git a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala index e13bfd96342dd..d37de761f1385 100755 --- a/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala +++ b/core/src/test/scala/unit/kafka/server/LogRecoveryTest.scala @@ -26,6 +26,7 @@ import java.io.File import org.apache.kafka.clients.producer.{KafkaProducer, ProducerRecord} import org.apache.kafka.common.serialization.{IntegerSerializer, StringSerializer} +import org.apache.kafka.common.utils.Utils import org.junit.{After, Before, Test} import org.junit.Assert._ @@ -94,7 +95,7 @@ class LogRecoveryTest extends ZooKeeperTestHarness { producer.close() for (server <- servers) { server.shutdown() - CoreUtils.rm(server.config.logDirs(0)) + Utils.delete(new File(server.config.logDirs(0))) } super.tearDown() } diff --git a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala index 1d5148b0e1588..29eaf2da4893f 100755 --- a/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala +++ b/core/src/test/scala/unit/kafka/server/OffsetCommitTest.scala @@ -19,13 +19,14 @@ package kafka.server import kafka.api.{GroupCoordinatorRequest, OffsetCommitRequest, OffsetFetchRequest} import kafka.consumer.SimpleConsumer -import kafka.common.{OffsetMetadata, OffsetMetadataAndError, OffsetAndMetadata, TopicAndPartition} +import kafka.common.{OffsetAndMetadata, OffsetMetadata, OffsetMetadataAndError, TopicAndPartition} import kafka.utils._ import kafka.utils.TestUtils._ import kafka.zk.ZooKeeperTestHarness import org.apache.kafka.common.protocol.Errors - +import org.apache.kafka.common.utils.Utils import org.junit.{After, Before, Test} +import org.junit.Assert._ import java.util.Properties import java.io.File @@ -33,8 +34,6 @@ import java.io.File import scala.util.Random import scala.collection._ -import org.junit.Assert._ - class OffsetCommitTest extends ZooKeeperTestHarness { val random: Random = new Random() val group = "test-group" @@ -71,7 +70,7 @@ class OffsetCommitTest extends ZooKeeperTestHarness { override def tearDown() { simpleConsumer.close server.shutdown - CoreUtils.rm(logDir) + Utils.delete(logDir) super.tearDown() } diff --git a/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala index c26ff133a3f25..8e25366aed073 100755 --- a/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerGenerateBrokerIdTest.scala @@ -51,7 +51,7 @@ class ServerGenerateBrokerIdTest extends ZooKeeperTestHarness { server1.startup() assertEquals(server1.config.brokerId, 1001) server1.shutdown() - CoreUtils.rm(server1.config.logDirs) + CoreUtils.delete(server1.config.logDirs) TestUtils.verifyNonDaemonThreadsStatus(this.getClass.getName) } @@ -75,9 +75,9 @@ class ServerGenerateBrokerIdTest extends ZooKeeperTestHarness { assertTrue(verifyBrokerMetadata(server1.config.logDirs,1001)) assertTrue(verifyBrokerMetadata(server2.config.logDirs,0)) assertTrue(verifyBrokerMetadata(server3.config.logDirs,1002)) - CoreUtils.rm(server1.config.logDirs) - CoreUtils.rm(server2.config.logDirs) - CoreUtils.rm(server3.config.logDirs) + CoreUtils.delete(server1.config.logDirs) + CoreUtils.delete(server2.config.logDirs) + CoreUtils.delete(server3.config.logDirs) TestUtils.verifyNonDaemonThreadsStatus(this.getClass.getName) } @@ -93,7 +93,7 @@ class ServerGenerateBrokerIdTest extends ZooKeeperTestHarness { assertEquals(server3.config.brokerId,3) server3.shutdown() assertTrue(verifyBrokerMetadata(server3.config.logDirs,3)) - CoreUtils.rm(server3.config.logDirs) + CoreUtils.delete(server3.config.logDirs) TestUtils.verifyNonDaemonThreadsStatus(this.getClass.getName) } @@ -116,7 +116,7 @@ class ServerGenerateBrokerIdTest extends ZooKeeperTestHarness { server1.startup() server1.shutdown() assertTrue(verifyBrokerMetadata(config1.logDirs, 1001)) - CoreUtils.rm(server1.config.logDirs) + CoreUtils.delete(server1.config.logDirs) TestUtils.verifyNonDaemonThreadsStatus(this.getClass.getName) } @@ -133,7 +133,7 @@ class ServerGenerateBrokerIdTest extends ZooKeeperTestHarness { case e: kafka.common.InconsistentBrokerIdException => //success } server1.shutdown() - CoreUtils.rm(server1.config.logDirs) + CoreUtils.delete(server1.config.logDirs) TestUtils.verifyNonDaemonThreadsStatus(this.getClass.getName) } @@ -170,8 +170,8 @@ class ServerGenerateBrokerIdTest extends ZooKeeperTestHarness { // verify correct broker metadata was written assertTrue(verifyBrokerMetadata(serverA.config.logDirs,1)) assertTrue(verifyBrokerMetadata(newServerB.config.logDirs,2)) - CoreUtils.rm(serverA.config.logDirs) - CoreUtils.rm(newServerB.config.logDirs) + CoreUtils.delete(serverA.config.logDirs) + CoreUtils.delete(newServerB.config.logDirs) TestUtils.verifyNonDaemonThreadsStatus(this.getClass.getName) } diff --git a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala index 67f62d9cf7569..bc71edd5df60b 100755 --- a/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerShutdownTest.scala @@ -104,7 +104,7 @@ class ServerShutdownTest extends ZooKeeperTestHarness { consumer.close() producer.close() server.shutdown() - CoreUtils.rm(server.config.logDirs) + CoreUtils.delete(server.config.logDirs) verifyNonDaemonThreadsStatus } @@ -117,7 +117,7 @@ class ServerShutdownTest extends ZooKeeperTestHarness { server.startup() server.shutdown() server.awaitShutdown() - CoreUtils.rm(server.config.logDirs) + CoreUtils.delete(server.config.logDirs) verifyNonDaemonThreadsStatus } @@ -145,7 +145,7 @@ class ServerShutdownTest extends ZooKeeperTestHarness { server.shutdown() server.awaitShutdown() } - CoreUtils.rm(server.config.logDirs) + CoreUtils.delete(server.config.logDirs) verifyNonDaemonThreadsStatus } diff --git a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala index b321a0266bf9f..9b49365c77060 100755 --- a/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala +++ b/core/src/test/scala/unit/kafka/server/ServerStartupTest.scala @@ -40,7 +40,7 @@ class ServerStartupTest extends ZooKeeperTestHarness { assertTrue(pathExists) server.shutdown() - CoreUtils.rm(server.config.logDirs) + CoreUtils.delete(server.config.logDirs) } @Test @@ -66,7 +66,7 @@ class ServerStartupTest extends ZooKeeperTestHarness { assertEquals(brokerRegistration, zkUtils.readData(ZkUtils.BrokerIdsPath + "/" + brokerId)._1) server1.shutdown() - CoreUtils.rm(server1.config.logDirs) + CoreUtils.delete(server1.config.logDirs) } @Test @@ -80,6 +80,6 @@ class ServerStartupTest extends ZooKeeperTestHarness { assertEquals(brokerId, server.metadataCache.getAliveBrokers.head.id) server.shutdown() - CoreUtils.rm(server.config.logDirs) + CoreUtils.delete(server.config.logDirs) } } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 0730468a496b5..a1e79128dda98 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -21,9 +21,8 @@ import java.io._ import java.nio._ import java.nio.file.Files import java.nio.channels._ -import java.util -import java.util.concurrent.{Callable, TimeUnit, Executors} -import java.util.{Collections, Random, Properties} +import java.util.concurrent.{Callable, Executors, TimeUnit} +import java.util.{Collections, Properties, Random} import java.security.cert.X509Certificate import javax.net.ssl.X509TrustManager import charset.Charset @@ -52,6 +51,7 @@ import org.apache.kafka.clients.CommonClientConfigs import org.apache.kafka.common.network.Mode import org.apache.kafka.common.record.CompressionType import org.apache.kafka.common.serialization.{ByteArraySerializer, Serializer} +import org.apache.kafka.common.utils.Utils import scala.collection.Map import scala.collection.JavaConversions._ @@ -100,7 +100,7 @@ object TestUtils extends Logging { Runtime.getRuntime().addShutdownHook(new Thread() { override def run() = { - CoreUtils.rm(f) + Utils.delete(f) } }) f @@ -1115,7 +1115,7 @@ object TestUtils extends Logging { } } catch { case ie: InterruptedException => failWithTimeout() - case e => exceptions += e + case e: Throwable => exceptions += e } finally { threadPool.shutdownNow() } diff --git a/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala b/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala index 5fa2f6519d8f2..1030c46d93bd1 100755 --- a/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala +++ b/core/src/test/scala/unit/kafka/zk/EmbeddedZookeeper.scala @@ -21,10 +21,9 @@ import org.apache.zookeeper.server.ZooKeeperServer import org.apache.zookeeper.server.NIOServerCnxnFactory import kafka.utils.TestUtils import java.net.InetSocketAddress -import javax.security.auth.login.Configuration + import kafka.utils.CoreUtils -import org.apache.kafka.common.security.JaasUtils -import org.apache.kafka.common.utils.Utils.getPort +import org.apache.kafka.common.utils.Utils class EmbeddedZookeeper() { val snapshotDir = TestUtils.tempDir() @@ -40,8 +39,8 @@ class EmbeddedZookeeper() { def shutdown() { CoreUtils.swallow(zookeeper.shutdown()) CoreUtils.swallow(factory.shutdown()) - CoreUtils.rm(logDir) - CoreUtils.rm(snapshotDir) + Utils.delete(logDir) + Utils.delete(snapshotDir) } } From 5089f547d5d64a0235e1b4adc327a0cb05eb4ca8 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Tue, 29 Mar 2016 13:30:56 -0700 Subject: [PATCH 095/206] HOTFIX: RocksDBStore must clear dirty flags after flush guozhangwang Without clearing the dirty flags, RocksDBStore will perform flush for every new record. This bug made the store performance painfully slower. Author: Yasuhiro Matsuda Reviewers: Guozhang Wang Closes #1163 from ymatsuda/clear_dirty_flag --- .../streams/state/internals/RocksDBStore.java | 27 +++++++++++-------- 1 file changed, 16 insertions(+), 11 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index b206f37962afd..fe327f6af8a2d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -165,7 +165,7 @@ public void init(ProcessorContext context, StateStore root) { public void apply(K key, RocksDBCacheEntry entry) { // flush all the dirty entries to RocksDB if this evicted entry is dirty if (entry.isDirty) { - flush(); + flushCache(); } } }); @@ -226,7 +226,6 @@ public V get(K key) { RocksDBCacheEntry entry = cache.get(key); if (entry == null) { - byte[] rawKey = serdes.rawKey(key); V value = serdes.valueFrom(getInternal(serdes.rawKey(key))); cache.put(key, new RocksDBCacheEntry(value)); @@ -251,8 +250,8 @@ private byte[] getInternal(byte[] rawKey) { @Override public void put(K key, V value) { if (cache != null) { - cache.put(key, new RocksDBCacheEntry(value, true)); cacheDirtyKeys.add(key); + cache.put(key, new RocksDBCacheEntry(value, true)); } else { byte[] rawKey = serdes.rawKey(key); byte[] rawValue = serdes.rawValue(value); @@ -298,7 +297,7 @@ public void putAll(List> entries) { put(entry.key, entry.value); } - // this function is only called in flush() + // this function is only called in flushCache() private void putAllInternal(List> entries) { WriteBatch batch = new WriteBatch(); @@ -324,7 +323,7 @@ public V delete(K key) { public KeyValueIterator range(K from, K to) { // we need to flush the cache if necessary before returning the iterator if (cache != null) - flush(); + flushCache(); return new RocksDBRangeIterator(db.newIterator(), serdes, from, to); } @@ -333,15 +332,14 @@ public KeyValueIterator range(K from, K to) { public KeyValueIterator all() { // we need to flush the cache if necessary before returning the iterator if (cache != null) - flush(); + flushCache(); RocksIterator innerIter = db.newIterator(); innerIter.seekToFirst(); return new RocksDbIterator(innerIter, serdes); } - @Override - public void flush() { + private void flushCache() { // flush of the cache entries if necessary if (cache != null) { List> putBatch = new ArrayList<>(cache.keys.size()); @@ -350,7 +348,7 @@ public void flush() { for (K key : cacheDirtyKeys) { RocksDBCacheEntry entry = cache.get(key); - assert entry.isDirty; + entry.isDirty = false; byte[] rawKey = serdes.rawKey(key); @@ -386,12 +384,19 @@ public void flush() { cacheDirtyKeys.clear(); } - flushInternal(); - if (loggingEnabled) changeLogger.logChange(getter); } + @Override + public void flush() { + // flush of the cache entries if necessary + flushCache(); + + // flush RocksDB + flushInternal(); + } + public void flushInternal() { try { db.flush(fOptions); From 27583953528138ff3503fb8f300c97971718e48a Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Tue, 29 Mar 2016 14:18:57 -0700 Subject: [PATCH 096/206] KAFKA-3425: add missing upgrade notes Author: Jason Gustafson Reviewers: Grant Henke, Ashish Singh, Ismael Juma, Guozhang Wang Closes #1159 from hachikuji/KAFKA-3425 --- docs/upgrade.html | 6 +++++- 1 file changed, 5 insertions(+), 1 deletion(-) diff --git a/docs/upgrade.html b/docs/upgrade.html index 060c3deb33dd1..b9c4bec012c10 100644 --- a/docs/upgrade.html +++ b/docs/upgrade.html @@ -79,12 +79,16 @@
      Potential breaking c
    12. MessageReader's package was changed from kafka.tools to kafka.common
    13. MirrorMakerMessageHandler no longer exposes the handle(record: MessageAndMetadata[Array[Byte], Array[Byte]]) method as it was never called.
    14. The 0.7 KafkaMigrationTool is no longer packaged with Kafka. If you need to migrate from 0.7 to 0.10.0, please migrate to 0.8 first and then follow the documented upgrade process to upgrade from 0.8 to 0.10.0.
    15. +
    16. The new consumer has standardized its APIs to accept java.util.Collection as the sequence type for method parameters. Existing code may have to be updated to work with the 0.10.0 client library.
    17. Notable changes in 0.10.0.0
        -
      • The default value of the configuration parameter receive.buffer.bytes is now 64K for the new consumer
      • +
      • The default value of the configuration parameter receive.buffer.bytes is now 64K for the new consumer.
      • +
      • The new consumer now exposes the configuration parameter exclude.internal.topics to restrict internal topics (such as the consumer offsets topic) from accidentally being included in regular expression subscriptions. By default, it is enabled.
      • +
      • The old Scala producer has been deprecated. Users should migrate their code to the Java producer included in the kafka-clients JAR as soon as possible.
      • +
      • The new consumer API has been marked stable.

      Upgrading from 0.8.0, 0.8.1.X or 0.8.2.X to 0.9.0.0

      From aaafd23df800d41e4b16fd399015991b2e426dc5 Mon Sep 17 00:00:00 2001 From: Geoff Anderson Date: Wed, 30 Mar 2016 00:50:29 -0700 Subject: [PATCH 097/206] MINOR: Advance system test ducktape dependency from 0.3.10 to 0.4.0 Previous version of ducktape was found to have a memory leak which caused occasional failures in nightly runs. Author: Geoff Anderson Reviewers: Ewen Cheslack-Postava Closes #1165 from granders/minor-advance-ducktape-to-0.4.0 --- tests/setup.py | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/tests/setup.py b/tests/setup.py index 865c3160f4de0..de3ea62a9d8e7 100644 --- a/tests/setup.py +++ b/tests/setup.py @@ -30,5 +30,5 @@ license="apache2.0", packages=find_packages(), include_package_data=True, - install_requires=["ducktape==0.3.10", "requests>=2.5.0"] + install_requires=["ducktape==0.4.0", "requests>=2.5.0"] ) From 2788f2dc73dd62c35c87842d02d23a0ed6a62e97 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Wed, 30 Mar 2016 14:26:01 -0700 Subject: [PATCH 098/206] MINOR: a simple benchmark for Streams guozhangwang miguno Author: Yasuhiro Matsuda Reviewers: Guozhang Wang Closes #1164 from ymatsuda/perf --- .../processor/internals/StreamTask.java | 4 +- .../kafka/streams/perf/SimpleBenchmark.java | 413 ++++++++++++++++++ 2 files changed, 415 insertions(+), 2 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index c4cc2baf0e026..afa303ca1514f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -167,11 +167,11 @@ public int process() { this.currNode = recordInfo.node(); TopicPartition partition = recordInfo.partition(); - log.debug("Start processing one record [" + currRecord + "]"); + log.debug("Start processing one record [{}]", currRecord); this.currNode.process(currRecord.key(), currRecord.value()); - log.debug("Completed processing one record [" + currRecord + "]"); + log.debug("Completed processing one record [{}]", currRecord); // update the consumed offset map after processing is done consumedOffsets.put(partition, currRecord.offset()); diff --git a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java new file mode 100644 index 0000000000000..c88309046d69a --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java @@ -0,0 +1,413 @@ +/** + * 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 org.apache.kafka.streams.perf; + +import org.apache.kafka.clients.consumer.ConsumerConfig; +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +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.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.serialization.ByteArrayDeserializer; +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.kafka.common.serialization.LongDeserializer; +import org.apache.kafka.common.serialization.LongSerializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.state.KeyValueStore; +import org.apache.kafka.streams.state.Stores; + +import java.io.File; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.Properties; + +public class SimpleBenchmark { + + private final String kafka; + private final String zookeeper; + private final File stateDir; + + private static final String SOURCE_TOPIC = "simpleBenchmarkSourceTopic"; + private static final String SINK_TOPIC = "simpleBenchmarkSinkTopic"; + + private static final long NUM_RECORDS = 10000000L; + private static final Long END_KEY = NUM_RECORDS - 1; + private static final int KEY_SIZE = 8; + private static final int VALUE_SIZE = 100; + private static final int RECORD_SIZE = KEY_SIZE + VALUE_SIZE; + + private static final Serde BYTE_SERDE = Serdes.ByteArray(); + private static final Serde LONG_SERDE = Serdes.Long(); + + public SimpleBenchmark(File stateDir, String kafka, String zookeeper) { + super(); + this.stateDir = stateDir; + this.kafka = kafka; + this.zookeeper = zookeeper; + } + + public static void main(String[] args) throws Exception { + final File stateDir = new File("/tmp/kafka-streams-simple-benchmark"); + stateDir.mkdir(); + + final File rocksdbDir = new File(stateDir, "rocksdb-test"); + rocksdbDir.mkdir(); + + + final String kafka = "localhost:9092"; + final String zookeeper = "localhost:2181"; + + SimpleBenchmark benchmark = new SimpleBenchmark(stateDir, kafka, zookeeper); + + // producer performance + benchmark.produce(); + // consumer performance + benchmark.consume(); + // simple stream performance source->process + benchmark.processStream(); + // simple stream performance source->sink + benchmark.processStreamWithSink(); + // simple stream performance source->store + benchmark.processStreamWithStateStore(); + } + + public void processStream() { + CountDownLatch latch = new CountDownLatch(1); + + final KafkaStreams streams = createKafkaStreams(stateDir, kafka, zookeeper, latch); + + Thread thread = new Thread() { + public void run() { + streams.start(); + } + }; + thread.start(); + + long startTime = System.currentTimeMillis(); + + while (latch.getCount() > 0) { + try { + latch.await(); + } catch (InterruptedException ex) { + Thread.interrupted(); + } + } + + long endTime = System.currentTimeMillis(); + + System.out.println("Streams Performance [MB/sec read]: " + megaBytePerSec(endTime - startTime)); + + streams.close(); + try { + thread.join(); + } catch (Exception ex) { + // ignore + } + } + + public void processStreamWithSink() { + CountDownLatch latch = new CountDownLatch(1); + + final KafkaStreams streams = createKafkaStreamsWithSink(stateDir, kafka, zookeeper, latch); + + Thread thread = new Thread() { + public void run() { + streams.start(); + } + }; + thread.start(); + + long startTime = System.currentTimeMillis(); + + while (latch.getCount() > 0) { + try { + latch.await(); + } catch (InterruptedException ex) { + Thread.interrupted(); + } + } + + long endTime = System.currentTimeMillis(); + + System.out.println("Streams Performance [MB/sec read+write]: " + megaBytePerSec(endTime - startTime)); + + streams.close(); + try { + thread.join(); + } catch (Exception ex) { + // ignore + } + } + + public void processStreamWithStateStore() { + CountDownLatch latch = new CountDownLatch(1); + + final KafkaStreams streams = createKafkaStreamsWithStateStore(stateDir, kafka, zookeeper, latch); + + Thread thread = new Thread() { + public void run() { + streams.start(); + } + }; + thread.start(); + + long startTime = System.currentTimeMillis(); + + while (latch.getCount() > 0) { + try { + latch.await(); + } catch (InterruptedException ex) { + Thread.interrupted(); + } + } + + long endTime = System.currentTimeMillis(); + + System.out.println("Streams Performance [MB/sec read+store]: " + megaBytePerSec(endTime - startTime)); + + streams.close(); + try { + thread.join(); + } catch (Exception ex) { + // ignore + } + } + + public void produce() { + Properties props = new Properties(); + props.put(ProducerConfig.CLIENT_ID_CONFIG, "simple-benchmark-produce"); + props.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + props.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, LongSerializer.class); + props.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, ByteArraySerializer.class); + + KafkaProducer producer = new KafkaProducer<>(props); + + byte[] value = new byte[VALUE_SIZE]; + long startTime = System.currentTimeMillis(); + + for (int i = 0; i < NUM_RECORDS; i++) { + producer.send(new ProducerRecord<>(SOURCE_TOPIC, (long) i, value)); + } + producer.close(); + + long endTime = System.currentTimeMillis(); + + System.out.println("Producer Performance [MB/sec write]: " + megaBytePerSec(endTime - startTime)); + } + + public void consume() { + Properties props = new Properties(); + props.put(ConsumerConfig.CLIENT_ID_CONFIG, "simple-benchmark-consumer"); + props.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, LongDeserializer.class); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class); + + KafkaConsumer consumer = new KafkaConsumer<>(props); + + List partitions = getAllPartitions(consumer, SOURCE_TOPIC); + consumer.assign(partitions); + consumer.seekToBeginning(partitions); + + Long key = null; + + long startTime = System.currentTimeMillis(); + + while (true) { + ConsumerRecords records = consumer.poll(500); + if (records.isEmpty()) { + if (END_KEY.equals(key)) + break; + } else { + for (ConsumerRecord record : records) { + key = record.key(); + } + } + } + + long endTime = System.currentTimeMillis(); + + System.out.println("Consumer Performance [MB/sec read]: " + megaBytePerSec(endTime - startTime)); + } + + private KafkaStreams createKafkaStreams(File stateDir, String kafka, String zookeeper, final CountDownLatch latch) { + Properties props = new Properties(); + props.put(StreamsConfig.APPLICATION_ID_CONFIG, "simple-benchmark-streams"); + props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString()); + props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, zookeeper); + props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1); + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + + KStreamBuilder builder = new KStreamBuilder(); + + KStream source = builder.stream(LONG_SERDE, BYTE_SERDE, SOURCE_TOPIC); + + source.process(new ProcessorSupplier() { + @Override + public Processor get() { + return new Processor() { + + @Override + public void init(ProcessorContext context) { + } + + @Override + public void process(Long key, byte[] value) { + if (END_KEY.equals(key)) { + latch.countDown(); + } + } + + @Override + public void punctuate(long timestamp) { + } + + @Override + public void close() { + } + }; + } + }); + + return new KafkaStreams(builder, props); + } + + private KafkaStreams createKafkaStreamsWithSink(File stateDir, String kafka, String zookeeper, final CountDownLatch latch) { + Properties props = new Properties(); + props.put(StreamsConfig.APPLICATION_ID_CONFIG, "simple-benchmark-streams-with-sink"); + props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString()); + props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, zookeeper); + props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1); + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + + KStreamBuilder builder = new KStreamBuilder(); + + KStream source = builder.stream(LONG_SERDE, BYTE_SERDE, SOURCE_TOPIC); + + source.to(LONG_SERDE, BYTE_SERDE, SINK_TOPIC); + source.process(new ProcessorSupplier() { + @Override + public Processor get() { + return new Processor() { + + @Override + public void init(ProcessorContext context) { + } + + @Override + public void process(Long key, byte[] value) { + if (END_KEY.equals(key)) { + latch.countDown(); + } + } + + @Override + public void punctuate(long timestamp) { + } + + @Override + public void close() { + } + }; + } + }); + + return new KafkaStreams(builder, props); + } + + + private KafkaStreams createKafkaStreamsWithStateStore(File stateDir, String kafka, String zookeeper, final CountDownLatch latch) { + Properties props = new Properties(); + props.put(StreamsConfig.APPLICATION_ID_CONFIG, "simple-benchmark-streams-with-store"); + props.put(StreamsConfig.STATE_DIR_CONFIG, stateDir.toString()); + props.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, kafka); + props.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, zookeeper); + props.put(StreamsConfig.NUM_STREAM_THREADS_CONFIG, 1); + props.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "earliest"); + + KStreamBuilder builder = new KStreamBuilder(); + + builder.addStateStore(Stores.create("store").withLongKeys().withByteArrayValues().persistent().build()); + + KStream source = builder.stream(LONG_SERDE, BYTE_SERDE, SOURCE_TOPIC); + + source.process(new ProcessorSupplier() { + @Override + public Processor get() { + return new Processor() { + + KeyValueStore store; + + @SuppressWarnings("unchecked") + @Override + public void init(ProcessorContext context) { + store = (KeyValueStore) context.getStateStore("store"); + } + + @Override + public void process(Long key, byte[] value) { + store.put(key, value); + + if (END_KEY.equals(key)) { + latch.countDown(); + } + } + + @Override + public void punctuate(long timestamp) { + } + + @Override + public void close() { + } + }; + } + }, "store"); + + return new KafkaStreams(builder, props); + } + + private double megaBytePerSec(long time) { + return (double) (RECORD_SIZE * NUM_RECORDS / 1024 / 1024) / ((double) time / 1000); + } + + private List getAllPartitions(KafkaConsumer consumer, String... topics) { + ArrayList partitions = new ArrayList<>(); + + for (String topic : topics) { + for (PartitionInfo info : consumer.partitionsFor(topic)) { + partitions.add(new TopicPartition(info.topic(), info.partition())); + } + } + return partitions; + } + +} From 78d91dcd8805d850038df52718380a6f956abad7 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Wed, 30 Mar 2016 19:30:34 -0700 Subject: [PATCH 099/206] KAFKA-3475; Introduce our own `MiniKdc` This also fixes KAFKA-3453 and KAFKA-2866. Author: Ismael Juma Reviewers: Gwen Shapira Closes #1155 from ijuma/kafka-3475-introduce-our-minikdc --- build.gradle | 15 +- .../org/apache/kafka/common/utils/Utils.java | 2 +- .../main/scala/kafka/utils/CoreUtils.scala | 15 +- core/src/test/resources/minikdc-krb5.conf | 25 + core/src/test/resources/minikdc.ldiff | 47 ++ .../integration/kafka/api/SaslSetup.scala | 10 +- .../kafka/security/minikdc/MiniKdc.scala | 433 ++++++++++++++++++ .../integration/KafkaServerTestHarness.scala | 12 +- gradle/dependencies.gradle | 12 +- tests/kafkatest/services/security/minikdc.py | 12 +- 10 files changed, 557 insertions(+), 26 deletions(-) create mode 100644 core/src/test/resources/minikdc-krb5.conf create mode 100644 core/src/test/resources/minikdc.ldiff create mode 100644 core/src/test/scala/kafka/security/minikdc/MiniKdc.scala diff --git a/build.gradle b/build.gradle index 13a8b4e5ac18c..d6f82a4f91298 100644 --- a/build.gradle +++ b/build.gradle @@ -338,7 +338,17 @@ project(':core') { testCompile project(':clients').sourceSets.test.output testCompile libs.bcpkix testCompile libs.easymock - testCompile libs.hadoopMiniKdc + testCompile(libs.apacheda) { + exclude group: 'xml-apis', module: 'xml-apis' + } + testCompile libs.apachedsCoreApi + testCompile libs.apachedsInterceptorKerberos + testCompile libs.apachedsProtocolShared + testCompile libs.apachedsProtocolKerberos + testCompile libs.apachedsProtocolLdap + testCompile libs.apachedsLdifPartition + testCompile libs.apachedsMavibotPartition + testCompile libs.apachedsJdbmPartition testCompile libs.junit testCompile libs.scalaTest } @@ -368,6 +378,9 @@ project(':core') { duplicatesStrategy 'exclude' } + systemTestLibs { + dependsOn testJar + } task genProtocolErrorDocs(type: JavaExec) { classpath = sourceSets.main.runtimeClasspath diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java index 01675487ff99f..2a988222d403b 100755 --- a/clients/src/main/java/org/apache/kafka/common/utils/Utils.java +++ b/clients/src/main/java/org/apache/kafka/common/utils/Utils.java @@ -476,7 +476,7 @@ public static String stackTrace(Throwable e) { * @param daemon Should the thread block JVM shutdown? * @return The unstarted thread */ - public static Thread newThread(String name, Runnable runnable, Boolean daemon) { + public static Thread newThread(String name, Runnable runnable, boolean daemon) { Thread thread = new Thread(runnable, name); thread.setDaemon(daemon); thread.setUncaughtExceptionHandler(new Thread.UncaughtExceptionHandler() { diff --git a/core/src/main/scala/kafka/utils/CoreUtils.scala b/core/src/main/scala/kafka/utils/CoreUtils.scala index fe2bebf195cf2..5b6c59f48f320 100755 --- a/core/src/main/scala/kafka/utils/CoreUtils.scala +++ b/core/src/main/scala/kafka/utils/CoreUtils.scala @@ -57,13 +57,14 @@ object CoreUtils extends Logging { } /** - * Create a daemon thread - * @param name The name of the thread - * @param fun The function to execute in the thread - * @return The unstarted thread - */ - def daemonThread(name: String, fun: => Unit): Thread = - Utils.daemonThread(name, runnable(fun)) + * Create a thread + * @param name The name of the thread + * @param daemon Whether the thread should block JVM shutdown + * @param fun The function to execute in the thread + * @return The unstarted thread + */ + def newThread(name: String, daemon: Boolean)(fun: => Unit): Thread = + Utils.newThread(name, runnable(fun), daemon) /** * Do the given action and log any exceptions thrown without rethrowing them diff --git a/core/src/test/resources/minikdc-krb5.conf b/core/src/test/resources/minikdc-krb5.conf new file mode 100644 index 0000000000000..0603404875525 --- /dev/null +++ b/core/src/test/resources/minikdc-krb5.conf @@ -0,0 +1,25 @@ +# +# Licensed to the Apache Software Foundation (ASF) under one +# or more contributor license agreements. See the NOTICE file +# distributed with this work for additional information +# regarding copyright ownership. The ASF licenses this file +# to you under the Apache License, Version 2.0 (the +# "License"); you may not use this file except in compliance +# with the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# +[libdefaults] +default_realm = {0} +udp_preference_limit = 1 + +[realms] +{0} = '{' + kdc = {1}:{2} +'}' diff --git a/core/src/test/resources/minikdc.ldiff b/core/src/test/resources/minikdc.ldiff new file mode 100644 index 0000000000000..75e4dfdadf2ec --- /dev/null +++ b/core/src/test/resources/minikdc.ldiff @@ -0,0 +1,47 @@ +# +# 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. +# +dn: ou=users,dc=${0},dc=${1} +objectClass: organizationalUnit +objectClass: top +ou: users + +dn: uid=krbtgt,ou=users,dc=${0},dc=${1} +objectClass: top +objectClass: person +objectClass: inetOrgPerson +objectClass: krb5principal +objectClass: krb5kdcentry +cn: KDC Service +sn: Service +uid: krbtgt +userPassword: secret +krb5PrincipalName: krbtgt/${2}.${3}@${2}.${3} +krb5KeyVersionNumber: 0 + +dn: uid=ldap,ou=users,dc=${0},dc=${1} +objectClass: top +objectClass: person +objectClass: inetOrgPerson +objectClass: krb5principal +objectClass: krb5kdcentry +cn: LDAP +sn: Service +uid: ldap +userPassword: secret +krb5PrincipalName: ldap/${4}@${2}.${3} +krb5KeyVersionNumber: 0 diff --git a/core/src/test/scala/integration/kafka/api/SaslSetup.scala b/core/src/test/scala/integration/kafka/api/SaslSetup.scala index c36b28802b4f8..8255e6a696fdd 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSetup.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSetup.scala @@ -17,11 +17,11 @@ package kafka.api -import java.io.{File} +import java.io.File import javax.security.auth.login.Configuration -import kafka.utils.{JaasTestUtils,TestUtils} -import org.apache.hadoop.minikdc.MiniKdc +import kafka.security.minikdc.MiniKdc +import kafka.utils.{JaasTestUtils, TestUtils} import org.apache.kafka.common.security.JaasUtils import org.apache.kafka.common.security.kerberos.LoginManager @@ -39,8 +39,8 @@ case object Both extends SaslSetupMode * currently to setup a keytab and jaas files. */ trait SaslSetup { - private val workDir = new File(System.getProperty("test.dir", "build/tmp/test-workDir")) - private val kdcConf = MiniKdc.createConf() + private val workDir = TestUtils.tempDir() + private val kdcConf = MiniKdc.createConfig private val kdc = new MiniKdc(kdcConf, workDir) def startSasl(mode: SaslSetupMode = Both) { diff --git a/core/src/test/scala/kafka/security/minikdc/MiniKdc.scala b/core/src/test/scala/kafka/security/minikdc/MiniKdc.scala new file mode 100644 index 0000000000000..14807bcabf819 --- /dev/null +++ b/core/src/test/scala/kafka/security/minikdc/MiniKdc.scala @@ -0,0 +1,433 @@ +/** + * 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.security.minikdc + +import java.io._ +import java.net.InetSocketAddress +import java.nio.charset.StandardCharsets +import java.nio.file.Files +import java.text.MessageFormat +import java.util.{Locale, Properties, UUID} + +import kafka.utils.{CoreUtils, Logging} + +import scala.collection.JavaConverters._ +import org.apache.commons.io.IOUtils +import org.apache.commons.lang.text.StrSubstitutor +import org.apache.directory.api.ldap.model.entry.{DefaultEntry, Entry} +import org.apache.directory.api.ldap.model.ldif.LdifReader +import org.apache.directory.api.ldap.model.name.Dn +import org.apache.directory.api.ldap.schema.extractor.impl.DefaultSchemaLdifExtractor +import org.apache.directory.api.ldap.schema.loader.LdifSchemaLoader +import org.apache.directory.api.ldap.schema.manager.impl.DefaultSchemaManager +import org.apache.directory.server.constants.ServerDNConstants +import org.apache.directory.server.core.DefaultDirectoryService +import org.apache.directory.server.core.api.{CacheService, DirectoryService, InstanceLayout} +import org.apache.directory.server.core.api.schema.SchemaPartition +import org.apache.directory.server.core.kerberos.KeyDerivationInterceptor +import org.apache.directory.server.core.partition.impl.btree.jdbm.{JdbmIndex, JdbmPartition} +import org.apache.directory.server.core.partition.ldif.LdifPartition +import org.apache.directory.server.kerberos.KerberosConfig +import org.apache.directory.server.kerberos.kdc.KdcServer +import org.apache.directory.server.kerberos.shared.crypto.encryption.KerberosKeyFactory +import org.apache.directory.server.kerberos.shared.keytab.{Keytab, KeytabEntry} +import org.apache.directory.server.protocol.shared.transport.{TcpTransport, UdpTransport} +import org.apache.directory.server.xdbm.Index +import org.apache.directory.shared.kerberos.KerberosTime +import org.apache.kafka.common.utils.Utils + +/** + * Mini KDC based on Apache Directory Server that can be embedded in tests or used from command line as a standalone + * KDC. + * + * MiniKdc sets 2 System properties when started and unsets them when stopped: + * + * - java.security.krb5.conf: set to the MiniKDC real/host/port + * - sun.security.krb5.debug: set to the debug value provided in the configuration + * + * As a result of this, multiple MiniKdc instances should not be started concurrently in the same JVM. + * + * MiniKdc default configuration values are: + * + * - org.name=EXAMPLE (used to create the REALM) + * - org.domain=COM (used to create the REALM) + * - kdc.bind.address=localhost + * - kdc.port=0 (ephemeral port) + * - instance=DefaultKrbServer + * - max.ticket.lifetime=86400000 (1 day) + * - max.renewable.lifetime604800000 (7 days) + * - transport=TCP + * - debug=false + * + * The generated krb5.conf forces TCP connections. + * + * Acknowledgements: this class is derived from the MiniKdc class in the hadoop-minikdc project (git commit + * d8d8ed35f00b15ee0f2f8aaf3fe7f7b42141286b). + * + * @constructor creates a new MiniKdc instance. + * @param config the MiniKdc configuration + * @param workDir the working directory which will contain krb5.conf, Apache DS files and any other files needed by + * MiniKdc. + * @throws Exception thrown if the MiniKdc could not be created. + */ +class MiniKdc(config: Properties, workDir: File) extends Logging { + + if (!config.keySet.containsAll(MiniKdc.RequiredProperties.asJava)) { + val missingProperties = MiniKdc.RequiredProperties.filterNot(config.keySet.asScala) + throw new IllegalArgumentException(s"Missing configuration properties: $missingProperties") + } + + info("Configuration:") + info("---------------------------------------------------------------") + config.asScala.foreach { case (key, value) => + info(s"\t$key: $value") + } + info("---------------------------------------------------------------") + + private val orgName = config.getProperty(MiniKdc.OrgName) + private val orgDomain = config.getProperty(MiniKdc.OrgDomain) + private val dnString = s"dc=$orgName,dc=$orgDomain" + private val realm = s"${orgName.toUpperCase(Locale.ENGLISH)}.${orgDomain.toUpperCase(Locale.ENGLISH)}" + private val krb5conf = new File(workDir, "krb5.conf") + + private var _port = config.getProperty(MiniKdc.KdcPort).toInt + private var ds: DirectoryService = null + private var kdc: KdcServer = null + + def port: Int = _port + + def host: String = config.getProperty(MiniKdc.KdcBindAddress) + + def start() { + if (kdc != null) + throw new RuntimeException("KDC already started") + initDirectoryService() + initKdcServer() + initJvmKerberosConfig() + } + + private def initDirectoryService() { + ds = new DefaultDirectoryService + ds.setInstanceLayout(new InstanceLayout(workDir)) + ds.setCacheService(new CacheService) + + // first load the schema + val instanceLayout = ds.getInstanceLayout + val schemaPartitionDirectory = new File(instanceLayout.getPartitionsDirectory, "schema") + val extractor = new DefaultSchemaLdifExtractor(instanceLayout.getPartitionsDirectory) + extractor.extractOrCopy + + val loader = new LdifSchemaLoader(schemaPartitionDirectory) + val schemaManager = new DefaultSchemaManager(loader) + schemaManager.loadAllEnabled() + ds.setSchemaManager(schemaManager) + // Init the LdifPartition with schema + val schemaLdifPartition = new LdifPartition(schemaManager, ds.getDnFactory) + schemaLdifPartition.setPartitionPath(schemaPartitionDirectory.toURI) + + // The schema partition + val schemaPartition = new SchemaPartition(schemaManager) + schemaPartition.setWrappedPartition(schemaLdifPartition) + ds.setSchemaPartition(schemaPartition) + + val systemPartition = new JdbmPartition(ds.getSchemaManager, ds.getDnFactory) + systemPartition.setId("system") + systemPartition.setPartitionPath(new File(ds.getInstanceLayout.getPartitionsDirectory, systemPartition.getId).toURI) + systemPartition.setSuffixDn(new Dn(ServerDNConstants.SYSTEM_DN)) + systemPartition.setSchemaManager(ds.getSchemaManager) + ds.setSystemPartition(systemPartition) + + ds.getChangeLog.setEnabled(false) + ds.setDenormalizeOpAttrsEnabled(true) + ds.addLast(new KeyDerivationInterceptor) + + // create one partition + val orgName = config.getProperty(MiniKdc.OrgName).toLowerCase(Locale.ENGLISH) + val orgDomain = config.getProperty(MiniKdc.OrgDomain).toLowerCase(Locale.ENGLISH) + val partition = new JdbmPartition(ds.getSchemaManager, ds.getDnFactory) + partition.setId(orgName) + partition.setPartitionPath(new File(ds.getInstanceLayout.getPartitionsDirectory, orgName).toURI) + val dn = new Dn(dnString) + partition.setSuffixDn(dn) + ds.addPartition(partition) + + // indexes + val indexedAttributes = Set[Index[_, String]]( + new JdbmIndex[Entry]("objectClass", false), + new JdbmIndex[Entry]("dc", false), + new JdbmIndex[Entry]("ou", false) + ).asJava + partition.setIndexedAttributes(indexedAttributes) + + // And start the ds + ds.setInstanceId(config.getProperty(MiniKdc.Instance)) + ds.startup() + + // context entry, after ds.startup() + val entry = ds.newEntry(dn) + entry.add("objectClass", "top", "domain") + entry.add("dc", orgName) + ds.getAdminSession.add(entry) + } + + private def initKdcServer() { + + def addInitialEntriesToDirectoryService(bindAddress: String) { + val map = Map ( + "0" -> orgName.toLowerCase(Locale.ENGLISH), + "1" -> orgDomain.toLowerCase(Locale.ENGLISH), + "2" -> orgName.toUpperCase(Locale.ENGLISH), + "3" -> orgDomain.toUpperCase(Locale.ENGLISH), + "4" -> bindAddress + ) + val inputStream = MiniKdc.getResourceAsStream("minikdc.ldiff") + try addEntriesToDirectoryService(StrSubstitutor.replace(IOUtils.toString(inputStream), map.asJava)) + finally CoreUtils.swallow(inputStream.close()) + } + + val bindAddress = config.getProperty(MiniKdc.KdcBindAddress) + addInitialEntriesToDirectoryService(bindAddress) + + val kerberosConfig = new KerberosConfig + kerberosConfig.setMaximumRenewableLifetime(config.getProperty(MiniKdc.MaxRenewableLifetime).toLong) + kerberosConfig.setMaximumTicketLifetime(config.getProperty(MiniKdc.MaxTicketLifetime).toLong) + kerberosConfig.setSearchBaseDn(dnString) + kerberosConfig.setPaEncTimestampRequired(false) + kdc = new KdcServer(kerberosConfig) + kdc.setDirectoryService(ds) + + // transport + val transport = config.getProperty(MiniKdc.Transport) + val absTransport = transport.trim match { + case "TCP" => new TcpTransport(bindAddress, port, 3, 50) + case "UDP" => new UdpTransport(port) + case _ => throw new IllegalArgumentException(s"Invalid transport: $transport") + } + kdc.addTransports(absTransport) + kdc.setServiceName(config.getProperty(MiniKdc.Instance)) + kdc.start() + + // if using ephemeral port, update port number for binding + if (port == 0) + _port = absTransport.getAcceptor.getLocalAddress.asInstanceOf[InetSocketAddress].getPort + + info(s"MiniKdc listening at port: $port") + } + + private def initJvmKerberosConfig(): Unit = { + writeKrb5Conf() + System.setProperty(MiniKdc.JavaSecurityKrb5Conf, krb5conf.getAbsolutePath) + System.setProperty(MiniKdc.SunSecurityKrb5Debug, config.getProperty(MiniKdc.Debug, "false")) + info(s"MiniKdc setting JVM krb5.conf to: ${krb5conf.getAbsolutePath}") + refreshJvmKerberosConfig() + } + + private def writeKrb5Conf() { + val stringBuilder = new StringBuilder + val reader = new BufferedReader( + new InputStreamReader(MiniKdc.getResourceAsStream("minikdc-krb5.conf"), StandardCharsets.UTF_8)) + try { + var line: String = null + while ({line = reader.readLine(); line != null}) { + stringBuilder.append(line).append("{3}") + } + } finally CoreUtils.swallow(reader.close()) + val output = MessageFormat.format(stringBuilder.toString, realm, host, port.toString, System.lineSeparator()) + Files.write(krb5conf.toPath, output.getBytes(StandardCharsets.UTF_8)) + } + + private def refreshJvmKerberosConfig(): Unit = { + val klass = + if (System.getProperty("java.vendor").contains("IBM")) + Class.forName("com.ibm.security.krb5.internal.Config") + else + Class.forName("sun.security.krb5.Config") + klass.getMethod("refresh").invoke(klass) + } + + def stop() { + if (kdc != null) { + System.clearProperty(MiniKdc.JavaSecurityKrb5Conf) + System.clearProperty(MiniKdc.SunSecurityKrb5Debug) + kdc.stop() + try ds.shutdown() + catch { + case ex: Exception => error("Could not shutdown ApacheDS properly", ex) + } + } + } + + /** + * Creates a principal in the KDC with the specified user and password. + * + * An exception will be thrown if the principal cannot be created. + * + * @param principal principal name, do not include the domain. + * @param password password. + */ + private def createPrincipal(principal: String, password: String) { + val ldifContent = s""" + |dn: uid=$principal,ou=users,dc=${orgName.toLowerCase(Locale.ENGLISH)},dc=${orgDomain.toLowerCase(Locale.ENGLISH)} + |objectClass: top + |objectClass: person + |objectClass: inetOrgPerson + |objectClass: krb5principal + |objectClass: krb5kdcentry + |cn: $principal + |sn: $principal + |uid: $principal + |userPassword: $password + |krb5PrincipalName: ${principal}@${realm} + |krb5KeyVersionNumber: 0""".stripMargin + addEntriesToDirectoryService(ldifContent) + } + + /** + * Creates multiple principals in the KDC and adds them to a keytab file. + * + * An exception will be thrown if the principal cannot be created. + * + * @param keytabFile keytab file to add the created principals + * @param principals principals to add to the KDC, do not include the domain. + */ + def createPrincipal(keytabFile: File, principals: String*) { + val generatedPassword = UUID.randomUUID.toString + val keytab = new Keytab + val entries = principals.flatMap { principal => + createPrincipal(principal, generatedPassword) + val principalWithRealm = s"${principal}@${realm}" + val timestamp = new KerberosTime + KerberosKeyFactory.getKerberosKeys(principalWithRealm, generatedPassword).asScala.values.map { encryptionKey => + val keyVersion = encryptionKey.getKeyVersion.toByte + new KeytabEntry(principalWithRealm, 1, timestamp, keyVersion, encryptionKey) + } + } + keytab.setEntries(entries.asJava) + keytab.write(keytabFile) + } + + private def addEntriesToDirectoryService(ldifContent: String): Unit = { + val reader = new LdifReader(new StringReader(ldifContent)) + try { + for (ldifEntry <- reader.asScala) + ds.getAdminSession.add(new DefaultEntry(ds.getSchemaManager, ldifEntry.getEntry)) + } finally CoreUtils.swallow(reader.close()) + } + +} + +object MiniKdc { + + val JavaSecurityKrb5Conf = "java.security.krb5.conf" + val SunSecurityKrb5Debug = "sun.security.krb5.debug" + + def main(args: Array[String]) { + args match { + case Array(workDirPath, configPath, keytabPath, principals@ _*) if principals.nonEmpty => + val workDir = new File(workDirPath) + if (!workDir.exists) + throw new RuntimeException(s"Specified work directory does not exist: ${workDir.getAbsolutePath}") + val config = createConfig + val configFile = new File(configPath) + if (!configFile.exists) + throw new RuntimeException(s"Specified configuration does not exist: ${configFile.getAbsolutePath}") + + val userConfig = Utils.loadProps(configFile.getAbsolutePath) + userConfig.asScala.foreach { case (key, value) => + config.put(key, value) + } + val keytabFile = new File(keytabPath).getAbsoluteFile + start(workDir, config, keytabFile, principals) + case _ => + println("Arguments: []+") + sys.exit(1) + } + } + + private def start(workDir: File, config: Properties, keytabFile: File, principals: Seq[String]) { + val miniKdc = new MiniKdc(config, workDir) + miniKdc.start() + miniKdc.createPrincipal(keytabFile, principals: _*) + val infoMessage = s""" + | + |Standalone MiniKdc Running + |--------------------------------------------------- + | Realm : ${miniKdc.realm} + | Running at : ${miniKdc.host}:${miniKdc.port} + | krb5conf : ${miniKdc.krb5conf} + | + | created keytab : $keytabFile + | with principals : ${principals.mkString(", ")} + | + |Hit or kill to stop it + |--------------------------------------------------- + | + """.stripMargin + println(infoMessage) + Runtime.getRuntime.addShutdownHook(CoreUtils.newThread("minikdc-shutdown-hook", daemon = false) { + miniKdc.stop() + }) + } + + val OrgName = "org.name" + val OrgDomain = "org.domain" + val KdcBindAddress = "kdc.bind.address" + val KdcPort = "kdc.port" + val Instance = "instance" + val MaxTicketLifetime = "max.ticket.lifetime" + val MaxRenewableLifetime = "max.renewable.lifetime" + val Transport = "transport" + val Debug = "debug" + + private val RequiredProperties = Set(OrgName, OrgDomain, KdcBindAddress, KdcPort, Instance, Transport, + MaxTicketLifetime, MaxRenewableLifetime) + + private val DefaultConfig = Map( + KdcBindAddress -> "localhost", + KdcPort -> "0", + Instance -> "DefaultKrbServer", + OrgName -> "Example", + OrgDomain -> "COM", + Transport -> "TCP", + MaxTicketLifetime -> "86400000", + MaxRenewableLifetime -> "604800000", + Debug -> "false" + ) + + /** + * Convenience method that returns MiniKdc default configuration. + * + * The returned configuration is a copy, it can be customized before using + * it to create a MiniKdc. + */ + def createConfig: Properties = { + val properties = new Properties + DefaultConfig.foreach { case (k, v) => properties.setProperty(k, v) } + properties + } + + @throws[IOException] + def getResourceAsStream(resourceName: String): InputStream = { + val cl = Option(Thread.currentThread.getContextClassLoader).getOrElse(classOf[MiniKdc].getClassLoader) + Option(cl.getResourceAsStream(resourceName)).getOrElse { + throw new IOException(s"Can not read resource file `$resourceName`") + } + } + +} diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index 676772f331188..2ca64f2fc1143 100755 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -77,17 +77,15 @@ trait KafkaServerTestHarness extends ZooKeeperTestHarness { // The following method does nothing by default, but // if the test case requires setting up a cluster ACL, // then it needs to be implemented. - setClusterAcl match { - case Some(f) => - f() - case None => // Nothing to do - } + setClusterAcl.foreach(_.apply) } @After override def tearDown() { - servers.foreach(_.shutdown()) - servers.foreach(server => CoreUtils.delete(server.config.logDirs)) + if (servers != null) { + servers.foreach(_.shutdown()) + servers.foreach(server => CoreUtils.delete(server.config.logDirs)) + } super.tearDown } diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 47158d64bf910..6ed317a216bc0 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -23,6 +23,8 @@ ext { } versions += [ + apacheda: "1.0.0-M33", + apacheds: "2.0.0-M21", argparse4j: "0.5.0", bcpkix: "1.54", hadoop: "2.7.2", @@ -65,8 +67,16 @@ versions["baseScala"] = versions.scala.substring(0, versions.scala.lastIndexOf(" libs += [ argparse4j: "net.sourceforge.argparse4j:argparse4j:$versions.argparse4j", + apacheda: "org.apache.directory.api:api-all:$versions.apacheda", + apachedsCoreApi: "org.apache.directory.server:apacheds-core-api:$versions.apacheds", + apachedsInterceptorKerberos: "org.apache.directory.server:apacheds-interceptor-kerberos:$versions.apacheds", + apachedsProtocolShared: "org.apache.directory.server:apacheds-protocol-shared:$versions.apacheds", + apachedsProtocolKerberos: "org.apache.directory.server:apacheds-protocol-kerberos:$versions.apacheds", + apachedsProtocolLdap: "org.apache.directory.server:apacheds-protocol-ldap:$versions.apacheds", + apachedsLdifPartition: "org.apache.directory.server:apacheds-ldif-partition:$versions.apacheds", + apachedsMavibotPartition: "org.apache.directory.server:apacheds-mavibot-partition:$versions.apacheds", + apachedsJdbmPartition: "org.apache.directory.server:apacheds-jdbm-partition:$versions.apacheds", bcpkix: "org.bouncycastle:bcpkix-jdk15on:$versions.bcpkix", - hadoopMiniKdc: "org.apache.hadoop:hadoop-minikdc:$versions.hadoop", easymock: "org.easymock:easymock:$versions.easymock", jacksonDatabind: "com.fasterxml.jackson.core:jackson-databind:$versions.jackson", jacksonJaxrsJsonProvider: "com.fasterxml.jackson.jaxrs:jackson-jaxrs-json-provider:$versions.jackson", diff --git a/tests/kafkatest/services/security/minikdc.py b/tests/kafkatest/services/security/minikdc.py index 3b3a5f1656025..b376e268f7f8b 100644 --- a/tests/kafkatest/services/security/minikdc.py +++ b/tests/kafkatest/services/security/minikdc.py @@ -67,10 +67,9 @@ def start_node(self, node): principals = 'client ' + kafka_principals + self.extra_principals self.logger.info("Starting MiniKdc with principals " + principals) - lib_dir = "/opt/%s/core/build/dependant-testlibs" % kafka_dir(node) - kdc_jars = node.account.ssh_capture("ls " + lib_dir) - classpath = ":".join([os.path.join(lib_dir, jar.strip()) for jar in kdc_jars]) - cmd = "CLASSPATH=%s /opt/%s/bin/kafka-run-class.sh org.apache.hadoop.minikdc.MiniKdc %s %s %s %s 1>> %s 2>> %s &" % (classpath, kafka_dir(node), MiniKdc.WORK_DIR, MiniKdc.PROPS_FILE, MiniKdc.KEYTAB_FILE, principals, MiniKdc.LOG_FILE, MiniKdc.LOG_FILE) + jar_paths = self.core_jar_paths(node, "dependant-testlibs") + self.core_jar_paths(node, "libs") + classpath = ":".join(jar_paths) + cmd = "CLASSPATH=%s /opt/%s/bin/kafka-run-class.sh kafka.security.minikdc.MiniKdc %s %s %s %s 1>> %s 2>> %s &" % (classpath, kafka_dir(node), MiniKdc.WORK_DIR, MiniKdc.PROPS_FILE, MiniKdc.KEYTAB_FILE, principals, MiniKdc.LOG_FILE, MiniKdc.LOG_FILE) self.logger.debug("Attempting to start MiniKdc on %s with command: %s" % (str(node.account), cmd)) with node.account.monitor_log(MiniKdc.LOG_FILE) as monitor: node.account.ssh(cmd) @@ -82,6 +81,11 @@ def start_node(self, node): #KDC is set to bind openly (via 0.0.0.0). Change krb5.conf to hold the specific KDC address self.replace_in_file(MiniKdc.LOCAL_KRB5CONF_FILE, '0.0.0.0', node.account.hostname) + def core_jar_paths(self, node, lib_dir_name): + lib_dir = "/opt/%s/core/build/%s" % (kafka_dir(node), lib_dir_name) + jars = node.account.ssh_capture("ls " + lib_dir) + return [os.path.join(lib_dir, jar.strip()) for jar in jars] + def stop_node(self, node): self.logger.info("Stopping %s on %s" % (type(self).__name__, node.account.hostname)) node.account.kill_process("apacheds", allow_fail=False) From 9f6a6f97134a1d4969c91c4b4e9037b376e03440 Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Wed, 30 Mar 2016 19:33:37 -0700 Subject: [PATCH 100/206] KAFKA-3381: Add system test for SimpleConsumerShell Author: Ashish Singh Reviewers: Geoff Anderson Closes #1053 from SinghAsDev/KAFKA-3381 --- .../services/simple_consumer_shell.py | 69 +++++++++++++++++ .../tests/simple_consumer_shell_test.py | 75 +++++++++++++++++++ 2 files changed, 144 insertions(+) create mode 100644 tests/kafkatest/services/simple_consumer_shell.py create mode 100644 tests/kafkatest/tests/simple_consumer_shell_test.py diff --git a/tests/kafkatest/services/simple_consumer_shell.py b/tests/kafkatest/services/simple_consumer_shell.py new file mode 100644 index 0000000000000..8deee85d0d273 --- /dev/null +++ b/tests/kafkatest/services/simple_consumer_shell.py @@ -0,0 +1,69 @@ +# 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. + +from ducktape.services.background_thread import BackgroundThreadService + +from kafkatest.services.kafka.directory import kafka_dir + + +class SimpleConsumerShell(BackgroundThreadService): + + logs = { + "simple_consumer_shell_log": { + "path": "/mnt/simple_consumer_shell.log", + "collect_default": False} + } + + def __init__(self, context, num_nodes, kafka, topic, partition=0): + super(SimpleConsumerShell, self).__init__(context, num_nodes) + + self.kafka = kafka + self.topic = topic + self.partition = partition + self.output = "" + + def _worker(self, idx, node): + cmd = self.start_cmd(node) + self.logger.debug("SimpleConsumerShell %d command: %s" % (idx, cmd)) + self.output = "" + self.logger.debug(cmd) + for line in node.account.ssh_capture(cmd): + self.output += line + self.logger.debug(self.output) + + def start_cmd(self, node): + cmd = "/opt/%s/bin/" % kafka_dir(node) + cmd += "kafka-run-class.sh kafka.tools.SimpleConsumerShell" + cmd += " --topic %s --broker-list %s --partition %s --no-wait-at-logend" % (self.topic, self.kafka.bootstrap_servers(), self.partition) + + cmd += " 2>> /mnt/get_simple_consumer_shell.log | tee -a /mnt/get_simple_consumer_shell.log &" + return cmd + + def get_output(self): + return self.output + + def stop_node(self, node): + node.account.kill_process("SimpleConsumerShell", allow_fail=False) + if self.worker_threads is None: + return + + # block until the corresponding thread exits + if len(self.worker_threads) >= self.idx(node): + # Need to guard this because stop is preemptively called before the worker threads are added and started + self.worker_threads[self.idx(node) - 1].join() + + def clean_node(self, node): + node.account.kill_process("SimpleConsumerShell", clean_shutdown=False, allow_fail=False) + node.account.ssh("rm -rf /mnt/simple_consumer_shell.log", allow_fail=False) diff --git a/tests/kafkatest/tests/simple_consumer_shell_test.py b/tests/kafkatest/tests/simple_consumer_shell_test.py new file mode 100644 index 0000000000000..74a7eeb911e30 --- /dev/null +++ b/tests/kafkatest/tests/simple_consumer_shell_test.py @@ -0,0 +1,75 @@ +# 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. + + +from ducktape.utils.util import wait_until +from ducktape.tests.test import Test +from kafkatest.services.simple_consumer_shell import SimpleConsumerShell +from kafkatest.services.verifiable_producer import VerifiableProducer + +from kafkatest.services.zookeeper import ZookeeperService +from kafkatest.services.kafka import KafkaService +TOPIC = "topic-simple-consumer-shell" +MAX_MESSAGES = 100 +NUM_PARTITIONS = 1 +REPLICATION_FACTOR = 1 + +class SimpleConsumerShellTest(Test): + """ + Tests SimpleConsumerShell tool + """ + def __init__(self, test_context): + super(SimpleConsumerShellTest, self).__init__(test_context) + self.num_zk = 1 + self.num_brokers = 1 + self.messages_received_count = 0 + self.topics = { + TOPIC: {'partitions': NUM_PARTITIONS, 'replication-factor': REPLICATION_FACTOR} + } + + self.zk = ZookeeperService(test_context, self.num_zk) + + def setUp(self): + self.zk.start() + + def start_kafka(self): + self.kafka = KafkaService( + self.test_context, self.num_brokers, + self.zk, topics=self.topics) + self.kafka.start() + + def run_producer(self): + # This will produce to kafka cluster + self.producer = VerifiableProducer(self.test_context, num_nodes=1, kafka=self.kafka, topic=TOPIC, throughput=1000, max_messages=MAX_MESSAGES) + self.producer.start() + wait_until(lambda: self.producer.num_acked == MAX_MESSAGES, timeout_sec=10, + err_msg="Timeout awaiting messages to be produced and acked") + + def start_simple_consumer_shell(self): + self.simple_consumer_shell = SimpleConsumerShell(self.test_context, 1, self.kafka, TOPIC) + self.simple_consumer_shell.start() + + def test_simple_consumer_shell(self): + """ + Tests if SimpleConsumerShell is fetching expected records + :return: None + """ + self.start_kafka() + self.run_producer() + self.start_simple_consumer_shell() + + # Assert that SimpleConsumerShell is fetching expected number of messages + wait_until(lambda: self.simple_consumer_shell.get_output().count("\n") == (MAX_MESSAGES + 1), timeout_sec=10, + err_msg="Timed out waiting to receive expected number of messages.") \ No newline at end of file From 623ab1e7c6497c000bc9c9978637f20542a3191c Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Thu, 31 Mar 2016 10:07:54 -0700 Subject: [PATCH 101/206] KAFKA-3451: Add basic HTML coverage report generation to gradle Author: Grant Henke Reviewers: Gwen Shapira, Ismael Juma, Ewen Cheslack-Postava Closes #1121 from granthenke/coverage --- README.md | 3 + build.gradle | 59 +++++++++++++++++++ core/src/main/scala/kafka/utils/ZkUtils.scala | 11 ++-- gradle/dependencies.gradle | 3 + 4 files changed, 71 insertions(+), 5 deletions(-) diff --git a/README.md b/README.md index bc2ca3afb07e1..97c5fef392e2a 100644 --- a/README.md +++ b/README.md @@ -47,6 +47,9 @@ Change the log4j setting in either `clients/src/test/resources/log4j.properties` ./gradlew -i -Dtest.single=RequestResponseSerializationTest core:test +### Generating test coverage reports ### + ./gradlew reportCoverage + ### Building a binary release gzipped tar ball ### ./gradlew clean ./gradlew releaseTarGz diff --git a/build.gradle b/build.gradle index d6f82a4f91298..da0152b990f97 100644 --- a/build.gradle +++ b/build.gradle @@ -26,12 +26,15 @@ buildscript { // For Apache Rat plugin to ignore non-Git files classpath "org.ajoberstar:grgit:1.5.0" classpath 'com.github.ben-manes:gradle-versions-plugin:0.12.0' + classpath 'org.scoverage:gradle-scoverage:2.0.1' } } allprojects { apply plugin: 'idea' apply plugin: 'eclipse' + apply plugin: "jacoco" + repositories { mavenCentral() } @@ -249,8 +252,50 @@ subprojects { configProperties = [importControlFile: "$rootDir/checkstyle/import-control.xml"] } test.dependsOn('checkstyleMain', 'checkstyleTest') + + // Ignore core since its a scala project + if (it.path != ':core') { + // NOTE: Gradles Jacoco plugin does not support "offline instrumentation" this means that classes mocked by PowerMock + // may report 0 coverage, since the source was modified after initial instrumentation. + // See https://github.com/jacoco/jacoco/issues/51 + jacocoTestReport { + dependsOn tasks.test + sourceSets sourceSets.main + reports { + html.enabled = true + xml.enabled = true + csv.enabled = false + } + } + } +} + +// Aggregates all jacoco results into the root project directory +task jacocoRootReport(type: org.gradle.testing.jacoco.tasks.JacocoReport) { + def javaProjects = subprojects.findAll { it.path != ':core' } + + description = 'Generates an aggregate report from all subprojects' + dependsOn(javaProjects.test) + + additionalSourceDirs = files(javaProjects.sourceSets.main.allSource.srcDirs) + sourceDirectories = files(javaProjects.sourceSets.main.allSource.srcDirs) + classDirectories = files(javaProjects.sourceSets.main.output) + executionData = files(javaProjects.jacocoTestReport.executionData) + + reports { + html.enabled = true + xml.enabled = true + } + + // workaround to ignore projects that don't have any tests at all + onlyIf = { true } + doFirst { + executionData = files(executionData.findAll { it.exists() }) + } } +task reportCoverage(dependsOn: ['jacocoRootReport', 'core:reportScoverage']) + for ( sv in ['2_10', '2_11'] ) { String svInDot = sv.replaceAll( "_", ".") @@ -320,6 +365,7 @@ project(':core') { println "Building project 'core' with Scala version ${versions.scala}" apply plugin: 'scala' + apply plugin: "org.scoverage" archivesBaseName = "kafka_${versions.baseScala}" dependencies { @@ -351,7 +397,20 @@ project(':core') { testCompile libs.apachedsJdbmPartition testCompile libs.junit testCompile libs.scalaTest + + scoverage libs.scoveragePlugin + scoverage libs.scoverageRuntime + } + + jacocoTestReport.enabled = false + scoverage { + reportDir = file("${rootProject.buildDir}/scoverage") + highlighting = false + } + checkScoverage { + minimumRate = 0.0 } + checkScoverage.shouldRunAfter('test') configurations { // manually excludes some unnecessary dependencies diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 49d3cfaaf8bfe..bd8ec7ec87765 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -452,12 +452,13 @@ class ZkUtils(val zkClient: ZkClient, } catch { case e1: ZkBadVersionException => optionalChecker match { - case Some(checker) => return checker(this, path, data) - case _ => debug("Checker method is not passed skipping zkData match") + case Some(checker) => checker(this, path, data) + case _ => + debug("Checker method is not passed skipping zkData match") + warn("Conditional update of path %s with data %s and expected version %d failed due to %s" + .format(path, data,expectVersion, e1.getMessage)) + (false, -1) } - warn("Conditional update of path %s with data %s and expected version %d failed due to %s".format(path, data, - expectVersion, e1.getMessage)) - (false, -1) case e2: Exception => warn("Conditional update of path %s with data %s and expected version %d failed due to %s".format(path, data, expectVersion, e2.getMessage)) diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index 6ed317a216bc0..f0358385e93af 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -41,6 +41,7 @@ versions += [ rocksDB: "4.1.0", scalaTest: "2.2.6", scalaParserCombinators: "1.0.4", + scoverage: "1.1.1", slf4j: "1.7.18", snappy: "1.1.2.1", zkclient: "0.8", @@ -96,6 +97,8 @@ libs += [ scalaCompiler: "org.scala-lang:scala-compiler:$versions.scala", scalaTest: "org.scalatest:scalatest_$versions.baseScala:$versions.scalaTest", scalaParserCombinators: "org.scala-lang.modules:scala-parser-combinators_$versions.baseScala:$versions.scalaParserCombinators", + scoveragePlugin: "org.scoverage:scalac-scoverage-plugin_$versions.baseScala:$versions.scoverage", + scoverageRuntime: "org.scoverage:scalac-scoverage-runtime_$versions.baseScala:$versions.scoverage", slf4jApi: "org.slf4j:slf4j-api:$versions.slf4j", slf4jlog4j: "org.slf4j:slf4j-log4j12:$versions.slf4j", snappy: "org.xerial.snappy:snappy-java:$versions.snappy", From 77142f6baeb35e6de9608a7372113f72ea330936 Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Fri, 1 Apr 2016 10:52:26 -0700 Subject: [PATCH 102/206] KAFKA-2910: Close Zookeeper clients in unit tests Zookeeper clients that are not closed after the server is shutdown keep trying to reconnect, reloading JAAS configuration. This impacts subsequent tests which rely on JAAS config to be reset. Author: Rajini Sivaram Reviewers: Flavio Junqueira , Ewen Cheslack-Postava Closes #1171 from rajinisivaram/KAFKA-2910 --- core/src/test/scala/unit/kafka/utils/TestUtils.scala | 1 + core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala | 4 +++- core/src/test/scala/unit/kafka/zk/ZKPathTest.scala | 8 ++++++++ 3 files changed, 12 insertions(+), 1 deletion(-) diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index a1e79128dda98..be7741df5cbaa 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -585,6 +585,7 @@ object TestUtils extends Logging { def updateConsumerOffset(config : ConsumerConfig, path : String, offset : Long) = { val zkUtils = ZkUtils(config.zkConnect, config.zkSessionTimeoutMs, config.zkConnectionTimeoutMs, false) zkUtils.updatePersistentPath(path, offset.toString) + zkUtils.close() } diff --git a/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala b/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala index fb53d775cf557..32c7a5dc59116 100644 --- a/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala +++ b/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala @@ -91,6 +91,7 @@ class ZKEphemeralTest(val secure: Boolean) extends ZooKeeperTestHarness { zkUtils = ZkUtils(zkConnect, zkSessionTimeoutMs, config.zkConnectionTimeoutMs, JaasUtils.isZkSecurityEnabled()) val nodeExists = zkUtils.pathExists("/tmp/zktest") Assert.assertFalse(nodeExists) + zkUtils.close() } /***** @@ -137,7 +138,7 @@ class ZKEphemeralTest(val secure: Boolean) extends ZooKeeperTestHarness { val zk1 = zkUtils.zkConnection.getZookeeper //Creates a second session - val (_, zkConnection2) = ZkUtils.createZkClientAndConnection(zkConnect, zkSessionTimeoutMs, zkConnectionTimeout) + val (zkClient2, zkConnection2) = ZkUtils.createZkClientAndConnection(zkConnect, zkSessionTimeoutMs, zkConnectionTimeout) val zk2 = zkConnection2.getZookeeper var zwe = new ZKCheckedEphemeral(path, "", zk2, JaasUtils.isZkSecurityEnabled()) @@ -153,6 +154,7 @@ class ZKEphemeralTest(val secure: Boolean) extends ZooKeeperTestHarness { gotException = true } Assert.assertTrue(gotException) + zkClient2.close() } /** diff --git a/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala b/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala index 65dd5899c85ff..92fae022a728c 100644 --- a/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala +++ b/core/src/test/scala/unit/kafka/zk/ZKPathTest.scala @@ -43,6 +43,7 @@ class ZKPathTest extends ZooKeeperTestHarness { case configException: ConfigException => case exception: Throwable => fail("Should have thrown ConfigException") } + zkUtils.close() } @Test @@ -57,6 +58,7 @@ class ZKPathTest extends ZooKeeperTestHarness { } assertTrue("Failed to create persistent path", zkUtils.pathExists(path)) + zkUtils.close() } @Test @@ -73,6 +75,7 @@ class ZKPathTest extends ZooKeeperTestHarness { case configException: ConfigException => case exception: Throwable => fail("Should have thrown ConfigException") } + zkUtils.close() } @Test @@ -87,6 +90,7 @@ class ZKPathTest extends ZooKeeperTestHarness { } assertTrue("Failed to create persistent path", zkUtils.pathExists(path)) + zkUtils.close() } @Test @@ -103,6 +107,7 @@ class ZKPathTest extends ZooKeeperTestHarness { case configException: ConfigException => case exception: Throwable => fail("Should have thrown ConfigException") } + zkUtils.close() } @Test @@ -117,6 +122,7 @@ class ZKPathTest extends ZooKeeperTestHarness { } assertTrue("Failed to create ephemeral path", zkUtils.pathExists(path)) + zkUtils.close() } @Test @@ -133,6 +139,7 @@ class ZKPathTest extends ZooKeeperTestHarness { case configException: ConfigException => case exception: Throwable => fail("Should have thrown ConfigException") } + zkUtils.close() } @Test @@ -149,5 +156,6 @@ class ZKPathTest extends ZooKeeperTestHarness { } assertTrue("Failed to create persistent path", zkUtils.pathExists(actualPath)) + zkUtils.close() } } From ae939467e8aec38f47e2474e74e7ab7ea29c2840 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 1 Apr 2016 13:14:47 -0700 Subject: [PATCH 103/206] MINOR: add null check for aggregate and reduce operators Author: Guozhang Wang Reviewers: Yasuhiro Matsuda, Gwen Shapira Closes #1175 from guozhangwang/KSNullPointerException --- .../kafka/streams/kstream/internals/KStreamAggregate.java | 5 +++++ .../kafka/streams/kstream/internals/KStreamReduce.java | 5 +++++ .../kafka/streams/kstream/internals/KTableAggregate.java | 5 +++++ .../apache/kafka/streams/kstream/internals/KTableReduce.java | 5 +++++ 4 files changed, 20 insertions(+) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java index f41bfa6ac6530..871a12d42dfb4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.kstream.internals; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Initializer; import org.apache.kafka.streams.processor.AbstractProcessor; @@ -62,6 +63,10 @@ public void init(ProcessorContext context) { @Override public void process(K key, V value) { + // the keys should never be null + if (key == null) + throw new StreamsException("Record key for KStream aggregate operator with state " + storeName + " should not be null."); + T oldAgg = store.get(key); if (oldAgg == null) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java index 0ec046533b408..e37fe341289ca 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.kstream.internals; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.Reducer; import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; @@ -59,6 +60,10 @@ public void init(ProcessorContext context) { @Override public void process(K key, V value) { + // the keys should never be null + if (key == null) + throw new StreamsException("Record key for KStream reduce operator with state " + storeName + " should not be null."); + V oldAgg = store.get(key); V newAgg = oldAgg; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java index 6ce776a786710..806c6e95000c6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.kstream.internals; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Initializer; import org.apache.kafka.streams.processor.AbstractProcessor; @@ -64,6 +65,10 @@ public void init(ProcessorContext context) { @Override public void process(K key, Change value) { + // the keys should never be null + if (key == null) + throw new StreamsException("Record key for KTable aggregate operator with state " + storeName + " should not be null."); + T oldAgg = store.get(key); if (oldAgg == null) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java index 0d1b55a8a82c3..d56b3aec1ebba 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.kstream.internals; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.Reducer; import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; @@ -61,6 +62,10 @@ public void init(ProcessorContext context) { @Override public void process(K key, Change value) { + // the keys should never be null + if (key == null) + throw new StreamsException("Record key for KTable reduce operator with state " + storeName + " should not be null."); + V oldAgg = store.get(key); V newAgg = oldAgg; From b865413cd8b71f905ad4ee695f193dcf91ee0595 Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Fri, 1 Apr 2016 13:56:11 -0700 Subject: [PATCH 104/206] MINOR: Fix BNF output for protocol arrays conataining primitives in docs Before this patch arrays containing primitive types were not output: ``` Metadata Request (Version: 0) => [topics] ``` After this patch the type is listed: ``` Metadata Request (Version: 0) => [topics] topics => STRING ``` Author: Grant Henke Reviewers: Ashish Singh, Gwen Shapira Closes #1174 from granthenke/protocol-arrays --- .../java/org/apache/kafka/common/protocol/Protocol.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index 43110b5152899..475a4f2c362e1 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -792,13 +792,13 @@ private static void schemaToBnfHtml(Schema schema, StringBuilder b, int indentSi b.append(field.name); b.append("] "); Type innerType = ((ArrayOf) field.type).type(); - if (innerType instanceof Schema && !subTypes.containsKey(field.name)) - subTypes.put(field.name, (Schema) innerType); + if (!subTypes.containsKey(field.name)) + subTypes.put(field.name, innerType); } else if (field.type instanceof Schema) { b.append(field.name); b.append(" "); if (!subTypes.containsKey(field.name)) - subTypes.put(field.name, (Schema) field.type); + subTypes.put(field.name, field.type); } else { b.append(field.name); b.append(" "); From a4d6f1dce5b1d3f45505d732ad3dc9bcb92bfd98 Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Fri, 1 Apr 2016 14:12:49 -0700 Subject: [PATCH 105/206] MINOR: Add check for empty topics iterator in ReplicaVerificationTool. Author: Ashish Singh Reviewers: Guozhang Wang, Gwen Shapira Closes #1167 from SinghAsDev/minorFixRelicaLagTool --- .../main/scala/kafka/tools/ReplicaVerificationTool.scala | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala index fe4968d8cf008..71bf0c0407d4f 100644 --- a/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala +++ b/core/src/main/scala/kafka/tools/ReplicaVerificationTool.scala @@ -128,6 +128,12 @@ object ReplicaVerificationTool extends Logging { else false ) + + if (filteredTopicMetadata.isEmpty) { + error("No topics found. " + topicWhiteListOpt + ", if specified, is either filtering out all topics or there is no topic.") + System.exit(1) + } + val topicPartitionReplicaList: Seq[TopicPartitionReplica] = filteredTopicMetadata.flatMap( topicMetadataResponse => topicMetadataResponse.partitionsMetadata.flatMap( From 89fd97f8c934b99b1ad994e36435e0686ef7b85a Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Fri, 1 Apr 2016 15:25:35 -0700 Subject: [PATCH 106/206] KAFKA-2844; Separate keytabs for sasl tests Use a different keytab for server and client in SASL tests Also: * Improve approach used to build the JAAS files programmatically * Delete stale `kafka_jaas.conf` file * Move `FourLetterWords` to its own file, add `Zk` prefix and clean-up its usage Author: Ismael Juma Reviewers: Harsha Chintalapani, Gwen Shapira Closes #533 from ijuma/separate-keytabs-for-sasl-tests --- core/src/test/resources/kafka_jaas.conf | 29 ---- .../integration/kafka/api/SaslSetup.scala | 39 ++--- .../security/auth/ZkAuthorizationTest.scala | 47 ++---- .../unit/kafka/utils/JaasTestUtils.scala | 156 +++++++++++------- .../scala/unit/kafka/zk/ZKEphemeralTest.scala | 66 ++++---- .../unit/kafka/zk/ZkFourLetterWords.scala | 47 ++++++ .../unit/kafka/zk/ZooKeeperTestHarness.scala | 48 ++---- 7 files changed, 217 insertions(+), 215 deletions(-) delete mode 100644 core/src/test/resources/kafka_jaas.conf create mode 100644 core/src/test/scala/unit/kafka/zk/ZkFourLetterWords.scala diff --git a/core/src/test/resources/kafka_jaas.conf b/core/src/test/resources/kafka_jaas.conf deleted file mode 100644 index b097e260b7a82..0000000000000 --- a/core/src/test/resources/kafka_jaas.conf +++ /dev/null @@ -1,29 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ -KafkaClient { - com.sun.security.auth.module.Krb5LoginModule required debug=true - useKeyTab=true - storeKey=true - serviceName="kafka" - keyTab="$keytab-location" - principal="client@EXAMPLE.COM"; -}; - -KafkaServer { - com.sun.security.auth.module.Krb5LoginModule required debug=true - useKeyTab=true - storeKey=true - serviceName="kafka" - keyTab="$keytab-location" - principal="kafka/localhost@EXAMPLE.COM"; -}; diff --git a/core/src/test/scala/integration/kafka/api/SaslSetup.scala b/core/src/test/scala/integration/kafka/api/SaslSetup.scala index 8255e6a696fdd..967cae1ea5634 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSetup.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSetup.scala @@ -35,8 +35,7 @@ case object KafkaSasl extends SaslSetupMode case object Both extends SaslSetupMode /* - * Trait used in SaslTestHarness and EndToEndAuthorizationTest - * currently to setup a keytab and jaas files. + * Trait used in SaslTestHarness and EndToEndAuthorizationTest to setup keytab and jaas files. */ trait SaslSetup { private val workDir = TestUtils.tempDir() @@ -46,34 +45,26 @@ trait SaslSetup { def startSasl(mode: SaslSetupMode = Both) { // Important if tests leak consumers, producers or brokers LoginManager.closeAll() - val keytabFile = createKeytabAndSetConfiguration(mode) + val (serverKeytabFile, clientKeytabFile) = createKeytabsAndSetConfiguration(mode) kdc.start() - kdc.createPrincipal(keytabFile, "client", "kafka/localhost") + kdc.createPrincipal(serverKeytabFile, "kafka/localhost") + kdc.createPrincipal(clientKeytabFile, "client") if (mode == Both || mode == ZkSasl) System.setProperty("zookeeper.authProvider.1", "org.apache.zookeeper.server.auth.SASLAuthenticationProvider") } - protected def createKeytabAndSetConfiguration(mode: SaslSetupMode): File = { - val (keytabFile, jaasFile) = createKeytabAndJaasFiles(mode) + protected def createKeytabsAndSetConfiguration(mode: SaslSetupMode): (File, File) = { + val serverKeytabFile = TestUtils.tempFile() + val clientKeytabFile = TestUtils.tempFile() + val jaasFile = mode match { + case ZkSasl => JaasTestUtils.writeZkFile() + case KafkaSasl => JaasTestUtils.writeKafkaFile(serverKeytabFile, clientKeytabFile) + case Both => JaasTestUtils.writeZkAndKafkaFiles(serverKeytabFile, clientKeytabFile) + } // This will cause a reload of the Configuration singleton when `getConfiguration` is called Configuration.setConfiguration(null) - System.setProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM, jaasFile.getAbsolutePath) - keytabFile - } - - private def createKeytabAndJaasFiles(mode: SaslSetupMode): (File, File) = { - val keytabFile = TestUtils.tempFile() - val jaasFileName: String = mode match { - case ZkSasl => - JaasTestUtils.genZkFile - case KafkaSasl => - JaasTestUtils.genKafkaFile(keytabFile.getAbsolutePath) - case _ => - JaasTestUtils.genZkAndKafkaFile(keytabFile.getAbsolutePath) - } - val jaasFile = new File(jaasFileName) - - (keytabFile, jaasFile) + System.setProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM, jaasFile) + (serverKeytabFile, clientKeytabFile) } def closeSasl() { @@ -81,7 +72,7 @@ trait SaslSetup { // Important if tests leak consumers, producers or brokers LoginManager.closeAll() System.clearProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM) - System.clearProperty("zookeeper.authProvider.1"); + System.clearProperty("zookeeper.authProvider.1") Configuration.setConfiguration(null) } } diff --git a/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala b/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala index 6a533b3fdbc54..ab5324c9c3705 100644 --- a/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala +++ b/core/src/test/scala/unit/kafka/security/auth/ZkAuthorizationTest.scala @@ -22,17 +22,17 @@ import kafka.utils.{Logging, ZkUtils} import kafka.zk.ZooKeeperTestHarness import org.apache.kafka.common.KafkaException import org.apache.kafka.common.security.JaasUtils -import org.apache.zookeeper.data.{ACL, Stat} +import org.apache.zookeeper.data.{ACL} import org.junit.Assert._ -import org.junit.{After, Before, BeforeClass, Test} +import org.junit.{After, Before, Test} import scala.collection.JavaConverters._ import scala.util.{Try, Success, Failure} import javax.security.auth.login.Configuration +class ZkAuthorizationTest extends ZooKeeperTestHarness with Logging { + val jaasFile = kafka.utils.JaasTestUtils.writeZkFile + val authProvider = "zookeeper.authProvider.1" -class ZkAuthorizationTest extends ZooKeeperTestHarness with Logging{ - val jaasFile: String = kafka.utils.JaasTestUtils.genZkFile - val authProvider: String = "zookeeper.authProvider.1" @Before override def setUp() { Configuration.setConfiguration(null) @@ -65,12 +65,7 @@ class ZkAuthorizationTest extends ZooKeeperTestHarness with Logging{ JaasUtils.isZkSecurityEnabled() fail("Should have thrown an exception") } catch { - case e: KafkaException => { - // Expected - } - case e: Exception => { - fail(e.toString) - } + case e: KafkaException => // Expected } } @@ -241,10 +236,10 @@ class ZkAuthorizationTest extends ZooKeeperTestHarness with Logging{ case false => list.size == 1 } isListSizeCorrect && list.asScala.forall( - secure match { - case true => isAclSecure - case false => isAclUnsecure - }) + secure match { + case true => isAclSecure + case false => isAclUnsecure + }) } /** @@ -255,15 +250,9 @@ class ZkAuthorizationTest extends ZooKeeperTestHarness with Logging{ private def isAclSecure(acl: ACL): Boolean = { info(s"ACL $acl") acl.getPerms match { - case 1 => { - acl.getId.getScheme.equals("world") - } - case 31 => { - acl.getId.getScheme.equals("sasl") - } - case _: Int => { - false - } + case 1 => acl.getId.getScheme.equals("world") + case 31 => acl.getId.getScheme.equals("sasl") + case _ => false } } @@ -273,12 +262,8 @@ class ZkAuthorizationTest extends ZooKeeperTestHarness with Logging{ private def isAclUnsecure(acl: ACL): Boolean = { info(s"ACL $acl") acl.getPerms match { - case 31 => { - acl.getId.getScheme.equals("world") - } - case _: Int => { - false - } + case 31 => acl.getId.getScheme.equals("world") + case _ => false } } @@ -323,7 +308,7 @@ class ZkAuthorizationTest extends ZooKeeperTestHarness with Logging{ case "/" => result // For all other paths, try to delete it case path => - try{ + try { zkUtils.deletePath(path) Failure(new Exception(s"Have been able to delete $path")) } catch { diff --git a/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala b/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala index cf088302dc850..a14cd3f94cc6b 100644 --- a/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala @@ -16,72 +16,110 @@ */ package kafka.utils +import java.io.{File, BufferedWriter, FileWriter} object JaasTestUtils { - // ZooKeeper vals - val zkServerContextName = "Server" - val zkClientContextName = "Client" - val userSuperPasswd = "adminpasswd" - val user = "fpj" - val userPasswd = "fpjsecret" - val zkModule = "org.apache.zookeeper.server.auth.DigestLoginModule" - //Kafka vals - val kafkaServerContextName = "KafkaServer" - val kafkaClientContextName = "KafkaClient" - val kafkaServerPrincipal = "client@EXAMPLE.COM" - val kafkaClientPrincipal = "kafka/localhost@EXAMPLE.COM" - val kafkaModule = "com.sun.security.auth.module.Krb5LoginModule" - - def genZkFile: String = { - val jaasFile = java.io.File.createTempFile("jaas", ".conf") - val jaasOutputStream = new java.io.FileOutputStream(jaasFile) - writeZkToOutputStream(jaasOutputStream) - jaasOutputStream.close() - jaasFile.deleteOnExit() + + case class Krb5LoginModule(contextName: String, + useKeyTab: Boolean, + storeKey: Boolean, + keyTab: String, + principal: String, + debug: Boolean, + serviceName: Option[String]) { + def toJaasSection: JaasSection = { + JaasSection( + contextName, + "com.sun.security.auth.module.Krb5LoginModule", + debug = debug, + entries = Map( + "useKeyTab" -> useKeyTab.toString, + "storeKey" -> storeKey.toString, + "keyTab" -> keyTab, + "principal" -> principal + ) ++ serviceName.map(s => Map("serviceName" -> s)).getOrElse(Map.empty) + ) + } + } + + case class JaasSection(contextName: String, + moduleName: String, + debug: Boolean, + entries: Map[String, String]) { + override def toString: String = { + s"""|$contextName { + | $moduleName required + | debug=$debug + | ${entries.map { case (k, v) => s"""$k="$v"""" }.mkString("", "\n| ", ";")} + |}; + |""".stripMargin + } + } + + private val ZkServerContextName = "Server" + private val ZkClientContextName = "Client" + private val ZkUserSuperPasswd = "adminpasswd" + private val ZkUser = "fpj" + private val ZkUserPassword = "fpjsecret" + private val ZkModule = "org.apache.zookeeper.server.auth.DigestLoginModule" + + private val KafkaServerContextName = "KafkaServer" + private val KafkaServerPrincipal = "kafka/localhost@EXAMPLE.COM" + private val KafkaClientContextName = "KafkaClient" + private val KafkaClientPrincipal = "client@EXAMPLE.COM" + + def writeZkFile(): String = { + val jaasFile = TestUtils.tempFile() + writeToFile(jaasFile, zkSections) jaasFile.getCanonicalPath } - - def genKafkaFile(keytabLocation: String): String = { - val jaasFile = java.io.File.createTempFile("jaas", ".conf") - val jaasOutputStream = new java.io.FileOutputStream(jaasFile) - writeKafkaToOutputStream(jaasOutputStream, keytabLocation) - jaasOutputStream.close() - jaasFile.deleteOnExit() + + def writeKafkaFile(serverKeyTabLocation: File, clientKeyTabLocation: File): String = { + val jaasFile = TestUtils.tempFile() + writeToFile(jaasFile, kafkaSections(serverKeyTabLocation, clientKeyTabLocation)) jaasFile.getCanonicalPath } - - def genZkAndKafkaFile(keytabLocation: String): String = { - val jaasFile = java.io.File.createTempFile("jaas", ".conf") - val jaasOutputStream = new java.io.FileOutputStream(jaasFile) - writeKafkaToOutputStream(jaasOutputStream, keytabLocation) - jaasOutputStream.write("\n\n".getBytes) - writeZkToOutputStream(jaasOutputStream) - jaasOutputStream.close() - jaasFile.deleteOnExit() + + def writeZkAndKafkaFiles(serverKeyTabLocation: File, clientKeyTabLocation: File): String = { + val jaasFile = TestUtils.tempFile() + writeToFile(jaasFile, kafkaSections(serverKeyTabLocation, clientKeyTabLocation) ++ zkSections) jaasFile.getCanonicalPath } - - private def writeZkToOutputStream(jaasOutputStream: java.io.FileOutputStream) { - jaasOutputStream.write(s"$zkServerContextName {\n\t$zkModule required\n".getBytes) - jaasOutputStream.write(s"""\tuser_super="$userSuperPasswd"\n""".getBytes) - jaasOutputStream.write(s"""\tuser_$user="$userPasswd";\n};\n\n""".getBytes) - jaasOutputStream.write(s"""$zkClientContextName {\n\t$zkModule required\n""".getBytes) - jaasOutputStream.write(s"""\tusername="$user"\n""".getBytes) - jaasOutputStream.write(s"""\tpassword="$userPasswd";\n};""".getBytes) + + private def zkSections: Seq[JaasSection] = Seq( + JaasSection(ZkServerContextName, ZkModule, false, Map("user_super" -> ZkUserSuperPasswd, s"user_$ZkUser" -> ZkUserPassword)), + JaasSection(ZkClientContextName, ZkModule, false, Map("username" -> ZkUser, "password" -> ZkUserPassword)) + ) + + private def kafkaSections(serverKeytabLocation: File, clientKeytabLocation: File): Seq[JaasSection] = { + Seq( + Krb5LoginModule( + KafkaServerContextName, + useKeyTab = true, + storeKey = true, + keyTab = serverKeytabLocation.getAbsolutePath, + principal = KafkaServerPrincipal, + debug = true, + serviceName = Some("kafka")), + Krb5LoginModule( + KafkaClientContextName, + useKeyTab = true, + storeKey = true, + keyTab = clientKeytabLocation.getAbsolutePath, + principal = KafkaClientPrincipal, + debug = true, + serviceName = Some("kafka") + ) + ).map(_.toJaasSection) } - - private def writeKafkaToOutputStream(jaasOutputStream: java.io.FileOutputStream, keytabLocation: String) { - jaasOutputStream.write(s"$kafkaClientContextName {\n\t$kafkaModule required debug=true\n".getBytes) - jaasOutputStream.write(s"\tuseKeyTab=true\n".getBytes) - jaasOutputStream.write(s"\tstoreKey=true\n".getBytes) - jaasOutputStream.write(s"""\tserviceName="kafka"\n""".getBytes) - jaasOutputStream.write(s"""\tkeyTab="$keytabLocation"\n""".getBytes) - jaasOutputStream.write(s"""\tprincipal="$kafkaServerPrincipal";\n};\n\n""".getBytes) - jaasOutputStream.write(s"""$kafkaServerContextName {\n\t$kafkaModule required debug=true\n""".getBytes) - jaasOutputStream.write(s"\tuseKeyTab=true\n".getBytes) - jaasOutputStream.write(s"\tstoreKey=true\n".getBytes) - jaasOutputStream.write(s"""\tserviceName="kafka"\n""".getBytes) - jaasOutputStream.write(s"""\tkeyTab="$keytabLocation"\n""".getBytes) - jaasOutputStream.write(s"""\tprincipal="$kafkaClientPrincipal";\n};""".getBytes) + + private def jaasSectionsToString(jaasSections: Seq[JaasSection]): String = + jaasSections.mkString + + private def writeToFile(file: File, jaasSections: Seq[JaasSection]) { + val writer = new BufferedWriter(new FileWriter(file)) + try writer.write(jaasSectionsToString(jaasSections)) + finally writer.close() } -} \ No newline at end of file + +} diff --git a/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala b/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala index 32c7a5dc59116..c2c25ed4a8bf3 100644 --- a/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala +++ b/core/src/test/scala/unit/kafka/zk/ZKEphemeralTest.scala @@ -17,10 +17,11 @@ package kafka.zk -import java.util.ArrayList -import java.util.Collection +import java.lang.Iterable import javax.security.auth.login.Configuration +import scala.collection.JavaConverters._ + import kafka.consumer.ConsumerConfig import kafka.utils.ZkUtils import kafka.utils.ZKCheckedEphemeral @@ -30,26 +31,24 @@ import org.apache.zookeeper.CreateMode import org.apache.zookeeper.WatchedEvent import org.apache.zookeeper.Watcher import org.apache.zookeeper.ZooDefs.Ids -import org.I0Itec.zkclient.exception.{ZkException,ZkNodeExistsException} +import org.I0Itec.zkclient.exception.ZkNodeExistsException import org.junit.{After, Before, Test, Assert} -import org.junit.runners.Parameterized; -import org.junit.runners.Parameterized.Parameters; -import org.junit.runner.RunWith; +import org.junit.runners.Parameterized +import org.junit.runners.Parameterized.Parameters +import org.junit.runner.RunWith object ZKEphemeralTest { + @Parameters - def enableSecurityOptions: Collection[Array[java.lang.Boolean]] = { - val list = new ArrayList[Array[java.lang.Boolean]]() - list.add(Array(true)) - list.add(Array(false)) - list - } + def enableSecurityOptions: Iterable[Array[java.lang.Boolean]] = + Seq[Array[java.lang.Boolean]](Array(true), Array(false)).asJava + } @RunWith(value = classOf[Parameterized]) class ZKEphemeralTest(val secure: Boolean) extends ZooKeeperTestHarness { - val jaasFile: String = kafka.utils.JaasTestUtils.genZkFile - val authProvider: String = "zookeeper.authProvider.1" + val jaasFile = kafka.utils.JaasTestUtils.writeZkFile() + val authProvider = "zookeeper.authProvider.1" var zkSessionTimeoutMs = 1000 @Before @@ -103,17 +102,14 @@ class ZKEphemeralTest(val secure: Boolean) extends ZooKeeperTestHarness { */ @Test def testZkWatchedEphemeral = { - var path = "/zwe-test" - testCreation(path) - path = "/zwe-test-parent/zwe-test" - testCreation(path) + testCreation("/zwe-test") + testCreation("/zwe-test-parent/zwe-test") } private def testCreation(path: String) { val zk = zkUtils.zkConnection.getZookeeper val zwe = new ZKCheckedEphemeral(path, "", zk, JaasUtils.isZkSecurityEnabled()) var created = false - var counter = 10 zk.exists(path, new Watcher() { def process(event: WatchedEvent) { @@ -140,19 +136,19 @@ class ZKEphemeralTest(val secure: Boolean) extends ZooKeeperTestHarness { //Creates a second session val (zkClient2, zkConnection2) = ZkUtils.createZkClientAndConnection(zkConnect, zkSessionTimeoutMs, zkConnectionTimeout) val zk2 = zkConnection2.getZookeeper - var zwe = new ZKCheckedEphemeral(path, "", zk2, JaasUtils.isZkSecurityEnabled()) + val zwe = new ZKCheckedEphemeral(path, "", zk2, JaasUtils.isZkSecurityEnabled()) // Creates znode for path in the first session zk1.create(path, Array[Byte](), Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL) //Bootstraps the ZKWatchedEphemeral object - var gotException = false; - try { - zwe.create() - } catch { - case e: ZkNodeExistsException => - gotException = true - } + val gotException = + try { + zwe.create() + false + } catch { + case e: ZkNodeExistsException => true + } Assert.assertTrue(gotException) zkClient2.close() } @@ -168,15 +164,15 @@ class ZKEphemeralTest(val secure: Boolean) extends ZooKeeperTestHarness { // Creates znode for path in the first session zk.create(path, Array[Byte](), Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL) - var zwe = new ZKCheckedEphemeral(path, "", zk, JaasUtils.isZkSecurityEnabled()) + val zwe = new ZKCheckedEphemeral(path, "", zk, JaasUtils.isZkSecurityEnabled()) //Bootstraps the ZKWatchedEphemeral object - var gotException = false; - try { - zwe.create() - } catch { - case e: ZkNodeExistsException => - gotException = true - } + val gotException = + try { + zwe.create() + false + } catch { + case e: ZkNodeExistsException => true + } Assert.assertFalse(gotException) } } diff --git a/core/src/test/scala/unit/kafka/zk/ZkFourLetterWords.scala b/core/src/test/scala/unit/kafka/zk/ZkFourLetterWords.scala new file mode 100644 index 0000000000000..6eaee704b5257 --- /dev/null +++ b/core/src/test/scala/unit/kafka/zk/ZkFourLetterWords.scala @@ -0,0 +1,47 @@ +/** + * 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.zk + +import java.io.IOException +import java.net.{SocketTimeoutException, Socket, InetAddress, InetSocketAddress} + +/** + * ZooKeeper responds to a small set of commands. Each command is composed of four letters. You issue the commands to + * ZooKeeper via telnet or nc, at the client port. + * + * Three of the more interesting commands: "stat" gives some general information about the server and connected + * clients, while "srvr" and "cons" give extended details on server and connections respectively. + */ +object ZkFourLetterWords { + def sendStat(host: String, port: Int, timeout: Int) { + val hostAddress = + if (host != null) new InetSocketAddress(host, port) + else new InetSocketAddress(InetAddress.getByName(null), port) + val sock = new Socket() + try { + sock.connect(hostAddress, timeout) + val outStream = sock.getOutputStream + outStream.write("stat".getBytes) + outStream.flush() + } catch { + case e: SocketTimeoutException => throw new IOException("Exception while sending 4lw") + } finally { + sock.close + } + } +} diff --git a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala index d618ba621f6bf..95f4e350954c7 100755 --- a/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala +++ b/core/src/test/scala/unit/kafka/zk/ZooKeeperTestHarness.scala @@ -17,40 +17,12 @@ package kafka.zk -import java.io._ -import java.net._ import javax.security.auth.login.Configuration -import org.I0Itec.zkclient.{ZkClient, ZkConnection} import kafka.utils.{ZkUtils, Logging, CoreUtils} import org.junit.{After, Before} import org.scalatest.junit.JUnitSuite import org.apache.kafka.common.security.JaasUtils -object FourLetterWords { - def sendStat(host: String, port: Int, timeout: Int) { - val hostAddress = if (host != null) - new InetSocketAddress(host, port) - else - new InetSocketAddress(InetAddress.getByName(null), port) - val sock = new Socket() - var reader: BufferedReader = null - sock.connect(hostAddress, timeout) - try { - val outstream = sock.getOutputStream - outstream.write("stat".getBytes) - outstream.flush - } catch { - case e: SocketTimeoutException => { - throw new IOException("Exception while sending 4lw") - } - } finally { - sock.close - if (reader != null) - reader.close - } - } -} - trait ZooKeeperTestHarness extends JUnitSuite with Logging { var zookeeper: EmbeddedZookeeper = null var zkPort: Int = -1 @@ -73,18 +45,20 @@ trait ZooKeeperTestHarness extends JUnitSuite with Logging { CoreUtils.swallow(zkUtils.close()) if (zookeeper != null) CoreUtils.swallow(zookeeper.shutdown()) - - var isDown = false - while(!isDown) { + + def isDown(): Boolean = { try { - FourLetterWords.sendStat("127.0.0.1", zkPort, 3000) - } catch { - case _: Throwable => { - info("Server is down") - isDown = true - } + ZkFourLetterWords.sendStat("127.0.0.1", zkPort, 3000) + false + } catch { case _: Throwable => + debug("Server is down") + true } } + + Iterator.continually(isDown()).exists(identity) + Configuration.setConfiguration(null) } + } From 75ec67eda833e3da2b5925d11ab043d1ef4be9ba Mon Sep 17 00:00:00 2001 From: Flavio Junqueira Date: Fri, 1 Apr 2016 15:57:39 -0700 Subject: [PATCH 107/206] KAFKA-2930: Update references to ZooKeeper in the docs. Author: Flavio Junqueira Reviewers: Ismael Juma, Gwen Shapira Closes #615 from fpj/KAFKA-2930 --- docs/ops.html | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/docs/ops.html b/docs/ops.html index 541a01ddcaf86..b239a0eda550d 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -934,17 +934,17 @@

      Audit

      6.7 ZooKeeper

      Stable version

      -At LinkedIn, we are running ZooKeeper 3.3.*. Version 3.3.3 has known serious issues regarding ephemeral node deletion and session expirations. After running into those issues in production, we upgraded to 3.3.4 and have been running that smoothly for over a year now. +The current stable branch is 3.4 and the latest release of that branch is 3.4.6, which is the one ZkClient 0.7 uses. ZkClient is the client layer Kafka uses to interact with ZooKeeper.

      Operationalizing ZooKeeper

      Operationally, we do the following for a healthy ZooKeeper installation:
        -
      • Redundancy in the physical/hardware/network layout: try not to put them all in the same rack, decent (but don't go nuts) hardware, try to keep redundant power and network paths, etc.
      • -
      • I/O segregation: if you do a lot of write type traffic you'll almost definitely want the transaction logs on a different disk group than application logs and snapshots (the write to the ZooKeeper service has a synchronous write to disk, which can be slow).
      • +
      • Redundancy in the physical/hardware/network layout: try not to put them all in the same rack, decent (but don't go nuts) hardware, try to keep redundant power and network paths, etc. A typical ZooKeeper ensemble has 5 or 7 servers, which tolerates 2 and 3 servers down, respectively. If you have a small deployment, then using 3 servers is acceptable, but keep in mind that you'll only be able to tolerate 1 server down in this case.
      • +
      • I/O segregation: if you do a lot of write type traffic you'll almost definitely want the transaction logs on a dedicated disk group. Writes to the transaction log are synchronous (but batched for performance), and consequently, concurrent writes can significantly affect performance. ZooKeeper snapshots can be one such a source of concurrent writes, and ideally should be written on a disk group separate from the transaction log. Snapshots are writtent to disk asynchronously, so it is typically ok to share with the operating system and message log files. You can configure a server to use a separate disk group with the dataLogDir parameter.
      • Application segregation: Unless you really understand the application patterns of other apps that you want to install on the same box, it can be a good idea to run ZooKeeper in isolation (though this can be a balancing act with the capabilities of the hardware).
      • Use care with virtualization: It can work, depending on your cluster layout and read/write patterns and SLAs, but the tiny overheads introduced by the virtualization layer can add up and throw off ZooKeeper, as it can be very time sensitive
      • -
      • ZooKeeper configuration and monitoring: It's java, make sure you give it 'enough' heap space (We usually run them with 3-5G, but that's mostly due to the data set size we have here). Unfortunately we don't have a good formula for it. As far as monitoring, both JMX and the 4 letter words (4lw) commands are very useful, they do overlap in some cases (and in those cases we prefer the 4 letter commands, they seem more predictable, or at the very least, they work better with the LI monitoring infrastructure)
      • -
      • Don't overbuild the cluster: large clusters, especially in a write heavy usage pattern, means a lot of intracluster communication (quorums on the writes and subsequent cluster member updates), but don't underbuild it (and risk swamping the cluster).
      • -
      • Try to run on a 3-5 node cluster: ZooKeeper writes use quorums and inherently that means having an odd number of machines in a cluster. Remember that a 5 node cluster will cause writes to slow down compared to a 3 node cluster, but will allow more fault tolerance.
      • +
      • ZooKeeper configuration: It's java, make sure you give it 'enough' heap space (We usually run them with 3-5G, but that's mostly due to the data set size we have here). Unfortunately we don't have a good formula for it, but keep in mind that allowing for more ZooKeeper state means that snapshots can become large, and large snapshots affect recovery time. In fact, if the snapshot becomes too large (a few gigabytes), then you may need to increase the initLimit parameter to give enough time for servers to recover and join the ensemble.
      • +
      • Monitoring: Both JMX and the 4 letter words (4lw) commands are very useful, they do overlap in some cases (and in those cases we prefer the 4 letter commands, they seem more predictable, or at the very least, they work better with the LI monitoring infrastructure)
      • +
      • Don't overbuild the cluster: large clusters, especially in a write heavy usage pattern, means a lot of intracluster communication (quorums on the writes and subsequent cluster member updates), but don't underbuild it (and risk swamping the cluster). Having more servers adds to your read capacity.
      Overall, we try to keep the ZooKeeper system as small as will handle the load (plus standard growth capacity planning) and as simple as possible. We try not to do anything fancy with the configuration or application layout as compared to the official release as well as keep it as self contained as possible. For these reasons, we tend to skip the OS packaged versions, since it has a tendency to try to put things in the OS standard hierarchy, which can be 'messy', for want of a better way to word it. From bd5325dd8be7c5cf920acee2aa33b3c288bd551a Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Fri, 1 Apr 2016 17:14:29 -0700 Subject: [PATCH 108/206] MINOR: small code optimizations in streams guozhangwang Author: Yasuhiro Matsuda Reviewers: Guozhang Wang Closes #1176 from ymatsuda/optimize --- .../processor/internals/PartitionGroup.java | 6 +++- .../processor/internals/StreamTask.java | 12 +++---- .../processor/internals/StreamThread.java | 35 +++++++------------ 3 files changed, 24 insertions(+), 29 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java index b487ff5b36ee4..3d8f792c7c8e8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java @@ -49,6 +49,10 @@ public ProcessorNode node() { public TopicPartition partition() { return queue.partition(); } + + public RecordQueue queue() { + return queue; + } } // since task is thread-safe, we do not need to synchronize on local variables @@ -88,7 +92,7 @@ public StampedRecord nextRecord(RecordInfo info) { // get the first record from this queue. record = queue.poll(); - if (queue.size() > 0) { + if (!queue.isEmpty()) { queuesByTime.offer(queue); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index afa303ca1514f..61aeced9624a8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -179,7 +179,7 @@ public int process() { // after processing this record, if its partition queue's buffered size has been // decreased to the threshold, we can then resume the consumption on this partition - if (partitionGroup.numBuffered(partition) == this.maxBufferedSize) { + if (recordInfo.queue().size() == this.maxBufferedSize) { consumer.resume(singleton(partition)); requiresPoll = true; } @@ -320,13 +320,13 @@ private RecordQueue createRecordQueue(TopicPartition partition, SourceNode sourc @SuppressWarnings("unchecked") public void forward(K key, V value) { ProcessorNode thisNode = currNode; - for (ProcessorNode childNode : (List>) thisNode.children()) { - currNode = childNode; - try { + try { + for (ProcessorNode childNode : (List>) thisNode.children()) { + currNode = childNode; childNode.process(key, value); - } finally { - currNode = thisNode; } + } finally { + currNode = thisNode; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 7d6b98f9b8680..c2a8e06e9b119 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -350,9 +350,12 @@ private void runLoop() { requiresPoll = requiresPoll || task.requiresPoll(); sensors.processTimeSensor.record(time.milliseconds() - startProcess); - } - maybePunctuate(); + maybePunctuate(task); + + if (task.commitNeeded()) + commitOne(task, time.milliseconds()); + } // if pollTimeMs has passed since the last poll, we poll to respond to a possible rebalance // even when we paused all partitions. @@ -424,18 +427,16 @@ private boolean stillRunning() { return true; } - private void maybePunctuate() { - for (StreamTask task : activeTasks.values()) { - try { - long now = time.milliseconds(); + private void maybePunctuate(StreamTask task) { + try { + long now = time.milliseconds(); - if (task.maybePunctuate(now)) - sensors.punctuateTimeSensor.record(time.milliseconds() - now); + if (task.maybePunctuate(now)) + sensors.punctuateTimeSensor.record(time.milliseconds() - now); - } catch (KafkaException e) { - log.error("Failed to punctuate active task #" + task.id() + " in thread [" + this.getName() + "]: ", e); - throw e; - } + } catch (KafkaException e) { + log.error("Failed to punctuate active task #" + task.id() + " in thread [" + this.getName() + "]: ", e); + throw e; } } @@ -449,16 +450,6 @@ protected void maybeCommit() { lastCommit = now; processStandbyRecords = true; - } else { - for (StreamTask task : activeTasks.values()) { - try { - if (task.commitNeeded()) - commitOne(task, time.milliseconds()); - } catch (KafkaException e) { - log.error("Failed to commit active task #" + task.id() + " in thread [" + this.getName() + "]: ", e); - throw e; - } - } } } From dd5480a47eb0f45214c179b7f14ffaf493164222 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Sat, 2 Apr 2016 23:02:19 -0700 Subject: [PATCH 109/206] KAFKA-3486: fix autocommit when partitions assigned manually Author: Jason Gustafson Reviewers: Ewen Cheslack-Postava Closes #1169 from hachikuji/KAFKA-3486 --- .../internals/ConsumerCoordinator.java | 54 +++----- .../internals/ConsumerCoordinatorTest.java | 122 ++++++++++++++++-- 2 files changed, 134 insertions(+), 42 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index e582ce3fc71b7..a3649878bfe23 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -110,7 +110,13 @@ public ConsumerCoordinator(ConsumerNetworkClient client, addMetadataListener(); - this.autoCommitTask = autoCommitEnabled ? new AutoCommitTask(autoCommitIntervalMs) : null; + if (autoCommitEnabled) { + this.autoCommitTask = new AutoCommitTask(autoCommitIntervalMs); + this.autoCommitTask.reschedule(); + } else { + this.autoCommitTask = null; + } + this.sensors = new ConsumerCoordinatorMetrics(metrics, metricGrpPrefix); this.interceptors = interceptors; this.excludeInternalTopics = excludeInternalTopics; @@ -187,9 +193,9 @@ protected void onJoinComplete(int generation, // give the assignor a chance to update internal state based on the received assignment assignor.onAssignment(assignment); - // restart the autocommit task if needed + // reschedule the auto commit starting from now if (autoCommitEnabled) - autoCommitTask.enable(); + autoCommitTask.reschedule(); // execute the user's callback after rebalance ConsumerRebalanceListener listener = subscriptions.listener(); @@ -384,52 +390,36 @@ public void commitOffsetsSync(Map offsets) { private class AutoCommitTask implements DelayedTask { private final long interval; - private boolean enabled = false; - private boolean requestInFlight = false; public AutoCommitTask(long interval) { this.interval = interval; } - public void enable() { - if (!enabled) { - // there shouldn't be any instances scheduled, but call unschedule anyway to ensure - // that this task is only ever scheduled once - client.unschedule(this); - this.enabled = true; - - if (!requestInFlight) { - long now = time.milliseconds(); - client.schedule(this, interval + now); - } - } - } - - public void disable() { - this.enabled = false; - client.unschedule(this); + private void reschedule() { + client.schedule(this, time.milliseconds() + interval); } private void reschedule(long at) { - if (enabled) - client.schedule(this, at); + client.schedule(this, at); } public void run(final long now) { - if (!enabled) - return; - if (coordinatorUnknown()) { log.debug("Cannot auto-commit offsets for group {} since the coordinator is unknown", groupId); - client.schedule(this, now + retryBackoffMs); + reschedule(now + retryBackoffMs); + return; + } + + if (needRejoin()) { + // skip the commit when we're rejoining since we'll commit offsets synchronously + // before the revocation callback is invoked + reschedule(now + interval); return; } - requestInFlight = true; commitOffsetsAsync(subscriptions.allConsumed(), new OffsetCommitCallback() { @Override public void onComplete(Map offsets, Exception exception) { - requestInFlight = false; if (exception == null) { reschedule(now + interval); } else if (exception instanceof SendFailedException) { @@ -446,10 +436,6 @@ public void onComplete(Map offsets, Exception private void maybeAutoCommitOffsetsSync() { if (autoCommitEnabled) { - // disable periodic commits prior to committing synchronously. note that they will - // be re-enabled after a rebalance completes - autoCommitTask.disable(); - try { commitOffsetsSync(subscriptions.allConsumed()); } catch (WakeupException e) { diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index 8844adc80b964..623e5ef93c1e0 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -77,11 +77,11 @@ public class ConsumerCoordinatorTest { private String topicName = "test"; private String groupId = "test-group"; private TopicPartition tp = new TopicPartition(topicName, 0); - private int sessionTimeoutMs = 10; - private int heartbeatIntervalMs = 2; + private int sessionTimeoutMs = 10000; + private int heartbeatIntervalMs = 5000; private long retryBackoffMs = 100; private boolean autoCommitEnabled = false; - private long autoCommitIntervalMs = 5000; + private long autoCommitIntervalMs = 2000; private MockPartitionAssignor partitionAssignor = new MockPartitionAssignor(); private List assignors = Arrays.asList(partitionAssignor); private MockTime time; @@ -110,7 +110,7 @@ public void setup() { this.partitionAssignor.clear(); client.setNode(node); - this.coordinator = buildCoordinator(metrics, assignors, ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_DEFAULT); + this.coordinator = buildCoordinator(metrics, assignors, ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_DEFAULT, autoCommitEnabled); } @After @@ -546,7 +546,7 @@ public void testExcludeInternalTopicsConfigOption() { @Test public void testIncludeInternalTopicsConfigOption() { - coordinator = buildCoordinator(new Metrics(), assignors, false); + coordinator = buildCoordinator(new Metrics(), assignors, false, false); subscriptions.subscribe(Pattern.compile(".*"), rebalanceListener); metadata.update(TestUtils.singletonCluster(TopicConstants.GROUP_METADATA_TOPIC_NAME, 2), time.milliseconds()); @@ -632,6 +632,107 @@ public void testCommitOffsetOnly() { assertEquals(100L, subscriptions.committed(tp).offset()); } + @Test + public void testAutoCommitDynamicAssignment() { + final String consumerId = "consumer"; + + ConsumerCoordinator coordinator = buildCoordinator(new Metrics(), assignors, + ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_DEFAULT, true); + + subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); + subscriptions.needReassignment(); + + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); + coordinator.ensurePartitionAssignment(); + + subscriptions.seek(tp, 100); + + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + time.sleep(autoCommitIntervalMs); + consumerClient.poll(0); + + assertEquals(100L, subscriptions.committed(tp).offset()); + } + + @Test + public void testAutoCommitDynamicAssignmentRebalance() { + final String consumerId = "consumer"; + + ConsumerCoordinator coordinator = buildCoordinator(new Metrics(), assignors, + ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_DEFAULT, true); + + subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); + subscriptions.needReassignment(); + + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // haven't joined, so should not cause a commit + time.sleep(autoCommitIntervalMs); + consumerClient.poll(0); + + client.prepareResponse(joinGroupFollowerResponse(1, consumerId, "leader", Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); + coordinator.ensurePartitionAssignment(); + + subscriptions.seek(tp, 100); + + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + time.sleep(autoCommitIntervalMs); + consumerClient.poll(0); + + assertEquals(100L, subscriptions.committed(tp).offset()); + } + + @Test + public void testAutoCommitManualAssignment() { + ConsumerCoordinator coordinator = buildCoordinator(new Metrics(), assignors, + ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_DEFAULT, true); + + subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.seek(tp, 100); + + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + time.sleep(autoCommitIntervalMs); + consumerClient.poll(0); + + assertEquals(100L, subscriptions.committed(tp).offset()); + } + + @Test + public void testAutoCommitManualAssignmentCoordinatorUnknown() { + ConsumerCoordinator coordinator = buildCoordinator(new Metrics(), assignors, + ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_DEFAULT, true); + + subscriptions.assignFromUser(Arrays.asList(tp)); + subscriptions.seek(tp, 100); + + // no commit initially since coordinator is unknown + consumerClient.poll(0); + time.sleep(autoCommitIntervalMs); + consumerClient.poll(0); + + assertNull(subscriptions.committed(tp)); + + // now find the coordinator + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // sleep only for the retry backoff + time.sleep(retryBackoffMs); + client.prepareResponse(offsetCommitResponse(Collections.singletonMap(tp, Errors.NONE.code()))); + consumerClient.poll(0); + + assertEquals(100L, subscriptions.committed(tp).offset()); + } + @Test public void testCommitOffsetMetadata() { subscriptions.assignFromUser(Arrays.asList(tp)); @@ -896,7 +997,8 @@ public void testProtocolMetadataOrder() { RangeAssignor range = new RangeAssignor(); try (Metrics metrics = new Metrics(time)) { - ConsumerCoordinator coordinator = buildCoordinator(metrics, Arrays.asList(roundRobin, range), ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_DEFAULT); + ConsumerCoordinator coordinator = buildCoordinator(metrics, Arrays.asList(roundRobin, range), + ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_DEFAULT, false); List metadata = coordinator.metadata(); assertEquals(2, metadata.size()); assertEquals(roundRobin.name(), metadata.get(0).name()); @@ -904,7 +1006,8 @@ public void testProtocolMetadataOrder() { } try (Metrics metrics = new Metrics(time)) { - ConsumerCoordinator coordinator = buildCoordinator(metrics, Arrays.asList(range, roundRobin), ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_DEFAULT); + ConsumerCoordinator coordinator = buildCoordinator(metrics, Arrays.asList(range, roundRobin), + ConsumerConfig.EXCLUDE_INTERNAL_TOPICS_DEFAULT, false); List metadata = coordinator.metadata(); assertEquals(2, metadata.size()); assertEquals(range.name(), metadata.get(0).name()); @@ -912,7 +1015,10 @@ public void testProtocolMetadataOrder() { } } - private ConsumerCoordinator buildCoordinator(Metrics metrics, List assignors, boolean excludeInternalTopics) { + private ConsumerCoordinator buildCoordinator(Metrics metrics, + List assignors, + boolean excludeInternalTopics, + boolean autoCommitEnabled) { return new ConsumerCoordinator( consumerClient, groupId, From ef3f053155c73af053a36a2e50047c5a49cd30bf Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Sun, 3 Apr 2016 13:44:05 -0700 Subject: [PATCH 110/206] KAFKA-3419: clarify difference between topic subscription and partition assignment Author: Jason Gustafson Reviewers: Ashish Singh, Ismael Juma, Guozhang Wang Closes #1158 from hachikuji/KAFKA-3419 --- .../kafka/clients/consumer/KafkaConsumer.java | 41 ++++++++++--------- 1 file changed, 21 insertions(+), 20 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index b15d07f80969e..c457c83692a59 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -125,8 +125,9 @@ * commits (note that offsets are always committed for a given consumer group), etc. * See Storing Offsets Outside Kafka for more details *

      - * It is also possible for the consumer to manually specify the partitions that are assigned to it through {@link #assign(Collection)}, - * which disables this dynamic partition assignment. + * It is also possible for the consumer to manually assign specific partitions + * (similar to the older "simple" consumer) using {@link #assign(Collection)}. In this case, dynamic partition + * assignment and consumer group coordination will be disabled. * *

      Usage Examples

      * The consumer APIs offer flexibility to cover a variety of consumption use cases. Here are some examples to @@ -242,27 +243,23 @@ * Note: The committed offset should always be the offset of the next message that your application will read. * Thus, when calling {@link #commitSync(Map) commitSync(offsets)} you should add one to the offset of the last message processed. * - *

      Subscribing To Specific Partitions

      + *

      Manual Partition Assignment

      * - * In the previous examples we subscribed to the topics we were interested in and let Kafka give our particular process - * a fair share of the partitions for those topics. This provides a simple load balancing mechanism so multiple - * instances of our program can divided up the work of processing records. + * In the previous examples, we subscribed to the topics we were interested in and let Kafka dynamically assign a + * fair share of the partitions for those topics based on the active consumers in the group. However, in + * some cases you may need finer control over the specific partitions that are assigned. For example: *

      - * In this mode the consumer will just get the partitions it subscribes to and if the consumer instance fails no attempt - * will be made to rebalance partitions to other instances. - *

      - * There are several cases where this makes sense: *

        - *
      • The first case is if the process is maintaining some kind of local state associated with that partition (like a - * local on-disk key-value store) and hence it should only get records for the partition it is maintaining on disk. - *
      • Another case is if the process itself is highly available and will be restarted if it fails (perhaps using a + *
      • If the process is maintaining some kind of local state associated with that partition (like a + * local on-disk key-value store), then it should only get records for the partition it is maintaining on disk. + *
      • If the process itself is highly available and will be restarted if it fails (perhaps using a * cluster management framework like YARN, Mesos, or AWS facilities, or as part of a stream processing framework). In - * this case there is no need for Kafka to detect the failure and reassign the partition, rather the consuming process + * this case there is no need for Kafka to detect the failure and reassign the partition since the consuming process * will be restarted on another machine. *
      *

      - * This mode is easy to specify, rather than subscribing to the topic, the consumer just subscribes to particular - * partitions: + * To use this mode, instead of subscribing to the topic using {@link #subscribe(Collection) subscribe}, you just call + * {@link #assign(Collection)} with the full list of partitions that you want to consume. * *

        *     String topic = "foo";
      @@ -271,11 +268,15 @@
        *     consumer.assign(Arrays.asList(partition0, partition1));
        * 
      * - * The group that the consumer specifies is still used for committing offsets, but now the set of partitions will only - * be changed if the consumer specifies new partitions, and no attempt at failure detection will be made. + * Once assigned, you can call {@link #poll(long) poll} in a loop, just as in the preceding examples to consume + * records. The group that the consumer specifies is still used for committing offsets, but now the set of partitions + * will only change with another call to {@link #assign(Collection) assign}. Manual partition assignment does + * not use group coordination, so consumer failures will not cause assigned partitions to be rebalanced. Each consumer + * acts independently even if it shares a groupId with another consumer. To avoid offset commit conflicts, you should + * usually ensure that the groupId is unique for each consumer instance. *

      - * It isn't possible to mix both subscription to specific partitions (with no load balancing) and to topics (with load - * balancing) using the same consumer instance. + * Note that it isn't possible to mix manual partition assignment (i.e. using {@link #assign(Collection) assign}) + * with dynamic partition assignment through topic subscription (i.e. using {@link #subscribe(Collection) subscribe}). * *

      Storing Offsets Outside Kafka

      * From b94a7812beeb840926871df84838f35ca7b76ffe Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Sun, 3 Apr 2016 16:34:46 -0700 Subject: [PATCH 111/206] KAFKA-3495; NetworkClient.blockingSendAndReceive` should rely on requestTimeout Also removed the code for handling negative timeouts in `blockingReady` as `Selector.poll` has not supported that for a while. Author: Ismael Juma Reviewers: Jun Rao Closes #1177 from ijuma/kafka-3495-blocking-send-and-receive-request-timeout --- .../apache/kafka/clients/NetworkClient.java | 2 +- .../controller/ControllerChannelManager.scala | 4 +- .../main/scala/kafka/server/KafkaServer.scala | 9 +-- .../kafka/server/ReplicaFetcherThread.scala | 4 +- .../utils/NetworkClientBlockingOps.scala | 66 ++++++++++--------- 5 files changed, 40 insertions(+), 45 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index 4d01cdeb2e27d..d22b508cd878f 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -390,7 +390,7 @@ private void processDisconnection(List responses, String nodeId, } /** - * Iterate over all the inflight requests and expire any requests that have exceeded the configured the requestTimeout. + * Iterate over all the inflight requests and expire any requests that have exceeded the configured requestTimeout. * The connection to the node associated with the request will be terminated and will be treated as a disconnection. * * @param responses The list of responses to update diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index b376d15e4eb1f..e9731fd4e091c 100755 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -178,9 +178,7 @@ class RequestSendThread(val controllerId: Int, val requestHeader = apiVersion.fold(networkClient.nextRequestHeader(apiKey))(networkClient.nextRequestHeader(apiKey, _)) val send = new RequestSend(brokerNode.idString, requestHeader, request.toStruct) val clientRequest = new ClientRequest(time.milliseconds(), true, send, null) - clientResponse = networkClient.blockingSendAndReceive(clientRequest, socketTimeoutMs)(time).getOrElse { - throw new SocketTimeoutException(s"No response received within $socketTimeoutMs ms") - } + clientResponse = networkClient.blockingSendAndReceive(clientRequest)(time) isSendSuccessful = true } } catch { diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index e29494baa1d99..f998d82104d36 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -320,9 +320,6 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr val socketTimeoutMs = config.controllerSocketTimeoutMs - def socketTimeoutException: Throwable = - new SocketTimeoutException(s"Did not receive response within $socketTimeoutMs") - def networkClientControlledShutdown(retries: Int): Boolean = { val metadataUpdater = new ManualMetadataUpdater() val networkClient = { @@ -388,16 +385,14 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr try { if (!networkClient.blockingReady(node(prevController), socketTimeoutMs)) - throw socketTimeoutException + throw new SocketTimeoutException(s"Failed to connect within $socketTimeoutMs ms") // send the controlled shutdown request val requestHeader = networkClient.nextRequestHeader(ApiKeys.CONTROLLED_SHUTDOWN_KEY) val send = new RequestSend(node(prevController).idString, requestHeader, new ControlledShutdownRequest(config.brokerId).toStruct) val request = new ClientRequest(kafkaMetricsTime.milliseconds(), true, send, null) - val clientResponse = networkClient.blockingSendAndReceive(request, socketTimeoutMs).getOrElse { - throw socketTimeoutException - } + val clientResponse = networkClient.blockingSendAndReceive(request) val shutdownResponse = new ControlledShutdownResponse(clientResponse.responseBody) if (shutdownResponse.errorCode == Errors.NONE.code && shutdownResponse.partitionsRemaining.isEmpty) { diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index de7269f833230..26838cac96dbe 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -233,9 +233,7 @@ class ReplicaFetcherThread(name: String, else { val send = new RequestSend(sourceBroker.id.toString, header, request.toStruct) val clientRequest = new ClientRequest(time.milliseconds(), true, send, null) - networkClient.blockingSendAndReceive(clientRequest, socketTimeout)(time).getOrElse { - throw new SocketTimeoutException(s"No response received within $socketTimeout ms") - } + networkClient.blockingSendAndReceive(clientRequest)(time) } } catch { diff --git a/core/src/main/scala/kafka/utils/NetworkClientBlockingOps.scala b/core/src/main/scala/kafka/utils/NetworkClientBlockingOps.scala index 9ed9d29a2932e..fd4af6e949b61 100644 --- a/core/src/main/scala/kafka/utils/NetworkClientBlockingOps.scala +++ b/core/src/main/scala/kafka/utils/NetworkClientBlockingOps.scala @@ -55,6 +55,7 @@ class NetworkClientBlockingOps(val client: NetworkClient) extends AnyVal { * care. */ def blockingReady(node: Node, timeout: Long)(implicit time: JTime): Boolean = { + require(timeout >=0, "timeout should be >= 0") client.ready(node, time.milliseconds()) || pollUntil(timeout) { (_, now) => if (client.isReady(node, now)) true @@ -65,19 +66,18 @@ class NetworkClientBlockingOps(val client: NetworkClient) extends AnyVal { } /** - * Invokes `client.send` followed by 1 or more `client.poll` invocations until a response is received, - * the timeout expires or a disconnection happens. + * Invokes `client.send` followed by 1 or more `client.poll` invocations until a response is received or a + * disconnection happens (which can happen for a number of reasons including a request timeout). * - * It returns `true` if the call completes normally or `false` if the timeout expires. In the case of a disconnection, - * an `IOException` is thrown instead. + * In case of a disconnection, an `IOException` is thrown. * * This method is useful for implementing blocking behaviour on top of the non-blocking `NetworkClient`, use it with * care. */ - def blockingSendAndReceive(request: ClientRequest, timeout: Long)(implicit time: JTime): Option[ClientResponse] = { + def blockingSendAndReceive(request: ClientRequest)(implicit time: JTime): ClientResponse = { client.send(request, time.milliseconds()) - pollUntilFound(timeout) { case (responses, _) => + pollContinuously { responses => val response = responses.find { response => response.request.request.header.correlationId == request.request.header.correlationId } @@ -102,41 +102,45 @@ class NetworkClientBlockingOps(val client: NetworkClient) extends AnyVal { * care. */ private def pollUntil(timeout: Long)(predicate: (Seq[ClientResponse], Long) => Boolean)(implicit time: JTime): Boolean = { - pollUntilFound(timeout) { (responses, now) => - if (predicate(responses, now)) Some(true) - else None - }.fold(false)(_ => true) - } - - /** - * Invokes `client.poll` until `collect` returns `Some` or the timeout expires. - * - * It returns the result of `collect` if the call completes normally or `None` if the timeout expires. Exceptions - * thrown via `collect` are not handled and will bubble up. - * - * This method is useful for implementing blocking behaviour on top of the non-blocking `NetworkClient`, use it with - * care. - */ - private def pollUntilFound[T](timeout: Long)(collect: (Seq[ClientResponse], Long) => Option[T])(implicit time: JTime): Option[T] = { - val methodStartTime = time.milliseconds() val timeoutExpiryTime = methodStartTime + timeout @tailrec - def recurse(iterationStartTime: Long): Option[T] = { - val pollTimeout = if (timeout < 0) timeout else timeoutExpiryTime - iterationStartTime + def recursivePoll(iterationStartTime: Long): Boolean = { + val pollTimeout = timeoutExpiryTime - iterationStartTime val responses = client.poll(pollTimeout, iterationStartTime).asScala - val result = collect(responses, iterationStartTime) - if (result.isDefined) result + if (predicate(responses, iterationStartTime)) true else { val afterPollTime = time.milliseconds() - if (timeout < 0 || afterPollTime < timeoutExpiryTime) - recurse(afterPollTime) - else None + if (afterPollTime < timeoutExpiryTime) recursivePoll(afterPollTime) + else false + } + } + + recursivePoll(methodStartTime) + } + + /** + * Invokes `client.poll` until `collect` returns `Some`. The value inside `Some` is returned. + * + * Exceptions thrown via `collect` are not handled and will bubble up. + * + * This method is useful for implementing blocking behaviour on top of the non-blocking `NetworkClient`, use it with + * care. + */ + private def pollContinuously[T](collect: Seq[ClientResponse] => Option[T])(implicit time: JTime): T = { + + @tailrec + def recursivePoll: T = { + // rely on request timeout to ensure we don't block forever + val responses = client.poll(Long.MaxValue, time.milliseconds()).asScala + collect(responses) match { + case Some(result) => result + case None => recursivePoll } } - recurse(methodStartTime) + recursivePoll } } From 3a20ba305517ace78ea5f54554bd3b333f0b7d3d Mon Sep 17 00:00:00 2001 From: Paul Cavallaro Date: Sun, 3 Apr 2016 18:14:53 -0700 Subject: [PATCH 112/206] MINOR: Fix small typo in design section Sentence was missing "as", minor grammar clean up. Author: Paul Cavallaro Reviewers: Ewen Cheslack-Postava Closes #1151 from paulcavallaro/docs-fix --- docs/design.html | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/design.html b/docs/design.html index ad40431eeb18c..a97a0adecc0bc 100644 --- a/docs/design.html +++ b/docs/design.html @@ -300,7 +300,7 @@

      4.8 Log Compaction

      This retention policy can be set per-topic, so a single cluster can have some topics where retention is enforced by size or time and other topics where retention is enforced by compaction.

      -This functionality is inspired by one of LinkedIn's oldest and most successful pieces of infrastructure—a database changelog caching service called Databus. Unlike most log-structured storage systems Kafka is built for subscription and organizes data for fast linear reads and writes. Unlike Databus, Kafka acts a source-of-truth store so it is useful even in situations where the upstream data source would not otherwise be replayable. +This functionality is inspired by one of LinkedIn's oldest and most successful pieces of infrastructure—a database changelog caching service called Databus. Unlike most log-structured storage systems Kafka is built for subscription and organizes data for fast linear reads and writes. Unlike Databus, Kafka acts as a source-of-truth store so it is useful even in situations where the upstream data source would not otherwise be replayable.

      Log Compaction Basics

      From 83cf38545be4614bd1f6b1759ada851fb38d63b0 Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Sun, 3 Apr 2016 19:04:48 -0700 Subject: [PATCH 113/206] MINOR: Clean up of SourceTaskOffsetCommiter Author: Liquan Pei Reviewers: Ewen Cheslack-Postava Closes #1170 from Ishiihara/minor-cleanup --- .../kafka/connect/runtime/SourceTaskOffsetCommitter.java | 9 +++------ .../java/org/apache/kafka/connect/runtime/Worker.java | 2 +- 2 files changed, 4 insertions(+), 7 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitter.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitter.java index bee24e7b1fca3..c7f869eb3a5dd 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitter.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/SourceTaskOffsetCommitter.java @@ -17,7 +17,6 @@ package org.apache.kafka.connect.runtime; -import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.util.ConnectorTaskId; import org.slf4j.Logger; @@ -45,13 +44,11 @@ class SourceTaskOffsetCommitter { private static final Logger log = LoggerFactory.getLogger(SourceTaskOffsetCommitter.class); - private Time time; private WorkerConfig config; private ScheduledExecutorService commitExecutorService = null; - private HashMap committers = new HashMap<>(); + private final HashMap committers = new HashMap<>(); - SourceTaskOffsetCommitter(Time time, WorkerConfig config) { - this.time = time; + SourceTaskOffsetCommitter(WorkerConfig config) { this.config = config; commitExecutorService = Executors.newSingleThreadScheduledExecutor(); } @@ -96,7 +93,7 @@ public void remove(ConnectorTaskId id) { } } - public void commit(ConnectorTaskId id, WorkerSourceTask workerTask) { + private void commit(ConnectorTaskId id, WorkerSourceTask workerTask) { final ScheduledCommitTask task; synchronized (committers) { task = committers.get(id); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java index 1a9ff110db58c..e1a806a2c3030 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java @@ -120,7 +120,7 @@ public void start() { producer = new KafkaProducer<>(producerProps); offsetBackingStore.start(); - sourceTaskOffsetCommitter = new SourceTaskOffsetCommitter(time, config); + sourceTaskOffsetCommitter = new SourceTaskOffsetCommitter(config); log.info("Worker started"); } From 45c585b4f7e3d5e5dd5297b4d121badbd2052922 Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Sun, 3 Apr 2016 20:04:36 -0700 Subject: [PATCH 114/206] KAFKA-3483: Restructure ducktape tests to simplify running subsets of tests MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit … tests Author: Grant Henke Reviewers: Geoff Anderson , Ewen Cheslack-Postava Closes #1162 from granthenke/ducktape-structure --- tests/kafkatest/benchmarks/__init__.py | 14 ++++++++++++++ tests/kafkatest/benchmarks/core/__init__.py | 15 +++++++++++++++ .../{tests => benchmarks/core}/benchmark_test.py | 0 tests/kafkatest/tests/client/__init__.py | 15 +++++++++++++++ .../tests/{ => client}/compression_test.py | 0 .../{ => client}/consumer_rolling_upgrade_test.py | 0 .../kafkatest/tests/{ => client}/consumer_test.py | 0 .../{ => client}/message_format_change_test.py | 0 tests/kafkatest/tests/{ => client}/quota_test.py | 0 tests/kafkatest/tests/connect/__init__.py | 15 +++++++++++++++ .../{ => connect}/connect_distributed_test.py | 0 .../tests/{ => connect}/connect_rest_test.py | 0 .../kafkatest/tests/{ => connect}/connect_test.py | 0 .../templates/connect-distributed.properties | 0 .../templates/connect-file-sink.properties | 0 .../templates/connect-file-source.properties | 0 .../templates/connect-standalone.properties | 0 tests/kafkatest/tests/core/__init__.py | 15 +++++++++++++++ .../compatibility_test_new_broker_test.py | 0 .../{ => core}/consumer_group_command_test.py | 0 .../tests/{ => core}/get_offset_shell_test.py | 0 .../tests/{ => core}/mirror_maker_test.py | 0 .../tests/{ => core}/reassign_partitions_test.py | 0 .../tests/{ => core}/replication_test.py | 0 .../{ => core}/security_rolling_upgrade_test.py | 0 .../{ => core}/simple_consumer_shell_test.py | 0 tests/kafkatest/tests/{ => core}/upgrade_test.py | 0 .../{ => core}/zookeeper_security_upgrade_test.py | 0 tests/kafkatest/tests/streams/__init__.py | 15 +++++++++++++++ .../tests/{ => streams}/streams_bounce_test.py | 0 .../tests/{ => streams}/streams_smoke_test.py | 0 tests/kafkatest/tests/tools/__init__.py | 15 +++++++++++++++ .../tests/{ => tools}/log4j_appender_test.py | 0 33 files changed, 104 insertions(+) create mode 100644 tests/kafkatest/benchmarks/__init__.py create mode 100644 tests/kafkatest/benchmarks/core/__init__.py rename tests/kafkatest/{tests => benchmarks/core}/benchmark_test.py (100%) create mode 100644 tests/kafkatest/tests/client/__init__.py rename tests/kafkatest/tests/{ => client}/compression_test.py (100%) rename tests/kafkatest/tests/{ => client}/consumer_rolling_upgrade_test.py (100%) rename tests/kafkatest/tests/{ => client}/consumer_test.py (100%) rename tests/kafkatest/tests/{ => client}/message_format_change_test.py (100%) rename tests/kafkatest/tests/{ => client}/quota_test.py (100%) create mode 100644 tests/kafkatest/tests/connect/__init__.py rename tests/kafkatest/tests/{ => connect}/connect_distributed_test.py (100%) rename tests/kafkatest/tests/{ => connect}/connect_rest_test.py (100%) rename tests/kafkatest/tests/{ => connect}/connect_test.py (100%) rename tests/kafkatest/tests/{ => connect}/templates/connect-distributed.properties (100%) rename tests/kafkatest/tests/{ => connect}/templates/connect-file-sink.properties (100%) rename tests/kafkatest/tests/{ => connect}/templates/connect-file-source.properties (100%) rename tests/kafkatest/tests/{ => connect}/templates/connect-standalone.properties (100%) create mode 100644 tests/kafkatest/tests/core/__init__.py rename tests/kafkatest/tests/{ => core}/compatibility_test_new_broker_test.py (100%) rename tests/kafkatest/tests/{ => core}/consumer_group_command_test.py (100%) rename tests/kafkatest/tests/{ => core}/get_offset_shell_test.py (100%) rename tests/kafkatest/tests/{ => core}/mirror_maker_test.py (100%) rename tests/kafkatest/tests/{ => core}/reassign_partitions_test.py (100%) rename tests/kafkatest/tests/{ => core}/replication_test.py (100%) rename tests/kafkatest/tests/{ => core}/security_rolling_upgrade_test.py (100%) rename tests/kafkatest/tests/{ => core}/simple_consumer_shell_test.py (100%) rename tests/kafkatest/tests/{ => core}/upgrade_test.py (100%) rename tests/kafkatest/tests/{ => core}/zookeeper_security_upgrade_test.py (100%) create mode 100644 tests/kafkatest/tests/streams/__init__.py rename tests/kafkatest/tests/{ => streams}/streams_bounce_test.py (100%) rename tests/kafkatest/tests/{ => streams}/streams_smoke_test.py (100%) create mode 100644 tests/kafkatest/tests/tools/__init__.py rename tests/kafkatest/tests/{ => tools}/log4j_appender_test.py (100%) diff --git a/tests/kafkatest/benchmarks/__init__.py b/tests/kafkatest/benchmarks/__init__.py new file mode 100644 index 0000000000000..ec2014340d78f --- /dev/null +++ b/tests/kafkatest/benchmarks/__init__.py @@ -0,0 +1,14 @@ +# 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. diff --git a/tests/kafkatest/benchmarks/core/__init__.py b/tests/kafkatest/benchmarks/core/__init__.py new file mode 100644 index 0000000000000..ebc9bb3a9a03f --- /dev/null +++ b/tests/kafkatest/benchmarks/core/__init__.py @@ -0,0 +1,15 @@ +# 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. +# see kafka.server.KafkaConfig for additional details and defaults diff --git a/tests/kafkatest/tests/benchmark_test.py b/tests/kafkatest/benchmarks/core/benchmark_test.py similarity index 100% rename from tests/kafkatest/tests/benchmark_test.py rename to tests/kafkatest/benchmarks/core/benchmark_test.py diff --git a/tests/kafkatest/tests/client/__init__.py b/tests/kafkatest/tests/client/__init__.py new file mode 100644 index 0000000000000..ebc9bb3a9a03f --- /dev/null +++ b/tests/kafkatest/tests/client/__init__.py @@ -0,0 +1,15 @@ +# 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. +# see kafka.server.KafkaConfig for additional details and defaults diff --git a/tests/kafkatest/tests/compression_test.py b/tests/kafkatest/tests/client/compression_test.py similarity index 100% rename from tests/kafkatest/tests/compression_test.py rename to tests/kafkatest/tests/client/compression_test.py diff --git a/tests/kafkatest/tests/consumer_rolling_upgrade_test.py b/tests/kafkatest/tests/client/consumer_rolling_upgrade_test.py similarity index 100% rename from tests/kafkatest/tests/consumer_rolling_upgrade_test.py rename to tests/kafkatest/tests/client/consumer_rolling_upgrade_test.py diff --git a/tests/kafkatest/tests/consumer_test.py b/tests/kafkatest/tests/client/consumer_test.py similarity index 100% rename from tests/kafkatest/tests/consumer_test.py rename to tests/kafkatest/tests/client/consumer_test.py diff --git a/tests/kafkatest/tests/message_format_change_test.py b/tests/kafkatest/tests/client/message_format_change_test.py similarity index 100% rename from tests/kafkatest/tests/message_format_change_test.py rename to tests/kafkatest/tests/client/message_format_change_test.py diff --git a/tests/kafkatest/tests/quota_test.py b/tests/kafkatest/tests/client/quota_test.py similarity index 100% rename from tests/kafkatest/tests/quota_test.py rename to tests/kafkatest/tests/client/quota_test.py diff --git a/tests/kafkatest/tests/connect/__init__.py b/tests/kafkatest/tests/connect/__init__.py new file mode 100644 index 0000000000000..ebc9bb3a9a03f --- /dev/null +++ b/tests/kafkatest/tests/connect/__init__.py @@ -0,0 +1,15 @@ +# 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. +# see kafka.server.KafkaConfig for additional details and defaults diff --git a/tests/kafkatest/tests/connect_distributed_test.py b/tests/kafkatest/tests/connect/connect_distributed_test.py similarity index 100% rename from tests/kafkatest/tests/connect_distributed_test.py rename to tests/kafkatest/tests/connect/connect_distributed_test.py diff --git a/tests/kafkatest/tests/connect_rest_test.py b/tests/kafkatest/tests/connect/connect_rest_test.py similarity index 100% rename from tests/kafkatest/tests/connect_rest_test.py rename to tests/kafkatest/tests/connect/connect_rest_test.py diff --git a/tests/kafkatest/tests/connect_test.py b/tests/kafkatest/tests/connect/connect_test.py similarity index 100% rename from tests/kafkatest/tests/connect_test.py rename to tests/kafkatest/tests/connect/connect_test.py diff --git a/tests/kafkatest/tests/templates/connect-distributed.properties b/tests/kafkatest/tests/connect/templates/connect-distributed.properties similarity index 100% rename from tests/kafkatest/tests/templates/connect-distributed.properties rename to tests/kafkatest/tests/connect/templates/connect-distributed.properties diff --git a/tests/kafkatest/tests/templates/connect-file-sink.properties b/tests/kafkatest/tests/connect/templates/connect-file-sink.properties similarity index 100% rename from tests/kafkatest/tests/templates/connect-file-sink.properties rename to tests/kafkatest/tests/connect/templates/connect-file-sink.properties diff --git a/tests/kafkatest/tests/templates/connect-file-source.properties b/tests/kafkatest/tests/connect/templates/connect-file-source.properties similarity index 100% rename from tests/kafkatest/tests/templates/connect-file-source.properties rename to tests/kafkatest/tests/connect/templates/connect-file-source.properties diff --git a/tests/kafkatest/tests/templates/connect-standalone.properties b/tests/kafkatest/tests/connect/templates/connect-standalone.properties similarity index 100% rename from tests/kafkatest/tests/templates/connect-standalone.properties rename to tests/kafkatest/tests/connect/templates/connect-standalone.properties diff --git a/tests/kafkatest/tests/core/__init__.py b/tests/kafkatest/tests/core/__init__.py new file mode 100644 index 0000000000000..ebc9bb3a9a03f --- /dev/null +++ b/tests/kafkatest/tests/core/__init__.py @@ -0,0 +1,15 @@ +# 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. +# see kafka.server.KafkaConfig for additional details and defaults diff --git a/tests/kafkatest/tests/compatibility_test_new_broker_test.py b/tests/kafkatest/tests/core/compatibility_test_new_broker_test.py similarity index 100% rename from tests/kafkatest/tests/compatibility_test_new_broker_test.py rename to tests/kafkatest/tests/core/compatibility_test_new_broker_test.py diff --git a/tests/kafkatest/tests/consumer_group_command_test.py b/tests/kafkatest/tests/core/consumer_group_command_test.py similarity index 100% rename from tests/kafkatest/tests/consumer_group_command_test.py rename to tests/kafkatest/tests/core/consumer_group_command_test.py diff --git a/tests/kafkatest/tests/get_offset_shell_test.py b/tests/kafkatest/tests/core/get_offset_shell_test.py similarity index 100% rename from tests/kafkatest/tests/get_offset_shell_test.py rename to tests/kafkatest/tests/core/get_offset_shell_test.py diff --git a/tests/kafkatest/tests/mirror_maker_test.py b/tests/kafkatest/tests/core/mirror_maker_test.py similarity index 100% rename from tests/kafkatest/tests/mirror_maker_test.py rename to tests/kafkatest/tests/core/mirror_maker_test.py diff --git a/tests/kafkatest/tests/reassign_partitions_test.py b/tests/kafkatest/tests/core/reassign_partitions_test.py similarity index 100% rename from tests/kafkatest/tests/reassign_partitions_test.py rename to tests/kafkatest/tests/core/reassign_partitions_test.py diff --git a/tests/kafkatest/tests/replication_test.py b/tests/kafkatest/tests/core/replication_test.py similarity index 100% rename from tests/kafkatest/tests/replication_test.py rename to tests/kafkatest/tests/core/replication_test.py diff --git a/tests/kafkatest/tests/security_rolling_upgrade_test.py b/tests/kafkatest/tests/core/security_rolling_upgrade_test.py similarity index 100% rename from tests/kafkatest/tests/security_rolling_upgrade_test.py rename to tests/kafkatest/tests/core/security_rolling_upgrade_test.py diff --git a/tests/kafkatest/tests/simple_consumer_shell_test.py b/tests/kafkatest/tests/core/simple_consumer_shell_test.py similarity index 100% rename from tests/kafkatest/tests/simple_consumer_shell_test.py rename to tests/kafkatest/tests/core/simple_consumer_shell_test.py diff --git a/tests/kafkatest/tests/upgrade_test.py b/tests/kafkatest/tests/core/upgrade_test.py similarity index 100% rename from tests/kafkatest/tests/upgrade_test.py rename to tests/kafkatest/tests/core/upgrade_test.py diff --git a/tests/kafkatest/tests/zookeeper_security_upgrade_test.py b/tests/kafkatest/tests/core/zookeeper_security_upgrade_test.py similarity index 100% rename from tests/kafkatest/tests/zookeeper_security_upgrade_test.py rename to tests/kafkatest/tests/core/zookeeper_security_upgrade_test.py diff --git a/tests/kafkatest/tests/streams/__init__.py b/tests/kafkatest/tests/streams/__init__.py new file mode 100644 index 0000000000000..ebc9bb3a9a03f --- /dev/null +++ b/tests/kafkatest/tests/streams/__init__.py @@ -0,0 +1,15 @@ +# 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. +# see kafka.server.KafkaConfig for additional details and defaults diff --git a/tests/kafkatest/tests/streams_bounce_test.py b/tests/kafkatest/tests/streams/streams_bounce_test.py similarity index 100% rename from tests/kafkatest/tests/streams_bounce_test.py rename to tests/kafkatest/tests/streams/streams_bounce_test.py diff --git a/tests/kafkatest/tests/streams_smoke_test.py b/tests/kafkatest/tests/streams/streams_smoke_test.py similarity index 100% rename from tests/kafkatest/tests/streams_smoke_test.py rename to tests/kafkatest/tests/streams/streams_smoke_test.py diff --git a/tests/kafkatest/tests/tools/__init__.py b/tests/kafkatest/tests/tools/__init__.py new file mode 100644 index 0000000000000..ebc9bb3a9a03f --- /dev/null +++ b/tests/kafkatest/tests/tools/__init__.py @@ -0,0 +1,15 @@ +# 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. +# see kafka.server.KafkaConfig for additional details and defaults diff --git a/tests/kafkatest/tests/log4j_appender_test.py b/tests/kafkatest/tests/tools/log4j_appender_test.py similarity index 100% rename from tests/kafkatest/tests/log4j_appender_test.py rename to tests/kafkatest/tests/tools/log4j_appender_test.py From 31e263e8294e94de3d2c44d2ab3a827ab904e247 Mon Sep 17 00:00:00 2001 From: Yasuhiro Matsuda Date: Mon, 4 Apr 2016 14:57:15 -0700 Subject: [PATCH 115/206] HOTFIX: set timestamp in SinkNode guozhangwang Setting the timestamp in produced records in SinkNode. This forces the producer record's timestamp same as the context's timestamp. Author: Yasuhiro Matsuda Reviewers: Guozhang Wang Closes #1137 from ymatsuda/set_timestamp_in_sinknode --- .../org/apache/kafka/streams/processor/internals/SinkNode.java | 2 +- .../src/test/java/org/apache/kafka/test/KStreamTestDriver.java | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java index ffc72fd928753..31a558bf51047 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java @@ -57,7 +57,7 @@ public void init(ProcessorContext context) { public void process(K key, V value) { // send to all the registered topics RecordCollector collector = ((RecordCollector.Supplier) context).recordCollector(); - collector.send(new ProducerRecord<>(topic, key, value), keySerializer, valSerializer, partitioner); + collector.send(new ProducerRecord<>(topic, null, context.timestamp(), key, value), keySerializer, valSerializer, partitioner); } @Override diff --git a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java index 05713c19c3ec9..0c56c26e17655 100644 --- a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java @@ -57,6 +57,7 @@ public KStreamTestDriver(KStreamBuilder builder, this.topology = builder.build("X", null); this.stateDir = stateDir; this.context = new MockProcessorContext(this, stateDir, keySerde, valSerde, new MockRecordCollector()); + this.context.setTime(0L); for (StateStoreSupplier stateStoreSupplier : topology.stateStoreSuppliers()) { StateStore store = stateStoreSupplier.get(); From c3c9289c12342065d6c671098aaace23a328c7ff Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Mon, 4 Apr 2016 18:49:29 -0700 Subject: [PATCH 116/206] KAFKA-3464: Add system tests for Connect with Kafka security enabled Author: Ewen Cheslack-Postava Reviewers: Ismael Juma, Gwen Shapira Closes #1141 from ewencp/kafka-3464-connect-security-system-tests --- tests/kafkatest/services/connect.py | 6 +++ tests/kafkatest/services/mirror_maker.py | 1 - .../services/security/security_config.py | 24 +++++++----- .../tests/connect/connect_distributed_test.py | 36 ++++++++++++++---- tests/kafkatest/tests/connect/connect_test.py | 37 ++++++++++++++----- .../templates/connect-distributed.properties | 7 +++- .../templates/connect-standalone.properties | 4 +- 7 files changed, 86 insertions(+), 29 deletions(-) diff --git a/tests/kafkatest/services/connect.py b/tests/kafkatest/services/connect.py index 76336e1257412..51dade3471b5b 100644 --- a/tests/kafkatest/services/connect.py +++ b/tests/kafkatest/services/connect.py @@ -48,6 +48,7 @@ class ConnectServiceBase(Service): def __init__(self, context, num_nodes, kafka, files): super(ConnectServiceBase, self).__init__(context, num_nodes) self.kafka = kafka + self.security_config = kafka.security_config.client_config() self.files = files def pids(self, node): @@ -89,6 +90,7 @@ def restart(self): def clean_node(self, node): node.account.kill_process("connect", clean_shutdown=False, allow_fail=True) + self.security_config.clean_node(node) node.account.ssh("rm -rf " + " ".join([self.CONFIG_FILE, self.LOG4J_CONFIG_FILE, self.PID_FILE, self.LOG_FILE, self.STDOUT_FILE, self.STDERR_FILE] + self.config_filenames() + self.files), allow_fail=False) def config_filenames(self): @@ -153,6 +155,7 @@ def node(self): def start_cmd(self, node, connector_configs): cmd = "( export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\"; " % self.LOG4J_CONFIG_FILE + cmd += "export KAFKA_OPTS=%s; " % self.security_config.kafka_opts cmd += "/opt/%s/bin/connect-standalone.sh %s " % (kafka_dir(node), self.CONFIG_FILE) cmd += " ".join(connector_configs) cmd += " & echo $! >&3 ) 1>> %s 2>> %s 3> %s" % (self.STDOUT_FILE, self.STDERR_FILE, self.PID_FILE) @@ -161,6 +164,7 @@ def start_cmd(self, node, connector_configs): def start_node(self, node): node.account.ssh("mkdir -p %s" % self.PERSISTENT_ROOT, allow_fail=False) + self.security_config.setup_node(node) node.account.create_file(self.CONFIG_FILE, self.config_template_func(node)) node.account.create_file(self.LOG4J_CONFIG_FILE, self.render('connect_log4j.properties', log_file=self.LOG_FILE)) remote_connector_configs = [] @@ -190,6 +194,7 @@ def __init__(self, context, num_nodes, kafka, files, offsets_topic="connect-offs def start_cmd(self, node): cmd = "( export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\"; " % self.LOG4J_CONFIG_FILE + cmd += "export KAFKA_OPTS=%s; " % self.security_config.kafka_opts cmd += "/opt/%s/bin/connect-distributed.sh %s " % (kafka_dir(node), self.CONFIG_FILE) cmd += " & echo $! >&3 ) 1>> %s 2>> %s 3> %s" % (self.STDOUT_FILE, self.STDERR_FILE, self.PID_FILE) return cmd @@ -197,6 +202,7 @@ def start_cmd(self, node): def start_node(self, node): node.account.ssh("mkdir -p %s" % self.PERSISTENT_ROOT, allow_fail=False) + self.security_config.setup_node(node) node.account.create_file(self.CONFIG_FILE, self.config_template_func(node)) node.account.create_file(self.LOG4J_CONFIG_FILE, self.render('connect_log4j.properties', log_file=self.LOG_FILE)) if self.connector_config_templates: diff --git a/tests/kafkatest/services/mirror_maker.py b/tests/kafkatest/services/mirror_maker.py index 4386788d5f5ab..cb4b2c1ac9a02 100644 --- a/tests/kafkatest/services/mirror_maker.py +++ b/tests/kafkatest/services/mirror_maker.py @@ -1,4 +1,3 @@ - # 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. diff --git a/tests/kafkatest/services/security/security_config.py b/tests/kafkatest/services/security/security_config.py index b5efba81e4f80..1bbabd2359b02 100644 --- a/tests/kafkatest/services/security/security_config.py +++ b/tests/kafkatest/services/security/security_config.py @@ -17,6 +17,7 @@ import subprocess from ducktape.template import TemplateRenderer from kafkatest.services.security.minikdc import MiniKdc +import itertools class Keytool(object): @@ -172,17 +173,22 @@ def kafka_opts(self): else: return "" - def __str__(self): + def props(self, prefix=''): """ - Return properties as string with line separators. + Return properties as string with line separators, optionally with a prefix. This is used to append security config properties to a properties file. + :param prefix: prefix to add to each property + :return: a string containing line-separated properties """ + if self.security_protocol == SecurityConfig.PLAINTEXT: + return "" + config_lines = (prefix + key + "=" + value for key, value in self.properties.iteritems()) + # Extra blank lines ensure this can be appended/prepended safely + return "\n".join(itertools.chain([""], config_lines, [""])) - prop_str = "" - if self.security_protocol != SecurityConfig.PLAINTEXT: - for key, value in self.properties.items(): - prop_str += ("\n" + key + "=" + value) - prop_str += "\n" - return prop_str - + def __str__(self): + """ + Return properties as a string with line separators. + """ + return self.props() diff --git a/tests/kafkatest/tests/connect/connect_distributed_test.py b/tests/kafkatest/tests/connect/connect_distributed_test.py index 9aa16abfbd7a8..698a827b1712a 100644 --- a/tests/kafkatest/tests/connect/connect_distributed_test.py +++ b/tests/kafkatest/tests/connect/connect_distributed_test.py @@ -13,15 +13,19 @@ # See the License for the specific language governing permissions and # limitations under the License. -from kafkatest.tests.kafka_test import KafkaTest +from ducktape.tests.test import Test + +from kafkatest.services.zookeeper import ZookeeperService +from kafkatest.services.kafka import KafkaService from kafkatest.services.connect import ConnectDistributedService, VerifiableSource, VerifiableSink from kafkatest.services.console_consumer import ConsoleConsumer +from kafkatest.services.security.security_config import SecurityConfig from ducktape.utils.util import wait_until from ducktape.mark import matrix import subprocess, itertools, time from collections import Counter -class ConnectDistributedTest(KafkaTest): +class ConnectDistributedTest(Test): """ Simple test of Kafka Connect in distributed mode, producing data from files on one cluster and consuming it on another, validating the total output is identical to the input. @@ -45,22 +49,39 @@ class ConnectDistributedTest(KafkaTest): SCHEMA = { "type": "string", "optional": False } def __init__(self, test_context): - super(ConnectDistributedTest, self).__init__(test_context, num_zk=1, num_brokers=1, topics={ + super(ConnectDistributedTest, self).__init__(test_context) + self.num_zk = 1 + self.num_brokers = 1 + self.topics = { 'test' : { 'partitions': 1, 'replication-factor': 1 } - }) + } + + self.zk = ZookeeperService(test_context, self.num_zk) - self.cc = ConnectDistributedService(test_context, 3, self.kafka, [self.INPUT_FILE, self.OUTPUT_FILE]) - self.cc.log_level = "DEBUG" self.key_converter = "org.apache.kafka.connect.json.JsonConverter" self.value_converter = "org.apache.kafka.connect.json.JsonConverter" self.schemas = True - def test_file_source_and_sink(self): + def setup_services(self, security_protocol=SecurityConfig.PLAINTEXT): + self.kafka = KafkaService(self.test_context, self.num_brokers, self.zk, + security_protocol=security_protocol, interbroker_security_protocol=security_protocol, + topics=self.topics) + + self.cc = ConnectDistributedService(self.test_context, 3, self.kafka, [self.INPUT_FILE, self.OUTPUT_FILE]) + self.cc.log_level = "DEBUG" + + self.zk.start() + self.kafka.start() + + + @matrix(security_protocol=[SecurityConfig.PLAINTEXT, SecurityConfig.SASL_SSL]) + def test_file_source_and_sink(self, security_protocol): """ Tests that a basic file connector works across clean rolling bounces. This validates that the connector is correctly created, tasks instantiated, and as nodes restart the work is rebalanced across nodes. """ + self.setup_services(security_protocol=security_protocol) self.cc.set_configs(lambda node: self.render("connect-distributed.properties", node=node)) self.cc.start() @@ -94,6 +115,7 @@ def test_bounce(self, clean): """ num_tasks = 3 + self.setup_services() self.cc.set_configs(lambda node: self.render("connect-distributed.properties", node=node)) self.cc.start() diff --git a/tests/kafkatest/tests/connect/connect_test.py b/tests/kafkatest/tests/connect/connect_test.py index 90f219a24271a..7b57402bf7cef 100644 --- a/tests/kafkatest/tests/connect/connect_test.py +++ b/tests/kafkatest/tests/connect/connect_test.py @@ -13,14 +13,18 @@ # See the License for the specific language governing permissions and # limitations under the License. -from kafkatest.tests.kafka_test import KafkaTest +from ducktape.tests.test import Test + +from kafkatest.services.zookeeper import ZookeeperService +from kafkatest.services.kafka import KafkaService from kafkatest.services.connect import ConnectStandaloneService from kafkatest.services.console_consumer import ConsoleConsumer +from kafkatest.services.security.security_config import SecurityConfig from ducktape.utils.util import wait_until -from ducktape.mark import parametrize +from ducktape.mark import parametrize, matrix import hashlib, subprocess, json -class ConnectStandaloneFileTest(KafkaTest): +class ConnectStandaloneFileTest(Test): """ Simple test of Kafka Connect that produces data from a file in one standalone process and consumes it on another, validating the output is @@ -42,24 +46,39 @@ class ConnectStandaloneFileTest(KafkaTest): SCHEMA = { "type": "string", "optional": False } def __init__(self, test_context): - super(ConnectStandaloneFileTest, self).__init__(test_context, num_zk=1, num_brokers=1, topics={ + super(ConnectStandaloneFileTest, self).__init__(test_context) + self.num_zk = 1 + self.num_brokers = 1 + self.topics = { 'test' : { 'partitions': 1, 'replication-factor': 1 } - }) + } - self.source = ConnectStandaloneService(test_context, self.kafka, [self.INPUT_FILE, self.OFFSETS_FILE]) - self.sink = ConnectStandaloneService(test_context, self.kafka, [self.OUTPUT_FILE, self.OFFSETS_FILE]) - self.consumer_validator = ConsoleConsumer(test_context, 1, self.kafka, self.TOPIC, consumer_timeout_ms=1000) + self.zk = ZookeeperService(test_context, self.num_zk) @parametrize(converter="org.apache.kafka.connect.json.JsonConverter", schemas=True) @parametrize(converter="org.apache.kafka.connect.json.JsonConverter", schemas=False) @parametrize(converter="org.apache.kafka.connect.storage.StringConverter", schemas=None) - def test_file_source_and_sink(self, converter="org.apache.kafka.connect.json.JsonConverter", schemas=True): + @matrix(security_protocol=[SecurityConfig.PLAINTEXT, SecurityConfig.SASL_SSL]) + def test_file_source_and_sink(self, converter="org.apache.kafka.connect.json.JsonConverter", schemas=True, security_protocol='PLAINTEXT'): assert converter != None, "converter type must be set" # Template parameters self.key_converter = converter self.value_converter = converter self.schemas = schemas + self.kafka = KafkaService(self.test_context, self.num_brokers, self.zk, + security_protocol=security_protocol, interbroker_security_protocol=security_protocol, + topics=self.topics) + + self.source = ConnectStandaloneService(self.test_context, self.kafka, [self.INPUT_FILE, self.OFFSETS_FILE]) + self.sink = ConnectStandaloneService(self.test_context, self.kafka, [self.OUTPUT_FILE, self.OFFSETS_FILE]) + self.consumer_validator = ConsoleConsumer(self.test_context, 1, self.kafka, self.TOPIC, + consumer_timeout_ms=1000, new_consumer=True) + + + self.zk.start() + self.kafka.start() + self.source.set_configs(lambda node: self.render("connect-standalone.properties", node=node), [self.render("connect-file-source.properties")]) self.sink.set_configs(lambda node: self.render("connect-standalone.properties", node=node), [self.render("connect-file-sink.properties")]) diff --git a/tests/kafkatest/tests/connect/templates/connect-distributed.properties b/tests/kafkatest/tests/connect/templates/connect-distributed.properties index 7a7440a4d9072..48f5f789fae1b 100644 --- a/tests/kafkatest/tests/connect/templates/connect-distributed.properties +++ b/tests/kafkatest/tests/connect/templates/connect-distributed.properties @@ -13,7 +13,10 @@ # See the License for the specific language governing permissions and # limitations under the License. -bootstrap.servers={{ kafka.bootstrap_servers() }} +bootstrap.servers={{ kafka.bootstrap_servers(kafka.security_config.security_protocol) }} +{{ kafka.security_config.client_config().props() }} +{{ kafka.security_config.client_config().props("producer.") }} +{{ kafka.security_config.client_config().props("consumer.") }} group.id={{ group|default("connect-cluster") }} @@ -43,4 +46,4 @@ rest.advertised.host.name = {{ node.account.hostname }} # Reduce session timeouts so tests that kill workers don't need to wait as long to recover session.timeout.ms=10000 -consumer.session.timeout.ms=10000 \ No newline at end of file +consumer.session.timeout.ms=10000 diff --git a/tests/kafkatest/tests/connect/templates/connect-standalone.properties b/tests/kafkatest/tests/connect/templates/connect-standalone.properties index bf1daf7bcc079..09c648720c7f8 100644 --- a/tests/kafkatest/tests/connect/templates/connect-standalone.properties +++ b/tests/kafkatest/tests/connect/templates/connect-standalone.properties @@ -13,7 +13,9 @@ # See the License for the specific language governing permissions and # limitations under the License. -bootstrap.servers={{ kafka.bootstrap_servers() }} +bootstrap.servers={{ kafka.bootstrap_servers(kafka.security_config.security_protocol) }} +{{ kafka.security_config.client_config().props("producer.") }} +{{ kafka.security_config.client_config().props("consumer.") }} key.converter={{ key_converter|default("org.apache.kafka.connect.json.JsonConverter") }} value.converter={{ value_converter|default("org.apache.kafka.connect.json.JsonConverter") }} From 4ff667711ffd477872adbd3e3e39bd94eac6e763 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Mon, 4 Apr 2016 21:28:59 -0700 Subject: [PATCH 117/206] KAFKA-2998: log warnings when client is disconnected from bootstrap brokers Author: Jason Gustafson Reviewers: Grant Henke, Guozhang Wang Closes #769 from hachikuji/KAFKA-2998 --- .../org/apache/kafka/clients/NetworkClient.java | 8 ++++++++ .../java/org/apache/kafka/common/Cluster.java | 17 +++++++++++++++-- 2 files changed, 23 insertions(+), 2 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index d22b508cd878f..d2eaace98b833 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -556,6 +556,14 @@ public boolean maybeHandleDisconnection(ClientRequest request) { ApiKeys requestKey = ApiKeys.forId(request.request().header().apiKey()); if (requestKey == ApiKeys.METADATA) { + Cluster cluster = metadata.fetch(); + if (cluster.isBootstrapConfigured()) { + int nodeId = Integer.parseInt(request.request().destination()); + Node node = cluster.nodeById(nodeId); + if (node != null) + log.warn("Bootstrap broker {}:{} disconnected", node.host(), node.port()); + } + metadataFetchInProgress = false; return true; } diff --git a/clients/src/main/java/org/apache/kafka/common/Cluster.java b/clients/src/main/java/org/apache/kafka/common/Cluster.java index 8e85df8f0903e..e1bf581b3e5d1 100644 --- a/clients/src/main/java/org/apache/kafka/common/Cluster.java +++ b/clients/src/main/java/org/apache/kafka/common/Cluster.java @@ -29,6 +29,7 @@ */ public final class Cluster { + private final boolean isBootstrapConfigured; private final List nodes; private final Set unauthorizedTopics; private final Map partitionsByTopicPartition; @@ -45,11 +46,19 @@ public final class Cluster { public Cluster(Collection nodes, Collection partitions, Set unauthorizedTopics) { + this(false, nodes, partitions, unauthorizedTopics); + } + + private Cluster(boolean isBootstrapConfigured, + Collection nodes, + Collection partitions, + Set unauthorizedTopics) { + this.isBootstrapConfigured = isBootstrapConfigured; + // make a randomized, unmodifiable copy of the nodes List copy = new ArrayList<>(nodes); Collections.shuffle(copy); this.nodes = Collections.unmodifiableList(copy); - this.nodesById = new HashMap<>(); for (Node node : nodes) this.nodesById.put(node.id(), node); @@ -115,7 +124,7 @@ public static Cluster bootstrap(List addresses) { int nodeId = -1; for (InetSocketAddress address : addresses) nodes.add(new Node(nodeId--, address.getHostString(), address.getPort())); - return new Cluster(nodes, new ArrayList(0), Collections.emptySet()); + return new Cluster(true, nodes, new ArrayList(0), Collections.emptySet()); } /** @@ -214,6 +223,10 @@ public Set unauthorizedTopics() { return unauthorizedTopics; } + public boolean isBootstrapConfigured() { + return isBootstrapConfigured; + } + @Override public String toString() { return "Cluster(nodes = " + this.nodes + ", partitions = " + this.partitionsByTopicPartition.values() + ")"; From 732b111f458454552411ce787c0b9f9fff8db642 Mon Sep 17 00:00:00 2001 From: Matt McClure Date: Mon, 4 Apr 2016 22:07:20 -0700 Subject: [PATCH 118/206] KAFKA-3384: Conform to POSIX kill usage I believe this addresses KAFKA-3384. The POSIX kill manpage is at http://pubs.opengroup.org/onlinepubs/9699919799/utilities/kill.html Author: Matt McClure Reviewers: Geoff Anderson , Ewen Cheslack-Postava Closes #1148 from matthewlmcclure/KAFKA-3384 --- bin/kafka-server-stop.sh | 2 +- bin/zookeeper-server-stop.sh | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/bin/kafka-server-stop.sh b/bin/kafka-server-stop.sh index f75ded78ea491..d3c660cab5a77 100755 --- a/bin/kafka-server-stop.sh +++ b/bin/kafka-server-stop.sh @@ -19,6 +19,6 @@ if [ -z "$PIDS" ]; then echo "No kafka server to stop" exit 1 else - kill -SIGTERM $PIDS + kill -s TERM $PIDS fi diff --git a/bin/zookeeper-server-stop.sh b/bin/zookeeper-server-stop.sh index 07c79102b466c..f771064cb5508 100755 --- a/bin/zookeeper-server-stop.sh +++ b/bin/zookeeper-server-stop.sh @@ -19,6 +19,6 @@ if [ -z "$PIDS" ]; then echo "No zookeeper server to stop" exit 1 else - kill -SIGTERM $PIDS + kill -s TERM $PIDS fi From aee8ebb46cd9393a0886c42cb88b080d065da397 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 5 Apr 2016 11:46:04 -0700 Subject: [PATCH 119/206] KAFKA-3510; OffsetIndex thread safety * Make all fields accessed outside of a lock `volatile` * Only allow mutation within the class * Remove unnecessary `AtomicInteger` since mutation always happens inside a lock Author: Ismael Juma Reviewers: Guozhang Wang Closes #1188 from ijuma/kafka-3510-offset-index-thread-safety --- core/src/main/scala/kafka/log/Log.scala | 2 +- .../src/main/scala/kafka/log/LogSegment.scala | 2 +- .../main/scala/kafka/log/OffsetIndex.scala | 173 +++++++++--------- .../scala/kafka/tools/DumpLogSegments.scala | 2 +- .../unit/kafka/log/OffsetIndexTest.scala | 10 +- 5 files changed, 98 insertions(+), 91 deletions(-) diff --git a/core/src/main/scala/kafka/log/Log.scala b/core/src/main/scala/kafka/log/Log.scala index 81c19fae93185..8465b649ac770 100644 --- a/core/src/main/scala/kafka/log/Log.scala +++ b/core/src/main/scala/kafka/log/Log.scala @@ -215,7 +215,7 @@ class Log(val dir: File, val fileName = logFile.getName val startOffset = fileName.substring(0, fileName.length - LogFileSuffix.length).toLong val indexFile = new File(CoreUtils.replaceSuffix(logFile.getPath, LogFileSuffix, IndexFileSuffix) + SwapFileSuffix) - val index = new OffsetIndex(file = indexFile, baseOffset = startOffset, maxIndexSize = config.maxIndexSize) + val index = new OffsetIndex(indexFile, baseOffset = startOffset, maxIndexSize = config.maxIndexSize) val swapSegment = new LogSegment(new FileMessageSet(file = swapFile), index = index, baseOffset = startOffset, diff --git a/core/src/main/scala/kafka/log/LogSegment.scala b/core/src/main/scala/kafka/log/LogSegment.scala index 9fc68a4a92350..3a4bbc86d6c80 100755 --- a/core/src/main/scala/kafka/log/LogSegment.scala +++ b/core/src/main/scala/kafka/log/LogSegment.scala @@ -55,7 +55,7 @@ class LogSegment(val log: FileMessageSet, def this(dir: File, startOffset: Long, indexIntervalBytes: Int, maxIndexSize: Int, rollJitterMs: Long, time: Time, fileAlreadyExists: Boolean = false, initFileSize: Int = 0, preallocate: Boolean = false) = this(new FileMessageSet(file = Log.logFilename(dir, startOffset), fileAlreadyExists = fileAlreadyExists, initFileSize = initFileSize, preallocate = preallocate), - new OffsetIndex(file = Log.indexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize), + new OffsetIndex(Log.indexFilename(dir, startOffset), baseOffset = startOffset, maxIndexSize = maxIndexSize), startOffset, indexIntervalBytes, rollJitterMs, diff --git a/core/src/main/scala/kafka/log/OffsetIndex.scala b/core/src/main/scala/kafka/log/OffsetIndex.scala index e95c9d139ecd5..ce35d6874c7c0 100755 --- a/core/src/main/scala/kafka/log/OffsetIndex.scala +++ b/core/src/main/scala/kafka/log/OffsetIndex.scala @@ -24,7 +24,6 @@ import java.io._ import java.nio._ import java.nio.channels._ import java.util.concurrent.locks._ -import java.util.concurrent.atomic._ import kafka.utils._ import kafka.utils.CoreUtils.inLock import kafka.common.InvalidOffsetException @@ -54,62 +53,70 @@ import kafka.common.InvalidOffsetException * All external APIs translate from relative offsets to full offsets, so users of this class do not interact with the internal * storage format. */ -class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSize: Int = -1) extends Logging { +class OffsetIndex(@volatile private[this] var _file: File, val baseOffset: Long, val maxIndexSize: Int = -1) extends Logging { private val lock = new ReentrantLock /* initialize the memory mapping for this index */ - private var mmap: MappedByteBuffer = - { - val newlyCreated = file.createNewFile() - val raf = new RandomAccessFile(file, "rw") - try { - /* pre-allocate the file if necessary */ - if(newlyCreated) { - if(maxIndexSize < 8) - throw new IllegalArgumentException("Invalid max index size: " + maxIndexSize) - raf.setLength(roundToExactMultiple(maxIndexSize, 8)) - } - - /* memory-map the file */ - val len = raf.length() - val idx = raf.getChannel.map(FileChannel.MapMode.READ_WRITE, 0, len) - - /* set the position in the index for the next entry */ - if(newlyCreated) - idx.position(0) - else - // if this is a pre-existing index, assume it is all valid and set position to last entry - idx.position(roundToExactMultiple(idx.limit, 8)) - idx - } finally { - CoreUtils.swallow(raf.close()) + @volatile + private[this] var mmap: MappedByteBuffer = { + val newlyCreated = _file.createNewFile() + val raf = new RandomAccessFile(_file, "rw") + try { + /* pre-allocate the file if necessary */ + if (newlyCreated) { + if (maxIndexSize < 8) + throw new IllegalArgumentException("Invalid max index size: " + maxIndexSize) + raf.setLength(roundToExactMultiple(maxIndexSize, 8)) } + + /* memory-map the file */ + val len = raf.length() + val idx = raf.getChannel.map(FileChannel.MapMode.READ_WRITE, 0, len) + + /* set the position in the index for the next entry */ + if (newlyCreated) + idx.position(0) + else + // if this is a pre-existing index, assume it is all valid and set position to last entry + idx.position(roundToExactMultiple(idx.limit, 8)) + idx + } finally { + CoreUtils.swallow(raf.close()) } - + } + /* the number of eight-byte entries currently in the index */ - private var size = new AtomicInteger(mmap.position / 8) - - /** - * The maximum number of eight-byte entries this index can hold - */ @volatile - var maxEntries = mmap.limit / 8 - - /* the last offset in the index */ - var lastOffset = readLastEntry.offset + private[this] var _entries = mmap.position / 8 + + /* The maximum number of eight-byte entries this index can hold */ + @volatile + private[this] var _maxEntries = mmap.limit / 8 + + @volatile + private[this] var _lastOffset = readLastEntry.offset debug("Loaded index file %s with maxEntries = %d, maxIndexSize = %d, entries = %d, lastOffset = %d, file position = %d" - .format(file.getAbsolutePath, maxEntries, maxIndexSize, entries(), lastOffset, mmap.position)) + .format(_file.getAbsolutePath, _maxEntries, maxIndexSize, _entries, _lastOffset, mmap.position)) + + /** The maximum number of entries this index can hold */ + def maxEntries: Int = _maxEntries + + /** The last offset in the index */ + def lastOffset: Long = _lastOffset + + /** The index file */ + def file: File = _file /** * The last entry in the index */ def readLastEntry(): OffsetPosition = { inLock(lock) { - size.get match { + _entries match { case 0 => OffsetPosition(baseOffset, 0) - case s => OffsetPosition(baseOffset + relativeOffset(this.mmap, s-1), physical(this.mmap, s-1)) + case s => OffsetPosition(baseOffset + relativeOffset(mmap, s - 1), physical(mmap, s - 1)) } } } @@ -149,22 +156,22 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi val relOffset = targetOffset - baseOffset // check if the index is empty - if(entries == 0) + if (_entries == 0) return -1 // check if the target offset is smaller than the least offset - if(relativeOffset(idx, 0) > relOffset) + if (relativeOffset(idx, 0) > relOffset) return -1 // binary search for the entry var lo = 0 - var hi = entries-1 - while(lo < hi) { + var hi = _entries - 1 + while (lo < hi) { val mid = ceil(hi/2.0 + lo/2.0).toInt val found = relativeOffset(idx, mid) - if(found == relOffset) + if (found == relOffset) return mid - else if(found < relOffset) + else if (found < relOffset) lo = mid else hi = mid - 1 @@ -185,8 +192,8 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi */ def entry(n: Int): OffsetPosition = { maybeLock(lock) { - if(n >= entries) - throw new IllegalArgumentException("Attempt to fetch the %dth entry from an index of size %d.".format(n, entries)) + if(n >= _entries) + throw new IllegalArgumentException("Attempt to fetch the %dth entry from an index of size %d.".format(n, _entries)) val idx = mmap.duplicate OffsetPosition(relativeOffset(idx, n), physical(idx, n)) } @@ -197,17 +204,17 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi */ def append(offset: Long, position: Int) { inLock(lock) { - require(!isFull, "Attempt to append to a full index (size = " + size + ").") - if (size.get == 0 || offset > lastOffset) { - debug("Adding index entry %d => %d to %s.".format(offset, position, file.getName)) - this.mmap.putInt((offset - baseOffset).toInt) - this.mmap.putInt(position) - this.size.incrementAndGet() - this.lastOffset = offset - require(entries * 8 == mmap.position, entries + " entries but file position in index is " + mmap.position + ".") + require(!isFull, "Attempt to append to a full index (size = " + _entries + ").") + if (_entries == 0 || offset > _lastOffset) { + debug("Adding index entry %d => %d to %s.".format(offset, position, _file.getName)) + mmap.putInt((offset - baseOffset).toInt) + mmap.putInt(position) + _entries += 1 + _lastOffset = offset + require(_entries * 8 == mmap.position, _entries + " entries but file position in index is " + mmap.position + ".") } else { throw new InvalidOffsetException("Attempt to append an offset (%d) to position %d no larger than the last offset appended (%d) to %s." - .format(offset, entries, lastOffset, file.getAbsolutePath)) + .format(offset, _entries, _lastOffset, _file.getAbsolutePath)) } } } @@ -215,7 +222,7 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi /** * True iff there are no more slots available in this index */ - def isFull: Boolean = entries >= this.maxEntries + def isFull: Boolean = _entries >= _maxEntries /** * Truncate the entire index, deleting all entries @@ -252,9 +259,9 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi */ private def truncateToEntries(entries: Int) { inLock(lock) { - this.size.set(entries) - mmap.position(this.size.get * 8) - this.lastOffset = readLastEntry.offset + _entries = entries + mmap.position(_entries * 8) + _lastOffset = readLastEntry.offset } } @@ -264,7 +271,7 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi */ def trimToValidSize() { inLock(lock) { - resize(entries * 8) + resize(_entries * 8) } } @@ -276,18 +283,18 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi */ def resize(newSize: Int) { inLock(lock) { - val raf = new RandomAccessFile(file, "rw") + val raf = new RandomAccessFile(_file, "rw") val roundedNewSize = roundToExactMultiple(newSize, 8) - val position = this.mmap.position + val position = mmap.position /* Windows won't let us modify the file length while the file is mmapped :-( */ - if(Os.isWindows) - forceUnmap(this.mmap) + if (Os.isWindows) + forceUnmap(mmap) try { raf.setLength(roundedNewSize) - this.mmap = raf.getChannel().map(FileChannel.MapMode.READ_WRITE, 0, roundedNewSize) - this.maxEntries = this.mmap.limit / 8 - this.mmap.position(position) + mmap = raf.getChannel().map(FileChannel.MapMode.READ_WRITE, 0, roundedNewSize) + _maxEntries = mmap.limit / 8 + mmap.position(position) } finally { CoreUtils.swallow(raf.close()) } @@ -319,19 +326,19 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi * Delete this index file */ def delete(): Boolean = { - info("Deleting index " + this.file.getAbsolutePath) - if(Os.isWindows) - CoreUtils.swallow(forceUnmap(this.mmap)) - this.file.delete() + info("Deleting index " + _file.getAbsolutePath) + if (Os.isWindows) + CoreUtils.swallow(forceUnmap(mmap)) + _file.delete() } /** The number of entries in this index */ - def entries() = size.get + def entries = _entries /** * The number of bytes actually used by this index */ - def sizeInBytes() = 8 * entries + def sizeInBytes() = 8 * _entries /** Close the index */ def close() { @@ -343,8 +350,8 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi * @throws IOException if rename fails */ def renameTo(f: File) { - try Utils.atomicMoveWithFallback(file.toPath, f.toPath) - finally this.file = f + try Utils.atomicMoveWithFallback(_file.toPath, f.toPath) + finally _file = f } /** @@ -352,13 +359,13 @@ class OffsetIndex(@volatile var file: File, val baseOffset: Long, val maxIndexSi * @throws IllegalArgumentException if any problems are found */ def sanityCheck() { - require(entries == 0 || lastOffset > baseOffset, + require(_entries == 0 || lastOffset > baseOffset, "Corrupt index found, index file (%s) has non-zero size but the last offset is %d and the base offset is %d" - .format(file.getAbsolutePath, lastOffset, baseOffset)) - val len = file.length() - require(len % 8 == 0, - "Index file " + file.getName + " is corrupt, found " + len + - " bytes which is not positive or not a multiple of 8.") + .format(_file.getAbsolutePath, lastOffset, baseOffset)) + val len = _file.length() + require(len % 8 == 0, + "Index file " + _file.getName + " is corrupt, found " + len + + " bytes which is not positive or not a multiple of 8.") } /** diff --git a/core/src/main/scala/kafka/tools/DumpLogSegments.scala b/core/src/main/scala/kafka/tools/DumpLogSegments.scala index e882a300aac9b..dc99672136b2b 100755 --- a/core/src/main/scala/kafka/tools/DumpLogSegments.scala +++ b/core/src/main/scala/kafka/tools/DumpLogSegments.scala @@ -124,7 +124,7 @@ object DumpLogSegments { val startOffset = file.getName().split("\\.")(0).toLong val logFile = new File(file.getAbsoluteFile.getParent, file.getName.split("\\.")(0) + Log.LogFileSuffix) val messageSet = new FileMessageSet(logFile, false) - val index = new OffsetIndex(file = file, baseOffset = startOffset) + val index = new OffsetIndex(file, baseOffset = startOffset) //Check that index passes sanityCheck, this is the check that determines if indexes will be rebuilt on startup or not. if (indexSanityOnly) { diff --git a/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala b/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala index dfd7b54fcf20c..869e618b9ef57 100644 --- a/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala +++ b/core/src/test/scala/unit/kafka/log/OffsetIndexTest.scala @@ -34,7 +34,7 @@ class OffsetIndexTest extends JUnitSuite { @Before def setup() { - this.idx = new OffsetIndex(file = nonExistantTempFile(), baseOffset = 45L, maxIndexSize = 30 * 8) + this.idx = new OffsetIndex(nonExistantTempFile(), baseOffset = 45L, maxIndexSize = 30 * 8) } @After @@ -103,7 +103,7 @@ class OffsetIndexTest extends JUnitSuite { idx.append(first.offset, first.position) idx.append(sec.offset, sec.position) idx.close() - val idxRo = new OffsetIndex(file = idx.file, baseOffset = idx.baseOffset) + val idxRo = new OffsetIndex(idx.file, baseOffset = idx.baseOffset) assertEquals(first, idxRo.lookup(first.offset)) assertEquals(sec, idxRo.lookup(sec.offset)) assertEquals(sec.offset, idxRo.lastOffset) @@ -113,7 +113,7 @@ class OffsetIndexTest extends JUnitSuite { @Test def truncate() { - val idx = new OffsetIndex(file = nonExistantTempFile(), baseOffset = 0L, maxIndexSize = 10 * 8) + val idx = new OffsetIndex(nonExistantTempFile(), baseOffset = 0L, maxIndexSize = 10 * 8) idx.truncate() for(i <- 1 until 10) idx.append(i, i) @@ -140,7 +140,7 @@ class OffsetIndexTest extends JUnitSuite { idx.append(5, 5) idx.truncate() - assertEquals("Full truncation should leave no entries", 0, idx.entries()) + assertEquals("Full truncation should leave no entries", 0, idx.entries) idx.append(0, 0) } @@ -169,4 +169,4 @@ class OffsetIndexTest extends JUnitSuite { file.delete() file } -} \ No newline at end of file +} From 893e79af88016f1d8659e72d48a25d4a825b8867 Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 5 Apr 2016 18:16:48 -0400 Subject: [PATCH 120/206] KAFKA-3489; Update request metrics if a client closes a connection while the broker response is in flight I also fixed a few issues in `SocketServerTest` and included a few clean-ups. Author: Ismael Juma Reviewers: Jun Rao Closes #1172 from ijuma/kafka-3489-update-request-metrics-if-client-closes --- .../apache/kafka/common/network/Selector.java | 6 +- .../scala/kafka/network/RequestChannel.scala | 53 ++--- .../scala/kafka/network/SocketServer.scala | 185 ++++++++++-------- .../unit/kafka/network/SocketServerTest.scala | 141 ++++++++++--- 4 files changed, 257 insertions(+), 128 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/network/Selector.java b/clients/src/main/java/org/apache/kafka/common/network/Selector.java index 698b99c3b8384..c333741818218 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/Selector.java +++ b/clients/src/main/java/org/apache/kafka/common/network/Selector.java @@ -491,7 +491,7 @@ public boolean isChannelReady(String id) { private KafkaChannel channelOrFail(String id) { KafkaChannel channel = this.channels.get(id); if (channel == null) - throw new IllegalStateException("Attempt to retrieve channel for which there is no open connection. Connection id " + id + " existing connections " + channels.keySet().toString()); + throw new IllegalStateException("Attempt to retrieve channel for which there is no open connection. Connection id " + id + " existing connections " + channels.keySet()); return channel; } @@ -551,7 +551,7 @@ private void addToStagedReceives(KafkaChannel channel, NetworkReceive receive) { * checks if there are any staged receives and adds to completedReceives */ private void addToCompletedReceives() { - if (this.stagedReceives.size() > 0) { + if (!this.stagedReceives.isEmpty()) { Iterator>> iter = this.stagedReceives.entrySet().iterator(); while (iter.hasNext()) { Map.Entry> entry = iter.next(); @@ -561,7 +561,7 @@ private void addToCompletedReceives() { NetworkReceive networkReceive = deque.poll(); this.completedReceives.add(networkReceive); this.sensors.recordBytesReceived(channel.id(), networkReceive.payload().limit()); - if (deque.size() == 0) + if (deque.isEmpty()) iter.remove(); } } diff --git a/core/src/main/scala/kafka/network/RequestChannel.scala b/core/src/main/scala/kafka/network/RequestChannel.scala index 1105802e0890d..17c5b9b3fbeda 100644 --- a/core/src/main/scala/kafka/network/RequestChannel.scala +++ b/core/src/main/scala/kafka/network/RequestChannel.scala @@ -117,36 +117,39 @@ object RequestChannel extends Logging { if (apiRemoteCompleteTimeMs < 0) apiRemoteCompleteTimeMs = responseCompleteTimeMs - val requestQueueTime = (requestDequeueTimeMs - startTimeMs).max(0L) - val apiLocalTime = (apiLocalCompleteTimeMs - requestDequeueTimeMs).max(0L) - val apiRemoteTime = (apiRemoteCompleteTimeMs - apiLocalCompleteTimeMs).max(0L) - val apiThrottleTime = (responseCompleteTimeMs - apiRemoteCompleteTimeMs).max(0L) - val responseQueueTime = (responseDequeueTimeMs - responseCompleteTimeMs).max(0L) - val responseSendTime = (endTimeMs - responseDequeueTimeMs).max(0L) + val requestQueueTime = math.max(requestDequeueTimeMs - startTimeMs, 0) + val apiLocalTime = math.max(apiLocalCompleteTimeMs - requestDequeueTimeMs, 0) + val apiRemoteTime = math.max(apiRemoteCompleteTimeMs - apiLocalCompleteTimeMs, 0) + val apiThrottleTime = math.max(responseCompleteTimeMs - apiRemoteCompleteTimeMs, 0) + val responseQueueTime = math.max(responseDequeueTimeMs - responseCompleteTimeMs, 0) + val responseSendTime = math.max(endTimeMs - responseDequeueTimeMs, 0) val totalTime = endTimeMs - startTimeMs - var metricsList = List(RequestMetrics.metricsMap(ApiKeys.forId(requestId).name)) - if (requestId == ApiKeys.FETCH.id) { - val isFromFollower = requestObj.asInstanceOf[FetchRequest].isFromFollower - metricsList ::= ( if (isFromFollower) - RequestMetrics.metricsMap(RequestMetrics.followFetchMetricName) - else - RequestMetrics.metricsMap(RequestMetrics.consumerFetchMetricName) ) - } - metricsList.foreach{ - m => m.requestRate.mark() - m.requestQueueTimeHist.update(requestQueueTime) - m.localTimeHist.update(apiLocalTime) - m.remoteTimeHist.update(apiRemoteTime) - m.throttleTimeHist.update(apiThrottleTime) - m.responseQueueTimeHist.update(responseQueueTime) - m.responseSendTimeHist.update(responseSendTime) - m.totalTimeHist.update(totalTime) + val fetchMetricNames = + if (requestId == ApiKeys.FETCH.id) { + val isFromFollower = requestObj.asInstanceOf[FetchRequest].isFromFollower + Seq( + if (isFromFollower) RequestMetrics.followFetchMetricName + else RequestMetrics.consumerFetchMetricName + ) + } + else Seq.empty + val metricNames = fetchMetricNames :+ ApiKeys.forId(requestId).name + metricNames.foreach { metricName => + val m = RequestMetrics.metricsMap(metricName) + m.requestRate.mark() + m.requestQueueTimeHist.update(requestQueueTime) + m.localTimeHist.update(apiLocalTime) + m.remoteTimeHist.update(apiRemoteTime) + m.throttleTimeHist.update(apiThrottleTime) + m.responseQueueTimeHist.update(responseQueueTime) + m.responseSendTimeHist.update(responseSendTime) + m.totalTimeHist.update(totalTime) } - if(requestLogger.isTraceEnabled) + if (requestLogger.isTraceEnabled) requestLogger.trace("Completed request:%s from connection %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d,securityProtocol:%s,principal:%s" .format(requestDesc(true), connectionId, totalTime, requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime, securityProtocol, session.principal)) - else if(requestLogger.isDebugEnabled) + else if (requestLogger.isDebugEnabled) requestLogger.debug("Completed request:%s from connection %s;totalTime:%d,requestQueueTime:%d,localTime:%d,remoteTime:%d,responseQueueTime:%d,sendTime:%d,securityProtocol:%s,principal:%s" .format(requestDesc(false), connectionId, totalTime, requestQueueTime, apiLocalTime, apiRemoteTime, responseQueueTime, responseSendTime, securityProtocol, session.principal)) } diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index 5c31ac656607a..f1ec2ef6c4515 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -31,9 +31,8 @@ import kafka.common.KafkaException import kafka.metrics.KafkaMetricsGroup import kafka.server.KafkaConfig import kafka.utils._ -import org.apache.kafka.common.MetricName import org.apache.kafka.common.metrics._ -import org.apache.kafka.common.network.{Selector => KSelector, LoginType, Mode, ChannelBuilders} +import org.apache.kafka.common.network.{ChannelBuilders, KafkaChannel, LoginType, Mode, Selector => KSelector} import org.apache.kafka.common.security.auth.KafkaPrincipal import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.kafka.common.protocol.types.SchemaException @@ -41,7 +40,7 @@ import org.apache.kafka.common.utils.{Time, Utils} import scala.collection._ import JavaConverters._ -import scala.util.control.{NonFatal, ControlThrowable} +import scala.util.control.{ControlThrowable, NonFatal} /** * An NIO socket server. The threading model is @@ -83,8 +82,6 @@ class SocketServer(val config: KafkaConfig, val metrics: Metrics, val time: Time val sendBufferSize = config.socketSendBufferBytes val recvBufferSize = config.socketReceiveBufferBytes - val maxRequestSize = config.socketRequestMaxBytes - val connectionsMaxIdleMs = config.connectionsMaxIdleMs val brokerId = config.brokerId var processorBeginIndex = 0 @@ -92,18 +89,8 @@ class SocketServer(val config: KafkaConfig, val metrics: Metrics, val time: Time val protocol = endpoint.protocolType val processorEndIndex = processorBeginIndex + numProcessorThreads - for (i <- processorBeginIndex until processorEndIndex) { - processors(i) = new Processor(i, - time, - maxRequestSize, - requestChannel, - connectionQuotas, - connectionsMaxIdleMs, - protocol, - config.values, - metrics - ) - } + for (i <- processorBeginIndex until processorEndIndex) + processors(i) = newProcessor(i, connectionQuotas, protocol) val acceptor = new Acceptor(endpoint, sendBufferSize, recvBufferSize, brokerId, processors.slice(processorBeginIndex, processorEndIndex), connectionQuotas) @@ -148,10 +135,27 @@ class SocketServer(val config: KafkaConfig, val metrics: Metrics, val time: Time } } + /* `protected` for test usage */ + protected[network] def newProcessor(id: Int, connectionQuotas: ConnectionQuotas, protocol: SecurityProtocol): Processor = { + new Processor(id, + time, + config.socketRequestMaxBytes, + requestChannel, + connectionQuotas, + config.connectionsMaxIdleMs, + protocol, + config.values, + metrics + ) + } + /* For test usage */ private[network] def connectionCount(address: InetAddress): Int = Option(connectionQuotas).fold(0)(_.get(address)) + /* For test usage */ + private[network] def processor(index: Int): Processor = processors(index) + } /** @@ -376,10 +380,7 @@ private[kafka] class Processor(val id: Int, private val newConnections = new ConcurrentLinkedQueue[SocketChannel]() private val inflightResponses = mutable.Map[String, RequestChannel.Response]() - private val channelBuilder = ChannelBuilders.create(protocol, Mode.SERVER, LoginType.SERVER, channelConfigs) - private val metricTags = new util.HashMap[String, String]() - metricTags.put("networkProcessor", id.toString) - + private val metricTags = Map("networkProcessor" -> id.toString).asJava newGauge("IdlePercent", new Gauge[Double] { @@ -398,65 +399,27 @@ private[kafka] class Processor(val id: Int, "socket-server", metricTags, false, - channelBuilder) + ChannelBuilders.create(protocol, Mode.SERVER, LoginType.SERVER, channelConfigs)) override def run() { startupComplete() - while(isRunning) { + while (isRunning) { try { // setup any new connections that have been queued up configureNewConnections() // register any new responses for writing processNewResponses() - - try { - selector.poll(300) - } catch { - case e @ (_: IllegalStateException | _: IOException) => - error("Closing processor %s due to illegal state or IO exception".format(id)) - swallow(closeAll()) - shutdownComplete() - throw e - } - selector.completedReceives.asScala.foreach { receive => - try { - val channel = selector.channel(receive.source) - val session = RequestChannel.Session(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, channel.principal.getName), - channel.socketAddress) - val req = RequestChannel.Request(processor = id, connectionId = receive.source, session = session, buffer = receive.payload, startTimeMs = time.milliseconds, securityProtocol = protocol) - requestChannel.sendRequest(req) - selector.mute(receive.source) - } catch { - case e @ (_: InvalidRequestException | _: SchemaException) => - // note that even though we got an exception, we can assume that receive.source is valid. Issues with constructing a valid receive object were handled earlier - error("Closing socket for " + receive.source + " because of error", e) - close(selector, receive.source) - } - } - - selector.completedSends.asScala.foreach { send => - val resp = inflightResponses.remove(send.destination).getOrElse { - throw new IllegalStateException(s"Send for ${send.destination} completed, but not in `inflightResponses`") - } - resp.request.updateRequestMetrics() - selector.unmute(send.destination) - } - - selector.disconnected.asScala.foreach { connectionId => - val remoteHost = ConnectionId.fromString(connectionId).getOrElse { - throw new IllegalStateException(s"connectionId has unexpected format: $connectionId") - }.remoteHost - // the channel has been closed by the selector but the quotas still need to be updated - connectionQuotas.dec(InetAddress.getByName(remoteHost)) - } - + poll() + processCompletedReceives() + processCompletedSends() + processDisconnected() } catch { // We catch all the throwables here to prevent the processor thread from exiting. We do this because - // letting a processor exit might cause bigger impact on the broker. Usually the exceptions thrown would + // letting a processor exit might cause a bigger impact on the broker. Usually the exceptions thrown would // be either associated with a specific socket channel or a bad request. We just ignore the bad socket channel // or request. This behavior might need to be reviewed if we see an exception that need the entire broker to stop. - case e : ControlThrowable => throw e - case e : Throwable => + case e: ControlThrowable => throw e + case e: Throwable => error("Processor got uncaught exception.", e) } } @@ -468,7 +431,7 @@ private[kafka] class Processor(val id: Int, private def processNewResponses() { var curr = requestChannel.receiveResponse(id) - while(curr != null) { + while (curr != null) { try { curr.responseAction match { case RequestChannel.NoOpAction => @@ -478,9 +441,7 @@ private[kafka] class Processor(val id: Int, trace("Socket server received empty response to send, registering for read: " + curr) selector.unmute(curr.request.connectionId) case RequestChannel.SendAction => - trace("Socket server received response to send, registering for write and sending data: " + curr) - selector.send(curr.responseSend) - inflightResponses += (curr.request.connectionId -> curr) + sendResponse(curr) case RequestChannel.CloseConnectionAction => curr.request.updateRequestMetrics trace("Closing socket connection actively according to the response code.") @@ -492,6 +453,71 @@ private[kafka] class Processor(val id: Int, } } + /* `protected` for test usage */ + protected[network] def sendResponse(response: RequestChannel.Response) { + trace(s"Socket server received response to send, registering for write and sending data: $response") + val channel = selector.channel(response.responseSend.destination) + // `channel` can be null if the selector closed the connection because it was idle for too long + if (channel == null) { + warn(s"Attempting to send response via channel for which there is no open connection, connection id $id") + response.request.updateRequestMetrics() + } + else { + selector.send(response.responseSend) + inflightResponses += (response.request.connectionId -> response) + } + } + + private def poll() { + try selector.poll(300) + catch { + case e @ (_: IllegalStateException | _: IOException) => + error(s"Closing processor $id due to illegal state or IO exception") + swallow(closeAll()) + shutdownComplete() + throw e + } + } + + private def processCompletedReceives() { + selector.completedReceives.asScala.foreach { receive => + try { + val channel = selector.channel(receive.source) + val session = RequestChannel.Session(new KafkaPrincipal(KafkaPrincipal.USER_TYPE, channel.principal.getName), + channel.socketAddress) + val req = RequestChannel.Request(processor = id, connectionId = receive.source, session = session, buffer = receive.payload, startTimeMs = time.milliseconds, securityProtocol = protocol) + requestChannel.sendRequest(req) + selector.mute(receive.source) + } catch { + case e @ (_: InvalidRequestException | _: SchemaException) => + // note that even though we got an exception, we can assume that receive.source is valid. Issues with constructing a valid receive object were handled earlier + error(s"Closing socket for ${receive.source} because of error", e) + close(selector, receive.source) + } + } + } + + private def processCompletedSends() { + selector.completedSends.asScala.foreach { send => + val resp = inflightResponses.remove(send.destination).getOrElse { + throw new IllegalStateException(s"Send for ${send.destination} completed, but not in `inflightResponses`") + } + resp.request.updateRequestMetrics() + selector.unmute(send.destination) + } + } + + private def processDisconnected() { + selector.disconnected.asScala.foreach { connectionId => + val remoteHost = ConnectionId.fromString(connectionId).getOrElse { + throw new IllegalStateException(s"connectionId has unexpected format: $connectionId") + }.remoteHost + inflightResponses.remove(connectionId).foreach(_.request.updateRequestMetrics()) + // the channel has been closed by the selector but the quotas still need to be updated + connectionQuotas.dec(InetAddress.getByName(remoteHost)) + } + } + /** * Queue up a new connection for reading */ @@ -504,10 +530,10 @@ private[kafka] class Processor(val id: Int, * Register any new connections that have been queued up */ private def configureNewConnections() { - while(!newConnections.isEmpty) { + while (!newConnections.isEmpty) { val channel = newConnections.poll() try { - debug("Processor " + id + " listening to new connection from " + channel.socket.getRemoteSocketAddress) + debug(s"Processor $id listening to new connection from ${channel.socket.getRemoteSocketAddress}") val localHost = channel.socket().getLocalAddress.getHostAddress val localPort = channel.socket().getLocalPort val remoteHost = channel.socket().getInetAddress.getHostAddress @@ -515,12 +541,12 @@ private[kafka] class Processor(val id: Int, val connectionId = ConnectionId(localHost, localPort, remoteHost, remotePort).toString selector.register(connectionId, channel) } catch { - // We explicitly catch all non fatal exceptions and close the socket to avoid socket leak. The other - // throwables will be caught in processor and logged as uncaught exception. + // We explicitly catch all non fatal exceptions and close the socket to avoid a socket leak. The other + // throwables will be caught in processor and logged as uncaught exceptions. case NonFatal(e) => - // need to close the channel here to avoid socket leak. + // need to close the channel here to avoid a socket leak. close(channel) - error("Processor " + id + " closed connection from " + channel.getRemoteAddress, e) + error(s"Processor $id closed connection from ${channel.getRemoteAddress}", e) } } } @@ -535,6 +561,9 @@ private[kafka] class Processor(val id: Int, selector.close() } + /* For test usage */ + private[network] def channel(connectionId: String): Option[KafkaChannel] = + Option(selector.channel(connectionId)) /** * Wakeup the thread for selection. diff --git a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala index 5d28894e32bff..81e5232e85cdb 100644 --- a/core/src/test/scala/unit/kafka/network/SocketServerTest.scala +++ b/core/src/test/scala/unit/kafka/network/SocketServerTest.scala @@ -39,7 +39,7 @@ import org.junit.Assert._ import org.junit._ import org.scalatest.junit.JUnitSuite -import scala.collection.Map +import scala.collection.mutable.ArrayBuffer class SocketServerTest extends JUnitSuite { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 0) @@ -55,6 +55,7 @@ class SocketServerTest extends JUnitSuite { val metrics = new Metrics val server = new SocketServer(config, metrics, new SystemTime) server.startup() + val sockets = new ArrayBuffer[Socket] def sendRequest(socket: Socket, request: Array[Byte], id: Option[Short] = None) { val outgoing = new DataOutputStream(socket.getOutputStream) @@ -79,7 +80,12 @@ class SocketServerTest extends JUnitSuite { /* A simple request handler that just echos back the response */ def processRequest(channel: RequestChannel) { - val request = channel.receiveRequest + val request = channel.receiveRequest(2000) + assertNotNull("receiveRequest timed out", request) + processRequest(channel, request) + } + + def processRequest(channel: RequestChannel, request: RequestChannel.Request) { val byteBuffer = ByteBuffer.allocate(request.header.sizeOf + request.body.sizeOf) request.header.writeTo(byteBuffer) request.body.writeTo(byteBuffer) @@ -89,13 +95,18 @@ class SocketServerTest extends JUnitSuite { channel.sendResponse(new RequestChannel.Response(request.processor, request, send)) } - def connect(s: SocketServer = server, protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT) = - new Socket("localhost", server.boundPort(protocol)) + def connect(s: SocketServer = server, protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT) = { + val socket = new Socket("localhost", s.boundPort(protocol)) + sockets += socket + socket + } @After - def cleanup() { + def tearDown() { metrics.close() server.shutdown() + sockets.foreach(_.close()) + sockets.clear() } private def producerRequestBytes: Array[Byte] = { @@ -183,7 +194,7 @@ class SocketServerTest extends JUnitSuite { @Test def testMaxConnectionsPerIp() { - // make the maximum allowable number of connections and then leak them + // make the maximum allowable number of connections val conns = (0 until server.config.maxConnectionsPerIp).map(_ => connect()) // now try one more (should fail) val conn = connect() @@ -201,27 +212,30 @@ class SocketServerTest extends JUnitSuite { sendRequest(conn2, serializedBytes) val request = server.requestChannel.receiveRequest(2000) assertNotNull(request) - conn2.close() - conns.tail.foreach(_.close()) } @Test - def testMaxConnectionsPerIPOverrides() { - val overrideNum = 6 - val overrides = Map("localhost" -> overrideNum) + def testMaxConnectionsPerIpOverrides() { + val overrideNum = server.config.maxConnectionsPerIp + 1 val overrideProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 0) + overrideProps.put(KafkaConfig.MaxConnectionsPerIpOverridesProp, s"localhost:$overrideNum") val serverMetrics = new Metrics() - val overrideServer: SocketServer = new SocketServer(KafkaConfig.fromProps(overrideProps), serverMetrics, new SystemTime()) + val overrideServer = new SocketServer(KafkaConfig.fromProps(overrideProps), serverMetrics, new SystemTime()) try { overrideServer.startup() - // make the maximum allowable number of connections and then leak them - val conns = ((0 until overrideNum).map(i => connect(overrideServer))) + // make the maximum allowable number of connections + val conns = (0 until overrideNum).map(_ => connect(overrideServer)) + + // it should succeed + val serializedBytes = producerRequestBytes + sendRequest(conns.last, serializedBytes) + val request = overrideServer.requestChannel.receiveRequest(2000) + assertNotNull(request) + // now try one more (should fail) val conn = connect(overrideServer) conn.setSoTimeout(3000) assertEquals(-1, conn.getInputStream.read()) - conn.close() - conns.foreach(_.close()) } finally { overrideServer.shutdown() serverMetrics.close() @@ -229,16 +243,16 @@ class SocketServerTest extends JUnitSuite { } @Test - def testSslSocketServer(): Unit = { + def testSslSocketServer() { val trustStoreFile = File.createTempFile("truststore", ".jks") val overrideProps = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, interBrokerSecurityProtocol = Some(SecurityProtocol.SSL), trustStoreFile = Some(trustStoreFile)) overrideProps.put(KafkaConfig.ListenersProp, "SSL://localhost:0") val serverMetrics = new Metrics - val overrideServer: SocketServer = new SocketServer(KafkaConfig.fromProps(overrideProps), serverMetrics, new SystemTime) - overrideServer.startup() + val overrideServer = new SocketServer(KafkaConfig.fromProps(overrideProps), serverMetrics, new SystemTime) try { + overrideServer.startup() val sslContext = SSLContext.getInstance("TLSv1.2") sslContext.init(null, Array(TestUtils.trustAllCerts), new java.security.SecureRandom()) val socketFactory = sslContext.getSocketFactory @@ -271,12 +285,95 @@ class SocketServerTest extends JUnitSuite { } @Test - def testSessionPrincipal(): Unit = { + def testSessionPrincipal() { val socket = connect() val bytes = new Array[Byte](40) sendRequest(socket, bytes, Some(0)) - assertEquals(KafkaPrincipal.ANONYMOUS, server.requestChannel.receiveRequest().session.principal) - socket.close() + assertEquals(KafkaPrincipal.ANONYMOUS, server.requestChannel.receiveRequest(2000).session.principal) + } + + /* Test that we update request metrics if the client closes the connection while the broker response is in flight. */ + @Test + def testClientDisconnectionUpdatesRequestMetrics() { + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 0) + val serverMetrics = new Metrics + var conn: Socket = null + val overrideServer = new SocketServer(KafkaConfig.fromProps(props), serverMetrics, new SystemTime) { + override def newProcessor(id: Int, connectionQuotas: ConnectionQuotas, protocol: SecurityProtocol): Processor = { + new Processor(id, time, config.socketRequestMaxBytes, requestChannel, connectionQuotas, + config.connectionsMaxIdleMs, protocol, config.values, metrics) { + override protected[network] def sendResponse(response: RequestChannel.Response) { + conn.close() + super.sendResponse(response) + } + } + } + } + try { + overrideServer.startup() + conn = connect(overrideServer) + val serializedBytes = producerRequestBytes + sendRequest(conn, serializedBytes) + + val channel = overrideServer.requestChannel + val request = channel.receiveRequest(2000) + + val requestMetrics = RequestMetrics.metricsMap(ApiKeys.forId(request.requestId).name) + def totalTimeHistCount(): Long = requestMetrics.totalTimeHist.count + val expectedTotalTimeCount = totalTimeHistCount() + 1 + + // send a large buffer to ensure that the broker detects the client disconnection while writing to the socket channel. + // On Mac OS X, the initial write seems to always succeed and it is able to write up to 102400 bytes on the initial + // write. If the buffer is smaller than this, the write is considered complete and the disconnection is not + // detected. If the buffer is larger than 102400 bytes, a second write is attempted and it fails with an + // IOException. + val send = new NetworkSend(request.connectionId, ByteBuffer.allocate(550000)) + channel.sendResponse(new RequestChannel.Response(request.processor, request, send)) + TestUtils.waitUntilTrue(() => totalTimeHistCount() == expectedTotalTimeCount, + s"request metrics not updated, expected: $expectedTotalTimeCount, actual: ${totalTimeHistCount()}") + + } finally { + overrideServer.shutdown() + serverMetrics.close() + } + } + + /* + * Test that we update request metrics if the channel has been removed from the selector when the broker calls + * `selector.send` (selector closes old connections, for example). + */ + @Test + def testBrokerSendAfterChannelClosedUpdatesRequestMetrics() { + val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect, port = 0) + props.setProperty(KafkaConfig.ConnectionsMaxIdleMsProp, "100") + val serverMetrics = new Metrics + var conn: Socket = null + val overrideServer = new SocketServer(KafkaConfig.fromProps(props), serverMetrics, new SystemTime) + try { + overrideServer.startup() + conn = connect(overrideServer) + val serializedBytes = producerRequestBytes + sendRequest(conn, serializedBytes) + val channel = overrideServer.requestChannel + val request = channel.receiveRequest(2000) + + TestUtils.waitUntilTrue(() => overrideServer.processor(request.processor).channel(request.connectionId).isEmpty, + s"Idle connection `${request.connectionId}` was not closed by selector") + + val requestMetrics = RequestMetrics.metricsMap(ApiKeys.forId(request.requestId).name) + def totalTimeHistCount(): Long = requestMetrics.totalTimeHist.count + val expectedTotalTimeCount = totalTimeHistCount() + 1 + + processRequest(channel, request) + + TestUtils.waitUntilTrue(() => totalTimeHistCount() == expectedTotalTimeCount, + s"request metrics not updated, expected: $expectedTotalTimeCount, actual: ${totalTimeHistCount()}") + + } finally { + overrideServer.shutdown() + serverMetrics.close() + } + } } From 8218a4a153d9138f29486e74cf77c88d1786997a Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Tue, 5 Apr 2016 15:17:46 -0700 Subject: [PATCH 121/206] KAFKA-3508: Fix transient SimpleACLAuthorizerTest failures Allows the the maximum retires when writing to zookeeper to be overridden in tests and sets the value to Int.MaxValue to avoid transient failure. Author: Grant Henke Reviewers: Ismael Juma , Ewen Cheslack-Postava Closes #1156 from granthenke/transient-acl-test --- .../scala/kafka/security/auth/SimpleAclAuthorizer.scala | 2 +- .../unit/kafka/security/auth/SimpleAclAuthorizerTest.scala | 6 +++++- 2 files changed, 6 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala index 1a06af2ad8fd2..18fff453e0858 100644 --- a/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala +++ b/core/src/main/scala/kafka/security/auth/SimpleAclAuthorizer.scala @@ -79,7 +79,7 @@ class SimpleAclAuthorizer extends Authorizer with Logging { // The maximum number of times we should try to update the resource acls in zookeeper before failing; // This should never occur, but is a safeguard just in case. - private val maxUpdateRetries = 10 + protected[auth] var maxUpdateRetries = 10 private val retryBackoffMs = 100 private val retryBackoffJitterMs = 50 diff --git a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala index bdadb15f430d9..7fcc33dd8c1b1 100644 --- a/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala +++ b/core/src/test/scala/unit/kafka/security/auth/SimpleAclAuthorizerTest.scala @@ -44,6 +44,10 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness { override def setUp() { super.setUp() + // Increase maxUpdateRetries to avoid transient failures + simpleAclAuthorizer.maxUpdateRetries = Int.MaxValue + simpleAclAuthorizer2.maxUpdateRetries = Int.MaxValue + val props = TestUtils.createBrokerConfig(0, zkConnect) props.put(SimpleAclAuthorizer.SuperUsersProp, superUsers) @@ -307,7 +311,7 @@ class SimpleAclAuthorizerTest extends ZooKeeperTestHarness { def testHighConcurrencyModificationOfResourceAcls() { val commonResource = new Resource(Topic, "test") - val acls = (0 to 100).map { i => + val acls = (0 to 50).map { i => val useri = new KafkaPrincipal(KafkaPrincipal.USER_TYPE, i.toString) new Acl(useri, Allow, WildCardHost, Read) } From 99d232922734e115f8e3585d91ccfb0ef067385d Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Tue, 5 Apr 2016 15:56:09 -0700 Subject: [PATCH 122/206] KAFKA-3477: extended KStream/KTable API to specify custom partitioner for sinks Author: mjsax Reviewers: Guozhang Wang Closes #1180 from mjsax/kafka-3477-streamPartitioner-DSL --- .../apache/kafka/streams/KafkaStreams.java | 2 +- .../apache/kafka/streams/kstream/KStream.java | 88 ++++++++++++++++--- .../apache/kafka/streams/kstream/KTable.java | 88 ++++++++++++++++--- .../kstream/internals/KStreamImpl.java | 39 +++++--- .../streams/kstream/internals/KTableImpl.java | 30 ++++++- 5 files changed, 206 insertions(+), 41 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index 20958e4a4730c..e8fda103fdc61 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -49,7 +49,7 @@ * A {@link KafkaStreams} instance can co-ordinate with any other instances with the same application ID (whether in this same process, on other processes * on this machine, or on remote machines) as a single (possibly distributed) stream processing client. These instances will divide up the work * based on the assignment of the input topic partitions so that all partitions are being - * consumed. If instances are added or failed, all instances will rebelance the partition assignment among themselves + * consumed. If instances are added or failed, all instances will rebalance the partition assignment among themselves * to balance processing load. *

      * Internally the {@link KafkaStreams} instance contains a normal {@link org.apache.kafka.clients.producer.KafkaProducer KafkaProducer} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index 2313b8bf749c9..e4933cb08e71f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.StreamPartitioner; /** * KStream is an abstraction of a record stream of key-value pairs. @@ -92,44 +93,105 @@ public interface KStream { /** * Materialize this stream to a topic, also creates a new instance of {@link KStream} from the topic - * using default serializers and deserializers. + * using default serializers and deserializers and producer's {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner}. * This is equivalent to calling {@link #to(String)} and {@link org.apache.kafka.streams.kstream.KStreamBuilder#stream(String...)}. * * @param topic the topic name */ KStream through(String topic); + /** + * Materialize this stream to a topic, also creates a new instance of {@link KStream} from the topic + * using default serializers and deserializers and a customizable {@link StreamPartitioner} to determine the distribution of records to partitions. + * This is equivalent to calling {@link #to(StreamPartitioner, String)} and {@link org.apache.kafka.streams.kstream.KStreamBuilder#stream(String...)}. + * + * @param partitioner the function used to determine how records are distributed among partitions of the topic, + * if not specified producer's {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} will be used + * @param topic the topic name + */ + KStream through(StreamPartitioner partitioner, String topic); + /** * Materialize this stream to a topic, also creates a new instance of {@link KStream} from the topic. + * If {@code keySerde} provides a {@link org.apache.kafka.streams.kstream.internals.WindowedSerializer} + * for the key {@link org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner} is used + * — otherwise producer's {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} is used. * This is equivalent to calling {@link #to(Serde, Serde, String)} and * {@link org.apache.kafka.streams.kstream.KStreamBuilder#stream(Serde, Serde, String...)}. * - * @param keySerde key serde used to send key-value pairs, - * if not specified the default key serde defined in the configuration will be used - * @param valSerde value serde used to send key-value pairs, - * if not specified the default value serde defined in the configuration will be used - * @param topic the topic name + * @param keySerde key serde used to send key-value pairs, + * if not specified the default key serde defined in the configuration will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default value serde defined in the configuration will be used + * @param topic the topic name */ KStream through(Serde keySerde, Serde valSerde, String topic); /** - * Materialize this stream to a topic using default serializers specified in the config. + * Materialize this stream to a topic, also creates a new instance of {@link KStream} from the topic + * using a customizable {@link StreamPartitioner} to determine the distribution of records to partitions. + * This is equivalent to calling {@link #to(Serde, Serde, StreamPartitioner, String)} and + * {@link org.apache.kafka.streams.kstream.KStreamBuilder#stream(Serde, Serde, String...)}. + * + * @param keySerde key serde used to send key-value pairs, + * if not specified the default key serde defined in the configuration will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default value serde defined in the configuration will be used + * @param partitioner the function used to determine how records are distributed among partitions of the topic, + * if not specified and {@code keySerde} provides a {@link org.apache.kafka.streams.kstream.internals.WindowedSerializer} for the key + * {@link org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner} will be used + * — otherwise {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} will be used + * @param topic the topic name + */ + KStream through(Serde keySerde, Serde valSerde, StreamPartitioner partitioner, String topic); + + /** + * Materialize this stream to a topic using default serializers specified in the config + * and producer's {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner}. * * @param topic the topic name */ void to(String topic); /** - * Materialize this stream to a topic. + * Materialize this stream to a topic using default serializers specified in the config and a customizable + * {@link StreamPartitioner} to determine the distribution of records to partitions. * - * @param keySerde key serde used to send key-value pairs, - * if not specified the default serde defined in the configs will be used - * @param valSerde value serde used to send key-value pairs, - * if not specified the default serde defined in the configs will be used - * @param topic the topic name + * @param partitioner the function used to determine how records are distributed among partitions of the topic, + * if not specified producer's {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} will be used + * @param topic the topic name + */ + void to(StreamPartitioner partitioner, String topic); + + /** + * Materialize this stream to a topic. If {@code keySerde} provides a + * {@link org.apache.kafka.streams.kstream.internals.WindowedSerializer} for the key + * {@link org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner} is used + * — otherwise producer's {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} is used. + * + * @param keySerde key serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * @param topic the topic name */ void to(Serde keySerde, Serde valSerde, String topic); + /** + * Materialize this stream to a topic using a customizable {@link StreamPartitioner} to determine the distribution of records to partitions. + * + * @param keySerde key serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * @param partitioner the function used to determine how records are distributed among partitions of the topic, + * if not specified and {@code keySerde} provides a {@link org.apache.kafka.streams.kstream.internals.WindowedSerializer} for the key + * {@link org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner} will be used + * — otherwise {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} will be used + * @param topic the topic name + */ + void to(Serde keySerde, Serde valSerde, StreamPartitioner partitioner, String topic); + /** * Create a new {@link KStream} instance by applying a {@link org.apache.kafka.streams.kstream.Transformer} to all elements in this stream, one element at a time. * diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index 30ea882563791..581ee282360eb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.annotation.InterfaceStability; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.processor.StreamPartitioner; /** * KTable is an abstraction of a changelog stream from a primary-keyed table. @@ -54,44 +55,105 @@ public interface KTable { /** * Materialize this stream to a topic, also creates a new instance of {@link KTable} from the topic - * using default serializers and deserializers. + * using default serializers and deserializers and producer's {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner}. * This is equivalent to calling {@link #to(String)} and {@link org.apache.kafka.streams.kstream.KStreamBuilder#table(String)}. * * @param topic the topic name */ KTable through(String topic); + /** + * Materialize this stream to a topic, also creates a new instance of {@link KTable} from the topic using default serializers + * and deserializers and a customizable {@link StreamPartitioner} to determine the distribution of records to partitions. + * This is equivalent to calling {@link #to(String)} and {@link org.apache.kafka.streams.kstream.KStreamBuilder#table(String)}. + * + * @param partitioner the function used to determine how records are distributed among partitions of the topic, + * if not specified producer's {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} will be used + * @param topic the topic name + */ + KTable through(StreamPartitioner partitioner, String topic); + /** * Materialize this stream to a topic, also creates a new instance of {@link KTable} from the topic. + * If {@code keySerde} provides a {@link org.apache.kafka.streams.kstream.internals.WindowedSerializer} + * for the key {@link org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner} is used + * — otherwise producer's {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} is used. * This is equivalent to calling {@link #to(Serde, Serde, String)} and * {@link org.apache.kafka.streams.kstream.KStreamBuilder#table(Serde, Serde, String)}. * - * @param keySerde key serde used to send key-value pairs, - * if not specified the default key serde defined in the configuration will be used - * @param valSerde value serde used to send key-value pairs, - * if not specified the default value serde defined in the configuration will be used - * @param topic the topic name + * @param keySerde key serde used to send key-value pairs, + * if not specified the default key serde defined in the configuration will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default value serde defined in the configuration will be used + * @param topic the topic name */ KTable through(Serde keySerde, Serde valSerde, String topic); /** - * Materialize this stream to a topic using default serializers specified in the config. + * Materialize this stream to a topic, also creates a new instance of {@link KTable} from the topic + * using a customizable {@link StreamPartitioner} to determine the distribution of records to partitions. + * This is equivalent to calling {@link #to(Serde, Serde, StreamPartitioner, String)} and + * {@link org.apache.kafka.streams.kstream.KStreamBuilder#table(Serde, Serde, String)}. + * + * @param keySerde key serde used to send key-value pairs, + * if not specified the default key serde defined in the configuration will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default value serde defined in the configuration will be used + * @param partitioner the function used to determine how records are distributed among partitions of the topic, + * if not specified and {@code keySerde} provides a {@link org.apache.kafka.streams.kstream.internals.WindowedSerializer} for the key + * {@link org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner} will be used + * — otherwise {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} will be used + * @param topic the topic name + */ + KTable through(Serde keySerde, Serde valSerde, StreamPartitioner partitioner, String topic); + + /** + * Materialize this stream to a topic using default serializers specified in the config + * and producer's {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner}. * * @param topic the topic name */ void to(String topic); /** - * Materialize this stream to a topic. + * Materialize this stream to a topic using default serializers specified in the config + * and a customizable {@link StreamPartitioner} to determine the distribution of records to partitions. + * + * @param partitioner the function used to determine how records are distributed among partitions of the topic, + * if not specified producer's {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} will be used + * @param topic the topic name + */ + void to(StreamPartitioner partitioner, String topic); + + /** + * Materialize this stream to a topic. If {@code keySerde} provides a + * {@link org.apache.kafka.streams.kstream.internals.WindowedSerializer} for the key + * {@link org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner} is used + * — otherwise producer's {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} is used. * - * @param keySerde key serde used to send key-value pairs, - * if not specified the default serde defined in the configs will be used - * @param valSerde value serde used to send key-value pairs, - * if not specified the default serde defined in the configs will be used - * @param topic the topic name + * @param keySerde key serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * @param topic the topic name */ void to(Serde keySerde, Serde valSerde, String topic); + /** + * Materialize this stream to a topic using a customizable {@link StreamPartitioner} to determine the distribution of records to partitions. + * + * @param keySerde key serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * @param partitioner the function used to determine how records are distributed among partitions of the topic, + * if not specified and {@code keySerde} provides a {@link org.apache.kafka.streams.kstream.internals.WindowedSerializer} for the key + * {@link org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner} will be used + * — otherwise {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} will be used + * @param topic the topic name + */ + void to(Serde keySerde, Serde valSerde, StreamPartitioner partitioner, String topic); + /** * Convert this stream to a new instance of {@link KStream}. */ diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 5889e078c3306..0fb3984245f10 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -194,37 +194,56 @@ public static KStream merge(KStreamBuilder topology, KStream[ } @Override - public KStream through(Serde keySerde, Serde valSerde, String topic) { - to(keySerde, valSerde, topic); + public KStream through(Serde keySerde, Serde valSerde, StreamPartitioner partitioner, String topic) { + to(keySerde, valSerde, partitioner, topic); return topology.stream(keySerde, valSerde, topic); } + @Override + public KStream through(Serde keySerde, Serde valSerde, String topic) { + return through(keySerde, valSerde, null, topic); + } + + @Override + public KStream through(StreamPartitioner partitioner, String topic) { + return through(null, null, partitioner, topic); + } + @Override public KStream through(String topic) { - return through(null, null, topic); + return through(null, null, null, topic); } @Override public void to(String topic) { - to(null, null, topic); + to(null, null, null, topic); + } + + @Override + public void to(StreamPartitioner partitioner, String topic) { + to(null, null, partitioner, topic); } - @SuppressWarnings("unchecked") @Override public void to(Serde keySerde, Serde valSerde, String topic) { + to(keySerde, valSerde, null, topic); + } + + @SuppressWarnings("unchecked") + @Override + public void to(Serde keySerde, Serde valSerde, StreamPartitioner partitioner, String topic) { String name = topology.newName(SINK_NAME); - StreamPartitioner streamPartitioner = null; Serializer keySerializer = keySerde == null ? null : keySerde.serializer(); Serializer valSerializer = keySerde == null ? null : valSerde.serializer(); - - if (keySerializer != null && keySerializer instanceof WindowedSerializer) { + + if (partitioner == null && keySerializer != null && keySerializer instanceof WindowedSerializer) { WindowedSerializer windowedSerializer = (WindowedSerializer) keySerializer; - streamPartitioner = (StreamPartitioner) new WindowedStreamPartitioner(windowedSerializer); + partitioner = (StreamPartitioner) new WindowedStreamPartitioner(windowedSerializer); } - topology.addSink(name, topic, keySerializer, valSerializer, streamPartitioner, this.name); + topology.addSink(name, topic, keySerializer, valSerializer, partitioner, this.name); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index fd464a08d7e92..156f2db4b18c3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -32,6 +32,7 @@ import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.processor.StateStoreSupplier; +import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.state.Stores; import java.util.Collections; @@ -133,25 +134,46 @@ public KTable mapValues(ValueMapper mapper) { @Override public KTable through(Serde keySerde, Serde valSerde, + StreamPartitioner partitioner, String topic) { - to(keySerde, valSerde, topic); + to(keySerde, valSerde, partitioner, topic); return topology.table(keySerde, valSerde, topic); } + @Override + public KTable through(Serde keySerde, Serde valSerde, String topic) { + return through(keySerde, valSerde, null, topic); + } + + @Override + public KTable through(StreamPartitioner partitioner, String topic) { + return through(null, null, partitioner, topic); + } + @Override public KTable through(String topic) { - return through(null, null, topic); + return through(null, null, null, topic); } @Override public void to(String topic) { - to(null, null, topic); + to(null, null, null, topic); + } + + @Override + public void to(StreamPartitioner partitioner, String topic) { + to(null, null, partitioner, topic); } @Override public void to(Serde keySerde, Serde valSerde, String topic) { - this.toStream().to(keySerde, valSerde, topic); + this.toStream().to(keySerde, valSerde, null, topic); + } + + @Override + public void to(Serde keySerde, Serde valSerde, StreamPartitioner partitioner, String topic) { + this.toStream().to(keySerde, valSerde, partitioner, topic); } @Override From 8dbd688b1617968329087317fa6bde8b8df0392e Mon Sep 17 00:00:00 2001 From: Eno Thereska Date: Thu, 7 Apr 2016 10:20:17 -0700 Subject: [PATCH 123/206] KAFKA-3497: Streams ProcessorContext should support forward() based on child name Author: Eno Thereska Reviewers: Yuto Kawamura, Michael G. Noll, Guozhang Wang Closes #1194 from enothereska/kafka-3497-forward --- .../streams/processor/ProcessorContext.java | 9 +++ .../internals/ProcessorContextImpl.java | 5 ++ .../internals/StandbyContextImpl.java | 5 ++ .../processor/internals/StreamTask.java | 16 ++++++ .../internals/ProcessorTopologyTest.java | 56 +++++++++++++++++++ .../apache/kafka/test/KStreamTestDriver.java | 16 ++++++ .../kafka/test/MockProcessorContext.java | 6 ++ 7 files changed, 113 insertions(+) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java index 434996e2fd563..8bac3e38363da 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java @@ -106,9 +106,18 @@ public interface ProcessorContext { * Forwards a key/value pair to one of the downstream processors designated by childIndex * @param key key * @param value value + * @param childIndex index in list of children of this node */ void forward(K key, V value, int childIndex); + /** + * Forwards a key/value pair to one of the downstream processors designated by the downstream processor name + * @param key key + * @param value value + * @param childName name of downstream processor + */ + void forward(K key, V value, String childName); + /** * Requests a commit */ diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index 888b89e2deade..5bda856cc00c0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -167,6 +167,11 @@ public void forward(K key, V value, int childIndex) { task.forward(key, value, childIndex); } + @Override + public void forward(K key, V value, String childName) { + task.forward(key, value, childName); + } + @Override public void commit() { task.needCommit(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java index 3ad06e2b1d8eb..d5a9683db805c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java @@ -141,6 +141,11 @@ public void forward(K key, V value, int childIndex) { throw new UnsupportedOperationException(); } + @Override + public void forward(K key, V value, String childName) { + throw new UnsupportedOperationException(); + } + @Override public void commit() { throw new UnsupportedOperationException(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 61aeced9624a8..a48498010ba09 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -342,4 +342,20 @@ public void forward(K key, V value, int childIndex) { } } + @SuppressWarnings("unchecked") + public void forward(K key, V value, String childName) { + ProcessorNode thisNode = currNode; + for (ProcessorNode childNode : (List>) thisNode.children()) { + if (childNode.name().equals(childName)) { + currNode = childNode; + try { + childNode.process(key, value); + } finally { + currNode = thisNode; + } + break; + } + } + } + } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java index ef081761eb4e4..1095fcf513ff2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorTopologyTest.java @@ -156,6 +156,28 @@ public void testDrivingMultiplexingTopology() { assertNextOutputRecord(OUTPUT_TOPIC_2, "key5", "value5(2)"); } + @Test + public void testDrivingMultiplexByNameTopology() { + driver = new ProcessorTopologyTestDriver(config, createMultiplexByNameTopology()); + driver.process(INPUT_TOPIC, "key1", "value1", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key1", "value1(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key1", "value1(2)"); + + driver.process(INPUT_TOPIC, "key2", "value2", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key2", "value2(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key2", "value2(2)"); + + driver.process(INPUT_TOPIC, "key3", "value3", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key4", "value4", STRING_SERIALIZER, STRING_SERIALIZER); + driver.process(INPUT_TOPIC, "key5", "value5", STRING_SERIALIZER, STRING_SERIALIZER); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key3", "value3(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key4", "value4(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_1, "key5", "value5(1)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key3", "value3(2)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key4", "value4(2)"); + assertNextOutputRecord(OUTPUT_TOPIC_2, "key5", "value5(2)"); + } + @Test public void testDrivingStatefulTopology() { String storeName = "entries"; @@ -215,6 +237,13 @@ protected TopologyBuilder createMultiplexingTopology() { .addSink("sink2", OUTPUT_TOPIC_2, "processor"); } + protected TopologyBuilder createMultiplexByNameTopology() { + return new TopologyBuilder().addSource("source", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC) + .addProcessor("processor", define(new MultiplexByNameProcessor(2)), "source") + .addSink("sink0", OUTPUT_TOPIC_1, "processor") + .addSink("sink1", OUTPUT_TOPIC_2, "processor"); + } + protected TopologyBuilder createStatefulTopology(String storeName) { return new TopologyBuilder().addSource("source", STRING_DESERIALIZER, STRING_DESERIALIZER, INPUT_TOPIC) .addProcessor("processor", define(new StatefulProcessor(storeName)), "source") @@ -267,6 +296,33 @@ public void punctuate(long streamTime) { } } + /** + * A processor that forwards slightly-modified messages to each named child. + * Note: the children are assumed to be named "sink{child number}", e.g., sink1, or sink2, etc. + */ + protected static class MultiplexByNameProcessor extends AbstractProcessor { + + private final int numChildren; + + public MultiplexByNameProcessor(int numChildren) { + this.numChildren = numChildren; + } + + @Override + public void process(String key, String value) { + for (int i = 0; i != numChildren; ++i) { + context().forward(key, value + "(" + (i + 1) + ")", "sink" + i); + } + } + + @Override + public void punctuate(long streamTime) { + for (int i = 0; i != numChildren; ++i) { + context().forward(Long.toString(streamTime), "punctuate(" + (i + 1) + ")", "sink" + i); + } + } + } + /** * A processor that stores each key-value pair in an in-memory key-value store registered with the context. When * {@link #punctuate(long)} is called, it outputs the total number of entries in the store. diff --git a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java index 0c56c26e17655..5cfee6b581062 100644 --- a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java @@ -120,6 +120,22 @@ public void forward(K key, V value, int childIndex) { } } + @SuppressWarnings("unchecked") + public void forward(K key, V value, String childName) { + ProcessorNode thisNode = currNode; + for (ProcessorNode childNode : (List>) thisNode.children()) { + if (childNode.name().equals(childName)) { + currNode = childNode; + try { + childNode.process(key, value); + } finally { + currNode = thisNode; + } + break; + } + } + } + public Map allStateStores() { return context.allStateStores(); } diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java index e57e1c7fabc43..d3b808177a70c 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -158,6 +158,12 @@ public void forward(K key, V value, int childIndex) { driver.forward(key, value, childIndex); } + @Override + @SuppressWarnings("unchecked") + public void forward(K key, V value, String childName) { + driver.forward(key, value, childName); + } + @Override public void commit() { throw new UnsupportedOperationException("commit() not supported."); From 9c34df1511a769b272893b75ec1ed90d38cc9576 Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Thu, 7 Apr 2016 15:48:50 -0700 Subject: [PATCH 124/206] KAFKA-3488; Avoid failing of unsent requests in consumer where possible Fail unsent requests only when returning from KafkaConsumer.poll(). Author: Rajini Sivaram Reviewers: Jason Gustafson , Ismael Juma , Ewen Cheslack-Postava Closes #1183 from rajinisivaram/KAFKA-3488 --- .../kafka/clients/consumer/KafkaConsumer.java | 3 +- .../internals/AbstractCoordinator.java | 2 + .../internals/ConsumerCoordinator.java | 3 - .../internals/ConsumerNetworkClient.java | 58 ++++++++++------- .../internals/SendFailedException.java | 27 -------- ...GroupCoordinatorNotAvailableException.java | 1 + .../internals/ConsumerCoordinatorTest.java | 2 +- .../internals/ConsumerNetworkClientTest.java | 63 ++++++++++++++++++- .../consumer/internals/FetcherTest.java | 2 +- .../distributed/WorkerGroupMember.java | 3 +- .../distributed/WorkerCoordinatorTest.java | 2 +- .../main/scala/kafka/admin/AdminClient.scala | 23 +++---- 12 files changed, 118 insertions(+), 71 deletions(-) delete mode 100644 clients/src/main/java/org/apache/kafka/clients/consumer/internals/SendFailedException.java diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index c457c83692a59..557643102eeb0 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -584,7 +584,8 @@ private KafkaConsumer(ConsumerConfig config, config.getInt(ConsumerConfig.SEND_BUFFER_CONFIG), config.getInt(ConsumerConfig.RECEIVE_BUFFER_CONFIG), config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG), time); - this.client = new ConsumerNetworkClient(netClient, metadata, time, retryBackoffMs); + this.client = new ConsumerNetworkClient(netClient, metadata, time, retryBackoffMs, + config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG)); OffsetResetStrategy offsetResetStrategy = OffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase()); this.subscriptions = new SubscriptionState(offsetResetStrategy); List assignors = config.getConfiguredInstances( diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index c79d8e7d30393..1e6757e9298cd 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -17,6 +17,7 @@ import org.apache.kafka.common.Node; import org.apache.kafka.common.errors.DisconnectException; import org.apache.kafka.common.errors.GroupAuthorizationException; +import org.apache.kafka.common.errors.GroupCoordinatorNotAvailableException; import org.apache.kafka.common.errors.IllegalGenerationException; import org.apache.kafka.common.errors.RebalanceInProgressException; import org.apache.kafka.common.errors.UnknownMemberIdException; @@ -521,6 +522,7 @@ public boolean coordinatorUnknown() { protected void coordinatorDead() { if (this.coordinator != null) { log.info("Marking the coordinator {} dead for group {}", this.coordinator, groupId); + client.failUnsentRequests(this.coordinator, GroupCoordinatorNotAvailableException.INSTANCE); this.coordinator = null; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index a3649878bfe23..86b60d0ed7553 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -422,9 +422,6 @@ public void run(final long now) { public void onComplete(Map offsets, Exception exception) { if (exception == null) { reschedule(now + interval); - } else if (exception instanceof SendFailedException) { - log.debug("Failed to send automatic offset commit for group {}", groupId); - reschedule(now); } else { log.warn("Auto offset commit failed for group {}: {}", groupId, exception.getMessage()); reschedule(now + interval); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java index b70994d87fc44..4119954eaf596 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java @@ -19,6 +19,7 @@ import org.apache.kafka.clients.RequestCompletionHandler; import org.apache.kafka.common.Node; import org.apache.kafka.common.errors.DisconnectException; +import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.protocol.ApiKeys; import org.apache.kafka.common.requests.AbstractRequest; @@ -40,12 +41,6 @@ /** * Higher level consumer access to the network layer with basic support for futures and * task scheduling. NOT thread-safe! - * - * TODO: The current implementation is simplistic in that it provides a facility for queueing requests - * prior to delivery, but it makes no effort to retry requests which cannot be sent at the time - * {@link #poll(long)} is called. This makes the behavior of the queue predictable and easy to - * understand, but there are opportunities to provide timeout or retry capabilities in the future. - * How we do this may depend on KAFKA-2120, so for now, we retain the simplistic behavior. */ public class ConsumerNetworkClient implements Closeable { private static final Logger log = LoggerFactory.getLogger(ConsumerNetworkClient.class); @@ -57,17 +52,20 @@ public class ConsumerNetworkClient implements Closeable { private final Metadata metadata; private final Time time; private final long retryBackoffMs; + private final long unsentExpiryMs; // wakeup enabled flag need to be volatile since it is allowed to be accessed concurrently volatile private boolean wakeupsEnabled = true; public ConsumerNetworkClient(KafkaClient client, Metadata metadata, Time time, - long retryBackoffMs) { + long retryBackoffMs, + long requestTimeoutMs) { this.client = client; this.metadata = metadata; this.time = time; this.retryBackoffMs = retryBackoffMs; + this.unsentExpiryMs = requestTimeoutMs; } /** @@ -227,8 +225,8 @@ private void poll(long timeout, long now, boolean executeDelayedTasks) { // cleared or a connect finished in the poll trySend(now); - // fail all requests that couldn't be sent - failUnsentRequests(); + // fail requests that couldn't be sent if they have expired + failExpiredRequests(now); } /** @@ -274,29 +272,48 @@ private void checkDisconnects(long now) { Map.Entry> requestEntry = iterator.next(); Node node = requestEntry.getKey(); if (client.connectionFailed(node)) { + // Remove entry before invoking request callback to avoid callbacks handling + // coordinator failures traversing the unsent list again. + iterator.remove(); for (ClientRequest request : requestEntry.getValue()) { RequestFutureCompletionHandler handler = (RequestFutureCompletionHandler) request.callback(); handler.onComplete(new ClientResponse(request, now, true, null)); } - iterator.remove(); } } } - private void failUnsentRequests() { - // clear all unsent requests and fail their corresponding futures - for (Map.Entry> requestEntry: unsent.entrySet()) { - Iterator iterator = requestEntry.getValue().iterator(); - while (iterator.hasNext()) { - ClientRequest request = iterator.next(); - RequestFutureCompletionHandler handler = - (RequestFutureCompletionHandler) request.callback(); - handler.raise(SendFailedException.INSTANCE); + private void failExpiredRequests(long now) { + // clear all expired unsent requests and fail their corresponding futures + Iterator>> iterator = unsent.entrySet().iterator(); + while (iterator.hasNext()) { + Map.Entry> requestEntry = iterator.next(); + Iterator requestIterator = requestEntry.getValue().iterator(); + while (requestIterator.hasNext()) { + ClientRequest request = requestIterator.next(); + if (request.createdTimeMs() < now - unsentExpiryMs) { + RequestFutureCompletionHandler handler = + (RequestFutureCompletionHandler) request.callback(); + handler.raise(new TimeoutException("Failed to send request after " + unsentExpiryMs + " ms.")); + requestIterator.remove(); + } else + break; + } + if (requestEntry.getValue().isEmpty()) iterator.remove(); + } + } + + protected void failUnsentRequests(Node node, RuntimeException e) { + // clear unsent requests to node and fail their corresponding futures + List unsentRequests = unsent.remove(node); + if (unsentRequests != null) { + for (ClientRequest request : unsentRequests) { + RequestFutureCompletionHandler handler = (RequestFutureCompletionHandler) request.callback(); + handler.raise(e); } } - unsent.clear(); } private boolean trySend(long now) { @@ -320,7 +337,6 @@ private boolean trySend(long now) { private void clientPoll(long timeout, long now) { client.poll(timeout, now); if (wakeupsEnabled && wakeup.get()) { - failUnsentRequests(); wakeup.set(false); throw new WakeupException(); } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SendFailedException.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SendFailedException.java deleted file mode 100644 index 3312a2c5e6b74..0000000000000 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/SendFailedException.java +++ /dev/null @@ -1,27 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE - * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file - * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the - * License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on - * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the - * specific language governing permissions and limitations under the License. - */ -package org.apache.kafka.clients.consumer.internals; - -import org.apache.kafka.common.errors.RetriableException; - -/** - * Exception used in {@link ConsumerNetworkClient} to indicate the failure - * to transmit a request to the networking layer. This could be either because - * the client is still connecting to the given host or its send buffer is full. - */ -public class SendFailedException extends RetriableException { - public static final SendFailedException INSTANCE = new SendFailedException(); - - private static final long serialVersionUID = 1L; - -} diff --git a/clients/src/main/java/org/apache/kafka/common/errors/GroupCoordinatorNotAvailableException.java b/clients/src/main/java/org/apache/kafka/common/errors/GroupCoordinatorNotAvailableException.java index c0949e3bc9e9f..554b8856d57da 100644 --- a/clients/src/main/java/org/apache/kafka/common/errors/GroupCoordinatorNotAvailableException.java +++ b/clients/src/main/java/org/apache/kafka/common/errors/GroupCoordinatorNotAvailableException.java @@ -18,6 +18,7 @@ * not yet been created. */ public class GroupCoordinatorNotAvailableException extends RetriableException { + public static final GroupCoordinatorNotAvailableException INSTANCE = new GroupCoordinatorNotAvailableException(); private static final long serialVersionUID = 1L; diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index 623e5ef93c1e0..2189c304fb92e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -103,7 +103,7 @@ public void setup() { this.subscriptions = new SubscriptionState(OffsetResetStrategy.EARLIEST); this.metadata = new Metadata(0, Long.MAX_VALUE); this.metadata.update(cluster, time.milliseconds()); - this.consumerClient = new ConsumerNetworkClient(client, metadata, time, 100); + this.consumerClient = new ConsumerNetworkClient(client, metadata, time, 100, 1000); this.metrics = new Metrics(time); this.rebalanceListener = new MockRebalanceListener(); this.defaultOffsetCommitCallback = new MockCommitCallback(); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java index 1692010a4ffa4..f0f2a973294e1 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClientTest.java @@ -27,6 +27,8 @@ import org.apache.kafka.test.TestUtils; import org.junit.Test; +import java.util.concurrent.atomic.AtomicBoolean; + import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; @@ -40,7 +42,7 @@ public class ConsumerNetworkClientTest { private Cluster cluster = TestUtils.singletonCluster(topicName, 1); private Node node = cluster.nodes().get(0); private Metadata metadata = new Metadata(0, Long.MAX_VALUE); - private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100); + private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100, 1000); @Test public void send() { @@ -104,6 +106,65 @@ public void wakeup() { assertTrue(future.isDone()); } + @Test + public void sendExpiry() throws InterruptedException { + long unsentExpiryMs = 10; + final AtomicBoolean isReady = new AtomicBoolean(); + final AtomicBoolean disconnected = new AtomicBoolean(); + client = new MockClient(time) { + @Override + public boolean ready(Node node, long now) { + if (isReady.get()) + return super.ready(node, now); + else + return false; + } + @Override + public boolean connectionFailed(Node node) { + return disconnected.get(); + } + }; + // Queue first send, sleep long enough for this to expire and then queue second send + consumerClient = new ConsumerNetworkClient(client, metadata, time, 100, unsentExpiryMs); + RequestFuture future1 = consumerClient.send(node, ApiKeys.METADATA, heartbeatRequest()); + assertEquals(1, consumerClient.pendingRequestCount()); + assertEquals(1, consumerClient.pendingRequestCount(node)); + assertFalse(future1.isDone()); + + time.sleep(unsentExpiryMs + 1); + RequestFuture future2 = consumerClient.send(node, ApiKeys.METADATA, heartbeatRequest()); + assertEquals(2, consumerClient.pendingRequestCount()); + assertEquals(2, consumerClient.pendingRequestCount(node)); + assertFalse(future2.isDone()); + + // First send should have expired and second send still pending + consumerClient.poll(0); + assertTrue(future1.isDone()); + assertFalse(future1.succeeded()); + assertEquals(1, consumerClient.pendingRequestCount()); + assertEquals(1, consumerClient.pendingRequestCount(node)); + assertFalse(future2.isDone()); + + // Enable send, the un-expired send should succeed on poll + isReady.set(true); + client.prepareResponse(heartbeatResponse(Errors.NONE.code())); + consumerClient.poll(future2); + ClientResponse clientResponse = future2.value(); + HeartbeatResponse response = new HeartbeatResponse(clientResponse.responseBody()); + assertEquals(Errors.NONE.code(), response.errorCode()); + + // Disable ready flag to delay send and queue another send. Disconnection should remove pending send + isReady.set(false); + RequestFuture future3 = consumerClient.send(node, ApiKeys.METADATA, heartbeatRequest()); + assertEquals(1, consumerClient.pendingRequestCount()); + assertEquals(1, consumerClient.pendingRequestCount(node)); + disconnected.set(true); + consumerClient.poll(0); + assertTrue(future3.isDone()); + assertFalse(future3.succeeded()); + assertEquals(0, consumerClient.pendingRequestCount()); + assertEquals(0, consumerClient.pendingRequestCount(node)); + } private HeartbeatRequest heartbeatRequest() { return new HeartbeatRequest("group", 1, "memberId"); diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 58c38412588ae..9002e819b4f6e 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -86,7 +86,7 @@ public class FetcherTest { private SubscriptionState subscriptionsNoAutoReset = new SubscriptionState(OffsetResetStrategy.NONE); private Metrics metrics = new Metrics(time); private static final double EPSILON = 0.0001; - private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100); + private ConsumerNetworkClient consumerClient = new ConsumerNetworkClient(client, metadata, time, 100, 1000); private MemoryRecords records = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE); private MemoryRecords nextRecords = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), CompressionType.NONE); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java index 7294ed4608ecc..57028ef1386c0 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java @@ -99,7 +99,8 @@ public WorkerGroupMember(DistributedConfig config, config.getInt(CommonClientConfigs.SEND_BUFFER_CONFIG), config.getInt(CommonClientConfigs.RECEIVE_BUFFER_CONFIG), config.getInt(CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG), time); - this.client = new ConsumerNetworkClient(netClient, metadata, time, retryBackoffMs); + this.client = new ConsumerNetworkClient(netClient, metadata, time, retryBackoffMs, + config.getInt(CommonClientConfigs.REQUEST_TIMEOUT_MS_CONFIG)); this.coordinator = new WorkerCoordinator(this.client, config.getString(DistributedConfig.GROUP_ID_CONFIG), config.getInt(DistributedConfig.SESSION_TIMEOUT_MS_CONFIG), diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java index abb62b932db6c..bf33cb32c0ffe 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java @@ -88,7 +88,7 @@ public void setup() { this.client = new MockClient(time); this.metadata = new Metadata(0, Long.MAX_VALUE); this.metadata.update(cluster, time.milliseconds()); - this.consumerClient = new ConsumerNetworkClient(client, metadata, time, 100); + this.consumerClient = new ConsumerNetworkClient(client, metadata, time, 100, 1000); this.metrics = new Metrics(time); this.rebalanceListener = new MockRebalanceListener(); this.configStorage = PowerMock.createMock(KafkaConfigStorage.class); diff --git a/core/src/main/scala/kafka/admin/AdminClient.scala b/core/src/main/scala/kafka/admin/AdminClient.scala index b8573153e5b8d..ef76ffc40ee1e 100644 --- a/core/src/main/scala/kafka/admin/AdminClient.scala +++ b/core/src/main/scala/kafka/admin/AdminClient.scala @@ -20,7 +20,7 @@ import kafka.common.KafkaException import kafka.coordinator.{GroupOverview, GroupSummary, MemberSummary} import kafka.utils.Logging import org.apache.kafka.clients._ -import org.apache.kafka.clients.consumer.internals.{ConsumerNetworkClient, ConsumerProtocol, RequestFuture, SendFailedException} +import org.apache.kafka.clients.consumer.internals.{ConsumerNetworkClient, ConsumerProtocol, RequestFuture} import org.apache.kafka.common.config.ConfigDef.{Importance, Type} import org.apache.kafka.common.config.{AbstractConfig, ConfigDef} import org.apache.kafka.common.errors.DisconnectException @@ -43,21 +43,15 @@ class AdminClient(val time: Time, private def send(target: Node, api: ApiKeys, request: AbstractRequest): Struct = { - var now = time.milliseconds() - val deadline = now + requestTimeoutMs var future: RequestFuture[ClientResponse] = null - do { - future = client.send(target, api, request) - client.poll(future) + future = client.send(target, api, request) + client.poll(future) - if (future.succeeded()) - return future.value().responseBody() - - now = time.milliseconds() - } while (now < deadline && future.exception().isInstanceOf[SendFailedException]) - - throw future.exception() + if (future.succeeded()) + return future.value().responseBody() + else + throw future.exception() } private def sendAnyNode(api: ApiKeys, request: AbstractRequest): Struct = { @@ -244,7 +238,8 @@ object AdminClient { networkClient, metadata, time, - DefaultRetryBackoffMs) + DefaultRetryBackoffMs, + DefaultRequestTimeoutMs) new AdminClient( time, From 8b9b07e5d6aed2552d1cdfba27b0211af39c691f Mon Sep 17 00:00:00 2001 From: Anna Povzner Date: Thu, 7 Apr 2016 18:17:40 -0700 Subject: [PATCH 125/206] MINOR: ensure original use of prop_file in verifiable producer This PR: https://github.com/apache/kafka/pull/958 fixed the use of prop_file in the situation when we have multiple producers (before, every producer will add to the config). However, it assumes that self.prop_file is initially "". This is correct for all existing tests, but it precludes us from extending verifiable producer and adding more properties to the producer config (same as console consumer). This is a small PR to change the behavior to the original, but also make verifiable producer use prop_file method to be consistent with console consumer. Also few more fixes to verifiable producer came up during the review: -- fixed each_produced_at_least() method -- more straightforward use of compression types granders please review. Author: Anna Povzner Reviewers: Geoff Anderson , Ewen Cheslack-Postava Closes #1192 from apovzner/fix_verifiable_producer --- .../kafkatest/services/verifiable_producer.py | 34 ++++++++++--------- 1 file changed, 18 insertions(+), 16 deletions(-) diff --git a/tests/kafkatest/services/verifiable_producer.py b/tests/kafkatest/services/verifiable_producer.py index f2ea421de33f9..0096a34875e50 100644 --- a/tests/kafkatest/services/verifiable_producer.py +++ b/tests/kafkatest/services/verifiable_producer.py @@ -52,12 +52,8 @@ def __init__(self, context, num_nodes, kafka, topic, max_messages=-1, throughput num_nodes = 1 * is_int_with_prefix recommended if num_nodes > 1, because otherwise each producer will produce exactly same messages, and validation may miss missing messages. - :param compression_types: If None, all producers will not use compression; or a list of one or - more compression types (including "none"). Each producer will pick a compression type - from the list in round-robin fashion. Example: compression_types = ["none", "snappy"] and - num_nodes = 3, then producer 1 and 2 will not use compression, and producer 3 will use - compression type = snappy. If in this example, num_nodes is 1, then first (and only) - producer will not use compression. + :param compression_types: If None, all producers will not use compression; or a list of + compression types, one per producer (could be "none"). """ super(VerifiableProducer, self).__init__(context, num_nodes) @@ -67,30 +63,36 @@ def __init__(self, context, num_nodes, kafka, topic, max_messages=-1, throughput self.throughput = throughput self.message_validator = message_validator self.compression_types = compression_types + if self.compression_types is not None: + assert len(self.compression_types) == num_nodes, "Specify one compression type per node" + + self.security_config = self.kafka.security_config.client_config() for node in self.nodes: node.version = version self.acked_values = [] self.not_acked_values = [] self.produced_count = {} - self.prop_file = "" + + def prop_file(self, node): + idx = self.idx(node) + prop_file = str(self.security_config) + if self.compression_types is not None: + compression_index = idx - 1 + self.logger.info("VerifiableProducer (index = %d) will use compression type = %s", idx, + self.compression_types[compression_index]) + prop_file += "\ncompression.type=%s\n" % self.compression_types[compression_index] + return prop_file def _worker(self, idx, node): node.account.ssh("mkdir -p %s" % VerifiableProducer.PERSISTENT_ROOT, allow_fail=False) # Create and upload log properties - self.security_config = self.kafka.security_config.client_config(self.prop_file) - producer_prop_file = str(self.security_config) log_config = self.render('tools_log4j.properties', log_file=VerifiableProducer.LOG_FILE) node.account.create_file(VerifiableProducer.LOG4J_CONFIG, log_config) # Create and upload config file - if self.compression_types is not None: - compression_index = (idx - 1) % len(self.compression_types) - self.logger.info("VerifiableProducer (index = %d) will use compression type = %s", idx, - self.compression_types[compression_index]) - producer_prop_file += "\ncompression.type=%s\n" % self.compression_types[compression_index] - + producer_prop_file = self.prop_file(node) self.logger.info("verifiable_producer.properties:") self.logger.info(producer_prop_file) node.account.create_file(VerifiableProducer.CONFIG_FILE, producer_prop_file) @@ -197,7 +199,7 @@ def num_not_acked(self): def each_produced_at_least(self, count): with self.lock: - for idx in range(1, self.num_nodes): + for idx in range(1, self.num_nodes + 1): if self.produced_count.get(idx) is None or self.produced_count[idx] < count: return False return True From 3a58407e2e0aee0bb2c5e343fbe98c7f28fc6f3e Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 8 Apr 2016 08:59:50 -0700 Subject: [PATCH 126/206] KAFKA-3505: Fix punctuate generated record metadata Author: Guozhang Wang Reviewers: Anna Povzner , Ewen Cheslack-Postava Closes #1190 from guozhangwang/K3505 --- .../clients/consumer/ConsumerRecordTest.java | 2 +- .../streams/processor/ProcessorContext.java | 17 +++- .../processor/internals/PartitionGroup.java | 17 ++-- .../internals/ProcessorContextImpl.java | 11 ++- .../processor/internals/ProcessorNode.java | 2 +- .../internals/PunctuationSchedule.java | 2 +- .../internals/StandbyContextImpl.java | 18 ++-- .../processor/internals/StreamTask.java | 13 ++- .../processor/internals/StreamThread.java | 7 +- .../internals/KStreamKStreamJoinTest.java | 72 ++++++++-------- .../internals/KStreamKStreamLeftJoinTest.java | 36 ++++---- .../internals/KStreamKTableLeftJoinTest.java | 14 ++-- .../internals/KStreamWindowAggregateTest.java | 24 +++--- .../kstream/internals/KTableFilterTest.java | 36 ++++---- .../internals/KTableKTableJoinTest.java | 42 +++++----- .../internals/KTableKTableLeftJoinTest.java | 42 +++++----- .../internals/KTableKTableOuterJoinTest.java | 48 +++++------ .../internals/KTableMapValuesTest.java | 16 ++-- .../kstream/internals/KTableSourceTest.java | 16 ++-- .../internals/PartitionGroupTest.java | 82 +++++++++++++++++-- .../processor/internals/StreamTaskTest.java | 79 +++++++++++++++++- .../kafka/test/MockProcessorContext.java | 2 +- .../apache/kafka/test/MockProcessorNode.java | 49 +++++++++++ .../kafka/test/MockProcessorSupplier.java | 38 +++++++-- 24 files changed, 458 insertions(+), 227 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/test/MockProcessorNode.java diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordTest.java index d1d3b24afeff8..41e9160f1226b 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/ConsumerRecordTest.java @@ -31,7 +31,7 @@ public void testOldConstructor() { String key = "key"; String value = "value"; - ConsumerRecord record = new ConsumerRecord(topic, partition, offset, key, value); + ConsumerRecord record = new ConsumerRecord<>(topic, partition, offset, key, value); assertEquals(topic, record.topic()); assertEquals(partition, record.partition()); assertEquals(offset, record.offset()); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java index 8bac3e38363da..815b5b4e26f59 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorContext.java @@ -124,30 +124,39 @@ public interface ProcessorContext { void commit(); /** - * Returns the topic name of the current input record + * Returns the topic name of the current input record; could be null if it is not + * available (for example, if this method is invoked from the punctuate call) * * @return the topic name */ String topic(); /** - * Returns the partition id of the current input record + * Returns the partition id of the current input record; could be -1 if it is not + * available (for example, if this method is invoked from the punctuate call) * * @return the partition id */ int partition(); /** - * Returns the offset of the current input record + * Returns the offset of the current input record; could be -1 if it is not + * available (for example, if this method is invoked from the punctuate call) * * @return the offset */ long offset(); /** - * Returns the timestamp of the current input record. The timestamp is extracted from + * Returns the current timestamp. + * + * If it is triggered while processing a record streamed from the source processor, timestamp is defined as the timestamp of the current input record; the timestamp is extracted from * {@link org.apache.kafka.clients.consumer.ConsumerRecord ConsumerRecord} by {@link TimestampExtractor}. * + * If it is triggered while processing a record generated not from the source processor (for example, + * if this method is invoked from the punctuate call), timestamp is defined as the current + * task's stream time, which is defined as the smallest among all its input stream partition timestamps. + * * @return the timestamp */ long timestamp(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java index 3d8f792c7c8e8..ec89d47e08b7f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java @@ -135,17 +135,14 @@ public Set partitions() { * partition timestamp among all its partitions */ public long timestamp() { - if (queuesByTime.isEmpty()) { - // if there is no data in all partitions, return the smallest of their last known times - long timestamp = Long.MAX_VALUE; - for (RecordQueue queue : partitionQueues.values()) { - if (timestamp > queue.timestamp()) - timestamp = queue.timestamp(); - } - return timestamp; - } else { - return queuesByTime.peek().timestamp(); + // we should always return the smallest timestamp of all partitions + // to avoid group partition time goes backward + long timestamp = Long.MAX_VALUE; + for (RecordQueue queue : partitionQueues.values()) { + if (timestamp > queue.timestamp()) + timestamp = queue.timestamp(); } + return timestamp; } public int numBuffered(TopicPartition partition) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index 5bda856cc00c0..1c398ac69b341 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -30,6 +30,8 @@ public class ProcessorContextImpl implements ProcessorContext, RecordCollector.Supplier { + public static final String NONEXIST_TOPIC = "__null_topic__"; + private final TaskId id; private final StreamTask task; private final StreamsMetrics metrics; @@ -118,7 +120,7 @@ public StateStore getStateStore(String name) { if (node == null) throw new TopologyBuilderException("Accessing from an unknown node"); - // TODO: restore this once we fix the ValueGetter initialiation issue + // TODO: restore this once we fix the ValueGetter initialization issue //if (!node.stateStores.contains(name)) // throw new TopologyBuilderException("Processor " + node.name() + " has no access to StateStore " + name); @@ -130,7 +132,12 @@ public String topic() { if (task.record() == null) throw new IllegalStateException("This should not happen as topic() should only be called while a record is processed"); - return task.record().topic(); + String topic = task.record().topic(); + + if (topic.equals(NONEXIST_TOPIC)) + return null; + else + return topic; } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java index 6db83a11c997a..50e3a0b27b3db 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorNode.java @@ -48,7 +48,7 @@ public final String name() { return name; } - public final Processor processor() { + public final Processor processor() { return processor; } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java index dc9a50d32c249..758cfb001e035 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java @@ -22,7 +22,7 @@ public class PunctuationSchedule extends Stamped { final long interval; public PunctuationSchedule(ProcessorNode node, long interval) { - this(node, System.currentTimeMillis(), interval); + this(node, 0, interval); } public PunctuationSchedule(ProcessorNode node, long time, long interval) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java index d5a9683db805c..ea008b827e554 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java @@ -108,37 +108,37 @@ public void register(StateStore store, boolean loggingEnabled, StateRestoreCallb @Override public StateStore getStateStore(String name) { - throw new UnsupportedOperationException(); + throw new UnsupportedOperationException("getStateStore() not supported."); } @Override public String topic() { - throw new UnsupportedOperationException(); + throw new UnsupportedOperationException("topic() not supported."); } @Override public int partition() { - throw new UnsupportedOperationException(); + throw new UnsupportedOperationException("partition() not supported."); } @Override public long offset() { - throw new UnsupportedOperationException(); + throw new UnsupportedOperationException("offset() not supported."); } @Override public long timestamp() { - throw new UnsupportedOperationException(); + throw new UnsupportedOperationException("timestamp() not supported."); } @Override public void forward(K key, V value) { - throw new UnsupportedOperationException(); + throw new UnsupportedOperationException("forward() not supported."); } @Override public void forward(K key, V value, int childIndex) { - throw new UnsupportedOperationException(); + throw new UnsupportedOperationException("forward() not supported."); } @Override @@ -148,11 +148,11 @@ public void forward(K key, V value, String childName) { @Override public void commit() { - throw new UnsupportedOperationException(); + throw new UnsupportedOperationException("commit() not supported."); } @Override public void schedule(long interval) { - throw new UnsupportedOperationException(); + throw new UnsupportedOperationException("schedule() not supported."); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index a48498010ba09..53d0a8df2848f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -43,6 +43,8 @@ public class StreamTask extends AbstractTask implements Punctuator { private static final Logger log = LoggerFactory.getLogger(StreamTask.class); + private static final ConsumerRecord DUMMY_RECORD = new ConsumerRecord<>(ProcessorContextImpl.NONEXIST_TOPIC, -1, -1L, null, null); + private final int maxBufferedSize; private final PartitionGroup partitionGroup; @@ -202,11 +204,11 @@ public boolean requiresPoll() { /** * Possibly trigger registered punctuation functions if - * current time has reached the defined stamp - * - * @param timestamp + * current partition group timestamp has reached the defined stamp */ - public boolean maybePunctuate(long timestamp) { + public boolean maybePunctuate() { + long timestamp = partitionGroup.timestamp(); + return punctuationQueue.mayPunctuate(timestamp, this); } @@ -216,10 +218,13 @@ public void punctuate(ProcessorNode node, long timestamp) { throw new IllegalStateException("Current node is not null"); currNode = node; + currRecord = new StampedRecord(DUMMY_RECORD, timestamp); + try { node.processor().punctuate(timestamp); } finally { currNode = null; + currRecord = null; } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index c2a8e06e9b119..38dc356a672c9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -341,8 +341,9 @@ private void runLoop() { totalNumBuffered = 0; + // try to process one fetch record from each task via the topology, and also trigger punctuate + // functions if necessary, which may result in more records going through the topology in this loop if (!activeTasks.isEmpty()) { - // try to process one record from each task for (StreamTask task : activeTasks.values()) { long startProcess = time.milliseconds(); @@ -431,7 +432,9 @@ private void maybePunctuate(StreamTask task) { try { long now = time.milliseconds(); - if (task.maybePunctuate(now)) + // check whether we should punctuate based on the task's partition group timestamp; + // which are essentially based on record timestamp. + if (task.maybePunctuate()) sensors.punctuateTimeSensor.record(time.milliseconds() - now); } catch (KafkaException e) { diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java index d24ab15461c44..19a941177e00d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java @@ -90,7 +90,7 @@ public void testJoin() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - processor.checkAndClearResult(); + processor.checkAndClearProcessResult(); // push two items to the other stream. this should produce two items. // w1 = { 0:X0, 1:X1 } @@ -102,7 +102,7 @@ public void testJoin() throws Exception { driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+Y0", "1:X1+Y1"); + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); // push all four items to the primary stream. this should produce two items. // w1 = { 0:X0, 1:X1 } @@ -114,7 +114,7 @@ public void testJoin() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+Y0", "1:X1+Y1"); + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); // push all items to the other stream. this should produce six items. // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 } @@ -126,7 +126,7 @@ public void testJoin() throws Exception { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+YY0", "0:X0+YY0", "1:X1+YY1", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + processor.checkAndClearProcessResult("0:X0+YY0", "0:X0+YY0", "1:X1+YY1", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); // push all four items to the primary stream. this should produce six items. // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 } @@ -138,7 +138,7 @@ public void testJoin() throws Exception { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:XX0+Y0", "0:XX0+YY0", "1:XX1+Y1", "1:XX1+YY1", "2:XX2+YY2", "3:XX3+YY3"); + processor.checkAndClearProcessResult("0:XX0+Y0", "0:XX0+YY0", "1:XX1+Y1", "1:XX1+YY1", "2:XX2+YY2", "3:XX3+YY3"); // push two items to the other stream. this should produce six item. // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3, 0:XX0, 1:XX1, 2:XX2, 3:XX3 } @@ -150,7 +150,7 @@ public void testJoin() throws Exception { driver.process(topic2, expectedKeys[i], "YYY" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+YYY0", "0:X0+YYY0", "0:XX0+YYY0", "1:X1+YYY1", "1:X1+YYY1", "1:XX1+YYY1"); + processor.checkAndClearProcessResult("0:X0+YYY0", "0:X0+YYY0", "0:XX0+YYY0", "1:X1+YYY1", "1:X1+YYY1", "1:XX1+YYY1"); } finally { Utils.delete(baseDir); @@ -195,7 +195,7 @@ public void testOuterJoin() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+null", "1:X1+null"); + processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); // push two items to the other stream. this should produce two items. // w1 = { 0:X0, 1:X1 } @@ -207,7 +207,7 @@ public void testOuterJoin() throws Exception { driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+Y0", "1:X1+Y1"); + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); // push all four items to the primary stream. this should produce four items. // w1 = { 0:X0, 1:X1 } @@ -219,7 +219,7 @@ public void testOuterJoin() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null"); + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null"); // push all items to the other stream. this should produce six items. // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 } @@ -231,7 +231,7 @@ public void testOuterJoin() throws Exception { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+YY0", "0:X0+YY0", "1:X1+YY1", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + processor.checkAndClearProcessResult("0:X0+YY0", "0:X0+YY0", "1:X1+YY1", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); // push all four items to the primary stream. this should produce six items. // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 } @@ -243,7 +243,7 @@ public void testOuterJoin() throws Exception { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:XX0+Y0", "0:XX0+YY0", "1:XX1+Y1", "1:XX1+YY1", "2:XX2+YY2", "3:XX3+YY3"); + processor.checkAndClearProcessResult("0:XX0+Y0", "0:XX0+YY0", "1:XX1+Y1", "1:XX1+YY1", "2:XX2+YY2", "3:XX3+YY3"); // push two items to the other stream. this should produce six item. // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3, 0:XX0, 1:XX1, 2:XX2, 3:XX3 } @@ -255,7 +255,7 @@ public void testOuterJoin() throws Exception { driver.process(topic2, expectedKeys[i], "YYY" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+YYY0", "0:X0+YYY0", "0:XX0+YYY0", "1:X1+YYY1", "1:X1+YYY1", "1:XX1+YYY1"); + processor.checkAndClearProcessResult("0:X0+YYY0", "0:X0+YYY0", "0:XX0+YYY0", "1:X1+YYY1", "1:X1+YYY1", "1:XX1+YYY1"); } finally { Utils.delete(baseDir); @@ -302,7 +302,7 @@ public void testWindowing() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - processor.checkAndClearResult(); + processor.checkAndClearProcessResult(); // push two items to the other stream. this should produce two items. // w1 = { 0:X0, 1:X1 } @@ -314,7 +314,7 @@ public void testWindowing() throws Exception { driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+Y0", "1:X1+Y1"); + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); // clear logically time = 1000L; @@ -323,7 +323,7 @@ public void testWindowing() throws Exception { driver.setTime(time + i); driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - processor.checkAndClearResult(); + processor.checkAndClearProcessResult(); // gradually expires items in w1 // w1 = { 0:X0, 1:X1, 2:X2, 3:X3 } @@ -335,35 +335,35 @@ public void testWindowing() throws Exception { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - processor.checkAndClearResult("1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + processor.checkAndClearProcessResult("1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - processor.checkAndClearResult("2:X2+YY2", "3:X3+YY3"); + processor.checkAndClearProcessResult("2:X2+YY2", "3:X3+YY3"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - processor.checkAndClearResult("3:X3+YY3"); + processor.checkAndClearProcessResult("3:X3+YY3"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - processor.checkAndClearResult(); + processor.checkAndClearProcessResult(); // go back to the time before expiration @@ -373,35 +373,35 @@ public void testWindowing() throws Exception { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - processor.checkAndClearResult(); + processor.checkAndClearProcessResult(); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+YY0"); + processor.checkAndClearProcessResult("0:X0+YY0"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+YY0", "1:X1+YY1"); + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2"); + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); // clear (logically) time = 2000L; @@ -411,7 +411,7 @@ public void testWindowing() throws Exception { driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } - processor.checkAndClearResult(); + processor.checkAndClearProcessResult(); // gradually expires items in w2 // w2 = { 0:Y0, 1:Y1, 2:Y2, 3:Y3 } @@ -422,35 +422,35 @@ public void testWindowing() throws Exception { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); + processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); + processor.checkAndClearProcessResult("1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("2:XX2+Y2", "3:XX3+Y3"); + processor.checkAndClearProcessResult("2:XX2+Y2", "3:XX3+Y3"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("3:XX3+Y3"); + processor.checkAndClearProcessResult("3:XX3+Y3"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult(); + processor.checkAndClearProcessResult(); // go back to the time before expiration @@ -460,35 +460,35 @@ public void testWindowing() throws Exception { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult(); + processor.checkAndClearProcessResult(); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:XX0+Y0"); + processor.checkAndClearProcessResult("0:XX0+Y0"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:XX0+Y0", "1:XX1+Y1"); + processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2"); + processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); + processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); } finally { Utils.delete(baseDir); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java index 166e8ba038571..65226d338f28c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java @@ -88,7 +88,7 @@ public void testLeftJoin() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+null", "1:X1+null"); + processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); // push two items to the other stream. this should produce two items. // w {} @@ -98,7 +98,7 @@ public void testLeftJoin() throws Exception { driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } - processor.checkAndClearResult(); + processor.checkAndClearProcessResult(); // push all four items to the primary stream. this should produce four items. // w = { 0:Y0, 1:Y1 } @@ -108,7 +108,7 @@ public void testLeftJoin() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null"); + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null"); // push all items to the other stream. this should produce no items. // w = { 0:Y0, 1:Y1 } @@ -118,7 +118,7 @@ public void testLeftJoin() throws Exception { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - processor.checkAndClearResult(); + processor.checkAndClearProcessResult(); // push all four items to the primary stream. this should produce four items. // w = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 @@ -128,7 +128,7 @@ public void testLeftJoin() throws Exception { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:XX0+Y0", "0:XX0+YY0", "1:XX1+Y1", "1:XX1+YY1", "2:XX2+YY2", "3:XX3+YY3"); + processor.checkAndClearProcessResult("0:XX0+Y0", "0:XX0+YY0", "1:XX1+Y1", "1:XX1+YY1", "2:XX2+YY2", "3:XX3+YY3"); } finally { Utils.delete(baseDir); @@ -173,7 +173,7 @@ public void testWindowing() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+null", "1:X1+null"); + processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); // push two items to the other stream. this should produce no items. // w = {} @@ -183,7 +183,7 @@ public void testWindowing() throws Exception { driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } - processor.checkAndClearResult(); + processor.checkAndClearProcessResult(); // clear logically time = 1000L; @@ -196,7 +196,7 @@ public void testWindowing() throws Exception { driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } - processor.checkAndClearResult(); + processor.checkAndClearProcessResult(); // gradually expire items in window. // w = { 0:Y0, 1:Y1, 2:Y2, 3:Y3 } @@ -207,35 +207,35 @@ public void testWindowing() throws Exception { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); + processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:XX0+null", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); + processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:XX0+null", "1:XX1+null", "2:XX2+Y2", "3:XX3+Y3"); + processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+Y2", "3:XX3+Y3"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:XX0+null", "1:XX1+null", "2:XX2+null", "3:XX3+Y3"); + processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+null", "3:XX3+Y3"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:XX0+null", "1:XX1+null", "2:XX2+null", "3:XX3+null"); + processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+null", "3:XX3+null"); // go back to the time before expiration @@ -245,35 +245,35 @@ public void testWindowing() throws Exception { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:XX0+null", "1:XX1+null", "2:XX2+null", "3:XX3+null"); + processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+null", "3:XX3+null"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:XX0+Y0", "1:XX1+null", "2:XX2+null", "3:XX3+null"); + processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+null", "2:XX2+null", "3:XX3+null"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+null", "3:XX3+null"); + processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+null", "3:XX3+null"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+null"); + processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+null"); driver.setTime(++time); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); + processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); } finally { Utils.delete(baseDir); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java index 8e672a2311d30..3acb59a4a0c29 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java @@ -95,7 +95,7 @@ public void testJoin() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+null", "1:X1+null"); + processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); // push two items to the other stream. this should not produce any item. @@ -103,7 +103,7 @@ public void testJoin() throws Exception { driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } - processor.checkAndClearResult(); + processor.checkAndClearProcessResult(); // push all four items to the primary stream. this should produce four items. @@ -111,14 +111,14 @@ public void testJoin() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null"); + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null"); // push all items to the other stream. this should not produce any item for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - processor.checkAndClearResult(); + processor.checkAndClearProcessResult(); // push all four items to the primary stream. this should produce four items. @@ -126,7 +126,7 @@ public void testJoin() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); // push two items with null to the other stream as deletes. this should not produce any item. @@ -134,7 +134,7 @@ public void testJoin() throws Exception { driver.process(topic2, expectedKeys[i], null); } - processor.checkAndClearResult(); + processor.checkAndClearProcessResult(); // push all four items to the primary stream. this should produce four items. @@ -142,7 +142,7 @@ public void testJoin() throws Exception { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:XX0+null", "1:XX1+null", "2:XX2+YY2", "3:XX3+YY3"); + processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+YY2", "3:XX3+YY3"); } finally { Utils.delete(baseDir); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java index e19510f679307..3c7a1bdc60485 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java @@ -182,15 +182,15 @@ public String apply(String p1, String p2) { driver.setTime(4L); driver.process(topic1, "A", "1"); - proc1.checkAndClearResult( + proc1.checkAndClearProcessResult( "[A@0]:0+1", "[B@0]:0+2", "[C@0]:0+3", "[D@0]:0+4", "[A@0]:0+1+1" ); - proc2.checkAndClearResult(); - proc3.checkAndClearResult( + proc2.checkAndClearProcessResult(); + proc3.checkAndClearProcessResult( "[A@0]:null", "[B@0]:null", "[C@0]:null", @@ -209,15 +209,15 @@ public String apply(String p1, String p2) { driver.setTime(9L); driver.process(topic1, "C", "3"); - proc1.checkAndClearResult( + proc1.checkAndClearProcessResult( "[A@0]:0+1+1+1", "[A@5]:0+1", "[B@0]:0+2+2", "[B@5]:0+2", "[D@0]:0+4+4", "[D@5]:0+4", "[B@0]:0+2+2+2", "[B@5]:0+2+2", "[C@0]:0+3+3", "[C@5]:0+3" ); - proc2.checkAndClearResult(); - proc3.checkAndClearResult( + proc2.checkAndClearProcessResult(); + proc3.checkAndClearProcessResult( "[A@0]:null", "[A@5]:null", "[B@0]:null", "[B@5]:null", "[D@0]:null", "[D@5]:null", @@ -236,15 +236,15 @@ public String apply(String p1, String p2) { driver.setTime(4L); driver.process(topic2, "A", "a"); - proc1.checkAndClearResult(); - proc2.checkAndClearResult( + proc1.checkAndClearProcessResult(); + proc2.checkAndClearProcessResult( "[A@0]:0+a", "[B@0]:0+b", "[C@0]:0+c", "[D@0]:0+d", "[A@0]:0+a+a" ); - proc3.checkAndClearResult( + proc3.checkAndClearProcessResult( "[A@0]:0+1+1+1%0+a", "[B@0]:0+2+2+2%0+b", "[C@0]:0+3+3%0+c", @@ -262,15 +262,15 @@ public String apply(String p1, String p2) { driver.setTime(9L); driver.process(topic2, "C", "c"); - proc1.checkAndClearResult(); - proc2.checkAndClearResult( + proc1.checkAndClearProcessResult(); + proc2.checkAndClearProcessResult( "[A@0]:0+a+a+a", "[A@5]:0+a", "[B@0]:0+b+b", "[B@5]:0+b", "[D@0]:0+d+d", "[D@5]:0+d", "[B@0]:0+b+b+b", "[B@5]:0+b+b", "[C@0]:0+c+c", "[C@5]:0+c" ); - proc3.checkAndClearResult( + proc3.checkAndClearProcessResult( "[A@0]:0+1+1+1%0+a+a+a", "[A@5]:0+1%0+a", "[B@0]:0+2+2+2%0+b+b", "[B@5]:0+2+2%0+b", "[D@0]:0+4+4%0+d+d", "[D@5]:0+4%0+d", diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java index 78d274eb695bd..ee26058193c91 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java @@ -74,8 +74,8 @@ public boolean test(String key, Integer value) { driver.process(topic1, "A", null); driver.process(topic1, "B", null); - proc2.checkAndClearResult("A:null", "B:2", "C:null", "D:4", "A:null", "B:null"); - proc3.checkAndClearResult("A:1", "B:null", "C:3", "D:null", "A:null", "B:null"); + proc2.checkAndClearProcessResult("A:null", "B:2", "C:null", "D:4", "A:null", "B:null"); + proc3.checkAndClearProcessResult("A:1", "B:null", "C:3", "D:null", "A:null", "B:null"); } @Test @@ -193,25 +193,25 @@ public boolean test(String key, Integer value) { driver.process(topic1, "B", 1); driver.process(topic1, "C", 1); - proc1.checkAndClearResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)"); - proc2.checkAndClearResult("A:(null<-null)", "B:(null<-null)", "C:(null<-null)"); + proc1.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)"); + proc2.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)", "C:(null<-null)"); driver.process(topic1, "A", 2); driver.process(topic1, "B", 2); - proc1.checkAndClearResult("A:(2<-null)", "B:(2<-null)"); - proc2.checkAndClearResult("A:(2<-null)", "B:(2<-null)"); + proc1.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)"); + proc2.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)"); driver.process(topic1, "A", 3); - proc1.checkAndClearResult("A:(3<-null)"); - proc2.checkAndClearResult("A:(null<-null)"); + proc1.checkAndClearProcessResult("A:(3<-null)"); + proc2.checkAndClearProcessResult("A:(null<-null)"); driver.process(topic1, "A", null); driver.process(topic1, "B", null); - proc1.checkAndClearResult("A:(null<-null)", "B:(null<-null)"); - proc2.checkAndClearResult("A:(null<-null)", "B:(null<-null)"); + proc1.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)"); + proc2.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)"); } finally { Utils.delete(stateDir); @@ -250,25 +250,25 @@ public boolean test(String key, Integer value) { driver.process(topic1, "B", 1); driver.process(topic1, "C", 1); - proc1.checkAndClearResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)"); - proc2.checkAndClearResult("A:(null<-null)", "B:(null<-null)", "C:(null<-null)"); + proc1.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)"); + proc2.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)", "C:(null<-null)"); driver.process(topic1, "A", 2); driver.process(topic1, "B", 2); - proc1.checkAndClearResult("A:(2<-1)", "B:(2<-1)"); - proc2.checkAndClearResult("A:(2<-null)", "B:(2<-null)"); + proc1.checkAndClearProcessResult("A:(2<-1)", "B:(2<-1)"); + proc2.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)"); driver.process(topic1, "A", 3); - proc1.checkAndClearResult("A:(3<-2)"); - proc2.checkAndClearResult("A:(null<-2)"); + proc1.checkAndClearProcessResult("A:(3<-2)"); + proc2.checkAndClearProcessResult("A:(null<-2)"); driver.process(topic1, "A", null); driver.process(topic1, "B", null); - proc1.checkAndClearResult("A:(null<-3)", "B:(null<-2)"); - proc2.checkAndClearResult("A:(null<-null)", "B:(null<-2)"); + proc1.checkAndClearProcessResult("A:(null<-3)", "B:(null<-2)"); + proc2.checkAndClearProcessResult("A:(null<-null)", "B:(null<-2)"); } finally { Utils.delete(stateDir); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java index 5f30574cd5a10..f6ebbe1a0831b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java @@ -100,7 +100,7 @@ public void testJoin() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - processor.checkAndClearResult("0:null", "1:null"); + processor.checkAndClearProcessResult("0:null", "1:null"); checkJoinedValues(getter, kv(0, null), kv(1, null), kv(2, null), kv(3, null)); // push two items to the other stream. this should produce two items. @@ -109,7 +109,7 @@ public void testJoin() throws Exception { driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+Y0", "1:X1+Y1"); + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null)); // push all four items to the primary stream. this should produce four items. @@ -118,7 +118,7 @@ public void testJoin() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+Y0", "1:X1+Y1", "2:null", "3:null"); + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:null", "3:null"); checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null)); // push all items to the other stream. this should produce four items. @@ -126,7 +126,7 @@ public void testJoin() throws Exception { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); // push all four items to the primary stream. this should produce four items. @@ -135,7 +135,7 @@ public void testJoin() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); // push two items with null to the other stream as deletes. this should produce two item. @@ -144,7 +144,7 @@ public void testJoin() throws Exception { driver.process(topic2, expectedKeys[i], null); } - processor.checkAndClearResult("0:null", "1:null"); + processor.checkAndClearProcessResult("0:null", "1:null"); checkJoinedValues(getter, kv(0, null), kv(1, null), kv(2, "X2+YY2"), kv(3, "X3+YY3")); // push all four items to the primary stream. this should produce four items. @@ -153,7 +153,7 @@ public void testJoin() throws Exception { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:null", "1:null", "2:XX2+YY2", "3:XX3+YY3"); + processor.checkAndClearProcessResult("0:null", "1:null", "2:XX2+YY2", "3:XX3+YY3"); checkJoinedValues(getter, kv(0, null), kv(1, null), kv(2, "XX2+YY2"), kv(3, "XX3+YY3")); } finally { @@ -195,7 +195,7 @@ public void testNotSendingOldValues() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - proc.checkAndClearResult("0:(null<-null)", "1:(null<-null)"); + proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)"); // push two items to the other stream. this should produce two items. @@ -203,7 +203,7 @@ public void testNotSendingOldValues() throws Exception { driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)"); + proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)"); // push all four items to the primary stream. this should produce four items. @@ -211,14 +211,14 @@ public void testNotSendingOldValues() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(null<-null)", "3:(null<-null)"); + proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(null<-null)", "3:(null<-null)"); // push all items to the other stream. this should produce four items. for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); + proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); // push all four items to the primary stream. this should produce four items. @@ -226,7 +226,7 @@ public void testNotSendingOldValues() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); + proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); // push two items with null to the other stream as deletes. this should produce two item. @@ -234,7 +234,7 @@ public void testNotSendingOldValues() throws Exception { driver.process(topic2, expectedKeys[i], null); } - proc.checkAndClearResult("0:(null<-null)", "1:(null<-null)"); + proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)"); // push all four items to the primary stream. this should produce four items. @@ -242,7 +242,7 @@ public void testNotSendingOldValues() throws Exception { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - proc.checkAndClearResult("0:(null<-null)", "1:(null<-null)", "2:(XX2+YY2<-null)", "3:(XX3+YY3<-null)"); + proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)", "2:(XX2+YY2<-null)", "3:(XX3+YY3<-null)"); } finally { Utils.delete(baseDir); @@ -285,7 +285,7 @@ public void testSendingOldValues() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - proc.checkAndClearResult("0:(null<-null)", "1:(null<-null)"); + proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)"); // push two items to the other stream. this should produce two items. @@ -293,7 +293,7 @@ public void testSendingOldValues() throws Exception { driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)"); + proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)"); // push all four items to the primary stream. this should produce four items. @@ -301,14 +301,14 @@ public void testSendingOldValues() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(null<-null)", "3:(null<-null)"); + proc.checkAndClearProcessResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(null<-null)", "3:(null<-null)"); // push all items to the other stream. this should produce four items. for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+YY0<-X0+Y0)", "1:(X1+YY1<-X1+Y1)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); + proc.checkAndClearProcessResult("0:(X0+YY0<-X0+Y0)", "1:(X1+YY1<-X1+Y1)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); // push all four items to the primary stream. this should produce four items. @@ -316,7 +316,7 @@ public void testSendingOldValues() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+YY0<-X0+YY0)", "1:(X1+YY1<-X1+YY1)", "2:(X2+YY2<-X2+YY2)", "3:(X3+YY3<-X3+YY3)"); + proc.checkAndClearProcessResult("0:(X0+YY0<-X0+YY0)", "1:(X1+YY1<-X1+YY1)", "2:(X2+YY2<-X2+YY2)", "3:(X3+YY3<-X3+YY3)"); // push two items with null to the other stream as deletes. this should produce two item. @@ -324,7 +324,7 @@ public void testSendingOldValues() throws Exception { driver.process(topic2, expectedKeys[i], null); } - proc.checkAndClearResult("0:(null<-X0+YY0)", "1:(null<-X1+YY1)"); + proc.checkAndClearProcessResult("0:(null<-X0+YY0)", "1:(null<-X1+YY1)"); // push all four items to the primary stream. this should produce four items. @@ -332,7 +332,7 @@ public void testSendingOldValues() throws Exception { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - proc.checkAndClearResult("0:(null<-null)", "1:(null<-null)", "2:(XX2+YY2<-X2+YY2)", "3:(XX3+YY3<-X3+YY3)"); + proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)", "2:(XX2+YY2<-X2+YY2)", "3:(XX3+YY3<-X3+YY3)"); } finally { Utils.delete(baseDir); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java index f92c5ca81414f..449ea0501f439 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java @@ -105,7 +105,7 @@ public void testJoin() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+null", "1:X1+null"); + processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); checkJoinedValues(getter, kv(0, "X0+null"), kv(1, "X1+null"), kv(2, null), kv(3, null)); // push two items to the other stream. this should produce two items. @@ -114,7 +114,7 @@ public void testJoin() throws Exception { driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+Y0", "1:X1+Y1"); + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null)); // push all four items to the primary stream. this should produce four items. @@ -123,7 +123,7 @@ public void testJoin() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null"); + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null"); checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, "X2+null"), kv(3, "X3+null")); // push all items to the other stream. this should produce four items. @@ -131,7 +131,7 @@ public void testJoin() throws Exception { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); // push all four items to the primary stream. this should produce four items. @@ -140,7 +140,7 @@ public void testJoin() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); // push two items with null to the other stream as deletes. this should produce two item. @@ -149,7 +149,7 @@ public void testJoin() throws Exception { driver.process(topic2, expectedKeys[i], null); } - processor.checkAndClearResult("0:X0+null", "1:X1+null"); + processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); checkJoinedValues(getter, kv(0, "X0+null"), kv(1, "X1+null"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); // push all four items to the primary stream. this should produce four items. @@ -158,7 +158,7 @@ public void testJoin() throws Exception { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:XX0+null", "1:XX1+null", "2:XX2+YY2", "3:XX3+YY3"); + processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+YY2", "3:XX3+YY3"); checkJoinedValues(getter, kv(0, "XX0+null"), kv(1, "XX1+null"), kv(2, "XX2+YY2"), kv(3, "XX3+YY3")); } finally { @@ -200,7 +200,7 @@ public void testNotSendingOldValue() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+null<-null)", "1:(X1+null<-null)"); + proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)"); // push two items to the other stream. this should produce two items. @@ -208,7 +208,7 @@ public void testNotSendingOldValue() throws Exception { driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)"); + proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)"); // push all four items to the primary stream. this should produce four items. @@ -216,14 +216,14 @@ public void testNotSendingOldValue() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(X2+null<-null)", "3:(X3+null<-null)"); + proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(X2+null<-null)", "3:(X3+null<-null)"); // push all items to the other stream. this should produce four items. for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); + proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); // push all four items to the primary stream. this should produce four items. @@ -231,7 +231,7 @@ public void testNotSendingOldValue() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); + proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); // push two items with null to the other stream as deletes. this should produce two item. @@ -239,7 +239,7 @@ public void testNotSendingOldValue() throws Exception { driver.process(topic2, expectedKeys[i], null); } - proc.checkAndClearResult("0:(X0+null<-null)", "1:(X1+null<-null)"); + proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)"); // push all four items to the primary stream. this should produce four items. @@ -247,7 +247,7 @@ public void testNotSendingOldValue() throws Exception { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - proc.checkAndClearResult("0:(XX0+null<-null)", "1:(XX1+null<-null)", "2:(XX2+YY2<-null)", "3:(XX3+YY3<-null)"); + proc.checkAndClearProcessResult("0:(XX0+null<-null)", "1:(XX1+null<-null)", "2:(XX2+YY2<-null)", "3:(XX3+YY3<-null)"); } finally { Utils.delete(baseDir); @@ -290,7 +290,7 @@ public void testSendingOldValue() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+null<-null)", "1:(X1+null<-null)"); + proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)"); // push two items to the other stream. this should produce two items. @@ -298,7 +298,7 @@ public void testSendingOldValue() throws Exception { driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+Y0<-X0+null)", "1:(X1+Y1<-X1+null)"); + proc.checkAndClearProcessResult("0:(X0+Y0<-X0+null)", "1:(X1+Y1<-X1+null)"); // push all four items to the primary stream. this should produce four items. @@ -306,14 +306,14 @@ public void testSendingOldValue() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(X2+null<-null)", "3:(X3+null<-null)"); + proc.checkAndClearProcessResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(X2+null<-null)", "3:(X3+null<-null)"); // push all items to the other stream. this should produce four items. for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+YY0<-X0+Y0)", "1:(X1+YY1<-X1+Y1)", "2:(X2+YY2<-X2+null)", "3:(X3+YY3<-X3+null)"); + proc.checkAndClearProcessResult("0:(X0+YY0<-X0+Y0)", "1:(X1+YY1<-X1+Y1)", "2:(X2+YY2<-X2+null)", "3:(X3+YY3<-X3+null)"); // push all four items to the primary stream. this should produce four items. @@ -321,7 +321,7 @@ public void testSendingOldValue() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+YY0<-X0+YY0)", "1:(X1+YY1<-X1+YY1)", "2:(X2+YY2<-X2+YY2)", "3:(X3+YY3<-X3+YY3)"); + proc.checkAndClearProcessResult("0:(X0+YY0<-X0+YY0)", "1:(X1+YY1<-X1+YY1)", "2:(X2+YY2<-X2+YY2)", "3:(X3+YY3<-X3+YY3)"); // push two items with null to the other stream as deletes. this should produce two item. @@ -329,7 +329,7 @@ public void testSendingOldValue() throws Exception { driver.process(topic2, expectedKeys[i], null); } - proc.checkAndClearResult("0:(X0+null<-X0+YY0)", "1:(X1+null<-X1+YY1)"); + proc.checkAndClearProcessResult("0:(X0+null<-X0+YY0)", "1:(X1+null<-X1+YY1)"); // push all four items to the primary stream. this should produce four items. @@ -337,7 +337,7 @@ public void testSendingOldValue() throws Exception { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - proc.checkAndClearResult("0:(XX0+null<-X0+null)", "1:(XX1+null<-X1+null)", "2:(XX2+YY2<-X2+YY2)", "3:(XX3+YY3<-X3+YY3)"); + proc.checkAndClearProcessResult("0:(XX0+null<-X0+null)", "1:(XX1+null<-X1+null)", "2:(XX2+YY2<-X2+YY2)", "3:(XX3+YY3<-X3+YY3)"); } finally { Utils.delete(baseDir); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java index 6cc77e0b1b0c6..ea7476ae2e1b3 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java @@ -100,7 +100,7 @@ public void testJoin() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+null", "1:X1+null"); + processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); checkJoinedValues(getter, kv(0, "X0+null"), kv(1, "X1+null"), kv(2, null), kv(3, null)); // push two items to the other stream. this should produce two items. @@ -109,7 +109,7 @@ public void testJoin() throws Exception { driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+Y0", "1:X1+Y1"); + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null)); // push all four items to the primary stream. this should produce four items. @@ -118,7 +118,7 @@ public void testJoin() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null"); + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null"); checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, "X2+null"), kv(3, "X3+null")); // push all items to the other stream. this should produce four items. @@ -126,7 +126,7 @@ public void testJoin() throws Exception { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); // push all four items to the primary stream. this should produce four items. @@ -135,7 +135,7 @@ public void testJoin() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - processor.checkAndClearResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); // push two items with null to the other stream as deletes. this should produce two item. @@ -144,7 +144,7 @@ public void testJoin() throws Exception { driver.process(topic2, expectedKeys[i], null); } - processor.checkAndClearResult("0:X0+null", "1:X1+null"); + processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); checkJoinedValues(getter, kv(0, "X0+null"), kv(1, "X1+null"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); // push all four items to the primary stream. this should produce four items. @@ -153,7 +153,7 @@ public void testJoin() throws Exception { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - processor.checkAndClearResult("0:XX0+null", "1:XX1+null", "2:XX2+YY2", "3:XX3+YY3"); + processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+YY2", "3:XX3+YY3"); checkJoinedValues(getter, kv(0, "XX0+null"), kv(1, "XX1+null"), kv(2, "XX2+YY2"), kv(3, "XX3+YY3")); // push middle two items to the primary stream with null. this should produce two items. @@ -162,7 +162,7 @@ public void testJoin() throws Exception { driver.process(topic1, expectedKeys[i], null); } - processor.checkAndClearResult("1:null", "2:null+YY2"); + processor.checkAndClearProcessResult("1:null", "2:null+YY2"); checkJoinedValues(getter, kv(0, "XX0+null"), kv(1, null), kv(2, "null+YY2"), kv(3, "XX3+YY3")); } finally { @@ -204,7 +204,7 @@ public void testNotSendingOldValue() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+null<-null)", "1:(X1+null<-null)"); + proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)"); // push two items to the other stream. this should produce two items. @@ -212,7 +212,7 @@ public void testNotSendingOldValue() throws Exception { driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)"); + proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)"); // push all four items to the primary stream. this should produce four items. @@ -220,14 +220,14 @@ public void testNotSendingOldValue() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(X2+null<-null)", "3:(X3+null<-null)"); + proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(X2+null<-null)", "3:(X3+null<-null)"); // push all items to the other stream. this should produce four items. for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); + proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); // push all four items to the primary stream. this should produce four items. @@ -235,7 +235,7 @@ public void testNotSendingOldValue() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); + proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); // push two items with null to the other stream as deletes. this should produce two item. @@ -243,7 +243,7 @@ public void testNotSendingOldValue() throws Exception { driver.process(topic2, expectedKeys[i], null); } - proc.checkAndClearResult("0:(X0+null<-null)", "1:(X1+null<-null)"); + proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)"); // push all four items to the primary stream. this should produce four items. @@ -251,7 +251,7 @@ public void testNotSendingOldValue() throws Exception { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - proc.checkAndClearResult("0:(XX0+null<-null)", "1:(XX1+null<-null)", "2:(XX2+YY2<-null)", "3:(XX3+YY3<-null)"); + proc.checkAndClearProcessResult("0:(XX0+null<-null)", "1:(XX1+null<-null)", "2:(XX2+YY2<-null)", "3:(XX3+YY3<-null)"); // push middle two items to the primary stream with null. this should produce two items. @@ -259,7 +259,7 @@ public void testNotSendingOldValue() throws Exception { driver.process(topic1, expectedKeys[i], null); } - proc.checkAndClearResult("1:(null<-null)", "2:(null+YY2<-null)"); + proc.checkAndClearProcessResult("1:(null<-null)", "2:(null+YY2<-null)"); } finally { Utils.delete(baseDir); @@ -302,7 +302,7 @@ public void testSendingOldValue() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+null<-null)", "1:(X1+null<-null)"); + proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)"); // push two items to the other stream. this should produce two items. @@ -310,7 +310,7 @@ public void testSendingOldValue() throws Exception { driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+Y0<-X0+null)", "1:(X1+Y1<-X1+null)"); + proc.checkAndClearProcessResult("0:(X0+Y0<-X0+null)", "1:(X1+Y1<-X1+null)"); // push all four items to the primary stream. this should produce four items. @@ -318,14 +318,14 @@ public void testSendingOldValue() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(X2+null<-null)", "3:(X3+null<-null)"); + proc.checkAndClearProcessResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(X2+null<-null)", "3:(X3+null<-null)"); // push all items to the other stream. this should produce four items. for (int i = 0; i < expectedKeys.length; i++) { driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+YY0<-X0+Y0)", "1:(X1+YY1<-X1+Y1)", "2:(X2+YY2<-X2+null)", "3:(X3+YY3<-X3+null)"); + proc.checkAndClearProcessResult("0:(X0+YY0<-X0+Y0)", "1:(X1+YY1<-X1+Y1)", "2:(X2+YY2<-X2+null)", "3:(X3+YY3<-X3+null)"); // push all four items to the primary stream. this should produce four items. @@ -333,7 +333,7 @@ public void testSendingOldValue() throws Exception { driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); } - proc.checkAndClearResult("0:(X0+YY0<-X0+YY0)", "1:(X1+YY1<-X1+YY1)", "2:(X2+YY2<-X2+YY2)", "3:(X3+YY3<-X3+YY3)"); + proc.checkAndClearProcessResult("0:(X0+YY0<-X0+YY0)", "1:(X1+YY1<-X1+YY1)", "2:(X2+YY2<-X2+YY2)", "3:(X3+YY3<-X3+YY3)"); // push two items with null to the other stream as deletes. this should produce two item. @@ -341,7 +341,7 @@ public void testSendingOldValue() throws Exception { driver.process(topic2, expectedKeys[i], null); } - proc.checkAndClearResult("0:(X0+null<-X0+YY0)", "1:(X1+null<-X1+YY1)"); + proc.checkAndClearProcessResult("0:(X0+null<-X0+YY0)", "1:(X1+null<-X1+YY1)"); // push all four items to the primary stream. this should produce four items. @@ -349,7 +349,7 @@ public void testSendingOldValue() throws Exception { driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - proc.checkAndClearResult("0:(XX0+null<-X0+null)", "1:(XX1+null<-X1+null)", "2:(XX2+YY2<-X2+YY2)", "3:(XX3+YY3<-X3+YY3)"); + proc.checkAndClearProcessResult("0:(XX0+null<-X0+null)", "1:(XX1+null<-X1+null)", "2:(XX2+YY2<-X2+YY2)", "3:(XX3+YY3<-X3+YY3)"); // push middle two items to the primary stream with null. this should produce two items. @@ -357,7 +357,7 @@ public void testSendingOldValue() throws Exception { driver.process(topic1, expectedKeys[i], null); } - proc.checkAndClearResult("1:(null<-XX1+null)", "2:(null+YY2<-XX2+YY2)"); + proc.checkAndClearProcessResult("1:(null<-XX1+null)", "2:(null+YY2<-XX2+YY2)"); } finally { Utils.delete(baseDir); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java index 9ec12583b5bbd..9cafe8b3bb0c4 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java @@ -223,20 +223,20 @@ public Integer apply(String value) { driver.process(topic1, "B", "01"); driver.process(topic1, "C", "01"); - proc.checkAndClearResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)"); + proc.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)"); driver.process(topic1, "A", "02"); driver.process(topic1, "B", "02"); - proc.checkAndClearResult("A:(2<-null)", "B:(2<-null)"); + proc.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)"); driver.process(topic1, "A", "03"); - proc.checkAndClearResult("A:(3<-null)"); + proc.checkAndClearProcessResult("A:(3<-null)"); driver.process(topic1, "A", null); - proc.checkAndClearResult("A:(null<-null)"); + proc.checkAndClearProcessResult("A:(null<-null)"); } finally { Utils.delete(stateDir); @@ -276,20 +276,20 @@ public Integer apply(String value) { driver.process(topic1, "B", "01"); driver.process(topic1, "C", "01"); - proc.checkAndClearResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)"); + proc.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)"); driver.process(topic1, "A", "02"); driver.process(topic1, "B", "02"); - proc.checkAndClearResult("A:(2<-1)", "B:(2<-1)"); + proc.checkAndClearProcessResult("A:(2<-1)", "B:(2<-1)"); driver.process(topic1, "A", "03"); - proc.checkAndClearResult("A:(3<-2)"); + proc.checkAndClearProcessResult("A:(3<-2)"); driver.process(topic1, "A", null); - proc.checkAndClearResult("A:(null<-3)"); + proc.checkAndClearProcessResult("A:(null<-3)"); } finally { Utils.delete(stateDir); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java index 51276f3228271..7c158e2bb6b20 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java @@ -131,21 +131,21 @@ public void testNotSedingOldValue() throws IOException { driver.process(topic1, "B", "01"); driver.process(topic1, "C", "01"); - proc1.checkAndClearResult("A:(01<-null)", "B:(01<-null)", "C:(01<-null)"); + proc1.checkAndClearProcessResult("A:(01<-null)", "B:(01<-null)", "C:(01<-null)"); driver.process(topic1, "A", "02"); driver.process(topic1, "B", "02"); - proc1.checkAndClearResult("A:(02<-null)", "B:(02<-null)"); + proc1.checkAndClearProcessResult("A:(02<-null)", "B:(02<-null)"); driver.process(topic1, "A", "03"); - proc1.checkAndClearResult("A:(03<-null)"); + proc1.checkAndClearProcessResult("A:(03<-null)"); driver.process(topic1, "A", null); driver.process(topic1, "B", null); - proc1.checkAndClearResult("A:(null<-null)", "B:(null<-null)"); + proc1.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)"); } finally { Utils.delete(stateDir); @@ -176,21 +176,21 @@ public void testSedingOldValue() throws IOException { driver.process(topic1, "B", "01"); driver.process(topic1, "C", "01"); - proc1.checkAndClearResult("A:(01<-null)", "B:(01<-null)", "C:(01<-null)"); + proc1.checkAndClearProcessResult("A:(01<-null)", "B:(01<-null)", "C:(01<-null)"); driver.process(topic1, "A", "02"); driver.process(topic1, "B", "02"); - proc1.checkAndClearResult("A:(02<-01)", "B:(02<-01)"); + proc1.checkAndClearProcessResult("A:(02<-01)", "B:(02<-01)"); driver.process(topic1, "A", "03"); - proc1.checkAndClearResult("A:(03<-02)"); + proc1.checkAndClearProcessResult("A:(03<-02)"); driver.process(topic1, "A", null); driver.process(topic1, "B", null); - proc1.checkAndClearResult("A:(null<-03)", "B:(null<-02)"); + proc1.checkAndClearProcessResult("A:(null<-03)", "B:(null<-02)"); } finally { Utils.delete(stateDir); diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java index 5bf1b5e04199a..a1c07af7d3d09 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/PartitionGroupTest.java @@ -21,7 +21,6 @@ import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.common.TopicPartition; -import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.IntegerDeserializer; import org.apache.kafka.common.serialization.IntegerSerializer; @@ -60,17 +59,17 @@ public void testTimeTracking() { // add three 3 records with timestamp 1, 3, 5 to partition-1 List> list1 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 3, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 5, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)); + new ConsumerRecord<>("topic", 1, 1L, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 3L, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 5L, recordKey, recordValue)); group.addRawRecords(partition1, list1); // add three 3 records with timestamp 2, 4, 6 to partition-2 List> list2 = Arrays.asList( - new ConsumerRecord<>("topic", 1, 2, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 4, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), - new ConsumerRecord<>("topic", 1, 6, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue)); + new ConsumerRecord<>("topic", 2, 2L, recordKey, recordValue), + new ConsumerRecord<>("topic", 2, 4L, recordKey, recordValue), + new ConsumerRecord<>("topic", 2, 6L, recordKey, recordValue)); group.addRawRecords(partition2, list2); @@ -82,7 +81,7 @@ public void testTimeTracking() { StampedRecord record; PartitionGroup.RecordInfo info = new PartitionGroup.RecordInfo(); - // get one record + // get one record, now the time should be advanced record = group.nextRecord(info); assertEquals(partition1, info.partition()); assertEquals(1L, record.timestamp); @@ -99,5 +98,72 @@ record = group.nextRecord(info); assertEquals(2, group.numBuffered(partition1)); assertEquals(2, group.numBuffered(partition2)); assertEquals(3L, group.timestamp()); + + // add three 3 records with timestamp 2, 4, 6 to partition-1 again + List> list3 = Arrays.asList( + new ConsumerRecord<>("topic", 1, 2L, recordKey, recordValue), + new ConsumerRecord<>("topic", 1, 4L, recordKey, recordValue)); + + group.addRawRecords(partition1, list3); + + assertEquals(6, group.numBuffered()); + assertEquals(4, group.numBuffered(partition1)); + assertEquals(2, group.numBuffered(partition2)); + assertEquals(3L, group.timestamp()); + + // get one record, time should not be advanced + record = group.nextRecord(info); + assertEquals(partition1, info.partition()); + assertEquals(3L, record.timestamp); + assertEquals(5, group.numBuffered()); + assertEquals(3, group.numBuffered(partition1)); + assertEquals(2, group.numBuffered(partition2)); + assertEquals(3L, group.timestamp()); + + // get one more record, now time should be advanced + record = group.nextRecord(info); + assertEquals(partition1, info.partition()); + assertEquals(5L, record.timestamp); + assertEquals(4, group.numBuffered()); + assertEquals(2, group.numBuffered(partition1)); + assertEquals(2, group.numBuffered(partition2)); + assertEquals(3L, group.timestamp()); + + // get one more record, time should not be advanced + record = group.nextRecord(info); + assertEquals(partition1, info.partition()); + assertEquals(2L, record.timestamp); + assertEquals(3, group.numBuffered()); + assertEquals(1, group.numBuffered(partition1)); + assertEquals(2, group.numBuffered(partition2)); + assertEquals(4L, group.timestamp()); + + // get one more record, now time should be advanced + record = group.nextRecord(info); + assertEquals(partition2, info.partition()); + assertEquals(4L, record.timestamp); + assertEquals(2, group.numBuffered()); + assertEquals(1, group.numBuffered(partition1)); + assertEquals(1, group.numBuffered(partition2)); + assertEquals(4L, group.timestamp()); + + // get one more record, time should not be advanced + record = group.nextRecord(info); + assertEquals(partition1, info.partition()); + assertEquals(4L, record.timestamp); + assertEquals(1, group.numBuffered()); + assertEquals(0, group.numBuffered(partition1)); + assertEquals(1, group.numBuffered(partition2)); + assertEquals(4L, group.timestamp()); + + // get one more record, time should not be advanced + record = group.nextRecord(info); + assertEquals(partition2, info.partition()); + assertEquals(6L, record.timestamp); + assertEquals(0, group.numBuffered()); + assertEquals(0, group.numBuffered(partition1)); + assertEquals(0, group.numBuffered(partition2)); + assertEquals(4L, group.timestamp()); + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index 33fa5c4a091a7..dd4894744010a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -32,6 +32,7 @@ import org.apache.kafka.streams.StreamsConfig; import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.processor.TaskId; +import org.apache.kafka.test.MockProcessorNode; import org.apache.kafka.test.MockSourceNode; import org.apache.kafka.test.MockTimestampExtractor; import org.junit.Test; @@ -46,6 +47,7 @@ import java.util.Set; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; import static org.junit.Assert.assertTrue; public class StreamTaskTest { @@ -58,10 +60,12 @@ public class StreamTaskTest { private final TopicPartition partition2 = new TopicPartition("topic2", 1); private final Set partitions = Utils.mkSet(partition1, partition2); - private final MockSourceNode source1 = new MockSourceNode<>(intDeserializer, intDeserializer); - private final MockSourceNode source2 = new MockSourceNode<>(intDeserializer, intDeserializer); + private final MockSourceNode source1 = new MockSourceNode<>(intDeserializer, intDeserializer); + private final MockSourceNode source2 = new MockSourceNode<>(intDeserializer, intDeserializer); + private final MockProcessorNode processor = new MockProcessorNode<>(10L); + private final ProcessorTopology topology = new ProcessorTopology( - Arrays.asList((ProcessorNode) source1, (ProcessorNode) source2), + Arrays.asList((ProcessorNode) source1, (ProcessorNode) source2, (ProcessorNode) processor), new HashMap() { { put("topic1", source1); @@ -94,6 +98,8 @@ private StreamsConfig createConfig(final File baseDir) throws Exception { @Before public void setup() { consumer.assign(Arrays.asList(partition1, partition2)); + source1.addChild(processor); + source2.addChild(processor); } @SuppressWarnings("unchecked") @@ -211,6 +217,73 @@ public void testPauseResume() throws Exception { } } + @SuppressWarnings("unchecked") + @Test + public void testMaybePunctuate() throws Exception { + File baseDir = Files.createTempDirectory("test").toFile(); + try { + StreamsConfig config = createConfig(baseDir); + StreamTask task = new StreamTask(new TaskId(0, 0), "applicationId", partitions, topology, consumer, producer, restoreStateConsumer, config, null); + + task.addRecords(partition1, records( + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue) + )); + + task.addRecords(partition2, records( + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 15, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 25, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue) + )); + + assertTrue(task.maybePunctuate()); + + assertEquals(5, task.process()); + assertEquals(1, source1.numReceived); + assertEquals(0, source2.numReceived); + + assertFalse(task.maybePunctuate()); + + assertEquals(4, task.process()); + assertEquals(1, source1.numReceived); + assertEquals(1, source2.numReceived); + + assertTrue(task.maybePunctuate()); + + assertEquals(3, task.process()); + assertEquals(2, source1.numReceived); + assertEquals(1, source2.numReceived); + + assertFalse(task.maybePunctuate()); + + assertEquals(2, task.process()); + assertEquals(2, source1.numReceived); + assertEquals(2, source2.numReceived); + + assertTrue(task.maybePunctuate()); + + assertEquals(1, task.process()); + assertEquals(3, source1.numReceived); + assertEquals(2, source2.numReceived); + + assertFalse(task.maybePunctuate()); + + assertEquals(0, task.process()); + assertEquals(3, source1.numReceived); + assertEquals(3, source2.numReceived); + + assertFalse(task.maybePunctuate()); + + processor.supplier.checkAndClearPunctuateResult(10L, 20L, 30L); + + task.close(); + + } finally { + Utils.delete(baseDir); + } + } + private Iterable> records(ConsumerRecord... recs) { return Arrays.asList(recs); } diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java index d3b808177a70c..287af5a61b191 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -143,7 +143,7 @@ public StateStore getStateStore(String name) { @Override public void schedule(long interval) { - throw new UnsupportedOperationException("schedule() not supported"); + throw new UnsupportedOperationException("schedule() not supported."); } @Override diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorNode.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorNode.java new file mode 100644 index 0000000000000..cf8a52600705f --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorNode.java @@ -0,0 +1,49 @@ +/** + * 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 org.apache.kafka.test; + +import org.apache.kafka.streams.processor.internals.ProcessorNode; + +import java.util.Collections; +import java.util.concurrent.atomic.AtomicInteger; + +public class MockProcessorNode extends ProcessorNode { + + public static final String NAME = "MOCK-PROCESS-"; + public static final AtomicInteger INDEX = new AtomicInteger(1); + + public int numReceived = 0; + + public final MockProcessorSupplier supplier; + + public MockProcessorNode(long scheduleInterval) { + this(new MockProcessorSupplier(scheduleInterval)); + } + + private MockProcessorNode(MockProcessorSupplier supplier) { + super(NAME + INDEX.getAndIncrement(), supplier.get(), Collections.emptySet()); + + this.supplier = supplier; + } + + @Override + public void process(K key, V value) { + this.numReceived++; + processor().process(key, value); + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java index b402525beaa88..921c365cad033 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java @@ -17,6 +17,7 @@ package org.apache.kafka.test; +import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.ProcessorSupplier; @@ -30,16 +31,28 @@ public class MockProcessorSupplier implements ProcessorSupplier { public final ArrayList processed = new ArrayList<>(); public final ArrayList punctuated = new ArrayList<>(); + private final long scheduleInterval; + + public MockProcessorSupplier() { + this(-1L); + } + + public MockProcessorSupplier(long scheduleInterval) { + this.scheduleInterval = scheduleInterval; + } + @Override public Processor get() { return new MockProcessor(); } - public class MockProcessor implements Processor { + public class MockProcessor extends AbstractProcessor { @Override public void init(ProcessorContext context) { - // do nothing + super.init(context); + if (scheduleInterval > 0L) + context.schedule(scheduleInterval); } @Override @@ -49,21 +62,30 @@ public void process(K key, V value) { @Override public void punctuate(long streamTime) { + assertEquals(streamTime, context().timestamp()); + assertEquals(null, context().topic()); + assertEquals(-1, context().partition()); + assertEquals(-1L, context().offset()); + punctuated.add(streamTime); } + } - @Override - public void close() { - // do nothing + public void checkAndClearProcessResult(String... expected) { + assertEquals("the number of outputs:", expected.length, processed.size()); + + for (int i = 0; i < expected.length; i++) { + assertEquals("output[" + i + "]:", expected[i], processed.get(i)); } + processed.clear(); } - public void checkAndClearResult(String... expected) { - assertEquals("the number of outputs:", expected.length, processed.size()); + public void checkAndClearPunctuateResult(long... expected) { + assertEquals("the number of outputs:", expected.length, punctuated.size()); for (int i = 0; i < expected.length; i++) { - assertEquals("output[" + i + "]:", expected[i], processed.get(i)); + assertEquals("output[" + i + "]:", expected[i], (long) punctuated.get(i)); } processed.clear(); From 9beafae23a83774fc1d9ea811d449eac34240363 Mon Sep 17 00:00:00 2001 From: Eno Thereska Date: Fri, 8 Apr 2016 09:17:05 -0700 Subject: [PATCH 127/206] KAFKA-3512: Added foreach operator miguno guozhangwang please have a look if you can. Author: Eno Thereska Reviewers: Michael G. Noll , Guozhang Wang Closes #1193 from enothereska/kafka-3512-ForEach --- .../kafka/streams/kstream/ForeachAction.java | 35 ++++++++ .../apache/kafka/streams/kstream/KStream.java | 8 ++ .../apache/kafka/streams/kstream/KTable.java | 8 ++ .../kstream/internals/KStreamForeach.java | 44 ++++++++++ .../kstream/internals/KStreamImpl.java | 8 ++ .../streams/kstream/internals/KTableImpl.java | 15 ++++ .../kstream/internals/KStreamForeachTest.java | 85 +++++++++++++++++++ .../kstream/internals/KTableForeachTest.java | 85 +++++++++++++++++++ 8 files changed, 288 insertions(+) create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamForeach.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamForeachTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableForeachTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java new file mode 100644 index 0000000000000..83064e8034efd --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java @@ -0,0 +1,35 @@ +/** + * 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 org.apache.kafka.streams.kstream; + + + +/** + * The ForeachAction interface for performing an action on a key-value pair. + * Note that this action is stateless. If stateful processing is required, consider + * using {@link KStream#transform(TransformerSupplier, String...)} or + * {@link KStream#process(ProcessorSupplier, String...)} instead. + * + * @param original key type + * @param original value type + */ +public interface ForeachAction { + void apply(K key, V value); +} + + diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index e4933cb08e71f..a55e7264eb2ac 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -100,6 +100,14 @@ public interface KStream { */ KStream through(String topic); + /** + * Perform an action on each element of {@link KStream}. + * Note that this is a terminal operation that returns void. + * + * @param action An action to perform on each element + */ + void foreach(ForeachAction action); + /** * Materialize this stream to a topic, also creates a new instance of {@link KStream} from the topic * using default serializers and deserializers and a customizable {@link StreamPartitioner} to determine the distribution of records to partitions. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index 581ee282360eb..1f6ee687c12c9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -297,4 +297,12 @@ KTable count(KeyValueMapper selector, * @param the key type of the aggregated {@link KTable} */ KTable count(KeyValueMapper selector, String name); + + /** + * Perform an action on each element of {@link KTable}. + * Note that this is a terminal operation that returns void. + * + * @param action An action to perform on each element + */ + void foreach(ForeachAction action); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamForeach.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamForeach.java new file mode 100644 index 0000000000000..2fd7ef9ad0bb7 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamForeach.java @@ -0,0 +1,44 @@ +/** + * 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 org.apache.kafka.streams.kstream.internals; + +import org.apache.kafka.streams.kstream.ForeachAction; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +class KStreamForeach implements ProcessorSupplier { + + private final ForeachAction action; + + public KStreamForeach(ForeachAction action) { + this.action = action; + } + + @Override + public Processor get() { + return new KStreamForeachProcessor(); + } + + private class KStreamForeachProcessor extends AbstractProcessor { + @Override + public void process(K key, V value) { + action.apply(key, value); + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 0fb3984245f10..c266328429730 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.kstream.Aggregator; +import org.apache.kafka.streams.kstream.ForeachAction; import org.apache.kafka.streams.kstream.Initializer; import org.apache.kafka.streams.kstream.JoinWindows; import org.apache.kafka.streams.kstream.KStreamBuilder; @@ -92,6 +93,8 @@ public class KStreamImpl extends AbstractStream implements KStream sourceNodes) { super(topology, name, sourceNodes); } @@ -201,6 +204,11 @@ public KStream through(Serde keySerde, Serde valSerde, StreamPartiti } @Override + public void foreach(ForeachAction action) { + String name = topology.newName(FOREACH_NAME); + + topology.addProcessor(name, new KStreamForeach(action), this.name); + } public KStream through(Serde keySerde, Serde valSerde, String topic) { return through(keySerde, valSerde, null, topic); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index 156f2db4b18c3..8de9a0bbb31fc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -20,6 +20,7 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.kstream.Aggregator; +import org.apache.kafka.streams.kstream.ForeachAction; import org.apache.kafka.streams.kstream.Initializer; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; @@ -76,6 +77,8 @@ public class KTableImpl extends AbstractStream implements KTable processorSupplier; private final Serde keySerde; @@ -141,6 +144,18 @@ public KTable through(Serde keySerde, return topology.table(keySerde, valSerde, topic); } + @Override + public void foreach(final ForeachAction action) { + String name = topology.newName(FOREACH_NAME); + KStreamForeach> processorSupplier = new KStreamForeach(new ForeachAction>() { + @Override + public void apply(K key, Change value) { + action.apply(key, value.newValue); + } + }); + topology.addProcessor(name, processorSupplier, this.name); + } + @Override public KTable through(Serde keySerde, Serde valSerde, String topic) { return through(keySerde, valSerde, null, topic); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamForeachTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamForeachTest.java new file mode 100644 index 0000000000000..65737790bcf39 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamForeachTest.java @@ -0,0 +1,85 @@ +/** + * 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 org.apache.kafka.streams.kstream.internals; + +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.kstream.ForeachAction; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.test.KStreamTestDriver; +import org.junit.Test; +import java.util.List; +import java.util.ArrayList; +import java.util.Arrays; + +import static org.junit.Assert.assertEquals; + +public class KStreamForeachTest { + + final private String topicName = "topic"; + + final private Serde intSerde = Serdes.Integer(); + final private Serde stringSerde = Serdes.String(); + + @Test + public void testForeach() { + // Given + List> inputRecords = Arrays.asList( + new KeyValue<>(0, "zero"), + new KeyValue<>(1, "one"), + new KeyValue<>(2, "two"), + new KeyValue<>(3, "three") + ); + + List> expectedRecords = Arrays.asList( + new KeyValue<>(0, "ZERO"), + new KeyValue<>(2, "ONE"), + new KeyValue<>(4, "TWO"), + new KeyValue<>(6, "THREE") + ); + + final List> actualRecords = new ArrayList<>(); + ForeachAction action = + new ForeachAction() { + @Override + public void apply(Integer key, String value) { + actualRecords.add(new KeyValue<>(key * 2, value.toUpperCase())); + } + }; + + // When + KStreamBuilder builder = new KStreamBuilder(); + KStream stream = builder.stream(intSerde, stringSerde, topicName); + stream.foreach(action); + + // Then + KStreamTestDriver driver = new KStreamTestDriver(builder); + for (KeyValue record: inputRecords) { + driver.process(topicName, record.key, record.value); + } + + assertEquals(expectedRecords.size(), actualRecords.size()); + for (int i = 0; i < expectedRecords.size(); i++) { + KeyValue expectedRecord = expectedRecords.get(i); + KeyValue actualRecord = actualRecords.get(i); + assertEquals(expectedRecord, actualRecord); + } + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableForeachTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableForeachTest.java new file mode 100644 index 0000000000000..4b612a5211489 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableForeachTest.java @@ -0,0 +1,85 @@ +/** + * 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 org.apache.kafka.streams.kstream.internals; + +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.kstream.ForeachAction; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.test.KStreamTestDriver; +import org.junit.Test; +import java.util.List; +import java.util.ArrayList; +import java.util.Arrays; + +import static org.junit.Assert.assertEquals; + +public class KTableForeachTest { + + final private String topicName = "topic"; + + final private Serde intSerde = Serdes.Integer(); + final private Serde stringSerde = Serdes.String(); + + @Test + public void testForeach() { + // Given + List> inputRecords = Arrays.asList( + new KeyValue<>(0, "zero"), + new KeyValue<>(1, "one"), + new KeyValue<>(2, "two"), + new KeyValue<>(3, "three") + ); + + List> expectedRecords = Arrays.asList( + new KeyValue<>(0, "ZERO"), + new KeyValue<>(2, "ONE"), + new KeyValue<>(4, "TWO"), + new KeyValue<>(6, "THREE") + ); + + final List> actualRecords = new ArrayList<>(); + ForeachAction action = + new ForeachAction() { + @Override + public void apply(Integer key, String value) { + actualRecords.add(new KeyValue<>(key * 2, value.toUpperCase())); + } + }; + + // When + KStreamBuilder builder = new KStreamBuilder(); + KTable table = builder.table(intSerde, stringSerde, topicName); + table.foreach(action); + + // Then + KStreamTestDriver driver = new KStreamTestDriver(builder); + for (KeyValue record: inputRecords) { + driver.process(topicName, record.key, record.value); + } + + assertEquals(expectedRecords.size(), actualRecords.size()); + for (int i = 0; i < expectedRecords.size(); i++) { + KeyValue expectedRecord = expectedRecords.get(i); + KeyValue actualRecord = actualRecords.get(i); + assertEquals(expectedRecord, actualRecord); + } + } +} From 4d467c2ec275b5659c2da0ca196409dffaa3caf3 Mon Sep 17 00:00:00 2001 From: =?UTF-8?q?Stig=20Rohde=20D=C3=B8ssing?= Date: Fri, 8 Apr 2016 09:44:51 -0700 Subject: [PATCH 128/206] KAFKA-725: Return OffsetOutOfRange error from ReplicaManager when non-follower attempts reading an offset that's above high watermark. MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit This should make Log.read act the same when startOffset is larger than maxOffset as it would if startOffset was larger than logEndOffset. The current behavior can result in an IllegalArgumentException from LogSegment if a consumer attempts to fetch an offset above the high watermark which is present in the leader's log. It seems more correct if Log.read presents the view of the log to consumers as if it simply ended at maxOffset (high watermark). I've tried to describe an example scenario of this happening here https://issues.apache.org/jira/browse/KAFKA-725?focusedCommentId=15221673&page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel#comment-15221673 I'm not sure I understand why ReplicaManager sets maxOffset to the high watermark, and not high watermark + 1. Isn't the high watermark the last committed message, and readable by consumers? Tests passed for me locally on second try, seems like it just hit a flaky test. Author: Stig Rohde Døssing Reviewers: Jiangjie Qin , Guozhang Wang Closes #1178 from srdo/KAFKA-725 --- .../scala/kafka/server/ReplicaManager.scala | 8 +- .../test/scala/unit/kafka/log/LogTest.scala | 9 +- .../kafka/server/ReplicaManagerTest.scala | 118 ++++++++++++++---- 3 files changed, 104 insertions(+), 31 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index f050e27c7e13f..22657f4c2a8cf 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -522,8 +522,12 @@ class ReplicaManager(val config: KafkaConfig, getReplicaOrException(topic, partition) // decide whether to only fetch committed data (i.e. messages below high watermark) - val maxOffsetOpt = if (readOnlyCommitted) - Some(localReplica.highWatermark.messageOffset) + val maxOffsetOpt = if (readOnlyCommitted) { + val maxOffset = localReplica.highWatermark.messageOffset + if(offset > maxOffset) + throw new OffsetOutOfRangeException("Request for offset %d beyond high watermark %d when reading from only committed offsets".format(offset, maxOffset)) + Some(maxOffset) + } else None diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 4d75d53c254a2..3f6a2753a555c 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -228,6 +228,7 @@ class LogTest extends JUnitSuite { /** * Test reading at the boundary of the log, specifically * - reading from the logEndOffset should give an empty message set + * - reading from the the maxOffset should give an empty message set * - reading beyond the log end offset should throw an OffsetOutOfRangeException */ @Test @@ -236,19 +237,21 @@ class LogTest extends JUnitSuite { val logProps = new Properties() logProps.put(LogConfig.SegmentBytesProp, 1024: java.lang.Integer) val log = new Log(logDir, LogConfig(logProps), recoveryPoint = 0L, time.scheduler, time = time) - assertEquals("Reading just beyond end of log should produce 0 byte read.", 0, log.read(1024, 1000).messageSet.sizeInBytes) + log.append(new ByteBufferMessageSet(NoCompressionCodec, messages = new Message("42".getBytes))) + assertEquals("Reading just beyond end of log should produce 0 byte read.", 0, log.read(1025, 1000).messageSet.sizeInBytes) try { - log.read(0, 1024) + log.read(0, 1025) fail("Expected exception on invalid read.") } catch { case e: OffsetOutOfRangeException => "This is good." } try { - log.read(1025, 1000) + log.read(1026, 1000) fail("Expected exception on invalid read.") } catch { case e: OffsetOutOfRangeException => // This is good. } + assertEquals("Reading from maxOffset should produce 0 byte read.", 0, log.read(1024, 1000, Some(1024)).messageSet.sizeInBytes) } /** diff --git a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala index ee14af4af518d..c2c670e6b69cd 100644 --- a/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ReplicaManagerTest.scala @@ -35,8 +35,8 @@ import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.utils.{MockTime => JMockTime} import org.apache.kafka.common.{Node, TopicPartition} import org.easymock.EasyMock -import org.junit.Assert.{assertEquals, assertTrue} -import org.junit.Test +import org.junit.Assert.{assertEquals, assertTrue, assertFalse} +import org.junit.{Test, Before, After} import scala.collection.JavaConverters._ import scala.collection.Map @@ -44,17 +44,28 @@ import scala.collection.Map class ReplicaManagerTest { val topic = "test-topic" + val time = new MockTime() + val jTime = new JMockTime + val metrics = new Metrics + var zkClient : ZkClient = _ + var zkUtils : ZkUtils = _ + + @Before + def setUp() { + zkClient = EasyMock.createMock(classOf[ZkClient]) + zkUtils = ZkUtils(zkClient, isZkSecurityEnabled = false) + } + + @After + def tearDown() { + metrics.close(); + } @Test def testHighWaterMarkDirectoryMapping() { val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) val config = KafkaConfig.fromProps(props) - val zkClient = EasyMock.createMock(classOf[ZkClient]) - val zkUtils = ZkUtils(zkClient, isZkSecurityEnabled = false) val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) - val time = new MockTime() - val jTime = new JMockTime - val metrics = new Metrics val rm = new ReplicaManager(config, metrics, time, jTime, zkUtils, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false)) try { @@ -64,7 +75,6 @@ class ReplicaManagerTest { } finally { // shutdown the replica manager upon test completion rm.shutdown(false) - metrics.close() } } @@ -73,12 +83,7 @@ class ReplicaManagerTest { val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath) val config = KafkaConfig.fromProps(props) - val zkClient = EasyMock.createMock(classOf[ZkClient]) - val zkUtils = ZkUtils(zkClient, false) val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) - val time = new MockTime() - val jTime = new JMockTime - val metrics = new Metrics val rm = new ReplicaManager(config, metrics, time, jTime, zkUtils, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false)) try { @@ -88,7 +93,6 @@ class ReplicaManagerTest { } finally { // shutdown the replica manager upon test completion rm.shutdown(checkpointHW = false) - metrics.close() } } @@ -96,12 +100,7 @@ class ReplicaManagerTest { def testIllegalRequiredAcks() { val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) val config = KafkaConfig.fromProps(props) - val zkClient = EasyMock.createMock(classOf[ZkClient]) - val zkUtils = ZkUtils(zkClient, isZkSecurityEnabled = false) val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) - val time = new MockTime() - val jTime = new JMockTime - val metrics = new Metrics val rm = new ReplicaManager(config, metrics, time, jTime, zkUtils, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false), Option(this.getClass.getName)) try { @@ -116,7 +115,6 @@ class ReplicaManagerTest { responseCallback = callback) } finally { rm.shutdown(checkpointHW = false) - metrics.close() } TestUtils.verifyNonDaemonThreadsStatus(this.getClass.getName) @@ -127,12 +125,7 @@ class ReplicaManagerTest { val props = TestUtils.createBrokerConfig(0, TestUtils.MockZkConnect) props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath) val config = KafkaConfig.fromProps(props) - val zkClient = EasyMock.createMock(classOf[ZkClient]) - val zkUtils = ZkUtils(zkClient, isZkSecurityEnabled = false) val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) - val time = new MockTime() - val jTime = new JMockTime - val metrics = new Metrics val rm = new ReplicaManager(config, metrics, time, jTime, zkUtils, new MockScheduler(time), mockLogMgr, new AtomicBoolean(false)) @@ -192,7 +185,80 @@ class ReplicaManagerTest { assertTrue(fetchCallbackFired) } finally { rm.shutdown(checkpointHW = false) - metrics.close() + } + } + + @Test + def testFetchBeyondHighWatermarkNotAllowedForConsumer() { + val props = TestUtils.createBrokerConfig(1, TestUtils.MockZkConnect) + props.put("log.dir", TestUtils.tempRelativeDir("data").getAbsolutePath) + props.put("broker.id", Int.box(0)) + val config = KafkaConfig.fromProps(props) + val mockLogMgr = TestUtils.createLogManager(config.logDirs.map(new File(_)).toArray) + val rm = new ReplicaManager(config, metrics, time, jTime, zkUtils, new MockScheduler(time), mockLogMgr, + new AtomicBoolean(false), Option(this.getClass.getName)) + try { + val aliveBrokers = Seq(new Broker(0, "host0", 0), new Broker(1, "host1", 1), new Broker(1, "host2", 2)) + val metadataCache = EasyMock.createMock(classOf[MetadataCache]) + EasyMock.expect(metadataCache.getAliveBrokers).andReturn(aliveBrokers).anyTimes() + EasyMock.replay(metadataCache) + + val brokerList : java.util.List[Integer] = Seq[Integer](0, 1, 2).asJava + val brokerSet : java.util.Set[Integer] = Set[Integer](0, 1, 2).asJava + + val partition = rm.getOrCreatePartition(topic, 0) + partition.getOrCreateReplica(0) + + // Make this replica the leader. + val leaderAndIsrRequest1 = new LeaderAndIsrRequest(0, 0, + collection.immutable.Map(new TopicPartition(topic, 0) -> new PartitionState(0, 0, 0, brokerList, 0, brokerSet)).asJava, + Set(new Node(0, "host1", 0), new Node(1, "host2", 1), new Node(2, "host2", 2)).asJava) + rm.becomeLeaderOrFollower(0, leaderAndIsrRequest1, metadataCache, (_, _) => {}) + rm.getLeaderReplicaIfLocal(topic, 0) + + def produceCallback(responseStatus: Map[TopicPartition, PartitionResponse]) = {} + + // Append a message. + for(i <- 1 to 2) + rm.appendMessages( + timeout = 1000, + requiredAcks = -1, + internalTopicsAllowed = false, + messagesPerPartition = Map(new TopicPartition(topic, 0) -> new ByteBufferMessageSet(new Message("message %d".format(i).getBytes))), + responseCallback = produceCallback) + + var fetchCallbackFired = false + var fetchError = 0 + def fetchCallback(responseStatus: Map[TopicAndPartition, FetchResponsePartitionData]) = { + fetchError = responseStatus.values.head.error + fetchCallbackFired = true + } + + // Fetch a message above the high watermark as a follower + rm.fetchMessages( + timeout = 1000, + replicaId = 1, + fetchMinBytes = 1, + fetchInfo = collection.immutable.Map(new TopicAndPartition(topic, 0) -> new PartitionFetchInfo(1, 100000)), + responseCallback = fetchCallback) + + + assertTrue(fetchCallbackFired) + assertEquals("Should not give an exception", Errors.NONE.code, fetchError) + fetchCallbackFired = false + + // Fetch a message above the high watermark as a consumer + rm.fetchMessages( + timeout = 1000, + replicaId = -1, + fetchMinBytes = 1, + fetchInfo = collection.immutable.Map(new TopicAndPartition(topic, 0) -> new PartitionFetchInfo(1, 100000)), + responseCallback = fetchCallback) + + assertTrue(fetchCallbackFired) + assertEquals("Should give OffsetOutOfRangeException", Errors.OFFSET_OUT_OF_RANGE.code, fetchError) + } finally { + rm.shutdown(checkpointHW = false) } } } From cbdd70ec0de4b4d6d04b62580a5a67ea69311c4a Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Fri, 8 Apr 2016 10:04:46 -0700 Subject: [PATCH 129/206] MINOR: improve logging of consumer system tests Author: Jason Gustafson Reviewers: Ewen Cheslack-Postava Closes #1199 from hachikuji/improve-consumer-systests --- tests/kafkatest/tests/client/consumer_test.py | 6 ++++++ .../java/org/apache/kafka/tools/VerifiableConsumer.java | 7 +++++++ 2 files changed, 13 insertions(+) diff --git a/tests/kafkatest/tests/client/consumer_test.py b/tests/kafkatest/tests/client/consumer_test.py index 084b19d5ec772..534f65cbc3c9e 100644 --- a/tests/kafkatest/tests/client/consumer_test.py +++ b/tests/kafkatest/tests/client/consumer_test.py @@ -66,6 +66,12 @@ def rolling_bounce_brokers(self, consumer, num_bounces=5, clean_shutdown=True): self.await_all_members(consumer) self.await_consumed_messages(consumer) + def setup_consumer(self, topic, **kwargs): + # collect verifiable consumer events since this makes debugging much easier + consumer = super(OffsetValidationTest, self).setup_consumer(topic, **kwargs) + self.mark_for_collect(consumer, 'verifiable_consumer_stdout') + return consumer + def test_broker_rolling_bounce(self): """ Verify correct consumer behavior when the brokers are consecutively restarted. diff --git a/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java b/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java index 25b87bd35ca1d..1880d7aa1c589 100644 --- a/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java +++ b/tools/src/main/java/org/apache/kafka/tools/VerifiableConsumer.java @@ -256,9 +256,16 @@ public void close() { } private static abstract class ConsumerEvent { + private final long timestamp = System.currentTimeMillis(); + @JsonProperty public abstract String name(); + @JsonProperty + public long timestamp() { + return timestamp; + } + @JsonProperty("class") public String clazz() { return VerifiableConsumer.class.getName(); From 2a8fa287862912491d1e118c52a2d194d8480075 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Fri, 8 Apr 2016 12:55:21 -0700 Subject: [PATCH 130/206] KAFKA-3528: handle wakeups while rebalancing more gracefully Author: Jason Gustafson Reviewers: Ewen Cheslack-Postava Closes #1201 from hachikuji/KAFKA-3528 --- .../internals/AbstractCoordinator.java | 22 +++++++--- .../internals/ConsumerCoordinatorTest.java | 40 +++++++++++++++++++ 2 files changed, 57 insertions(+), 5 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index 1e6757e9298cd..496a1141e06b6 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -218,13 +218,25 @@ public void ensureActiveGroup() { } RequestFuture future = sendJoinGroupRequest(); + future.addListener(new RequestFutureListener() { + @Override + public void onSuccess(ByteBuffer value) { + // handle join completion in the callback so that the callback will be invoked + // even if the consumer is woken up before finishing the rebalance + onJoinComplete(generation, memberId, protocol, value); + needsJoinPrepare = true; + heartbeatTask.reset(); + } + + @Override + public void onFailure(RuntimeException e) { + // we handle failures below after the request finishes. if the join completes + // after having been woken up, the exception is ignored and we will rejoin + } + }); client.poll(future); - if (future.succeeded()) { - onJoinComplete(generation, memberId, protocol, future.value()); - needsJoinPrepare = true; - heartbeatTask.reset(); - } else { + if (future.failed()) { RuntimeException exception = future.exception(); if (exception instanceof UnknownMemberIdException || exception instanceof RebalanceInProgressException || diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index 2189c304fb92e..b864d692fde44 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -35,6 +35,7 @@ import org.apache.kafka.common.errors.DisconnectException; import org.apache.kafka.common.errors.GroupAuthorizationException; import org.apache.kafka.common.errors.OffsetMetadataTooLarge; +import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.internals.TopicConstants; import org.apache.kafka.common.metrics.Metrics; import org.apache.kafka.common.protocol.Errors; @@ -322,6 +323,45 @@ public boolean matches(ClientRequest request) { assertEquals(Collections.singleton(tp), rebalanceListener.assigned); } + @Test + public void testWakeupDuringJoin() { + final String consumerId = "leader"; + + subscriptions.subscribe(Arrays.asList(topicName), rebalanceListener); + subscriptions.needReassignment(); + + // ensure metadata is up-to-date for leader + metadata.setTopics(Arrays.asList(topicName)); + metadata.update(cluster, time.milliseconds()); + + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + Map> memberSubscriptions = Collections.singletonMap(consumerId, Arrays.asList(topicName)); + partitionAssignor.prepare(Collections.singletonMap(consumerId, Arrays.asList(tp))); + + // prepare only the first half of the join and then trigger the wakeup + client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE.code())); + consumerClient.wakeup(); + + try { + coordinator.ensurePartitionAssignment(); + } catch (WakeupException e) { + // ignore + } + + // now complete the second half + client.prepareResponse(syncGroupResponse(Arrays.asList(tp), Errors.NONE.code())); + coordinator.ensurePartitionAssignment(); + + assertFalse(subscriptions.partitionAssignmentNeeded()); + assertEquals(Collections.singleton(tp), subscriptions.assignedPartitions()); + assertEquals(1, rebalanceListener.revokedCount); + assertEquals(Collections.emptySet(), rebalanceListener.revoked); + assertEquals(1, rebalanceListener.assignedCount); + assertEquals(Collections.singleton(tp), rebalanceListener.assigned); + } + @Test public void testNormalJoinGroupFollower() { final String consumerId = "consumer"; From 8c59565761a42984335294683c3501df8427ce62 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Fri, 8 Apr 2016 13:30:46 -0700 Subject: [PATCH 131/206] KAFKA-3521: validate null keys in Streams DSL implementations Author: Guozhang Wang Reviewers: Ewen Cheslack-Postava Closes #1197 from guozhangwang/K3521 --- .../kstream/internals/ChangedSerializer.java | 16 ++++++++++++++- .../kstream/internals/KStreamJoinWindow.java | 8 ++++++-- .../kstream/internals/KStreamKStreamJoin.java | 5 +++++ .../internals/KStreamKTableLeftJoin.java | 6 +++++- .../kstream/internals/KStreamTransform.java | 10 +++++----- .../internals/KStreamWindowAggregate.java | 5 +++++ .../internals/KStreamWindowReduce.java | 5 +++++ .../kstream/internals/KTableKTableJoin.java | 5 +++++ .../internals/KTableKTableLeftJoin.java | 5 +++++ .../internals/KTableKTableOuterJoin.java | 5 +++++ .../internals/KTableKTableRightJoin.java | 5 +++++ .../internals/KTableRepartitionMap.java | 5 +++++ .../internals/StandbyContextImpl.java | 20 +++++++++---------- 13 files changed, 81 insertions(+), 19 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java index e9b7cada80d88..5ea0791e33fa9 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java @@ -18,6 +18,7 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.errors.StreamsException; import java.nio.ByteBuffer; import java.util.Map; @@ -39,8 +40,21 @@ public void configure(Map configs, boolean isKey) { @Override public byte[] serialize(String topic, Change data) { + byte[] serializedKey; + // only one of the old / new values would be not null - byte[] serializedKey = inner.serialize(topic, data.newValue != null ? data.newValue : data.oldValue); + if (data.newValue != null) { + if (data.oldValue != null) + throw new StreamsException("Both old and new values are not null (" + data.oldValue + + " : " + data.newValue + ") in ChangeSerializer, which is not allowed."); + + serializedKey = inner.serialize(topic, data.newValue); + } else { + if (data.oldValue == null) + throw new StreamsException("Both old and new values are null in ChangeSerializer, which is not allowed."); + + serializedKey = inner.serialize(topic, data.oldValue); + } ByteBuffer buf = ByteBuffer.allocate(serializedKey.length + NEWFLAG_SIZE); buf.put(serializedKey); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java index 5b83b282dc89c..94e0b88f3fe0f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java @@ -55,8 +55,12 @@ public void init(ProcessorContext context) { @Override public void process(K key, V value) { - context().forward(key, value); - window.put(key, value); + // if the key is null, we do not need to put the record into window store + // since it will never be considered for join operations + if (key != null) { + context().forward(key, value); + window.put(key, value); + } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java index a4ac9b32f1bcf..d8caf3caa1cf7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java @@ -18,6 +18,7 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; @@ -63,6 +64,10 @@ public void init(ProcessorContext context) { @Override public void process(K key, V1 value) { + // the keys should never be null + if (key == null) + throw new StreamsException("Record key for KStream-KStream join operator with other window state store " + otherWindowName + " should not be null."); + boolean needOuterJoin = KStreamKStreamJoin.this.outer; long timeFrom = Math.max(0L, context().timestamp() - joinBeforeMs); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoin.java index dfca019f51dbe..92b9b59f94518 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoin.java @@ -55,7 +55,11 @@ public void init(ProcessorContext context) { @Override public void process(K key, V1 value) { - context().forward(key, joiner.apply(value, valueGetter.get(key))); + // if the key is null, we do not need proceed joining + // the record with the table + if (key != null) { + context().forward(key, joiner.apply(value, valueGetter.get(key))); + } } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransform.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransform.java index 4299c66d11e93..09dddfea91878 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransform.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransform.java @@ -20,6 +20,7 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.Transformer; import org.apache.kafka.streams.kstream.TransformerSupplier; +import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.ProcessorSupplier; @@ -34,13 +35,12 @@ public KStreamTransform(TransformerSupplier> transformerS @Override public Processor get() { - return new KStreamTransformProcessor(transformerSupplier.get()); + return new KStreamTransformProcessor<>(transformerSupplier.get()); } - public static class KStreamTransformProcessor implements Processor { + public static class KStreamTransformProcessor extends AbstractProcessor { private final Transformer> transformer; - private ProcessorContext context; public KStreamTransformProcessor(Transformer> transformer) { this.transformer = transformer; @@ -48,14 +48,14 @@ public KStreamTransformProcessor(Transformer> transform @Override public void init(ProcessorContext context) { + super.init(context); transformer.init(context); - this.context = context; } @Override public void process(K1 key, V1 value) { KeyValue pair = transformer.transform(key, value); - context.forward(pair.key, pair.value); + context().forward(pair.key, pair.value); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java index 76964f963b9dc..f36cc8c8558fd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java @@ -72,6 +72,11 @@ public void init(ProcessorContext context) { @Override public void process(K key, V value) { + // if the key is null, we do not need proceed aggregating the record + // the record with the table + if (key == null) + return; + // first get the matching windows long timestamp = context().timestamp(); Map matchedWindows = windows.windowsFor(timestamp); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduce.java index d532e7945cafd..6c05ce30008eb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduce.java @@ -69,6 +69,11 @@ public void init(ProcessorContext context) { @Override public void process(K key, V value) { + // if the key is null, we do not need proceed aggregating the record + // the record with the table + if (key == null) + return; + // first get the matching windows long timestamp = context().timestamp(); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java index 6eb27b648139a..24c8da6307093 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.kstream.internals; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; @@ -61,6 +62,10 @@ public void init(ProcessorContext context) { @Override public void process(K key, Change change) { + // the keys should never be null + if (key == null) + throw new StreamsException("Record key for KTable join operator should not be null."); + R newValue = null; R oldValue = null; V2 value2 = null; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java index 00e872eee5666..4bf45ed068039 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.kstream.internals; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; @@ -61,6 +62,10 @@ public void init(ProcessorContext context) { @Override public void process(K key, Change change) { + // the keys should never be null + if (key == null) + throw new StreamsException("Record key for KTable left-join operator should not be null."); + R newValue = null; R oldValue = null; V2 value2 = null; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java index 6ab0ae91cfbcb..49eed539d00f2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.kstream.internals; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; @@ -61,6 +62,10 @@ public void init(ProcessorContext context) { @Override public void process(K key, Change change) { + // the keys should never be null + if (key == null) + throw new StreamsException("Record key for KTable outer-join operator should not be null."); + R newValue = null; R oldValue = null; V2 value2 = valueGetter.get(key); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java index a6a13fc4b8ccb..7443d4a623151 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java @@ -17,6 +17,7 @@ package org.apache.kafka.streams.kstream.internals; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; @@ -62,6 +63,10 @@ public void init(ProcessorContext context) { @Override public void process(K key, Change change) { + // the keys should never be null + if (key == null) + throw new StreamsException("Record key for KTable right-join operator should not be null."); + R newValue = null; R oldValue = null; V2 value2 = valueGetter.get(key); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java index ff69c3781b16c..142a2796c2834 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java @@ -18,6 +18,7 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.errors.StreamsException; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.processor.AbstractProcessor; import org.apache.kafka.streams.processor.Processor; @@ -77,6 +78,10 @@ private class KTableMapProcessor extends AbstractProcessor> { public void process(K key, Change change) { KeyValue newPair = computeValue(key, change.newValue); + // the selected repartition key should never be null + if (newPair.key == null) + throw new StreamsException("Record key for KTable repartition operator should not be null."); + context().forward(newPair.key, new Change<>(newPair.value, null)); if (change.oldValue != null) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java index ea008b827e554..468fe741448ae 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java @@ -108,51 +108,51 @@ public void register(StateStore store, boolean loggingEnabled, StateRestoreCallb @Override public StateStore getStateStore(String name) { - throw new UnsupportedOperationException("getStateStore() not supported."); + throw new UnsupportedOperationException("this should not happen: getStateStore() not supported in standby tasks."); } @Override public String topic() { - throw new UnsupportedOperationException("topic() not supported."); + throw new UnsupportedOperationException("this should not happen: topic() not supported in standby tasks."); } @Override public int partition() { - throw new UnsupportedOperationException("partition() not supported."); + throw new UnsupportedOperationException("this should not happen: partition() not supported in standby tasks."); } @Override public long offset() { - throw new UnsupportedOperationException("offset() not supported."); + throw new UnsupportedOperationException("this should not happen: offset() not supported in standby tasks."); } @Override public long timestamp() { - throw new UnsupportedOperationException("timestamp() not supported."); + throw new UnsupportedOperationException("this should not happen: timestamp() not supported in standby tasks."); } @Override public void forward(K key, V value) { - throw new UnsupportedOperationException("forward() not supported."); + throw new UnsupportedOperationException("this should not happen: forward() not supported in standby tasks."); } @Override public void forward(K key, V value, int childIndex) { - throw new UnsupportedOperationException("forward() not supported."); + throw new UnsupportedOperationException("this should not happen: forward() not supported in standby tasks."); } @Override public void forward(K key, V value, String childName) { - throw new UnsupportedOperationException(); + throw new UnsupportedOperationException("this should not happen: forward() not supported in standby tasks."); } @Override public void commit() { - throw new UnsupportedOperationException("commit() not supported."); + throw new UnsupportedOperationException("this should not happen: commit() not supported in standby tasks."); } @Override public void schedule(long interval) { - throw new UnsupportedOperationException("schedule() not supported."); + throw new UnsupportedOperationException("this should not happen: schedule() not supported in standby tasks."); } } From 7c27989860e7e063bfec3c679cb6bc8fd52abc84 Mon Sep 17 00:00:00 2001 From: bbejeck Date: Sun, 10 Apr 2016 17:43:47 -0700 Subject: [PATCH 132/206] KAFKA-3338: Add print and writeAsText to KStream/KTable in Kafka Streams Addresses comments from previous PR [#1187] Changed print and writeAsText method return signature to void Flush System.out on close Changed IllegalStateException to TopologyBuilderException Updated MockProcessorContext.topic method to return a String Renamed KStreamPrinter to KeyValuePrinter Updated the printing of null keys to 'null' to match ConsoleConsumer Updated JavaDoc stating need to override toString Author: bbejeck Reviewers: Dan Norwood, Guozhang Wang Closes #1209 from bbejeck/KAFKA-3338_Adding_print/writeAsText_to_Streams_DSL --- .../apache/kafka/streams/kstream/KStream.java | 46 +++++ .../apache/kafka/streams/kstream/KTable.java | 43 +++++ .../kstream/internals/KStreamImpl.java | 40 ++++- .../streams/kstream/internals/KTableImpl.java | 38 +++- .../kstream/internals/KeyValuePrinter.java | 124 +++++++++++++ .../KeyValuePrinterProcessorTest.java | 165 ++++++++++++++++++ .../kafka/test/MockProcessorContext.java | 2 +- 7 files changed, 454 insertions(+), 4 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinter.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinterProcessorTest.java diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index a55e7264eb2ac..27475aa18ad5c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -63,6 +63,52 @@ public interface KStream { */ KStream mapValues(ValueMapper mapper); + /** + * Print the elements of this stream to System.out + * + * Implementors will need to override toString for keys and values that are not of + * type String, Integer etc to get meaningful information. + */ + void print(); + + + /** + * Print the elements of this stream to System.out + * + * @param keySerde key serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * + * Implementors will need to override toString for keys and values that are not of + * type String, Integer etc to get meaningful information. + */ + void print(Serde keySerde, Serde valSerde); + + + /** + * Write the elements of this stream to a file at the given path. + * + * @param filePath name of file to write to + * + * Implementors will need to override toString for keys and values that are not of + * type String, Integer etc to get meaningful information. + */ + void writeAsText(String filePath); + + /** + * @param filePath name of file to write to + * @param keySerde key serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * + * Implementors will need to override toString for keys and values that are not of + * type String, Integer etc to get meaningful information. + */ + + void writeAsText(String filePath, Serde keySerde, Serde valSerde); + /** * Create a new instance of {@link KStream} by transforming each element in this stream into zero or more elements in the new stream. * diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index 1f6ee687c12c9..bb6878f3ae235 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -53,6 +53,49 @@ public interface KTable { */ KTable mapValues(ValueMapper mapper); + + /** + * Print the elements of this stream to System.out + * + * Implementors will need to override toString for keys and values that are not of + * type String, Integer etc to get meaningful information. + */ + void print(); + + /** + * Print the elements of this stream to System.out + * @param keySerde key serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * + * Implementors will need to override toString for keys and values that are not of + * type String, Integer etc to get meaningful information. + */ + void print(Serde keySerde, Serde valSerde); + + /** + * Write the elements of this stream to a file at the given path. + * @param filePath name of file to write to + * + * Implementors will need to override toString for keys and values that are not of + * type String, Integer etc to get meaningful information. + */ + void writeAsText(String filePath); + + /** + * + * @param filePath name of file to write to + * @param keySerde key serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * @param valSerde value serde used to send key-value pairs, + * if not specified the default serde defined in the configs will be used + * + * Implementors will need to override toString for keys and values that are not of + * type String, Integer etc to get meaningful information. + */ + void writeAsText(String filePath, Serde keySerde, Serde valSerde); + /** * Materialize this stream to a topic, also creates a new instance of {@link KTable} from the topic * using default serializers and deserializers and producer's {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner}. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index c266328429730..9707aeebbcb99 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -20,13 +20,14 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.errors.TopologyBuilderException; import org.apache.kafka.streams.kstream.Aggregator; -import org.apache.kafka.streams.kstream.ForeachAction; import org.apache.kafka.streams.kstream.Initializer; import org.apache.kafka.streams.kstream.JoinWindows; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.ForeachAction; import org.apache.kafka.streams.kstream.Reducer; import org.apache.kafka.streams.kstream.TransformerSupplier; import org.apache.kafka.streams.kstream.ValueJoiner; @@ -42,7 +43,9 @@ import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.state.Stores; - +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.PrintStream; import java.lang.reflect.Array; import java.util.HashSet; import java.util.Set; @@ -79,6 +82,8 @@ public class KStreamImpl extends AbstractStream implements KStream KStream mapValues(ValueMapper mapper) { return new KStreamImpl<>(topology, name, sourceNodes); } + @Override + public void print() { + print(null, null); + } + + @Override + public void print(Serde keySerde, Serde valSerde) { + String name = topology.newName(PRINTING_NAME); + topology.addProcessor(name, new KeyValuePrinter<>(keySerde, valSerde), this.name); + } + + + @Override + public void writeAsText(String filePath) { + writeAsText(filePath, null, null); + } + + @Override + public void writeAsText(String filePath, Serde keySerde, Serde valSerde) { + String name = topology.newName(PRINTING_NAME); + try { + + PrintStream printStream = new PrintStream(new FileOutputStream(filePath)); + topology.addProcessor(name, new KeyValuePrinter<>(printStream, keySerde, valSerde), this.name); + + } catch (FileNotFoundException e) { + String message = "Unable to write stream to file at [" + filePath + "] " + e.getMessage(); + throw new TopologyBuilderException(message); + } + } + @Override public KStream flatMap(KeyValueMapper>> mapper) { String name = topology.newName(FLATMAP_NAME); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index 8de9a0bbb31fc..adc8b91203294 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -19,13 +19,14 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.errors.TopologyBuilderException; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.ForeachAction; import org.apache.kafka.streams.kstream.Initializer; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; -import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Predicate; import org.apache.kafka.streams.kstream.Reducer; @@ -36,6 +37,9 @@ import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.state.Stores; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.PrintStream; import java.util.Collections; import java.util.Set; @@ -69,6 +73,8 @@ public class KTableImpl extends AbstractStream implements KTable KTable mapValues(ValueMapper mapper) { return new KTableImpl<>(topology, name, processorSupplier, sourceNodes); } + @Override + public void print() { + print(null, null); + } + + @Override + public void print(Serde keySerde, Serde valSerde) { + String name = topology.newName(PRINTING_NAME); + topology.addProcessor(name, new KeyValuePrinter<>(keySerde, valSerde), this.name); + } + + + @Override + public void writeAsText(String filePath) { + writeAsText(filePath, null, null); + } + + @Override + public void writeAsText(String filePath, Serde keySerde, Serde valSerde) { + String name = topology.newName(PRINTING_NAME); + try { + PrintStream printStream = new PrintStream(new FileOutputStream(filePath)); + topology.addProcessor(name, new KeyValuePrinter<>(printStream, keySerde, valSerde), this.name); + } catch (FileNotFoundException e) { + String message = "Unable to write stream to file at [" + filePath + "] " + e.getMessage(); + throw new TopologyBuilderException(message); + } + } + + @Override public KTable through(Serde keySerde, Serde valSerde, diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinter.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinter.java new file mode 100644 index 0000000000000..d1c1d8b2e0334 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinter.java @@ -0,0 +1,124 @@ +/** + * 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 org.apache.kafka.streams.kstream.internals; + +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.streams.processor.AbstractProcessor; +import org.apache.kafka.streams.processor.Processor; +import org.apache.kafka.streams.processor.ProcessorContext; +import org.apache.kafka.streams.processor.ProcessorSupplier; + +import java.io.PrintStream; + + +class KeyValuePrinter implements ProcessorSupplier { + + private final PrintStream printStream; + private Serde keySerde; + private Serde valueSerde; + private boolean notStandardOut; + + + KeyValuePrinter(PrintStream printStream, Serde keySerde, Serde valueSerde) { + this.keySerde = keySerde; + this.valueSerde = valueSerde; + if (printStream == null) { + this.printStream = System.out; + } else { + this.printStream = printStream; + notStandardOut = true; + } + } + + KeyValuePrinter(PrintStream printStream) { + this(printStream, null, null); + } + + KeyValuePrinter(Serde keySerde, Serde valueSerde) { + this(System.out, keySerde, valueSerde); + } + + KeyValuePrinter() { + this(System.out, null, null); + } + + @Override + public Processor get() { + return new KeyValuePrinterProcessor(this.printStream, this.keySerde, this.valueSerde); + } + + + private class KeyValuePrinterProcessor extends AbstractProcessor { + private final PrintStream printStream; + private Serde keySerde; + private Serde valueSerde; + private ProcessorContext processorContext; + + private KeyValuePrinterProcessor(PrintStream printStream, Serde keySerde, Serde valueSerde) { + this.printStream = printStream; + this.keySerde = keySerde; + this.valueSerde = valueSerde; + } + + @Override + public void init(ProcessorContext context) { + this.processorContext = context; + + if (this.keySerde == null) { + keySerde = this.processorContext.keySerde(); + } + + if (this.valueSerde == null) { + valueSerde = this.processorContext.valueSerde(); + } + } + + @Override + public void process(K key, V value) { + K keyToPrint = (K) maybeDeserialize(key, keySerde.deserializer()); + V valueToPrint = (V) maybeDeserialize(value, valueSerde.deserializer()); + + printStream.println(keyToPrint + " , " + valueToPrint); + + this.processorContext.forward(key, value); + } + + + private Object maybeDeserialize(Object receivedElement, Deserializer deserializer) { + if (receivedElement == null) { + return null; + } + + if (receivedElement instanceof byte[]) { + return deserializer.deserialize(this.processorContext.topic(), (byte[]) receivedElement); + } + + return receivedElement; + } + + @Override + public void close() { + if (notStandardOut) { + this.printStream.close(); + } else { + this.printStream.flush(); + } + } + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinterProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinterProcessorTest.java new file mode 100644 index 0000000000000..22948abcc6f41 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinterProcessorTest.java @@ -0,0 +1,165 @@ +/** + * 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 org.apache.kafka.streams.kstream.internals; + + +import org.apache.kafka.common.errors.SerializationException; +import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.test.KStreamTestDriver; +import org.junit.Test; + +import java.io.ByteArrayOutputStream; +import java.io.PrintStream; +import java.nio.charset.Charset; +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class KeyValuePrinterProcessorTest { + + private String topicName = "topic"; + private Serde stringSerde = Serdes.String(); + private Serde bytesSerde = Serdes.ByteArray(); + private ByteArrayOutputStream baos = new ByteArrayOutputStream(); + private KStreamBuilder builder = new KStreamBuilder(); + private PrintStream printStream = new PrintStream(baos); + + + @Test + public void testPrintKeyValueDefaultSerde() throws Exception { + + KeyValuePrinter keyValuePrinter = new KeyValuePrinter<>(printStream); + String[] suppliedKeys = {"foo", "bar", null}; + String[] suppliedValues = {"value1", "value2", "value3"}; + String[] expectedValues = {"foo , value1", "bar , value2", "null , value3"}; + + + KStream stream = builder.stream(stringSerde, stringSerde, topicName); + stream.process(keyValuePrinter); + + KStreamTestDriver driver = new KStreamTestDriver(builder); + for (int i = 0; i < suppliedKeys.length; i++) { + driver.process(topicName, suppliedKeys[i], suppliedValues[i]); + } + + String[] capturedValues = new String(baos.toByteArray(), Charset.forName("UTF-8")).split("\n"); + + for (int i = 0; i < capturedValues.length; i++) { + assertEquals(capturedValues[i], expectedValues[i]); + } + } + + + @Test + public void testPrintKeyValueWithProvidedSerde() throws Exception { + + Serde mockObjectSerde = Serdes.serdeFrom(new MockSerializer(), new MockDeserializer()); + KeyValuePrinter keyValuePrinter = new KeyValuePrinter<>(printStream, stringSerde, mockObjectSerde); + KStream stream = builder.stream(stringSerde, mockObjectSerde, topicName); + + stream.process(keyValuePrinter); + + KStreamTestDriver driver = new KStreamTestDriver(builder); + + String suppliedKey = null; + byte[] suppliedValue = "{\"name\":\"print\", \"label\":\"test\"}".getBytes(Charset.forName("UTF-8")); + + driver.process(topicName, suppliedKey, suppliedValue); + String expectedPrintedValue = "null , name:print label:test"; + String capturedValue = new String(baos.toByteArray(), Charset.forName("UTF-8")).trim(); + + assertEquals(capturedValue, expectedPrintedValue); + + } + + private static class MockObject { + public String name; + public String label; + + public MockObject() { + } + + MockObject(String name, String label) { + this.name = name; + this.label = label; + } + + @Override + public String toString() { + return "name:" + name + " label:" + label; + } + } + + + private static class MockDeserializer implements Deserializer { + + private com.fasterxml.jackson.databind.ObjectMapper objectMapper = new com.fasterxml.jackson.databind.ObjectMapper(); + + @Override + public void configure(Map configs, boolean isKey) { + + } + + @Override + public MockObject deserialize(String topic, byte[] data) { + MockObject mockObject; + try { + mockObject = objectMapper.readValue(data, MockObject.class); + } catch (Exception e) { + throw new SerializationException(e); + } + return mockObject; + } + + @Override + public void close() { + + } + } + + + private static class MockSerializer implements Serializer { + private final com.fasterxml.jackson.databind.ObjectMapper objectMapper = new com.fasterxml.jackson.databind.ObjectMapper(); + + @Override + public void configure(Map configs, boolean isKey) { + + } + + @Override + public byte[] serialize(String topic, MockObject data) { + try { + return objectMapper.writeValueAsBytes(data); + } catch (Exception e) { + throw new SerializationException("Error serializing JSON message", e); + } + } + + @Override + public void close() { + + } + } + + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java index 287af5a61b191..1d478ddf9fd89 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -171,7 +171,7 @@ public void commit() { @Override public String topic() { - throw new UnsupportedOperationException("topic() not supported."); + return "mockTopic"; } @Override From c76b6e6d9bad2278076054f5175a2b053383388f Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Sun, 10 Apr 2016 23:09:43 -0700 Subject: [PATCH 133/206] HOTFIX: special handling first ever triggered punctuate Author: Guozhang Wang Reviewers: Anna Povzner Closes #1208 from guozhangwang/KPunctuate --- .../processor/internals/PunctuationQueue.java | 2 +- .../processor/internals/PunctuationSchedule.java | 13 +++++++++---- .../streams/processor/internals/StreamTaskTest.java | 10 +++++----- 3 files changed, 15 insertions(+), 10 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java index d7d7eee90ef78..824e20a295f63 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationQueue.java @@ -43,7 +43,7 @@ public boolean mayPunctuate(long timestamp, Punctuator punctuator) { PunctuationSchedule sched = top; pq.poll(); punctuator.punctuate(sched.node(), timestamp); - pq.add(sched.next()); + pq.add(sched.next(timestamp)); punctuated = true; top = pq.peek(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java index 758cfb001e035..98919d2716832 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PunctuationSchedule.java @@ -22,11 +22,11 @@ public class PunctuationSchedule extends Stamped { final long interval; public PunctuationSchedule(ProcessorNode node, long interval) { - this(node, 0, interval); + this(node, 0L, interval); } public PunctuationSchedule(ProcessorNode node, long time, long interval) { - super(node, time + interval); + super(node, time); this.interval = interval; } @@ -34,8 +34,13 @@ public ProcessorNode node() { return value; } - public PunctuationSchedule next() { - return new PunctuationSchedule(value, timestamp, interval); + public PunctuationSchedule next(long currTimestamp) { + // we need to special handle the case when it is firstly triggered (i.e. the timestamp + // is equal to the interval) by reschedule based on the currTimestamp + if (timestamp == 0L) + return new PunctuationSchedule(value, currTimestamp + interval, interval); + else + return new PunctuationSchedule(value, timestamp + interval, interval); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java index dd4894744010a..6014c36ae5eb9 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamTaskTest.java @@ -226,15 +226,15 @@ public void testMaybePunctuate() throws Exception { StreamTask task = new StreamTask(new TaskId(0, 0), "applicationId", partitions, topology, consumer, producer, restoreStateConsumer, config, null); task.addRecords(partition1, records( - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 10, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), new ConsumerRecord<>(partition1.topic(), partition1.partition(), 20, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), - new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue) + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 30, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), + new ConsumerRecord<>(partition1.topic(), partition1.partition(), 40, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue) )); task.addRecords(partition2, records( - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 15, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), new ConsumerRecord<>(partition2.topic(), partition2.partition(), 25, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), - new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue) + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 35, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue), + new ConsumerRecord<>(partition2.topic(), partition2.partition(), 45, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, recordKey, recordValue) )); assertTrue(task.maybePunctuate()); @@ -275,7 +275,7 @@ public void testMaybePunctuate() throws Exception { assertFalse(task.maybePunctuate()); - processor.supplier.checkAndClearPunctuateResult(10L, 20L, 30L); + processor.supplier.checkAndClearPunctuateResult(20L, 30L, 40L); task.close(); From 411b04a3c933b5253f34342bf24f63a9473f9ab5 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Mon, 11 Apr 2016 10:37:04 -0700 Subject: [PATCH 134/206] MINOR: Make VerifiableProducer in system tests lookup security configuration dynamically instead of at construction. Author: Ewen Cheslack-Postava Reviewers: Anna Povzner Closes #1207 from ewencp/minor-dynamic-security-config-verifiable-producer --- tests/kafkatest/services/verifiable_producer.py | 7 +++++-- 1 file changed, 5 insertions(+), 2 deletions(-) diff --git a/tests/kafkatest/services/verifiable_producer.py b/tests/kafkatest/services/verifiable_producer.py index 0096a34875e50..414da84ad9e64 100644 --- a/tests/kafkatest/services/verifiable_producer.py +++ b/tests/kafkatest/services/verifiable_producer.py @@ -66,14 +66,17 @@ def __init__(self, context, num_nodes, kafka, topic, max_messages=-1, throughput if self.compression_types is not None: assert len(self.compression_types) == num_nodes, "Specify one compression type per node" - self.security_config = self.kafka.security_config.client_config() - for node in self.nodes: node.version = version self.acked_values = [] self.not_acked_values = [] self.produced_count = {} + + @property + def security_config(self): + return self.kafka.security_config.client_config() + def prop_file(self, node): idx = self.idx(node) prop_file = str(self.security_config) From 1ec842a3ea9bbe414a5c59f90569fbc3348bdfb8 Mon Sep 17 00:00:00 2001 From: Igor Stepanov Date: Mon, 11 Apr 2016 12:02:49 -0700 Subject: [PATCH 135/206] MINOR: fix incorrect exception message in KafkaProducer While playing with client got the next exception: ```java java.lang.IllegalArgumentException: Invalid partition given with record: 1 is not in the range [0...1]. ``` It's obviously incorrect, so I've fixed it. Author: Igor Stepanov Reviewers: Guozhang Wang Closes #1210 from stepio/trunk --- .../apache/kafka/clients/producer/KafkaProducer.java | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 6acc0599e8518..d60e28e9c91eb 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -711,13 +711,11 @@ private int partition(ProducerRecord record, byte[] serializedKey , byte[] Integer partition = record.partition(); if (partition != null) { List partitions = cluster.partitionsForTopic(record.topic()); - int numPartitions = partitions.size(); + int lastPartition = partitions.size() - 1; // they have given us a partition, use it - if (partition < 0 || partition >= numPartitions) - throw new IllegalArgumentException("Invalid partition given with record: " + partition - + " is not in the range [0..." - + numPartitions - + "]."); + if (partition < 0 || partition > lastPartition) { + throw new IllegalArgumentException(String.format("Invalid partition given with record: %d is not in the range [0...%d].", partition, lastPartition)); + } return partition; } return this.partitioner.partition(record.topic(), record.key(), serializedKey, record.value(), serializedValue, From 40fd456649b5df29d030da46865b5e7e0ca6db15 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Mon, 11 Apr 2016 12:33:48 -0700 Subject: [PATCH 136/206] KAFKA-3519: Refactor Transformer's transform / punctuate to return nullable values Author: Guozhang Wang Reviewers: Dan Norwood, Anna Povzner Closes #1204 from guozhangwang/KTransformR --- .../apache/kafka/streams/kstream/Transformer.java | 9 +++++---- .../kafka/streams/kstream/ValueTransformer.java | 5 +++-- .../streams/kstream/internals/KStreamImpl.java | 4 +--- .../kstream/internals/KStreamTransform.java | 9 +++++++-- .../kstream/internals/KStreamTransformValues.java | 11 +++++++---- .../streams/kstream/internals/KTableImpl.java | 2 +- .../kstream/internals/KStreamTransformTest.java | 10 +++++++--- .../internals/KStreamTransformValuesTest.java | 8 ++++++-- .../org/apache/kafka/test/KStreamTestDriver.java | 15 +++++++++++++++ .../apache/kafka/test/MockProcessorContext.java | 6 +++--- .../apache/kafka/test/MockProcessorSupplier.java | 3 ++- 11 files changed, 57 insertions(+), 25 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java index 8069dca60602e..5197e94769f8f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java @@ -32,7 +32,7 @@ public interface Transformer { * Initialize this transformer with the given context. The framework ensures this is called once per processor when the topology * that contains it is initialized. *

      - * If this tranformer is to be {@link #punctuate(long) called periodically} by the framework, then this method should + * If this transformer is to be {@link #punctuate(long) called periodically} by the framework, then this method should * {@link ProcessorContext#schedule(long) schedule itself} with the provided context. * * @param context the context; may not be null @@ -44,17 +44,18 @@ public interface Transformer { * * @param key the key for the message * @param value the value for the message - * @return new value + * @return new value; if null no key-value pair will be forwarded to down stream */ R transform(K key, V value); /** - * Perform any periodic operations, if this processor {@link ProcessorContext#schedule(long) schedule itself} with the context + * Perform any periodic operations and possibly generate a key, if this processor {@link ProcessorContext#schedule(long) schedule itself} with the context * during {@link #init(ProcessorContext) initialization}. * * @param timestamp the stream time when this method is being called + * @return new value; if null it will not be forwarded to down stream */ - void punctuate(long timestamp); + R punctuate(long timestamp); /** * Close this processor and clean up any resources. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java index 1a0679d681595..63214fde2238b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java @@ -47,12 +47,13 @@ public interface ValueTransformer { R transform(V value); /** - * Perform any periodic operations, if this processor {@link ProcessorContext#schedule(long) schedule itself} with the context + * Perform any periodic operations and possibly return a new value, if this processor {@link ProcessorContext#schedule(long) schedule itself} with the context * during {@link #init(ProcessorContext) initialization}. * * @param timestamp the stream time when this method is being called + * @return new value; if null it will not be forwarded to down stream */ - void punctuate(long timestamp); + R punctuate(long timestamp); /** * Close this processor and clean up any resources. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 9707aeebbcb99..a02cfb4562465 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -86,8 +86,6 @@ public class KStreamImpl extends AbstractStream implements KStream through(Serde keySerde, Serde valSerde, StreamPartiti public void foreach(ForeachAction action) { String name = topology.newName(FOREACH_NAME); - topology.addProcessor(name, new KStreamForeach(action), this.name); + topology.addProcessor(name, new KStreamForeach<>(action), this.name); } public KStream through(Serde keySerde, Serde valSerde, String topic) { return through(keySerde, valSerde, null, topic); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransform.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransform.java index 09dddfea91878..af100a2e04348 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransform.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransform.java @@ -55,12 +55,17 @@ public void init(ProcessorContext context) { @Override public void process(K1 key, V1 value) { KeyValue pair = transformer.transform(key, value); - context().forward(pair.key, pair.value); + + if (pair != null) + context().forward(pair.key, pair.value); } @Override public void punctuate(long timestamp) { - transformer.punctuate(timestamp); + KeyValue pair = transformer.punctuate(timestamp); + + if (pair != null) + context().forward(pair.key, pair.value); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java index 6f989e6833cfc..cb9aab10caa99 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValues.java @@ -27,18 +27,18 @@ public class KStreamTransformValues implements ProcessorSupplier private final ValueTransformerSupplier valueTransformerSupplier; - public KStreamTransformValues(ValueTransformerSupplier valueTransformerSupplier) { + public KStreamTransformValues(ValueTransformerSupplier valueTransformerSupplier) { this.valueTransformerSupplier = valueTransformerSupplier; } @Override public Processor get() { - return new KStreamTransformValuesProcessor(valueTransformerSupplier.get()); + return new KStreamTransformValuesProcessor<>(valueTransformerSupplier.get()); } public static class KStreamTransformValuesProcessor implements Processor { - private final ValueTransformer valueTransformer; + private final ValueTransformer valueTransformer; private ProcessorContext context; public KStreamTransformValuesProcessor(ValueTransformer valueTransformer) { @@ -58,7 +58,10 @@ public void process(K key, V value) { @Override public void punctuate(long timestamp) { - valueTransformer.punctuate(timestamp); + R ret = valueTransformer.punctuate(timestamp); + + if (ret != null) + context.forward(null, ret); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index adc8b91203294..ee2c9313e53d6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -183,7 +183,7 @@ public KTable through(Serde keySerde, @Override public void foreach(final ForeachAction action) { String name = topology.newName(FOREACH_NAME); - KStreamForeach> processorSupplier = new KStreamForeach(new ForeachAction>() { + KStreamForeach> processorSupplier = new KStreamForeach<>(new ForeachAction>() { @Override public void apply(K key, Change value) { action.apply(key, value.newValue); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java index 4244de541640c..a0a61f2d63b1f 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java @@ -59,7 +59,8 @@ public KeyValue transform(Integer key, Integer value) { } @Override - public void punctuate(long timestamp) { + public KeyValue punctuate(long timestamp) { + return KeyValue.pair(-1, (int) timestamp); } @Override @@ -80,9 +81,12 @@ public void close() { driver.process(topicName, expectedKeys[i], expectedKeys[i] * 10); } - assertEquals(4, processor.processed.size()); + driver.punctuate(2); + driver.punctuate(3); - String[] expected = {"2:10", "20:110", "200:1110", "2000:11110"}; + assertEquals(6, processor.processed.size()); + + String[] expected = {"2:10", "20:110", "200:1110", "2000:11110", "-1:2", "-1:3"}; for (int i = 0; i < expected.length; i++) { assertEquals(expected[i], processor.processed.get(i)); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java index 52abdf728c93e..f5f9698baac43 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java @@ -58,7 +58,8 @@ public Integer transform(Integer value) { } @Override - public void punctuate(long timestamp) { + public Integer punctuate(long timestamp) { + return (int) timestamp; } @Override @@ -82,7 +83,10 @@ public void close() { assertEquals(4, processor.processed.size()); - String[] expected = {"1:10", "10:110", "100:1110", "1000:11110"}; + driver.punctuate(2); + driver.punctuate(3); + + String[] expected = {"1:10", "10:110", "100:1110", "1000:11110", "null:2", "null:3"}; for (int i = 0; i < expected.length; i++) { assertEquals(expected[i], processor.processed.get(i)); diff --git a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java index 5cfee6b581062..2ee87303ee7ce 100644 --- a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java @@ -87,6 +87,21 @@ public void process(String topicName, Object key, Object value) { } } + public void punctuate(long timestamp) { + setTime(timestamp); + + for (ProcessorNode processor : topology.processors()) { + if (processor.processor() != null) { + currNode = processor; + try { + processor.processor().punctuate(timestamp); + } finally { + currNode = null; + } + } + } + } + public void setTime(long timestamp) { context.setTime(timestamp); } diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java index 1d478ddf9fd89..2e2c221fc7bb3 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorContext.java @@ -171,17 +171,17 @@ public void commit() { @Override public String topic() { - return "mockTopic"; + return null; } @Override public int partition() { - throw new UnsupportedOperationException("partition() not supported."); + return -1; } @Override public long offset() { - throw new UnsupportedOperationException("offset() not supported."); + return -1L; } @Override diff --git a/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java b/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java index 921c365cad033..9cf0eb264d83d 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java +++ b/streams/src/test/java/org/apache/kafka/test/MockProcessorSupplier.java @@ -57,7 +57,8 @@ public void init(ProcessorContext context) { @Override public void process(K key, V value) { - processed.add(key + ":" + value); + processed.add((key == null ? "null" : key) + ":" + + (value == null ? "null" : value)); } @Override From 319c6e7195143c1fa4a1e5b34ad923c2b73f373d Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Tue, 12 Apr 2016 11:34:46 -0700 Subject: [PATCH 137/206] MINOR: Add missing `@Override` to `KStreamImpl.through` Author: Ismael Juma Reviewers: Guozhang Wang Closes #1216 from ijuma/add-missing-override-to-through --- .../org/apache/kafka/streams/kstream/internals/KStreamImpl.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index a02cfb4562465..97a7aac411bab 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -243,6 +243,8 @@ public void foreach(ForeachAction action) { topology.addProcessor(name, new KStreamForeach<>(action), this.name); } + + @Override public KStream through(Serde keySerde, Serde valSerde, String topic) { return through(keySerde, valSerde, null, topic); } From ba9456de2ebf1a34bdf5f6f62a701875822e1923 Mon Sep 17 00:00:00 2001 From: Eno Thereska Date: Tue, 12 Apr 2016 13:00:54 -0700 Subject: [PATCH 138/206] KAFKA-3439; Added exceptions thrown Author: Eno Thereska Reviewers: Guozhang Wang Closes #1213 from enothereska/KAFKA-3439-throws --- .../apache/kafka/streams/KafkaStreams.java | 2 ++ .../kstream/internals/AbstractStream.java | 3 ++ .../kstream/internals/ChangedSerializer.java | 4 +++ .../kstream/internals/KStreamAggregate.java | 3 ++ .../kstream/internals/KStreamImpl.java | 3 ++ .../kstream/internals/KStreamJoinWindow.java | 3 ++ .../kstream/internals/KStreamKStreamJoin.java | 3 ++ .../kstream/internals/KStreamReduce.java | 3 ++ .../kstream/internals/KTableAggregate.java | 3 ++ .../streams/kstream/internals/KTableImpl.java | 3 ++ .../kstream/internals/KTableKTableJoin.java | 3 ++ .../internals/KTableKTableLeftJoin.java | 3 ++ .../internals/KTableKTableOuterJoin.java | 3 ++ .../internals/KTableKTableRightJoin.java | 3 ++ .../kstream/internals/KTableReduce.java | 3 ++ .../internals/KTableRepartitionMap.java | 6 ++++ .../processor/DefaultPartitionGrouper.java | 3 ++ .../kafka/streams/processor/TaskId.java | 9 +++++ .../streams/processor/TopologyBuilder.java | 5 +++ .../processor/internals/AbstractTask.java | 6 ++++ .../internals/InternalTopicManager.java | 6 ++++ .../internals/MinTimestampTracker.java | 3 ++ .../processor/internals/PartitionGroup.java | 3 ++ .../internals/ProcessorContextImpl.java | 18 ++++++++++ .../internals/ProcessorStateManager.java | 14 ++++++++ .../processor/internals/QuickUnion.java | 3 ++ .../streams/processor/internals/SinkNode.java | 3 ++ .../internals/StandbyContextImpl.java | 33 +++++++++++++++++++ .../internals/StreamPartitionAssignor.java | 4 +++ .../processor/internals/StreamTask.java | 7 ++++ .../processor/internals/StreamThread.java | 5 +++ .../internals/assignment/AssignmentInfo.java | 7 ++++ .../assignment/SubscriptionInfo.java | 6 ++++ .../kafka/streams/state/StateSerdes.java | 1 + .../apache/kafka/streams/state/Stores.java | 4 +++ .../state/internals/MemoryLRUCache.java | 6 ++++ .../state/internals/OffsetCheckpoint.java | 23 +++++++++++++ .../streams/state/internals/RocksDBStore.java | 9 +++++ .../state/internals/RocksDBWindowStore.java | 3 ++ 39 files changed, 232 insertions(+) diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index e8fda103fdc61..4d1306d228b74 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -138,6 +138,7 @@ public KafkaStreams(TopologyBuilder builder, StreamsConfig config) { /** * Start the stream instance by starting all its threads. + * @throws IllegalStateException if process was already started */ public synchronized void start() { log.debug("Starting Kafka Stream process"); @@ -157,6 +158,7 @@ public synchronized void start() { /** * Shutdown this stream instance by signaling all the threads to stop, * and then wait for them to join. + * @throws IllegalStateException if process has not started yet */ public synchronized void close() { log.debug("Stopping Kafka Stream process"); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java index c5374650ae89e..ebada92839625 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/AbstractStream.java @@ -36,6 +36,9 @@ public AbstractStream(KStreamBuilder topology, String name, Set sourceNo this.sourceNodes = sourceNodes; } + /** + * @throws TopologyBuilderException if the streams are not joinable + */ protected Set ensureJoinableWith(AbstractStream other) { Set thisSourceNodes = sourceNodes; Set otherSourceNodes = other.sourceNodes; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java index 5ea0791e33fa9..5dbbac9812f9a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java @@ -38,6 +38,10 @@ public void configure(Map configs, boolean isKey) { // do nothing } + /** + * @throws StreamsException if both old and new values of data are null, or if + * both values are not null + */ @Override public byte[] serialize(String topic, Change data) { byte[] serializedKey; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java index 871a12d42dfb4..b6d14927496bb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamAggregate.java @@ -61,6 +61,9 @@ public void init(ProcessorContext context) { store = (KeyValueStore) context.getStateStore(storeName); } + /** + * @throws StreamsException if key is null + */ @Override public void process(K key, V value) { // the keys should never be null diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index 97a7aac411bab..70300214a5699 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -155,6 +155,9 @@ public void writeAsText(String filePath) { writeAsText(filePath, null, null); } + /** + * @throws TopologyBuilderException if file is not found + */ @Override public void writeAsText(String filePath, Serde keySerde, Serde valSerde) { String name = topology.newName(PRINTING_NAME); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java index 94e0b88f3fe0f..864dc9c26962b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamJoinWindow.java @@ -28,6 +28,9 @@ class KStreamJoinWindow implements ProcessorSupplier { private final String windowName; + /** + * @throws TopologyBuilderException if retention period of the join window is less than expected + */ KStreamJoinWindow(String windowName, long windowSizeMs, long retentionPeriodMs) { this.windowName = windowName; diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java index d8caf3caa1cf7..d13d11208d913 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoin.java @@ -62,6 +62,9 @@ public void init(ProcessorContext context) { otherWindow = (WindowStore) context.getStateStore(otherWindowName); } + /** + * @throws StreamsException if key is null + */ @Override public void process(K key, V1 value) { // the keys should never be null diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java index e37fe341289ca..ed6e2167d07a5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamReduce.java @@ -58,6 +58,9 @@ public void init(ProcessorContext context) { store = (KeyValueStore) context.getStateStore(storeName); } + /** + * @throws StreamsException if key is null + */ @Override public void process(K key, V value) { // the keys should never be null diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java index 806c6e95000c6..4a7c7c00c99d0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableAggregate.java @@ -63,6 +63,9 @@ public void init(ProcessorContext context) { store = (KeyValueStore) context.getStateStore(storeName); } + /** + * @throws StreamsException if key is null + */ @Override public void process(K key, Change value) { // the keys should never be null diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index ee2c9313e53d6..f78169e0d4764 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -157,6 +157,9 @@ public void writeAsText(String filePath) { writeAsText(filePath, null, null); } + /** + * @throws TopologyBuilderException if file is not found + */ @Override public void writeAsText(String filePath, Serde keySerde, Serde valSerde) { String name = topology.newName(PRINTING_NAME); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java index 24c8da6307093..36424d11e026e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoin.java @@ -60,6 +60,9 @@ public void init(ProcessorContext context) { valueGetter.init(context); } + /** + * @throws StreamsException if key is null + */ @Override public void process(K key, Change change) { // the keys should never be null diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java index 4bf45ed068039..996ebc3e8242a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoin.java @@ -60,6 +60,9 @@ public void init(ProcessorContext context) { valueGetter.init(context); } + /** + * @throws StreamsException if key is null + */ @Override public void process(K key, Change change) { // the keys should never be null diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java index 49eed539d00f2..2a0d47724d553 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoin.java @@ -60,6 +60,9 @@ public void init(ProcessorContext context) { valueGetter.init(context); } + /** + * @throws StreamsException if key is null + */ @Override public void process(K key, Change change) { // the keys should never be null diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java index 7443d4a623151..fa41ed3baf167 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableKTableRightJoin.java @@ -61,6 +61,9 @@ public void init(ProcessorContext context) { valueGetter.init(context); } + /** + * @throws StreamsException if key is null + */ @Override public void process(K key, Change change) { // the keys should never be null diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java index d56b3aec1ebba..bab6bf3bbbaec 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableReduce.java @@ -60,6 +60,9 @@ public void init(ProcessorContext context) { store = (KeyValueStore) context.getStateStore(storeName); } + /** + * @throws StreamsException if key is null + */ @Override public void process(K key, Change value) { // the keys should never be null diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java index 142a2796c2834..2a7cf1b56f059 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableRepartitionMap.java @@ -57,6 +57,9 @@ public KTableValueGetter> get() { }; } + /** + * @throws IllegalStateException since this method should never be called + */ @Override public void enableSendingOldValues() { // this should never be called @@ -74,6 +77,9 @@ private KeyValue computeValue(K key, V value) { private class KTableMapProcessor extends AbstractProcessor> { + /** + * @throws StreamsException if key is null + */ @Override public void process(K key, Change change) { KeyValue newPair = computeValue(key, change.newValue); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java b/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java index 999f6a94f3bf5..405ecd590a354 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/DefaultPartitionGrouper.java @@ -69,6 +69,9 @@ public Map> partitionGroups(Map return Collections.unmodifiableMap(groups); } + /** + * @throws StreamsException if no metadata can be received for a topic + */ protected int maxNumPartitions(Cluster metadata, Set topics) { int maxNumPartitions = 0; for (String topic : topics) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java b/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java index ff21047c20b9d..fa7c73c112ad5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java @@ -41,6 +41,9 @@ public String toString() { return topicGroupId + "_" + partition; } + /** + * @throws TaskIdFormatException if the string is not a valid TaskId + */ public static TaskId parse(String string) { int index = string.indexOf('_'); if (index <= 0 || index + 1 >= string.length()) throw new TaskIdFormatException(string); @@ -55,11 +58,17 @@ public static TaskId parse(String string) { } } + /** + * @throws IOException if cannot write to output stream + */ public void writeTo(DataOutputStream out) throws IOException { out.writeInt(topicGroupId); out.writeInt(partition); } + /** + * @throws IOException if cannot read from input stream + */ public static TaskId readFrom(DataInputStream in) throws IOException { return new TaskId(in.readInt(), in.readInt()); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java index 7f5d6454d55c2..487d5fe8684f6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java @@ -221,6 +221,7 @@ public final TopologyBuilder addSource(String name, String... topics) { * {@link org.apache.kafka.streams.StreamsConfig stream configuration} * @param topics the name of one or more Kafka topics that this source is to consume * @return this builder instance so methods can be chained together; never null + * @throws TopologyBuilderException if processor is already added or if topics have already been registered by another source */ public final TopologyBuilder addSource(String name, Deserializer keyDeserializer, Deserializer valDeserializer, String... topics) { if (nodeFactories.containsKey(name)) @@ -328,6 +329,7 @@ public final TopologyBuilder addSink(String name, String topic, Serializer keySe * @see #addSink(String, String, String...) * @see #addSink(String, String, StreamPartitioner, String...) * @see #addSink(String, String, Serializer, Serializer, String...) + * @throws TopologyBuilderException if parent processor is not added yet, or if this processor's name is equal to the parent's name */ public final TopologyBuilder addSink(String name, String topic, Serializer keySerializer, Serializer valSerializer, StreamPartitioner partitioner, String... parentNames) { if (nodeFactories.containsKey(name)) @@ -359,6 +361,7 @@ public final TopologyBuilder addSink(String name, String topic, Serialize * @param parentNames the name of one or more source or processor nodes whose output messages this processor should receive * and process * @return this builder instance so methods can be chained together; never null + * @throws TopologyBuilderException if parent processor is not added yet, or if this processor's name is equal to the parent's name */ public final TopologyBuilder addProcessor(String name, ProcessorSupplier supplier, String... parentNames) { if (nodeFactories.containsKey(name)) @@ -386,6 +389,7 @@ public final TopologyBuilder addProcessor(String name, ProcessorSupplier supplie * * @param supplier the supplier used to obtain this state store {@link StateStore} instance * @return this builder instance so methods can be chained together; never null + * @throws TopologyBuilderException if state store supplier is already added */ public final TopologyBuilder addStateStore(StateStoreSupplier supplier, boolean isInternal, String... processorNames) { if (stateFactories.containsKey(supplier.name())) { @@ -438,6 +442,7 @@ public final TopologyBuilder connectProcessorAndStateStores(String processorName * * @param processorNames the name of the processors * @return this builder instance so methods can be chained together; never null + * @throws TopologyBuilderException if less than two processors are specified, or if one of the processors is not added yet */ public final TopologyBuilder connectProcessors(String... processorNames) { if (processorNames.length < 2) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java index b3b653754e8ae..c85ecde8ef5cc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/AbstractTask.java @@ -44,6 +44,9 @@ public abstract class AbstractTask { protected final Set partitions; protected ProcessorContext processorContext; + /** + * @throws ProcessorStateException if the state manager cannot be created + */ protected AbstractTask(TaskId id, String applicationId, Collection partitions, @@ -101,6 +104,9 @@ public final ProcessorContext context() { public abstract void commit(); + /** + * @throws ProcessorStateException if there is an error while closing the state manager + */ public void close() { try { stateMgr.close(recordCollectorOffsets()); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java index 3725c4c4b19ca..536a4478ee3f6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java @@ -51,6 +51,9 @@ public class InternalTopicManager { private class ZKStringSerializer implements ZkSerializer { + /** + * @throws AssertionError if the byte String encoding type is not supported + */ @Override public byte[] serialize(Object data) { try { @@ -60,6 +63,9 @@ public byte[] serialize(Object data) { } } + /** + * @throws AssertionError if the byte String encoding type is not supported + */ @Override public Object deserialize(byte[] bytes) { try { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java index 717df2c54abcd..655b8b8551dac 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/MinTimestampTracker.java @@ -32,6 +32,9 @@ public class MinTimestampTracker implements TimestampTracker { // record's timestamp private long lastKnownTime = NOT_KNOWN; + /** + * @throws NullPointerException if the element is null + */ public void addElement(Stamped elem) { if (elem == null) throw new NullPointerException(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java index ec89d47e08b7f..8c6078a10bca3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/PartitionGroup.java @@ -145,6 +145,9 @@ public long timestamp() { return timestamp; } + /** + * @throws IllegalStateException if the record's partition does not belong to this partition group + */ public int numBuffered(TopicPartition partition) { RecordQueue recordQueue = partitionQueues.get(partition); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java index 1c398ac69b341..10e7d68b44c0e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorContextImpl.java @@ -105,6 +105,9 @@ public StreamsMetrics metrics() { return metrics; } + /** + * @throws IllegalStateException if this method is called before {@link #initialized()} + */ @Override public void register(StateStore store, boolean loggingEnabled, StateRestoreCallback stateRestoreCallback) { if (initialized) @@ -113,6 +116,9 @@ public void register(StateStore store, boolean loggingEnabled, StateRestoreCallb stateMgr.register(store, loggingEnabled, stateRestoreCallback); } + /** + * @throws TopologyBuilderException if an attempt is made to access this state store from an unknown node + */ @Override public StateStore getStateStore(String name) { ProcessorNode node = task.node(); @@ -127,6 +133,9 @@ public StateStore getStateStore(String name) { return stateMgr.getStore(name); } + /** + * @throws IllegalStateException if the task's record is null + */ @Override public String topic() { if (task.record() == null) @@ -140,6 +149,9 @@ public String topic() { return topic; } + /** + * @throws IllegalStateException if the task's record is null + */ @Override public int partition() { if (task.record() == null) @@ -148,6 +160,9 @@ public int partition() { return task.record().partition(); } + /** + * @throws IllegalStateException if the task's record is null + */ @Override public long offset() { if (this.task.record() == null) @@ -156,6 +171,9 @@ public long offset() { return this.task.record().offset(); } + /** + * @throws IllegalStateException if the task's record is null + */ @Override public long timestamp() { if (task.record() == null) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index 665d39f8264e5..003b988273fb5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -67,6 +67,9 @@ public class ProcessorStateManager { private final boolean isStandby; private final Map restoreCallbacks; // used for standby tasks, keyed by state topic name + /** + * @throws IOException if any error happens while creating or locking the state directory + */ public ProcessorStateManager(String applicationId, int defaultPartition, Collection sources, File baseDir, Consumer restoreConsumer, boolean isStandby) throws IOException { this.applicationId = applicationId; this.defaultPartition = defaultPartition; @@ -110,6 +113,9 @@ public static String storeChangelogTopic(String applicationId, String storeName) return applicationId + "-" + storeName + STATE_CHANGELOG_TOPIC_SUFFIX; } + /** + * @throws IOException if any error happens when locking the state directory + */ public static FileLock lockStateDirectory(File stateDir) throws IOException { return lockStateDirectory(stateDir, 0); } @@ -143,6 +149,11 @@ public File baseDir() { return this.baseDir; } + /** + * @throws IllegalArgumentException if the store name has already been registered or if it is not a valid name + * (e.g., when it conflicts with the names of internal topics, like the checkpoint file name) + * @throws StreamsException if the store's change log does not contain the partition + */ public void register(StateStore store, boolean loggingEnabled, StateRestoreCallback stateRestoreCallback) { if (store.name().equals(CHECKPOINT_FILE_NAME)) throw new IllegalArgumentException("Illegal store name: " + CHECKPOINT_FILE_NAME); @@ -313,6 +324,9 @@ public void flush() { } } + /** + * @throws IOException if any error happens when flushing or closing the state stores + */ public void close(Map ackedOffsets) throws IOException { try { if (!stores.isEmpty()) { diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/QuickUnion.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/QuickUnion.java index 087cbd2591e25..4e789fa6bce1e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/QuickUnion.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/QuickUnion.java @@ -32,6 +32,9 @@ public boolean exists(T id) { return ids.containsKey(id); } + /** + * @throws NoSuchElementException if the parent of this node is null + */ public T root(T id) { T current = id; T parent = ids.get(current); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java index 31a558bf51047..e9c2760344dcf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java @@ -40,6 +40,9 @@ public SinkNode(String name, String topic, Serializer keySerializer, Serializ this.partitioner = partitioner; } + /** + * @throws UnsupportedOperationException if this method adds a child to a sink node + */ @Override public void addChild(ProcessorNode child) { throw new UnsupportedOperationException("sink node does not allow addChild"); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java index 468fe741448ae..d4b47e2c954ea 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StandbyContextImpl.java @@ -98,6 +98,9 @@ public StreamsMetrics metrics() { return metrics; } + /** + * @throws IllegalStateException + */ @Override public void register(StateStore store, boolean loggingEnabled, StateRestoreCallback stateRestoreCallback) { if (initialized) @@ -106,51 +109,81 @@ public void register(StateStore store, boolean loggingEnabled, StateRestoreCallb stateMgr.register(store, loggingEnabled, stateRestoreCallback); } + /** + * @throws UnsupportedOperationException + */ @Override public StateStore getStateStore(String name) { throw new UnsupportedOperationException("this should not happen: getStateStore() not supported in standby tasks."); } + /** + * @throws UnsupportedOperationException + */ @Override public String topic() { throw new UnsupportedOperationException("this should not happen: topic() not supported in standby tasks."); } + /** + * @throws UnsupportedOperationException + */ @Override public int partition() { throw new UnsupportedOperationException("this should not happen: partition() not supported in standby tasks."); } + /** + * @throws UnsupportedOperationException + */ @Override public long offset() { throw new UnsupportedOperationException("this should not happen: offset() not supported in standby tasks."); } + /** + * @throws UnsupportedOperationException + */ @Override public long timestamp() { throw new UnsupportedOperationException("this should not happen: timestamp() not supported in standby tasks."); } + /** + * @throws UnsupportedOperationException + */ @Override public void forward(K key, V value) { throw new UnsupportedOperationException("this should not happen: forward() not supported in standby tasks."); } + /** + * @throws UnsupportedOperationException + */ @Override public void forward(K key, V value, int childIndex) { throw new UnsupportedOperationException("this should not happen: forward() not supported in standby tasks."); } + /** + * @throws UnsupportedOperationException + */ @Override public void forward(K key, V value, String childName) { throw new UnsupportedOperationException("this should not happen: forward() not supported in standby tasks."); } + /** + * @throws UnsupportedOperationException + */ @Override public void commit() { throw new UnsupportedOperationException("this should not happen: commit() not supported in standby tasks."); } + /** + * @throws UnsupportedOperationException + */ @Override public void schedule(long interval) { throw new UnsupportedOperationException("this should not happen: schedule() not supported in standby tasks."); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java index 1dd082d8cf5c1..bc42c8253fa16 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java @@ -96,6 +96,7 @@ public int compare(TopicPartition p1, TopicPartition p2) { * We need to have the PartitionAssignor and its StreamThread to be mutually accessible * since the former needs later's cached metadata while sending subscriptions, * and the latter needs former's returned assignment when adding tasks. + * @throws KafkaException if the stream thread is not specified */ @Override public void configure(Map configs) { @@ -382,6 +383,9 @@ public Map assign(Cluster metadata, Map partitions = new ArrayList<>(assignment.partitions()); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java index 53d0a8df2848f..d9efb6debb4b0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamTask.java @@ -212,6 +212,9 @@ public boolean maybePunctuate() { return punctuationQueue.mayPunctuate(timestamp, this); } + /** + * @throws IllegalStateException if the current node is not null + */ @Override public void punctuate(ProcessorNode node, long timestamp) { if (currNode != null) @@ -280,6 +283,7 @@ public void needCommit() { * Schedules a punctuation for the processor * * @param interval the interval in milliseconds + * @throws IllegalStateException if the current node is not null */ public void schedule(long interval) { if (currNode == null) @@ -288,6 +292,9 @@ public void schedule(long interval) { punctuationQueue.schedule(new PunctuationSchedule(currNode, interval)); } + /** + * @throws RuntimeException if an error happens during closing of processor nodes + */ @Override public void close() { this.partitionGroup.close(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index 38dc356a672c9..f02683e587170 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -239,6 +239,8 @@ private Consumer createRestoreConsumer() { /** * Execute the stream processors + * @throws KafkaException for any Kafka-related exceptions + * @throws Exception for any other non-Kafka exceptions */ @Override public void run() { @@ -760,6 +762,9 @@ public void recordLatency(Sensor sensor, long startNs, long endNs) { sensor.record((endNs - startNs) / 1000000, endNs); } + /** + * @throws IllegalArgumentException if tags is not constructed in key-value pairs + */ @Override public Sensor addLatencySensor(String scopeName, String entityName, String operationName, String... tags) { // extract the additional tags if there are any diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java index c2175bbf5ab39..0486e57ef7117 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/assignment/AssignmentInfo.java @@ -54,6 +54,10 @@ protected AssignmentInfo(int version, List activeTasks, Map prevTasks, Set this.standbyTasks = standbyTasks; } + /** + * @throws TaskAssignmentException if method fails to encode the data + */ public ByteBuffer encode() { if (version == CURRENT_VERSION) { ByteBuffer buf = ByteBuffer.allocate(4 /* version */ + 16 /* process id */ + 4 + prevTasks.size() * 8 + 4 + standbyTasks.size() * 8); @@ -78,6 +81,9 @@ public ByteBuffer encode() { } } + /** + * @throws TaskAssignmentException if method fails to decode the data + */ public static SubscriptionInfo decode(ByteBuffer data) { // ensure we are at the beginning of the ByteBuffer data.rewind(); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java b/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java index 9daac98136f01..933bf72b394ce 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java @@ -46,6 +46,7 @@ public static StateSerdes withBuiltinTypes(String topic, Class k * @param stateName the name of the state * @param keySerde the serde for keys; cannot be null * @param valueSerde the serde for values; cannot be null + * @throws IllegalArgumentException if key or value serde is null */ @SuppressWarnings("unchecked") public StateSerdes(String stateName, diff --git a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java index 4e281876b0767..9f1e53c8ff666 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/Stores.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/Stores.java @@ -50,6 +50,10 @@ public InMemoryKeyValueFactory inMemory() { return new InMemoryKeyValueFactory() { private int capacity = Integer.MAX_VALUE; + /** + * @param capacity the maximum capacity of the in-memory cache; should be one less than a power of 2 + * @throws IllegalArgumentException if the capacity of the store is zero or negative + */ @Override public InMemoryKeyValueFactory maxEntries(int capacity) { if (capacity < 1) throw new IllegalArgumentException("The capacity must be positive"); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java index a859bd2cbbe1c..76dd74439baef 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java @@ -124,11 +124,17 @@ public V delete(K key) { return value; } + /** + * @throws UnsupportedOperationException + */ @Override public KeyValueIterator range(K from, K to) { throw new UnsupportedOperationException("MemoryLRUCache does not support range() function."); } + /** + * @throws UnsupportedOperationException + */ @Override public KeyValueIterator all() { throw new UnsupportedOperationException("MemoryLRUCache does not support all() function."); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/OffsetCheckpoint.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/OffsetCheckpoint.java index 853fc5d867075..ff17e68897f50 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/OffsetCheckpoint.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/OffsetCheckpoint.java @@ -55,11 +55,17 @@ public class OffsetCheckpoint { private final File file; private final Object lock; + /** + * @throws IOException + */ public OffsetCheckpoint(File file) throws IOException { this.file = file; this.lock = new Object(); } + /** + * @throws IOException if any file operation fails with an IO exception + */ public void write(Map offsets) throws IOException { synchronized (lock) { // write to temp file and then swap with the existing file @@ -84,11 +90,17 @@ public void write(Map offsets) throws IOException { } } + /** + * @throws IOException if file write operations failed with any IO exception + */ private void writeIntLine(BufferedWriter writer, int number) throws IOException { writer.write(Integer.toString(number)); writer.newLine(); } + /** + * @throws IOException if file write operations failed with any IO exception + */ private void writeEntry(BufferedWriter writer, TopicPartition part, long offset) throws IOException { writer.write(part.topic()); writer.write(' '); @@ -98,6 +110,11 @@ private void writeEntry(BufferedWriter writer, TopicPartition part, long offset) writer.newLine(); } + + /** + * @throws IOException if any file operation fails with an IO exception + * @throws IllegalArgumentException if the offset checkpoint version is unknown + */ public Map read() throws IOException { synchronized (lock) { BufferedReader reader; @@ -141,6 +158,9 @@ public Map read() throws IOException { } } + /** + * @throws IOException if file read ended prematurely + */ private int readInt(BufferedReader reader) throws IOException { String line = reader.readLine(); if (line == null) @@ -148,6 +168,9 @@ private int readInt(BufferedReader reader) throws IOException { return Integer.parseInt(line); } + /** + * @throws IOException if there is any IO exception during delete + */ public void delete() throws IOException { file.delete(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index fe327f6af8a2d..944d40802f06f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -397,6 +397,9 @@ public void flush() { flushInternal(); } + /** + * @throws ProcessorStateException if flushing failed because of any internal store exceptions + */ public void flushInternal() { try { db.flush(fOptions); @@ -433,6 +436,9 @@ public boolean hasNext() { return iter.isValid(); } + /** + * @throws NoSuchElementException if no next element exist + */ @Override public KeyValue next() { if (!hasNext()) @@ -443,6 +449,9 @@ public KeyValue next() { return entry; } + /** + * @throws UnsupportedOperationException + */ @Override public void remove() { throw new UnsupportedOperationException("RocksDB iterator does not support remove"); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java index 9851c0489b886..5955d214b0f2a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java @@ -86,6 +86,9 @@ public boolean hasNext() { return false; } + /** + * @throws NoSuchElementException if no next element exists + */ @Override public KeyValue next() { if (index >= iterators.length) From 34a594472133c3bc6a2886cf58ded0065f26d28a Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Tue, 12 Apr 2016 13:47:02 -0700 Subject: [PATCH 139/206] MINOR: Remove unused hadoop version All dependencies on hadoop were removed with MiniKDC. This removes the left over version entry. Author: Grant Henke Reviewers: Ismael Juma Closes #1214 from granthenke/remove-hadoop --- gradle/dependencies.gradle | 1 - 1 file changed, 1 deletion(-) diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle index f0358385e93af..87e83cba9e1bb 100644 --- a/gradle/dependencies.gradle +++ b/gradle/dependencies.gradle @@ -27,7 +27,6 @@ versions += [ apacheds: "2.0.0-M21", argparse4j: "0.5.0", bcpkix: "1.54", - hadoop: "2.7.2", easymock: "3.4", jackson: "2.6.3", jetty: "9.2.15.v20160210", From e79d9af3cfbb8884e00424f84f3c687114497998 Mon Sep 17 00:00:00 2001 From: Dongjoon Hyun Date: Tue, 12 Apr 2016 13:48:18 -0700 Subject: [PATCH 140/206] KAFKA-3461: Fix typos in Kafka web documentations. This PR fixes 8 typos in HTML files of `docs` module. I wrote explicitly here since Github sometimes does not highlight the corrections on long lines correctly. - docs/api.html: compatability => compatibility - docs/connect.html: simultaneoulsy => simultaneously - docs/implementation.html: LATIEST_TIME => LATEST_TIME, nPartions => nPartitions - docs/migration.html: Decomission => Decommission - docs/ops.html: stoping => stopping, ConumserGroupCommand => ConsumerGroupCommand, youre => you're Author: Dongjoon Hyun Reviewers: Ismael Juma Closes #1138 from dongjoon-hyun/KAFKA-3461 --- docs/api.html | 2 +- docs/connect.html | 28 ++++++++++++++-------------- docs/implementation.html | 4 ++-- docs/migration.html | 2 +- docs/ops.html | 6 +++--- 5 files changed, 21 insertions(+), 21 deletions(-) diff --git a/docs/api.html b/docs/api.html index d303244427d73..8d5be9b030da9 100644 --- a/docs/api.html +++ b/docs/api.html @@ -15,7 +15,7 @@ limitations under the License. --> -Apache Kafka includes new java clients (in the org.apache.kafka.clients package). These are meant to supplant the older Scala clients, but for compatability they will co-exist for some time. These clients are available in a separate jar with minimal dependencies, while the old Scala clients remain packaged with the server. +Apache Kafka includes new java clients (in the org.apache.kafka.clients package). These are meant to supplant the older Scala clients, but for compatibility they will co-exist for some time. These clients are available in a separate jar with minimal dependencies, while the old Scala clients remain packaged with the server.

      2.1 Producer API

      diff --git a/docs/connect.html b/docs/connect.html index dc6ad6e9eb5b0..88b8c2b5c34c9 100644 --- a/docs/connect.html +++ b/docs/connect.html @@ -108,7 +108,7 @@
      Connec To copy data between Kafka and another system, users create a Connector for the system they want to pull data from or push data to. Connectors come in two flavors: SourceConnectors import data from another system (e.g. JDBCSourceConnector would import a relational database into Kafka) and SinkConnectors export data (e.g. HDFSSinkConnector would export the contents of a Kafka topic to an HDFS file). -Connectors do not perform any data copying themselves: their configuration describes the data to be copied, and the Connector is responsible for breaking that job into a set of Tasks that can be distributed to workers. These Tasks also come in two corresponding flavors: SourceTaskand SinkTask. +Connectors do not perform any data copying themselves: their configuration describes the data to be copied, and the Connector is responsible for breaking that job into a set of Tasks that can be distributed to workers. These Tasks also come in two corresponding flavors: SourceTask and SinkTask. With an assignment in hand, each Task must copy its subset of the data to or from Kafka. In Kafka Connect, it should always be possible to frame these assignments as a set of input and output streams consisting of records with consistent schemas. Sometimes this mapping is obvious: each file in a set of log files can be considered a stream with each parsed line forming a record using the same schema and offsets stored as byte offsets in the file. In other cases it may require more effort to map to this model: a JDBC connector can map each table to a stream, but the offset is less clear. One possible mapping uses a timestamp column to generate queries incrementally returning new data, and the last queried timestamp can be used as the offset. @@ -242,11 +242,11 @@
      Task Example - Sourc Again, we've omitted some details, but we can see the important steps: the poll() method is going to be called repeatedly, and for each call it will loop trying to read records from the file. For each line it reads, it also tracks the file offset. It uses this information to create an output SourceRecord with four pieces of information: the source partition (there is only one, the single file being read), source offset (byte offset in the file), output topic name, and output value (the line, and we include a schema indicating this value will always be a string). Other variants of the SourceRecord constructor can also include a specific output partition and a key. -Note that this implementation uses the normal Java InputStreaminterface and may sleep if data is not available. This is acceptable because Kafka Connect provides each task with a dedicated thread. While task implementations have to conform to the basic poll()interface, they have a lot of flexibility in how they are implemented. In this case, an NIO-based implementation would be more efficient, but this simple approach works, is quick to implement, and is compatible with older versions of Java. +Note that this implementation uses the normal Java InputStream interface and may sleep if data is not available. This is acceptable because Kafka Connect provides each task with a dedicated thread. While task implementations have to conform to the basic poll() interface, they have a lot of flexibility in how they are implemented. In this case, an NIO-based implementation would be more efficient, but this simple approach works, is quick to implement, and is compatible with older versions of Java.
      Sink Tasks
      -The previous section described how to implement a simple SourceTask. Unlike SourceConnectorand SinkConnector, SourceTaskand SinkTaskhave very different interfaces because SourceTaskuses a pull interface and SinkTaskuses a push interface. Both share the common lifecycle methods, but the SinkTaskinterface is quite different: +The previous section described how to implement a simple SourceTask. Unlike SourceConnector and SinkConnector, SourceTask and SinkTask have very different interfaces because SourceTask uses a pull interface and SinkTask uses a push interface. Both share the common lifecycle methods, but the SinkTask interface is quite different:
       public abstract class SinkTask implements Task {
      @@ -257,17 +257,17 @@ 
      Sink Tasks
      public abstract void flush(Map<TopicPartition, Long> offsets);
      -The SinkTask documentation contains full details, but this interface is nearly as simple as the the SourceTask. The put()method should contain most of the implementation, accepting sets of SinkRecords, performing any required translation, and storing them in the destination system. This method does not need to ensure the data has been fully written to the destination system before returning. In fact, in many cases internal buffering will be useful so an entire batch of records can be sent at once, reducing the overhead of inserting events into the downstream data store. The SinkRecordscontain essentially the same information as SourceRecords: Kafka topic, partition, offset and the event key and value. +The SinkTask documentation contains full details, but this interface is nearly as simple as the the SourceTask. The put() method should contain most of the implementation, accepting sets of SinkRecords, performing any required translation, and storing them in the destination system. This method does not need to ensure the data has been fully written to the destination system before returning. In fact, in many cases internal buffering will be useful so an entire batch of records can be sent at once, reducing the overhead of inserting events into the downstream data store. The SinkRecords contain essentially the same information as SourceRecords: Kafka topic, partition, offset and the event key and value. -The flush()method is used during the offset commit process, which allows tasks to recover from failures and resume from a safe point such that no events will be missed. The method should push any outstanding data to the destination system and then block until the write has been acknowledged. The offsetsparameter can often be ignored, but is useful in some cases where implementations want to store offset information in the destination store to provide exactly-once -delivery. For example, an HDFS connector could do this and use atomic move operations to make sure the flush()operation atomically commits the data and offsets to a final location in HDFS. +The flush() method is used during the offset commit process, which allows tasks to recover from failures and resume from a safe point such that no events will be missed. The method should push any outstanding data to the destination system and then block until the write has been acknowledged. The offsets parameter can often be ignored, but is useful in some cases where implementations want to store offset information in the destination store to provide exactly-once +delivery. For example, an HDFS connector could do this and use atomic move operations to make sure the flush() operation atomically commits the data and offsets to a final location in HDFS.
      Resuming from Previous Offsets
      -The SourceTaskimplementation included a stream ID (the input filename) and offset (position in the file) with each record. The framework uses this to commit offsets periodically so that in the case of a failure, the task can recover and minimize the number of events that are reprocessed and possibly duplicated (or to resume from the most recent offset if Kafka Connect was stopped gracefully, e.g. in standalone mode or due to a job reconfiguration). This commit process is completely automated by the framework, but only the connector knows how to seek back to the right position in the input stream to resume from that location. +The SourceTask implementation included a stream ID (the input filename) and offset (position in the file) with each record. The framework uses this to commit offsets periodically so that in the case of a failure, the task can recover and minimize the number of events that are reprocessed and possibly duplicated (or to resume from the most recent offset if Kafka Connect was stopped gracefully, e.g. in standalone mode or due to a job reconfiguration). This commit process is completely automated by the framework, but only the connector knows how to seek back to the right position in the input stream to resume from that location. -To correctly resume upon startup, the task can use the SourceContextpassed into its initialize()method to access the offset data. In initialize(), we would add a bit more code to read the offset (if it exists) and seek to that position: +To correctly resume upon startup, the task can use the SourceContext passed into its initialize() method to access the offset data. In initialize(), we would add a bit more code to read the offset (if it exists) and seek to that position:
           stream = new FileInputStream(filename);
      @@ -285,7 +285,7 @@ 

      Dynamic Input/Output Str Kafka Connect is intended to define bulk data copying jobs, such as copying an entire database rather than creating many jobs to copy each table individually. One consequence of this design is that the set of input or output streams for a connector can vary over time. -Source connectors need to monitor the source system for changes, e.g. table additions/deletions in a database. When they pick up changes, they should notify the framework via the ConnectorContextobject that reconfiguration is necessary. For example, in a SourceConnector: +Source connectors need to monitor the source system for changes, e.g. table additions/deletions in a database. When they pick up changes, they should notify the framework via the ConnectorContext object that reconfiguration is necessary. For example, in a SourceConnector:
      @@ -293,11 +293,11 @@ 

      Dynamic Input/Output Str this.context.requestTaskReconfiguration();

      -The framework will promptly request new configuration information and update the tasks, allowing them to gracefully commit their progress before reconfiguring them. Note that in the SourceConnectorthis monitoring is currently left up to the connector implementation. If an extra thread is required to perform this monitoring, the connector must allocate it itself. +The framework will promptly request new configuration information and update the tasks, allowing them to gracefully commit their progress before reconfiguring them. Note that in the SourceConnector this monitoring is currently left up to the connector implementation. If an extra thread is required to perform this monitoring, the connector must allocate it itself. -Ideally this code for monitoring changes would be isolated to the Connectorand tasks would not need to worry about them. However, changes can also affect tasks, most commonly when one of their input streams is destroyed in the input system, e.g. if a table is dropped from a database. If the Taskencounters the issue before the Connector, which will be common if the Connectorneeds to poll for changes, the Taskwill need to handle the subsequent error. Thankfully, this can usually be handled simply by catching and handling the appropriate exception. +Ideally this code for monitoring changes would be isolated to the Connector and tasks would not need to worry about them. However, changes can also affect tasks, most commonly when one of their input streams is destroyed in the input system, e.g. if a table is dropped from a database. If the Task encounters the issue before the Connector, which will be common if the Connector needs to poll for changes, the Task will need to handle the subsequent error. Thankfully, this can usually be handled simply by catching and handling the appropriate exception. -SinkConnectors usually only have to handle the addition of streams, which may translate to new entries in their outputs (e.g., a new database table). The framework manages any changes to the Kafka input, such as when the set of input topics changes because of a regex subscription. SinkTasksshould expect new input streams, which may require creating new resources in the downstream system, such as a new table in a database. The trickiest situation to handle in these cases may be conflicts between multiple SinkTasksseeing a new input stream for the first time and simultaneoulsy trying to create the new resource. SinkConnectors, on the other hand, will generally require no special code for handling a dynamic set of streams. +SinkConnectors usually only have to handle the addition of streams, which may translate to new entries in their outputs (e.g., a new database table). The framework manages any changes to the Kafka input, such as when the set of input topics changes because of a regex subscription. SinkTasks should expect new input streams, which may require creating new resources in the downstream system, such as a new table in a database. The trickiest situation to handle in these cases may be conflicts between multiple SinkTasks seeing a new input stream for the first time and simultaneously trying to create the new resource. SinkConnectors, on the other hand, will generally require no special code for handling a dynamic set of streams.

      Working with Schemas

      @@ -305,7 +305,7 @@

      Working with Schemas

      data API. Most structured records will need to interact with two classes in addition to primitive types: Schema and Struct. -The API documentation provides a complete reference, but here is a simple example creating a Schemaand Struct: +The API documentation provides a complete reference, but here is a simple example creating a Schema and Struct:
       Schema schema = SchemaBuilder.struct().name(NAME)
      @@ -322,7 +322,7 @@ 

      Working with Schemas

      ALTER TABLEcommand. The connector must be able to detect these changes and react appropriately. +However, many connectors will have dynamic schemas. One simple example of this is a database connector. Considering even just a single table, the schema will not be predefined for the entire connector (as it varies from table to table). But it also may not be fixed for a single table over the lifetime of the connector since the user may execute an ALTER TABLE command. The connector must be able to detect these changes and react appropriately. Sink connectors are usually simpler because they are consuming data and therefore do not need to create schemas. However, they should take just as much care to validate that the schemas they receive have the expected format. When the schema does not match -- usually indicating the upstream producer is generating invalid data that cannot be correctly translated to the destination system -- sink connectors should throw an exception to indicate this error to the system. diff --git a/docs/implementation.html b/docs/implementation.html index ecd99e708ec9d..be81227c906bb 100644 --- a/docs/implementation.html +++ b/docs/implementation.html @@ -90,7 +90,7 @@
      Low-level API
      * Get a list of valid offsets (up to maxSize) before the given time. * The result is a list of offsets, in descending order. * @param time: time in millisecs, - * if set to OffsetRequest$.MODULE$.LATIEST_TIME(), get from the latest offset available. + * if set to OffsetRequest$.MODULE$.LATEST_TIME(), get from the latest offset available. * if set to OffsetRequest$.MODULE$.EARLIEST_TIME(), get from the earliest offset available. */ public long[] getOffsetsBefore(String topic, int partition, long time, int maxNumOffsets); @@ -292,7 +292,7 @@

      Broker Node Registry

      Broker Topic Registry

      -/brokers/topics/[topic]/[0...N] --> nPartions (ephemeral node)
      +/brokers/topics/[topic]/[0...N] --> nPartitions (ephemeral node)
       

      diff --git a/docs/migration.html b/docs/migration.html index 2da6a7e26ac94..5240d8664335b 100644 --- a/docs/migration.html +++ b/docs/migration.html @@ -27,7 +27,7 @@

      Migration Steps

    18. Use the 0.7 to 0.8 migration tool to mirror data from the 0.7 cluster into the 0.8 cluster.
    19. When the 0.8 cluster is fully caught up, redeploy all data consumers running the 0.8 client and reading from the 0.8 cluster.
    20. Finally migrate all 0.7 producers to 0.8 client publishing data to the 0.8 cluster. -
    21. Decomission the 0.7 cluster. +
    22. Decommission the 0.7 cluster.
    23. Drink. diff --git a/docs/ops.html b/docs/ops.html index b239a0eda550d..8b1cc234c6407 100644 --- a/docs/ops.html +++ b/docs/ops.html @@ -70,7 +70,7 @@

      Modifying topi

      Graceful shutdown

      -The Kafka cluster will automatically detect any broker shutdown or failure and elect new leaders for the partitions on that machine. This will occur whether a server fails or it is brought down intentionally for maintenance or configuration changes. For the latter cases Kafka supports a more graceful mechanism for stoping a server than just killing it. +The Kafka cluster will automatically detect any broker shutdown or failure and elect new leaders for the partitions on that machine. This will occur whether a server fails or it is brought down intentionally for maintenance or configuration changes. For the latter cases Kafka supports a more graceful mechanism for stopping a server than just killing it. When a server is stopped gracefully it has two optimizations it will take advantage of:
        @@ -138,7 +138,7 @@

        Checking consu

        Managing Consumer Groups

        -With the ConumserGroupCommand tool, we can list, delete, or describe consumer groups. For example, to list all consumer groups across all topics: +With the ConsumerGroupCommand tool, we can list, delete, or describe consumer groups. For example, to list all consumer groups across all topics:
          > bin/kafka-consumer-groups.sh --zookeeper localhost:2181 --list
        @@ -156,7 +156,7 @@ 

        Managing C

        -When youre using the new consumer-groups API where the broker handles coordination of partition handling and rebalance, you can manage the groups with the "--new-consumer" flags: +When you're using the new consumer-groups API where the broker handles coordination of partition handling and rebalance, you can manage the groups with the "--new-consumer" flags:
          > bin/kafka-consumer-groups.sh --new-consumer --bootstrap-server broker1:9092 --list
        
        From 667ff7ef737612773c50908d2b3cc829bb5132c7 Mon Sep 17 00:00:00 2001
        From: Eno Thereska 
        Date: Tue, 12 Apr 2016 17:38:20 -0700
        Subject: [PATCH 141/206] KAFKA-3504; Log compaction for changelog partition
        
        Author: Eno Thereska 
        
        Reviewers: Ismael Juma , Guozhang Wang 
        
        Closes #1203 from enothereska/KAFKA-3504-logcompaction
        ---
         .../internals/InternalTopicManager.java       |  29 ++++-
         .../internals/StreamPartitionAssignor.java    | 120 +++++++++---------
         .../StreamPartitionAssignorTest.java          |   2 +-
         3 files changed, 86 insertions(+), 65 deletions(-)
        
        diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java
        index 536a4478ee3f6..4477fb7ccad2f 100644
        --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java
        +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/InternalTopicManager.java
        @@ -36,6 +36,7 @@
         import java.util.HashMap;
         import java.util.List;
         import java.util.Map;
        +import java.util.Properties;
         
         public class InternalTopicManager {
         
        @@ -45,6 +46,10 @@ public class InternalTopicManager {
             private static final String ZK_TOPIC_PATH = "/brokers/topics";
             private static final String ZK_BROKER_PATH = "/brokers/ids";
             private static final String ZK_DELETE_TOPIC_PATH = "/admin/delete_topics";
        +    private static final String ZK_ENTITY_CONFIG_PATH = "/config/topics";
        +    // TODO: the following LogConfig dependency should be removed after KIP-4
        +    private static final String CLEANUP_POLICY_PROP = "cleanup.policy";
        +    private static final String COMPACT = "compact";
         
             private final ZkClient zkClient;
             private final int replicationFactor;
        @@ -89,7 +94,7 @@ public InternalTopicManager(String zkConnect, int replicationFactor) {
                 this.replicationFactor = replicationFactor;
             }
         
        -    public void makeReady(String topic, int numPartitions) {
        +    public void makeReady(String topic, int numPartitions, boolean compactTopic) {
                 boolean topicNotReady = true;
         
                 while (topicNotReady) {
        @@ -97,7 +102,7 @@ public void makeReady(String topic, int numPartitions) {
         
                     if (topicMetadata == null) {
                         try {
        -                    createTopic(topic, numPartitions, replicationFactor);
        +                    createTopic(topic, numPartitions, replicationFactor, compactTopic);
                         } catch (ZkNodeExistsException e) {
                             // ignore and continue
                         }
        @@ -158,9 +163,10 @@ private Map> getTopicMetadata(String topic) {
                 }
             }
         
        -    private void createTopic(String topic, int numPartitions, int replicationFactor) throws ZkNodeExistsException {
        +    private void createTopic(String topic, int numPartitions, int replicationFactor, boolean compactTopic) throws ZkNodeExistsException {
                 log.debug("Creating topic {} with {} partitions from ZK in partition assignor.", topic, numPartitions);
        -
        +        Properties prop = new Properties();
        +        ObjectMapper mapper = new ObjectMapper();
                 List brokers = getBrokers();
                 int numBrokers = brokers.size();
                 if (numBrokers < replicationFactor) {
        @@ -178,14 +184,25 @@ private void createTopic(String topic, int numPartitions, int replicationFactor)
                     }
                     assignment.put(i, brokerList);
                 }
        +        // write out config first just like in AdminUtils.scala createOrUpdateTopicPartitionAssignmentPathInZK()
        +        if (compactTopic) {
        +            prop.put(CLEANUP_POLICY_PROP, COMPACT);
        +            try {
        +                Map dataMap = new HashMap<>();
        +                dataMap.put("version", 1);
        +                dataMap.put("config", prop);
        +                String data = mapper.writeValueAsString(dataMap);
        +                zkClient.createPersistent(ZK_ENTITY_CONFIG_PATH + "/" + topic, data, ZooDefs.Ids.OPEN_ACL_UNSAFE);
        +            } catch (JsonProcessingException e) {
        +                throw new StreamsException("Error while creating topic config in ZK for internal topic " + topic, e);
        +            }
        +        }
         
                 // try to write to ZK with open ACL
                 try {
                     Map dataMap = new HashMap<>();
                     dataMap.put("version", 1);
                     dataMap.put("partitions", assignment);
        -
        -            ObjectMapper mapper = new ObjectMapper();
                     String data = mapper.writeValueAsString(dataMap);
         
                     zkClient.createPersistent(ZK_TOPIC_PATH + "/" + topic, data, ZooDefs.Ids.OPEN_ACL_UNSAFE);
        diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java
        index bc42c8253fa16..341e66a6862c0 100644
        --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java
        +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignor.java
        @@ -147,6 +147,63 @@ public Subscription subscription(Set topics) {
                 return new Subscription(new ArrayList<>(topics), data.encode());
             }
         
        +    /**
        +     * Internal helper function that creates a Kafka topic
        +     * @param topicToTaskIds Map that contains the topic names to be created
        +     * @param compactTopic If true, the topic should be a compacted topic. This is used for
        +     *                     change log topics usually.
        +     * @param outPartitionInfo If true, compute and return all partitions created
        +     * @param postPartitionPhase If true, the computation for calculating the number of partitions
        +     *                           is slightly different. Set to true after the initial topic-to-partition
        +     *                           assignment.
        +     * @return
        +     */
        +    private Map prepareTopic(Map> topicToTaskIds,
        +                                                            boolean compactTopic,
        +                                                            boolean outPartitionInfo,
        +                                                            boolean postPartitionPhase) {
        +        Map partitionInfos = new HashMap<>();
        +        // if ZK is specified, prepare the internal source topic before calling partition grouper
        +        if (internalTopicManager != null) {
        +            log.debug("Starting to validate internal topics in partition assignor.");
        +
        +            for (Map.Entry> entry : topicToTaskIds.entrySet()) {
        +                String topic = entry.getKey();
        +                int numPartitions = 0;
        +                if (postPartitionPhase) {
        +                    // the expected number of partitions is the max value of TaskId.partition + 1
        +                    for (TaskId task : entry.getValue()) {
        +                        if (numPartitions < task.partition + 1)
        +                            numPartitions = task.partition + 1;
        +                    }
        +                } else {
        +                    // should have size 1 only
        +                    numPartitions = -1;
        +                    for (TaskId task : entry.getValue()) {
        +                        numPartitions = task.partition;
        +                    }
        +                }
        +
        +                internalTopicManager.makeReady(topic, numPartitions, compactTopic);
        +
        +                // wait until the topic metadata has been propagated to all brokers
        +                List partitions;
        +                do {
        +                    partitions = streamThread.restoreConsumer.partitionsFor(topic);
        +                } while (partitions == null || partitions.size() != numPartitions);
        +
        +                if (outPartitionInfo) {
        +                    for (PartitionInfo partition : partitions)
        +                        partitionInfos.put(new TopicPartition(partition.topic(), partition.partition()), partition);
        +                }
        +            }
        +
        +            log.info("Completed validating internal topics in partition assignor.");
        +        }
        +
        +        return partitionInfos;
        +    }
        +
             @Override
             public Map assign(Cluster metadata, Map subscriptions) {
                 // This assigns tasks to consumer clients in two steps.
        @@ -227,35 +284,7 @@ public Map assign(Cluster metadata, Map internalPartitionInfos = new HashMap<>();
        -
        -        // if ZK is specified, prepare the internal source topic before calling partition grouper
        -        if (internalTopicManager != null) {
        -            log.debug("Starting to validate internal source topics in partition assignor.");
        -
        -            for (Map.Entry> entry : internalSourceTopicToTaskIds.entrySet()) {
        -                String topic = entry.getKey();
        -
        -                // should have size 1 only
        -                int numPartitions = -1;
        -                for (TaskId task : entry.getValue()) {
        -                    numPartitions = task.partition;
        -                }
        -
        -                internalTopicManager.makeReady(topic, numPartitions);
        -
        -                // wait until the topic metadata has been propagated to all brokers
        -                List partitions;
        -                do {
        -                    partitions = streamThread.restoreConsumer.partitionsFor(topic);
        -                } while (partitions == null || partitions.size() != numPartitions);
        -
        -                for (PartitionInfo partition : partitions)
        -                    internalPartitionInfos.put(new TopicPartition(partition.topic(), partition.partition()), partition);
        -            }
        -
        -            log.info("Completed validating internal source topics in partition assignor.");
        -        }
        +        Map internalPartitionInfos = prepareTopic(internalSourceTopicToTaskIds, false, true, false);
                 internalSourceTopicToTaskIds.clear();
         
                 Cluster metadataWithInternalTopics = metadata;
        @@ -350,35 +379,10 @@ public Map assign(Cluster metadata, Map> topicToTaskIds = new HashMap<>();
        -            topicToTaskIds.putAll(stateChangelogTopicToTaskIds);
        -            topicToTaskIds.putAll(internalSourceTopicToTaskIds);
        -
        -            for (Map.Entry> entry : topicToTaskIds.entrySet()) {
        -                String topic = entry.getKey();
        -
        -                // the expected number of partitions is the max value of TaskId.partition + 1
        -                int numPartitions = 0;
        -                for (TaskId task : entry.getValue()) {
        -                    if (numPartitions < task.partition + 1)
        -                        numPartitions = task.partition + 1;
        -                }
        -
        -                internalTopicManager.makeReady(topic, numPartitions);
        -
        -                // wait until the topic metadata has been propagated to all brokers
        -                List partitions;
        -                do {
        -                    partitions = streamThread.restoreConsumer.partitionsFor(topic);
        -                } while (partitions == null || partitions.size() != numPartitions);
        -            }
        -
        -            log.info("Completed validating changelog topics in partition assignor.");
        -        }
        +        // if ZK is specified, validate the internal topics again
        +        prepareTopic(internalSourceTopicToTaskIds, false /* compactTopic */, false, true);
        +        // change log topics should be compacted
        +        prepareTopic(stateChangelogTopicToTaskIds, true /* compactTopic */, false, true);
         
                 return assignment;
             }
        diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java
        index be851bff67b9a..3e8b110fde9b6 100644
        --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java
        +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/StreamPartitionAssignorTest.java
        @@ -556,7 +556,7 @@ public MockInternalTopicManager(MockConsumer restoreConsumer) {
                 }
         
                 @Override
        -        public void makeReady(String topic, int numPartitions) {
        +        public void makeReady(String topic, int numPartitions, boolean compactTopic) {
                     readyTopics.put(topic, numPartitions);
         
                     List partitions = new ArrayList<>();
        
        From c1694833d5c095e47e5767f38c3e85bbe927a0a7 Mon Sep 17 00:00:00 2001
        From: Ismael Juma 
        Date: Wed, 13 Apr 2016 13:50:49 -0700
        Subject: [PATCH 142/206] KAFKA-3490; Multiple version support for ducktape
         performance tests
        
        Author: Ismael Juma 
        Author: Geoff Anderson 
        
        Reviewers: Geoff Anderson , Ewen Cheslack-Postava 
        
        Closes #1173 from ijuma/kafka-3490-multiple-version-support-perf-tests
        ---
         .../benchmarks/core/benchmark_test.py         |  85 +++++-----
         .../test_performance_services.py              |  88 ++++++++++
         .../services/performance/__init__.py          |   4 +-
         .../performance/consumer_performance.py       |  45 +++--
         .../performance/end_to_end_latency.py         |  62 ++++++-
         .../services/performance/performance.py       |  23 +++
         .../performance/producer_performance.py       | 159 +++++++++++++-----
         .../services/templates/tools_log4j.properties |   2 +-
         vagrant/base.sh                               |   2 +
         9 files changed, 364 insertions(+), 106 deletions(-)
         create mode 100644 tests/kafkatest/sanity_checks/test_performance_services.py
        
        diff --git a/tests/kafkatest/benchmarks/core/benchmark_test.py b/tests/kafkatest/benchmarks/core/benchmark_test.py
        index 9c2e32db4e975..d252e5dead264 100644
        --- a/tests/kafkatest/benchmarks/core/benchmark_test.py
        +++ b/tests/kafkatest/benchmarks/core/benchmark_test.py
        @@ -20,7 +20,8 @@
         
         from kafkatest.services.zookeeper import ZookeeperService
         from kafkatest.services.kafka import KafkaService
        -from kafkatest.services.performance import ProducerPerformanceService, EndToEndLatencyService, ConsumerPerformanceService
        +from kafkatest.services.kafka.version import TRUNK, KafkaVersion
        +from kafkatest.services.performance import ProducerPerformanceService, EndToEndLatencyService, ConsumerPerformanceService, throughput, latency, compute_aggregate_throughput
         
         
         TOPIC_REP_ONE = "topic-replication-factor-one"
        @@ -54,11 +55,12 @@ def __init__(self, test_context):
             def setUp(self):
                 self.zk.start()
         
        -    def start_kafka(self, security_protocol, interbroker_security_protocol):
        +    def start_kafka(self, security_protocol, interbroker_security_protocol, version):
                 self.kafka = KafkaService(
                     self.test_context, self.num_brokers,
                     self.zk, security_protocol=security_protocol,
        -            interbroker_security_protocol=interbroker_security_protocol, topics=self.topics)
        +            interbroker_security_protocol=interbroker_security_protocol, topics=self.topics,
        +            version=version)
                 self.kafka.log_level = "INFO"  # We don't DEBUG logging here
                 self.kafka.start()
         
        @@ -67,7 +69,8 @@ def start_kafka(self, security_protocol, interbroker_security_protocol):
             @parametrize(acks=-1, topic=TOPIC_REP_THREE)
             @parametrize(acks=1, topic=TOPIC_REP_THREE, num_producers=3)
             @matrix(acks=[1], topic=[TOPIC_REP_THREE], message_size=[10, 100, 1000, 10000, 100000], security_protocol=['PLAINTEXT', 'SSL'])
        -    def test_producer_throughput(self, acks, topic, num_producers=1, message_size=DEFAULT_RECORD_SIZE, security_protocol='PLAINTEXT'):
        +    def test_producer_throughput(self, acks, topic, num_producers=1, message_size=DEFAULT_RECORD_SIZE, security_protocol='PLAINTEXT',
        +                                 client_version=str(TRUNK), broker_version=str(TRUNK)):
                 """
                 Setup: 1 node zk + 3 node kafka cluster
                 Produce ~128MB worth of messages to a topic with 6 partitions. Required acks, topic replication factor,
        @@ -76,13 +79,16 @@ def test_producer_throughput(self, acks, topic, num_producers=1, message_size=DE
                 Collect and return aggregate throughput statistics after all messages have been acknowledged.
                 (This runs ProducerPerformance.java under the hood)
                 """
        -        self.start_kafka(security_protocol, security_protocol)
        +        client_version = KafkaVersion(client_version)
        +        broker_version = KafkaVersion(broker_version)
        +        self.validate_versions(client_version, broker_version)
        +        self.start_kafka(security_protocol, security_protocol, broker_version)
                 # Always generate the same total amount of data
                 nrecords = int(self.target_data_size / message_size)
         
                 self.producer = ProducerPerformanceService(
                     self.test_context, num_producers, self.kafka, topic=topic,
        -            num_records=nrecords, record_size=message_size,  throughput=-1,
        +            num_records=nrecords, record_size=message_size,  throughput=-1, version=client_version,
                     settings={
                         'acks': acks,
                         'batch.size': self.batch_size,
        @@ -92,7 +98,8 @@ def test_producer_throughput(self, acks, topic, num_producers=1, message_size=DE
         
             @parametrize(security_protocol='SSL', interbroker_security_protocol='PLAINTEXT')
             @matrix(security_protocol=['PLAINTEXT', 'SSL'])
        -    def test_long_term_producer_throughput(self, security_protocol, interbroker_security_protocol=None):
        +    def test_long_term_producer_throughput(self, security_protocol, interbroker_security_protocol=None,
        +                                           client_version=str(TRUNK), broker_version=str(TRUNK)):
                 """
                 Setup: 1 node zk + 3 node kafka cluster
                 Produce 10e6 100 byte messages to a topic with 6 partitions, replication-factor 3, and acks=1.
        @@ -101,13 +108,16 @@ def test_long_term_producer_throughput(self, security_protocol, interbroker_secu
         
                 (This runs ProducerPerformance.java under the hood)
                 """
        +        client_version = KafkaVersion(client_version)
        +        broker_version = KafkaVersion(broker_version)
        +        self.validate_versions(client_version, broker_version)
                 if interbroker_security_protocol is None:
                     interbroker_security_protocol = security_protocol
        -        self.start_kafka(security_protocol, interbroker_security_protocol)
        +        self.start_kafka(security_protocol, interbroker_security_protocol, broker_version)
                 self.producer = ProducerPerformanceService(
                     self.test_context, 1, self.kafka,
                     topic=TOPIC_REP_THREE, num_records=self.msgs_large, record_size=DEFAULT_RECORD_SIZE,
        -            throughput=-1, settings={'acks': 1, 'batch.size': self.batch_size, 'buffer.memory': self.buffer_memory},
        +            throughput=-1, version=client_version, settings={'acks': 1, 'batch.size': self.batch_size, 'buffer.memory': self.buffer_memory},
                     intermediate_stats=True
                 )
                 self.producer.run()
        @@ -135,10 +145,10 @@ def test_long_term_producer_throughput(self, security_protocol, interbroker_secu
                 self.logger.info("\n".join(summary))
                 return data
         
        -
             @parametrize(security_protocol='SSL', interbroker_security_protocol='PLAINTEXT')
             @matrix(security_protocol=['PLAINTEXT', 'SSL', 'SASL_PLAINTEXT', 'SASL_SSL'])
        -    def test_end_to_end_latency(self, security_protocol, interbroker_security_protocol=None):
        +    def test_end_to_end_latency(self, security_protocol, interbroker_security_protocol=None,
        +                                client_version=str(TRUNK), broker_version=str(TRUNK)):
                 """
                 Setup: 1 node zk + 3 node kafka cluster
                 Produce (acks = 1) and consume 10e3 messages to a topic with 6 partitions and replication-factor 3,
        @@ -148,13 +158,16 @@ def test_end_to_end_latency(self, security_protocol, interbroker_security_protoc
         
                 (Under the hood, this simply runs EndToEndLatency.scala)
                 """
        +        client_version = KafkaVersion(client_version)
        +        broker_version = KafkaVersion(broker_version)
        +        self.validate_versions(client_version, broker_version)
                 if interbroker_security_protocol is None:
                     interbroker_security_protocol = security_protocol
        -        self.start_kafka(security_protocol, interbroker_security_protocol)
        +        self.start_kafka(security_protocol, interbroker_security_protocol, broker_version)
                 self.logger.info("BENCHMARK: End to end latency")
                 self.perf = EndToEndLatencyService(
                     self.test_context, 1, self.kafka,
        -            topic=TOPIC_REP_THREE, num_records=10000
        +            topic=TOPIC_REP_THREE, num_records=10000, version=client_version
                 )
                 self.perf.run()
                 return latency(self.perf.results[0]['latency_50th_ms'],  self.perf.results[0]['latency_99th_ms'], self.perf.results[0]['latency_999th_ms'])
        @@ -162,7 +175,8 @@ def test_end_to_end_latency(self, security_protocol, interbroker_security_protoc
             @parametrize(security_protocol='PLAINTEXT', new_consumer=False)
             @parametrize(security_protocol='SSL', interbroker_security_protocol='PLAINTEXT')
             @matrix(security_protocol=['PLAINTEXT', 'SSL'])
        -    def test_producer_and_consumer(self, security_protocol, interbroker_security_protocol=None, new_consumer=True):
        +    def test_producer_and_consumer(self, security_protocol, interbroker_security_protocol=None, new_consumer=True,
        +                                   client_version=str(TRUNK), broker_version=str(TRUNK)):
                 """
                 Setup: 1 node zk + 3 node kafka cluster
                 Concurrently produce and consume 10e6 messages with a single producer and a single consumer,
        @@ -172,15 +186,18 @@ def test_producer_and_consumer(self, security_protocol, interbroker_security_pro
         
                 (Under the hood, this runs ProducerPerformance.java, and ConsumerPerformance.scala)
                 """
        +        client_version = KafkaVersion(client_version)
        +        broker_version = KafkaVersion(broker_version)
        +        self.validate_versions(client_version, broker_version)
                 if interbroker_security_protocol is None:
                     interbroker_security_protocol = security_protocol
        -        self.start_kafka(security_protocol, interbroker_security_protocol)
        +        self.start_kafka(security_protocol, interbroker_security_protocol, broker_version)
                 num_records = 10 * 1000 * 1000  # 10e6
         
                 self.producer = ProducerPerformanceService(
                     self.test_context, 1, self.kafka,
                     topic=TOPIC_REP_THREE,
        -            num_records=num_records, record_size=DEFAULT_RECORD_SIZE, throughput=-1,
        +            num_records=num_records, record_size=DEFAULT_RECORD_SIZE, throughput=-1, version=client_version,
                     settings={'acks': 1, 'batch.size': self.batch_size, 'buffer.memory': self.buffer_memory}
                 )
                 self.consumer = ConsumerPerformanceService(
        @@ -200,21 +217,25 @@ def test_producer_and_consumer(self, security_protocol, interbroker_security_pro
             @parametrize(security_protocol='PLAINTEXT', new_consumer=False)
             @parametrize(security_protocol='SSL', interbroker_security_protocol='PLAINTEXT')
             @matrix(security_protocol=['PLAINTEXT', 'SSL'])
        -    def test_consumer_throughput(self, security_protocol, interbroker_security_protocol=None, new_consumer=True, num_consumers=1):
        +    def test_consumer_throughput(self, security_protocol, interbroker_security_protocol=None, new_consumer=True, num_consumers=1,
        +                                 client_version=str(TRUNK), broker_version=str(TRUNK)):
                 """
                 Consume 10e6 100-byte messages with 1 or more consumers from a topic with 6 partitions
                 (using new consumer iff new_consumer == True), and report throughput.
                 """
        +        client_version = KafkaVersion(client_version)
        +        broker_version = KafkaVersion(broker_version)
        +        self.validate_versions(client_version, broker_version)
                 if interbroker_security_protocol is None:
                     interbroker_security_protocol = security_protocol
        -        self.start_kafka(security_protocol, interbroker_security_protocol)
        +        self.start_kafka(security_protocol, interbroker_security_protocol, broker_version)
                 num_records = 10 * 1000 * 1000  # 10e6
         
                 # seed kafka w/messages
                 self.producer = ProducerPerformanceService(
                     self.test_context, 1, self.kafka,
                     topic=TOPIC_REP_THREE,
        -            num_records=num_records, record_size=DEFAULT_RECORD_SIZE, throughput=-1,
        +            num_records=num_records, record_size=DEFAULT_RECORD_SIZE, throughput=-1, version=client_version,
                     settings={'acks': 1, 'batch.size': self.batch_size, 'buffer.memory': self.buffer_memory}
                 )
                 self.producer.run()
        @@ -227,27 +248,5 @@ def test_consumer_throughput(self, security_protocol, interbroker_security_proto
                 self.consumer.run()
                 return compute_aggregate_throughput(self.consumer)
         
        -
        -def throughput(records_per_sec, mb_per_sec):
        -    """Helper method to ensure uniform representation of throughput data"""
        -    return {
        -        "records_per_sec": records_per_sec,
        -        "mb_per_sec": mb_per_sec
        -    }
        -
        -
        -def latency(latency_50th_ms, latency_99th_ms, latency_999th_ms):
        -    """Helper method to ensure uniform representation of latency data"""
        -    return {
        -        "latency_50th_ms": latency_50th_ms,
        -        "latency_99th_ms": latency_99th_ms,
        -        "latency_999th_ms": latency_999th_ms
        -    }
        -
        -
        -def compute_aggregate_throughput(perf):
        -    """Helper method for computing throughput after running a performance service."""
        -    aggregate_rate = sum([r['records_per_sec'] for r in perf.results])
        -    aggregate_mbps = sum([r['mbps'] for r in perf.results])
        -
        -    return throughput(aggregate_rate, aggregate_mbps)
        +    def validate_versions(self, client_version, broker_version):
        +        assert client_version <= broker_version, "Client version %s should be <= than broker version %s" (client_version, broker_version)
        diff --git a/tests/kafkatest/sanity_checks/test_performance_services.py b/tests/kafkatest/sanity_checks/test_performance_services.py
        new file mode 100644
        index 0000000000000..16d5d32132fea
        --- /dev/null
        +++ b/tests/kafkatest/sanity_checks/test_performance_services.py
        @@ -0,0 +1,88 @@
        +# 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.
        +
        +from ducktape.tests.test import Test
        +from ducktape.mark import parametrize
        +
        +from kafkatest.services.zookeeper import ZookeeperService
        +from kafkatest.services.kafka import KafkaService
        +from kafkatest.services.kafka.version import TRUNK, LATEST_0_8_2, LATEST_0_9, KafkaVersion
        +from kafkatest.services.performance import ProducerPerformanceService, ConsumerPerformanceService, EndToEndLatencyService
        +from kafkatest.services.performance import latency, compute_aggregate_throughput
        +
        +
        +class PerformanceServiceTest(Test):
        +    def __init__(self, test_context):
        +        super(PerformanceServiceTest, self).__init__(test_context)
        +        self.record_size = 100
        +        self.num_records = 10000
        +        self.topic = "topic"
        +
        +        self.zk = ZookeeperService(test_context, 1)
        +
        +    def setUp(self):
        +        self.zk.start()
        +
        +    # We are keeping 0.8.2 here so that we don't inadvertently break support for it. Since this is just a sanity check,
        +    # the overhead should be manageable.
        +    @parametrize(version=str(LATEST_0_8_2))
        +    @parametrize(version=str(LATEST_0_9), new_consumer=False)
        +    @parametrize(version=str(LATEST_0_9), new_consumer=True)
        +    @parametrize(version=str(TRUNK), new_consumer=False)
        +    @parametrize(version=str(TRUNK), new_consumer=True)
        +    def test_version(self, version=str(LATEST_0_9), new_consumer=False):
        +        """
        +        Sanity check out producer performance service - verify that we can run the service with a small
        +        number of messages. The actual stats here are pretty meaningless since the number of messages is quite small.
        +        """
        +        version = KafkaVersion(version)
        +        self.kafka = KafkaService(
        +            self.test_context, 1,
        +            self.zk, topics={self.topic: {'partitions': 1, 'replication-factor': 1}}, version=version)
        +        self.kafka.start()
        +
        +        # check basic run of producer performance
        +        self.producer_perf = ProducerPerformanceService(
        +            self.test_context, 1, self.kafka, topic=self.topic,
        +            num_records=self.num_records, record_size=self.record_size,
        +            throughput=1000000000,  # Set impossibly for no throttling for equivalent behavior between 0.8.X and 0.9.X
        +            version=version,
        +            settings={
        +                'acks': 1,
        +                'batch.size': 8*1024,
        +                'buffer.memory': 64*1024*1024})
        +        self.producer_perf.run()
        +        producer_perf_data = compute_aggregate_throughput(self.producer_perf)
        +
        +        # check basic run of end to end latency
        +        self.end_to_end = EndToEndLatencyService(
        +            self.test_context, 1, self.kafka,
        +            topic=self.topic, num_records=self.num_records, version=version)
        +        self.end_to_end.run()
        +        end_to_end_data = latency(self.end_to_end.results[0]['latency_50th_ms'],  self.end_to_end.results[0]['latency_99th_ms'], self.end_to_end.results[0]['latency_999th_ms'])
        +
        +        # check basic run of consumer performance service
        +        self.consumer_perf = ConsumerPerformanceService(
        +            self.test_context, 1, self.kafka, new_consumer=new_consumer,
        +            topic=self.topic, version=version, messages=self.num_records)
        +        self.consumer_perf.group = "test-consumer-group"
        +        self.consumer_perf.run()
        +        consumer_perf_data = compute_aggregate_throughput(self.consumer_perf)
        +
        +        return {
        +            "producer_performance": producer_perf_data,
        +            "end_to_end_latency": end_to_end_data,
        +            "consumer_performance": consumer_perf_data
        +        }
        diff --git a/tests/kafkatest/services/performance/__init__.py b/tests/kafkatest/services/performance/__init__.py
        index a72e3b792bd9f..9eddcaa6dbcc1 100644
        --- a/tests/kafkatest/services/performance/__init__.py
        +++ b/tests/kafkatest/services/performance/__init__.py
        @@ -13,7 +13,7 @@
         # See the License for the specific language governing permissions and
         # limitations under the License.
         
        -from performance import PerformanceService
        +from performance import PerformanceService, throughput, latency, compute_aggregate_throughput
         from end_to_end_latency import EndToEndLatencyService
         from producer_performance import ProducerPerformanceService
        -from consumer_performance import ConsumerPerformanceService
        \ No newline at end of file
        +from consumer_performance import ConsumerPerformanceService
        diff --git a/tests/kafkatest/services/performance/consumer_performance.py b/tests/kafkatest/services/performance/consumer_performance.py
        index f8289bcd31649..def27b15fa6d7 100644
        --- a/tests/kafkatest/services/performance/consumer_performance.py
        +++ b/tests/kafkatest/services/performance/consumer_performance.py
        @@ -14,8 +14,9 @@
         # limitations under the License.
         
         from kafkatest.services.performance import PerformanceService
        -from kafkatest.services.kafka.directory import kafka_dir
         from kafkatest.services.security.security_config import SecurityConfig
        +from kafkatest.services.kafka.directory import kafka_dir
        +from kafkatest.services.kafka.version import TRUNK, V_0_9_0_0
         
         import os
         
        @@ -69,7 +70,7 @@ class ConsumerPerformanceService(PerformanceService):
                     "collect_default": True}
             }
         
        -    def __init__(self, context, num_nodes, kafka, topic, messages, new_consumer=False, settings={}):
        +    def __init__(self, context, num_nodes, kafka, topic, messages, version=TRUNK, new_consumer=False, settings={}):
                 super(ConsumerPerformanceService, self).__init__(context, num_nodes)
                 self.kafka = kafka
                 self.security_config = kafka.security_config.client_config()
        @@ -78,6 +79,13 @@ def __init__(self, context, num_nodes, kafka, topic, messages, new_consumer=Fals
                 self.new_consumer = new_consumer
                 self.settings = settings
         
        +        assert version >= V_0_9_0_0 or (not new_consumer), \
        +            "new_consumer is only supported if version >= 0.9.0.0, version %s" % str(version)
        +
        +        security_protocol = self.security_config.security_protocol
        +        assert version >= V_0_9_0_0 or security_protocol == SecurityConfig.PLAINTEXT, \
        +            "Security protocol %s is only supported if version >= 0.9.0.0, version %s" % (self.security_config, str(version))
        +
                 # These less-frequently used settings can be updated manually after instantiation
                 self.fetch_size = None
                 self.socket_buffer_size = None
        @@ -86,6 +94,9 @@ def __init__(self, context, num_nodes, kafka, topic, messages, new_consumer=Fals
                 self.group = None
                 self.from_latest = None
         
        +        for node in self.nodes:
        +            node.version = version
        +
             @property
             def args(self):
                 """Dictionary of arguments used to start the Consumer Performance script."""
        @@ -127,7 +138,10 @@ def start_cmd(self, node):
                 cmd += " /opt/%s/bin/kafka-consumer-perf-test.sh" % kafka_dir(node)
                 for key, value in self.args.items():
                     cmd += " --%s %s" % (key, value)
        -        cmd += " --consumer.config %s" % ConsumerPerformanceService.CONFIG_FILE
        +
        +        if node.version >= V_0_9_0_0:
        +            # This is only used for security settings
        +            cmd += " --consumer.config %s" % ConsumerPerformanceService.CONFIG_FILE
         
                 for key, value in self.settings.items():
                     cmd += " %s=%s" % (str(key), str(value))
        @@ -136,6 +150,22 @@ def start_cmd(self, node):
                                                                 'stderr': ConsumerPerformanceService.STDERR_CAPTURE}
                 return cmd
         
        +    def parse_results(self, line, version):
        +        parts = line.split(',')
        +        if version >= V_0_9_0_0:
        +            result = {
        +                'total_mb': float(parts[2]),
        +                'mbps': float(parts[3]),
        +                'records_per_sec': float(parts[5]),
        +            }
        +        else:
        +            result = {
        +                'total_mb': float(parts[3]),
        +                'mbps': float(parts[4]),
        +                'records_per_sec': float(parts[6]),
        +            }
        +        return result
        +
             def _worker(self, idx, node):
                 node.account.ssh("mkdir -p %s" % ConsumerPerformanceService.PERSISTENT_ROOT, allow_fail=False)
         
        @@ -149,11 +179,6 @@ def _worker(self, idx, node):
                 last = None
                 for line in node.account.ssh_capture(cmd):
                     last = line
        -        # Parse and save the last line's information
        -        parts = last.split(',')
         
        -        self.results[idx-1] = {
        -            'total_mb': float(parts[2]),
        -            'mbps': float(parts[3]),
        -            'records_per_sec': float(parts[5]),
        -        }
        +        # Parse and save the last line's information
        +        self.results[idx-1] = self.parse_results(last, node.version)
        diff --git a/tests/kafkatest/services/performance/end_to_end_latency.py b/tests/kafkatest/services/performance/end_to_end_latency.py
        index 049eebc5b82ce..08eff70cf1ef8 100644
        --- a/tests/kafkatest/services/performance/end_to_end_latency.py
        +++ b/tests/kafkatest/services/performance/end_to_end_latency.py
        @@ -17,9 +17,11 @@
         from kafkatest.services.security.security_config import SecurityConfig
         
         from kafkatest.services.kafka.directory import kafka_dir
        +from kafkatest.services.kafka.version import TRUNK, V_0_9_0_0
         
         
         class EndToEndLatencyService(PerformanceService):
        +    MESSAGE_BYTES = 21  # 0.8.X messages are fixed at 21 bytes, so we'll match that for other versions
         
             logs = {
                 "end_to_end_latency_log": {
        @@ -27,37 +29,79 @@ class EndToEndLatencyService(PerformanceService):
                     "collect_default": True},
             }
         
        -    def __init__(self, context, num_nodes, kafka, topic, num_records, consumer_fetch_max_wait=100, acks=1):
        +    def __init__(self, context, num_nodes, kafka, topic, num_records, version=TRUNK, consumer_fetch_max_wait=100, acks=1):
                 super(EndToEndLatencyService, self).__init__(context, num_nodes)
                 self.kafka = kafka
                 self.security_config = kafka.security_config.client_config()
        +
        +        security_protocol = self.security_config.security_protocol
        +        assert version >= V_0_9_0_0 or security_protocol == SecurityConfig.PLAINTEXT, \
        +            "Security protocol %s is only supported if version >= 0.9.0.0, version %s" % (self.security_config, str(version))
        +
                 self.args = {
                     'topic': topic,
                     'num_records': num_records,
                     'consumer_fetch_max_wait': consumer_fetch_max_wait,
                     'acks': acks,
        -            'kafka_opts': self.security_config.kafka_opts
        +            'kafka_opts': self.security_config.kafka_opts,
        +            'message_bytes': EndToEndLatencyService.MESSAGE_BYTES
                 }
         
        -    def _worker(self, idx, node):
        -        args = self.args.copy()
        -        self.security_config.setup_node(node)
        +        for node in self.nodes:
        +            node.version = version
        +
        +    @property
        +    def security_config_file(self):
                 if self.security_config.security_protocol != SecurityConfig.PLAINTEXT:
                     security_config_file = SecurityConfig.CONFIG_DIR + "/security.properties"
        -            node.account.create_file(security_config_file, str(self.security_config))
                 else:
                     security_config_file = ""
        +        return security_config_file
        +
        +    def start_cmd(self, node):
        +        args = self.args.copy()
                 args.update({
                     'zk_connect': self.kafka.zk.connect_setting(),
                     'bootstrap_servers': self.kafka.bootstrap_servers(self.security_config.security_protocol),
        -            'security_config_file': security_config_file,
        +            'security_config_file': self.security_config_file,
                     'kafka_dir': kafka_dir(node)
                 })
         
        -        cmd = "KAFKA_OPTS=%(kafka_opts)s /opt/%(kafka_dir)s/bin/kafka-run-class.sh kafka.tools.EndToEndLatency " % args
        -        cmd += "%(bootstrap_servers)s %(topic)s %(num_records)d %(acks)d 20 %(security_config_file)s" % args
        +        if node.version >= V_0_9_0_0:
        +            """
        +            val brokerList = args(0)
        +            val topic = args(1)
        +            val numMessages = args(2).toInt
        +            val producerAcks = args(3)
        +            val messageLen = args(4).toInt
        +            """
        +
        +            cmd = "KAFKA_OPTS=%(kafka_opts)s /opt/%(kafka_dir)s/bin/kafka-run-class.sh kafka.tools.EndToEndLatency " % args
        +            cmd += "%(bootstrap_servers)s %(topic)s %(num_records)d %(acks)d %(message_bytes)d %(security_config_file)s" % args
        +        else:
        +            """
        +            val brokerList = args(0)
        +            val zkConnect = args(1)
        +            val topic = args(2)
        +            val numMessages = args(3).toInt
        +            val consumerFetchMaxWait = args(4).toInt
        +            val producerAcks = args(5).toInt
        +            """
        +
        +            # Set fetch max wait to 0 to match behavior in later versions
        +            cmd = "KAFKA_OPTS=%(kafka_opts)s /opt/%(kafka_dir)s/bin/kafka-run-class.sh kafka.tools.TestEndToEndLatency " % args
        +            cmd += "%(bootstrap_servers)s %(zk_connect)s %(topic)s %(num_records)d 0 %(acks)d" % args
        +
                 cmd += " | tee /mnt/end-to-end-latency.log"
         
        +        return cmd
        +
        +    def _worker(self, idx, node):
        +        self.security_config.setup_node(node)
        +        if self.security_config.security_protocol != SecurityConfig.PLAINTEXT:
        +            node.account.create_file(self.security_config_file, str(self.security_config))
        +
        +        cmd = self.start_cmd(node)
                 self.logger.debug("End-to-end latency %d command: %s", idx, cmd)
                 results = {}
                 for line in node.account.ssh_capture(cmd):
        diff --git a/tests/kafkatest/services/performance/performance.py b/tests/kafkatest/services/performance/performance.py
        index 6d286f60faddf..1eab1976278f5 100644
        --- a/tests/kafkatest/services/performance/performance.py
        +++ b/tests/kafkatest/services/performance/performance.py
        @@ -27,3 +27,26 @@ def clean_node(self, node):
                 node.account.kill_process("java", clean_shutdown=False, allow_fail=True)
                 node.account.ssh("rm -rf /mnt/*", allow_fail=False)
         
        +def throughput(records_per_sec, mb_per_sec):
        +    """Helper method to ensure uniform representation of throughput data"""
        +    return {
        +        "records_per_sec": records_per_sec,
        +        "mb_per_sec": mb_per_sec
        +    }
        +
        +
        +def latency(latency_50th_ms, latency_99th_ms, latency_999th_ms):
        +    """Helper method to ensure uniform representation of latency data"""
        +    return {
        +        "latency_50th_ms": latency_50th_ms,
        +        "latency_99th_ms": latency_99th_ms,
        +        "latency_999th_ms": latency_999th_ms
        +    }
        +
        +
        +def compute_aggregate_throughput(perf):
        +    """Helper method for computing throughput after running a performance service."""
        +    aggregate_rate = sum([r['records_per_sec'] for r in perf.results])
        +    aggregate_mbps = sum([r['mbps'] for r in perf.results])
        +
        +    return throughput(aggregate_rate, aggregate_mbps)
        diff --git a/tests/kafkatest/services/performance/producer_performance.py b/tests/kafkatest/services/performance/producer_performance.py
        index 7cbc7bb11a734..f4887edb8cb0e 100644
        --- a/tests/kafkatest/services/performance/producer_performance.py
        +++ b/tests/kafkatest/services/performance/producer_performance.py
        @@ -13,26 +13,56 @@
         # See the License for the specific language governing permissions and
         # limitations under the License.
         
        +from ducktape.utils.util import wait_until
        +
         from kafkatest.services.monitor.jmx import JmxMixin
         from kafkatest.services.performance import PerformanceService
        -import itertools
         from kafkatest.services.security.security_config import SecurityConfig
        -from kafkatest.services.kafka.directory import kafka_dir
        +from kafkatest.services.kafka.directory import kafka_dir, KAFKA_TRUNK
        +from kafkatest.services.kafka.version import TRUNK, V_0_9_0_0
        +
        +import os
        +import subprocess
        +
         
         class ProducerPerformanceService(JmxMixin, PerformanceService):
         
        -    logs = {
        -        "producer_performance_log": {
        -            "path": "/mnt/producer-performance.log",
        -            "collect_default": True},
        -    }
        +    PERSISTENT_ROOT = "/mnt/producer_performance"
        +    STDOUT_CAPTURE = os.path.join(PERSISTENT_ROOT, "producer_performance.stdout")
        +    STDERR_CAPTURE = os.path.join(PERSISTENT_ROOT, "producer_performance.stderr")
        +    LOG_DIR = os.path.join(PERSISTENT_ROOT, "logs")
        +    LOG_FILE = os.path.join(LOG_DIR, "producer_performance.log")
        +    LOG4J_CONFIG = os.path.join(PERSISTENT_ROOT, "tools-log4j.properties")
         
        -    def __init__(self, context, num_nodes, kafka, topic, num_records, record_size, throughput, settings={},
        +    def __init__(self, context, num_nodes, kafka, topic, num_records, record_size, throughput, version=TRUNK, settings={},
                          intermediate_stats=False, client_id="producer-performance", jmx_object_names=None, jmx_attributes=[]):
                 JmxMixin.__init__(self, num_nodes, jmx_object_names, jmx_attributes)
                 PerformanceService.__init__(self, context, num_nodes)
        +
        +        self.logs = {
        +            "producer_performance_stdout": {
        +                "path": ProducerPerformanceService.STDOUT_CAPTURE,
        +                "collect_default": True},
        +            "producer_performance_stderr": {
        +                "path": ProducerPerformanceService.STDERR_CAPTURE,
        +                "collect_default": True},
        +            "producer_performance_log": {
        +                "path": ProducerPerformanceService.LOG_FILE,
        +                "collect_default": True},
        +            "jmx_log": {
        +                "path": "/mnt/jmx_tool.log",
        +                "collect_default": jmx_object_names is not None
        +            }
        +
        +        }
        +
                 self.kafka = kafka
                 self.security_config = kafka.security_config.client_config()
        +
        +        security_protocol = self.security_config.security_protocol
        +        assert version >= V_0_9_0_0 or security_protocol == SecurityConfig.PLAINTEXT, \
        +            "Security protocol %s is only supported if version >= 0.9.0.0, version %s" % (self.security_config, str(version))
        +
                 self.args = {
                     'topic': topic,
                     'kafka_opts': self.security_config.kafka_opts,
        @@ -44,7 +74,10 @@ def __init__(self, context, num_nodes, kafka, topic, num_records, record_size, t
                 self.intermediate_stats = intermediate_stats
                 self.client_id = client_id
         
        -    def _worker(self, idx, node):
        +        for node in self.nodes:
        +            node.version = version
        +
        +    def start_cmd(self, node):
                 args = self.args.copy()
                 args.update({
                     'bootstrap_servers': self.kafka.bootstrap_servers(self.security_config.security_protocol),
        @@ -52,48 +85,92 @@ def _worker(self, idx, node):
                     'client_id': self.client_id,
                     'kafka_directory': kafka_dir(node)
                     })
        -        cmd = "JMX_PORT=%(jmx_port)d KAFKA_OPTS=%(kafka_opts)s /opt/%(kafka_directory)s/bin/kafka-run-class.sh org.apache.kafka.tools.ProducerPerformance " \
        +
        +        cmd = ""
        +
        +        if node.version < TRUNK:
        +            # In order to ensure more consistent configuration between versions, always use the ProducerPerformance
        +            # tool from trunk
        +            cmd += "for file in /opt/%s/tools/build/libs/kafka-tools*.jar; do CLASSPATH=$CLASSPATH:$file; done; " % KAFKA_TRUNK
        +            cmd += "for file in /opt/%s/tools/build/dependant-libs-${SCALA_VERSION}*/*.jar; do CLASSPATH=$CLASSPATH:$file; done; " % KAFKA_TRUNK
        +            cmd += "export CLASSPATH; "
        +
        +        cmd += " export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\"; " % ProducerPerformanceService.LOG4J_CONFIG
        +        cmd += "JMX_PORT=%(jmx_port)d KAFKA_OPTS=%(kafka_opts)s /opt/%(kafka_directory)s/bin/kafka-run-class.sh org.apache.kafka.tools.ProducerPerformance " \
                       "--topic %(topic)s --num-records %(num_records)d --record-size %(record_size)d --throughput %(throughput)d --producer-props bootstrap.servers=%(bootstrap_servers)s client.id=%(client_id)s" % args
         
                 self.security_config.setup_node(node)
                 if self.security_config.security_protocol != SecurityConfig.PLAINTEXT:
                     self.settings.update(self.security_config.properties)
        +
                 for key, value in self.settings.items():
                     cmd += " %s=%s" % (str(key), str(value))
        -        cmd += " | tee /mnt/producer-performance.log"
         
        +        cmd += " 2>>%s | tee %s" % (ProducerPerformanceService.STDERR_CAPTURE, ProducerPerformanceService.STDOUT_CAPTURE)
        +        return cmd
        +
        +    def pids(self, node):
        +        try:
        +            cmd = "jps | grep -i ProducerPerformance | awk '{print $1}'"
        +            pid_arr = [pid for pid in node.account.ssh_capture(cmd, allow_fail=True, callback=int)]
        +            return pid_arr
        +        except (subprocess.CalledProcessError, ValueError) as e:
        +            return []
        +
        +    def alive(self, node):
        +        return len(self.pids(node)) > 0
        +
        +    def _worker(self, idx, node):
        +
        +        node.account.ssh("mkdir -p %s" % ProducerPerformanceService.PERSISTENT_ROOT, allow_fail=False)
        +
        +        # Create and upload log properties
        +        log_config = self.render('tools_log4j.properties', log_file=ProducerPerformanceService.LOG_FILE)
        +        node.account.create_file(ProducerPerformanceService.LOG4J_CONFIG, log_config)
        +
        +        cmd = self.start_cmd(node)
                 self.logger.debug("Producer performance %d command: %s", idx, cmd)
         
        -        def parse_stats(line):
        -            parts = line.split(',')
        -            return {
        -                'records': int(parts[0].split()[0]),
        -                'records_per_sec': float(parts[1].split()[0]),
        -                'mbps': float(parts[1].split('(')[1].split()[0]),
        -                'latency_avg_ms': float(parts[2].split()[0]),
        -                'latency_max_ms': float(parts[3].split()[0]),
        -                'latency_50th_ms': float(parts[4].split()[0]),
        -                'latency_95th_ms': float(parts[5].split()[0]),
        -                'latency_99th_ms': float(parts[6].split()[0]),
        -                'latency_999th_ms': float(parts[7].split()[0]),
        -            }
        -        last = None
        +        # start ProducerPerformance process
                 producer_output = node.account.ssh_capture(cmd)
        +        wait_until(lambda: self.alive(node), timeout_sec=20, err_msg="ProducerPerformance failed to start")
        +        # block until there is at least one line of output
                 first_line = next(producer_output, None)
        +        if first_line is None:
        +            raise Exception("No output from ProducerPerformance")
        +
        +        self.start_jmx_tool(idx, node)
        +        wait_until(lambda: not self.alive(node), timeout_sec=1200, err_msg="ProducerPerformance failed to finish")
        +        self.read_jmx_output(idx, node)
        +
        +        # parse producer output from file
        +        last = None
        +        producer_output = node.account.ssh_capture("cat %s" % ProducerPerformanceService.STDOUT_CAPTURE)
        +        for line in producer_output:
        +            if self.intermediate_stats:
        +                try:
        +                    self.stats[idx-1].append(self.parse_stats(line))
        +                except:
        +                    # Sometimes there are extraneous log messages
        +                    pass
         
        -        if first_line is not None:
        -            self.start_jmx_tool(idx, node)
        -            for line in itertools.chain([first_line], producer_output):
        -                if self.intermediate_stats:
        -                    try:
        -                        self.stats[idx-1].append(parse_stats(line))
        -                    except:
        -                        # Sometimes there are extraneous log messages
        -                        pass
        -
        -                last = line
        -            try:
        -                self.results[idx-1] = parse_stats(last)
        -            except:
        -                raise Exception("Unable to parse aggregate performance statistics on node %d: %s" % (idx, last))
        -            self.read_jmx_output(idx, node)
        +            last = line
        +        try:
        +            self.results[idx-1] = self.parse_stats(last)
        +        except:
        +            raise Exception("Unable to parse aggregate performance statistics on node %d: %s" % (idx, last))
        +
        +    def parse_stats(self, line):
        +
        +        parts = line.split(',')
        +        return {
        +            'records': int(parts[0].split()[0]),
        +            'records_per_sec': float(parts[1].split()[0]),
        +            'mbps': float(parts[1].split('(')[1].split()[0]),
        +            'latency_avg_ms': float(parts[2].split()[0]),
        +            'latency_max_ms': float(parts[3].split()[0]),
        +            'latency_50th_ms': float(parts[4].split()[0]),
        +            'latency_95th_ms': float(parts[5].split()[0]),
        +            'latency_99th_ms': float(parts[6].split()[0]),
        +            'latency_999th_ms': float(parts[7].split()[0]),
        +        }
        diff --git a/tests/kafkatest/services/templates/tools_log4j.properties b/tests/kafkatest/services/templates/tools_log4j.properties
        index 6fec1d60598f2..55ae4e04e4031 100644
        --- a/tests/kafkatest/services/templates/tools_log4j.properties
        +++ b/tests/kafkatest/services/templates/tools_log4j.properties
        @@ -22,4 +22,4 @@ log4j.appender.FILE.ImmediateFlush=true
         # Set the append to true
         log4j.appender.FILE.Append=true
         log4j.appender.FILE.layout=org.apache.log4j.PatternLayout
        -log4j.appender.FILE.layout.conversionPattern=[%d] %p %m (%c)%n
        \ No newline at end of file
        +log4j.appender.FILE.layout.conversionPattern=[%d] %p %m (%c)%n
        diff --git a/vagrant/base.sh b/vagrant/base.sh
        index d271f87ec5f84..da7737cf14d90 100644
        --- a/vagrant/base.sh
        +++ b/vagrant/base.sh
        @@ -63,7 +63,9 @@ get_kafka() {
         }
         
         get_kafka 0.8.2.2
        +chmod a+rw /opt/kafka-0.8.2.2
         get_kafka 0.9.0.1
        +chmod a+rw /opt/kafka-0.9.0.1
         
         # For EC2 nodes, we want to use /mnt, which should have the local disk. On local
         # VMs, we can just create it if it doesn't exist and use it like we'd use
        
        From f4d3d2865894f1a1ade9d92ac27931fd35d16cae Mon Sep 17 00:00:00 2001
        From: Jason Gustafson 
        Date: Wed, 13 Apr 2016 18:09:08 -0700
        Subject: [PATCH 143/206] KAFKA-3470: treat commits as member heartbeats
        
        Author: Jason Gustafson 
        
        Reviewers: Ismael Juma , Guozhang Wang 
        
        Closes #1206 from hachikuji/KAFKA-3470
        ---
         .../kafka/coordinator/GroupCoordinator.scala  |  22 +++-
         .../scala/kafka/server/DelayedOperation.scala |  68 ++++++----
         .../scala/kafka/server/ReplicaManager.scala   |   4 +-
         .../main/scala/kafka/utils/timer/Timer.scala  |  51 +++++++-
         .../scala/kafka/utils/timer/TimerTask.scala   |   2 +-
         .../kafka/utils/timer/TimerTaskList.scala     |   7 +-
         .../scala/kafka/utils/timer/TimingWheel.scala |   2 +-
         .../kafka/TestPurgatoryPerformance.scala      |   2 +-
         .../GroupCoordinatorResponseTest.scala        | 121 ++++++++++++++++--
         .../kafka/server/DelayedOperationTest.scala   |   2 +-
         .../scala/unit/kafka/utils/MockTime.scala     |  12 ++
         .../unit/kafka/utils/timer/MockTimer.scala    |  57 +++++++++
         .../kafka/utils/timer/TimerTaskListTest.scala |  12 +-
         .../unit/kafka/utils/timer/TimerTest.scala    |  29 ++---
         14 files changed, 312 insertions(+), 79 deletions(-)
         create mode 100644 core/src/test/scala/unit/kafka/utils/timer/MockTimer.scala
        
        diff --git a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
        index 30a3a78f6b131..fb712541ec87c 100644
        --- a/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
        +++ b/core/src/main/scala/kafka/coordinator/GroupCoordinator.scala
        @@ -51,6 +51,8 @@ class GroupCoordinator(val brokerId: Int,
                                val groupConfig: GroupConfig,
                                val offsetConfig: OffsetConfig,
                                val groupManager: GroupMetadataManager,
        +                       val heartbeatPurgatory: DelayedOperationPurgatory[DelayedHeartbeat],
        +                       val joinPurgatory: DelayedOperationPurgatory[DelayedJoin],
                                time: Time) extends Logging {
           type JoinCallback = JoinGroupResult => Unit
           type SyncCallback = (Array[Byte], Short) => Unit
        @@ -59,9 +61,6 @@ class GroupCoordinator(val brokerId: Int,
         
           private val isActive = new AtomicBoolean(false)
         
        -  private var heartbeatPurgatory: DelayedOperationPurgatory[DelayedHeartbeat] = null
        -  private var joinPurgatory: DelayedOperationPurgatory[DelayedJoin] = null
        -
           def offsetsTopicConfigs: Properties = {
             val props = new Properties
             props.put(LogConfig.CleanupPolicyProp, LogConfig.Compact)
        @@ -80,8 +79,6 @@ class GroupCoordinator(val brokerId: Int,
            */
           def startup() {
             info("Starting up.")
        -    heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", brokerId)
        -    joinPurgatory = new DelayedOperationPurgatory[DelayedJoin]("Rebalance", brokerId)
             isActive.set(true)
             info("Startup complete.")
           }
        @@ -414,6 +411,8 @@ class GroupCoordinator(val brokerId: Int,
                   } else if (generationId != group.generationId) {
                     responseCallback(offsetMetadata.mapValues(_ => Errors.ILLEGAL_GENERATION.code))
                   } else {
        +            val member = group.get(memberId)
        +            completeAndScheduleNextHeartbeatExpiration(group, member)
                     delayedOffsetStore = Some(groupManager.prepareStoreOffsets(groupId, memberId, generationId,
                       offsetMetadata, responseCallback))
                   }
        @@ -729,6 +728,17 @@ object GroupCoordinator {
                     zkUtils: ZkUtils,
                     replicaManager: ReplicaManager,
                     time: Time): GroupCoordinator = {
        +    val heartbeatPurgatory = DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", config.brokerId)
        +    val joinPurgatory = DelayedOperationPurgatory[DelayedJoin]("Rebalance", config.brokerId)
        +    apply(config, zkUtils, replicaManager, heartbeatPurgatory, joinPurgatory, time)
        +  }
        +
        +  def apply(config: KafkaConfig,
        +            zkUtils: ZkUtils,
        +            replicaManager: ReplicaManager,
        +            heartbeatPurgatory: DelayedOperationPurgatory[DelayedHeartbeat],
        +            joinPurgatory: DelayedOperationPurgatory[DelayedJoin],
        +            time: Time): GroupCoordinator = {
             val offsetConfig = OffsetConfig(maxMetadataSize = config.offsetMetadataMaxSize,
               loadBufferSize = config.offsetsLoadBufferSize,
               offsetsRetentionMs = config.offsetsRetentionMinutes * 60 * 1000L,
        @@ -741,7 +751,7 @@ object GroupCoordinator {
               groupMaxSessionTimeoutMs = config.groupMaxSessionTimeoutMs)
         
             val groupManager = new GroupMetadataManager(config.brokerId, offsetConfig, replicaManager, zkUtils, time)
        -    new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, groupManager, time)
        +    new GroupCoordinator(config.brokerId, groupConfig, offsetConfig, groupManager, heartbeatPurgatory, joinPurgatory, time)
           }
         
         }
        diff --git a/core/src/main/scala/kafka/server/DelayedOperation.scala b/core/src/main/scala/kafka/server/DelayedOperation.scala
        index 0b535329364e2..220556891584f 100644
        --- a/core/src/main/scala/kafka/server/DelayedOperation.scala
        +++ b/core/src/main/scala/kafka/server/DelayedOperation.scala
        @@ -47,9 +47,7 @@ import com.yammer.metrics.core.Gauge
          *
          * A subclass of DelayedOperation needs to provide an implementation of both onComplete() and tryComplete().
          */
        -abstract class DelayedOperation(delayMs: Long) extends TimerTask with Logging {
        -
        -  override val expirationMs = delayMs + System.currentTimeMillis()
        +abstract class DelayedOperation(override val delayMs: Long) extends TimerTask with Logging {
         
           private val completed = new AtomicBoolean(false)
         
        @@ -110,19 +108,27 @@ abstract class DelayedOperation(delayMs: Long) extends TimerTask with Logging {
           }
         }
         
        +object DelayedOperationPurgatory {
        +
        +  def apply[T <: DelayedOperation](purgatoryName: String,
        +                                   brokerId: Int = 0,
        +                                   purgeInterval: Int = 1000): DelayedOperationPurgatory[T] = {
        +    val timer = new SystemTimer(purgatoryName)
        +    new DelayedOperationPurgatory[T](purgatoryName, timer, brokerId, purgeInterval)
        +  }
        +
        +}
        +
         /**
          * A helper purgatory class for bookkeeping delayed operations with a timeout, and expiring timed out operations.
          */
        -class DelayedOperationPurgatory[T <: DelayedOperation](purgatoryName: String, brokerId: Int = 0, purgeInterval: Int = 1000)
        +class DelayedOperationPurgatory[T <: DelayedOperation](purgatoryName: String,
        +                                                       timeoutTimer: Timer,
        +                                                       brokerId: Int = 0,
        +                                                       purgeInterval: Int = 1000,
        +                                                       reaperEnabled: Boolean = true)
                 extends Logging with KafkaMetricsGroup {
         
        -  // timeout timer
        -  private[this] val executor = Executors.newFixedThreadPool(1, new ThreadFactory() {
        -    def newThread(runnable: Runnable): Thread =
        -      Utils.newThread("executor-"+purgatoryName, runnable, false)
        -  })
        -  private[this] val timeoutTimer = new Timer(executor)
        -
           /* a list of operation watching keys */
           private val watchersForKey = new Pool[Any, Watchers](Some((key: Any) => new Watchers(key)))
         
        @@ -152,7 +158,8 @@ class DelayedOperationPurgatory[T <: DelayedOperation](purgatoryName: String, br
             metricsTags
           )
         
        -  expirationReaper.start()
        +  if (reaperEnabled)
        +    expirationReaper.start()
         
           /**
            * Check if the operation can be completed, if not watch it based on the given watch keys
        @@ -275,8 +282,9 @@ class DelayedOperationPurgatory[T <: DelayedOperation](purgatoryName: String, br
            * Shutdown the expire reaper thread
            */
           def shutdown() {
        -    expirationReaper.shutdown()
        -    executor.shutdown()
        +    if (reaperEnabled)
        +      expirationReaper.shutdown()
        +    timeoutTimer.shutdown()
           }
         
           /**
        @@ -338,6 +346,23 @@ class DelayedOperationPurgatory[T <: DelayedOperation](purgatoryName: String, br
             }
           }
         
        +  def advanceClock(timeoutMs: Long) {
        +    timeoutTimer.advanceClock(timeoutMs)
        +
        +    // Trigger a purge if the number of completed but still being watched operations is larger than
        +    // the purge threshold. That number is computed by the difference btw the estimated total number of
        +    // operations and the number of pending delayed operations.
        +    if (estimatedTotalOperations.get - delayed > purgeInterval) {
        +      // now set estimatedTotalOperations to delayed (the number of pending operations) since we are going to
        +      // clean up watchers. Note that, if more operations are completed during the clean up, we may end up with
        +      // a little overestimated total number of operations.
        +      estimatedTotalOperations.getAndSet(delayed)
        +      debug("Begin purging watch lists")
        +      val purged = allWatchers.map(_.purgeCompleted()).sum
        +      debug("Purged %d elements from watch lists.".format(purged))
        +    }
        +  }
        +
           /**
            * A background reaper to expire delayed operations that have timed out
            */
        @@ -346,20 +371,7 @@ class DelayedOperationPurgatory[T <: DelayedOperation](purgatoryName: String, br
             false) {
         
             override def doWork() {
        -      timeoutTimer.advanceClock(200L)
        -
        -      // Trigger a purge if the number of completed but still being watched operations is larger than
        -      // the purge threshold. That number is computed by the difference btw the estimated total number of
        -      // operations and the number of pending delayed operations.
        -      if (estimatedTotalOperations.get - delayed > purgeInterval) {
        -        // now set estimatedTotalOperations to delayed (the number of pending operations) since we are going to
        -        // clean up watchers. Note that, if more operations are completed during the clean up, we may end up with
        -        // a little overestimated total number of operations.
        -        estimatedTotalOperations.getAndSet(delayed)
        -        debug("Begin purging watch lists")
        -        val purged = allWatchers.map(_.purgeCompleted()).sum
        -        debug("Purged %d elements from watch lists.".format(purged))
        -      }
        +      advanceClock(200L)
             }
           }
         }
        diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala
        index 22657f4c2a8cf..9bbd29eaad307 100644
        --- a/core/src/main/scala/kafka/server/ReplicaManager.scala
        +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala
        @@ -122,9 +122,9 @@ class ReplicaManager(val config: KafkaConfig,
           private val lastIsrChangeMs = new AtomicLong(System.currentTimeMillis())
           private val lastIsrPropagationMs = new AtomicLong(System.currentTimeMillis())
         
        -  val delayedProducePurgatory = new DelayedOperationPurgatory[DelayedProduce](
        +  val delayedProducePurgatory = DelayedOperationPurgatory[DelayedProduce](
             purgatoryName = "Produce", config.brokerId, config.producerPurgatoryPurgeIntervalRequests)
        -  val delayedFetchPurgatory = new DelayedOperationPurgatory[DelayedFetch](
        +  val delayedFetchPurgatory = DelayedOperationPurgatory[DelayedFetch](
             purgatoryName = "Fetch", config.brokerId, config.fetchPurgatoryPurgeIntervalRequests)
         
           val leaderCount = newGauge(
        diff --git a/core/src/main/scala/kafka/utils/timer/Timer.scala b/core/src/main/scala/kafka/utils/timer/Timer.scala
        index bdd0e75b93e4b..2d7866521c870 100644
        --- a/core/src/main/scala/kafka/utils/timer/Timer.scala
        +++ b/core/src/main/scala/kafka/utils/timer/Timer.scala
        @@ -16,14 +16,52 @@
          */
         package kafka.utils.timer
         
        -import java.util.concurrent.{DelayQueue, ExecutorService, TimeUnit}
        +import java.util.concurrent.{DelayQueue, Executors, ThreadFactory, TimeUnit}
         import java.util.concurrent.atomic.AtomicInteger
         import java.util.concurrent.locks.ReentrantReadWriteLock
         
         import kafka.utils.threadsafe
        +import org.apache.kafka.common.utils.Utils
        +
        +trait Timer {
        +  /**
        +    * Add a new task to this executor. It will be executed after the task's delay
        +    * (beginning from the time of submission)
        +    * @param timerTask the task to add
        +    */
        +  def add(timerTask: TimerTask): Unit
        +
        +  /**
        +    * Advance the internal clock, executing any tasks whose expiration has been
        +    * reached within the duration of the passed timeout.
        +    * @param timeoutMs
        +    * @return whether or not any tasks were executed
        +    */
        +  def advanceClock(timeoutMs: Long): Boolean
        +
        +  /**
        +    * Get the number of tasks pending execution
        +    * @return the number of tasks
        +    */
        +  def size: Int
        +
        +  /**
        +    * Shutdown the timer service, leaving pending tasks unexecuted
        +    */
        +  def shutdown(): Unit
        +}
         
         @threadsafe
        -class Timer(taskExecutor: ExecutorService, tickMs: Long = 1, wheelSize: Int = 20, startMs: Long = System.currentTimeMillis) {
        +class SystemTimer(executorName: String,
        +                  tickMs: Long = 1,
        +                  wheelSize: Int = 20,
        +                  startMs: Long = System.currentTimeMillis) extends Timer {
        +
        +  // timeout timer
        +  private[this] val taskExecutor = Executors.newFixedThreadPool(1, new ThreadFactory() {
        +    def newThread(runnable: Runnable): Thread =
        +      Utils.newThread("executor-"+executorName, runnable, false)
        +  })
         
           private[this] val delayQueue = new DelayQueue[TimerTaskList]()
           private[this] val taskCounter = new AtomicInteger(0)
        @@ -43,7 +81,7 @@ class Timer(taskExecutor: ExecutorService, tickMs: Long = 1, wheelSize: Int = 20
           def add(timerTask: TimerTask): Unit = {
             readLock.lock()
             try {
        -      addTimerTaskEntry(new TimerTaskEntry(timerTask))
        +      addTimerTaskEntry(new TimerTaskEntry(timerTask, timerTask.delayMs + System.currentTimeMillis()))
             } finally {
               readLock.unlock()
             }
        @@ -82,6 +120,11 @@ class Timer(taskExecutor: ExecutorService, tickMs: Long = 1, wheelSize: Int = 20
             }
           }
         
        -  def size(): Int = taskCounter.get
        +  def size: Int = taskCounter.get
        +
        +  override def shutdown() {
        +    taskExecutor.shutdown()
        +  }
        +
         }
         
        diff --git a/core/src/main/scala/kafka/utils/timer/TimerTask.scala b/core/src/main/scala/kafka/utils/timer/TimerTask.scala
        index d6b3a2eb3f70a..66238540d34d6 100644
        --- a/core/src/main/scala/kafka/utils/timer/TimerTask.scala
        +++ b/core/src/main/scala/kafka/utils/timer/TimerTask.scala
        @@ -18,7 +18,7 @@ package kafka.utils.timer
         
         trait TimerTask extends Runnable {
         
        -  val expirationMs: Long // timestamp in millisecond
        +  val delayMs: Long // timestamp in millisecond
         
           private[this] var timerTaskEntry: TimerTaskEntry = null
         
        diff --git a/core/src/main/scala/kafka/utils/timer/TimerTaskList.scala b/core/src/main/scala/kafka/utils/timer/TimerTaskList.scala
        index c4aeb5d20f1e3..e862f4f11991a 100644
        --- a/core/src/main/scala/kafka/utils/timer/TimerTaskList.scala
        +++ b/core/src/main/scala/kafka/utils/timer/TimerTaskList.scala
        @@ -29,7 +29,7 @@ private[timer] class TimerTaskList(taskCounter: AtomicInteger) extends Delayed {
           // TimerTaskList forms a doubly linked cyclic list using a dummy root entry
           // root.next points to the head
           // root.prev points to the tail
        -  private[this] val root = new TimerTaskEntry(null)
        +  private[this] val root = new TimerTaskEntry(null, -1)
           root.next = root
           root.prev = root
         
        @@ -131,7 +131,7 @@ private[timer] class TimerTaskList(taskCounter: AtomicInteger) extends Delayed {
         
         }
         
        -private[timer] class TimerTaskEntry(val timerTask: TimerTask) {
        +private[timer] class TimerTaskEntry(val timerTask: TimerTask, val expirationMs: Long) extends Ordered[TimerTaskEntry] {
         
           @volatile
           var list: TimerTaskList = null
        @@ -157,5 +157,8 @@ private[timer] class TimerTaskEntry(val timerTask: TimerTask) {
             }
           }
         
        +  override def compare(that: TimerTaskEntry): Int = {
        +    this.expirationMs compare that.expirationMs
        +  }
         }
         
        diff --git a/core/src/main/scala/kafka/utils/timer/TimingWheel.scala b/core/src/main/scala/kafka/utils/timer/TimingWheel.scala
        index f5b6efe29349a..4535f3f3e8e17 100644
        --- a/core/src/main/scala/kafka/utils/timer/TimingWheel.scala
        +++ b/core/src/main/scala/kafka/utils/timer/TimingWheel.scala
        @@ -123,7 +123,7 @@ private[timer] class TimingWheel(tickMs: Long, wheelSize: Int, startMs: Long, ta
           }
         
           def add(timerTaskEntry: TimerTaskEntry): Boolean = {
        -    val expiration = timerTaskEntry.timerTask.expirationMs
        +    val expiration = timerTaskEntry.expirationMs
         
             if (timerTaskEntry.cancelled) {
               // Cancelled
        diff --git a/core/src/test/scala/other/kafka/TestPurgatoryPerformance.scala b/core/src/test/scala/other/kafka/TestPurgatoryPerformance.scala
        index 744be3bed0633..ba89fc882fc12 100644
        --- a/core/src/test/scala/other/kafka/TestPurgatoryPerformance.scala
        +++ b/core/src/test/scala/other/kafka/TestPurgatoryPerformance.scala
        @@ -95,7 +95,7 @@ object TestPurgatoryPerformance {
             val latencySamples = new LatencySamples(1000000, pct75, pct50)
             val intervalSamples = new IntervalSamples(1000000, requestRate)
         
        -    val purgatory = new DelayedOperationPurgatory[FakeOperation]("fake purgatory")
        +    val purgatory = DelayedOperationPurgatory[FakeOperation]("fake purgatory")
             val queue = new CompletionQueue()
         
             val gcNames = gcMXBeans.map(_.getName)
        diff --git a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
        index acdb660ca687e..beab1b542e173 100644
        --- a/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
        +++ b/core/src/test/scala/unit/kafka/coordinator/GroupCoordinatorResponseTest.scala
        @@ -17,14 +17,14 @@
         
         package kafka.coordinator
         
        +import kafka.utils.timer.MockTimer
         import org.apache.kafka.common.record.Record
         import org.junit.Assert._
        -import kafka.common.{OffsetAndMetadata, TopicAndPartition}
        +import kafka.common.OffsetAndMetadata
         import kafka.message.{Message, MessageSet}
        -import kafka.server.{ReplicaManager, KafkaConfig}
        +import kafka.server.{DelayedOperationPurgatory, ReplicaManager, KafkaConfig}
         import kafka.utils._
        -import org.apache.kafka.common.utils.SystemTime
        -import org.apache.kafka.common.TopicPartition
        +import org.apache.kafka.common.{utils, TopicPartition}
         import org.apache.kafka.common.protocol.Errors
         import org.apache.kafka.common.requests.{OffsetCommitRequest, JoinGroupRequest}
         import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse
        @@ -56,6 +56,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
           val ConsumerMinSessionTimeout = 10
           val ConsumerMaxSessionTimeout = 1000
           val DefaultSessionTimeout = 500
        +  var timer: MockTimer = null
           var groupCoordinator: GroupCoordinator = null
           var replicaManager: ReplicaManager = null
           var scheduler: KafkaScheduler = null
        @@ -87,7 +88,14 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
             EasyMock.expect(zkUtils.getPartitionAssignmentForTopics(Seq(TopicConstants.GROUP_METADATA_TOPIC_NAME))).andReturn(ret)
             EasyMock.replay(zkUtils)
         
        -    groupCoordinator = GroupCoordinator(KafkaConfig.fromProps(props), zkUtils, replicaManager, new SystemTime)
        +    timer = new MockTimer
        +
        +    val config = KafkaConfig.fromProps(props)
        +
        +    val heartbeatPurgatory = new DelayedOperationPurgatory[DelayedHeartbeat]("Heartbeat", timer, config.brokerId, reaperEnabled = false)
        +    val joinPurgatory = new DelayedOperationPurgatory[DelayedJoin]("Rebalance", timer, config.brokerId, reaperEnabled = false)
        +
        +    groupCoordinator = GroupCoordinator(config, zkUtils, replicaManager, heartbeatPurgatory, joinPurgatory, timer.time)
             groupCoordinator.startup()
         
             // add the partition into the owned partition list
        @@ -283,6 +291,90 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
             assertEquals(Errors.NONE.code, heartbeatResult)
           }
         
        +  @Test
        +  def testSessionTimeout() {
        +    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
        +
        +    val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols)
        +    val assignedConsumerId = joinGroupResult.memberId
        +    val generationId = joinGroupResult.generationId
        +    val joinGroupErrorCode = joinGroupResult.errorCode
        +    assertEquals(Errors.NONE.code, joinGroupErrorCode)
        +
        +    EasyMock.reset(replicaManager)
        +    val (_, syncGroupErrorCode) = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()))
        +    assertEquals(Errors.NONE.code, syncGroupErrorCode)
        +
        +    EasyMock.reset(replicaManager)
        +    EasyMock.expect(replicaManager.getPartition(TopicConstants.GROUP_METADATA_TOPIC_NAME, groupPartitionId)).andReturn(None)
        +    EasyMock.expect(replicaManager.getMessageFormatVersion(EasyMock.anyObject())).andReturn(Some(Message.MagicValue_V1)).anyTimes()
        +    EasyMock.replay(replicaManager)
        +
        +    timer.advanceClock(DefaultSessionTimeout + 100)
        +
        +    EasyMock.reset(replicaManager)
        +    val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1)
        +    assertEquals(Errors.UNKNOWN_MEMBER_ID.code, heartbeatResult)
        +  }
        +
        +  @Test
        +  def testHeartbeatMaintainsSession() {
        +    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
        +    val sessionTimeout = 1000
        +
        +    val joinGroupResult = joinGroup(groupId, memberId, sessionTimeout, protocolType, protocols)
        +    val assignedConsumerId = joinGroupResult.memberId
        +    val generationId = joinGroupResult.generationId
        +    val joinGroupErrorCode = joinGroupResult.errorCode
        +    assertEquals(Errors.NONE.code, joinGroupErrorCode)
        +
        +    EasyMock.reset(replicaManager)
        +    val (_, syncGroupErrorCode) = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()))
        +    assertEquals(Errors.NONE.code, syncGroupErrorCode)
        +
        +    timer.advanceClock(sessionTimeout / 2)
        +
        +    EasyMock.reset(replicaManager)
        +    var heartbeatResult = heartbeat(groupId, assignedConsumerId, 1)
        +    assertEquals(Errors.NONE.code, heartbeatResult)
        +
        +    timer.advanceClock(sessionTimeout / 2 + 100)
        +
        +    EasyMock.reset(replicaManager)
        +    heartbeatResult = heartbeat(groupId, assignedConsumerId, 1)
        +    assertEquals(Errors.NONE.code, heartbeatResult)
        +  }
        +
        +  @Test
        +  def testCommitMaintainsSession() {
        +    val sessionTimeout = 1000
        +    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
        +    val tp = new TopicPartition("topic", 0)
        +    val offset = OffsetAndMetadata(0)
        +
        +    val joinGroupResult = joinGroup(groupId, memberId, sessionTimeout, protocolType, protocols)
        +    val assignedConsumerId = joinGroupResult.memberId
        +    val generationId = joinGroupResult.generationId
        +    val joinGroupErrorCode = joinGroupResult.errorCode
        +    assertEquals(Errors.NONE.code, joinGroupErrorCode)
        +
        +    EasyMock.reset(replicaManager)
        +    val (_, syncGroupErrorCode) = syncGroupLeader(groupId, generationId, assignedConsumerId, Map(assignedConsumerId -> Array[Byte]()))
        +    assertEquals(Errors.NONE.code, syncGroupErrorCode)
        +
        +    timer.advanceClock(sessionTimeout / 2)
        +
        +    EasyMock.reset(replicaManager)
        +    val commitOffsetResult = commitOffsets(groupId, assignedConsumerId, generationId, immutable.Map(tp -> offset))
        +    assertEquals(Errors.NONE.code, commitOffsetResult(tp))
        +
        +    timer.advanceClock(sessionTimeout / 2 + 100)
        +
        +    EasyMock.reset(replicaManager)
        +    val heartbeatResult = heartbeat(groupId, assignedConsumerId, 1)
        +    assertEquals(Errors.NONE.code, heartbeatResult)
        +  }
        +
           @Test
           def testSyncGroupEmptyAssignment() {
             val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
        @@ -459,7 +551,10 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
             // with no leader SyncGroup, the follower's request should failure with an error indicating
             // that it should rejoin
             EasyMock.reset(replicaManager)
        -    val followerSyncFuture= sendSyncGroupFollower(groupId, nextGenerationId, otherJoinResult.memberId)
        +    val followerSyncFuture = sendSyncGroupFollower(groupId, nextGenerationId, otherJoinResult.memberId)
        +
        +    timer.advanceClock(DefaultSessionTimeout + 100)
        +
             val followerSyncResult = await(followerSyncFuture, DefaultSessionTimeout+100)
             assertEquals(Errors.REBALANCE_IN_PROGRESS.code, followerSyncResult._2)
           }
        @@ -628,17 +723,20 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
         
           @Test
           def testGenerationIdIncrementsOnRebalance() {
        -    val memberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
        -    val otherMemberId = JoinGroupRequest.UNKNOWN_MEMBER_ID
        -
        -    val joinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols)
        +    val joinGroupResult = joinGroup(groupId, JoinGroupRequest.UNKNOWN_MEMBER_ID, DefaultSessionTimeout, protocolType, protocols)
             val initialGenerationId = joinGroupResult.generationId
             val joinGroupErrorCode = joinGroupResult.errorCode
        +    val memberId = joinGroupResult.memberId
             assertEquals(1, initialGenerationId)
             assertEquals(Errors.NONE.code, joinGroupErrorCode)
         
             EasyMock.reset(replicaManager)
        -    val otherJoinGroupResult = joinGroup(groupId, otherMemberId, DefaultSessionTimeout, protocolType, protocols)
        +    val syncGroupResult = syncGroupLeader(groupId, initialGenerationId, memberId, Map(memberId -> Array[Byte]()))
        +    val syncGroupErrorCode = syncGroupResult._2
        +    assertEquals(Errors.NONE.code, syncGroupErrorCode)
        +
        +    EasyMock.reset(replicaManager)
        +    val otherJoinGroupResult = joinGroup(groupId, memberId, DefaultSessionTimeout, protocolType, protocols)
             val nextGenerationId = otherJoinGroupResult.generationId
             val otherJoinGroupErrorCode = otherJoinGroupResult.errorCode
             assertEquals(2, nextGenerationId)
        @@ -860,6 +958,7 @@ class GroupCoordinatorResponseTest extends JUnitSuite {
                                 protocolType: String,
                                 protocols: List[(String, Array[Byte])]): JoinGroupResult = {
             val responseFuture = sendJoinGroup(groupId, memberId, sessionTimeout, protocolType, protocols)
        +    timer.advanceClock(10)
             // should only have to wait as long as session timeout, but allow some extra time in case of an unexpected delay
             Await.result(responseFuture, Duration(sessionTimeout+100, TimeUnit.MILLISECONDS))
           }
        diff --git a/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala b/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala
        index df8d5b1b8642a..2c70137765abc 100644
        --- a/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala
        +++ b/core/src/test/scala/unit/kafka/server/DelayedOperationTest.scala
        @@ -26,7 +26,7 @@ class DelayedOperationTest {
         
           @Before
           def setUp() {
        -    purgatory = new DelayedOperationPurgatory[MockDelayedOperation](purgatoryName = "mock")
        +    purgatory = DelayedOperationPurgatory[MockDelayedOperation](purgatoryName = "mock")
           }
         
           @After
        diff --git a/core/src/test/scala/unit/kafka/utils/MockTime.scala b/core/src/test/scala/unit/kafka/utils/MockTime.scala
        index ee65748afefd5..0858e04628912 100644
        --- a/core/src/test/scala/unit/kafka/utils/MockTime.scala
        +++ b/core/src/test/scala/unit/kafka/utils/MockTime.scala
        @@ -19,6 +19,8 @@ package kafka.utils
         
         import java.util.concurrent._
         
        +import org.apache.kafka.common.utils
        +
         /**
          * A class used for unit testing things which depend on the Time interface.
          * 
        @@ -47,3 +49,13 @@ class MockTime(@volatile private var currentMs: Long) extends Time {
           override def toString() = "MockTime(%d)".format(milliseconds)
         
         }
        +
        +object MockTime {
        +  implicit def toCommonTime(time: MockTime): utils.Time = new utils.Time {
        +    override def nanoseconds(): Long = time.nanoseconds
        +
        +    override def milliseconds(): Long = time.milliseconds
        +
        +    override def sleep(ms: Long): Unit = time.sleep(ms)
        +  }
        +}
        \ No newline at end of file
        diff --git a/core/src/test/scala/unit/kafka/utils/timer/MockTimer.scala b/core/src/test/scala/unit/kafka/utils/timer/MockTimer.scala
        new file mode 100644
        index 0000000000000..d18a060976002
        --- /dev/null
        +++ b/core/src/test/scala/unit/kafka/utils/timer/MockTimer.scala
        @@ -0,0 +1,57 @@
        +/**
        +  * 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.utils.timer
        +
        +import kafka.utils.MockTime
        +
        +import scala.collection.mutable
        +
        +class MockTimer extends Timer {
        +
        +  val time = new MockTime
        +  private val taskQueue = mutable.PriorityQueue[TimerTaskEntry]()
        +
        +  def add(timerTask: TimerTask) {
        +    if (timerTask.delayMs <= 0)
        +      timerTask.run()
        +    else
        +      taskQueue.enqueue(new TimerTaskEntry(timerTask, timerTask.delayMs + time.milliseconds))
        +  }
        +
        +  def advanceClock(timeoutMs: Long): Boolean = {
        +    time.sleep(timeoutMs)
        +
        +    var executed = false
        +    val now = time.milliseconds
        +
        +    while (taskQueue.nonEmpty && now > taskQueue.head.expirationMs) {
        +      val taskEntry = taskQueue.dequeue()
        +      if (!taskEntry.cancelled) {
        +        val task = taskEntry.timerTask
        +        task.run()
        +        executed = true
        +      }
        +    }
        +
        +    executed
        +  }
        +
        +  def size: Int = taskQueue.size
        +
        +  override def shutdown(): Unit = {}
        +
        +}
        diff --git a/core/src/test/scala/unit/kafka/utils/timer/TimerTaskListTest.scala b/core/src/test/scala/unit/kafka/utils/timer/TimerTaskListTest.scala
        index a018ddef5f238..29c9067161ab7 100644
        --- a/core/src/test/scala/unit/kafka/utils/timer/TimerTaskListTest.scala
        +++ b/core/src/test/scala/unit/kafka/utils/timer/TimerTaskListTest.scala
        @@ -18,11 +18,11 @@ package kafka.utils.timer
         
         import org.junit.Assert._
         import java.util.concurrent.atomic._
        -import org.junit.{Test, After, Before}
        +import org.junit.Test
         
         class TimerTaskListTest {
         
        -  private class TestTask(val expirationMs: Long) extends TimerTask {
        +  private class TestTask(val delayMs: Long) extends TimerTask {
             def run(): Unit = { }
           }
         
        @@ -42,8 +42,8 @@ class TimerTaskListTest {
             val list3 = new TimerTaskList(sharedCounter)
         
             val tasks = (1 to 10).map { i =>
        -      val task = new TestTask(10L)
        -      list1.add(new TimerTaskEntry(task))
        +      val task = new TestTask(0L)
        +      list1.add(new TimerTaskEntry(task, 10L))
               assertEquals(i, sharedCounter.get)
               task
             }.toSeq
        @@ -54,7 +54,7 @@ class TimerTaskListTest {
             tasks.take(4).foreach { task =>
               val prevCount = sharedCounter.get
               // new TimerTaskEntry(task) will remove the existing entry from the list
        -      list2.add(new TimerTaskEntry(task))
        +      list2.add(new TimerTaskEntry(task, 10L))
               assertEquals(prevCount, sharedCounter.get)
             }
             assertEquals(10 - 4, size(list1))
        @@ -66,7 +66,7 @@ class TimerTaskListTest {
             tasks.drop(4).foreach { task =>
               val prevCount = sharedCounter.get
               // new TimerTaskEntry(task) will remove the existing entry from the list
        -      list3.add(new TimerTaskEntry(task))
        +      list3.add(new TimerTaskEntry(task, 10L))
               assertEquals(prevCount, sharedCounter.get)
             }
             assertEquals(0, size(list1))
        diff --git a/core/src/test/scala/unit/kafka/utils/timer/TimerTest.scala b/core/src/test/scala/unit/kafka/utils/timer/TimerTest.scala
        index 95de3780c6f47..54b73b887e4f4 100644
        --- a/core/src/test/scala/unit/kafka/utils/timer/TimerTest.scala
        +++ b/core/src/test/scala/unit/kafka/utils/timer/TimerTest.scala
        @@ -27,7 +27,7 @@ import scala.util.Random
         
         class TimerTest {
         
        -  private class TestTask(override val expirationMs: Long, id: Int, latch: CountDownLatch, output: ArrayBuffer[Int]) extends TimerTask {
        +  private class TestTask(override val delayMs: Long, id: Int, latch: CountDownLatch, output: ArrayBuffer[Int]) extends TimerTask {
             private[this] val completed = new AtomicBoolean(false)
             def run(): Unit = {
               if (completed.compareAndSet(false, true)) {
        @@ -37,32 +37,31 @@ class TimerTest {
             }
           }
         
        -  private[this] var executor: ExecutorService = null
        +  private[this] var timer: Timer = null
         
           @Before
           def setup() {
        -    executor = Executors.newSingleThreadExecutor()
        +    timer = new SystemTimer("test", tickMs = 1, wheelSize = 3)
           }
         
           @After
           def teardown(): Unit = {
        -    executor.shutdown()
        -    executor = null
        +    timer.shutdown()
           }
         
           @Test
           def testAlreadyExpiredTask(): Unit = {
        -    val startTime = System.currentTimeMillis()
        -    val timer = new Timer(taskExecutor = executor, tickMs = 1, wheelSize = 3, startMs = startTime)
             val output = new ArrayBuffer[Int]()
         
         
             val latches = (-5 until 0).map { i =>
               val latch = new CountDownLatch(1)
        -      timer.add(new TestTask(startTime + i, i, latch, output))
        +      timer.add(new TestTask(i, i, latch, output))
               latch
             }
         
        +    timer.advanceClock(0)
        +
             latches.take(5).foreach { latch =>
               assertEquals("already expired tasks should run immediately", true, latch.await(3, TimeUnit.SECONDS))
             }
        @@ -72,8 +71,6 @@ class TimerTest {
         
           @Test
           def testTaskExpiration(): Unit = {
        -    val startTime = System.currentTimeMillis()
        -    val timer = new Timer(taskExecutor = executor, tickMs = 1, wheelSize = 3, startMs = startTime)
             val output = new ArrayBuffer[Int]()
         
             val tasks = new ArrayBuffer[TestTask]()
        @@ -82,27 +79,27 @@ class TimerTest {
             val latches =
               (0 until 5).map { i =>
                 val latch = new CountDownLatch(1)
        -        tasks += new TestTask(startTime + i, i, latch, output)
        +        tasks += new TestTask(i, i, latch, output)
                 ids += i
                 latch
               } ++ (10 until 100).map { i =>
                 val latch = new CountDownLatch(2)
        -        tasks += new TestTask(startTime + i, i, latch, output)
        -        tasks += new TestTask(startTime + i, i, latch, output)
        +        tasks += new TestTask(i, i, latch, output)
        +        tasks += new TestTask(i, i, latch, output)
                 ids += i
                 ids += i
                 latch
               } ++ (100 until 500).map { i =>
                 val latch = new CountDownLatch(1)
        -        tasks += new TestTask(startTime + i, i, latch, output)
        +        tasks += new TestTask(i, i, latch, output)
                 ids += i
                 latch
               }
         
             // randomly submit requests
        -    Random.shuffle(tasks.toSeq).foreach { task => timer.add(task) }
        +    tasks.foreach { task => timer.add(task) }
         
        -    while (timer.advanceClock(1000)) {}
        +    while (timer.advanceClock(2000)) {}
         
             latches.foreach { latch => latch.await() }
         
        
        From 4fa456bc6e875c819e41aadadd1e37ba9bdb9cd0 Mon Sep 17 00:00:00 2001
        From: Ismael Juma 
        Date: Thu, 14 Apr 2016 10:44:05 -0700
        Subject: [PATCH 144/206] KAFKA-3557; Update rocksdb to 4.4.1 and patch updates
         to snappy and slf4j
        
        * The hope is that RocksDb 4.4.1 is more stable than 4.1.0 (occasional segfaults) and 4.2.0 (very frequent segfaults), release notes for 4.4.1: https://www.facebook.com/groups/rocksdb.dev/permalink/925995520832296/
        * slf4j 1.7.21 includes thread-safety fixes: http://www.slf4j.org/news.html
        * snappy 1.1.2.4 includes performance improvements requested by Spark, which apply to our usage: https://github.com/xerial/snappy-java/blob/master/Milestone.md
        
        I ran the stream tests several times and they passed every time while 4.2.0 segfaulted every time.
        
        Author: Ismael Juma 
        
        Reviewers: Eno Thereska , Guozhang Wang 
        
        Closes #1219 from ijuma/kafka-3557-update-rocks-db-4.4.1-snappy-slf4j
        ---
         gradle/dependencies.gradle | 6 +++---
         1 file changed, 3 insertions(+), 3 deletions(-)
        
        diff --git a/gradle/dependencies.gradle b/gradle/dependencies.gradle
        index 87e83cba9e1bb..8b292e8f9c749 100644
        --- a/gradle/dependencies.gradle
        +++ b/gradle/dependencies.gradle
        @@ -37,12 +37,12 @@ versions += [
           metrics: "2.2.0",
           powermock: "1.6.4",
           reflections: "0.9.10",
        -  rocksDB: "4.1.0",
        +  rocksDB: "4.4.1",
           scalaTest: "2.2.6",
           scalaParserCombinators: "1.0.4",
           scoverage: "1.1.1",
        -  slf4j: "1.7.18",
        -  snappy: "1.1.2.1",
        +  slf4j: "1.7.21",
        +  snappy: "1.1.2.4",
           zkclient: "0.8",
           zookeeper: "3.4.6",
         ]
        
        From 065ddf90195e09689512b55d0718a5ebdb3d42ad Mon Sep 17 00:00:00 2001
        From: Grant Henke 
        Date: Thu, 14 Apr 2016 22:02:19 -0700
        Subject: [PATCH 145/206] KAFKA-3549: Close consumers instantiated in consumer
         tests
        
        Author: Grant Henke 
        
        Reviewers: Ismael Juma , Ewen Cheslack-Postava 
        
        Closes #1217 from granthenke/close-consumers
        ---
         .../clients/consumer/KafkaConsumerTest.java   | 13 +++++++--
         .../kafka/api/BaseConsumerTest.scala          | 26 ++++++++---------
         .../kafka/api/PlaintextConsumerTest.scala     | 28 +++++++++++--------
         .../kafka/streams/perf/SimpleBenchmark.java   |  1 +
         .../streams/smoketest/SmokeTestDriver.java    |  2 +-
         5 files changed, 40 insertions(+), 30 deletions(-)
        
        diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
        index ff07461ac7845..227279533d2f0 100644
        --- a/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
        +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/KafkaConsumerTest.java
        @@ -77,6 +77,8 @@ public void testSubscription() {
                 consumer.unsubscribe();
                 Assert.assertTrue(consumer.subscription().isEmpty());
                 Assert.assertTrue(consumer.assignment().isEmpty());
        +
        +        consumer.close();
             }
         
             @Test(expected = IllegalArgumentException.class)
        @@ -85,10 +87,13 @@ public void testSeekNegative() {
                 props.setProperty(ConsumerConfig.CLIENT_ID_CONFIG, "testSeekNegative");
                 props.setProperty(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, "localhost:9999");
                 props.setProperty(ConsumerConfig.METRIC_REPORTER_CLASSES_CONFIG, MockMetricsReporter.class.getName());
        -
                 KafkaConsumer consumer = newConsumer();
        -        consumer.assign(Arrays.asList(new TopicPartition("nonExistTopic", 0)));
        -        consumer.seek(new TopicPartition("nonExistTopic", 0), -1);
        +        try {
        +            consumer.assign(Arrays.asList(new TopicPartition("nonExistTopic", 0)));
        +            consumer.seek(new TopicPartition("nonExistTopic", 0), -1);
        +        } finally {
        +            consumer.close();
        +        }
             }
         
             @Test
        @@ -129,6 +134,8 @@ public void testPause() {
         
                 consumer.unsubscribe();
                 Assert.assertTrue(consumer.paused().isEmpty());
        +
        +        consumer.close();
             }
         
             private KafkaConsumer newConsumer() {
        diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
        index 1408cd96a1657..916a0ab2d021d 100644
        --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
        +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala
        @@ -13,7 +13,6 @@
         package kafka.api
         
         import java.util
        -import kafka.coordinator.GroupCoordinator
         import org.apache.kafka.clients.consumer._
         import org.apache.kafka.clients.producer.{ProducerConfig, ProducerRecord}
         import org.apache.kafka.common.record.TimestampType
        @@ -92,6 +91,7 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging {
         
             this.consumerConfig.setProperty(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "true")
             val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
        +    consumers += consumer0
         
             val numRecords = 10000
             sendRecords(numRecords)
        @@ -184,6 +184,8 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging {
             this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100") // timeout quickly to avoid slow test
             this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30")
             val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
        +    consumers += consumer0
        +
             consumer0.subscribe(List(topic).asJava, listener)
         
             // the initial subscription should cause a callback execution
        @@ -209,8 +211,6 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging {
         
             // only expect one revocation since revoke is not invoked on initial membership
             assertEquals(2, listener.callsToRevoked)
        -
        -    consumer0.close()
           }
         
           @Test
        @@ -219,20 +219,17 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging {
             this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100") // timeout quickly to avoid slow test
             this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30")
             val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
        +    consumers += consumer0
         
        -    try {
        -      val listener = new TestConsumerReassignmentListener()
        -      consumer0.subscribe(List(topic).asJava, listener)
        +    val listener = new TestConsumerReassignmentListener()
        +    consumer0.subscribe(List(topic).asJava, listener)
         
        -      // the initial subscription should cause a callback execution
        -      while (listener.callsToAssigned == 0)
        -        consumer0.poll(50)
        +    // the initial subscription should cause a callback execution
        +    while (listener.callsToAssigned == 0)
        +      consumer0.poll(50)
         
        -      consumer0.subscribe(List[String]().asJava)
        -      assertEquals(0, consumer0.assignment.size())
        -    } finally {
        -      consumer0.close()
        -    }
        +    consumer0.subscribe(List[String]().asJava)
        +    assertEquals(0, consumer0.assignment.size())
           }
         
           @Test
        @@ -240,6 +237,7 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging {
             this.consumerConfig.setProperty(ConsumerConfig.SESSION_TIMEOUT_MS_CONFIG, "100") // timeout quickly to avoid slow test
             this.consumerConfig.setProperty(ConsumerConfig.HEARTBEAT_INTERVAL_MS_CONFIG, "30")
             val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
        +    consumers += consumer0
         
             sendRecords(5)
             consumer0.subscribe(List(topic).asJava)
        diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala
        index ff2e63d366aa9..349f7ad507722 100644
        --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala
        +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala
        @@ -48,6 +48,8 @@ class PlaintextConsumerTest extends BaseConsumerTest {
         
             this.consumerConfig.setProperty(ConsumerConfig.MAX_POLL_RECORDS_CONFIG, maxPollRecords.toString)
             val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
        +    consumers += consumer0
        +
             consumer0.assign(List(tp).asJava)
         
             consumeAndVerifyRecords(consumer0, numRecords = numRecords, startingOffset = 0,
        @@ -405,6 +407,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
           def testFetchInvalidOffset() {
             this.consumerConfig.setProperty(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none");
             val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
        +    consumers += consumer0
         
             // produce one record
             val totalRecords = 2
        @@ -426,8 +429,6 @@ class PlaintextConsumerTest extends BaseConsumerTest {
             assertNotNull(outOfRangePartitions)
             assertEquals(1, outOfRangePartitions.size)
             assertEquals(outOfRangePos.toLong, outOfRangePartitions.get(tp))
        -
        -    consumer0.close()
           }
         
           @Test
        @@ -435,6 +436,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
             val maxFetchBytes = 10 * 1024
             this.consumerConfig.setProperty(ConsumerConfig.MAX_PARTITION_FETCH_BYTES_CONFIG, maxFetchBytes.toString)
             val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
        +    consumers += consumer0
         
             // produce a record that is larger than the configured fetch size
             val record = new ProducerRecord[Array[Byte], Array[Byte]](tp.topic(), tp.partition(), "key".getBytes, new Array[Byte](maxFetchBytes + 1))
        @@ -450,8 +452,6 @@ class PlaintextConsumerTest extends BaseConsumerTest {
             assertEquals(1, oversizedPartitions.size)
             // the oversized message is at offset 0
             assertEquals(0L, oversizedPartitions.get(tp))
        -
        -    consumer0.close()
           }
         
           @Test
        @@ -460,6 +460,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
             this.consumerConfig.setProperty(ConsumerConfig.GROUP_ID_CONFIG, "roundrobin-group")
             this.consumerConfig.setProperty(ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, classOf[RoundRobinAssignor].getName)
             val consumer0 = new KafkaConsumer(this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
        +    consumers += consumer0
         
             // create two new topics, each having 2 partitions
             val topic1 = "topic1"
        @@ -512,13 +513,13 @@ class PlaintextConsumerTest extends BaseConsumerTest {
             val (rrConsumers, consumerPollers) = createConsumerGroupAndWaitForAssignment(10, List(topic1, topic2), subscriptions)
         
             // add one more consumer and validate re-assignment
        -    addConsumersToGroupAndWaitForGroupAssignment(1, rrConsumers, consumerPollers, List(topic1, topic2), subscriptions)
        +    addConsumersToGroupAndWaitForGroupAssignment(1, consumers, consumerPollers, List(topic1, topic2), subscriptions)
         
             // done with pollers and consumers
             for (poller <- consumerPollers)
               poller.shutdown()
         
        -    for (consumer <- rrConsumers)
        +    for (consumer <- consumers)
               consumer.unsubscribe()
           }
         
        @@ -688,6 +689,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
             producerProps.put(ProducerConfig.INTERCEPTOR_CLASSES_CONFIG, "org.apache.kafka.test.MockProducerInterceptor")
             producerProps.put("mock.interceptor.append", appendStr)
             val testProducer = new KafkaProducer[Array[Byte], Array[Byte]](producerProps, new ByteArraySerializer(), new ByteArraySerializer())
        +    producers += testProducer
         
             // producing records should succeed
             testProducer.send(new ProducerRecord(tp.topic(), tp.partition(), s"key".getBytes, s"value will not be modified".getBytes))
        @@ -695,6 +697,8 @@ class PlaintextConsumerTest extends BaseConsumerTest {
             // create consumer with interceptor that has different key and value types from the consumer
             this.consumerConfig.setProperty(ConsumerConfig.INTERCEPTOR_CLASSES_CONFIG, "org.apache.kafka.test.MockConsumerInterceptor")
             val testConsumer = new KafkaConsumer[Array[Byte], Array[Byte]](this.consumerConfig, new ByteArrayDeserializer(), new ByteArrayDeserializer())
        +    consumers += testConsumer
        +
             testConsumer.assign(List(tp).asJava)
             testConsumer.seek(tp, 0)
         
        @@ -702,9 +706,6 @@ class PlaintextConsumerTest extends BaseConsumerTest {
             val records = consumeRecords(testConsumer, 1)
             val record = records.get(0)
             assertEquals(s"value will not be modified", new String(record.value()))
        -
        -    testConsumer.close()
        -    testProducer.close()
           }
         
           def testConsumeMessagesWithCreateTime() {
        @@ -762,12 +763,14 @@ class PlaintextConsumerTest extends BaseConsumerTest {
         
             // create one more consumer and add it to the group; we will timeout this consumer
             val timeoutConsumer = new KafkaConsumer[Array[Byte], Array[Byte]](this.consumerConfig)
        -    val expandedConsumers = consumers ++ Buffer[KafkaConsumer[Array[Byte], Array[Byte]]](timeoutConsumer)
        +    // Close the consumer on test teardown, unless this test will manually
        +    if(!closeConsumer)
        +      consumers += timeoutConsumer
             val timeoutPoller = subscribeConsumerAndStartPolling(timeoutConsumer, List(topic, topic1))
        -    val expandedPollers = consumerPollers ++ Buffer[ConsumerAssignmentPoller](timeoutPoller)
        +    consumerPollers += timeoutPoller
         
             // validate the initial assignment
        -    validateGroupAssignment(expandedPollers, subscriptions, s"Did not get valid initial assignment for partitions ${subscriptions.asJava}")
        +    validateGroupAssignment(consumerPollers, subscriptions, s"Did not get valid initial assignment for partitions ${subscriptions.asJava}")
         
             // stop polling and close one of the consumers, should trigger partition re-assignment among alive consumers
             timeoutPoller.shutdown()
        @@ -859,6 +862,7 @@ class PlaintextConsumerTest extends BaseConsumerTest {
             val consumerGroup = Buffer[KafkaConsumer[Array[Byte], Array[Byte]]]()
             for (i <- 0 until consumerCount)
               consumerGroup += new KafkaConsumer[Array[Byte], Array[Byte]](this.consumerConfig)
        +    consumers ++= consumerGroup
         
             // create consumer pollers, wait for assignment and validate it
             val consumerPollers = subscribeConsumersAndWaitForAssignment(consumerGroup, topicsToSubscribe, subscriptions)
        diff --git a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java
        index c88309046d69a..a92fb1baea6b8 100644
        --- a/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java
        +++ b/streams/src/test/java/org/apache/kafka/streams/perf/SimpleBenchmark.java
        @@ -253,6 +253,7 @@ public void consume() {
         
                 long endTime = System.currentTimeMillis();
         
        +        consumer.close();
                 System.out.println("Consumer Performance [MB/sec read]: " + megaBytePerSec(endTime - startTime));
             }
         
        diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java
        index d7b0139618fd0..205ba4b85f6b6 100644
        --- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java
        +++ b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestDriver.java
        @@ -278,7 +278,7 @@ public static void verify(String kafka, Map> allData, int m
                         }
                     }
                 }
        -
        +        consumer.close();
         
                 System.out.println("-------------------");
                 System.out.println("Result Verification");
        
        From 5236bf60debbb0c08010315a92dd3fbfa482e871 Mon Sep 17 00:00:00 2001
        From: Liquan Pei 
        Date: Fri, 15 Apr 2016 15:51:31 -0700
        Subject: [PATCH 146/206] KAFKA-3526: Return string instead of object in
         ConfigKeyInfo and ConfigValueInfo
        
        Author: Liquan Pei 
        
        Reviewers: Ewen Cheslack-Postava 
        
        Closes #1200 from Ishiihara/config-string
        ---
         .../apache/kafka/common/config/ConfigDef.java | 35 +++++++++--
         .../kafka/common/config/ConfigDefTest.java    |  3 +-
         .../kafka/connect/runtime/AbstractHerder.java | 40 +++++++++---
         .../runtime/rest/entities/ConfigKeyInfo.java  |  6 +-
         .../rest/entities/ConfigValueInfo.java        | 12 ++--
         .../ConnectorPluginsResourceTest.java         | 63 +++++++++++++++----
         6 files changed, 120 insertions(+), 39 deletions(-)
        
        diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
        index 881cb0bf9353f..1df55d98dc3fd 100644
        --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
        +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java
        @@ -538,19 +538,18 @@ private void parseForValidate(String name, Map props, Map parsed, Map configs) {
                 if (!configKeys.containsKey(name)) {
                     return;
                 }
                 ConfigKey key = configKeys.get(name);
                 ConfigValue config = configs.get(name);
        -        Object value = parsed.get(name);
                 List recommendedValues;
                 if (key.recommender != null) {
                     try {
                         recommendedValues = key.recommender.validValues(name, parsed);
                         List originalRecommendedValues = config.recommendedValues();
        -
                         if (!originalRecommendedValues.isEmpty()) {
                             Set originalRecommendedValueSet = new HashSet<>(originalRecommendedValues);
                             Iterator it = recommendedValues.iterator();
        @@ -562,9 +561,6 @@ private void validate(String name, Map parsed, Map valueList = (List) parsedValue;
        +                return Utils.join(valueList, ",");
        +            case CLASS:
        +                Class clazz = (Class) parsedValue;
        +                return clazz.getCanonicalName();
        +            default:
        +                throw new IllegalStateException("Unknown type.");
        +        }
        +    }
        +
             /**
              * The config types
              */
        diff --git a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java
        index 022fb6bc62397..e20e422020af1 100644
        --- a/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java
        +++ b/clients/src/test/java/org/apache/kafka/common/config/ConfigDefTest.java
        @@ -236,12 +236,11 @@ public void testValidate() {
                 Map expected = new HashMap<>();
                 String errorMessageB = "Missing required configuration \"b\" which has no default value.";
                 String errorMessageC = "Missing required configuration \"c\" which has no default value.";
        -        String errorMessageD = "Invalid value for configuration d";
         
                 ConfigValue configA = new ConfigValue("a", 1, Arrays.asList(1, 2, 3), Collections.emptyList());
                 ConfigValue configB = new ConfigValue("b", null, Arrays.asList(4, 5), Arrays.asList(errorMessageB, errorMessageB));
                 ConfigValue configC = new ConfigValue("c", null, Arrays.asList(4, 5), Arrays.asList(errorMessageC));
        -        ConfigValue configD = new ConfigValue("d", 10, Arrays.asList(1, 2, 3), Arrays.asList(errorMessageD));
        +        ConfigValue configD = new ConfigValue("d", 10, Arrays.asList(1, 2, 3), Collections.emptyList());
         
                 expected.put("a", configA);
                 expected.put("b", configB);
        diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
        index a97c4db8faff7..1d87d605ce0ba 100644
        --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
        +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java
        @@ -19,6 +19,7 @@
         import org.apache.kafka.common.config.Config;
         import org.apache.kafka.common.config.ConfigDef;
         import org.apache.kafka.common.config.ConfigDef.ConfigKey;
        +import org.apache.kafka.common.config.ConfigDef.Type;
         import org.apache.kafka.common.config.ConfigValue;
         import org.apache.kafka.connect.connector.Connector;
         import org.apache.kafka.connect.errors.NotFoundException;
        @@ -230,16 +231,18 @@ public static ConfigInfos generateResult(String connType, Map
                     configValueMap.put(configName, configValue);
                     if (!configKeys.containsKey(configName)) {
                         configValue.addErrorMessage("Configuration is not defined: " + configName);
        -                configInfoList.add(new ConfigInfo(null, convertConfigValue(configValue)));
        +                configInfoList.add(new ConfigInfo(null, convertConfigValue(configValue, null)));
                     }
                 }
         
        -        for (String configName: configKeys.keySet()) {
        -            ConfigKeyInfo configKeyInfo = convertConfigKey(configKeys.get(configName));
        +        for (Map.Entry entry : configKeys.entrySet()) {
        +            String configName = entry.getKey();
        +            ConfigKeyInfo configKeyInfo = convertConfigKey(entry.getValue());
        +            Type type = entry.getValue().type;
                     ConfigValueInfo configValueInfo = null;
                     if (configValueMap.containsKey(configName)) {
                         ConfigValue configValue = configValueMap.get(configName);
        -                configValueInfo = convertConfigValue(configValue);
        +                configValueInfo = convertConfigValue(configValue, type);
                         errorCount += configValue.errorMessages().size();
                     }
                     configInfoList.add(new ConfigInfo(configKeyInfo, configValueInfo));
        @@ -249,11 +252,16 @@ public static ConfigInfos generateResult(String connType, Map
         
             private static ConfigKeyInfo convertConfigKey(ConfigKey configKey) {
                 String name = configKey.name;
        -        String type = configKey.type.name();
        -        Object defaultValue = configKey.defaultValue;
        +        Type type = configKey.type;
        +        String typeName = configKey.type.name();
        +
                 boolean required = false;
        -        if (defaultValue == ConfigDef.NO_DEFAULT_VALUE) {
        +        String defaultValue;
        +        if (configKey.defaultValue == ConfigDef.NO_DEFAULT_VALUE) {
        +            defaultValue = (String) configKey.defaultValue;
                     required = true;
        +        } else {
        +            defaultValue = ConfigDef.convertToString(configKey.defaultValue, type);
                 }
                 String importance = configKey.importance.name();
                 String documentation = configKey.documentation;
        @@ -262,11 +270,23 @@ private static ConfigKeyInfo convertConfigKey(ConfigKey configKey) {
                 String width = configKey.width.name();
                 String displayName = configKey.displayName;
                 List dependents = configKey.dependents;
        -        return new ConfigKeyInfo(name, type, required, defaultValue, importance, documentation, group, orderInGroup, width, displayName, dependents);
        +        return new ConfigKeyInfo(name, typeName, required, defaultValue, importance, documentation, group, orderInGroup, width, displayName, dependents);
             }
         
        -    private static ConfigValueInfo convertConfigValue(ConfigValue configValue) {
        -        return new ConfigValueInfo(configValue.name(), configValue.value(), configValue.recommendedValues(), configValue.errorMessages(), configValue.visible());
        +    private static ConfigValueInfo convertConfigValue(ConfigValue configValue, Type type) {
        +        String value = ConfigDef.convertToString(configValue.value(), type);
        +        List recommendedValues = new LinkedList<>();
        +
        +        if (type == Type.LIST) {
        +            for (Object object: configValue.recommendedValues()) {
        +                recommendedValues.add(ConfigDef.convertToString(object, Type.STRING));
        +            }
        +        } else {
        +            for (Object object : configValue.recommendedValues()) {
        +                recommendedValues.add(ConfigDef.convertToString(object, type));
        +            }
        +        }
        +        return new ConfigValueInfo(configValue.name(), value, recommendedValues, configValue.errorMessages(), configValue.visible());
             }
         
             private Connector getConnector(String connType) {
        diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigKeyInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigKeyInfo.java
        index f8137092efca4..ead24c57c5627 100644
        --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigKeyInfo.java
        +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigKeyInfo.java
        @@ -28,7 +28,7 @@ public class ConfigKeyInfo {
             private final String name;
             private final String type;
             private final boolean required;
        -    private final Object defaultValue;
        +    private final String defaultValue;
             private final String importance;
             private final String documentation;
             private final String group;
        @@ -41,7 +41,7 @@ public class ConfigKeyInfo {
             public ConfigKeyInfo(@JsonProperty("name") String name,
                                  @JsonProperty("type") String type,
                                  @JsonProperty("required") boolean required,
        -                         @JsonProperty("default_value") Object defaultValue,
        +                         @JsonProperty("default_value") String defaultValue,
                                  @JsonProperty("importance") String importance,
                                  @JsonProperty("documentation") String documentation,
                                  @JsonProperty("group") String group,
        @@ -78,7 +78,7 @@ public boolean required() {
             }
         
             @JsonProperty("default_value")
        -    public Object defaultValue() {
        +    public String defaultValue() {
                 return defaultValue;
             }
         
        diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigValueInfo.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigValueInfo.java
        index 51e7ee5b2f420..a6ae006907396 100644
        --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigValueInfo.java
        +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/entities/ConfigValueInfo.java
        @@ -25,16 +25,16 @@
         
         public class ConfigValueInfo {
             private String name;
        -    private Object value;
        -    private List recommendedValues;
        +    private String value;
        +    private List recommendedValues;
             private List errors;
             private boolean visible;
         
             @JsonCreator
             public ConfigValueInfo(
                 @JsonProperty("name") String name,
        -        @JsonProperty("value") Object value,
        -        @JsonProperty("recommended_values") List recommendedValues,
        +        @JsonProperty("value") String value,
        +        @JsonProperty("recommended_values") List recommendedValues,
                 @JsonProperty("errors") List errors,
                 @JsonProperty("visible") boolean visible) {
                 this.name = name;
        @@ -50,12 +50,12 @@ public String name() {
             }
         
             @JsonProperty
        -    public Object value() {
        +    public String value() {
                 return value;
             }
         
             @JsonProperty("recommended_values")
        -    public List recommendedValues() {
        +    public List recommendedValues() {
                 return recommendedValues;
             }
         
        diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java
        index 1049e7e0dd78f..732db3d725674 100644
        --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java
        +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java
        @@ -21,8 +21,10 @@
         
         import org.apache.kafka.common.config.Config;
         import org.apache.kafka.common.config.ConfigDef;
        +import org.apache.kafka.common.config.ConfigDef.Recommender;
         import org.apache.kafka.common.config.ConfigDef.Importance;
         import org.apache.kafka.common.config.ConfigDef.Type;
        +import org.apache.kafka.common.config.ConfigDef.Width;
         import org.apache.kafka.connect.connector.Connector;
         import org.apache.kafka.connect.connector.Task;
         import org.apache.kafka.connect.runtime.AbstractHerder;
        @@ -48,6 +50,7 @@
         import org.powermock.core.classloader.annotations.PrepareForTest;
         import org.powermock.modules.junit4.PowerMockRunner;
         
        +import java.util.Arrays;
         import java.util.Collections;
         import java.util.HashMap;
         import java.util.HashSet;
        @@ -68,7 +71,8 @@ public class ConnectorPluginsResourceTest {
             private static Map props = new HashMap<>();
             static {
                 props.put("test.string.config", "testString");
        -        props.put("test.int.config", "10");
        +        props.put("test.int.config", "1");
        +        props.put("test.list.config", "a,b");
             }
         
             private static final ConfigInfos CONFIG_INFOS;
        @@ -76,22 +80,27 @@ public class ConnectorPluginsResourceTest {
             static {
                 List configs = new LinkedList<>();
         
        -        ConfigKeyInfo configKeyInfo = new ConfigKeyInfo("test.string.config", "STRING", true, "", "HIGH", "Test configuration for string type.", null, -1, "NONE", "test.string.config", new LinkedList());
        -        ConfigValueInfo configValueInfo = new ConfigValueInfo("test.string.config", "testString", Collections.emptyList(), Collections.emptyList(), true);
        +        ConfigKeyInfo configKeyInfo = new ConfigKeyInfo("test.string.config", "STRING", true, "", "HIGH", "Test configuration for string type.", null, -1, "NONE", "test.string.config", Collections.emptyList());
        +        ConfigValueInfo configValueInfo = new ConfigValueInfo("test.string.config", "testString", Collections.emptyList(), Collections.emptyList(), true);
                 ConfigInfo configInfo = new ConfigInfo(configKeyInfo, configValueInfo);
                 configs.add(configInfo);
         
        -        configKeyInfo = new ConfigKeyInfo("test.int.config", "INT", true, "", "MEDIUM", "Test configuration for integer type.", null, -1, "NONE", "test.int.config", new LinkedList());
        -        configValueInfo = new ConfigValueInfo("test.int.config", 10, Collections.emptyList(), Collections.emptyList(), true);
        +        configKeyInfo = new ConfigKeyInfo("test.int.config", "INT", true, "", "MEDIUM", "Test configuration for integer type.", "Test", 1, "MEDIUM", "test.int.config", Collections.emptyList());
        +        configValueInfo = new ConfigValueInfo("test.int.config", "1", Arrays.asList("1", "2", "3"), Collections.emptyList(), true);
                 configInfo = new ConfigInfo(configKeyInfo, configValueInfo);
                 configs.add(configInfo);
         
        -        configKeyInfo = new ConfigKeyInfo("test.string.config.default", "STRING", false, "", "LOW", "Test configuration with default value.", null, -1, "NONE", "test.string.config.default", new LinkedList());
        -        configValueInfo = new ConfigValueInfo("test.string.config.default", "", Collections.emptyList(), Collections.emptyList(), true);
        +        configKeyInfo = new ConfigKeyInfo("test.string.config.default", "STRING", false, "", "LOW", "Test configuration with default value.", null, -1, "NONE", "test.string.config.default", Collections.emptyList());
        +        configValueInfo = new ConfigValueInfo("test.string.config.default", "", Collections.emptyList(), Collections.emptyList(), true);
                 configInfo = new ConfigInfo(configKeyInfo, configValueInfo);
                 configs.add(configInfo);
         
        -        CONFIG_INFOS = new ConfigInfos(ConnectorPluginsResourceTestConnector.class.getName(), 0, Collections.emptyList(), configs);
        +        configKeyInfo = new ConfigKeyInfo("test.list.config", "LIST", true, "", "HIGH", "Test configuration for list type.", "Test", 2, "LONG", "test.list.config", Collections.emptyList());
        +        configValueInfo = new ConfigValueInfo("test.list.config", "a,b", Arrays.asList("a", "b", "c"), Collections.emptyList(), true);
        +        configInfo = new ConfigInfo(configKeyInfo, configValueInfo);
        +        configs.add(configInfo);
        +
        +        CONFIG_INFOS = new ConfigInfos(ConnectorPluginsResourceTestConnector.class.getName(), 0, Collections.singletonList("Test"), configs);
             }
         
             @Mock
        @@ -143,14 +152,17 @@ public void testListConnectorPlugins() {
             /* Name here needs to be unique as we are testing the aliasing mechanism */
             public static class ConnectorPluginsResourceTestConnector extends Connector {
         
        -        public static final String TEST_STRING_CONFIG = "test.string.config";
        -        public static final String TEST_INT_CONFIG = "test.int.config";
        -        public static final String TEST_STRING_CONFIG_DEFAULT = "test.string.config.default";
        +        private static final String TEST_STRING_CONFIG = "test.string.config";
        +        private static final String TEST_INT_CONFIG = "test.int.config";
        +        private static final String TEST_STRING_CONFIG_DEFAULT = "test.string.config.default";
        +        private static final String TEST_LIST_CONFIG = "test.list.config";
        +        private static final String GROUP = "Test";
         
                 private static final ConfigDef CONFIG_DEF = new ConfigDef()
                     .define(TEST_STRING_CONFIG, Type.STRING, Importance.HIGH, "Test configuration for string type.")
        -            .define(TEST_INT_CONFIG, Type.INT, Importance.MEDIUM, "Test configuration for integer type.")
        -            .define(TEST_STRING_CONFIG_DEFAULT, Type.STRING, "", Importance.LOW, "Test configuration with default value.");
        +            .define(TEST_INT_CONFIG, Type.INT, Importance.MEDIUM, "Test configuration for integer type.", GROUP, 1, Width.MEDIUM, TEST_INT_CONFIG, new IntegerRecommender())
        +            .define(TEST_STRING_CONFIG_DEFAULT, Type.STRING, "", Importance.LOW, "Test configuration with default value.")
        +            .define(TEST_LIST_CONFIG, Type.LIST, Importance.HIGH, "Test configuration for list type.", GROUP, 2, Width.LONG, TEST_LIST_CONFIG, new ListRecommender());
         
                 @Override
                 public String version() {
        @@ -182,4 +194,29 @@ public ConfigDef config() {
                     return CONFIG_DEF;
                 }
             }
        +
        +    private static class IntegerRecommender implements Recommender {
        +
        +        @Override
        +        public List validValues(String name, Map parsedConfig) {
        +            return Arrays.asList(1, 2, 3);
        +        }
        +
        +        @Override
        +        public boolean visible(String name, Map parsedConfig) {
        +            return true;
        +        }
        +    }
        +
        +    private static class ListRecommender implements Recommender {
        +        @Override
        +        public List validValues(String name, Map parsedConfig) {
        +            return Arrays.asList("a", "b", "c");
        +        }
        +
        +        @Override
        +        public boolean visible(String name, Map parsedConfig) {
        +            return true;
        +        }
        +    }
         }
        
        From b1691cf49e9de850ac8a2675c487af9fb60bfdaa Mon Sep 17 00:00:00 2001
        From: Bill Bejeck 
        Date: Sat, 16 Apr 2016 20:18:27 -0700
        Subject: [PATCH 147/206] KAFKA-3430: Allow users to set key in KTable.toStream
         and in KStream.
        MIME-Version: 1.0
        Content-Type: text/plain; charset=UTF-8
        Content-Transfer-Encoding: 8bit
        
        … With KStream the method selectKey was added to enable getting a key from values before perfoming aggregation-by-key operations on original streams that have null keys.
        
        Author: bbejeck 
        
        Reviewers: Guozhang Wang 
        
        Closes #1222 from bbejeck/KAFKA-3430_allow_users_to_set_key_KTable_toStream
        ---
         .../apache/kafka/streams/kstream/KStream.java |  9 ++
         .../apache/kafka/streams/kstream/KTable.java  |  9 ++
         .../kstream/internals/KStreamImpl.java        | 15 ++++
         .../streams/kstream/internals/KTableImpl.java |  5 ++
         .../internals/KStreamSelectKeyTest.java       | 83 +++++++++++++++++
         .../kstream/internals/KTableMapKeysTest.java  | 88 +++++++++++++++++++
         6 files changed, 209 insertions(+)
         create mode 100644 streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java
         create mode 100644 streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java
        
        diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
        index 27475aa18ad5c..7e3562c558d34 100644
        --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
        +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java
        @@ -46,6 +46,15 @@ public interface KStream {
              */
             KStream filterNot(Predicate predicate);
         
        +
        +    /**
        +     * Create a new key from the current key and value.
        +     *
        +     * @param mapper  the instance of {@link KeyValueMapper}
        +     * @param    the new key type on the stream
        +     */
        +     KStream selectKey(KeyValueMapper mapper);
        +
             /**
              * Create a new instance of {@link KStream} by transforming each element in this stream into a different element in the new stream.
              *
        diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java
        index bb6878f3ae235..1e44cb5f38ae9 100644
        --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java
        +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java
        @@ -202,6 +202,15 @@ public interface KTable {
              */
             KStream toStream();
         
        +    /**
        +     *  Convert this stream to a new instance of {@link KStream} using the given {@link KeyValueMapper} to select
        +     *  the new key.
        +     *
        +     * @param mapper  @param mapper  the instance of {@link KeyValueMapper}
        +     * @param  the new key type
        +     */
        +     KStream toStream(KeyValueMapper mapper);
        +
             /**
              * Combine values of this stream with another {@link KTable} stream's elements of the same key using Inner Join.
              *
        diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java
        index 70300214a5699..a84b4aa980d5b 100644
        --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java
        +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java
        @@ -86,6 +86,8 @@ public class KStreamImpl extends AbstractStream implements KStream filterNot(final Predicate predicate) {
                 return new KStreamImpl<>(topology, name, sourceNodes);
             }
         
        +    @Override
        +    @SuppressWarnings("unchecked")
        +    public  KStream selectKey(final KeyValueMapper mapper) {
        +        String name = topology.newName(KEY_SELECT_NAME);
        +        topology.addProcessor(name, new KStreamMap<>(new KeyValueMapper>() {
        +            @Override
        +            public KeyValue apply(K key, V value) {
        +                return new KeyValue(mapper.apply(key, value), value);
        +            }
        +        }), this.name);
        +        return new KStreamImpl<>(topology, name, sourceNodes);
        +    }
        +
             @Override
             public  KStream map(KeyValueMapper> mapper) {
                 String name = topology.newName(MAP_NAME);
        diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java
        index f78169e0d4764..5c291f5f8e670 100644
        --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java
        +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java
        @@ -244,6 +244,11 @@ public V apply(Change change) {
                 return new KStreamImpl<>(topology, name, sourceNodes);
             }
         
        +    @Override
        +    public  KStream toStream(KeyValueMapper mapper) {
        +        return toStream().selectKey(mapper);
        +    }
        +
             @SuppressWarnings("unchecked")
             @Override
             public  KTable join(KTable other, ValueJoiner joiner) {
        diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java
        new file mode 100644
        index 0000000000000..5f19b9eb3886b
        --- /dev/null
        +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java
        @@ -0,0 +1,83 @@
        +/**
        + * 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 org.apache.kafka.streams.kstream.internals;
        +
        +
        +import org.apache.kafka.common.serialization.Serde;
        +import org.apache.kafka.common.serialization.Serdes;
        +import org.apache.kafka.streams.kstream.KStream;
        +import org.apache.kafka.streams.kstream.KStreamBuilder;
        +import org.apache.kafka.streams.kstream.KeyValueMapper;
        +import org.apache.kafka.test.KStreamTestDriver;
        +import org.apache.kafka.test.MockProcessorSupplier;
        +import org.junit.Test;
        +
        +import java.util.HashMap;
        +import java.util.Map;
        +
        +import static org.junit.Assert.assertEquals;
        +
        +public class KStreamSelectKeyTest {
        +
        +    private String topicName = "topic_key_select";
        +
        +    final private Serde integerSerde = Serdes.Integer();
        +    final private Serde stringSerde = Serdes.String();
        +
        +    @Test
        +    public void testSelectKey() {
        +        KStreamBuilder builder = new KStreamBuilder();
        +
        +        final Map keyMap = new HashMap<>();
        +        keyMap.put(1, "ONE");
        +        keyMap.put(2, "TWO");
        +        keyMap.put(3, "THREE");
        +
        +
        +        KeyValueMapper selector = new KeyValueMapper() {
        +            @Override
        +            public String apply(String key, Integer value) {
        +                return keyMap.get(value);
        +            }
        +        };
        +
        +        final String[] expected = new String[]{"ONE:1", "TWO:2", "THREE:3"};
        +        final int[] expectedValues = new int[]{1, 2, 3};
        +
        +        KStream  stream = builder.stream(stringSerde, integerSerde, topicName);
        +
        +        MockProcessorSupplier processor = new MockProcessorSupplier<>();
        +
        +        stream.selectKey(selector).process(processor);
        +
        +        KStreamTestDriver driver = new KStreamTestDriver(builder);
        +
        +        for (int expectedValue : expectedValues) {
        +            driver.process(topicName, null, expectedValue);
        +        }
        +
        +        assertEquals(3, processor.processed.size());
        +
        +        for (int i = 0; i < expected.length; i++) {
        +            assertEquals(expected[i], processor.processed.get(i));
        +        }
        +
        +    }
        +
        +}
        \ No newline at end of file
        diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java
        new file mode 100644
        index 0000000000000..ce1b9d62dd248
        --- /dev/null
        +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java
        @@ -0,0 +1,88 @@
        +/**
        + * 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 org.apache.kafka.streams.kstream.internals;
        +
        +
        +import org.apache.kafka.common.serialization.Serde;
        +import org.apache.kafka.common.serialization.Serdes;
        +import org.apache.kafka.streams.kstream.KStream;
        +import org.apache.kafka.streams.kstream.KStreamBuilder;
        +import org.apache.kafka.streams.kstream.KTable;
        +import org.apache.kafka.streams.kstream.KeyValueMapper;
        +import org.apache.kafka.test.KStreamTestDriver;
        +import org.apache.kafka.test.MockProcessorSupplier;
        +import org.junit.Test;
        +
        +import java.util.HashMap;
        +import java.util.Map;
        +
        +import static org.junit.Assert.assertEquals;
        +
        +public class KTableMapKeysTest {
        +
        +    final private Serde stringSerde = new Serdes.StringSerde();
        +    final private Serde  integerSerde = new Serdes.IntegerSerde();
        +
        +    @Test
        +    public void testMapKeysConvertingToStream() {
        +        final KStreamBuilder builder = new KStreamBuilder();
        +
        +        String topic1 = "topic_map_keys";
        +
        +        KTable table1 = builder.table(integerSerde, stringSerde, topic1);
        +
        +        final Map keyMap = new HashMap<>();
        +        keyMap.put(1, "ONE");
        +        keyMap.put(2, "TWO");
        +        keyMap.put(3, "THREE");
        +
        +        KeyValueMapper keyMapper = new KeyValueMapper() {
        +            @Override
        +            public  String apply(Integer key, String value) {
        +                return keyMap.get(key);
        +            }
        +        };
        +
        +        KStream convertedStream = table1.toStream(keyMapper);
        +
        +        final String[] expected = new String[]{"ONE:V_ONE", "TWO:V_TWO", "THREE:V_THREE"};
        +        final int[] originalKeys = new int[]{1, 2, 3};
        +        final String[] values = new String[]{"V_ONE", "V_TWO", "V_THREE"};
        +
        +
        +
        +        MockProcessorSupplier processor = new MockProcessorSupplier<>();
        +
        +        convertedStream.process(processor);
        +
        +        KStreamTestDriver driver = new KStreamTestDriver(builder);
        +
        +        for (int i = 0;  i < originalKeys.length; i++) {
        +            driver.process(topic1, originalKeys[i], values[i]);
        +        }
        +
        +        assertEquals(3, processor.processed.size());
        +
        +        for (int i = 0; i < expected.length; i++) {
        +            assertEquals(expected[i], processor.processed.get(i));
        +        }
        +    }
        +
        +
        +
        +}
        \ No newline at end of file
        
        From 89c67727c2793bf56b0b005a7d758beebedb5aed Mon Sep 17 00:00:00 2001
        From: Jason Gustafson 
        Date: Mon, 18 Apr 2016 10:50:58 -0700
        Subject: [PATCH 148/206] KAFKA-3506: Kafka Connect restart APIs
        
        Author: Jason Gustafson 
        
        Reviewers: Ewen Cheslack-Postava 
        
        Closes #1189 from hachikuji/KAFKA-3506
        ---
         .../apache/kafka/connect/runtime/Herder.java  |  18 +-
         .../apache/kafka/connect/runtime/Worker.java  |   7 +
         .../distributed/DistributedHerder.java        |  91 +++++-
         .../distributed/NotAssignedException.java     |  29 ++
         .../distributed/NotLeaderException.java       |  21 +-
         .../distributed/RequestTargetException.java   |  47 +++
         .../distributed/StaleConfigException.java     |  27 ++
         .../distributed/WorkerCoordinator.java        |  59 ++++
         .../distributed/WorkerGroupMember.java        |   9 +
         .../connect/runtime/rest/RestServer.java      |   9 +-
         .../rest/resources/ConnectorsResource.java    | 134 +++++---
         .../runtime/standalone/StandaloneHerder.java  |  60 +++-
         .../distributed/DistributedHerderTest.java    | 301 ++++++++++++++++++
         .../resources/ConnectorsResourceTest.java     | 143 +++++++--
         .../standalone/StandaloneHerderTest.java      | 154 +++++++++
         15 files changed, 1009 insertions(+), 100 deletions(-)
         create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotAssignedException.java
         create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/RequestTargetException.java
         create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/StaleConfigException.java
        
        diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java
        index 3ea4a81bb6da5..cce100e16018c 100644
        --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java
        +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java
        @@ -61,9 +61,9 @@ public interface Herder {
              * from the current configuration. However, note
              *
              * @returns A list of connector names
        -     * @throws org.apache.kafka.connect.runtime.distributed.NotLeaderException if this node can not resolve the request
        +     * @throws org.apache.kafka.connect.runtime.distributed.RequestTargetException if this node can not resolve the request
              *         (e.g., because it has not joined the cluster or does not have configs in sync with the group) and it is
        -     *         also not the leader
        +     *         not the leader or the task owner (e.g., task restart must be handled by the worker which owns the task)
              * @throws org.apache.kafka.connect.errors.ConnectException if this node is the leader, but still cannot resolve the
              *         request (e.g., it is not in sync with other worker's config state)
              */
        @@ -135,6 +135,20 @@ public interface Herder {
              */
             ConfigInfos validateConfigs(String connType, Map connectorConfig);
         
        +    /**
        +     * Restart the task with the given id.
        +     * @param id id of the task
        +     * @param cb callback to invoke upon completion
        +     */
        +    void restartTask(ConnectorTaskId id, Callback cb);
        +
        +    /**
        +     * Restart the connector.
        +     * @param connName name of the connector
        +     * @param cb callback to invoke upon completion
        +     */
        +    void restartConnector(String connName, Callback cb);
        +
             class Created {
                 private final boolean created;
                 private final T result;
        diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
        index e1a806a2c3030..22843d38b56f2 100644
        --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
        +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java
        @@ -242,6 +242,9 @@ private String connectorNames(Collection> connectors)
                 return names.substring(0, names.toString().length() - 2);
             }
         
        +    public boolean ownsTask(ConnectorTaskId taskId) {
        +        return tasks.containsKey(taskId);
        +    }
         
             private static Connector instantiateConnector(Class connClass) {
                 try {
        @@ -415,6 +418,10 @@ public String workerId() {
                 return workerId;
             }
         
        +    public boolean ownsConnector(String connName) {
        +        return this.connectors.containsKey(connName);
        +    }
        +
             private static class WorkerConnector  {
                 private final String connName;
                 private final ConnectorStatus.Listener lifecycleListener;
        diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java
        index 2fc8297a5f644..24d548d112a34 100644
        --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java
        +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java
        @@ -544,6 +544,79 @@ else if (!configState.connectors().contains(connName))
                 );
             }
         
        +    @Override
        +    public synchronized void restartConnector(final String connName, final Callback callback) {
        +        addRequest(new Callable() {
        +            @Override
        +            public Void call() throws Exception {
        +                // if config is out of sync, then a rebalance is likely to begin shortly, so rather than risking
        +                // a stale response, we return an error and let the user retry
        +                if (!isConfigSynced()) {
        +                    throw new StaleConfigException("Cannot complete request because config is out of sync");
        +                }
        +
        +                if (!configState.connectors().contains(connName)) {
        +                    callback.onCompletion(new NotFoundException("Unknown connector: " + connName), null);
        +                    return null;
        +                }
        +
        +                if (worker.ownsConnector(connName)) {
        +                    try {
        +                        worker.stopConnector(connName);
        +                        startConnector(connName);
        +                        callback.onCompletion(null, null);
        +                    } catch (Throwable t) {
        +                        callback.onCompletion(t, null);
        +                    }
        +                } else if (isLeader()) {
        +                    callback.onCompletion(new NotAssignedException("Cannot restart connector since it is not assigned to this member", member.ownerUrl(connName)), null);
        +                } else {
        +                    callback.onCompletion(new NotLeaderException("Cannot restart connector since it is not assigned to this member", leaderUrl()), null);
        +                }
        +                return null;
        +            }
        +        }, forwardErrorCallback(callback));
        +    }
        +
        +    @Override
        +    public synchronized void restartTask(final ConnectorTaskId id, final Callback callback) {
        +        addRequest(new Callable() {
        +            @Override
        +            public Void call() throws Exception {
        +                // if config is out of sync, then a rebalance is likely to begin shortly, so rather than risking
        +                // a stale response, we return an error and let the user retry
        +                if (!isConfigSynced()) {
        +                    throw new StaleConfigException("Cannot complete request because config is out of sync");
        +                }
        +
        +                if (!configState.connectors().contains(id.connector())) {
        +                    callback.onCompletion(new NotFoundException("Unknown connector: " + id.connector()), null);
        +                    return null;
        +                }
        +
        +                if (configState.taskConfig(id) == null) {
        +                    callback.onCompletion(new NotFoundException("Unknown task: " + id), null);
        +                    return null;
        +                }
        +
        +                if (worker.ownsTask(id)) {
        +                    try {
        +                        worker.stopAndAwaitTask(id);
        +                        startTask(id);
        +                        callback.onCompletion(null, null);
        +                    } catch (Throwable t) {
        +                        callback.onCompletion(t, null);
        +                    }
        +                } else if (isLeader()) {
        +                    callback.onCompletion(new NotAssignedException("Cannot restart task since it is not assigned to this member", member.ownerUrl(id)), null);
        +                } else {
        +                    callback.onCompletion(new NotLeaderException("Cannot restart task since it is not assigned to this member", leaderUrl()), null);
        +                }
        +                return null;
        +            }
        +        }, forwardErrorCallback(callback));
        +    }
        +
             @Override
             public int generation() {
                 return generation;
        @@ -679,10 +752,7 @@ private void startWork() {
                 }
                 for (ConnectorTaskId taskId : assignment.tasks()) {
                     try {
        -                log.info("Starting task {}", taskId);
        -                Map configs = configState.taskConfig(taskId);
        -                TaskConfig taskConfig = new TaskConfig(configs);
        -                worker.startTask(taskId, taskConfig, this);
        +                startTask(taskId);
                     } catch (ConfigException e) {
                         log.error("Couldn't instantiate task " + taskId + " because it has an invalid task " +
                                 "configuration. This task will not execute until reconfigured.", e);
        @@ -691,6 +761,13 @@ private void startWork() {
                 log.info("Finished starting connectors and tasks");
             }
         
        +    private void startTask(ConnectorTaskId taskId) {
        +        log.info("Starting task {}", taskId);
        +        Map configs = configState.taskConfig(taskId);
        +        TaskConfig taskConfig = new TaskConfig(configs);
        +        worker.startTask(taskId, taskConfig, this);
        +    }
        +
             // Helper for starting a connector with the given name, which will extract & parse the config, generate connector
             // context and add to the worker. This needs to be called from within the main worker thread for this herder.
             private void startConnector(String connectorName) {
        @@ -791,10 +868,14 @@ public void run() {
                 }
             }
         
        +    private boolean isConfigSynced() {
        +        return assignment != null && configState.offset() == assignment.offset();
        +    }
        +
             // Common handling for requests that get config data. Checks if we are in sync with the current config, which allows
             // us to answer requests directly. If we are not, handles invoking the callback with the appropriate error.
             private boolean checkConfigSynced(Callback callback) {
        -        if (assignment == null || configState.offset() != assignment.offset()) {
        +        if (!isConfigSynced()) {
                     if (!isLeader())
                         callback.onCompletion(new NotLeaderException("Cannot get config data because config is not in sync and this is not the leader", leaderUrl()), null);
                     else
        diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotAssignedException.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotAssignedException.java
        new file mode 100644
        index 0000000000000..a4211cc231799
        --- /dev/null
        +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotAssignedException.java
        @@ -0,0 +1,29 @@
        +/**
        + * 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 org.apache.kafka.connect.runtime.distributed; + +/** + * Thrown when a request intended for the owner of a task or connector is received by a worker which doesn't + * own it (typically the leader). + */ +public class NotAssignedException extends RequestTargetException { + + public NotAssignedException(String message, String ownerUrl) { + super(message, ownerUrl); + } + +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotLeaderException.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotLeaderException.java index 5f94b53cea808..9340eda433d54 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotLeaderException.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/NotLeaderException.java @@ -17,31 +17,14 @@ package org.apache.kafka.connect.runtime.distributed; -import org.apache.kafka.connect.errors.ConnectException; - /** * Indicates an operation was not permitted because it can only be performed on the leader and this worker is not currently * the leader. */ -public class NotLeaderException extends ConnectException { - private final String leaderUrl; +public class NotLeaderException extends RequestTargetException { public NotLeaderException(String msg, String leaderUrl) { - super(msg); - this.leaderUrl = leaderUrl; - } - - public NotLeaderException(String msg, String leaderUrl, Throwable throwable) { - super(msg, throwable); - this.leaderUrl = leaderUrl; + super(msg, leaderUrl); } - public NotLeaderException(String leaderUrl, Throwable throwable) { - super(throwable); - this.leaderUrl = leaderUrl; - } - - public String leaderUrl() { - return leaderUrl; - } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/RequestTargetException.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/RequestTargetException.java new file mode 100644 index 0000000000000..42a5f5dd6838d --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/RequestTargetException.java @@ -0,0 +1,47 @@ +/** + * 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 org.apache.kafka.connect.runtime.distributed; + +import org.apache.kafka.connect.errors.ConnectException; + +/** + * Raised when a request has been received by a worker which cannot handle it, + * but can forward it to the right target + */ +public class RequestTargetException extends ConnectException { + private final String forwardUrl; + + public RequestTargetException(String s, String forwardUrl) { + super(s); + this.forwardUrl = forwardUrl; + } + + public RequestTargetException(String s, Throwable throwable, String forwardUrl) { + super(s, throwable); + this.forwardUrl = forwardUrl; + } + + public RequestTargetException(Throwable throwable, String forwardUrl) { + super(throwable); + this.forwardUrl = forwardUrl; + } + + public String forwardUrl() { + return forwardUrl; + } + +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/StaleConfigException.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/StaleConfigException.java new file mode 100644 index 0000000000000..c615b37757056 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/StaleConfigException.java @@ -0,0 +1,27 @@ +/** + * 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 org.apache.kafka.connect.runtime.distributed; + +import org.apache.kafka.connect.errors.ConnectException; + +public class StaleConfigException extends ConnectException { + + public StaleConfigException(String s) { + super(s); + } + +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java index fa50fbf06bac1..d5802c6c36ed5 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java @@ -54,6 +54,7 @@ public final class WorkerCoordinator extends AbstractCoordinator implements Clos private final WorkerCoordinatorMetrics sensors; private ClusterConfigState configSnapshot; private final WorkerRebalanceListener listener; + private LeaderState leaderState; private boolean rejoinRequested; @@ -198,6 +199,8 @@ private Map performTaskAssignment(String leaderId, long maxO } } + this.leaderState = new LeaderState(allConfigs, connectorAssignments, taskAssignments); + return fillAssignmentsAndSerialize(allConfigs.keySet(), ConnectProtocol.Assignment.NO_ERROR, leaderId, allConfigs.get(leaderId).url(), maxOffset, connectorAssignments, taskAssignments); } @@ -228,6 +231,7 @@ private Map fillAssignmentsAndSerialize(Collection m @Override protected void onJoinPrepare(int generation, String memberId) { + this.leaderState = null; log.debug("Revoking previous assignment {}", assignmentSnapshot); if (assignmentSnapshot != null && !assignmentSnapshot.failed()) listener.onRevoked(assignmentSnapshot.leader(), assignmentSnapshot.connectors(), assignmentSnapshot.tasks()); @@ -247,6 +251,22 @@ public void close() { super.close(); } + private boolean isLeader() { + return assignmentSnapshot != null && memberId.equals(assignmentSnapshot.leader()); + } + + public String ownerUrl(String connector) { + if (needRejoin() || !isLeader()) + return null; + return leaderState.ownerUrl(connector); + } + + public String ownerUrl(ConnectorTaskId task) { + if (needRejoin() || !isLeader()) + return null; + return leaderState.ownerUrl(task); + } + private class WorkerCoordinatorMetrics { public final Metrics metrics; public final String metricGrpName; @@ -282,4 +302,43 @@ private static > List sorted(Collection members) { return res; } + private static Map invertAssignment(Map> assignment) { + Map inverted = new HashMap<>(); + for (Map.Entry> assignmentEntry : assignment.entrySet()) { + K key = assignmentEntry.getKey(); + for (V value : assignmentEntry.getValue()) + inverted.put(value, key); + } + return inverted; + } + + private static class LeaderState { + private final Map allMembers; + private final Map connectorOwners; + private final Map taskOwners; + + public LeaderState(Map allMembers, + Map> connectorAssignment, + Map> taskAssignment) { + this.allMembers = allMembers; + this.connectorOwners = invertAssignment(connectorAssignment); + this.taskOwners = invertAssignment(taskAssignment); + } + + private String ownerUrl(ConnectorTaskId id) { + String ownerId = taskOwners.get(id); + if (ownerId == null) + return null; + return allMembers.get(ownerId).url(); + } + + private String ownerUrl(String connector) { + String ownerId = connectorOwners.get(connector); + if (ownerId == null) + return null; + return allMembers.get(ownerId).url(); + } + + } + } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java index 57028ef1386c0..058f17114ba81 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java @@ -33,6 +33,7 @@ import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.storage.KafkaConfigStorage; +import org.apache.kafka.connect.util.ConnectorTaskId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -175,6 +176,14 @@ public void maybeLeaveGroup() { coordinator.maybeLeaveGroup(); } + public String ownerUrl(String connector) { + return coordinator.ownerUrl(connector); + } + + public String ownerUrl(ConnectorTaskId task) { + return coordinator.ownerUrl(task); + } + private void stop(boolean swallowException) { log.trace("Stopping the Connect group member."); AtomicReference firstException = new AtomicReference(); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java index 1505a01332d33..3475e1c5683c6 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java @@ -20,7 +20,6 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; - import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.runtime.Herder; import org.apache.kafka.connect.runtime.WorkerConfig; @@ -48,6 +47,9 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import javax.servlet.DispatcherType; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.UriBuilder; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -58,10 +60,6 @@ import java.util.List; import java.util.Map; -import javax.servlet.DispatcherType; -import javax.ws.rs.core.Response; -import javax.ws.rs.core.UriBuilder; - /** * Embedded server for the REST API that provides the control plane for Kafka Connect workers. */ @@ -272,4 +270,5 @@ public static String urlJoin(String base, String path) { else return base + path; } + } \ No newline at end of file diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java index b6e9f61a75b37..a53ed7d6ba01c 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java @@ -18,10 +18,10 @@ package org.apache.kafka.connect.runtime.rest.resources; import com.fasterxml.jackson.core.type.TypeReference; - import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.runtime.Herder; -import org.apache.kafka.connect.runtime.distributed.NotLeaderException; +import org.apache.kafka.connect.runtime.distributed.StaleConfigException; +import org.apache.kafka.connect.runtime.distributed.RequestTargetException; import org.apache.kafka.connect.runtime.rest.RestServer; import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; @@ -33,14 +33,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.net.URI; -import java.util.Collection; -import java.util.List; -import java.util.Map; -import java.util.concurrent.ExecutionException; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.TimeoutException; - import javax.servlet.ServletContext; import javax.ws.rs.Consumes; import javax.ws.rs.DELETE; @@ -50,8 +42,17 @@ import javax.ws.rs.Path; import javax.ws.rs.PathParam; import javax.ws.rs.Produces; +import javax.ws.rs.QueryParam; import javax.ws.rs.core.MediaType; import javax.ws.rs.core.Response; +import javax.ws.rs.core.UriBuilder; +import java.net.URI; +import java.util.Collection; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; @Path("/connectors") @Produces(MediaType.APPLICATION_JSON) @@ -75,16 +76,17 @@ public ConnectorsResource(Herder herder) { @GET @Path("/") - public Collection listConnectors() throws Throwable { + public Collection listConnectors(final @QueryParam("forward") Boolean forward) throws Throwable { FutureCallback> cb = new FutureCallback<>(); herder.connectors(cb); return completeOrForwardRequest(cb, "/connectors", "GET", null, new TypeReference>() { - }); + }, forward); } @POST @Path("/") - public Response createConnector(final CreateConnectorRequest createRequest) throws Throwable { + public Response createConnector(final @QueryParam("forward") Boolean forward, + final CreateConnectorRequest createRequest) throws Throwable { String name = createRequest.name(); Map configs = createRequest.config(); if (!configs.containsKey(ConnectorConfig.NAME_CONFIG)) @@ -93,24 +95,26 @@ public Response createConnector(final CreateConnectorRequest createRequest) thro FutureCallback> cb = new FutureCallback<>(); herder.putConnectorConfig(name, configs, false, cb); Herder.Created info = completeOrForwardRequest(cb, "/connectors", "POST", createRequest, - new TypeReference() { }, new CreatedConnectorInfoTranslator()); + new TypeReference() { }, new CreatedConnectorInfoTranslator(), forward); return Response.created(URI.create("/connectors/" + name)).entity(info.result()).build(); } @GET @Path("/{connector}") - public ConnectorInfo getConnector(final @PathParam("connector") String connector) throws Throwable { + public ConnectorInfo getConnector(final @PathParam("connector") String connector, + final @QueryParam("forward") Boolean forward) throws Throwable { FutureCallback cb = new FutureCallback<>(); herder.connectorInfo(connector, cb); - return completeOrForwardRequest(cb, "/connectors/" + connector, "GET", null); + return completeOrForwardRequest(cb, "/connectors/" + connector, "GET", null, forward); } @GET @Path("/{connector}/config") - public Map getConnectorConfig(final @PathParam("connector") String connector) throws Throwable { + public Map getConnectorConfig(final @PathParam("connector") String connector, + final @QueryParam("forward") Boolean forward) throws Throwable { FutureCallback> cb = new FutureCallback<>(); herder.connectorConfig(connector, cb); - return completeOrForwardRequest(cb, "/connectors/" + connector + "/config", "GET", null); + return completeOrForwardRequest(cb, "/connectors/" + connector + "/config", "GET", null, forward); } @GET @@ -122,11 +126,12 @@ public ConnectorStateInfo getConnectorStatus(final @PathParam("connector") Strin @PUT @Path("/{connector}/config") public Response putConnectorConfig(final @PathParam("connector") String connector, - final Map connectorConfig) throws Throwable { + final @QueryParam("forward") Boolean forward, + final Map connectorConfig) throws Throwable { FutureCallback> cb = new FutureCallback<>(); herder.putConnectorConfig(connector, connectorConfig, true, cb); Herder.Created createdInfo = completeOrForwardRequest(cb, "/connectors/" + connector + "/config", - "PUT", connectorConfig, new TypeReference() { }, new CreatedConnectorInfoTranslator()); + "PUT", connectorConfig, new TypeReference() { }, new CreatedConnectorInfoTranslator(), forward); Response.ResponseBuilder response; if (createdInfo.created()) response = Response.created(URI.create("/connectors/" + connector)); @@ -135,55 +140,102 @@ public Response putConnectorConfig(final @PathParam("connector") String connecto return response.entity(createdInfo.result()).build(); } + @POST + @Path("/{connector}/restart") + public void restartConnector(final @PathParam("connector") String connector, + final @QueryParam("forward") Boolean forward) throws Throwable { + FutureCallback cb = new FutureCallback<>(); + herder.restartConnector(connector, cb); + completeOrForwardRequest(cb, "/connectors/" + connector + "/restart", "POST", null, forward); + } + @GET @Path("/{connector}/tasks") - public List getTaskConfigs(final @PathParam("connector") String connector) throws Throwable { + public List getTaskConfigs(final @PathParam("connector") String connector, + final @QueryParam("forward") Boolean forward) throws Throwable { FutureCallback> cb = new FutureCallback<>(); herder.taskConfigs(connector, cb); return completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks", "GET", null, new TypeReference>() { - }); + }, forward); } @POST @Path("/{connector}/tasks") public void putTaskConfigs(final @PathParam("connector") String connector, + final @QueryParam("forward") Boolean forward, final List> taskConfigs) throws Throwable { FutureCallback cb = new FutureCallback<>(); herder.putTaskConfigs(connector, taskConfigs, cb); - completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks", "POST", taskConfigs); + completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks", "POST", taskConfigs, forward); } @GET @Path("/{connector}/tasks/{task}/status") - public ConnectorStateInfo.TaskState getTaskStatus(@PathParam("connector") String connector, - @PathParam("task") Integer task) throws Throwable { + public ConnectorStateInfo.TaskState getTaskStatus(final @PathParam("connector") String connector, + final @PathParam("task") Integer task) throws Throwable { return herder.taskStatus(new ConnectorTaskId(connector, task)); } + @POST + @Path("/{connector}/tasks/{task}/restart") + public void restartTask(final @PathParam("connector") String connector, + final @PathParam("task") Integer task, + final @QueryParam("forward") Boolean forward) throws Throwable { + FutureCallback cb = new FutureCallback<>(); + ConnectorTaskId taskId = new ConnectorTaskId(connector, task); + herder.restartTask(taskId, cb); + completeOrForwardRequest(cb, "/connectors/" + connector + "/tasks/" + task + "/restart", "POST", null, forward); + } + @DELETE @Path("/{connector}") - public void destroyConnector(final @PathParam("connector") String connector) throws Throwable { + public void destroyConnector(final @PathParam("connector") String connector, + final @QueryParam("forward") Boolean forward) throws Throwable { FutureCallback> cb = new FutureCallback<>(); herder.putConnectorConfig(connector, null, true, cb); - completeOrForwardRequest(cb, "/connectors/" + connector, "DELETE", null); + completeOrForwardRequest(cb, "/connectors/" + connector, "DELETE", null, forward); } // Wait for a FutureCallback to complete. If it succeeds, return the parsed response. If it fails, try to forward the // request to the leader. - private T completeOrForwardRequest( - FutureCallback cb, String path, String method, Object body, TypeReference resultType, - Translator translator) throws Throwable { + private T completeOrForwardRequest(FutureCallback cb, + String path, + String method, + Object body, + TypeReference resultType, + Translator translator, + Boolean forward) throws Throwable { try { return cb.get(REQUEST_TIMEOUT_MS, TimeUnit.MILLISECONDS); } catch (ExecutionException e) { - if (e.getCause() instanceof NotLeaderException) { - NotLeaderException notLeaderError = (NotLeaderException) e.getCause(); - String forwardUrl = RestServer.urlJoin(notLeaderError.leaderUrl(), path); - log.debug("Forwarding request to leader: {} {} {}", forwardUrl, method, body); - return translator.translate(RestServer.httpRequest(forwardUrl, method, body, resultType)); + Throwable cause = e.getCause(); + + if (cause instanceof RequestTargetException) { + if (forward == null || forward) { + // the only time we allow recursive forwarding is when no forward flag has + // been set, which should only be seen by the first worker to handle a user request. + // this gives two total hops to resolve the request before giving up. + boolean recursiveForward = forward == null; + RequestTargetException targetException = (RequestTargetException) cause; + String forwardUrl = UriBuilder.fromUri(targetException.forwardUrl()) + .path(path) + .queryParam("forward", recursiveForward) + .build() + .toString(); + log.debug("Forwarding request {} {} {}", forwardUrl, method, body); + return translator.translate(RestServer.httpRequest(forwardUrl, method, body, resultType)); + } else { + // we should find the right target for the query within two hops, so if + // we don't, it probably means that a rebalance has taken place. + throw new ConnectRestException(Response.Status.CONFLICT.getStatusCode(), + "Cannot complete request because of a conflicting operation (e.g. worker rebalance)"); + } + } else if (cause instanceof StaleConfigException) { + throw new ConnectRestException(Response.Status.CONFLICT.getStatusCode(), + "Cannot complete request momentarily due to stale configuration (typically caused by a concurrent config change)"); } - throw e.getCause(); + throw cause; } catch (TimeoutException e) { // This timeout is for the operation itself. None of the timeout error codes are relevant, so internal server // error is the best option @@ -193,12 +245,14 @@ private T completeOrForwardRequest( } } - private T completeOrForwardRequest(FutureCallback cb, String path, String method, Object body, TypeReference resultType) throws Throwable { - return completeOrForwardRequest(cb, path, method, body, resultType, new IdentityTranslator()); + private T completeOrForwardRequest(FutureCallback cb, String path, String method, Object body, + TypeReference resultType, Boolean forward) throws Throwable { + return completeOrForwardRequest(cb, path, method, body, resultType, new IdentityTranslator(), forward); } - private T completeOrForwardRequest(FutureCallback cb, String path, String method, Object body) throws Throwable { - return completeOrForwardRequest(cb, path, method, body, null, new IdentityTranslator()); + private T completeOrForwardRequest(FutureCallback cb, String path, String method, + Object body, Boolean forward) throws Throwable { + return completeOrForwardRequest(cb, path, method, body, null, new IdentityTranslator(), forward); } private interface Translator { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java index 9c48ed79dd622..a59336a2cede3 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java @@ -202,6 +202,44 @@ public void putTaskConfigs(String connName, List> configs, C throw new UnsupportedOperationException("Kafka Connect in standalone mode does not support externally setting task configurations."); } + @Override + public synchronized void restartTask(ConnectorTaskId taskId, Callback cb) { + if (!connectors.containsKey(taskId.connector())) + cb.onCompletion(new NotFoundException("Connector " + taskId.connector() + " not found", null), null); + + ConnectorState state = connectors.get(taskId.connector()); + if (state.taskConfigs.contains(taskId)) + cb.onCompletion(new NotFoundException("Task " + taskId + " not found", null), null); + + try { + worker.stopAndAwaitTask(taskId); + + Map taskConfig = state.taskConfigs.get(taskId.task()); + worker.startTask(taskId, new TaskConfig(taskConfig), this); + + cb.onCompletion(null, null); + } catch (Exception e) { + log.error("Failed to restart task {}", taskId, e); + cb.onCompletion(e, null); + } + } + + @Override + public synchronized void restartConnector(String connName, Callback cb) { + if (!connectors.containsKey(connName)) + cb.onCompletion(new NotFoundException("Connector " + connName + " not found", null), null); + + ConnectorState state = connectors.get(connName); + try { + worker.stopConnector(connName); + worker.startConnector(state.config, new HerderConnectorContext(this, connName), this); + cb.onCompletion(null, null); + } catch (Exception e) { + log.error("Failed to restart connector {}", connName, e); + cb.onCompletion(e, null); + } + } + /** * Start a connector in the worker and record its state. * @param connectorProps new connector configuration @@ -234,19 +272,23 @@ private void createConnectorTasks(String connName) { int index = 0; for (Map taskConfigMap : state.taskConfigs) { ConnectorTaskId taskId = new ConnectorTaskId(connName, index); - TaskConfig config = new TaskConfig(taskConfigMap); - try { - worker.startTask(taskId, config, this); - } catch (Throwable e) { - log.error("Failed to add task {}: ", taskId, e); - // Swallow this so we can continue updating the rest of the tasks - // FIXME what's the proper response? Kill all the tasks? Consider this the same as a task - // that died after starting successfully. - } + startTask(taskId, taskConfigMap); index++; } } + private void startTask(ConnectorTaskId taskId, Map taskConfigMap) { + TaskConfig config = new TaskConfig(taskConfigMap); + try { + worker.startTask(taskId, config, this); + } catch (Throwable e) { + log.error("Failed to add task {}: ", taskId, e); + // Swallow this so we can continue updating the rest of the tasks + // FIXME what's the proper response? Kill all the tasks? Consider this the same as a task + // that died after starting successfully. + } + } + private Set tasksFor(ConnectorState state) { Set tasks = new HashSet<>(); for (int i = 0; i < state.taskConfigs.size(); i++) diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java index aa747f6ec70a7..b667fa89296a7 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.connect.connector.ConnectorContext; import org.apache.kafka.connect.errors.AlreadyExistsException; +import org.apache.kafka.connect.errors.NotFoundException; import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.runtime.Herder; import org.apache.kafka.connect.runtime.TaskConfig; @@ -56,10 +57,13 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; @RunWith(PowerMockRunner.class) @PrepareForTest(DistributedHerder.class) @@ -314,6 +318,303 @@ public void testDestroyConnector() throws Exception { PowerMock.verifyAll(); } + @Test + public void testRestartConnector() throws Exception { + EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andStubReturn(TASK_CONFIGS); + + // get the initial assignment + EasyMock.expect(member.memberId()).andStubReturn("leader"); + expectRebalance(1, Collections.singletonList(CONN1), Collections.emptyList()); + expectPostRebalanceCatchup(SNAPSHOT); + member.poll(EasyMock.anyInt()); + PowerMock.expectLastCall(); + worker.startConnector(EasyMock.anyObject(), EasyMock.anyObject(), + EasyMock.eq(herder)); + PowerMock.expectLastCall(); + + // now handle the connector restart + member.wakeup(); + PowerMock.expectLastCall(); + member.ensureActive(); + PowerMock.expectLastCall(); + member.poll(EasyMock.anyInt()); + PowerMock.expectLastCall(); + + EasyMock.expect(worker.ownsConnector(CONN1)).andReturn(true); + + worker.stopConnector(CONN1); + PowerMock.expectLastCall(); + worker.startConnector(EasyMock.anyObject(), EasyMock.anyObject(), + EasyMock.eq(herder)); + PowerMock.expectLastCall(); + + PowerMock.replayAll(); + + herder.tick(); + FutureCallback callback = new FutureCallback<>(); + herder.restartConnector(CONN1, callback); + herder.tick(); + callback.get(1000L, TimeUnit.MILLISECONDS); + + PowerMock.verifyAll(); + } + + @Test + public void testRestartUnknownConnector() throws Exception { + // get the initial assignment + EasyMock.expect(member.memberId()).andStubReturn("leader"); + expectRebalance(1, Collections.emptyList(), Collections.emptyList()); + expectPostRebalanceCatchup(SNAPSHOT); + member.poll(EasyMock.anyInt()); + PowerMock.expectLastCall(); + + + // now handle the connector restart + member.wakeup(); + PowerMock.expectLastCall(); + member.ensureActive(); + PowerMock.expectLastCall(); + member.poll(EasyMock.anyInt()); + PowerMock.expectLastCall(); + + PowerMock.replayAll(); + + herder.tick(); + FutureCallback callback = new FutureCallback<>(); + herder.restartConnector(CONN2, callback); + herder.tick(); + try { + callback.get(1000L, TimeUnit.MILLISECONDS); + fail("Expected NotLeaderException to be raised"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof NotFoundException); + } + + PowerMock.verifyAll(); + } + + @Test + public void testRestartConnectorRedirectToLeader() throws Exception { + // get the initial assignment + EasyMock.expect(member.memberId()).andStubReturn("member"); + expectRebalance(1, Collections.emptyList(), Collections.emptyList()); + expectPostRebalanceCatchup(SNAPSHOT); + member.poll(EasyMock.anyInt()); + PowerMock.expectLastCall(); + + // now handle the connector restart + member.wakeup(); + PowerMock.expectLastCall(); + member.ensureActive(); + PowerMock.expectLastCall(); + member.poll(EasyMock.anyInt()); + PowerMock.expectLastCall(); + + EasyMock.expect(worker.ownsConnector(CONN1)).andReturn(false); + + PowerMock.replayAll(); + + herder.tick(); + FutureCallback callback = new FutureCallback<>(); + herder.restartConnector(CONN1, callback); + herder.tick(); + + try { + callback.get(1000L, TimeUnit.MILLISECONDS); + fail("Expected NotLeaderException to be raised"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof NotLeaderException); + } + + PowerMock.verifyAll(); + } + + @Test + public void testRestartConnectorRedirectToOwner() throws Exception { + // get the initial assignment + EasyMock.expect(member.memberId()).andStubReturn("leader"); + expectRebalance(1, Collections.emptyList(), Collections.emptyList()); + expectPostRebalanceCatchup(SNAPSHOT); + member.poll(EasyMock.anyInt()); + PowerMock.expectLastCall(); + + // now handle the connector restart + member.wakeup(); + PowerMock.expectLastCall(); + member.ensureActive(); + PowerMock.expectLastCall(); + member.poll(EasyMock.anyInt()); + PowerMock.expectLastCall(); + + String ownerUrl = "ownerUrl"; + EasyMock.expect(worker.ownsConnector(CONN1)).andReturn(false); + EasyMock.expect(member.ownerUrl(CONN1)).andReturn(ownerUrl); + + PowerMock.replayAll(); + + herder.tick(); + FutureCallback callback = new FutureCallback<>(); + herder.restartConnector(CONN1, callback); + herder.tick(); + + try { + callback.get(1000L, TimeUnit.MILLISECONDS); + fail("Expected NotLeaderException to be raised"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof NotAssignedException); + NotAssignedException notAssignedException = (NotAssignedException) e.getCause(); + assertEquals(ownerUrl, notAssignedException.forwardUrl()); + } + + PowerMock.verifyAll(); + } + + @Test + public void testRestartTask() throws Exception { + EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andStubReturn(TASK_CONFIGS); + + // get the initial assignment + EasyMock.expect(member.memberId()).andStubReturn("leader"); + expectRebalance(1, Collections.emptyList(), Collections.singletonList(TASK0)); + expectPostRebalanceCatchup(SNAPSHOT); + member.poll(EasyMock.anyInt()); + PowerMock.expectLastCall(); + worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.eq(herder)); + PowerMock.expectLastCall(); + + // now handle the task restart + member.wakeup(); + PowerMock.expectLastCall(); + member.ensureActive(); + PowerMock.expectLastCall(); + member.poll(EasyMock.anyInt()); + PowerMock.expectLastCall(); + + EasyMock.expect(worker.ownsTask(TASK0)).andReturn(true); + + worker.stopAndAwaitTask(TASK0); + PowerMock.expectLastCall(); + worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.eq(herder)); + PowerMock.expectLastCall(); + + PowerMock.replayAll(); + + herder.tick(); + FutureCallback callback = new FutureCallback<>(); + herder.restartTask(TASK0, callback); + herder.tick(); + callback.get(1000L, TimeUnit.MILLISECONDS); + + PowerMock.verifyAll(); + } + + @Test + public void testRestartUnknownTask() throws Exception { + // get the initial assignment + EasyMock.expect(member.memberId()).andStubReturn("member"); + expectRebalance(1, Collections.emptyList(), Collections.emptyList()); + expectPostRebalanceCatchup(SNAPSHOT); + member.poll(EasyMock.anyInt()); + PowerMock.expectLastCall(); + + member.wakeup(); + PowerMock.expectLastCall(); + member.ensureActive(); + PowerMock.expectLastCall(); + member.poll(EasyMock.anyInt()); + PowerMock.expectLastCall(); + + PowerMock.replayAll(); + + FutureCallback callback = new FutureCallback<>(); + herder.tick(); + herder.restartTask(new ConnectorTaskId("blah", 0), callback); + herder.tick(); + + try { + callback.get(1000L, TimeUnit.MILLISECONDS); + fail("Expected NotLeaderException to be raised"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof NotFoundException); + } + + PowerMock.verifyAll(); + } + + @Test + public void testRestartTaskRedirectToLeader() throws Exception { + // get the initial assignment + EasyMock.expect(member.memberId()).andStubReturn("member"); + expectRebalance(1, Collections.emptyList(), Collections.emptyList()); + expectPostRebalanceCatchup(SNAPSHOT); + member.poll(EasyMock.anyInt()); + PowerMock.expectLastCall(); + + // now handle the task restart + EasyMock.expect(worker.ownsTask(TASK0)).andReturn(false); + member.wakeup(); + PowerMock.expectLastCall(); + member.ensureActive(); + PowerMock.expectLastCall(); + member.poll(EasyMock.anyInt()); + PowerMock.expectLastCall(); + + PowerMock.replayAll(); + + herder.tick(); + FutureCallback callback = new FutureCallback<>(); + herder.restartTask(TASK0, callback); + herder.tick(); + + try { + callback.get(1000L, TimeUnit.MILLISECONDS); + fail("Expected NotLeaderException to be raised"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof NotLeaderException); + } + + PowerMock.verifyAll(); + } + + @Test + public void testRestartTaskRedirectToOwner() throws Exception { + // get the initial assignment + EasyMock.expect(member.memberId()).andStubReturn("leader"); + expectRebalance(1, Collections.emptyList(), Collections.emptyList()); + expectPostRebalanceCatchup(SNAPSHOT); + member.poll(EasyMock.anyInt()); + PowerMock.expectLastCall(); + + // now handle the task restart + String ownerUrl = "ownerUrl"; + EasyMock.expect(worker.ownsTask(TASK0)).andReturn(false); + EasyMock.expect(member.ownerUrl(TASK0)).andReturn(ownerUrl); + member.wakeup(); + PowerMock.expectLastCall(); + member.ensureActive(); + PowerMock.expectLastCall(); + member.poll(EasyMock.anyInt()); + PowerMock.expectLastCall(); + + PowerMock.replayAll(); + + herder.tick(); + FutureCallback callback = new FutureCallback<>(); + herder.restartTask(TASK0, callback); + herder.tick(); + + try { + callback.get(1000L, TimeUnit.MILLISECONDS); + fail("Expected NotLeaderException to be raised"); + } catch (ExecutionException e) { + assertTrue(e.getCause() instanceof NotAssignedException); + NotAssignedException notAssignedException = (NotAssignedException) e.getCause(); + assertEquals(ownerUrl, notAssignedException.forwardUrl()); + } + + PowerMock.verifyAll(); + } + @Test public void testConnectorConfigAdded() { // If a connector was added, we need to rebalance diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java index 970f56c9522cc..fa7d997924c72 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java @@ -24,6 +24,7 @@ import org.apache.kafka.connect.errors.NotFoundException; import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.runtime.Herder; +import org.apache.kafka.connect.runtime.distributed.NotAssignedException; import org.apache.kafka.connect.runtime.distributed.NotLeaderException; import org.apache.kafka.connect.runtime.rest.RestServer; import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; @@ -64,6 +65,7 @@ public class ConnectorsResourceTest { private static final String LEADER_URL = "http://leader:8083/"; private static final String CONNECTOR_NAME = "test"; private static final String CONNECTOR2_NAME = "test2"; + private static final Boolean FORWARD = true; private static final Map CONNECTOR_CONFIG = new HashMap<>(); static { CONNECTOR_CONFIG.put("name", CONNECTOR_NAME); @@ -103,7 +105,7 @@ public void testListConnectors() throws Throwable { PowerMock.replayAll(); - Collection connectors = connectorsResource.listConnectors(); + Collection connectors = connectorsResource.listConnectors(FORWARD); // Ordering isn't guaranteed, compare sets assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_NAME, CONNECTOR2_NAME)), new HashSet<>(connectors)); @@ -116,13 +118,13 @@ public void testListConnectorsNotLeader() throws Throwable { herder.connectors(EasyMock.capture(cb)); expectAndCallbackNotLeaderException(cb); // Should forward request - EasyMock.expect(RestServer.httpRequest(EasyMock.eq("http://leader:8083/connectors"), EasyMock.eq("GET"), + EasyMock.expect(RestServer.httpRequest(EasyMock.eq("http://leader:8083/connectors?forward=false"), EasyMock.eq("GET"), EasyMock.isNull(), EasyMock.anyObject(TypeReference.class))) .andReturn(new RestServer.HttpResponse<>(200, new HashMap>(), Arrays.asList(CONNECTOR2_NAME, CONNECTOR_NAME))); PowerMock.replayAll(); - Collection connectors = connectorsResource.listConnectors(); + Collection connectors = connectorsResource.listConnectors(FORWARD); // Ordering isn't guaranteed, compare sets assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_NAME, CONNECTOR2_NAME)), new HashSet<>(connectors)); @@ -138,7 +140,7 @@ public void testListConnectorsNotSynced() throws Throwable { PowerMock.replayAll(); // throws - connectorsResource.listConnectors(); + connectorsResource.listConnectors(FORWARD); } @Test @@ -151,7 +153,7 @@ public void testCreateConnector() throws Throwable { PowerMock.replayAll(); - connectorsResource.createConnector(body); + connectorsResource.createConnector(FORWARD, body); PowerMock.verifyAll(); } @@ -164,12 +166,12 @@ public void testCreateConnectorNotLeader() throws Throwable { herder.putConnectorConfig(EasyMock.eq(CONNECTOR_NAME), EasyMock.eq(body.config()), EasyMock.eq(false), EasyMock.capture(cb)); expectAndCallbackNotLeaderException(cb); // Should forward request - EasyMock.expect(RestServer.httpRequest(EasyMock.eq("http://leader:8083/connectors"), EasyMock.eq("POST"), EasyMock.eq(body), EasyMock.anyObject())) + EasyMock.expect(RestServer.httpRequest(EasyMock.eq("http://leader:8083/connectors?forward=false"), EasyMock.eq("POST"), EasyMock.eq(body), EasyMock.anyObject())) .andReturn(new RestServer.HttpResponse<>(201, new HashMap>(), new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG, CONNECTOR_TASK_NAMES))); PowerMock.replayAll(); - connectorsResource.createConnector(body); + connectorsResource.createConnector(FORWARD, body); PowerMock.verifyAll(); @@ -186,7 +188,7 @@ public void testCreateConnectorExists() throws Throwable { PowerMock.replayAll(); - connectorsResource.createConnector(body); + connectorsResource.createConnector(FORWARD, body); PowerMock.verifyAll(); } @@ -199,7 +201,7 @@ public void testDeleteConnector() throws Throwable { PowerMock.replayAll(); - connectorsResource.destroyConnector(CONNECTOR_NAME); + connectorsResource.destroyConnector(CONNECTOR_NAME, FORWARD); PowerMock.verifyAll(); } @@ -210,12 +212,12 @@ public void testDeleteConnectorNotLeader() throws Throwable { herder.putConnectorConfig(EasyMock.eq(CONNECTOR_NAME), EasyMock.>isNull(), EasyMock.eq(true), EasyMock.capture(cb)); expectAndCallbackNotLeaderException(cb); // Should forward request - EasyMock.expect(RestServer.httpRequest("http://leader:8083/connectors/" + CONNECTOR_NAME, "DELETE", null, null)) + EasyMock.expect(RestServer.httpRequest("http://leader:8083/connectors/" + CONNECTOR_NAME + "?forward=false", "DELETE", null, null)) .andReturn(new RestServer.HttpResponse<>(204, new HashMap>(), null)); PowerMock.replayAll(); - connectorsResource.destroyConnector(CONNECTOR_NAME); + connectorsResource.destroyConnector(CONNECTOR_NAME, FORWARD); PowerMock.verifyAll(); } @@ -229,7 +231,7 @@ public void testDeleteConnectorNotFound() throws Throwable { PowerMock.replayAll(); - connectorsResource.destroyConnector(CONNECTOR_NAME); + connectorsResource.destroyConnector(CONNECTOR_NAME, FORWARD); PowerMock.verifyAll(); } @@ -242,7 +244,7 @@ public void testGetConnector() throws Throwable { PowerMock.replayAll(); - ConnectorInfo connInfo = connectorsResource.getConnector(CONNECTOR_NAME); + ConnectorInfo connInfo = connectorsResource.getConnector(CONNECTOR_NAME, FORWARD); assertEquals(new ConnectorInfo(CONNECTOR_NAME, CONNECTOR_CONFIG, CONNECTOR_TASK_NAMES), connInfo); PowerMock.verifyAll(); @@ -256,7 +258,7 @@ public void testGetConnectorConfig() throws Throwable { PowerMock.replayAll(); - Map connConfig = connectorsResource.getConnectorConfig(CONNECTOR_NAME); + Map connConfig = connectorsResource.getConnectorConfig(CONNECTOR_NAME, FORWARD); assertEquals(CONNECTOR_CONFIG, connConfig); PowerMock.verifyAll(); @@ -270,7 +272,7 @@ public void testGetConnectorConfigConnectorNotFound() throws Throwable { PowerMock.replayAll(); - connectorsResource.getConnectorConfig(CONNECTOR_NAME); + connectorsResource.getConnectorConfig(CONNECTOR_NAME, FORWARD); PowerMock.verifyAll(); } @@ -283,7 +285,7 @@ public void testPutConnectorConfig() throws Throwable { PowerMock.replayAll(); - connectorsResource.putConnectorConfig(CONNECTOR_NAME, CONNECTOR_CONFIG); + connectorsResource.putConnectorConfig(CONNECTOR_NAME, FORWARD, CONNECTOR_CONFIG); PowerMock.verifyAll(); } @@ -296,7 +298,7 @@ public void testGetConnectorTaskConfigs() throws Throwable { PowerMock.replayAll(); - List taskInfos = connectorsResource.getTaskConfigs(CONNECTOR_NAME); + List taskInfos = connectorsResource.getTaskConfigs(CONNECTOR_NAME, FORWARD); assertEquals(TASK_INFOS, taskInfos); PowerMock.verifyAll(); @@ -310,7 +312,7 @@ public void testGetConnectorTaskConfigsConnectorNotFound() throws Throwable { PowerMock.replayAll(); - connectorsResource.getTaskConfigs(CONNECTOR_NAME); + connectorsResource.getTaskConfigs(CONNECTOR_NAME, FORWARD); PowerMock.verifyAll(); } @@ -323,7 +325,7 @@ public void testPutConnectorTaskConfigs() throws Throwable { PowerMock.replayAll(); - connectorsResource.putTaskConfigs(CONNECTOR_NAME, TASK_CONFIGS); + connectorsResource.putTaskConfigs(CONNECTOR_NAME, FORWARD, TASK_CONFIGS); PowerMock.verifyAll(); } @@ -336,7 +338,108 @@ public void testPutConnectorTaskConfigsConnectorNotFound() throws Throwable { PowerMock.replayAll(); - connectorsResource.putTaskConfigs(CONNECTOR_NAME, TASK_CONFIGS); + connectorsResource.putTaskConfigs(CONNECTOR_NAME, FORWARD, TASK_CONFIGS); + + PowerMock.verifyAll(); + } + + @Test(expected = NotFoundException.class) + public void testRestartConnectorNotFound() throws Throwable { + final Capture> cb = Capture.newInstance(); + herder.restartConnector(EasyMock.eq(CONNECTOR_NAME), EasyMock.capture(cb)); + expectAndCallbackException(cb, new NotFoundException("not found")); + + PowerMock.replayAll(); + + connectorsResource.restartConnector(CONNECTOR_NAME, FORWARD); + + PowerMock.verifyAll(); + } + + @Test + public void testRestartConnectorLeaderRedirect() throws Throwable { + final Capture> cb = Capture.newInstance(); + herder.restartConnector(EasyMock.eq(CONNECTOR_NAME), EasyMock.capture(cb)); + expectAndCallbackNotLeaderException(cb); + + EasyMock.expect(RestServer.httpRequest(EasyMock.eq("http://leader:8083/connectors/" + CONNECTOR_NAME + "/restart?forward=true"), + EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.anyObject())) + .andReturn(new RestServer.HttpResponse<>(202, new HashMap>(), null)); + + PowerMock.replayAll(); + + connectorsResource.restartConnector(CONNECTOR_NAME, null); + + PowerMock.verifyAll(); + } + + @Test + public void testRestartConnectorOwnerRedirect() throws Throwable { + final Capture> cb = Capture.newInstance(); + herder.restartConnector(EasyMock.eq(CONNECTOR_NAME), EasyMock.capture(cb)); + String ownerUrl = "http://owner:8083"; + expectAndCallbackException(cb, new NotAssignedException("not owner test", ownerUrl)); + + EasyMock.expect(RestServer.httpRequest(EasyMock.eq("http://owner:8083/connectors/" + CONNECTOR_NAME + "/restart?forward=false"), + EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.anyObject())) + .andReturn(new RestServer.HttpResponse<>(202, new HashMap>(), null)); + + PowerMock.replayAll(); + + connectorsResource.restartConnector(CONNECTOR_NAME, true); + + PowerMock.verifyAll(); + } + + @Test(expected = NotFoundException.class) + public void testRestartTaskNotFound() throws Throwable { + ConnectorTaskId taskId = new ConnectorTaskId(CONNECTOR_NAME, 0); + final Capture> cb = Capture.newInstance(); + herder.restartTask(EasyMock.eq(taskId), EasyMock.capture(cb)); + expectAndCallbackException(cb, new NotFoundException("not found")); + + PowerMock.replayAll(); + + connectorsResource.restartTask(CONNECTOR_NAME, 0, FORWARD); + + PowerMock.verifyAll(); + } + + @Test + public void testRestartTaskLeaderRedirect() throws Throwable { + ConnectorTaskId taskId = new ConnectorTaskId(CONNECTOR_NAME, 0); + + final Capture> cb = Capture.newInstance(); + herder.restartTask(EasyMock.eq(taskId), EasyMock.capture(cb)); + expectAndCallbackNotLeaderException(cb); + + EasyMock.expect(RestServer.httpRequest(EasyMock.eq("http://leader:8083/connectors/" + CONNECTOR_NAME + "/tasks/0/restart?forward=true"), + EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.anyObject())) + .andReturn(new RestServer.HttpResponse<>(202, new HashMap>(), null)); + + PowerMock.replayAll(); + + connectorsResource.restartTask(CONNECTOR_NAME, 0, null); + + PowerMock.verifyAll(); + } + + @Test + public void testRestartTaskOwnerRedirect() throws Throwable { + ConnectorTaskId taskId = new ConnectorTaskId(CONNECTOR_NAME, 0); + + final Capture> cb = Capture.newInstance(); + herder.restartTask(EasyMock.eq(taskId), EasyMock.capture(cb)); + String ownerUrl = "http://owner:8083"; + expectAndCallbackException(cb, new NotAssignedException("not owner test", ownerUrl)); + + EasyMock.expect(RestServer.httpRequest(EasyMock.eq("http://owner:8083/connectors/" + CONNECTOR_NAME + "/tasks/0/restart?forward=false"), + EasyMock.eq("POST"), EasyMock.isNull(), EasyMock.anyObject())) + .andReturn(new RestServer.HttpResponse<>(202, new HashMap>(), null)); + + PowerMock.replayAll(); + + connectorsResource.restartTask(CONNECTOR_NAME, 0, true); PowerMock.verifyAll(); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java index 3959ff8f767d2..05a64a157e288 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java @@ -155,6 +155,160 @@ public void testDestroyConnector() throws Exception { PowerMock.verifyAll(); } + @Test + public void testRestartConnector() throws Exception { + expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false); + + worker.stopConnector(CONNECTOR_NAME); + EasyMock.expectLastCall(); + + worker.startConnector(EasyMock.eq(new ConnectorConfig(connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class))), + EasyMock.anyObject(HerderConnectorContext.class), EasyMock.eq(herder)); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class), false, createCallback); + + FutureCallback cb = new FutureCallback<>(); + herder.restartConnector(CONNECTOR_NAME, cb); + cb.get(1000L, TimeUnit.MILLISECONDS); + + PowerMock.verifyAll(); + } + + @Test + public void testRestartConnectorFailureOnStop() throws Exception { + expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false); + + RuntimeException e = new RuntimeException(); + worker.stopConnector(CONNECTOR_NAME); + EasyMock.expectLastCall().andThrow(e); + + // the connector will not be started after the failure in start + + PowerMock.replayAll(); + + herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class), false, createCallback); + + FutureCallback cb = new FutureCallback<>(); + herder.restartConnector(CONNECTOR_NAME, cb); + try { + cb.get(1000L, TimeUnit.MILLISECONDS); + fail(); + } catch (ExecutionException exception) { + assertEquals(e, exception.getCause()); + } + + PowerMock.verifyAll(); + } + + @Test + public void testRestartConnectorFailureOnStart() throws Exception { + expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false); + + worker.stopConnector(CONNECTOR_NAME); + EasyMock.expectLastCall(); + + RuntimeException e = new RuntimeException(); + worker.startConnector(EasyMock.eq(new ConnectorConfig(connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class))), + EasyMock.anyObject(HerderConnectorContext.class), EasyMock.eq(herder)); + EasyMock.expectLastCall().andThrow(e); + + PowerMock.replayAll(); + + herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class), false, createCallback); + + FutureCallback cb = new FutureCallback<>(); + herder.restartConnector(CONNECTOR_NAME, cb); + try { + cb.get(1000L, TimeUnit.MILLISECONDS); + fail(); + } catch (ExecutionException exception) { + assertEquals(e, exception.getCause()); + } + + PowerMock.verifyAll(); + } + + @Test + public void testRestartTask() throws Exception { + ConnectorTaskId taskId = new ConnectorTaskId(CONNECTOR_NAME, 0); + expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false); + + worker.stopAndAwaitTask(taskId); + EasyMock.expectLastCall(); + + Map generatedTaskProps = taskConfig(BogusSourceTask.class, false); + worker.startTask(taskId, new TaskConfig(generatedTaskProps), herder); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class), false, createCallback); + + FutureCallback cb = new FutureCallback<>(); + herder.restartTask(taskId, cb); + cb.get(1000L, TimeUnit.MILLISECONDS); + + PowerMock.verifyAll(); + } + + @Test + public void testRestartTaskFailureOnStop() throws Exception { + ConnectorTaskId taskId = new ConnectorTaskId(CONNECTOR_NAME, 0); + expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false); + + RuntimeException e = new RuntimeException(); + worker.stopAndAwaitTask(taskId); + EasyMock.expectLastCall().andThrow(e); + + // task will not be started after the failure in stop + + PowerMock.replayAll(); + + herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class), false, createCallback); + + FutureCallback cb = new FutureCallback<>(); + herder.restartTask(taskId, cb); + try { + cb.get(1000L, TimeUnit.MILLISECONDS); + fail("Expected restart callback to raise an exception"); + } catch (ExecutionException exception) { + assertEquals(e, exception.getCause()); + } + PowerMock.verifyAll(); + } + + @Test + public void testRestartTaskFailureOnStart() throws Exception { + ConnectorTaskId taskId = new ConnectorTaskId(CONNECTOR_NAME, 0); + expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false); + + worker.stopAndAwaitTask(taskId); + EasyMock.expectLastCall(); + + RuntimeException e = new RuntimeException(); + Map generatedTaskProps = taskConfig(BogusSourceTask.class, false); + worker.startTask(taskId, new TaskConfig(generatedTaskProps), herder); + EasyMock.expectLastCall().andThrow(e); + + PowerMock.replayAll(); + + herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class), false, createCallback); + + FutureCallback cb = new FutureCallback<>(); + herder.restartTask(taskId, cb); + try { + cb.get(1000L, TimeUnit.MILLISECONDS); + fail("Expected restart callback to raise an exception"); + } catch (ExecutionException exception) { + assertEquals(e, exception.getCause()); + } + + PowerMock.verifyAll(); + } + @Test public void testCreateAndStop() throws Exception { connector = PowerMock.createMock(BogusSourceConnector.class); From 03817d5a26722c6e95647f6219abf2802b187c8d Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Mon, 18 Apr 2016 12:49:36 -0700 Subject: [PATCH 149/206] KAFKA-3529: Fix transient failure in testCommitAsync Author: Jason Gustafson Reviewers: Ewen Cheslack-Postava Closes #1234 from hachikuji/KAFKA-3529 --- .../scala/integration/kafka/api/BaseConsumerTest.scala | 9 ++------- 1 file changed, 2 insertions(+), 7 deletions(-) diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index 916a0ab2d021d..56dae7617973e 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -77,10 +77,6 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { // check async commit callbacks val commitCallback = new CountConsumerCommitCallback() this.consumers(0).commitAsync(commitCallback) - - // shouldn't make progress until poll is invoked - Thread.sleep(10) - assertEquals(0, commitCallback.successCount) awaitCommitCallback(this.consumers(0), commitCallback) } @@ -331,11 +327,10 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { protected def awaitCommitCallback[K, V](consumer: Consumer[K, V], commitCallback: CountConsumerCommitCallback, count: Int = 1): Unit = { - val startCount = commitCallback.successCount val started = System.currentTimeMillis() - while (commitCallback.successCount < startCount + count && System.currentTimeMillis() - started < 10000) + while (commitCallback.successCount < count && System.currentTimeMillis() - started < 10000) consumer.poll(50) - assertEquals(startCount + count, commitCallback.successCount) + assertEquals(count, commitCallback.successCount) } protected class CountConsumerCommitCallback extends OffsetCommitCallback { From a81ad2582ee0e533d335fe0dc5c5cc885dbf645d Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Mon, 18 Apr 2016 14:10:26 -0700 Subject: [PATCH 150/206] KAFKA-3563: Maintain MessageAndMetadata constructor compatibility Author: Grant Henke Reviewers: Ismael Juma , Ashish Singh , Ewen Cheslack-Postava Closes #1226 from granthenke/message_constructor --- core/src/main/scala/kafka/consumer/ConsumerIterator.scala | 6 +++--- core/src/main/scala/kafka/message/MessageAndMetadata.scala | 6 +++--- 2 files changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala index 07356517fed2f..df98db7a734e3 100755 --- a/core/src/main/scala/kafka/consumer/ConsumerIterator.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerIterator.scala @@ -104,10 +104,10 @@ class ConsumerIterator[K, V](private val channel: BlockingQueue[FetchedDataChunk currentTopicInfo.partitionId, item.message, item.offset, - item.message.timestamp, - item.message.timestampType, keyDecoder, - valueDecoder) + valueDecoder, + item.message.timestamp, + item.message.timestampType) } def clearCurrentChunk() { diff --git a/core/src/main/scala/kafka/message/MessageAndMetadata.scala b/core/src/main/scala/kafka/message/MessageAndMetadata.scala index ac9ef77d515e7..5c09cafdbac34 100755 --- a/core/src/main/scala/kafka/message/MessageAndMetadata.scala +++ b/core/src/main/scala/kafka/message/MessageAndMetadata.scala @@ -25,10 +25,10 @@ case class MessageAndMetadata[K, V](topic: String, partition: Int, private val rawMessage: Message, offset: Long, + keyDecoder: Decoder[K], valueDecoder: Decoder[V], timestamp: Long = Message.NoTimestamp, - timestampType: TimestampType = TimestampType.CREATE_TIME, - keyDecoder: Decoder[K], valueDecoder: Decoder[V]) { - + timestampType: TimestampType = TimestampType.CREATE_TIME) { + /** * Return the decoded message key and payload */ From a5f1158c317e22a79c4186d1acb04fb25ce6e56a Mon Sep 17 00:00:00 2001 From: Ismael Juma Date: Mon, 18 Apr 2016 14:23:46 -0700 Subject: [PATCH 151/206] KAFKA-3558; Add compression_type parameter to benchmarks in benchmark_test.py * Use a fixed `Random` seed in `EndToEndLatency.scala` for determinism * Add `compression_type` to and remove `consumer_fetch_max_wait` from `end_to_end_latency.py`. The latter was never used. * Tweak logging of `end_to_end_latency.py` to be similar to `consumer_performance.py`. * Add `compression_type` to `benchmark_test.py` methods and add `snappy` to `matrix` annotation * Use randomly generated bytes from a restricted range for `ProducerPerformance` payload. This is a simple fix for now. It can be improved in the PR for KAFKA-3554. Author: Ismael Juma Reviewers: Ewen Cheslack-Postava Closes #1225 from ijuma/kafka-3558-add-compression_type-benchmark_test.py --- .../kafka/tools/ConsumerPerformance.scala | 14 ++-- .../scala/kafka/tools/EndToEndLatency.scala | 18 ++-- .../benchmarks/core/benchmark_test.py | 56 +++++++++---- .../performance/end_to_end_latency.py | 84 ++++++++++--------- .../kafka/tools/ProducerPerformance.java | 7 +- 5 files changed, 105 insertions(+), 74 deletions(-) diff --git a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala index a38c04b4ff322..6480ff5ae4ddd 100644 --- a/core/src/main/scala/kafka/tools/ConsumerPerformance.scala +++ b/core/src/main/scala/kafka/tools/ConsumerPerformance.scala @@ -19,8 +19,6 @@ package kafka.tools import java.util -import org.apache.kafka.common.TopicPartition - import scala.collection.JavaConversions._ import java.util.concurrent.atomic.AtomicLong import java.nio.channels.ClosedByInterruptException @@ -85,10 +83,9 @@ object ConsumerPerformance { thread.start for (thread <- threadList) thread.join - if(consumerTimeout.get()) - endMs = System.currentTimeMillis - consumerConfig.consumerTimeoutMs - else - endMs = System.currentTimeMillis + endMs = + if (consumerTimeout.get()) System.currentTimeMillis - consumerConfig.consumerTimeoutMs + else System.currentTimeMillis consumerConnector.shutdown() } val elapsedSecs = (endMs - startMs) / 1000.0 @@ -279,9 +276,8 @@ object ConsumerPerformance { } catch { case _: InterruptedException => case _: ClosedByInterruptException => - case _: ConsumerTimeoutException => { - consumerTimeout.set(true); - } + case _: ConsumerTimeoutException => + consumerTimeout.set(true) case e: Throwable => e.printStackTrace() } totalMessagesRead.addAndGet(messagesRead) diff --git a/core/src/main/scala/kafka/tools/EndToEndLatency.scala b/core/src/main/scala/kafka/tools/EndToEndLatency.scala index 584d4fb7ee7e5..1c920888c2917 100755 --- a/core/src/main/scala/kafka/tools/EndToEndLatency.scala +++ b/core/src/main/scala/kafka/tools/EndToEndLatency.scala @@ -25,6 +25,7 @@ import org.apache.kafka.common.utils.Utils import org.apache.kafka.common.TopicPartition import scala.collection.JavaConversions._ +import scala.util.Random /** @@ -43,7 +44,7 @@ object EndToEndLatency { def main(args: Array[String]) { if (args.length != 5 && args.length != 6) { - System.err.println("USAGE: java " + getClass.getName + " broker_list topic num_messages producer_acks message_size_bytes [optional] ssl_properties_file") + System.err.println("USAGE: java " + getClass.getName + " broker_list topic num_messages producer_acks message_size_bytes [optional] properties_file") System.exit(1) } @@ -52,12 +53,14 @@ object EndToEndLatency { val numMessages = args(2).toInt val producerAcks = args(3) val messageLen = args(4).toInt - val sslPropsFile = if (args.length == 6) args(5) else "" + val propsFile = if (args.length > 5) Some(args(5)).filter(_.nonEmpty) else None if (!List("1", "all").contains(producerAcks)) throw new IllegalArgumentException("Latency testing requires synchronous acknowledgement. Please use 1 or all") - val consumerProps = if (sslPropsFile.equals("")) new Properties() else Utils.loadProps(sslPropsFile) + def loadProps: Properties = propsFile.map(Utils.loadProps).getOrElse(new Properties()) + + val consumerProps = loadProps consumerProps.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) consumerProps.put(ConsumerConfig.GROUP_ID_CONFIG, "test-group-" + System.currentTimeMillis()) consumerProps.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false") @@ -69,7 +72,7 @@ object EndToEndLatency { val consumer = new KafkaConsumer[Array[Byte], Array[Byte]](consumerProps) consumer.subscribe(List(topic)) - val producerProps = if (sslPropsFile.equals("")) new Properties() else Utils.loadProps(sslPropsFile) + val producerProps = loadProps producerProps.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, brokerList) producerProps.put(ProducerConfig.LINGER_MS_CONFIG, "0") //ensure writes are synchronous producerProps.put(ProducerConfig.MAX_BLOCK_MS_CONFIG, Long.MaxValue.toString) @@ -91,9 +94,10 @@ object EndToEndLatency { var totalTime = 0.0 val latencies = new Array[Long](numMessages) + val random = new Random(0) for (i <- 0 until numMessages) { - val message = randomBytesOfLen(messageLen) + val message = randomBytesOfLen(random, messageLen) val begin = System.nanoTime //Send message (of random bytes) synchronously then immediately poll for it @@ -141,7 +145,7 @@ object EndToEndLatency { finalise() } - def randomBytesOfLen(len: Int): Array[Byte] = { - Array.fill(len)((scala.util.Random.nextInt(26) + 65).toByte) + def randomBytesOfLen(random: Random, len: Int): Array[Byte] = { + Array.fill(len)((random.nextInt(26) + 65).toByte) } } diff --git a/tests/kafkatest/benchmarks/core/benchmark_test.py b/tests/kafkatest/benchmarks/core/benchmark_test.py index d252e5dead264..83f4b2a23adf7 100644 --- a/tests/kafkatest/benchmarks/core/benchmark_test.py +++ b/tests/kafkatest/benchmarks/core/benchmark_test.py @@ -68,9 +68,10 @@ def start_kafka(self, security_protocol, interbroker_security_protocol, version) @parametrize(acks=1, topic=TOPIC_REP_THREE) @parametrize(acks=-1, topic=TOPIC_REP_THREE) @parametrize(acks=1, topic=TOPIC_REP_THREE, num_producers=3) - @matrix(acks=[1], topic=[TOPIC_REP_THREE], message_size=[10, 100, 1000, 10000, 100000], security_protocol=['PLAINTEXT', 'SSL']) - def test_producer_throughput(self, acks, topic, num_producers=1, message_size=DEFAULT_RECORD_SIZE, security_protocol='PLAINTEXT', - client_version=str(TRUNK), broker_version=str(TRUNK)): + @matrix(acks=[1], topic=[TOPIC_REP_THREE], message_size=[10, 100, 1000, 10000, 100000], compression_type=["none", "snappy"], security_protocol=['PLAINTEXT', 'SSL']) + def test_producer_throughput(self, acks, topic, num_producers=1, message_size=DEFAULT_RECORD_SIZE, + compression_type="none", security_protocol='PLAINTEXT', client_version=str(TRUNK), + broker_version=str(TRUNK)): """ Setup: 1 node zk + 3 node kafka cluster Produce ~128MB worth of messages to a topic with 6 partitions. Required acks, topic replication factor, @@ -91,15 +92,17 @@ def test_producer_throughput(self, acks, topic, num_producers=1, message_size=DE num_records=nrecords, record_size=message_size, throughput=-1, version=client_version, settings={ 'acks': acks, + 'compression.type': compression_type, 'batch.size': self.batch_size, 'buffer.memory': self.buffer_memory}) self.producer.run() return compute_aggregate_throughput(self.producer) @parametrize(security_protocol='SSL', interbroker_security_protocol='PLAINTEXT') - @matrix(security_protocol=['PLAINTEXT', 'SSL']) - def test_long_term_producer_throughput(self, security_protocol, interbroker_security_protocol=None, - client_version=str(TRUNK), broker_version=str(TRUNK)): + @matrix(security_protocol=['PLAINTEXT', 'SSL'], compression_type=["none", "snappy"]) + def test_long_term_producer_throughput(self, compression_type="none", security_protocol='PLAINTEXT', + interbroker_security_protocol=None, client_version=str(TRUNK), + broker_version=str(TRUNK)): """ Setup: 1 node zk + 3 node kafka cluster Produce 10e6 100 byte messages to a topic with 6 partitions, replication-factor 3, and acks=1. @@ -117,7 +120,12 @@ def test_long_term_producer_throughput(self, security_protocol, interbroker_secu self.producer = ProducerPerformanceService( self.test_context, 1, self.kafka, topic=TOPIC_REP_THREE, num_records=self.msgs_large, record_size=DEFAULT_RECORD_SIZE, - throughput=-1, version=client_version, settings={'acks': 1, 'batch.size': self.batch_size, 'buffer.memory': self.buffer_memory}, + throughput=-1, version=client_version, settings={ + 'acks': 1, + 'compression.type': compression_type, + 'batch.size': self.batch_size, + 'buffer.memory': self.buffer_memory + }, intermediate_stats=True ) self.producer.run() @@ -146,9 +154,10 @@ def test_long_term_producer_throughput(self, security_protocol, interbroker_secu return data @parametrize(security_protocol='SSL', interbroker_security_protocol='PLAINTEXT') - @matrix(security_protocol=['PLAINTEXT', 'SSL', 'SASL_PLAINTEXT', 'SASL_SSL']) - def test_end_to_end_latency(self, security_protocol, interbroker_security_protocol=None, - client_version=str(TRUNK), broker_version=str(TRUNK)): + @matrix(security_protocol=['PLAINTEXT', 'SSL', 'SASL_PLAINTEXT', 'SASL_SSL'], compression_type=["none", "snappy"]) + def test_end_to_end_latency(self, compression_type="none", security_protocol="PLAINTEXT", + interbroker_security_protocol=None, client_version=str(TRUNK), + broker_version=str(TRUNK)): """ Setup: 1 node zk + 3 node kafka cluster Produce (acks = 1) and consume 10e3 messages to a topic with 6 partitions and replication-factor 3, @@ -167,15 +176,17 @@ def test_end_to_end_latency(self, security_protocol, interbroker_security_protoc self.logger.info("BENCHMARK: End to end latency") self.perf = EndToEndLatencyService( self.test_context, 1, self.kafka, - topic=TOPIC_REP_THREE, num_records=10000, version=client_version + topic=TOPIC_REP_THREE, num_records=10000, + compression_type=compression_type, version=client_version ) self.perf.run() return latency(self.perf.results[0]['latency_50th_ms'], self.perf.results[0]['latency_99th_ms'], self.perf.results[0]['latency_999th_ms']) @parametrize(security_protocol='PLAINTEXT', new_consumer=False) @parametrize(security_protocol='SSL', interbroker_security_protocol='PLAINTEXT') - @matrix(security_protocol=['PLAINTEXT', 'SSL']) - def test_producer_and_consumer(self, security_protocol, interbroker_security_protocol=None, new_consumer=True, + @matrix(security_protocol=['PLAINTEXT', 'SSL'], compression_type=["none", "snappy"]) + def test_producer_and_consumer(self, compression_type="none", security_protocol="PLAINTEXT", + interbroker_security_protocol=None, new_consumer=True, client_version=str(TRUNK), broker_version=str(TRUNK)): """ Setup: 1 node zk + 3 node kafka cluster @@ -198,7 +209,12 @@ def test_producer_and_consumer(self, security_protocol, interbroker_security_pro self.test_context, 1, self.kafka, topic=TOPIC_REP_THREE, num_records=num_records, record_size=DEFAULT_RECORD_SIZE, throughput=-1, version=client_version, - settings={'acks': 1, 'batch.size': self.batch_size, 'buffer.memory': self.buffer_memory} + settings={ + 'acks': 1, + 'compression.type': compression_type, + 'batch.size': self.batch_size, + 'buffer.memory': self.buffer_memory + } ) self.consumer = ConsumerPerformanceService( self.test_context, 1, self.kafka, topic=TOPIC_REP_THREE, new_consumer=new_consumer, messages=num_records) @@ -216,8 +232,9 @@ def test_producer_and_consumer(self, security_protocol, interbroker_security_pro @parametrize(security_protocol='PLAINTEXT', new_consumer=False) @parametrize(security_protocol='SSL', interbroker_security_protocol='PLAINTEXT') - @matrix(security_protocol=['PLAINTEXT', 'SSL']) - def test_consumer_throughput(self, security_protocol, interbroker_security_protocol=None, new_consumer=True, num_consumers=1, + @matrix(security_protocol=['PLAINTEXT', 'SSL'], compression_type=["none", "snappy"]) + def test_consumer_throughput(self, compression_type="none", security_protocol="PLAINTEXT", + interbroker_security_protocol=None, new_consumer=True, num_consumers=1, client_version=str(TRUNK), broker_version=str(TRUNK)): """ Consume 10e6 100-byte messages with 1 or more consumers from a topic with 6 partitions @@ -236,7 +253,12 @@ def test_consumer_throughput(self, security_protocol, interbroker_security_proto self.test_context, 1, self.kafka, topic=TOPIC_REP_THREE, num_records=num_records, record_size=DEFAULT_RECORD_SIZE, throughput=-1, version=client_version, - settings={'acks': 1, 'batch.size': self.batch_size, 'buffer.memory': self.buffer_memory} + settings={ + 'acks': 1, + 'compression.type': compression_type, + 'batch.size': self.batch_size, + 'buffer.memory': self.buffer_memory + } ) self.producer.run() diff --git a/tests/kafkatest/services/performance/end_to_end_latency.py b/tests/kafkatest/services/performance/end_to_end_latency.py index 08eff70cf1ef8..6d21151d530da 100644 --- a/tests/kafkatest/services/performance/end_to_end_latency.py +++ b/tests/kafkatest/services/performance/end_to_end_latency.py @@ -17,32 +17,53 @@ from kafkatest.services.security.security_config import SecurityConfig from kafkatest.services.kafka.directory import kafka_dir -from kafkatest.services.kafka.version import TRUNK, V_0_9_0_0 +from kafkatest.services.kafka.version import TRUNK, V_0_9_0_0, V_0_10_0_0 +import os class EndToEndLatencyService(PerformanceService): MESSAGE_BYTES = 21 # 0.8.X messages are fixed at 21 bytes, so we'll match that for other versions + # Root directory for persistent output + PERSISTENT_ROOT = "/mnt/end_to_end_latency" + LOG_DIR = os.path.join(PERSISTENT_ROOT, "logs") + STDOUT_CAPTURE = os.path.join(PERSISTENT_ROOT, "end_to_end_latency.stdout") + STDERR_CAPTURE = os.path.join(PERSISTENT_ROOT, "end_to_end_latency.stderr") + LOG_FILE = os.path.join(LOG_DIR, "end_to_end_latency.log") + LOG4J_CONFIG = os.path.join(PERSISTENT_ROOT, "tools-log4j.properties") + CONFIG_FILE = os.path.join(PERSISTENT_ROOT, "client.properties") + logs = { - "end_to_end_latency_log": { - "path": "/mnt/end-to-end-latency.log", + "end_to_end_latency_output": { + "path": STDOUT_CAPTURE, + "collect_default": True}, + "end_to_end_latency_stderr": { + "path": STDERR_CAPTURE, "collect_default": True}, + "end_to_end_latency_log": { + "path": LOG_FILE, + "collect_default": True} } - def __init__(self, context, num_nodes, kafka, topic, num_records, version=TRUNK, consumer_fetch_max_wait=100, acks=1): + + def __init__(self, context, num_nodes, kafka, topic, num_records, compression_type="none", version=TRUNK, acks=1): super(EndToEndLatencyService, self).__init__(context, num_nodes) self.kafka = kafka self.security_config = kafka.security_config.client_config() security_protocol = self.security_config.security_protocol - assert version >= V_0_9_0_0 or security_protocol == SecurityConfig.PLAINTEXT, \ - "Security protocol %s is only supported if version >= 0.9.0.0, version %s" % (self.security_config, str(version)) + + if version < V_0_9_0_0: + assert security_protocol == SecurityConfig.PLAINTEXT, \ + "Security protocol %s is only supported if version >= 0.9.0.0, version %s" % (self.security_config, str(version)) + assert compression_type == "none", \ + "Compression type %s is only supported if version >= 0.9.0.0, version %s" % (compression_type, str(version)) self.args = { 'topic': topic, 'num_records': num_records, - 'consumer_fetch_max_wait': consumer_fetch_max_wait, 'acks': acks, + 'compression_type': compression_type, 'kafka_opts': self.security_config.kafka_opts, 'message_bytes': EndToEndLatencyService.MESSAGE_BYTES } @@ -50,56 +71,41 @@ def __init__(self, context, num_nodes, kafka, topic, num_records, version=TRUNK, for node in self.nodes: node.version = version - @property - def security_config_file(self): - if self.security_config.security_protocol != SecurityConfig.PLAINTEXT: - security_config_file = SecurityConfig.CONFIG_DIR + "/security.properties" - else: - security_config_file = "" - return security_config_file - def start_cmd(self, node): args = self.args.copy() args.update({ 'zk_connect': self.kafka.zk.connect_setting(), 'bootstrap_servers': self.kafka.bootstrap_servers(self.security_config.security_protocol), - 'security_config_file': self.security_config_file, + 'config_file': EndToEndLatencyService.CONFIG_FILE, 'kafka_dir': kafka_dir(node) }) + cmd = "export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%s\"; " % EndToEndLatencyService.LOG4J_CONFIG if node.version >= V_0_9_0_0: - """ - val brokerList = args(0) - val topic = args(1) - val numMessages = args(2).toInt - val producerAcks = args(3) - val messageLen = args(4).toInt - """ - - cmd = "KAFKA_OPTS=%(kafka_opts)s /opt/%(kafka_dir)s/bin/kafka-run-class.sh kafka.tools.EndToEndLatency " % args - cmd += "%(bootstrap_servers)s %(topic)s %(num_records)d %(acks)d %(message_bytes)d %(security_config_file)s" % args + cmd += "KAFKA_OPTS=%(kafka_opts)s /opt/%(kafka_dir)s/bin/kafka-run-class.sh kafka.tools.EndToEndLatency " % args + cmd += "%(bootstrap_servers)s %(topic)s %(num_records)d %(acks)d %(message_bytes)d %(config_file)s" % args else: - """ - val brokerList = args(0) - val zkConnect = args(1) - val topic = args(2) - val numMessages = args(3).toInt - val consumerFetchMaxWait = args(4).toInt - val producerAcks = args(5).toInt - """ - # Set fetch max wait to 0 to match behavior in later versions - cmd = "KAFKA_OPTS=%(kafka_opts)s /opt/%(kafka_dir)s/bin/kafka-run-class.sh kafka.tools.TestEndToEndLatency " % args + cmd += "KAFKA_OPTS=%(kafka_opts)s /opt/%(kafka_dir)s/bin/kafka-run-class.sh kafka.tools.TestEndToEndLatency " % args cmd += "%(bootstrap_servers)s %(zk_connect)s %(topic)s %(num_records)d 0 %(acks)d" % args - cmd += " | tee /mnt/end-to-end-latency.log" + cmd += " 2>> %(stderr)s | tee -a %(stdout)s" % {'stdout': EndToEndLatencyService.STDOUT_CAPTURE, + 'stderr': EndToEndLatencyService.STDERR_CAPTURE} return cmd def _worker(self, idx, node): + node.account.ssh("mkdir -p %s" % EndToEndLatencyService.PERSISTENT_ROOT, allow_fail=False) + + log_config = self.render('tools_log4j.properties', log_file=EndToEndLatencyService.LOG_FILE) + + node.account.create_file(EndToEndLatencyService.LOG4J_CONFIG, log_config) + client_config = str(self.security_config) + if node.version >= V_0_9_0_0: + client_config += "compression_type=%(compression_type)s" % self.args + node.account.create_file(EndToEndLatencyService.CONFIG_FILE, client_config) + self.security_config.setup_node(node) - if self.security_config.security_protocol != SecurityConfig.PLAINTEXT: - node.account.create_file(self.security_config_file, str(self.security_config)) cmd = self.start_cmd(node) self.logger.debug("End-to-end latency %d command: %s", idx, cmd) diff --git a/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java b/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java index 18daf09b62dfd..b83227f0173d7 100644 --- a/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java +++ b/tools/src/main/java/org/apache/kafka/tools/ProducerPerformance.java @@ -17,6 +17,7 @@ import java.util.Arrays; import java.util.List; import java.util.Properties; +import java.util.Random; import org.apache.kafka.clients.producer.Callback; import org.apache.kafka.clients.producer.KafkaProducer; @@ -59,8 +60,10 @@ public static void main(String[] args) throws Exception { /* setup perf test */ byte[] payload = new byte[recordSize]; - Arrays.fill(payload, (byte) 1); - ProducerRecord record = new ProducerRecord(topicName, payload); + Random random = new Random(0); + for (int i = 0; i < payload.length; ++i) + payload[i] = (byte) (random.nextInt(26) + 65); + ProducerRecord record = new ProducerRecord<>(topicName, payload); Stats stats = new Stats(numRecords, 5000); long startMs = System.currentTimeMillis(); From f89f5fb907711e4696bee8a8d7bb53c6941ed693 Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Mon, 18 Apr 2016 18:49:17 -0700 Subject: [PATCH 152/206] KAFKA-3582; Remove references to Copcyat from Kafka Connect property files junrao Author: Liquan Pei Reviewers: Jun Rao Closes #1236 from Ishiihara/minor-fix --- config/connect-distributed.properties | 2 +- config/connect-standalone.properties | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/config/connect-distributed.properties b/config/connect-distributed.properties index 46bd3bcfcbb72..b25339f83e91d 100644 --- a/config/connect-distributed.properties +++ b/config/connect-distributed.properties @@ -30,7 +30,7 @@ key.converter.schemas.enable=true value.converter.schemas.enable=true # The internal converter used for offsets and config data is configurable and must be specified, but most users will -# always want to use the built-in default. Offset and config data is never visible outside of Copcyat in this format. +# always want to use the built-in default. Offset and config data is never visible outside of Kafka Connect in this format. internal.key.converter=org.apache.kafka.connect.json.JsonConverter internal.value.converter=org.apache.kafka.connect.json.JsonConverter internal.key.converter.schemas.enable=false diff --git a/config/connect-standalone.properties b/config/connect-standalone.properties index 8c4f98e327d2a..8760590a6da50 100644 --- a/config/connect-standalone.properties +++ b/config/connect-standalone.properties @@ -26,7 +26,7 @@ key.converter.schemas.enable=true value.converter.schemas.enable=true # The internal converter used for offsets and config data is configurable and must be specified, but most users will -# always want to use the built-in default. Offset and config data is never visible outside of Copcyat in this format. +# always want to use the built-in default. Offset and config data is never visible outside of Kafka Connect in this format. internal.key.converter=org.apache.kafka.connect.json.JsonConverter internal.value.converter=org.apache.kafka.connect.json.JsonConverter internal.key.converter.schemas.enable=false From 501fa37222ee7bb6c1883441af05fa883c51d93b Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Tue, 19 Apr 2016 11:06:31 -0700 Subject: [PATCH 153/206] KAFKA-3421: Update docs with new connector features ewencp gwenshap Docs. I also tried to clean up some typos. However, it seems that the we don't have two words without space in between in the source yet they showed up as no space in between in the generated doc. Author: Liquan Pei Reviewers: Ewen Cheslack-Postava Closes #1227 from Ishiihara/config-doc --- docs/connect.html | 34 ++++++++++++++++++++++------------ 1 file changed, 22 insertions(+), 12 deletions(-) diff --git a/docs/connect.html b/docs/connect.html index 88b8c2b5c34c9..5cd4130d691cb 100644 --- a/docs/connect.html +++ b/docs/connect.html @@ -25,7 +25,7 @@

        8.1 Overview

      1. Distributed and standalone modes - scale up to a large, centrally managed service supporting an entire organization or scale down to development, testing, and small production deployments
      2. REST interface - submit and manage connectors to your Kafka Connect cluster via an easy to use REST API
      3. Automatic offset management - with just a little information from connectors, Kafka Connect can manage the offset commit process automatically so connector developers do not need to worry about this error prone part of connector development
      4. -
      5. Distributed and scalable by default - Kafka Connect builds on the existing
      6. +
      7. Distributed and scalable by default - Kafka Connect builds on the existing group management protocol. More workers can be added to scale up a Kafka Connect cluster.
      8. Streaming/batch integration - leveraging Kafka's existing capabilities, Kafka Connect is an ideal solution for bridging streaming and batch data systems
      9. @@ -76,6 +76,8 @@

        Configuring Connecto
      10. tasks.max - The maximum number of tasks that should be created for this connector. The connector may create fewer tasks if it cannot achieve this level of parallelism.
      11. +The connector.class config supports several formats: the full name or alias of the class for this connector. If the connector is org.apache.kafka.connect.file.FileStreamSinkConnector, you can either specify this full name or use FileStreamSink or FileStreamSinkConnector to make the configuration a bit shorter. + Sink connectors also have one additional option to control their input:
        • topics - A list of topics to use as input for this connector
        • @@ -83,10 +85,9 @@

          Configuring Connecto For any other options, you should consult the documentation for the connector. -

          REST API

          -Since Kafka Connect is intended to be run as a service, it also supports a REST API for managing connectors. By default this service runs on port 8083. The following are the currently supported endpoints: +Since Kafka Connect is intended to be run as a service, it also provides a REST API for managing connectors. By default this service runs on port 8083. The following are the currently supported endpoints:
          • GET /connectors - return a list of active connectors
          • @@ -98,6 +99,13 @@

            REST API

          • DELETE /connectors/{name} - delete a connector, halting all tasks and deleting its configuration
          +Kafka Connect also provides a REST API for getting information about connector plugins: + +
            +
          • GET /connector-plugins- return a list of connector plugins installed in the Kafka Connect cluster. Note that the API only checks for connectors on the worker that handles the request, which means you may see inconsistent results, especially during a rolling upgrade if you add new connector jars
          • +
          • PUT /connector-plugins/{connector-type}/config/validate - validate the provided configuration values against the configuration definition. This API performs per config validation, returns suggested values and error messages during validation.
          • +
          +

          8.3 Connector Development Guide

          This guide describes how developers can write new connectors for Kafka Connect to move data between Kafka and other systems. It briefly reviews a few key concepts and then describes how to create a simple connector. @@ -183,6 +191,9 @@
          Connector } +Although not used in the example, SourceTask also provides two APIs to commit offsets in the source system: commit and commitSourceRecord. The APIs are provided for source systems which have an acknowledgement mechanism for messages. Overriding these methods allows the source connector to acknowledge messages in the source system, either in bulk or individually, once they have been written to Kafka. +The commit API stores the offsets in the source system, up to the offsets that have been returned by poll. The implementation of this API should block until the commit is complete. The commitSourceRecord API saves the offset in the source system for each SourceRecord after it is written to Kafka. As Kafka Connect will record offsets automatically, SourceTasks are not required to implement them. In cases where a connector does need to acknowledge messages in the source system, only one of the APIs is typically required. + Even with multiple tasks, this method implementation is usually pretty simple. It just has to determine the number of input tasks, which may require contacting the remote service it is pulling data from, and then divvy them up. Because some patterns for splitting work among tasks are so common, some utilities are provided in ConnectorUtils to simplify these cases. Note that this simple example does not include dynamic input. See the discussion in the next section for how to trigger updates to task configs. @@ -257,7 +268,7 @@
          Sink Tasks
          public abstract void flush(Map<TopicPartition, Long> offsets); -The SinkTask documentation contains full details, but this interface is nearly as simple as the the SourceTask. The put() method should contain most of the implementation, accepting sets of SinkRecords, performing any required translation, and storing them in the destination system. This method does not need to ensure the data has been fully written to the destination system before returning. In fact, in many cases internal buffering will be useful so an entire batch of records can be sent at once, reducing the overhead of inserting events into the downstream data store. The SinkRecords contain essentially the same information as SourceRecords: Kafka topic, partition, offset and the event key and value. +The SinkTask documentation contains full details, but this interface is nearly as simple as the SourceTask. The put() method should contain most of the implementation, accepting sets of SinkRecords, performing any required translation, and storing them in the destination system. This method does not need to ensure the data has been fully written to the destination system before returning. In fact, in many cases internal buffering will be useful so an entire batch of records can be sent at once, reducing the overhead of inserting events into the downstream data store. The SinkRecords contain essentially the same information as SourceRecords: Kafka topic, partition, offset and the event key and value. The flush() method is used during the offset commit process, which allows tasks to recover from failures and resume from a safe point such that no events will be missed. The method should push any outstanding data to the destination system and then block until the write has been acknowledged. The offsets parameter can often be ignored, but is useful in some cases where implementations want to store offset information in the destination store to provide exactly-once delivery. For example, an HDFS connector could do this and use atomic move operations to make sure the flush() operation atomically commits the data and offsets to a final location in HDFS. @@ -287,7 +298,6 @@

          Dynamic Input/Output Str Source connectors need to monitor the source system for changes, e.g. table additions/deletions in a database. When they pick up changes, they should notify the framework via the ConnectorContext object that reconfiguration is necessary. For example, in a SourceConnector: -
           if (inputsChanged())
               this.context.requestTaskReconfiguration();
          @@ -309,15 +319,15 @@ 

          Working with Schemas

          Schema schema = SchemaBuilder.struct().name(NAME) - .field("name", Schema.STRING_SCHEMA) - .field("age", Schema.INT_SCHEMA) - .field("admin", new SchemaBuilder.boolean().defaultValue(false).build()) - .build(); + .field("name", Schema.STRING_SCHEMA) + .field("age", Schema.INT_SCHEMA) + .field("admin", new SchemaBuilder.boolean().defaultValue(false).build()) + .build(); Struct struct = new Struct(schema) - .put("name", "Barbara Liskov") - .put("age", 75) - .build(); + .put("name", "Barbara Liskov") + .put("age", 75) + .build();
          If you are implementing a source connector, you'll need to decide when and how to create schemas. Where possible, you should avoid recomputing them as much as possible. For example, if your connector is guaranteed to have a fixed schema, create it statically and reuse a single instance. From 0bf61039c858af17260878d4815dbe7fb1645f90 Mon Sep 17 00:00:00 2001 From: Ishita Mandhan Date: Tue, 19 Apr 2016 17:39:04 -0700 Subject: [PATCH 154/206] MINOR: Fix typos in code comments This patch fixes all occurances of two consecutive 'the's in the code comments. Author: Ishita Mandhan (imandhaus.ibm.com) Author: Ishita Mandhan Reviewers: Guozhang Wang Closes #1240 from imandhan/typofixes --- .../kafka/clients/producer/internals/RecordAccumulator.java | 2 +- .../main/java/org/apache/kafka/common/network/LoginType.java | 2 +- .../main/java/org/apache/kafka/common/protocol/Protocol.java | 2 +- .../main/java/org/apache/kafka/connect/data/ConnectSchema.java | 2 +- core/src/main/scala/kafka/admin/TopicCommand.scala | 2 +- core/src/main/scala/kafka/utils/ZkUtils.scala | 2 +- core/src/test/scala/unit/kafka/log/LogTest.scala | 2 +- .../test/scala/unit/kafka/server/ClientQuotaManagerTest.scala | 2 +- docs/streams.html | 2 +- 9 files changed, 9 insertions(+), 9 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index 7f5b16f244eca..d963981524433 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -467,7 +467,7 @@ public void abortIncompleteBatches() { abortBatches(); } while (appendsInProgress()); // After this point, no thread will append any messages because they will see the close - // flag set. We need to do the last abort after no thread was appending in case the there was a new + // flag set. We need to do the last abort after no thread was appending in case there was a new // batch appended by the last appending thread. abortBatches(); this.batches.clear(); diff --git a/clients/src/main/java/org/apache/kafka/common/network/LoginType.java b/clients/src/main/java/org/apache/kafka/common/network/LoginType.java index 9216cb0c5c08f..a3a2b279e6b43 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/LoginType.java +++ b/clients/src/main/java/org/apache/kafka/common/network/LoginType.java @@ -20,7 +20,7 @@ /** * The type of the login context, it should be SERVER for the broker and CLIENT for the clients (i.e. consumer and - * producer). It provides the the login context name which defines the section of the JAAS configuration file to be used + * producer). It provides the login context name which defines the section of the JAAS configuration file to be used * for login. */ public enum LoginType { diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index 475a4f2c362e1..248b7ecc51c98 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -478,7 +478,7 @@ public class Protocol { "The current state of the group (one of: Dead, Stable, AwaitingSync, or PreparingRebalance, or empty if there is no active group)"), new Field("protocol_type", STRING, - "The current group protocol type (will be empty if the there is no active group)"), + "The current group protocol type (will be empty if there is no active group)"), new Field("protocol", STRING, "The current group protocol (only provided if the group is Stable)"), diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java b/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java index 37c056a692849..591644c1cbd3c 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/ConnectSchema.java @@ -299,7 +299,7 @@ public String toString() { /** - * Get the {@link Schema.Type} associated with the the given class. + * Get the {@link Schema.Type} associated with the given class. * * @param klass the Class to * @return the corresponding type, nor null if there is no matching type diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index dd4ea88a5aeaf..232db4ac1d5f8 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -379,7 +379,7 @@ object TopicCommand extends Logging { def shortMessageSizeWarning(maxMessageBytes: Int): String = { "\n\n" + "*****************************************************************************************************\n" + - "*** WARNING: you are creating a topic where the the max.message.bytes is greater than the consumer ***\n" + + "*** WARNING: you are creating a topic where the max.message.bytes is greater than the consumer ***\n" + "*** default. This operation is potentially dangerous. Consumers will get failures if their ***\n" + "*** fetch.message.max.bytes < the value you are using. ***\n" + "*****************************************************************************************************\n" + diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index bd8ec7ec87765..155b3fdd94222 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -668,7 +668,7 @@ class ZkUtils(val zkClient: ZkClient, } } - // Parses without deduplicating keys so the the data can be checked before allowing reassignment to proceed + // Parses without deduplicating keys so the data can be checked before allowing reassignment to proceed def parsePartitionReassignmentDataWithoutDedup(jsonData: String): Seq[(TopicAndPartition, Seq[Int])] = { Json.parseFull(jsonData) match { case Some(m) => diff --git a/core/src/test/scala/unit/kafka/log/LogTest.scala b/core/src/test/scala/unit/kafka/log/LogTest.scala index 3f6a2753a555c..8c973a45f4168 100755 --- a/core/src/test/scala/unit/kafka/log/LogTest.scala +++ b/core/src/test/scala/unit/kafka/log/LogTest.scala @@ -228,7 +228,7 @@ class LogTest extends JUnitSuite { /** * Test reading at the boundary of the log, specifically * - reading from the logEndOffset should give an empty message set - * - reading from the the maxOffset should give an empty message set + * - reading from the maxOffset should give an empty message set * - reading beyond the log end offset should throw an OffsetOutOfRangeException */ @Test diff --git a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala index 193acfd2bb771..69e83c03be88e 100644 --- a/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala +++ b/core/src/test/scala/unit/kafka/server/ClientQuotaManagerTest.scala @@ -104,7 +104,7 @@ class ClientQuotaManagerTest { assertEquals(10, numCallbacks) time.sleep(sleepTime) - // Callback can only be triggered after the the delay time passes + // Callback can only be triggered after the delay time passes clientMetrics.throttledRequestReaper.doWork() assertEquals(0, queueSizeMetric.value().toInt) assertEquals(11, numCallbacks) diff --git a/docs/streams.html b/docs/streams.html index 9b94bb32c06df..91fda368ae6d6 100644 --- a/docs/streams.html +++ b/docs/streams.html @@ -64,7 +64,7 @@
          Stream Processing Topology
          Time

          -A critical aspect in stream processing is the the notion of time, and how it is modeled and integrated. +A critical aspect in stream processing is the notion of time, and how it is modeled and integrated. For example, some operations such as windowing are defined based on time boundaries.

          From 280efe7f789d21f0c45657e14830451c1c8fc2b4 Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Wed, 20 Apr 2016 10:16:44 -0700 Subject: [PATCH 155/206] MINOR: Remove RollingBounceTest since its functionality is covered by the ReplicationTest system test RollingBounceTest is a system test that cannot be run reliably in unit tests and ReplicationTest is a superset of the functionality: in addition to verifying that bouncing leaders eventually results in a new leader, ReplicationTest also validates that data continues to be produced and consumed. Author: Ewen Cheslack-Postava Reviewers: Gwen Shapira Closes #1242 from ewencp/minor-remove-rolling-bounce-integration-test --- .../kafka/integration/RollingBounceTest.scala | 95 ------------------- 1 file changed, 95 deletions(-) delete mode 100755 core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala diff --git a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala b/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala deleted file mode 100755 index 5221855603a3a..0000000000000 --- a/core/src/test/scala/unit/kafka/integration/RollingBounceTest.scala +++ /dev/null @@ -1,95 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package kafka.integration - -import org.junit.{Test, After, Before} -import kafka.zk.ZooKeeperTestHarness -import kafka.utils.TestUtils._ -import org.junit.Assert._ -import kafka.utils.{CoreUtils, TestUtils} -import kafka.server.{KafkaConfig, KafkaServer} - -class RollingBounceTest extends ZooKeeperTestHarness { - - val partitionId = 0 - var servers: Seq[KafkaServer] = null - - @Before - override def setUp() { - super.setUp() - // controlled.shutdown.enable is true by default - val configs = (0 until 4).map(i => TestUtils.createBrokerConfig(i, zkConnect)) - configs(3).put("controlled.shutdown.retry.backoff.ms", "100") - - // start all the servers - servers = configs.map(c => TestUtils.createServer(KafkaConfig.fromProps(c))) - } - - @After - override def tearDown() { - servers.foreach(_.shutdown()) - servers.foreach(server => CoreUtils.delete(server.config.logDirs)) - super.tearDown() - } - - @Test - def testRollingBounce { - // start all the brokers - val topic1 = "new-topic1" - val topic2 = "new-topic2" - val topic3 = "new-topic3" - val topic4 = "new-topic4" - - // create topics with 1 partition, 2 replicas, one on each broker - createTopic(zkUtils, topic1, partitionReplicaAssignment = Map(0->Seq(0,1)), servers = servers) - createTopic(zkUtils, topic2, partitionReplicaAssignment = Map(0->Seq(1,2)), servers = servers) - createTopic(zkUtils, topic3, partitionReplicaAssignment = Map(0->Seq(2,3)), servers = servers) - createTopic(zkUtils, topic4, partitionReplicaAssignment = Map(0->Seq(0,3)), servers = servers) - - // Do a rolling bounce and check if leader transitions happen correctly - - // Bring down the leader for the first topic - bounceServer(topic1, 0) - - // Bring down the leader for the second topic - bounceServer(topic2, 1) - - // Bring down the leader for the third topic - bounceServer(topic3, 2) - - // Bring down the leader for the fourth topic - bounceServer(topic4, 3) - } - - private def bounceServer(topic: String, startIndex: Int) { - var prevLeader = 0 - if (isLeaderLocalOnBroker(topic, partitionId, servers(startIndex))) { - servers(startIndex).shutdown() - prevLeader = startIndex - } - else { - servers((startIndex + 1) % 4).shutdown() - prevLeader = (startIndex + 1) % 4 - } - var newleader = waitUntilLeaderIsElectedOrChanged(zkUtils, topic, partitionId) - // Ensure the new leader is different from the old - assertTrue("Leader transition did not happen for " + topic, newleader.getOrElse(-1) != -1 && (newleader.getOrElse(-1) != prevLeader)) - // Start the server back up again - servers(prevLeader).startup() - } -} From c9485b78a6e43747daf1314ae9532839fb7bc810 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Wed, 20 Apr 2016 14:09:59 -0700 Subject: [PATCH 156/206] KAFKA-2370: kafka connect pause/resume API Author: Jason Gustafson Reviewers: Liquan Pei , Ewen Cheslack-Postava Closes #1087 from hachikuji/KAFKA-2370 --- .../kafka/common/config/AbstractConfig.java | 3 +- .../kafka/connect/cli/ConnectDistributed.java | 10 +- .../kafka/connect/cli/ConnectStandalone.java | 2 +- .../kafka/connect/runtime/AbstractHerder.java | 52 +- .../kafka/connect/runtime/AbstractStatus.java | 1 + .../apache/kafka/connect/runtime/Connect.java | 6 +- .../connect/runtime/ConnectorStatus.java | 17 +- .../apache/kafka/connect/runtime/Herder.java | 15 + .../kafka/connect/runtime/TargetState.java | 36 ++ .../kafka/connect/runtime/TaskStatus.java | 12 + .../apache/kafka/connect/runtime/Worker.java | 107 ++--- .../connect/runtime/WorkerConnector.java | 206 ++++++++ .../kafka/connect/runtime/WorkerSinkTask.java | 76 +-- .../connect/runtime/WorkerSourceTask.java | 19 +- .../kafka/connect/runtime/WorkerTask.java | 128 +++-- .../distributed/ClusterConfigState.java | 46 +- .../distributed/DistributedHerder.java | 449 ++++++++++-------- ...ion.java => RebalanceNeededException.java} | 4 +- .../distributed/WorkerCoordinator.java | 6 +- .../distributed/WorkerGroupMember.java | 4 +- .../rest/resources/ConnectorsResource.java | 18 +- .../runtime/standalone/StandaloneHerder.java | 218 +++++---- .../connect/storage/ConfigBackingStore.java | 126 +++++ ...rage.java => KafkaConfigBackingStore.java} | 171 +++++-- .../storage/MemoryConfigBackingStore.java | 154 ++++++ .../connect/storage/StatusBackingStore.java | 2 +- .../connect/runtime/AbstractHerderTest.java | 23 +- .../connect/runtime/WorkerConnectorTest.java | 336 +++++++++++++ .../connect/runtime/WorkerSinkTaskTest.java | 115 ++++- .../runtime/WorkerSinkTaskThreadedTest.java | 33 +- .../connect/runtime/WorkerSourceTaskTest.java | 134 +++++- .../kafka/connect/runtime/WorkerTaskTest.java | 36 +- .../kafka/connect/runtime/WorkerTest.java | 30 +- .../distributed/DistributedHerderTest.java | 143 ++++-- .../distributed/WorkerCoordinatorTest.java | 12 +- .../standalone/StandaloneHerderTest.java | 35 +- ....java => KafkaConfigBackingStoreTest.java} | 67 +-- 37 files changed, 2173 insertions(+), 679 deletions(-) create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TargetState.java create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java rename connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/{StaleConfigException.java => RebalanceNeededException.java} (89%) create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConfigBackingStore.java rename connect/runtime/src/main/java/org/apache/kafka/connect/storage/{KafkaConfigStorage.java => KafkaConfigBackingStore.java} (83%) create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java create mode 100644 connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConnectorTest.java rename connect/runtime/src/test/java/org/apache/kafka/connect/storage/{KafkaConfigStorageTest.java => KafkaConfigBackingStoreTest.java} (89%) diff --git a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java index f833d7e784b05..8e36f40cfdbea 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java +++ b/clients/src/main/java/org/apache/kafka/common/config/AbstractConfig.java @@ -137,7 +137,8 @@ public Map originalsStrings() { Map copy = new RecordingMap<>(); for (Map.Entry entry : originals.entrySet()) { if (!(entry.getValue() instanceof String)) - throw new ClassCastException("Non-string value found in original settings"); + throw new ClassCastException("Non-string value found in original settings for key " + entry.getKey() + + ": " + (entry.getValue() == null ? null : entry.getValue().getClass().getName())); copy.put(entry.getKey(), (String) entry.getValue()); } return copy; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java index 849fa2f5ac1a3..e7a0c36c0c52f 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectDistributed.java @@ -26,6 +26,8 @@ import org.apache.kafka.connect.runtime.distributed.DistributedConfig; import org.apache.kafka.connect.runtime.distributed.DistributedHerder; import org.apache.kafka.connect.runtime.rest.RestServer; +import org.apache.kafka.connect.storage.ConfigBackingStore; +import org.apache.kafka.connect.storage.KafkaConfigBackingStore; import org.apache.kafka.connect.storage.KafkaOffsetBackingStore; import org.apache.kafka.connect.storage.KafkaStatusBackingStore; import org.apache.kafka.connect.storage.StatusBackingStore; @@ -74,8 +76,12 @@ public static void main(String[] args) throws Exception { StatusBackingStore statusBackingStore = new KafkaStatusBackingStore(time, worker.getInternalValueConverter()); statusBackingStore.configure(config); - DistributedHerder herder = new DistributedHerder(config, time, worker, statusBackingStore, advertisedUrl.toString()); - final Connect connect = new Connect(worker, herder, rest); + ConfigBackingStore configBackingStore = new KafkaConfigBackingStore(worker.getInternalValueConverter()); + configBackingStore.configure(config); + + DistributedHerder herder = new DistributedHerder(config, time, worker, statusBackingStore, configBackingStore, + advertisedUrl.toString()); + final Connect connect = new Connect(herder, rest); try { connect.start(); } catch (Exception e) { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java index 6c4335e6b8d0e..4ade18c878fc3 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/cli/ConnectStandalone.java @@ -76,7 +76,7 @@ public static void main(String[] args) throws Exception { Worker worker = new Worker(workerId, time, config, new FileOffsetBackingStore()); Herder herder = new StandaloneHerder(worker); - final Connect connect = new Connect(worker, herder, rest); + final Connect connect = new Connect(herder, rest); connect.start(); try { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java index 1d87d605ce0ba..a22f15c135e53 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java @@ -29,6 +29,7 @@ import org.apache.kafka.connect.runtime.rest.entities.ConfigValueInfo; import org.apache.kafka.connect.runtime.rest.entities.ConnectorPluginInfo; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; +import org.apache.kafka.connect.storage.ConfigBackingStore; import org.apache.kafka.connect.storage.StatusBackingStore; import org.apache.kafka.connect.tools.VerifiableSinkConnector; import org.apache.kafka.connect.tools.VerifiableSourceConnector; @@ -75,28 +76,37 @@ */ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, ConnectorStatus.Listener { + private final String workerId; protected final Worker worker; protected final StatusBackingStore statusBackingStore; - private final String workerId; + protected final ConfigBackingStore configBackingStore; private Map tempConnectors = new ConcurrentHashMap<>(); private static final List> SKIPPED_CONNECTORS = Arrays.>asList(VerifiableSourceConnector.class, VerifiableSinkConnector.class); private static List validConnectorPlugins; - public AbstractHerder(Worker worker, StatusBackingStore statusBackingStore, String workerId) { + public AbstractHerder(Worker worker, + String workerId, + StatusBackingStore statusBackingStore, + ConfigBackingStore configBackingStore) { this.worker = worker; - this.statusBackingStore = statusBackingStore; this.workerId = workerId; + this.statusBackingStore = statusBackingStore; + this.configBackingStore = configBackingStore; } protected abstract int generation(); protected void startServices() { + this.worker.start(); this.statusBackingStore.start(); + this.configBackingStore.start(); } protected void stopServices() { this.statusBackingStore.stop(); + this.configBackingStore.stop(); + this.worker.stop(); } @Override @@ -105,6 +115,18 @@ public void onStartup(String connector) { workerId, generation())); } + @Override + public void onPause(String connector) { + statusBackingStore.put(new ConnectorStatus(connector, ConnectorStatus.State.PAUSED, + workerId, generation())); + } + + @Override + public void onResume(String connector) { + statusBackingStore.put(new ConnectorStatus(connector, TaskStatus.State.RUNNING, + workerId, generation())); + } + @Override public void onShutdown(String connector) { statusBackingStore.putSafe(new ConnectorStatus(connector, ConnectorStatus.State.UNASSIGNED, @@ -132,6 +154,16 @@ public void onShutdown(ConnectorTaskId id) { statusBackingStore.putSafe(new TaskStatus(id, TaskStatus.State.UNASSIGNED, workerId, generation())); } + @Override + public void onResume(ConnectorTaskId id) { + statusBackingStore.put(new TaskStatus(id, TaskStatus.State.RUNNING, workerId, generation())); + } + + @Override + public void onPause(ConnectorTaskId id) { + statusBackingStore.put(new TaskStatus(id, TaskStatus.State.PAUSED, workerId, generation())); + } + @Override public void onDeletion(String connector) { for (TaskStatus status : statusBackingStore.getAll(connector)) @@ -139,6 +171,20 @@ public void onDeletion(String connector) { statusBackingStore.put(new ConnectorStatus(connector, ConnectorStatus.State.DESTROYED, workerId, generation())); } + @Override + public void pauseConnector(String connector) { + if (!configBackingStore.contains(connector)) + throw new NotFoundException("Unknown connector " + connector); + configBackingStore.putTargetState(connector, TargetState.PAUSED); + } + + @Override + public void resumeConnector(String connector) { + if (!configBackingStore.contains(connector)) + throw new NotFoundException("Unknown connector " + connector); + configBackingStore.putTargetState(connector, TargetState.STARTED); + } + @Override public ConnectorStateInfo connectorStatus(String connName) { ConnectorStatus connector = statusBackingStore.get(connName); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractStatus.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractStatus.java index 4f31be1bf7632..d00b81f60a0c5 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractStatus.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractStatus.java @@ -21,6 +21,7 @@ public abstract class AbstractStatus { public enum State { UNASSIGNED, RUNNING, + PAUSED, FAILED, DESTROYED, } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java index 49cf4bc6e0942..86f7f23f7a295 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Connect.java @@ -33,7 +33,6 @@ public class Connect { private static final Logger log = LoggerFactory.getLogger(Connect.class); - private final Worker worker; private final Herder herder; private final RestServer rest; private final CountDownLatch startLatch = new CountDownLatch(1); @@ -41,9 +40,8 @@ public class Connect { private final AtomicBoolean shutdown = new AtomicBoolean(false); private final ShutdownHook shutdownHook; - public Connect(Worker worker, Herder herder, RestServer rest) { + public Connect(Herder herder, RestServer rest) { log.debug("Kafka Connect instance created"); - this.worker = worker; this.herder = herder; this.rest = rest; shutdownHook = new ShutdownHook(); @@ -54,7 +52,6 @@ public void start() { log.info("Kafka Connect starting"); Runtime.getRuntime().addShutdownHook(shutdownHook); - worker.start(); herder.start(); rest.start(herder); @@ -72,7 +69,6 @@ public void stop() { rest.stop(); herder.stop(); - worker.stop(); log.info("Kafka Connect stopped"); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorStatus.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorStatus.java index d9a2eba960887..de5d3936ac924 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorStatus.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/ConnectorStatus.java @@ -16,6 +16,7 @@ **/ package org.apache.kafka.connect.runtime; + public class ConnectorStatus extends AbstractStatus { public ConnectorStatus(String connector, State state, String msg, String workerUrl, int generation) { @@ -35,13 +36,27 @@ public interface Listener { void onShutdown(String connector); /** - * Invoked from the Connector using {@link org.apache.kafka.connect.connector.ConnectorContext#raiseError(Exception)}. + * Invoked from the Connector using {@link org.apache.kafka.connect.connector.ConnectorContext#raiseError(Exception)} + * or if either {@link org.apache.kafka.connect.connector.Connector#start(java.util.Map)} or + * {@link org.apache.kafka.connect.connector.Connector#stop()} throw an exception. * Note that no shutdown event will follow after the task has been failed. * @param connector The connector name * @param cause Error raised from the connector. */ void onFailure(String connector, Throwable cause); + /** + * Invoked when the connector is paused through the REST API + * @param connector The connector name + */ + void onPause(String connector); + + /** + * Invoked after the connector has been resumed. + * @param connector The connector name + */ + void onResume(String connector); + /** * Invoked after successful startup of the connector. * @param connector The connector name diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java index cce100e16018c..ce8bcf99abcf0 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Herder.java @@ -149,6 +149,21 @@ public interface Herder { */ void restartConnector(String connName, Callback cb); + /** + * Pause the connector. This call will asynchronously suspend processing by the connector and all + * of its tasks. + * @param connector name of the connector + */ + void pauseConnector(String connector); + + /** + * Resume the connector. This call will asynchronously start the connector and its tasks (if + * not started already). + * @param connector name of the connector + */ + void resumeConnector(String connector); + + class Created { private final boolean created; private final T result; diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TargetState.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TargetState.java new file mode 100644 index 0000000000000..b59b3bbbd4793 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TargetState.java @@ -0,0 +1,36 @@ +/** + * 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 org.apache.kafka.connect.runtime; + +/** + * The target state of a connector is its desired state as indicated by the user + * through interaction with the REST API. When a connector is first created, its + * target state is "STARTED." This does not mean it has actually started, just that + * the Connect framework will attempt to start it after its tasks have been assigned. + * After the connector has been paused, the target state will change to PAUSED, + * and all the tasks will stop doing work. + * + * Target states are persisted in the config topic, which is read by all of the + * workers in the group. When a worker sees a new target state for a connector which + * is running, it will transition any tasks which it owns (i.e. which have been + * assigned to it by the leader) into the desired target state. Upon completion of + * a task rebalance, the worker will start the task in the last known target state. + */ +public enum TargetState { + STARTED, + PAUSED, +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskStatus.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskStatus.java index 3542eb84a9071..173a694e4f6b8 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskStatus.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/TaskStatus.java @@ -36,6 +36,18 @@ public interface Listener { */ void onStartup(ConnectorTaskId id); + /** + * Invoked after the task has been paused. + * @param id The id of the task + */ + void onPause(ConnectorTaskId id); + + /** + * Invoked after the task has been resumed. + * @param id The id of the task + */ + void onResume(ConnectorTaskId id); + /** * Invoked if the task raises an error. No shutdown event will follow. * @param id The id of the task diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java index 22843d38b56f2..a88d0f9280996 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/Worker.java @@ -26,7 +26,6 @@ import org.apache.kafka.connect.connector.ConnectorContext; import org.apache.kafka.connect.connector.Task; import org.apache.kafka.connect.errors.ConnectException; -import org.apache.kafka.connect.sink.SinkConnector; import org.apache.kafka.connect.sink.SinkTask; import org.apache.kafka.connect.source.SourceTask; import org.apache.kafka.connect.storage.Converter; @@ -132,10 +131,10 @@ public void stop() { long limit = started + config.getLong(WorkerConfig.TASK_SHUTDOWN_GRACEFUL_TIMEOUT_MS_CONFIG); for (Map.Entry entry : connectors.entrySet()) { - WorkerConnector conn = entry.getValue(); + WorkerConnector workerConnector = entry.getValue(); log.warn("Shutting down connector {} uncleanly; herder should have shut down connectors before the" + - "Worker is stopped.", conn); - conn.stop(); + "Worker is stopped.", entry.getKey()); + workerConnector.shutdown(); } Collection taskIds = tasks.keySet(); @@ -157,8 +156,12 @@ public void stop() { * @param connConfig connector configuration * @param ctx context for the connector * @param statusListener listener for notifications of connector status changes + * @param initialState the initial target state that the connector should be initialized to */ - public void startConnector(ConnectorConfig connConfig, ConnectorContext ctx, ConnectorStatus.Listener statusListener) { + public void startConnector(ConnectorConfig connConfig, + ConnectorContext ctx, + ConnectorStatus.Listener statusListener, + TargetState initialState) { String connName = connConfig.getString(ConnectorConfig.NAME_CONFIG); Class connClass = getConnectorClass(connConfig.getString(ConnectorConfig.CONNECTOR_CLASS_CONFIG)); @@ -171,22 +174,17 @@ public void startConnector(ConnectorConfig connConfig, ConnectorContext ctx, Con WorkerConnector workerConnector = new WorkerConnector(connName, connector, ctx, statusListener); log.info("Instantiated connector {} with version {} of type {}", connName, connector.version(), connClass.getName()); - workerConnector.initialize(); - try { - workerConnector.start(connConfig.originalsStrings()); - } catch (ConnectException e) { - throw new ConnectException("Connector threw an exception while starting", e); - } + workerConnector.initialize(connConfig); + workerConnector.transitionTo(initialState); connectors.put(connName, workerConnector); - log.info("Finished creating connector {}", connName); } /* Now that the configuration doesn't contain the actual class name, we need to be able to tell the herder whether a connector is a Sink */ public boolean isSinkConnector(String connName) { WorkerConnector workerConnector = connectors.get(connName); - return SinkConnector.class.isAssignableFrom(workerConnector.delegate.getClass()); + return workerConnector.isSinkConnector(); } public Connector getConnector(String connType) { @@ -263,7 +261,7 @@ public List> connectorTaskConfigs(String connName, int maxTa if (workerConnector == null) throw new ConnectException("Connector " + connName + " not found in this worker."); - Connector connector = workerConnector.delegate; + Connector connector = workerConnector.connector(); List> result = new ArrayList<>(); String taskClassName = connector.taskClass().getName(); for (Map taskProps : connector.taskConfigs(maxTasks)) { @@ -283,7 +281,7 @@ public void stopConnector(String connName) { if (connector == null) throw new ConnectException("Connector " + connName + " not found in this worker."); - connector.stop(); + connector.shutdown(); connectors.remove(connName); log.info("Stopped connector {}", connName); @@ -296,13 +294,24 @@ public Set connectorNames() { return connectors.keySet(); } + public boolean isRunning(String connName) { + WorkerConnector connector = connectors.get(connName); + if (connector == null) + throw new ConnectException("Connector " + connName + " not found in this worker."); + return connector.isRunning(); + } + /** * Add a new task. * @param id Globally unique ID for this task. * @param taskConfig the parsed task configuration * @param statusListener listener for notifications of task status changes + * @param initialState the initial target state that the task should be initialized to */ - public void startTask(ConnectorTaskId id, TaskConfig taskConfig, TaskStatus.Listener statusListener) { + public void startTask(ConnectorTaskId id, + TaskConfig taskConfig, + TaskStatus.Listener statusListener, + TargetState initialState) { log.info("Creating task {}", id); if (tasks.containsKey(id)) { @@ -316,11 +325,11 @@ public void startTask(ConnectorTaskId id, TaskConfig taskConfig, TaskStatus.List final Task task = instantiateTask(taskClass); log.info("Instantiated task {} with version {} of type {}", id, task.version(), taskClass.getName()); - final WorkerTask workerTask = buildWorkerTask(id, task, statusListener); + final WorkerTask workerTask = buildWorkerTask(id, task, statusListener, initialState); // Start the task before adding modifying any state, any exceptions are caught higher up the // call chain and there's no cleanup to do here - workerTask.initialize(taskConfig.originalsStrings()); + workerTask.initialize(taskConfig); executor.submit(workerTask); if (task instanceof SourceTask) { @@ -330,17 +339,21 @@ public void startTask(ConnectorTaskId id, TaskConfig taskConfig, TaskStatus.List tasks.put(id, workerTask); } - private WorkerTask buildWorkerTask(ConnectorTaskId id, Task task, TaskStatus.Listener lifecycleListener) { + private WorkerTask buildWorkerTask(ConnectorTaskId id, + Task task, + TaskStatus.Listener statusListener, + TargetState initialState) { // Decide which type of worker task we need based on the type of task. if (task instanceof SourceTask) { OffsetStorageReader offsetReader = new OffsetStorageReaderImpl(offsetBackingStore, id.connector(), internalKeyConverter, internalValueConverter); OffsetStorageWriter offsetWriter = new OffsetStorageWriter(offsetBackingStore, id.connector(), internalKeyConverter, internalValueConverter); - return new WorkerSourceTask(id, (SourceTask) task, lifecycleListener, keyConverter, valueConverter, producer, - offsetReader, offsetWriter, config, time); + return new WorkerSourceTask(id, (SourceTask) task, statusListener, initialState, keyConverter, + valueConverter, producer, offsetReader, offsetWriter, config, time); } else if (task instanceof SinkTask) { - return new WorkerSinkTask(id, (SinkTask) task, lifecycleListener, config, keyConverter, valueConverter, time); + return new WorkerSinkTask(id, (SinkTask) task, statusListener, initialState, config, keyConverter, + valueConverter, time); } else { log.error("Tasks must be a subclass of either SourceTask or SinkTask", task); throw new ConnectException("Tasks must be a subclass of either SourceTask or SinkTask"); @@ -422,51 +435,17 @@ public boolean ownsConnector(String connName) { return this.connectors.containsKey(connName); } - private static class WorkerConnector { - private final String connName; - private final ConnectorStatus.Listener lifecycleListener; - private final ConnectorContext ctx; - private final Connector delegate; + public void setTargetState(String connName, TargetState state) { + log.info("Setting connector {} state to {}", connName, state); - public WorkerConnector(String connName, - Connector delegate, - ConnectorContext ctx, - ConnectorStatus.Listener lifecycleListener) { - this.connName = connName; - this.ctx = ctx; - this.delegate = delegate; - this.lifecycleListener = lifecycleListener; - } - - public void initialize() { - delegate.initialize(ctx); - } - - public void start(Map props) { - try { - delegate.start(props); - lifecycleListener.onStartup(connName); - } catch (Throwable t) { - log.error("Error while starting connector {}", connName, t); - lifecycleListener.onFailure(connName, t); - } - } - - public void stop() { - try { - delegate.stop(); - lifecycleListener.onShutdown(connName); - } catch (Throwable t) { - log.error("Error while shutting down connector {}", connName, t); - lifecycleListener.onFailure(connName, t); - } - } + WorkerConnector connector = connectors.get(connName); + if (connector != null) + connector.transitionTo(state); - @Override - public String toString() { - return delegate.toString(); + for (Map.Entry taskEntry : tasks.entrySet()) { + if (taskEntry.getKey().connector().equals(connName)) + taskEntry.getValue().transitionTo(state); } - } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java new file mode 100644 index 0000000000000..7880095727919 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConnector.java @@ -0,0 +1,206 @@ +/** + * 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 org.apache.kafka.connect.runtime; + +import org.apache.kafka.connect.connector.Connector; +import org.apache.kafka.connect.connector.ConnectorContext; +import org.apache.kafka.connect.sink.SinkConnector; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; + +/** + * Container for connectors which is responsible for managing their lifecycle (e.g. handling startup, + * shutdown, pausing, etc.). Internally, we manage the runtime state of the connector and transition according + * to target state changes. Note that unlike connector tasks, the connector does not really have a "pause" + * state which is distinct from being stopped. We therefore treat pause operations as requests to momentarily + * stop the connector, and resume operations as requests to restart it (without reinitialization). Connector + * failures, whether in initialization or after startup, are treated as fatal, which means that we will not attempt + * to restart this connector instance after failure. What this means from a user perspective is that you must + * use the /restart REST API to restart a failed task. This behavior is consistent with task failures. + * + * Note that this class is NOT thread-safe. + */ +public class WorkerConnector { + private static final Logger log = LoggerFactory.getLogger(WorkerConnector.class); + + private enum State { + INIT, // initial state before startup + STOPPED, // the connector has been stopped/paused. + STARTED, // the connector has been started/resumed. + FAILED, // the connector has failed (no further transitions are possible after this state) + } + + private final String connName; + private final ConnectorStatus.Listener statusListener; + private final ConnectorContext ctx; + private final Connector connector; + + private Map config; + private State state; + + public WorkerConnector(String connName, + Connector connector, + ConnectorContext ctx, + ConnectorStatus.Listener statusListener) { + this.connName = connName; + this.ctx = ctx; + this.connector = connector; + this.statusListener = statusListener; + this.state = State.INIT; + } + + public void initialize(ConnectorConfig connectorConfig) { + log.debug("Initializing connector {} with config {}", connName, config); + + try { + this.config = connectorConfig.originalsStrings(); + + connector.initialize(new ConnectorContext() { + @Override + public void requestTaskReconfiguration() { + ctx.requestTaskReconfiguration(); + } + + @Override + public void raiseError(Exception e) { + log.error("Connector raised an error {}", connName, e); + onFailure(e); + ctx.raiseError(e); + } + }); + } catch (Throwable t) { + log.error("Error initializing connector {}", connName, t); + onFailure(t); + } + } + + private boolean doStart() { + try { + switch (state) { + case STARTED: + return false; + + case INIT: + case STOPPED: + connector.start(config); + this.state = State.STARTED; + return true; + + default: + throw new IllegalArgumentException("Cannot start connector in state " + state); + } + } catch (Throwable t) { + log.error("Error while starting connector {}", connName, t); + onFailure(t); + return false; + } + } + + private void onFailure(Throwable t) { + statusListener.onFailure(connName, t); + this.state = State.FAILED; + } + + private void resume() { + if (doStart()) + statusListener.onResume(connName); + } + + private void start() { + if (doStart()) + statusListener.onStartup(connName); + } + + public boolean isRunning() { + return state == State.STARTED; + } + + private void pause() { + try { + switch (state) { + case STOPPED: + return; + + case STARTED: + connector.stop(); + // fall through + + case INIT: + statusListener.onPause(connName); + this.state = State.STOPPED; + break; + + default: + throw new IllegalArgumentException("Cannot pause connector in state " + state); + } + } catch (Throwable t) { + log.error("Error while shutting down connector {}", connName, t); + statusListener.onFailure(connName, t); + this.state = State.FAILED; + } + } + + public void shutdown() { + try { + if (state == State.STARTED) + connector.stop(); + this.state = State.STOPPED; + } catch (Throwable t) { + log.error("Error while shutting down connector {}", connName, t); + this.state = State.FAILED; + } finally { + statusListener.onShutdown(connName); + } + } + + public void transitionTo(TargetState targetState) { + if (state == State.FAILED) { + log.warn("Cannot transition connector {} to {} since it has failed", connName, targetState); + return; + } + + log.debug("Transition connector {} to {}", connName, targetState); + if (targetState == TargetState.PAUSED) { + pause(); + } else if (targetState == TargetState.STARTED) { + if (state == State.INIT) + start(); + else + resume(); + } else { + throw new IllegalArgumentException("Unhandled target state " + targetState); + } + } + + public boolean isSinkConnector() { + return SinkConnector.class.isAssignableFrom(connector.getClass()); + } + + public Connector connector() { + return connector; + } + + @Override + public String toString() { + return "WorkerConnector{" + + "connName='" + connName + '\'' + + ", connector=" + connector + + '}'; + } +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java index 62934553959af..f5eaac4aa328f 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSinkTask.java @@ -43,11 +43,8 @@ import java.util.Arrays; import java.util.Collection; import java.util.HashMap; -import java.util.HashSet; -import java.util.Iterator; import java.util.List; import java.util.Map; -import java.util.Set; import static java.util.Collections.singleton; @@ -78,12 +75,13 @@ class WorkerSinkTask extends WorkerTask { public WorkerSinkTask(ConnectorTaskId id, SinkTask task, - TaskStatus.Listener lifecycleListener, + TaskStatus.Listener statusListener, + TargetState initialState, WorkerConfig workerConfig, Converter keyConverter, Converter valueConverter, Time time) { - super(id, lifecycleListener); + super(id, statusListener, initialState); this.workerConfig = workerConfig; this.task = task; @@ -103,10 +101,15 @@ public WorkerSinkTask(ConnectorTaskId id, } @Override - public void initialize(Map taskConfig) { - this.taskConfig = taskConfig; - this.consumer = createConsumer(); - this.context = new WorkerSinkTaskContext(consumer); + public void initialize(TaskConfig taskConfig) { + try { + this.taskConfig = taskConfig.originalsStrings(); + this.consumer = createConsumer(); + this.context = new WorkerSinkTaskContext(consumer); + } catch (Throwable t) { + log.error("Task {} failed initialization and will not be started.", t); + onFailure(t); + } } @Override @@ -125,6 +128,12 @@ protected void close() { consumer.close(); } + @Override + public void transitionTo(TargetState state) { + super.transitionTo(state); + consumer.wakeup(); + } + @Override public void execute() { initializeAndStart(); @@ -218,6 +227,12 @@ protected void poll(long timeoutMs) { deliverMessages(); } catch (WakeupException we) { log.trace("{} consumer woken up", id); + + if (shouldPause()) { + pauseAll(); + } else if (!pausedForRedelivery) { + resumeAll(); + } } } @@ -338,6 +353,16 @@ private void convertMessages(ConsumerRecords msgs) { } } + private void resumeAll() { + for (TopicPartition tp : consumer.assignment()) + if (!context.pausedPartitions().contains(tp)) + consumer.resume(singleton(tp)); + } + + private void pauseAll() { + consumer.pause(consumer.assignment()); + } + private void deliverMessages() { // Finally, deliver this batch to the sink try { @@ -350,9 +375,8 @@ private void deliverMessages() { // If we had paused all consumer topic partitions to try to redeliver data, then we should resume any that // the task had not explicitly paused if (pausedForRedelivery) { - for (TopicPartition tp : consumer.assignment()) - if (!context.pausedPartitions().contains(tp)) - consumer.resume(singleton(tp)); + if (!shouldPause()) + resumeAll(); pausedForRedelivery = false; } } catch (RetriableException e) { @@ -360,7 +384,7 @@ private void deliverMessages() { // If we're retrying a previous batch, make sure we've paused all topic partitions so we don't get new data, // but will still be able to poll in order to handle user-requested timeouts, keep group membership, etc. pausedForRedelivery = true; - consumer.pause(consumer.assignment()); + pauseAll(); // Let this exit normally, the batch will be reprocessed on the next loop. } catch (Throwable t) { log.error("Task {} threw an uncaught and unrecoverable exception", id, t); @@ -412,24 +436,14 @@ public void onPartitionsAssigned(Collection partitions) { // If we paused everything for redelivery (which is no longer relevant since we discarded the data), make // sure anything we paused that the task didn't request to be paused *and* which we still own is resumed. // Also make sure our tracking of paused partitions is updated to remove any partitions we no longer own. - if (pausedForRedelivery) { - pausedForRedelivery = false; - - Set assigned = new HashSet<>(partitions); - Set taskPaused = context.pausedPartitions(); - - for (TopicPartition tp : partitions) { - if (!taskPaused.contains(tp)) - consumer.resume(singleton(tp)); - } - - Iterator tpIter = taskPaused.iterator(); - while (tpIter.hasNext()) { - TopicPartition tp = tpIter.next(); - if (assigned.contains(tp)) - tpIter.remove(); - } - } + pausedForRedelivery = false; + + // Ensure that the paused partitions contains only assigned partitions and repause as necessary + context.pausedPartitions().retainAll(partitions); + if (shouldPause()) + pauseAll(); + else if (!context.pausedPartitions().isEmpty()) + consumer.pause(context.pausedPartitions()); // Instead of invoking the assignment callback on initialization, we guarantee the consumer is ready upon // task start. Since this callback gets invoked during that initial setup before we've started the task, we diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java index 3a43f968a102a..602af4a533f72 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerSourceTask.java @@ -76,7 +76,8 @@ class WorkerSourceTask extends WorkerTask { public WorkerSourceTask(ConnectorTaskId id, SourceTask task, - TaskStatus.Listener lifecycleListener, + TaskStatus.Listener statusListener, + TargetState initialState, Converter keyConverter, Converter valueConverter, KafkaProducer producer, @@ -84,7 +85,7 @@ public WorkerSourceTask(ConnectorTaskId id, OffsetStorageWriter offsetWriter, WorkerConfig workerConfig, Time time) { - super(id, lifecycleListener); + super(id, statusListener, initialState); this.workerConfig = workerConfig; this.task = task; @@ -104,8 +105,13 @@ public WorkerSourceTask(ConnectorTaskId id, } @Override - public void initialize(Map config) { - this.taskConfig = config; + public void initialize(TaskConfig taskConfig) { + try { + this.taskConfig = taskConfig.originalsStrings(); + } catch (Throwable t) { + log.error("Task {} failed initialization and will not be started.", t); + onFailure(t); + } } protected void close() { @@ -139,6 +145,11 @@ public void execute() { } while (!isStopping()) { + if (shouldPause()) { + awaitUnpause(); + continue; + } + if (toSend == null) { log.debug("Nothing to send to Kafka. Polling source for additional records"); toSend = task.poll(); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java index 7979fb008971a..846ca9572cded 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerTask.java @@ -21,33 +21,41 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import java.util.Map; import java.util.concurrent.CountDownLatch; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicReference; /** * Handles processing for an individual task. This interface only provides the basic methods * used by {@link Worker} to manage the tasks. Implementations combine a user-specified Task with * Kafka to create a data flow. + * + * Note on locking: since the task runs in its own thread, special care must be taken to ensure + * that state transitions are reported correctly, in particular since some state transitions are + * asynchronous (e.g. pause/resume). For example, changing the state to paused could cause a race + * if the task fails at the same time. To protect from these cases, we synchronize status updates + * using the WorkerTask's monitor. */ abstract class WorkerTask implements Runnable { private static final Logger log = LoggerFactory.getLogger(WorkerTask.class); protected final ConnectorTaskId id; - private final AtomicBoolean stopping; - private final AtomicBoolean running; - private final AtomicBoolean cancelled; + private final AtomicBoolean stopping; // indicates whether the Worker has asked the task to stop + private final AtomicBoolean cancelled; // indicates whether the Worker has cancelled the task (e.g. because of slow shutdown) private final CountDownLatch shutdownLatch; - private final TaskStatus.Listener lifecycleListener; + private final TaskStatus.Listener statusListener; + private final AtomicReference targetState; - public WorkerTask(ConnectorTaskId id, TaskStatus.Listener lifecycleListener) { + public WorkerTask(ConnectorTaskId id, + TaskStatus.Listener statusListener, + TargetState initialState) { this.id = id; this.stopping = new AtomicBoolean(false); - this.running = new AtomicBoolean(false); this.cancelled = new AtomicBoolean(false); this.shutdownLatch = new CountDownLatch(1); - this.lifecycleListener = lifecycleListener; + this.statusListener = statusListener; + this.targetState = new AtomicReference<>(initialState); } public ConnectorTaskId id() { @@ -58,14 +66,24 @@ public ConnectorTaskId id() { * Initialize the task for execution. * @param props initial configuration */ - public abstract void initialize(Map props); + public abstract void initialize(TaskConfig taskConfig); + + + private void triggerStop() { + synchronized (this) { + this.stopping.set(true); + + // wakeup any threads that are waiting for unpause + this.notifyAll(); + } + } /** * Stop this task from processing messages. This method does not block, it only triggers * shutdown. Use #{@link #awaitStop} to block until completion. */ public void stop() { - this.stopping.set(true); + triggerStop(); } /** @@ -83,9 +101,6 @@ public void cancel() { * @return true if successful, false if the timeout was reached */ public boolean awaitStop(long timeoutMs) { - if (!running.get()) - return true; - try { return shutdownLatch.await(timeoutMs, TimeUnit.MILLISECONDS); } catch (InterruptedException e) { @@ -101,10 +116,6 @@ protected boolean isStopping() { return stopping.get(); } - protected boolean isStopped() { - return !running.get(); - } - private void doClose() { try { close(); @@ -115,14 +126,17 @@ private void doClose() { } private void doRun() { - if (!this.running.compareAndSet(false, true)) - throw new IllegalStateException("The task cannot be started while still running"); - try { - if (stopping.get()) - return; + synchronized (this) { + if (stopping.get()) + return; + + if (targetState.get() == TargetState.PAUSED) + statusListener.onPause(id); + else + statusListener.onStartup(id); + } - lifecycleListener.onStartup(id); execute(); } catch (Throwable t) { log.error("Task {} threw an uncaught and unrecoverable exception", id, t); @@ -133,22 +147,80 @@ private void doRun() { } } + private void onShutdown() { + synchronized (this) { + triggerStop(); + + // if we were cancelled, skip the status update since the task may have already been + // started somewhere else + if (!cancelled.get()) + statusListener.onShutdown(id); + } + } + + protected void onFailure(Throwable t) { + synchronized (this) { + triggerStop(); + + // if we were cancelled, skip the status update since the task may have already been + // started somewhere else + if (!cancelled.get()) + statusListener.onFailure(id, t); + } + } + @Override public void run() { try { doRun(); - if (!cancelled.get()) - lifecycleListener.onShutdown(id); + onShutdown(); } catch (Throwable t) { - if (!cancelled.get()) - lifecycleListener.onFailure(id, t); + onFailure(t); if (t instanceof Error) throw t; } finally { - running.set(false); shutdownLatch.countDown(); } } + public boolean shouldPause() { + return this.targetState.get() == TargetState.PAUSED; + } + + /** + * Await task resumption. + * @return true if the task's target state is not paused, false if the task is shutdown before resumption + * @throws InterruptedException + */ + protected boolean awaitUnpause() throws InterruptedException { + synchronized (this) { + while (targetState.get() == TargetState.PAUSED) { + if (stopping.get()) + return false; + this.wait(); + } + return true; + } + } + + public void transitionTo(TargetState state) { + synchronized (this) { + // ignore the state change if we are stopping + if (stopping.get()) + return; + + TargetState oldState = this.targetState.getAndSet(state); + if (state != oldState) { + if (state == TargetState.PAUSED) { + statusListener.onPause(id); + } else if (state == TargetState.STARTED) { + statusListener.onResume(id); + this.notifyAll(); + } else + throw new IllegalArgumentException("Unhandled target state " + state); + } + } + } + } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ClusterConfigState.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ClusterConfigState.java index cc4a3c10f4be0..c5c217e6f4731 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ClusterConfigState.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ClusterConfigState.java @@ -17,10 +17,12 @@ package org.apache.kafka.connect.runtime.distributed; +import org.apache.kafka.connect.runtime.TargetState; import org.apache.kafka.connect.util.ConnectorTaskId; import java.util.ArrayList; import java.util.Collections; +import java.util.HashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -29,24 +31,32 @@ * An immutable snapshot of the configuration state of connectors and tasks in a Kafka Connect cluster. */ public class ClusterConfigState { - public static final ClusterConfigState EMPTY = new ClusterConfigState(-1, Collections.emptyMap(), - Collections.>emptyMap(), Collections.>emptyMap(), + public static final long NO_OFFSET = -1; + public static final ClusterConfigState EMPTY = new ClusterConfigState( + NO_OFFSET, + Collections.emptyMap(), + Collections.>emptyMap(), + Collections.emptyMap(), + Collections.>emptyMap(), Collections.emptySet()); private final long offset; private final Map connectorTaskCounts; private final Map> connectorConfigs; + private final Map connectorTargetStates; private final Map> taskConfigs; private final Set inconsistentConnectors; public ClusterConfigState(long offset, Map connectorTaskCounts, Map> connectorConfigs, + Map connectorTargetStates, Map> taskConfigs, Set inconsistentConnectors) { this.offset = offset; this.connectorTaskCounts = connectorTaskCounts; this.connectorConfigs = connectorConfigs; + this.connectorTargetStates = connectorTargetStates; this.taskConfigs = taskConfigs; this.inconsistentConnectors = inconsistentConnectors; } @@ -60,6 +70,15 @@ public long offset() { return offset; } + /** + * Check whether this snapshot contains configuration for a connector. + * @param connector name of the connector + * @return true if this state contains configuration for the connector, false otherwise + */ + public boolean contains(String connector) { + return connectorConfigs.containsKey(connector); + } + /** * Get a list of the connectors in this configuration */ @@ -76,6 +95,15 @@ public Map connectorConfig(String connector) { return connectorConfigs.get(connector); } + /** + * Get the target state of the connector + * @param connector name of the connector + * @return the target state + */ + public TargetState targetState(String connector) { + return connectorTargetStates.get(connector); + } + /** * Get the configuration for a task. * @param task id of the task @@ -85,6 +113,20 @@ public Map taskConfig(ConnectorTaskId task) { return taskConfigs.get(task); } + /** + * Get all task configs for a connector. + * @param connector name of the connector + * @return a map from the task id to its configuration + */ + public Map> allTaskConfigs(String connector) { + Map> taskConfigs = new HashMap<>(); + for (Map.Entry> taskConfigEntry : this.taskConfigs.entrySet()) { + if (taskConfigEntry.getKey().connector().equals(connector)) + taskConfigs.put(taskConfigEntry.getKey(), taskConfigEntry.getValue()); + } + return taskConfigs; + } + /** * Get the number of tasks assigned for the given connector. * @param connectorName name of the connector to look up tasks for diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java index 24d548d112a34..15126034fbd76 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java @@ -28,12 +28,13 @@ import org.apache.kafka.connect.runtime.AbstractHerder; import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.runtime.HerderConnectorContext; +import org.apache.kafka.connect.runtime.TargetState; import org.apache.kafka.connect.runtime.TaskConfig; import org.apache.kafka.connect.runtime.Worker; import org.apache.kafka.connect.runtime.rest.RestServer; import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; import org.apache.kafka.connect.runtime.rest.entities.TaskInfo; -import org.apache.kafka.connect.storage.KafkaConfigStorage; +import org.apache.kafka.connect.storage.ConfigBackingStore; import org.apache.kafka.connect.storage.StatusBackingStore; import org.apache.kafka.connect.util.Callback; import org.apache.kafka.connect.util.ConnectorTaskId; @@ -52,7 +53,6 @@ import java.util.Set; import java.util.concurrent.Callable; import java.util.concurrent.CountDownLatch; -import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; @@ -78,19 +78,27 @@ * by a single node at a time. Most importantly, this includes writing updated configurations for connectors and tasks, * (and therefore, also for creating, destroy, and scaling up/down connectors). *

          + *

          + * The DistributedHerder uses a single thread for most of its processing. This includes processing + * config changes, handling task rebalances and serving requests from the HTTP layer. The latter are pushed + * into a queue until the thread has time to handle them. A consequence of this is that requests can get blocked + * behind a worker rebalance. When the herder knows that a rebalance is expected, it typically returns an error + * immediately to the request, but this is not always possible (in particular when another worker has requested + * the rebalance). Similar to handling HTTP requests, config changes which are observed asynchronously by polling + * the config log are batched for handling in the work thread. + *

          */ public class DistributedHerder extends AbstractHerder implements Runnable { private static final Logger log = LoggerFactory.getLogger(DistributedHerder.class); private static final long RECONFIGURE_CONNECTOR_TASKS_BACKOFF_MS = 250; - private final KafkaConfigStorage configStorage; - private ClusterConfigState configState; private final Time time; private final int workerSyncTimeoutMs; private final int workerUnsyncBackoffMs; + private final ExecutorService forwardRequestExecutor; private final WorkerGroupMember member; private final AtomicBoolean stopping; private final CountDownLatch stopLatch = new CountDownLatch(1); @@ -100,6 +108,7 @@ public class DistributedHerder extends AbstractHerder implements Runnable { private boolean rebalanceResolved; private ConnectProtocol.Assignment assignment; private boolean canReadConfigs; + private ClusterConfigState configState; // To handle most external requests, like creating or destroying a connector, we can use a generic request where // the caller specifies all the code that should be executed. @@ -107,51 +116,44 @@ public class DistributedHerder extends AbstractHerder implements Runnable { // Config updates can be collected and applied together when possible. Also, we need to take care to rebalance when // needed (e.g. task reconfiguration, which requires everyone to coordinate offset commits). private Set connectorConfigUpdates = new HashSet<>(); + // Similarly collect target state changes (when observed by the config storage listener) for handling in the + // herder's main thread. + private Set connectorTargetStateChanges = new HashSet<>(); private boolean needsReconfigRebalance; private volatile int generation; - private final ExecutorService forwardRequestExecutor; - public DistributedHerder(DistributedConfig config, Time time, Worker worker, StatusBackingStore statusBackingStore, + ConfigBackingStore configBackingStore, String restUrl) { - this(config, worker.workerId(), worker, statusBackingStore, null, null, restUrl, time); + this(config, worker, worker.workerId(), statusBackingStore, configBackingStore, null, restUrl, time); + configBackingStore.setUpdateListener(new ConfigUpdateListener()); } // visible for testing DistributedHerder(DistributedConfig config, - String workerId, Worker worker, + String workerId, StatusBackingStore statusBackingStore, - KafkaConfigStorage configStorage, + ConfigBackingStore configStorage, WorkerGroupMember member, String restUrl, Time time) { - super(worker, statusBackingStore, workerId); + super(worker, workerId, statusBackingStore, configStorage); - if (configStorage != null) { - // For testing. Assume configuration has already been performed - this.configStorage = configStorage; - } else { - this.configStorage = new KafkaConfigStorage(worker.getInternalValueConverter(), connectorConfigCallback(), taskConfigCallback()); - this.configStorage.configure(config); - } - configState = ClusterConfigState.EMPTY; this.time = time; - this.workerSyncTimeoutMs = config.getInt(DistributedConfig.WORKER_SYNC_TIMEOUT_MS_CONFIG); this.workerUnsyncBackoffMs = config.getInt(DistributedConfig.WORKER_UNSYNC_BACKOFF_MS_CONFIG); + this.member = member != null ? member : new WorkerGroupMember(config, restUrl, this.configBackingStore, new RebalanceListener(), time); + this.forwardRequestExecutor = Executors.newSingleThreadExecutor(); - this.member = member != null ? member : new WorkerGroupMember(config, restUrl, this.configStorage, rebalanceListener(), time); stopping = new AtomicBoolean(false); - + configState = ClusterConfigState.EMPTY; rebalanceResolved = true; // If we still need to follow up after a rebalance occurred, starting up tasks needsReconfigRebalance = false; canReadConfigs = true; // We didn't try yet, but Configs are readable until proven otherwise - - forwardRequestExecutor = Executors.newSingleThreadExecutor(); } @Override @@ -160,19 +162,6 @@ public void start() { thread.start(); } - @Override - protected void startServices() { - super.startServices(); - configStorage.start(); - } - - @Override - protected void stopServices() { - super.stopServices(); - if (configStorage != null) - configStorage.stop(); - } - @Override public void run() { try { @@ -248,50 +237,48 @@ public void tick() { // Process any configuration updates Set connectorConfigUpdatesCopy = null; + Set connectorTargetStateChangesCopy = null; synchronized (this) { - if (needsReconfigRebalance || !connectorConfigUpdates.isEmpty()) { + if (needsReconfigRebalance || !connectorConfigUpdates.isEmpty() || !connectorTargetStateChanges.isEmpty()) { // Connector reconfigs only need local updates since there is no coordination between workers required. // However, if connectors were added or removed, work needs to be rebalanced since we have more work // items to distribute among workers. - ClusterConfigState newConfigState = configStorage.snapshot(); - if (!newConfigState.connectors().equals(configState.connectors())) - needsReconfigRebalance = true; - configState = newConfigState; + configState = configBackingStore.snapshot(); + if (needsReconfigRebalance) { // Task reconfigs require a rebalance. Request the rebalance, clean out state, and then restart // this loop, which will then ensure the rebalance occurs without any other requests being // processed until it completes. member.requestRejoin(); - // Any connector config updates will be addressed during the rebalance too + // Any connector config updates or target state changes will be addressed during the rebalance too connectorConfigUpdates.clear(); + connectorTargetStateChanges.clear(); needsReconfigRebalance = false; return; - } else if (!connectorConfigUpdates.isEmpty()) { - // We can't start/stop while locked since starting connectors can cause task updates that will - // require writing configs, which in turn make callbacks into this class from another thread that - // require acquiring a lock. This leads to deadlock. Instead, just copy the info we need and process - // the updates after unlocking. - connectorConfigUpdatesCopy = connectorConfigUpdates; - connectorConfigUpdates = new HashSet<>(); + } else { + if (!connectorConfigUpdates.isEmpty()) { + // We can't start/stop while locked since starting connectors can cause task updates that will + // require writing configs, which in turn make callbacks into this class from another thread that + // require acquiring a lock. This leads to deadlock. Instead, just copy the info we need and process + // the updates after unlocking. + connectorConfigUpdatesCopy = connectorConfigUpdates; + connectorConfigUpdates = new HashSet<>(); + } + + if (!connectorTargetStateChanges.isEmpty()) { + // Similarly for target state changes which can cause connectors to be restarted + connectorTargetStateChangesCopy = connectorTargetStateChanges; + connectorTargetStateChanges = new HashSet<>(); + } } } } - if (connectorConfigUpdatesCopy != null) { - // If we only have connector config updates, we can just bounce the updated connectors that are - // currently assigned to this worker. - Set localConnectors = assignment == null ? Collections.emptySet() : new HashSet<>(assignment.connectors()); - for (String connectorName : connectorConfigUpdatesCopy) { - if (!localConnectors.contains(connectorName)) - continue; - boolean remains = configState.connectors().contains(connectorName); - log.info("Handling connector-only config update by {} connector {}", - remains ? "restarting" : "stopping", connectorName); - worker.stopConnector(connectorName); - // The update may be a deletion, so verify we actually need to restart the connector - if (remains) - startConnector(connectorName); - } - } + + if (connectorConfigUpdatesCopy != null) + processConnectorConfigUpdates(connectorConfigUpdatesCopy); + + if (connectorTargetStateChangesCopy != null) + processTargetStateChanges(connectorTargetStateChangesCopy); // Let the group take any actions it needs to try { @@ -303,6 +290,36 @@ public void tick() { } } + private void processConnectorConfigUpdates(Set connectorConfigUpdates) { + // If we only have connector config updates, we can just bounce the updated connectors that are + // currently assigned to this worker. + Set localConnectors = assignment == null ? Collections.emptySet() : new HashSet<>(assignment.connectors()); + for (String connectorName : connectorConfigUpdates) { + if (!localConnectors.contains(connectorName)) + continue; + boolean remains = configState.contains(connectorName); + log.info("Handling connector-only config update by {} connector {}", + remains ? "restarting" : "stopping", connectorName); + worker.stopConnector(connectorName); + // The update may be a deletion, so verify we actually need to restart the connector + if (remains) + startConnector(connectorName); + } + } + + private void processTargetStateChanges(Set connectorTargetStateChanges) { + if (!connectorTargetStateChanges.isEmpty()) { + for (String connector : connectorTargetStateChanges) { + if (worker.connectorNames().contains(connector)) { + TargetState targetState = configState.targetState(connector); + worker.setTargetState(connector, targetState); + if (targetState == TargetState.STARTED) + reconfigureConnectorTasksWithRetry(connector); + } + } + } + } + // public for testing public void halt() { synchronized (this) { @@ -347,7 +364,6 @@ public void stop() { } } - forwardRequestExecutor.shutdown(); try { if (!forwardRequestExecutor.awaitTermination(10000, TimeUnit.MILLISECONDS)) @@ -367,7 +383,7 @@ public synchronized void connectors(final Callback> callback) new Callable() { @Override public Void call() throws Exception { - if (!checkConfigSynced(callback)) + if (checkRebalanceNeeded(callback)) return null; callback.onCompletion(null, configState.connectors()); @@ -386,10 +402,10 @@ public synchronized void connectorInfo(final String connName, final Callback() { @Override public Void call() throws Exception { - if (!checkConfigSynced(callback)) + if (checkRebalanceNeeded(callback)) return null; - if (!configState.connectors().contains(connName)) { + if (!configState.contains(connName)) { callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null); } else { callback.onCompletion(null, new ConnectorInfo(connName, configState.connectorConfig(connName), configState.tasks(connName))); @@ -441,27 +457,30 @@ public Void call() throws Exception { return null; } - boolean exists = configState.connectors().contains(connName); + boolean exists = configState.contains(connName); if (!allowReplace && exists) { callback.onCompletion(new AlreadyExistsException("Connector " + connName + " already exists"), null); return null; } - if (connConfig == null && !exists) { - callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null); + if (connConfig == null) { + if (!exists) { + callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null); + } else { + log.trace("Removing connector config {} {} {}", connName, allowReplace, configState.connectors()); + configBackingStore.removeConnectorConfig(connName); + callback.onCompletion(null, new Created(false, null)); + } return null; } log.trace("Submitting connector config {} {} {}", connName, allowReplace, configState.connectors()); - configStorage.putConnectorConfig(connName, connConfig); + configBackingStore.putConnectorConfig(connName, connConfig); - boolean created = !exists && connConfig != null; // Note that we use the updated connector config despite the fact that we don't have an updated // snapshot yet. The existing task info should still be accurate. - ConnectorInfo info = connConfig == null ? null : - new ConnectorInfo(connName, connConfig, configState.tasks(connName)); - callback.onCompletion(null, new Created<>(created, info)); - + ConnectorInfo info = new ConnectorInfo(connName, connConfig, configState.tasks(connName)); + callback.onCompletion(null, new Created<>(!exists, info)); return null; } }, @@ -501,10 +520,10 @@ public synchronized void taskConfigs(final String connName, final Callback() { @Override public Void call() throws Exception { - if (!checkConfigSynced(callback)) + if (checkRebalanceNeeded(callback)) return null; - if (!configState.connectors().contains(connName)) { + if (!configState.contains(connName)) { callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null); } else { List result = new ArrayList<>(); @@ -531,10 +550,10 @@ public synchronized void putTaskConfigs(final String connName, final List() { @Override public Void call() throws Exception { - // if config is out of sync, then a rebalance is likely to begin shortly, so rather than risking - // a stale response, we return an error and let the user retry - if (!isConfigSynced()) { - throw new StaleConfigException("Cannot complete request because config is out of sync"); - } + if (checkRebalanceNeeded(callback)) + return null; if (!configState.connectors().contains(connName)) { callback.onCompletion(new NotFoundException("Unknown connector: " + connName), null); @@ -583,11 +599,8 @@ public synchronized void restartTask(final ConnectorTaskId id, final Callback() { @Override public Void call() throws Exception { - // if config is out of sync, then a rebalance is likely to begin shortly, so rather than risking - // a stale response, we return an error and let the user retry - if (!isConfigSynced()) { - throw new StaleConfigException("Cannot complete request because config is out of sync"); - } + if (checkRebalanceNeeded(callback)) + return null; if (!configState.connectors().contains(id.connector())) { callback.onCompletion(new NotFoundException("Unknown connector: " + id.connector()), null); @@ -719,8 +732,8 @@ private boolean handleRebalanceCompleted() { private boolean readConfigToEnd(long timeoutMs) { log.info("Current config state offset {} is behind group assignment {}, reading to end of config log", configState.offset(), assignment.offset()); try { - configStorage.readToEnd().get(timeoutMs, TimeUnit.MILLISECONDS); - configState = configStorage.snapshot(); + configBackingStore.refresh(timeoutMs, TimeUnit.MILLISECONDS); + configState = configBackingStore.snapshot(); log.info("Finished reading to end of log and updated config snapshot, new config log offset: {}", configState.offset()); return true; } catch (TimeoutException e) { @@ -730,8 +743,6 @@ private boolean readConfigToEnd(long timeoutMs) { member.maybeLeaveGroup(); backoff(workerUnsyncBackoffMs); return false; - } catch (InterruptedException | ExecutionException e) { - throw new ConnectException("Error trying to catch up after assignment", e); } } @@ -763,9 +774,10 @@ private void startWork() { private void startTask(ConnectorTaskId taskId) { log.info("Starting task {}", taskId); + TargetState initialState = configState.targetState(taskId.connector()); Map configs = configState.taskConfig(taskId); TaskConfig taskConfig = new TaskConfig(configs); - worker.startTask(taskId, taskConfig, this); + worker.startTask(taskId, taskConfig, this, initialState); } // Helper for starting a connector with the given name, which will extract & parse the config, generate connector @@ -776,11 +788,14 @@ private void startConnector(String connectorName) { ConnectorConfig connConfig = new ConnectorConfig(configs); String connName = connConfig.getString(ConnectorConfig.NAME_CONFIG); ConnectorContext ctx = new HerderConnectorContext(DistributedHerder.this, connName); - worker.startConnector(connConfig, ctx, this); + TargetState initialState = configState.targetState(connectorName); + worker.startConnector(connConfig, ctx, this, initialState); + // Immediately request configuration since this could be a brand new connector. However, also only update those // task configs if they are actually different from the existing ones to avoid unnecessary updates when this is // just restoring an existing connector. - reconfigureConnectorTasksWithRetry(connName); + if (initialState == TargetState.STARTED) + reconfigureConnectorTasksWithRetry(connName); } private void reconfigureConnectorTasksWithRetry(final String connName) { @@ -817,6 +832,11 @@ public void onCompletion(Throwable error, Void result) { // by the system, then checks whether any configs have actually changed before submitting the new configs to storage private void reconfigureConnector(final String connName, final Callback cb) { try { + if (!worker.isRunning(connName)) { + log.info("Skipping reconfiguration of connector {} since it is not running", connName); + return; + } + Map configs = configState.connectorConfig(connName); ConnectorConfig connConfig = new ConnectorConfig(configs); List sinkTopics = null; @@ -843,7 +863,7 @@ private void reconfigureConnector(final String connName, final Callback cb } if (changed) { if (isLeader()) { - configStorage.putTaskConfigs(taskConfigListAsMap(connName, taskProps)); + configBackingStore.putTaskConfigs(connName, taskConfigListAsMap(connName, taskProps)); cb.onCompletion(null, null); } else { // We cannot forward the request on the same thread because this reconfiguration can happen in as a @@ -868,21 +888,14 @@ public void run() { } } - private boolean isConfigSynced() { - return assignment != null && configState.offset() == assignment.offset(); - } - - // Common handling for requests that get config data. Checks if we are in sync with the current config, which allows - // us to answer requests directly. If we are not, handles invoking the callback with the appropriate error. - private boolean checkConfigSynced(Callback callback) { - if (!isConfigSynced()) { - if (!isLeader()) - callback.onCompletion(new NotLeaderException("Cannot get config data because config is not in sync and this is not the leader", leaderUrl()), null); - else - callback.onCompletion(new ConnectException("Cannot get config data because this is the leader node, but it does not have the most up to date configs"), null); - return false; + private boolean checkRebalanceNeeded(Callback callback) { + // Raise an error if we are expecting a rebalance to begin. This prevents us from forwarding requests + // based on stale leadership or assignment information + if (needsReconfigRebalance) { + callback.onCompletion(new RebalanceNeededException("Request cannot be completed because a rebalance is expected"), null); + return true; } - return true; + return false; } private void addRequest(Callable action, Callback callback) { @@ -896,6 +909,60 @@ private void addRequest(long delayMs, Callable action, Callback call member.wakeup(); } + public class ConfigUpdateListener implements ConfigBackingStore.UpdateListener { + @Override + public void onConnectorConfigRemove(String connector) { + log.info("Connector {} config removed", connector); + + synchronized (DistributedHerder.this) { + // rebalance after connector removal to ensure that existing tasks are balanced among workers + if (configState.contains(connector)) + needsReconfigRebalance = true; + connectorConfigUpdates.add(connector); + } + member.wakeup(); + } + + @Override + public void onConnectorConfigUpdate(String connector) { + log.info("Connector {} config updated", connector); + + // Stage the update and wake up the work thread. Connector config *changes* only need the one connector + // to be bounced. However, this callback may also indicate a connector *addition*, which does require + // a rebalance, so we need to be careful about what operation we request. + synchronized (DistributedHerder.this) { + if (!configState.contains(connector)) + needsReconfigRebalance = true; + connectorConfigUpdates.add(connector); + } + member.wakeup(); + } + + @Override + public void onTaskConfigUpdate(Collection tasks) { + log.info("Tasks {} configs updated", tasks); + + // Stage the update and wake up the work thread. No need to record the set of tasks here because task reconfigs + // always need a rebalance to ensure offsets get committed. + // TODO: As an optimization, some task config updates could avoid a rebalance. In particular, single-task + // connectors clearly don't need any coordination. + synchronized (DistributedHerder.this) { + needsReconfigRebalance = true; + } + member.wakeup(); + } + + @Override + public void onConnectorTargetStateChange(String connector) { + log.info("Connector {} target state change", connector); + + synchronized (DistributedHerder.this) { + connectorTargetStateChanges.add(connector); + } + member.wakeup(); + } + } + private class HerderRequest implements Comparable { private final long at; private final Callable action; @@ -931,43 +998,8 @@ public void onCompletion(Throwable error, Void result) { }; } - - // Config callbacks are triggered from the KafkaConfigStorage thread - private Callback connectorConfigCallback() { - return new Callback() { - @Override - public void onCompletion(Throwable error, String connector) { - log.info("Connector {} config updated", connector); - // Stage the update and wake up the work thread. Connector config *changes* only need the one connector - // to be bounced. However, this callback may also indicate a connector *addition*, which does require - // a rebalance, so we need to be careful about what operation we request. - synchronized (DistributedHerder.this) { - connectorConfigUpdates.add(connector); - } - member.wakeup(); - } - }; - } - - private Callback> taskConfigCallback() { - return new Callback>() { - @Override - public void onCompletion(Throwable error, List tasks) { - log.info("Tasks {} configs updated", tasks); - // Stage the update and wake up the work thread. No need to record the set of tasks here because task reconfigs - // always need a rebalance to ensure offsets get committed. - // TODO: As an optimization, some task config updates could avoid a rebalance. In particular, single-task - // connectors clearly don't need any coordination. - synchronized (DistributedHerder.this) { - needsReconfigRebalance = true; - } - member.wakeup(); - } - }; - } - private void updateDeletedConnectorStatus() { - ClusterConfigState snapshot = configStorage.snapshot(); + ClusterConfigState snapshot = configBackingStore.snapshot(); Set connectors = snapshot.connectors(); for (String connector : statusBackingStore.connectors()) { if (!connectors.contains(connector)) { @@ -978,73 +1010,70 @@ private void updateDeletedConnectorStatus() { } // Rebalances are triggered internally from the group member, so these are always executed in the work thread. - private WorkerRebalanceListener rebalanceListener() { - return new WorkerRebalanceListener() { - @Override - public void onAssigned(ConnectProtocol.Assignment assignment, int generation) { - // This callback just logs the info and saves it. The actual response is handled in the main loop, which - // ensures the group member's logic for rebalancing can complete, potentially long-running steps to - // catch up (or backoff if we fail) not executed in a callback, and so we'll be able to invoke other - // group membership actions (e.g., we may need to explicitly leave the group if we cannot handle the - // assigned tasks). - log.info("Joined group and got assignment: {}", assignment); - synchronized (DistributedHerder.this) { - DistributedHerder.this.assignment = assignment; - DistributedHerder.this.generation = generation; - rebalanceResolved = false; - } - - // Delete the statuses of all connectors removed prior to the start of this reblaance. This has to - // be done after the rebalance completes to avoid race conditions as the previous generation attempts - // to change the state to UNASSIGNED after tasks have been stopped. - if (isLeader()) - updateDeletedConnectorStatus(); - - // We *must* interrupt any poll() call since this could occur when the poll starts, and we might then - // sleep in the poll() for a long time. Forcing a wakeup ensures we'll get to process this event in the - // main thread. - member.wakeup(); + public class RebalanceListener implements WorkerRebalanceListener { + @Override + public void onAssigned(ConnectProtocol.Assignment assignment, int generation) { + // This callback just logs the info and saves it. The actual response is handled in the main loop, which + // ensures the group member's logic for rebalancing can complete, potentially long-running steps to + // catch up (or backoff if we fail) not executed in a callback, and so we'll be able to invoke other + // group membership actions (e.g., we may need to explicitly leave the group if we cannot handle the + // assigned tasks). + log.info("Joined group and got assignment: {}", assignment); + synchronized (DistributedHerder.this) { + DistributedHerder.this.assignment = assignment; + DistributedHerder.this.generation = generation; + rebalanceResolved = false; } - @Override - public void onRevoked(String leader, Collection connectors, Collection tasks) { - log.info("Rebalance started"); - - // Note that since we don't reset the assignment, we we don't revoke leadership here. During a rebalance, - // it is still important to have a leader that can write configs, offsets, etc. - - if (rebalanceResolved) { - // TODO: Parallelize this. We should be able to request all connectors and tasks to stop, then wait on all of - // them to finish - // TODO: Technically we don't have to stop connectors at all until we know they've really been removed from - // this worker. Instead, we can let them continue to run but buffer any update requests (which should be - // rare anyway). This would avoid a steady stream of start/stop, which probably also includes lots of - // unnecessary repeated connections to the source/sink system. - for (String connectorName : connectors) - worker.stopConnector(connectorName); - - // TODO: We need to at least commit task offsets, but if we could commit offsets & pause them instead of - // stopping them then state could continue to be reused when the task remains on this worker. For example, - // this would avoid having to close a connection and then reopen it when the task is assigned back to this - // worker again. - if (!tasks.isEmpty()) { - worker.stopTasks(tasks); // trigger stop() for all tasks - worker.awaitStopTasks(tasks); // await stopping tasks - } + // Delete the statuses of all connectors removed prior to the start of this reblaance. This has to + // be done after the rebalance completes to avoid race conditions as the previous generation attempts + // to change the state to UNASSIGNED after tasks have been stopped. + if (isLeader()) + updateDeletedConnectorStatus(); - // Ensure that all status updates have been pushed to the storage system before rebalancing. - // Otherwise, we may inadvertently overwrite the state with a stale value after the rebalance - // completes. - statusBackingStore.flush(); - log.info("Finished stopping tasks in preparation for rebalance"); - } else { - log.info("Wasn't unable to resume work after last rebalance, can skip stopping connectors and tasks"); + // We *must* interrupt any poll() call since this could occur when the poll starts, and we might then + // sleep in the poll() for a long time. Forcing a wakeup ensures we'll get to process this event in the + // main thread. + member.wakeup(); + } + + @Override + public void onRevoked(String leader, Collection connectors, Collection tasks) { + log.info("Rebalance started"); + + // Note that since we don't reset the assignment, we we don't revoke leadership here. During a rebalance, + // it is still important to have a leader that can write configs, offsets, etc. + + if (rebalanceResolved) { + // TODO: Parallelize this. We should be able to request all connectors and tasks to stop, then wait on all of + // them to finish + // TODO: Technically we don't have to stop connectors at all until we know they've really been removed from + // this worker. Instead, we can let them continue to run but buffer any update requests (which should be + // rare anyway). This would avoid a steady stream of start/stop, which probably also includes lots of + // unnecessary repeated connections to the source/sink system. + for (String connectorName : connectors) + worker.stopConnector(connectorName); + + // TODO: We need to at least commit task offsets, but if we could commit offsets & pause them instead of + // stopping them then state could continue to be reused when the task remains on this worker. For example, + // this would avoid having to close a connection and then reopen it when the task is assigned back to this + // worker again. + if (!tasks.isEmpty()) { + worker.stopTasks(tasks); // trigger stop() for all tasks + worker.awaitStopTasks(tasks); // await stopping tasks } + + // Ensure that all status updates have been pushed to the storage system before rebalancing. + // Otherwise, we may inadvertently overwrite the state with a stale value after the rebalance + // completes. + statusBackingStore.flush(); + log.info("Finished stopping tasks in preparation for rebalance"); + } else { + log.info("Wasn't unable to resume work after last rebalance, can skip stopping connectors and tasks"); } - }; + } } - private static Map> taskConfigListAsMap(String connName, List> configs) { int index = 0; Map> result = new HashMap<>(); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/StaleConfigException.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/RebalanceNeededException.java similarity index 89% rename from connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/StaleConfigException.java rename to connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/RebalanceNeededException.java index c615b37757056..80c08a3d42c7f 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/StaleConfigException.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/RebalanceNeededException.java @@ -18,9 +18,9 @@ import org.apache.kafka.connect.errors.ConnectException; -public class StaleConfigException extends ConnectException { +public class RebalanceNeededException extends ConnectException { - public StaleConfigException(String s) { + public RebalanceNeededException(String s) { super(s); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java index d5802c6c36ed5..9c7496026c8f2 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinator.java @@ -24,7 +24,7 @@ import org.apache.kafka.common.requests.JoinGroupRequest.ProtocolMetadata; import org.apache.kafka.common.utils.CircularIterator; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.connect.storage.KafkaConfigStorage; +import org.apache.kafka.connect.storage.ConfigBackingStore; import org.apache.kafka.connect.util.ConnectorTaskId; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -49,7 +49,7 @@ public final class WorkerCoordinator extends AbstractCoordinator implements Clos public static final String DEFAULT_SUBPROTOCOL = "default"; private final String restUrl; - private final KafkaConfigStorage configStorage; + private final ConfigBackingStore configStorage; private ConnectProtocol.Assignment assignmentSnapshot; private final WorkerCoordinatorMetrics sensors; private ClusterConfigState configSnapshot; @@ -70,7 +70,7 @@ public WorkerCoordinator(ConsumerNetworkClient client, Time time, long retryBackoffMs, String restUrl, - KafkaConfigStorage configStorage, + ConfigBackingStore configStorage, WorkerRebalanceListener listener) { super(client, groupId, diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java index 058f17114ba81..85af549fe1d36 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/WorkerGroupMember.java @@ -32,8 +32,8 @@ import org.apache.kafka.common.network.Selector; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.Time; -import org.apache.kafka.connect.storage.KafkaConfigStorage; import org.apache.kafka.connect.util.ConnectorTaskId; +import org.apache.kafka.connect.storage.ConfigBackingStore; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -69,7 +69,7 @@ public class WorkerGroupMember { public WorkerGroupMember(DistributedConfig config, String restUrl, - KafkaConfigStorage configStorage, + ConfigBackingStore configStorage, WorkerRebalanceListener listener, Time time) { try { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java index a53ed7d6ba01c..fc713cabdd2ad 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java @@ -20,7 +20,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.runtime.Herder; -import org.apache.kafka.connect.runtime.distributed.StaleConfigException; +import org.apache.kafka.connect.runtime.distributed.RebalanceNeededException; import org.apache.kafka.connect.runtime.distributed.RequestTargetException; import org.apache.kafka.connect.runtime.rest.RestServer; import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; @@ -149,6 +149,20 @@ public void restartConnector(final @PathParam("connector") String connector, completeOrForwardRequest(cb, "/connectors/" + connector + "/restart", "POST", null, forward); } + @PUT + @Path("/{connector}/pause") + public Response pauseConnector(@PathParam("connector") String connector) { + herder.pauseConnector(connector); + return Response.accepted().build(); + } + + @PUT + @Path("/{connector}/resume") + public Response resumeConnector(@PathParam("connector") String connector) { + herder.resumeConnector(connector); + return Response.accepted().build(); + } + @GET @Path("/{connector}/tasks") public List getTaskConfigs(final @PathParam("connector") String connector, @@ -230,7 +244,7 @@ private T completeOrForwardRequest(FutureCallback cb, throw new ConnectRestException(Response.Status.CONFLICT.getStatusCode(), "Cannot complete request because of a conflicting operation (e.g. worker rebalance)"); } - } else if (cause instanceof StaleConfigException) { + } else if (cause instanceof RebalanceNeededException) { throw new ConnectRestException(Response.Status.CONFLICT.getStatusCode(), "Cannot complete request momentarily due to stale configuration (typically caused by a concurrent config change)"); } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java index a59336a2cede3..ad02e99587c12 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java @@ -23,10 +23,14 @@ import org.apache.kafka.connect.runtime.AbstractHerder; import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.runtime.HerderConnectorContext; +import org.apache.kafka.connect.runtime.TargetState; import org.apache.kafka.connect.runtime.TaskConfig; import org.apache.kafka.connect.runtime.Worker; +import org.apache.kafka.connect.runtime.distributed.ClusterConfigState; import org.apache.kafka.connect.runtime.rest.entities.ConnectorInfo; import org.apache.kafka.connect.runtime.rest.entities.TaskInfo; +import org.apache.kafka.connect.storage.ConfigBackingStore; +import org.apache.kafka.connect.storage.MemoryConfigBackingStore; import org.apache.kafka.connect.storage.MemoryStatusBackingStore; import org.apache.kafka.connect.storage.StatusBackingStore; import org.apache.kafka.connect.util.Callback; @@ -37,11 +41,8 @@ import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; -import java.util.HashSet; import java.util.List; import java.util.Map; -import java.util.Set; - /** * Single process, in-memory "herder". Useful for a standalone Kafka Connect process. @@ -49,17 +50,20 @@ public class StandaloneHerder extends AbstractHerder { private static final Logger log = LoggerFactory.getLogger(StandaloneHerder.class); - private HashMap connectors = new HashMap<>(); + private ClusterConfigState configState; public StandaloneHerder(Worker worker) { - this(worker.workerId(), worker, new MemoryStatusBackingStore()); + this(worker, worker.workerId(), new MemoryStatusBackingStore(), new MemoryConfigBackingStore()); } // visible for testing - StandaloneHerder(String workerId, - Worker worker, - StatusBackingStore statusBackingStore) { - super(worker, statusBackingStore, workerId); + StandaloneHerder(Worker worker, + String workerId, + StatusBackingStore statusBackingStore, + MemoryConfigBackingStore configBackingStore) { + super(worker, workerId, statusBackingStore, configBackingStore); + this.configState = ClusterConfigState.EMPTY; + configBackingStore.setUpdateListener(new ConfigUpdateListener()); } public synchronized void start() { @@ -74,7 +78,7 @@ public synchronized void stop() { // There's no coordination/hand-off to do here since this is all standalone. Instead, we // should just clean up the stuff we normally would, i.e. cleanly checkpoint and shutdown all // the tasks. - for (String connName : new HashSet<>(connectors.keySet())) { + for (String connName : configState.connectors()) { removeConnectorTasks(connName); try { worker.stopConnector(connName); @@ -82,8 +86,7 @@ public synchronized void stop() { log.error("Error shutting down connector {}: ", connName, e); } } - connectors.clear(); - + stopServices(); log.info("Herder stopped"); } @@ -94,27 +97,24 @@ public int generation() { @Override public synchronized void connectors(Callback> callback) { - callback.onCompletion(null, new ArrayList<>(connectors.keySet())); + callback.onCompletion(null, configState.connectors()); } @Override public synchronized void connectorInfo(String connName, Callback callback) { - ConnectorState state = connectors.get(connName); - if (state == null) { + ConnectorInfo connectorInfo = createConnectorInfo(connName); + if (connectorInfo == null) { callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null); return; } - callback.onCompletion(null, createConnectorInfo(state)); + callback.onCompletion(null, connectorInfo); } - private ConnectorInfo createConnectorInfo(ConnectorState state) { - if (state == null) + private ConnectorInfo createConnectorInfo(String connector) { + if (!configState.contains(connector)) return null; - - List taskIds = new ArrayList<>(); - for (int i = 0; i < state.taskConfigs.size(); i++) - taskIds.add(new ConnectorTaskId(state.name, i)); - return new ConnectorInfo(state.name, state.configOriginals, taskIds); + Map config = configState.connectorConfig(connector); + return new ConnectorInfo(connector, config, configState.tasks(connector)); } @Override @@ -133,12 +133,13 @@ public void onCompletion(Throwable error, ConnectorInfo result) { } @Override - public synchronized void putConnectorConfig(String connName, final Map config, + public synchronized void putConnectorConfig(String connName, + final Map config, boolean allowReplace, final Callback> callback) { try { boolean created = false; - if (connectors.containsKey(connName)) { + if (configState.contains(connName)) { if (!allowReplace) { callback.onCompletion(new AlreadyExistsException("Connector " + connName + " already exists"), null); return; @@ -147,7 +148,7 @@ public synchronized void putConnectorConfig(String connName, final Map(created, createConnectorInfo(connectors.get(connName)))); + callback.onCompletion(null, new Created<>(created, createConnectorInfo(connName))); else callback.onCompletion(null, new Created(false, null)); } catch (ConnectException e) { @@ -183,17 +184,14 @@ public synchronized void requestTaskReconfiguration(String connName) { @Override public synchronized void taskConfigs(String connName, Callback> callback) { - ConnectorState state = connectors.get(connName); - if (state == null) { + if (!configState.contains(connName)) { callback.onCompletion(new NotFoundException("Connector " + connName + " not found", null), null); return; } List result = new ArrayList<>(); - for (int i = 0; i < state.taskConfigs.size(); i++) { - TaskInfo info = new TaskInfo(new ConnectorTaskId(connName, i), state.taskConfigs.get(i)); - result.add(info); - } + for (ConnectorTaskId taskId : configState.tasks(connName)) + result.add(new TaskInfo(taskId, configState.taskConfig(taskId))); callback.onCompletion(null, result); } @@ -204,19 +202,17 @@ public void putTaskConfigs(String connName, List> configs, C @Override public synchronized void restartTask(ConnectorTaskId taskId, Callback cb) { - if (!connectors.containsKey(taskId.connector())) + if (!configState.contains(taskId.connector())) cb.onCompletion(new NotFoundException("Connector " + taskId.connector() + " not found", null), null); - ConnectorState state = connectors.get(taskId.connector()); - if (state.taskConfigs.contains(taskId)) + Map taskConfig = configState.taskConfig(taskId); + if (taskConfig == null) cb.onCompletion(new NotFoundException("Task " + taskId + " not found", null), null); + TargetState targetState = configState.targetState(taskId.connector()); try { worker.stopAndAwaitTask(taskId); - - Map taskConfig = state.taskConfigs.get(taskId.task()); - worker.startTask(taskId, new TaskConfig(taskConfig), this); - + worker.startTask(taskId, new TaskConfig(taskConfig), this, targetState); cb.onCompletion(null, null); } catch (Exception e) { log.error("Failed to restart task {}", taskId, e); @@ -226,13 +222,13 @@ public synchronized void restartTask(ConnectorTaskId taskId, Callback cb) @Override public synchronized void restartConnector(String connName, Callback cb) { - if (!connectors.containsKey(connName)) + if (!configState.contains(connName)) cb.onCompletion(new NotFoundException("Connector " + connName + " not found", null), null); - ConnectorState state = connectors.get(connName); + Map config = configState.connectorConfig(connName); try { worker.stopConnector(connName); - worker.startConnector(state.config, new HerderConnectorContext(this, connName), this); + startConnector(config); cb.onCompletion(null, null); } catch (Exception e) { log.error("Failed to restart connector {}", connName, e); @@ -248,86 +244,110 @@ public synchronized void restartConnector(String connName, Callback cb) { private String startConnector(Map connectorProps) { ConnectorConfig connConfig = new ConnectorConfig(connectorProps); String connName = connConfig.getString(ConnectorConfig.NAME_CONFIG); - ConnectorState state = connectors.get(connName); - worker.startConnector(connConfig, new HerderConnectorContext(this, connName), this); - if (state == null) { - connectors.put(connName, new ConnectorState(connectorProps, connConfig)); - } else { - state.configOriginals = connectorProps; - state.config = connConfig; - } + configBackingStore.putConnectorConfig(connName, connectorProps); + TargetState targetState = configState.targetState(connName); + worker.startConnector(connConfig, new HerderConnectorContext(this, connName), this, targetState); return connName; } + private Map> recomputeTaskConfigs(String connName) { + Map config = configState.connectorConfig(connName); + ConnectorConfig connConfig = new ConnectorConfig(config); - private List> recomputeTaskConfigs(String connName) { - ConnectorState state = connectors.get(connName); - return worker.connectorTaskConfigs(connName, - state.config.getInt(ConnectorConfig.TASKS_MAX_CONFIG), - state.config.getList(ConnectorConfig.TOPICS_CONFIG)); - } + List> taskConfigs = worker.connectorTaskConfigs(connName, + connConfig.getInt(ConnectorConfig.TASKS_MAX_CONFIG), + connConfig.getList(ConnectorConfig.TOPICS_CONFIG)); - private void createConnectorTasks(String connName) { - ConnectorState state = connectors.get(connName); - int index = 0; - for (Map taskConfigMap : state.taskConfigs) { - ConnectorTaskId taskId = new ConnectorTaskId(connName, index); - startTask(taskId, taskConfigMap); - index++; - } - } + int i = 0; + Map> taskConfigMap = new HashMap<>(); + for (Map taskConfig : taskConfigs) + taskConfigMap.put(new ConnectorTaskId(connName, i++), taskConfig); - private void startTask(ConnectorTaskId taskId, Map taskConfigMap) { - TaskConfig config = new TaskConfig(taskConfigMap); - try { - worker.startTask(taskId, config, this); - } catch (Throwable e) { - log.error("Failed to add task {}: ", taskId, e); - // Swallow this so we can continue updating the rest of the tasks - // FIXME what's the proper response? Kill all the tasks? Consider this the same as a task - // that died after starting successfully. - } + return taskConfigMap; } - private Set tasksFor(ConnectorState state) { - Set tasks = new HashSet<>(); - for (int i = 0; i < state.taskConfigs.size(); i++) - tasks.add(new ConnectorTaskId(state.name, i)); - return tasks; + private void createConnectorTasks(String connName, TargetState initialState) { + for (ConnectorTaskId taskId : configState.tasks(connName)) { + Map taskConfigMap = configState.taskConfig(taskId); + TaskConfig config = new TaskConfig(taskConfigMap); + try { + worker.startTask(taskId, config, this, initialState); + } catch (Throwable e) { + log.error("Failed to add task {}: ", taskId, e); + // Swallow this so we can continue updating the rest of the tasks + // FIXME what's the proper response? Kill all the tasks? Consider this the same as a task + // that died after starting successfully. + } + } } private void removeConnectorTasks(String connName) { - ConnectorState state = connectors.get(connName); - Set tasks = tasksFor(state); + Collection tasks = configState.tasks(connName); if (!tasks.isEmpty()) { worker.stopTasks(tasks); worker.awaitStopTasks(tasks); - state.taskConfigs = new ArrayList<>(); + configBackingStore.removeTaskConfigs(connName); } } private void updateConnectorTasks(String connName) { - List> newTaskConfigs = recomputeTaskConfigs(connName); - ConnectorState state = connectors.get(connName); - if (!newTaskConfigs.equals(state.taskConfigs)) { + if (!worker.isRunning(connName)) { + log.info("Skipping reconfiguration of connector {} since it is not running", connName); + return; + } + + Map> newTaskConfigs = recomputeTaskConfigs(connName); + Map> oldTaskConfigs = configState.allTaskConfigs(connName); + + if (!newTaskConfigs.equals(oldTaskConfigs)) { removeConnectorTasks(connName); - state.taskConfigs = newTaskConfigs; - createConnectorTasks(connName); + configBackingStore.putTaskConfigs(connName, newTaskConfigs); + createConnectorTasks(connName, configState.targetState(connName)); } } + // This update listener assumes synchronous updates the ConfigBackingStore, which only works + // with the MemoryConfigBackingStore. This allows us to write a change (e.g. through + // ConfigBackingStore.putConnectorConfig()) and then immediately read it back from an updated + // snapshot. + // TODO: To get any real benefit from the backing store abstraction, we should move some of + // the handling into the callbacks in this listener. + private class ConfigUpdateListener implements ConfigBackingStore.UpdateListener { + + @Override + public void onConnectorConfigRemove(String connector) { + synchronized (StandaloneHerder.this) { + configState = configBackingStore.snapshot(); + } + } + + @Override + public void onConnectorConfigUpdate(String connector) { + // TODO: move connector configuration update handling here to be consistent with + // the semantics of the config backing store - private static class ConnectorState { - public String name; - public Map configOriginals; - public ConnectorConfig config; - List> taskConfigs; + synchronized (StandaloneHerder.this) { + configState = configBackingStore.snapshot(); + } + } - public ConnectorState(Map configOriginals, ConnectorConfig config) { - this.name = config.getString(ConnectorConfig.NAME_CONFIG); - this.configOriginals = configOriginals; - this.config = config; - this.taskConfigs = new ArrayList<>(); + @Override + public void onTaskConfigUpdate(Collection tasks) { + synchronized (StandaloneHerder.this) { + configState = configBackingStore.snapshot(); + } + } + + @Override + public void onConnectorTargetStateChange(String connector) { + synchronized (StandaloneHerder.this) { + configState = configBackingStore.snapshot(); + TargetState targetState = configState.targetState(connector); + worker.setTargetState(connector, targetState); + if (targetState == TargetState.STARTED) + updateConnectorTasks(connector); + } } } + } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConfigBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConfigBackingStore.java new file mode 100644 index 0000000000000..5244842a2c984 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConfigBackingStore.java @@ -0,0 +1,126 @@ +/** + * 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 org.apache.kafka.connect.storage; + +import org.apache.kafka.connect.runtime.TargetState; +import org.apache.kafka.connect.runtime.WorkerConfig; +import org.apache.kafka.connect.runtime.distributed.ClusterConfigState; +import org.apache.kafka.connect.util.ConnectorTaskId; + +import java.util.Collection; +import java.util.Map; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.TimeoutException; + +public interface ConfigBackingStore { + + void configure(WorkerConfig config); + + void start(); + + void stop(); + + /** + * Get a snapshot of the current configuration state including all connector and task + * configurations. + * @return the cluster config state + */ + ClusterConfigState snapshot(); + + /** + * Check if the store has configuration for a connector. + * @param connector name of the connector + * @return true if the backing store contains configuration for the connector + */ + boolean contains(String connector); + + /** + * Update the configuration for a connector. + * @param connector name of the connector + * @param properties the connector configuration + */ + void putConnectorConfig(String connector, Map properties); + + /** + * Remove configuration for a connector + * @param connector name of the connector + */ + void removeConnectorConfig(String connector); + + /** + * Update the task configurations for a connector. + * @param connector name of the connector + * @param configs the new task configs + */ + void putTaskConfigs(String connector, Map> configs); + + /** + * Remove the task configs associated with a connector. + * @param connector name of the connector + */ + void removeTaskConfigs(String connector); + + /** + * Refresh the backing store. This forces the store to ensure that it has the latest + * configs that have been written. + * @param timeout max time to wait for the refresh to complete + * @param unit unit of timeout + * @throws TimeoutException if the timeout expires before the refresh has completed + */ + void refresh(long timeout, TimeUnit unit) throws TimeoutException; + + /** + * Transition a connector to a new target state (e.g. paused). + * @param connector name of the connector + * @param state the state to transition to + */ + void putTargetState(String connector, TargetState state); + + /** + * Set an update listener to get notifications when there are config/target state + * changes. + * @param listener non-null listener + */ + void setUpdateListener(UpdateListener listener); + + interface UpdateListener { + /** + * Invoked when a connector configuration has been removed + * @param connector name of the connector + */ + void onConnectorConfigRemove(String connector); + + /** + * Invoked when a connector configuration has been updated. + * @param connector name of the connector + */ + void onConnectorConfigUpdate(String connector); + + /** + * Invoked when task configs are updated. + * @param tasks all the tasks whose configs have been updated + */ + void onTaskConfigUpdate(Collection tasks); + + /** + * Invoked when the user has set a new target state (e.g. paused) + * @param connector name of the connector + */ + void onConnectorTargetStateChange(String connector); + } + +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigStorage.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java similarity index 83% rename from connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigStorage.java rename to connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java index 08c528c30c53f..8d2028852ecf8 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigStorage.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java @@ -33,6 +33,8 @@ import org.apache.kafka.connect.data.Struct; import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.errors.DataException; +import org.apache.kafka.connect.runtime.TargetState; +import org.apache.kafka.connect.runtime.WorkerConfig; import org.apache.kafka.connect.runtime.distributed.ClusterConfigState; import org.apache.kafka.connect.runtime.distributed.DistributedConfig; import org.apache.kafka.connect.util.Callback; @@ -50,7 +52,6 @@ import java.util.Set; import java.util.TreeSet; import java.util.concurrent.ExecutionException; -import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -146,8 +147,14 @@ * rebalance must be deferred. *

          */ -public class KafkaConfigStorage { - private static final Logger log = LoggerFactory.getLogger(KafkaConfigStorage.class); +public class KafkaConfigBackingStore implements ConfigBackingStore { + private static final Logger log = LoggerFactory.getLogger(KafkaConfigBackingStore.class); + + public static final String TARGET_STATE_PREFIX = "target-state-"; + + public static String TARGET_STATE_KEY(String connectorName) { + return TARGET_STATE_PREFIX + connectorName; + } public static final String CONNECTOR_PREFIX = "connector-"; @@ -178,14 +185,17 @@ public static String COMMIT_TASKS_KEY(String connectorName) { public static final Schema CONNECTOR_TASKS_COMMIT_V0 = SchemaBuilder.struct() .field("tasks", Schema.INT32_SCHEMA) .build(); + public static final Schema TARGET_STATE_V0 = SchemaBuilder.struct() + .field("state", Schema.STRING_SCHEMA) + .build(); private static final long READ_TO_END_TIMEOUT_MS = 30000; private final Object lock; private boolean starting; private final Converter converter; - private final Callback connectorConfigCallback; - private final Callback> tasksConfigCallback; + private UpdateListener updateListener; + private String topic; // Data is passed to the log already serialized. We use a converter to handle translating to/from generic Connect // format to serialized form @@ -200,23 +210,27 @@ public static String COMMIT_TASKS_KEY(String connectorName) { private Set inconsistent = new HashSet<>(); // The most recently read offset. This does not take into account deferred task updates/commits, so we may have // outstanding data to be applied. - private long offset; + private volatile long offset; // Connector -> Map[ConnectorTaskId -> Configs] - private Map>> deferredTaskUpdates = new HashMap<>(); + private final Map>> deferredTaskUpdates = new HashMap<>(); + private final Map connectorTargetStates = new HashMap<>(); - public KafkaConfigStorage(Converter converter, Callback connectorConfigCallback, Callback> tasksConfigCallback) { + public KafkaConfigBackingStore(Converter converter) { this.lock = new Object(); this.starting = false; this.converter = converter; - this.connectorConfigCallback = connectorConfigCallback; - this.tasksConfigCallback = tasksConfigCallback; + this.offset = -1; + } - offset = -1; + @Override + public void setUpdateListener(UpdateListener listener) { + this.updateListener = listener; } - public void configure(DistributedConfig config) { + @Override + public void configure(WorkerConfig config) { topic = config.getString(DistributedConfig.CONFIG_TOPIC_CONFIG); if (topic.equals("")) throw new ConfigException("Must specify topic for connector configuration."); @@ -232,28 +246,31 @@ public void configure(DistributedConfig config) { consumerProps.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, StringDeserializer.class.getName()); consumerProps.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); - configLog = createKafkaBasedLog(topic, producerProps, consumerProps, consumedCallback); + configLog = createKafkaBasedLog(topic, producerProps, consumerProps, new ConsumeCallback()); } + @Override public void start() { - log.info("Starting KafkaConfigStorage"); + log.info("Starting KafkaConfigBackingStore"); // During startup, callbacks are *not* invoked. You can grab a snapshot after starting -- just take care that // updates can continue to occur in the background starting = true; configLog.start(); starting = false; - log.info("Started KafkaConfigStorage"); + log.info("Started KafkaConfigBackingStore"); } + @Override public void stop() { - log.info("Closing KafkaConfigStorage"); + log.info("Closing KafkaConfigBackingStore"); configLog.stop(); - log.info("Closed KafkaConfigStorage"); + log.info("Closed KafkaConfigBackingStore"); } /** * Get a snapshot of the current state of the cluster. */ + @Override public ClusterConfigState snapshot() { synchronized (lock) { // Doing a shallow copy of the data is safe here because the complex nested data that is copied should all be @@ -262,12 +279,20 @@ public ClusterConfigState snapshot() { offset, new HashMap<>(connectorTaskCounts), new HashMap<>(connectorConfigs), + new HashMap<>(connectorTargetStates), new HashMap<>(taskConfigs), new HashSet<>(inconsistent) ); } } + @Override + public boolean contains(String connector) { + synchronized (lock) { + return connectorConfigs.containsKey(connector); + } + } + /** * Write this connector configuration to persistent storage and wait until it has been acknowledge and read back by * tailing the Kafka log with a consumer. @@ -275,18 +300,33 @@ public ClusterConfigState snapshot() { * @param connector name of the connector to write data for * @param properties the configuration to write */ + @Override public void putConnectorConfig(String connector, Map properties) { - byte[] serializedConfig; - if (properties == null) { - serializedConfig = null; - } else { - Struct connectConfig = new Struct(CONNECTOR_CONFIGURATION_V0); - connectConfig.put("properties", properties); - serializedConfig = converter.fromConnectData(topic, CONNECTOR_CONFIGURATION_V0, connectConfig); - } + log.debug("Writing connector configuration {} for connector {} configuration", properties, connector); + Struct connectConfig = new Struct(CONNECTOR_CONFIGURATION_V0); + connectConfig.put("properties", properties); + byte[] serializedConfig = converter.fromConnectData(topic, CONNECTOR_CONFIGURATION_V0, connectConfig); + updateConnectorConfig(connector, serializedConfig); + } + /** + * Remove configuration for a given connector. + * @param connector name of the connector to remove + */ + @Override + public void removeConnectorConfig(String connector) { + log.debug("Removing connector configuration for connector {}", connector); + updateConnectorConfig(connector, null); + configLog.send(TARGET_STATE_KEY(connector), null); + } + + @Override + public void removeTaskConfigs(String connector) { + throw new UnsupportedOperationException("Removal of tasks is not currently supported"); + } + + private void updateConnectorConfig(String connector, byte[] serializedConfig) { try { - log.debug("Writing connector configuration for connector " + connector + " configuration: " + properties); configLog.send(CONNECTOR_KEY(connector), serializedConfig); configLog.readToEnd().get(READ_TO_END_TIMEOUT_MS, TimeUnit.MILLISECONDS); } catch (InterruptedException | ExecutionException | TimeoutException e) { @@ -303,7 +343,8 @@ public void putConnectorConfig(String connector, Map properties) * @throws ConnectException if the task configurations do not resolve inconsistencies found in the existing root * and task configurations. */ - public void putTaskConfigs(Map> configs) { + @Override + public void putTaskConfigs(String connector, Map> configs) { // Make sure we're at the end of the log. We should be the only writer, but we want to make sure we don't have // any outstanding lagging data to consume. try { @@ -362,12 +403,22 @@ public void putTaskConfigs(Map> configs) { } } - public Future readToEnd() { - return configLog.readToEnd(); + @Override + public void refresh(long timeout, TimeUnit unit) throws TimeoutException { + try { + configLog.readToEnd().get(timeout, unit); + } catch (InterruptedException | ExecutionException e) { + throw new ConnectException("Error trying to read to end of config log", e); + } } - public void readToEnd(Callback cb) { - configLog.readToEnd(cb); + @Override + public void putTargetState(String connector, TargetState state) { + Struct connectTargetState = new Struct(TARGET_STATE_V0); + connectTargetState.put("state", state.name()); + byte[] serializedTargetState = converter.fromConnectData(topic, TARGET_STATE_V0, connectTargetState); + log.debug("Writing target state {} for connector {}", state, connector); + configLog.send(TARGET_STATE_KEY(connector), serializedTargetState); } private KafkaBasedLog createKafkaBasedLog(String topic, Map producerProps, @@ -375,12 +426,11 @@ private KafkaBasedLog createKafkaBasedLog(String topic, Map(topic, producerProps, consumerProps, consumedCallback, new SystemTime()); } - @SuppressWarnings("unchecked") - private final Callback> consumedCallback = new Callback>() { + private class ConsumeCallback implements Callback> { @Override public void onCompletion(Throwable error, ConsumerRecord record) { if (error != null) { - log.error("Unexpected in consumer callback for KafkaConfigStorage: ", error); + log.error("Unexpected in consumer callback for KafkaConfigBackingStore: ", error); return; } @@ -395,13 +445,43 @@ public void onCompletion(Throwable error, ConsumerRecord record) // *next record*, not the last one consumed. offset = record.offset() + 1; - if (record.key().startsWith(CONNECTOR_PREFIX)) { + if (record.key().startsWith(TARGET_STATE_PREFIX)) { + String connectorName = record.key().substring(TARGET_STATE_PREFIX.length()); + synchronized (lock) { + if (value.value() != null) { + if (!(value.value() instanceof Map)) { + log.error("Found target state ({}) in wrong format: {}", record.key(), value.value().getClass()); + return; + } + Object targetState = ((Map) value.value()).get("state"); + if (!(targetState instanceof String)) { + log.error("Invalid data for target state for connector ({}): 'state' field should be a Map but is {}", + connectorName, targetState == null ? null : targetState.getClass()); + return; + } + + try { + TargetState state = TargetState.valueOf((String) targetState); + log.trace("Setting target state for connector {} to {}", connectorName, targetState); + connectorTargetStates.put(connectorName, state); + } catch (IllegalArgumentException e) { + log.error("Invalid target state for connector ({}): {}", connectorName, targetState); + return; + } + } + } + + if (!starting) + updateListener.onConnectorTargetStateChange(connectorName); + } else if (record.key().startsWith(CONNECTOR_PREFIX)) { String connectorName = record.key().substring(CONNECTOR_PREFIX.length()); + boolean removed = false; synchronized (lock) { if (value.value() == null) { // Connector deletion will be written as a null value log.info("Removed connector " + connectorName + " due to null configuration. This is usually intentional and does not indicate an issue."); connectorConfigs.remove(connectorName); + removed = true; } else { // Connector configs can be applied and callbacks invoked immediately if (!(value.value() instanceof Map)) { @@ -410,15 +490,23 @@ public void onCompletion(Throwable error, ConsumerRecord record) } Object newConnectorConfig = ((Map) value.value()).get("properties"); if (!(newConnectorConfig instanceof Map)) { - log.error("Invalid data for connector config (" + connectorName + "): properties filed should be a Map but is " + newConnectorConfig.getClass()); + log.error("Invalid data for connector config ({}): properties field should be a Map but is {}", connectorName, + newConnectorConfig == null ? null : newConnectorConfig.getClass()); return; } - log.debug("Updating configuration for connector " + connectorName + " configuation: " + newConnectorConfig); + log.debug("Updating configuration for connector " + connectorName + " configuration: " + newConnectorConfig); connectorConfigs.put(connectorName, (Map) newConnectorConfig); + + if (!connectorTargetStates.containsKey(connectorName)) + connectorTargetStates.put(connectorName, TargetState.STARTED); } } - if (!starting) - connectorConfigCallback.onCompletion(null, connectorName); + if (!starting) { + if (removed) + updateListener.onConnectorConfigRemove(connectorName); + else + updateListener.onConnectorConfigUpdate(connectorName); + } } else if (record.key().startsWith(TASK_PREFIX)) { synchronized (lock) { ConnectorTaskId taskId = parseTaskId(record.key()); @@ -512,12 +600,13 @@ public void onCompletion(Throwable error, ConsumerRecord record) } if (!starting) - tasksConfigCallback.onCompletion(null, updatedTasks); + updateListener.onTaskConfigUpdate(updatedTasks); } else { log.error("Discarding config update record with invalid key: " + record.key()); } } - }; + + } private ConnectorTaskId parseTaskId(String key) { String[] parts = key.split("-"); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java new file mode 100644 index 0000000000000..ec5f2e6c47522 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java @@ -0,0 +1,154 @@ +/** + * 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 org.apache.kafka.connect.storage; + +import org.apache.kafka.connect.runtime.TargetState; +import org.apache.kafka.connect.runtime.WorkerConfig; +import org.apache.kafka.connect.runtime.distributed.ClusterConfigState; +import org.apache.kafka.connect.util.ConnectorTaskId; + +import java.util.Collections; +import java.util.HashMap; +import java.util.HashSet; +import java.util.Map; +import java.util.concurrent.TimeUnit; + +public class MemoryConfigBackingStore implements ConfigBackingStore { + + private Map connectors = new HashMap<>(); + private UpdateListener updateListener; + + @Override + public void configure(WorkerConfig config) { + } + + @Override + public synchronized void start() { + } + + @Override + public synchronized void stop() { + } + + @Override + public synchronized ClusterConfigState snapshot() { + Map connectorTaskCounts = new HashMap<>(); + Map> connectorConfigs = new HashMap<>(); + Map connectorTargetStates = new HashMap<>(); + Map> taskConfigs = new HashMap<>(); + + for (Map.Entry connectorStateEntry : connectors.entrySet()) { + String connector = connectorStateEntry.getKey(); + ConnectorState connectorState = connectorStateEntry.getValue(); + connectorTaskCounts.put(connector, connectorState.taskConfigs.size()); + connectorConfigs.put(connector, connectorState.connConfig); + connectorTargetStates.put(connector, connectorState.targetState); + taskConfigs.putAll(connectorState.taskConfigs); + } + + return new ClusterConfigState( + ClusterConfigState.NO_OFFSET, + connectorTaskCounts, + connectorConfigs, + connectorTargetStates, + taskConfigs, + Collections.emptySet()); + } + + @Override + public synchronized boolean contains(String connector) { + return connectors.containsKey(connector); + } + + @Override + public synchronized void putConnectorConfig(String connector, Map properties) { + ConnectorState state = connectors.get(connector); + if (state == null) + connectors.put(connector, new ConnectorState(properties)); + else + state.connConfig = properties; + + if (updateListener != null) + updateListener.onConnectorConfigUpdate(connector); + } + + @Override + public synchronized void removeConnectorConfig(String connector) { + ConnectorState state = connectors.remove(connector); + + if (updateListener != null && state != null) + updateListener.onConnectorConfigRemove(connector); + } + + @Override + public synchronized void removeTaskConfigs(String connector) { + ConnectorState state = connectors.get(connector); + if (state == null) + throw new IllegalArgumentException("Cannot remove tasks for non-existing connector"); + + HashSet taskIds = new HashSet<>(state.taskConfigs.keySet()); + state.taskConfigs.clear(); + + if (updateListener != null) + updateListener.onTaskConfigUpdate(taskIds); + } + + @Override + public synchronized void putTaskConfigs(String connector, Map> configs) { + ConnectorState state = connectors.get(connector); + if (state == null) + throw new IllegalArgumentException("Cannot put tasks for non-existing connector"); + + state.taskConfigs = configs; + + if (updateListener != null) + updateListener.onTaskConfigUpdate(configs.keySet()); + } + + @Override + public void refresh(long timeout, TimeUnit unit) { + } + + @Override + public synchronized void putTargetState(String connector, TargetState state) { + ConnectorState connectorState = connectors.get(connector); + if (connectorState == null) + throw new IllegalArgumentException("No connector `" + connector + "` configured"); + + connectorState.targetState = state; + + if (updateListener != null) + updateListener.onConnectorTargetStateChange(connector); + } + + @Override + public synchronized void setUpdateListener(UpdateListener listener) { + this.updateListener = listener; + } + + private static class ConnectorState { + private TargetState targetState; + private Map connConfig; + private Map> taskConfigs; + + public ConnectorState(Map connConfig) { + this.targetState = TargetState.STARTED; + this.connConfig = connConfig; + this.taskConfigs = new HashMap<>(); + } + } +} diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/StatusBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/StatusBackingStore.java index 268c5df970075..69f1fdc78fec7 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/StatusBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/StatusBackingStore.java @@ -102,5 +102,5 @@ public interface StatusBackingStore { * Configure class with the given key-value pairs * @param config config for StatusBackingStore */ - public void configure(WorkerConfig config); + void configure(WorkerConfig config); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java index e4084a809ae55..d7a2b2ac59895 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/AbstractHerderTest.java @@ -17,6 +17,7 @@ package org.apache.kafka.connect.runtime; import org.apache.kafka.connect.runtime.rest.entities.ConnectorStateInfo; +import org.apache.kafka.connect.storage.ConfigBackingStore; import org.apache.kafka.connect.storage.StatusBackingStore; import org.apache.kafka.connect.util.ConnectorTaskId; import org.easymock.Capture; @@ -40,20 +41,21 @@ public void connectorStatus() { int generation = 5; ConnectorTaskId taskId = new ConnectorTaskId(connector, 0); - StatusBackingStore store = strictMock(StatusBackingStore.class); + ConfigBackingStore configStore = strictMock(ConfigBackingStore.class); + StatusBackingStore statusStore = strictMock(StatusBackingStore.class); AbstractHerder herder = partialMockBuilder(AbstractHerder.class) - .withConstructor(Worker.class, StatusBackingStore.class, String.class) - .withArgs(worker, store, workerId) + .withConstructor(Worker.class, String.class, StatusBackingStore.class, ConfigBackingStore.class) + .withArgs(worker, workerId, statusStore, configStore) .addMockedMethod("generation") .createMock(); EasyMock.expect(herder.generation()).andStubReturn(generation); - EasyMock.expect(store.get(connector)) + EasyMock.expect(statusStore.get(connector)) .andReturn(new ConnectorStatus(connector, AbstractStatus.State.RUNNING, workerId, generation)); - EasyMock.expect(store.getAll(connector)) + EasyMock.expect(statusStore.getAll(connector)) .andReturn(Collections.singletonList( new TaskStatus(taskId, AbstractStatus.State.UNASSIGNED, workerId, generation))); @@ -81,21 +83,22 @@ public void taskStatus() { ConnectorTaskId taskId = new ConnectorTaskId("connector", 0); String workerId = "workerId"; - StatusBackingStore store = strictMock(StatusBackingStore.class); + ConfigBackingStore configStore = strictMock(ConfigBackingStore.class); + StatusBackingStore statusStore = strictMock(StatusBackingStore.class); AbstractHerder herder = partialMockBuilder(AbstractHerder.class) - .withConstructor(Worker.class, StatusBackingStore.class, String.class) - .withArgs(worker, store, workerId) + .withConstructor(Worker.class, String.class, StatusBackingStore.class, ConfigBackingStore.class) + .withArgs(worker, workerId, statusStore, configStore) .addMockedMethod("generation") .createMock(); EasyMock.expect(herder.generation()).andStubReturn(5); final Capture statusCapture = EasyMock.newCapture(); - store.putSafe(EasyMock.capture(statusCapture)); + statusStore.putSafe(EasyMock.capture(statusCapture)); EasyMock.expectLastCall(); - EasyMock.expect(store.get(taskId)).andAnswer(new IAnswer() { + EasyMock.expect(statusStore.get(taskId)).andAnswer(new IAnswer() { @Override public TaskStatus answer() throws Throwable { return statusCapture.getValue(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConnectorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConnectorTest.java new file mode 100644 index 0000000000000..a5f740908334a --- /dev/null +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerConnectorTest.java @@ -0,0 +1,336 @@ +/** + * 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 org.apache.kafka.connect.runtime; + +import org.apache.kafka.connect.connector.Connector; +import org.apache.kafka.connect.connector.ConnectorContext; +import org.easymock.EasyMock; +import org.easymock.EasyMockRunner; +import org.easymock.EasyMockSupport; +import org.easymock.Mock; +import org.junit.Test; +import org.junit.runner.RunWith; + +import java.util.HashMap; +import java.util.Map; + +import static org.easymock.EasyMock.expectLastCall; + +@RunWith(EasyMockRunner.class) +public class WorkerConnectorTest extends EasyMockSupport { + + public static final String CONNECTOR = "connector"; + public static final Map CONFIG = new HashMap<>(); + static { + CONFIG.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, TestConnector.class.getName()); + CONFIG.put(ConnectorConfig.NAME_CONFIG, CONNECTOR); + } + public static final ConnectorConfig CONNECTOR_CONFIG = new ConnectorConfig(CONFIG); + + @Mock Connector connector; + @Mock ConnectorContext ctx; + @Mock ConnectorStatus.Listener listener; + + @Test + public void testInitializeFailure() { + RuntimeException exception = new RuntimeException(); + + connector.initialize(EasyMock.notNull(ConnectorContext.class)); + expectLastCall().andThrow(exception); + + listener.onFailure(CONNECTOR, exception); + expectLastCall(); + + listener.onShutdown(CONNECTOR); + expectLastCall(); + + replayAll(); + + WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, ctx, listener); + + workerConnector.initialize(CONNECTOR_CONFIG); + workerConnector.shutdown(); + + verifyAll(); + } + + @Test + public void testFailureIsFinalState() { + RuntimeException exception = new RuntimeException(); + + connector.initialize(EasyMock.notNull(ConnectorContext.class)); + expectLastCall().andThrow(exception); + + listener.onFailure(CONNECTOR, exception); + expectLastCall(); + + // expect no call to onStartup() after failure + + listener.onShutdown(CONNECTOR); + expectLastCall(); + + replayAll(); + + WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, ctx, listener); + + workerConnector.initialize(CONNECTOR_CONFIG); + workerConnector.transitionTo(TargetState.STARTED); + workerConnector.shutdown(); + + verifyAll(); + } + + @Test + public void testStartupAndShutdown() { + connector.initialize(EasyMock.notNull(ConnectorContext.class)); + expectLastCall(); + + connector.start(CONFIG); + expectLastCall(); + + listener.onStartup(CONNECTOR); + expectLastCall(); + + connector.stop(); + expectLastCall(); + + listener.onShutdown(CONNECTOR); + expectLastCall(); + + replayAll(); + + WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, ctx, listener); + + workerConnector.initialize(CONNECTOR_CONFIG); + workerConnector.transitionTo(TargetState.STARTED); + workerConnector.shutdown(); + + verifyAll(); + } + + @Test + public void testStartupAndPause() { + connector.initialize(EasyMock.notNull(ConnectorContext.class)); + expectLastCall(); + + connector.start(CONFIG); + expectLastCall(); + + listener.onStartup(CONNECTOR); + expectLastCall(); + + connector.stop(); + expectLastCall(); + + listener.onPause(CONNECTOR); + expectLastCall(); + + listener.onShutdown(CONNECTOR); + expectLastCall(); + + replayAll(); + + WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, ctx, listener); + + workerConnector.initialize(CONNECTOR_CONFIG); + workerConnector.transitionTo(TargetState.STARTED); + workerConnector.transitionTo(TargetState.PAUSED); + workerConnector.shutdown(); + + verifyAll(); + } + + @Test + public void testOnResume() { + connector.initialize(EasyMock.notNull(ConnectorContext.class)); + expectLastCall(); + + listener.onPause(CONNECTOR); + expectLastCall(); + + connector.start(CONFIG); + expectLastCall(); + + listener.onResume(CONNECTOR); + expectLastCall(); + + connector.stop(); + expectLastCall(); + + listener.onShutdown(CONNECTOR); + expectLastCall(); + + replayAll(); + + WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, ctx, listener); + + workerConnector.initialize(CONNECTOR_CONFIG); + workerConnector.transitionTo(TargetState.PAUSED); + workerConnector.transitionTo(TargetState.STARTED); + workerConnector.shutdown(); + + verifyAll(); + } + + @Test + public void testStartupPaused() { + connector.initialize(EasyMock.notNull(ConnectorContext.class)); + expectLastCall(); + + // connector never gets started + + listener.onPause(CONNECTOR); + expectLastCall(); + + listener.onShutdown(CONNECTOR); + expectLastCall(); + + replayAll(); + + WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, ctx, listener); + + workerConnector.initialize(CONNECTOR_CONFIG); + workerConnector.transitionTo(TargetState.PAUSED); + workerConnector.shutdown(); + + verifyAll(); + } + + @Test + public void testStartupFailure() { + RuntimeException exception = new RuntimeException(); + + connector.initialize(EasyMock.notNull(ConnectorContext.class)); + expectLastCall(); + + connector.start(CONFIG); + expectLastCall().andThrow(exception); + + listener.onFailure(CONNECTOR, exception); + expectLastCall(); + + listener.onShutdown(CONNECTOR); + expectLastCall(); + + replayAll(); + + WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, ctx, listener); + + workerConnector.initialize(CONNECTOR_CONFIG); + workerConnector.transitionTo(TargetState.STARTED); + workerConnector.shutdown(); + + verifyAll(); + } + + @Test + public void testShutdownFailure() { + RuntimeException exception = new RuntimeException(); + + connector.initialize(EasyMock.notNull(ConnectorContext.class)); + expectLastCall(); + + connector.start(CONFIG); + expectLastCall(); + + listener.onStartup(CONNECTOR); + expectLastCall(); + + connector.stop(); + expectLastCall().andThrow(exception); + + listener.onShutdown(CONNECTOR); + expectLastCall(); + + replayAll(); + + WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, ctx, listener); + + workerConnector.initialize(CONNECTOR_CONFIG); + workerConnector.transitionTo(TargetState.STARTED); + workerConnector.shutdown(); + + verifyAll(); + } + + @Test + public void testTransitionStartedToStarted() { + connector.initialize(EasyMock.notNull(ConnectorContext.class)); + expectLastCall(); + + connector.start(CONFIG); + expectLastCall(); + + // expect only one call to onStartup() + listener.onStartup(CONNECTOR); + expectLastCall(); + + connector.stop(); + expectLastCall(); + + listener.onShutdown(CONNECTOR); + expectLastCall(); + + replayAll(); + + WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, ctx, listener); + + workerConnector.initialize(CONNECTOR_CONFIG); + workerConnector.transitionTo(TargetState.STARTED); + workerConnector.transitionTo(TargetState.STARTED); + workerConnector.shutdown(); + + verifyAll(); + } + + @Test + public void testTransitionPausedToPaused() { + connector.initialize(EasyMock.notNull(ConnectorContext.class)); + expectLastCall(); + + connector.start(CONFIG); + expectLastCall(); + + listener.onStartup(CONNECTOR); + expectLastCall(); + + connector.stop(); + expectLastCall(); + + listener.onPause(CONNECTOR); + expectLastCall(); + + listener.onShutdown(CONNECTOR); + expectLastCall(); + + replayAll(); + + WorkerConnector workerConnector = new WorkerConnector(CONNECTOR, connector, ctx, listener); + + workerConnector.initialize(CONNECTOR_CONFIG); + workerConnector.transitionTo(TargetState.STARTED); + workerConnector.transitionTo(TargetState.PAUSED); + workerConnector.transitionTo(TargetState.PAUSED); + workerConnector.shutdown(); + + verifyAll(); + } + + private static abstract class TestConnector extends Connector { + } + +} diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java index 7bc83de2deaf8..835e30f2e5994 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskTest.java @@ -23,6 +23,7 @@ import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.common.utils.Time; import org.apache.kafka.connect.data.Schema; @@ -49,14 +50,15 @@ import org.powermock.modules.junit4.PowerMockRunner; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collection; import java.util.Collections; import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; +import static java.util.Arrays.asList; import static java.util.Collections.singleton; import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; @@ -84,10 +86,12 @@ public class WorkerSinkTaskTest { private static final Map TASK_PROPS = new HashMap<>(); static { TASK_PROPS.put(SinkConnector.TOPICS_CONFIG, TOPIC); + TASK_PROPS.put(TaskConfig.TASK_CLASS_CONFIG, TestSinkTask.class.getName()); } - + private static final TaskConfig TASK_CONFIG = new TaskConfig(TASK_PROPS); private ConnectorTaskId taskId = new ConnectorTaskId("job", 0); + private TargetState initialState = TargetState.STARTED; private Time time; private WorkerSinkTask workerTask; @Mock @@ -120,11 +124,90 @@ public void setUp() { workerConfig = new StandaloneConfig(workerProps); workerTask = PowerMock.createPartialMock( WorkerSinkTask.class, new String[]{"createConsumer"}, - taskId, sinkTask, statusListener, workerConfig, keyConverter, valueConverter, time); + taskId, sinkTask, statusListener, initialState, workerConfig, keyConverter, valueConverter, time); recordsReturned = 0; } + @Test + public void testStartPaused() throws Exception { + workerTask = PowerMock.createPartialMock( + WorkerSinkTask.class, new String[]{"createConsumer"}, + taskId, sinkTask, statusListener, TargetState.PAUSED, workerConfig, keyConverter, valueConverter, time); + + expectInitializeTask(); + expectPollInitialAssignment(); + + Set partitions = new HashSet<>(asList(TOPIC_PARTITION, TOPIC_PARTITION2)); + EasyMock.expect(consumer.assignment()).andReturn(partitions); + consumer.pause(partitions); + PowerMock.expectLastCall(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + workerTask.poll(Long.MAX_VALUE); + + PowerMock.verifyAll(); + } + + @Test + public void testPause() throws Exception { + expectInitializeTask(); + expectPollInitialAssignment(); + + expectConsumerPoll(1); + expectConvertMessages(1); + sinkTask.put(EasyMock.>anyObject()); + EasyMock.expectLastCall(); + + Set partitions = new HashSet<>(asList(TOPIC_PARTITION, TOPIC_PARTITION2)); + + // Pause + statusListener.onPause(taskId); + EasyMock.expectLastCall(); + expectConsumerWakeup(); + EasyMock.expect(consumer.assignment()).andReturn(partitions); + consumer.pause(partitions); + PowerMock.expectLastCall(); + + // No records returned + expectConsumerPoll(0); + sinkTask.put(Collections.emptyList()); + EasyMock.expectLastCall(); + + // And unpause + statusListener.onResume(taskId); + EasyMock.expectLastCall(); + expectConsumerWakeup(); + EasyMock.expect(consumer.assignment()).andReturn(new HashSet<>(asList(TOPIC_PARTITION, TOPIC_PARTITION2))); + consumer.resume(singleton(TOPIC_PARTITION)); + PowerMock.expectLastCall(); + consumer.resume(singleton(TOPIC_PARTITION2)); + PowerMock.expectLastCall(); + + expectConsumerPoll(1); + expectConvertMessages(1); + sinkTask.put(EasyMock.>anyObject()); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + workerTask.initializeAndStart(); + workerTask.poll(Long.MAX_VALUE); // initial assignment + workerTask.poll(Long.MAX_VALUE); // fetch some data + workerTask.transitionTo(TargetState.PAUSED); + workerTask.poll(Long.MAX_VALUE); // wakeup + workerTask.poll(Long.MAX_VALUE); // now paused + workerTask.transitionTo(TargetState.STARTED); + workerTask.poll(Long.MAX_VALUE); // wakeup + workerTask.poll(Long.MAX_VALUE); // now unpaused + + PowerMock.verifyAll(); + } + @Test public void testPollRedelivery() throws Exception { expectInitializeTask(); @@ -137,7 +220,7 @@ public void testPollRedelivery() throws Exception { sinkTask.put(EasyMock.capture(records)); EasyMock.expectLastCall().andThrow(new RetriableException("retry")); // Pause - HashSet partitions = new HashSet<>(Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2)); + HashSet partitions = new HashSet<>(asList(TOPIC_PARTITION, TOPIC_PARTITION2)); EasyMock.expect(consumer.assignment()).andReturn(partitions); consumer.pause(partitions); PowerMock.expectLastCall(); @@ -155,7 +238,7 @@ public void testPollRedelivery() throws Exception { PowerMock.replayAll(); - workerTask.initialize(TASK_PROPS); + workerTask.initialize(TASK_CONFIG); workerTask.initializeAndStart(); workerTask.poll(Long.MAX_VALUE); workerTask.poll(Long.MAX_VALUE); @@ -174,7 +257,7 @@ public void testErrorInRebalancePartitionRevocation() throws Exception { PowerMock.replayAll(); - workerTask.initialize(TASK_PROPS); + workerTask.initialize(TASK_CONFIG); workerTask.initializeAndStart(); workerTask.poll(Long.MAX_VALUE); try { @@ -197,7 +280,7 @@ public void testErrorInRebalancePartitionAssignment() throws Exception { PowerMock.replayAll(); - workerTask.initialize(TASK_PROPS); + workerTask.initialize(TASK_CONFIG); workerTask.initializeAndStart(); workerTask.poll(Long.MAX_VALUE); try { @@ -213,7 +296,7 @@ public void testErrorInRebalancePartitionAssignment() throws Exception { private void expectInitializeTask() throws Exception { PowerMock.expectPrivate(workerTask, "createConsumer").andReturn(consumer); - consumer.subscribe(EasyMock.eq(Arrays.asList(TOPIC)), EasyMock.capture(rebalanceListener)); + consumer.subscribe(EasyMock.eq(asList(TOPIC)), EasyMock.capture(rebalanceListener)); PowerMock.expectLastCall(); sinkTask.initialize(EasyMock.capture(sinkTaskContext)); @@ -223,7 +306,7 @@ private void expectInitializeTask() throws Exception { } private void expectRebalanceRevocationError(RuntimeException e) { - final List partitions = Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2); + final List partitions = asList(TOPIC_PARTITION, TOPIC_PARTITION2); sinkTask.close(new HashSet<>(partitions)); EasyMock.expectLastCall().andThrow(e); @@ -239,7 +322,7 @@ public ConsumerRecords answer() throws Throwable { } private void expectRebalanceAssignmentError(RuntimeException e) { - final List partitions = Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2); + final List partitions = asList(TOPIC_PARTITION, TOPIC_PARTITION2); sinkTask.close(new HashSet<>(partitions)); EasyMock.expectLastCall(); @@ -268,7 +351,7 @@ public ConsumerRecords answer() throws Throwable { } private void expectPollInitialAssignment() { - final List partitions = Arrays.asList(TOPIC_PARTITION, TOPIC_PARTITION2); + final List partitions = asList(TOPIC_PARTITION, TOPIC_PARTITION2); sinkTask.open(partitions); EasyMock.expectLastCall(); @@ -287,6 +370,12 @@ public ConsumerRecords answer() throws Throwable { EasyMock.expectLastCall(); } + private void expectConsumerWakeup() { + consumer.wakeup(); + EasyMock.expectLastCall(); + EasyMock.expect(consumer.poll(EasyMock.anyLong())).andThrow(new WakeupException()); + } + private void expectConsumerPoll(final int numMessages) { EasyMock.expect(consumer.poll(EasyMock.anyLong())).andAnswer( new IAnswer>() { @@ -309,4 +398,8 @@ private void expectConvertMessages(final int numMessages) { EasyMock.expect(keyConverter.toConnectData(TOPIC, RAW_KEY)).andReturn(new SchemaAndValue(KEY_SCHEMA, KEY)).times(numMessages); EasyMock.expect(valueConverter.toConnectData(TOPIC, RAW_VALUE)).andReturn(new SchemaAndValue(VALUE_SCHEMA, VALUE)).times(numMessages); } + + private abstract static class TestSinkTask extends SinkTask { + } + } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java index 25f0bf4e66da7..25dbff575cbba 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSinkTaskThreadedTest.java @@ -89,9 +89,12 @@ public class WorkerSinkTaskThreadedTest extends ThreadedTest { private static final Map TASK_PROPS = new HashMap<>(); static { TASK_PROPS.put(SinkConnector.TOPICS_CONFIG, TOPIC); + TASK_PROPS.put(TaskConfig.TASK_CLASS_CONFIG, TestSinkTask.class.getName()); } + private static final TaskConfig TASK_CONFIG = new TaskConfig(TASK_PROPS); private ConnectorTaskId taskId = new ConnectorTaskId("job", 0); + private TargetState initialState = TargetState.STARTED; private Time time; @Mock private SinkTask sinkTask; private Capture sinkTaskContext = EasyMock.newCapture(); @@ -105,6 +108,7 @@ public class WorkerSinkTaskThreadedTest extends ThreadedTest { private long recordsReturned; + @SuppressWarnings("unchecked") @Override public void setup() { @@ -121,7 +125,7 @@ public void setup() { workerConfig = new StandaloneConfig(workerProps); workerTask = PowerMock.createPartialMock( WorkerSinkTask.class, new String[]{"createConsumer"}, - taskId, sinkTask, statusListener, workerConfig, keyConverter, valueConverter, time); + taskId, sinkTask, statusListener, initialState, workerConfig, keyConverter, valueConverter, time); recordsReturned = 0; } @@ -136,7 +140,7 @@ public void testPollsInBackground() throws Exception { PowerMock.replayAll(); - workerTask.initialize(TASK_PROPS); + workerTask.initialize(TASK_CONFIG); workerTask.initializeAndStart(); // First iteration initializes partition assignment @@ -147,7 +151,6 @@ public void testPollsInBackground() throws Exception { workerTask.iteration(); } workerTask.stop(); - workerTask.awaitStop(Long.MAX_VALUE); workerTask.close(); // Verify contents match expected values, i.e. that they were translated properly. With max @@ -180,7 +183,7 @@ public void testCommit() throws Exception { PowerMock.replayAll(); - workerTask.initialize(TASK_PROPS); + workerTask.initialize(TASK_CONFIG); workerTask.initializeAndStart(); // Initialize partition assignment @@ -193,7 +196,6 @@ public void testCommit() throws Exception { // Commit finishes synchronously for testing so we can check this immediately assertEquals(0, workerTask.commitFailures()); workerTask.stop(); - workerTask.awaitStop(Long.MAX_VALUE); workerTask.close(); assertEquals(2, capturedRecords.getValues().size()); @@ -220,7 +222,7 @@ public void testCommitTaskFlushFailure() throws Exception { PowerMock.replayAll(); - workerTask.initialize(TASK_PROPS); + workerTask.initialize(TASK_CONFIG); workerTask.initializeAndStart(); // Initialize partition assignment @@ -233,7 +235,6 @@ public void testCommitTaskFlushFailure() throws Exception { assertEquals(1, workerTask.commitFailures()); assertEquals(false, Whitebox.getInternalState(workerTask, "committing")); workerTask.stop(); - workerTask.awaitStop(Long.MAX_VALUE); workerTask.close(); PowerMock.verifyAll(); @@ -259,7 +260,7 @@ public void testCommitTaskSuccessAndFlushFailure() throws Exception { PowerMock.replayAll(); - workerTask.initialize(TASK_PROPS); + workerTask.initialize(TASK_CONFIG); workerTask.initializeAndStart(); // Initialize partition assignment @@ -274,7 +275,6 @@ public void testCommitTaskSuccessAndFlushFailure() throws Exception { assertEquals(1, workerTask.commitFailures()); assertEquals(false, Whitebox.getInternalState(workerTask, "committing")); workerTask.stop(); - workerTask.awaitStop(Long.MAX_VALUE); workerTask.close(); PowerMock.verifyAll(); @@ -292,7 +292,7 @@ public void testCommitConsumerFailure() throws Exception { PowerMock.replayAll(); - workerTask.initialize(TASK_PROPS); + workerTask.initialize(TASK_CONFIG); workerTask.initializeAndStart(); // Initialize partition assignment @@ -306,7 +306,6 @@ public void testCommitConsumerFailure() throws Exception { assertEquals(1, workerTask.commitFailures()); assertEquals(false, Whitebox.getInternalState(workerTask, "committing")); workerTask.stop(); - workerTask.awaitStop(Long.MAX_VALUE); workerTask.close(); PowerMock.verifyAll(); @@ -325,7 +324,7 @@ public void testCommitTimeout() throws Exception { PowerMock.replayAll(); - workerTask.initialize(TASK_PROPS); + workerTask.initialize(TASK_CONFIG); workerTask.initializeAndStart(); // Initialize partition assignment @@ -342,7 +341,6 @@ public void testCommitTimeout() throws Exception { assertEquals(1, workerTask.commitFailures()); assertEquals(false, Whitebox.getInternalState(workerTask, "committing")); workerTask.stop(); - workerTask.awaitStop(Long.MAX_VALUE); workerTask.close(); PowerMock.verifyAll(); @@ -405,13 +403,12 @@ public Object answer() throws Throwable { PowerMock.replayAll(); - workerTask.initialize(TASK_PROPS); + workerTask.initialize(TASK_CONFIG); workerTask.initializeAndStart(); workerTask.iteration(); workerTask.iteration(); workerTask.iteration(); workerTask.stop(); - workerTask.awaitStop(Long.MAX_VALUE); workerTask.close(); PowerMock.verifyAll(); @@ -449,13 +446,12 @@ public Object answer() throws Throwable { expectStopTask(3); PowerMock.replayAll(); - workerTask.initialize(TASK_PROPS); + workerTask.initialize(TASK_CONFIG); workerTask.initializeAndStart(); workerTask.iteration(); workerTask.iteration(); workerTask.iteration(); workerTask.stop(); - workerTask.awaitStop(Long.MAX_VALUE); workerTask.close(); PowerMock.verifyAll(); @@ -597,4 +593,7 @@ public Object answer() throws Throwable { return capturedCallback; } + private static abstract class TestSinkTask extends SinkTask { + } + } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java index 3dd07a699fe9c..0d805dae47ab1 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerSourceTaskTest.java @@ -58,6 +58,7 @@ import java.util.concurrent.Future; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; @@ -94,7 +95,12 @@ public class WorkerSourceTaskTest extends ThreadedTest { private Capture producerCallbacks; - private static final Map EMPTY_TASK_PROPS = Collections.emptyMap(); + private static final Map TASK_PROPS = new HashMap<>(); + static { + TASK_PROPS.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); + } + private static final TaskConfig TASK_CONFIG = new TaskConfig(TASK_PROPS); + private static final List RECORDS = Arrays.asList( new SourceRecord(PARTITION, OFFSET, "topic", null, KEY_SCHEMA, KEY, RECORD_SCHEMA, RECORD) ); @@ -115,8 +121,94 @@ public void setup() { } private void createWorkerTask() { - workerTask = new WorkerSourceTask(taskId, sourceTask, statusListener, keyConverter, valueConverter, producer, - offsetReader, offsetWriter, config, new SystemTime()); + createWorkerTask(TargetState.STARTED); + } + + private void createWorkerTask(TargetState initialState) { + workerTask = new WorkerSourceTask(taskId, sourceTask, statusListener, initialState, keyConverter, + valueConverter, producer, offsetReader, offsetWriter, config, new SystemTime()); + } + + @Test + public void testStartPaused() throws Exception { + final CountDownLatch startupLatch = new CountDownLatch(1); + + createWorkerTask(TargetState.PAUSED); + + sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); + EasyMock.expectLastCall(); + sourceTask.start(TASK_PROPS); + EasyMock.expectLastCall().andAnswer(new IAnswer() { + @Override + public Void answer() throws Throwable { + startupLatch.countDown(); + return null; + } + }); + statusListener.onPause(taskId); + EasyMock.expectLastCall(); + + // we shouldn't get any calls to poll() + + sourceTask.stop(); + EasyMock.expectLastCall(); + expectOffsetFlush(true); + + statusListener.onShutdown(taskId); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + executor.submit(workerTask); + assertTrue(startupLatch.await(5, TimeUnit.SECONDS)); + workerTask.stop(); + assertTrue(workerTask.awaitStop(1000)); + + PowerMock.verifyAll(); + } + + @Test + public void testPause() throws Exception { + createWorkerTask(); + + sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); + EasyMock.expectLastCall(); + sourceTask.start(TASK_PROPS); + EasyMock.expectLastCall(); + statusListener.onStartup(taskId); + EasyMock.expectLastCall(); + + AtomicInteger count = new AtomicInteger(0); + CountDownLatch pollLatch = expectPolls(10, count); + // In this test, we don't flush, so nothing goes any further than the offset writer + + statusListener.onPause(taskId); + EasyMock.expectLastCall(); + + sourceTask.stop(); + EasyMock.expectLastCall(); + expectOffsetFlush(true); + + statusListener.onShutdown(taskId); + EasyMock.expectLastCall(); + + PowerMock.replayAll(); + + workerTask.initialize(TASK_CONFIG); + executor.submit(workerTask); + awaitPolls(pollLatch); + + workerTask.transitionTo(TargetState.PAUSED); + + int priorCount = count.get(); + Thread.sleep(100); + assertEquals(priorCount, count.get()); + + workerTask.stop(); + assertTrue(workerTask.awaitStop(1000)); + + PowerMock.verifyAll(); } @Test @@ -125,7 +217,7 @@ public void testPollsInBackground() throws Exception { sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); EasyMock.expectLastCall(); - sourceTask.start(EMPTY_TASK_PROPS); + sourceTask.start(TASK_PROPS); EasyMock.expectLastCall(); statusListener.onStartup(taskId); EasyMock.expectLastCall(); @@ -142,7 +234,7 @@ public void testPollsInBackground() throws Exception { PowerMock.replayAll(); - workerTask.initialize(EMPTY_TASK_PROPS); + workerTask.initialize(TASK_CONFIG); executor.submit(workerTask); awaitPolls(pollLatch); workerTask.stop(); @@ -157,7 +249,7 @@ public void testFailureInPoll() throws Exception { sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); EasyMock.expectLastCall(); - sourceTask.start(EMPTY_TASK_PROPS); + sourceTask.start(TASK_PROPS); EasyMock.expectLastCall(); statusListener.onStartup(taskId); EasyMock.expectLastCall(); @@ -175,7 +267,7 @@ public void testFailureInPoll() throws Exception { PowerMock.replayAll(); - workerTask.initialize(EMPTY_TASK_PROPS); + workerTask.initialize(TASK_CONFIG); executor.submit(workerTask); awaitPolls(pollLatch); workerTask.stop(); @@ -191,7 +283,7 @@ public void testCommit() throws Exception { sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); EasyMock.expectLastCall(); - sourceTask.start(EMPTY_TASK_PROPS); + sourceTask.start(TASK_PROPS); EasyMock.expectLastCall(); statusListener.onStartup(taskId); EasyMock.expectLastCall(); @@ -210,7 +302,7 @@ public void testCommit() throws Exception { PowerMock.replayAll(); - workerTask.initialize(EMPTY_TASK_PROPS); + workerTask.initialize(TASK_CONFIG); executor.submit(workerTask); awaitPolls(pollLatch); assertTrue(workerTask.commitOffsets()); @@ -227,7 +319,7 @@ public void testCommitFailure() throws Exception { sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); EasyMock.expectLastCall(); - sourceTask.start(EMPTY_TASK_PROPS); + sourceTask.start(TASK_PROPS); EasyMock.expectLastCall(); statusListener.onStartup(taskId); EasyMock.expectLastCall(); @@ -245,7 +337,7 @@ public void testCommitFailure() throws Exception { PowerMock.replayAll(); - workerTask.initialize(EMPTY_TASK_PROPS); + workerTask.initialize(TASK_CONFIG); executor.submit(workerTask); awaitPolls(pollLatch); assertTrue(workerTask.commitOffsets()); @@ -317,7 +409,7 @@ public void testSlowTaskStart() throws Exception { sourceTask.initialize(EasyMock.anyObject(SourceTaskContext.class)); EasyMock.expectLastCall(); - sourceTask.start(EMPTY_TASK_PROPS); + sourceTask.start(TASK_PROPS); EasyMock.expectLastCall(); statusListener.onStartup(taskId); @@ -336,7 +428,7 @@ public Object answer() throws Throwable { PowerMock.replayAll(); - workerTask.initialize(EMPTY_TASK_PROPS); + workerTask.initialize(TASK_CONFIG); executor.submit(workerTask); // Stopping immediately while the other thread has work to do should result in no polling, no offset commits, // exiting the work thread immediately, and the stop() method will be invoked in the background thread since it @@ -348,8 +440,8 @@ public Object answer() throws Throwable { PowerMock.verifyAll(); } - private CountDownLatch expectPolls(int count) throws InterruptedException { - final CountDownLatch latch = new CountDownLatch(count); + private CountDownLatch expectPolls(int minimum, final AtomicInteger count) throws InterruptedException { + final CountDownLatch latch = new CountDownLatch(minimum); // Note that we stub these to allow any number of calls because the thread will continue to // run. The count passed in + latch returned just makes sure we get *at least* that number of // calls @@ -357,6 +449,7 @@ private CountDownLatch expectPolls(int count) throws InterruptedException { .andStubAnswer(new IAnswer>() { @Override public List answer() throws Throwable { + count.incrementAndGet(); latch.countDown(); return RECORDS; } @@ -366,6 +459,10 @@ public List answer() throws Throwable { return latch; } + private CountDownLatch expectPolls(int count) throws InterruptedException { + return expectPolls(count, new AtomicInteger()); + } + @SuppressWarnings("unchecked") private void expectSendRecordSyncFailure(Throwable error) throws InterruptedException { expectConvertKeyValue(false); @@ -446,8 +543,8 @@ private void expectConvertKeyValue(boolean anyTimes) { convertValueExpect.andReturn(SERIALIZED_RECORD); } - private void awaitPolls(CountDownLatch latch) throws InterruptedException { - latch.await(1000, TimeUnit.MILLISECONDS); + private boolean awaitPolls(CountDownLatch latch) throws InterruptedException { + return latch.await(1000, TimeUnit.MILLISECONDS); } @SuppressWarnings("unchecked") @@ -469,4 +566,7 @@ private void expectOffsetFlush(boolean succeed) throws Exception { } } + private abstract static class TestSourceTask extends SourceTask { + } + } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java index 20e3fe2809369..36803db2bca4c 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTaskTest.java @@ -16,12 +16,13 @@ **/ package org.apache.kafka.connect.runtime; +import org.apache.kafka.connect.sink.SinkTask; import org.apache.kafka.connect.util.ConnectorTaskId; import org.easymock.EasyMock; import org.easymock.IAnswer; import org.junit.Test; -import java.util.Collections; +import java.util.HashMap; import java.util.Map; import java.util.concurrent.CountDownLatch; @@ -32,7 +33,11 @@ public class WorkerTaskTest { - private static final Map EMPTY_TASK_PROPS = Collections.emptyMap(); + private static final Map TASK_PROPS = new HashMap<>(); + static { + TASK_PROPS.put(TaskConfig.TASK_CLASS_CONFIG, TestSinkTask.class.getName()); + } + private static final TaskConfig TASK_CONFIG = new TaskConfig(TASK_PROPS); @Test public void standardStartup() { @@ -41,14 +46,14 @@ public void standardStartup() { TaskStatus.Listener statusListener = EasyMock.createMock(TaskStatus.Listener.class); WorkerTask workerTask = partialMockBuilder(WorkerTask.class) - .withConstructor(ConnectorTaskId.class, TaskStatus.Listener.class) - .withArgs(taskId, statusListener) + .withConstructor(ConnectorTaskId.class, TaskStatus.Listener.class, TargetState.class) + .withArgs(taskId, statusListener, TargetState.STARTED) .addMockedMethod("initialize") .addMockedMethod("execute") .addMockedMethod("close") .createStrictMock(); - workerTask.initialize(EMPTY_TASK_PROPS); + workerTask.initialize(TASK_CONFIG); expectLastCall(); workerTask.execute(); @@ -65,7 +70,7 @@ public void standardStartup() { replay(workerTask); - workerTask.initialize(EMPTY_TASK_PROPS); + workerTask.initialize(TASK_CONFIG); workerTask.run(); workerTask.stop(); workerTask.awaitStop(1000L); @@ -80,14 +85,14 @@ public void stopBeforeStarting() { TaskStatus.Listener statusListener = EasyMock.createMock(TaskStatus.Listener.class); WorkerTask workerTask = partialMockBuilder(WorkerTask.class) - .withConstructor(ConnectorTaskId.class, TaskStatus.Listener.class) - .withArgs(taskId, statusListener) + .withConstructor(ConnectorTaskId.class, TaskStatus.Listener.class, TargetState.class) + .withArgs(taskId, statusListener, TargetState.STARTED) .addMockedMethod("initialize") .addMockedMethod("execute") .addMockedMethod("close") .createStrictMock(); - workerTask.initialize(EMPTY_TASK_PROPS); + workerTask.initialize(TASK_CONFIG); EasyMock.expectLastCall(); workerTask.close(); @@ -95,7 +100,7 @@ public void stopBeforeStarting() { replay(workerTask); - workerTask.initialize(EMPTY_TASK_PROPS); + workerTask.initialize(TASK_CONFIG); workerTask.stop(); workerTask.awaitStop(1000L); @@ -112,8 +117,8 @@ public void cancelBeforeStopping() throws Exception { TaskStatus.Listener statusListener = EasyMock.createMock(TaskStatus.Listener.class); WorkerTask workerTask = partialMockBuilder(WorkerTask.class) - .withConstructor(ConnectorTaskId.class, TaskStatus.Listener.class) - .withArgs(taskId, statusListener) + .withConstructor(ConnectorTaskId.class, TaskStatus.Listener.class, TargetState.class) + .withArgs(taskId, statusListener, TargetState.STARTED) .addMockedMethod("initialize") .addMockedMethod("execute") .addMockedMethod("close") @@ -130,7 +135,7 @@ public void run() { } }; - workerTask.initialize(EMPTY_TASK_PROPS); + workerTask.initialize(TASK_CONFIG); EasyMock.expectLastCall(); workerTask.execute(); @@ -152,7 +157,7 @@ public Void answer() throws Throwable { replay(workerTask); - workerTask.initialize(EMPTY_TASK_PROPS); + workerTask.initialize(TASK_CONFIG); workerTask.run(); workerTask.stop(); @@ -163,4 +168,7 @@ public Void answer() throws Throwable { verify(workerTask); } + private static abstract class TestSinkTask extends SinkTask { + } + } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java index 557d7891610ab..2004c993a2c6e 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/WorkerTest.java @@ -102,7 +102,7 @@ public void testStartAndStopConnector() throws Exception { props.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_ID); props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, WorkerTestConnector.class.getName()); - connector.initialize(ctx); + connector.initialize(EasyMock.anyObject(ConnectorContext.class)); EasyMock.expectLastCall(); connector.start(props); EasyMock.expectLastCall(); @@ -126,10 +126,10 @@ public void testStartAndStopConnector() throws Exception { ConnectorConfig config = new ConnectorConfig(props); assertEquals(Collections.emptySet(), worker.connectorNames()); - worker.startConnector(config, ctx, connectorStatusListener); + worker.startConnector(config, ctx, connectorStatusListener, TargetState.STARTED); assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames()); try { - worker.startConnector(config, ctx, connectorStatusListener); + worker.startConnector(config, ctx, connectorStatusListener, TargetState.STARTED); fail("Should have thrown exception when trying to add connector with same name."); } catch (ConnectException e) { // expected @@ -160,7 +160,7 @@ public void testAddConnectorByAlias() throws Exception { props.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_ID); props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, "WorkerTestConnector"); - connector.initialize(ctx); + connector.initialize(EasyMock.anyObject(ConnectorContext.class)); EasyMock.expectLastCall(); connector.start(props); EasyMock.expectLastCall(); @@ -184,7 +184,7 @@ public void testAddConnectorByAlias() throws Exception { ConnectorConfig config = new ConnectorConfig(props); assertEquals(Collections.emptySet(), worker.connectorNames()); - worker.startConnector(config, ctx, connectorStatusListener); + worker.startConnector(config, ctx, connectorStatusListener, TargetState.STARTED); assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames()); worker.stopConnector(CONNECTOR_ID); @@ -213,7 +213,7 @@ public void testAddConnectorByShortAlias() throws Exception { props.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_ID); props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, "WorkerTest"); - connector.initialize(ctx); + connector.initialize(EasyMock.anyObject(ConnectorContext.class)); EasyMock.expectLastCall(); connector.start(props); EasyMock.expectLastCall(); @@ -237,7 +237,7 @@ public void testAddConnectorByShortAlias() throws Exception { ConnectorConfig config = new ConnectorConfig(props); assertEquals(Collections.emptySet(), worker.connectorNames()); - worker.startConnector(config, ctx, connectorStatusListener); + worker.startConnector(config, ctx, connectorStatusListener, TargetState.STARTED); assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames()); worker.stopConnector(CONNECTOR_ID); @@ -279,7 +279,7 @@ public void testReconfigureConnectorTasks() throws Exception { props.put(ConnectorConfig.NAME_CONFIG, CONNECTOR_ID); props.put(ConnectorConfig.CONNECTOR_CLASS_CONFIG, WorkerTestConnector.class.getName()); - connector.initialize(ctx); + connector.initialize(EasyMock.anyObject(ConnectorContext.class)); EasyMock.expectLastCall(); connector.start(props); EasyMock.expectLastCall(); @@ -309,10 +309,10 @@ public void testReconfigureConnectorTasks() throws Exception { ConnectorConfig config = new ConnectorConfig(props); assertEquals(Collections.emptySet(), worker.connectorNames()); - worker.startConnector(config, ctx, connectorStatusListener); + worker.startConnector(config, ctx, connectorStatusListener, TargetState.STARTED); assertEquals(new HashSet<>(Arrays.asList(CONNECTOR_ID)), worker.connectorNames()); try { - worker.startConnector(config, ctx, connectorStatusListener); + worker.startConnector(config, ctx, connectorStatusListener, TargetState.STARTED); fail("Should have thrown exception when trying to add connector with same name."); } catch (ConnectException e) { // expected @@ -351,6 +351,7 @@ public void testAddRemoveTask() throws Exception { WorkerSourceTask.class, EasyMock.eq(TASK_ID), EasyMock.eq(task), EasyMock.anyObject(TaskStatus.Listener.class), + EasyMock.eq(TargetState.STARTED), EasyMock.anyObject(Converter.class), EasyMock.anyObject(Converter.class), EasyMock.anyObject(KafkaProducer.class), @@ -361,7 +362,7 @@ public void testAddRemoveTask() throws Exception { .andReturn(workerTask); Map origProps = new HashMap<>(); origProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); - workerTask.initialize(origProps); + workerTask.initialize(new TaskConfig(origProps)); EasyMock.expectLastCall(); workerTask.run(); EasyMock.expectLastCall(); @@ -379,7 +380,7 @@ public void testAddRemoveTask() throws Exception { worker = new Worker(WORKER_ID, new MockTime(), config, offsetBackingStore); worker.start(); assertEquals(Collections.emptySet(), worker.taskIds()); - worker.startTask(TASK_ID, new TaskConfig(origProps), taskStatusListener); + worker.startTask(TASK_ID, new TaskConfig(origProps), taskStatusListener, TargetState.STARTED); assertEquals(new HashSet<>(Arrays.asList(TASK_ID)), worker.taskIds()); worker.stopAndAwaitTask(TASK_ID); assertEquals(Collections.emptySet(), worker.taskIds()); @@ -418,6 +419,7 @@ public void testCleanupTasksOnStop() throws Exception { WorkerSourceTask.class, EasyMock.eq(TASK_ID), EasyMock.eq(task), EasyMock.anyObject(TaskStatus.Listener.class), + EasyMock.eq(TargetState.STARTED), EasyMock.anyObject(Converter.class), EasyMock.anyObject(Converter.class), EasyMock.anyObject(KafkaProducer.class), @@ -428,7 +430,7 @@ public void testCleanupTasksOnStop() throws Exception { .andReturn(workerTask); Map origProps = new HashMap<>(); origProps.put(TaskConfig.TASK_CLASS_CONFIG, TestSourceTask.class.getName()); - workerTask.initialize(origProps); + workerTask.initialize(new TaskConfig(origProps)); EasyMock.expectLastCall(); workerTask.run(); EasyMock.expectLastCall(); @@ -447,7 +449,7 @@ public void testCleanupTasksOnStop() throws Exception { worker = new Worker(WORKER_ID, new MockTime(), config, offsetBackingStore); worker.start(); - worker.startTask(TASK_ID, new TaskConfig(origProps), taskStatusListener); + worker.startTask(TASK_ID, new TaskConfig(origProps), taskStatusListener, TargetState.STARTED); worker.stop(); PowerMock.verifyAll(); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java index b667fa89296a7..e62b66366cb24 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/DistributedHerderTest.java @@ -24,6 +24,7 @@ import org.apache.kafka.connect.errors.NotFoundException; import org.apache.kafka.connect.runtime.ConnectorConfig; import org.apache.kafka.connect.runtime.Herder; +import org.apache.kafka.connect.runtime.TargetState; import org.apache.kafka.connect.runtime.TaskConfig; import org.apache.kafka.connect.runtime.Worker; import org.apache.kafka.connect.runtime.WorkerConfig; @@ -31,12 +32,12 @@ import org.apache.kafka.connect.runtime.rest.entities.TaskInfo; import org.apache.kafka.connect.source.SourceConnector; import org.apache.kafka.connect.source.SourceTask; -import org.apache.kafka.connect.storage.KafkaConfigStorage; +import org.apache.kafka.connect.storage.ConfigBackingStore; +import org.apache.kafka.connect.storage.KafkaConfigBackingStore; import org.apache.kafka.connect.storage.StatusBackingStore; import org.apache.kafka.connect.util.Callback; import org.apache.kafka.connect.util.ConnectorTaskId; import org.apache.kafka.connect.util.FutureCallback; -import org.apache.kafka.connect.util.TestFuture; import org.easymock.Capture; import org.easymock.EasyMock; import org.easymock.IAnswer; @@ -48,7 +49,6 @@ import org.powermock.core.classloader.annotations.PowerMockIgnore; import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; -import org.powermock.reflect.Whitebox; import java.util.ArrayList; import java.util.Arrays; @@ -127,13 +127,15 @@ public class DistributedHerderTest { TASK_CONFIGS_MAP.put(TASK2, TASK_CONFIG); } private static final ClusterConfigState SNAPSHOT = new ClusterConfigState(1, Collections.singletonMap(CONN1, 3), - Collections.singletonMap(CONN1, CONN1_CONFIG), TASK_CONFIGS_MAP, Collections.emptySet()); + Collections.singletonMap(CONN1, CONN1_CONFIG), Collections.singletonMap(CONN1, TargetState.STARTED), + TASK_CONFIGS_MAP, Collections.emptySet()); private static final ClusterConfigState SNAPSHOT_UPDATED_CONN1_CONFIG = new ClusterConfigState(1, Collections.singletonMap(CONN1, 3), - Collections.singletonMap(CONN1, CONN1_CONFIG_UPDATED), TASK_CONFIGS_MAP, Collections.emptySet()); + Collections.singletonMap(CONN1, CONN1_CONFIG_UPDATED), Collections.singletonMap(CONN1, TargetState.STARTED), + TASK_CONFIGS_MAP, Collections.emptySet()); private static final String WORKER_ID = "localhost:8083"; - @Mock private KafkaConfigStorage configStorage; + @Mock private KafkaConfigBackingStore configStorage; @Mock private StatusBackingStore statusBackingStore; @Mock private WorkerGroupMember member; private MockTime time; @@ -141,8 +143,7 @@ public class DistributedHerderTest { @Mock private Worker worker; @Mock private Callback> putConnectorCallback; - private Callback connectorConfigCallback; - private Callback> taskConfigCallback; + private ConfigBackingStore.UpdateListener configUpdateListener; private WorkerRebalanceListener rebalanceListener; @Before @@ -152,24 +153,27 @@ public void setUp() throws Exception { time = new MockTime(); herder = PowerMock.createPartialMock(DistributedHerder.class, new String[]{"backoff", "updateDeletedConnectorStatus"}, - new DistributedConfig(HERDER_CONFIG), WORKER_ID, worker, statusBackingStore, configStorage, member, MEMBER_URL, time); - connectorConfigCallback = Whitebox.invokeMethod(herder, "connectorConfigCallback"); - taskConfigCallback = Whitebox.invokeMethod(herder, "taskConfigCallback"); - rebalanceListener = Whitebox.invokeMethod(herder, "rebalanceListener"); + new DistributedConfig(HERDER_CONFIG), worker, WORKER_ID, statusBackingStore, configStorage, member, MEMBER_URL, time); + + configUpdateListener = herder.new ConfigUpdateListener(); + rebalanceListener = herder.new RebalanceListener(); + PowerMock.expectPrivate(herder, "updateDeletedConnectorStatus").andVoid().anyTimes(); } @Test - public void testJoinAssignment() { + public void testJoinAssignment() throws Exception { // Join group and get assignment EasyMock.expect(member.memberId()).andStubReturn("member"); expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1)); expectPostRebalanceCatchup(SNAPSHOT); worker.startConnector(EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder)); + EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); + EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); + EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS); - worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.eq(herder)); + worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -182,16 +186,17 @@ public void testJoinAssignment() { } @Test - public void testRebalance() { + public void testRebalance() throws Exception { // Join group and get assignment EasyMock.expect(member.memberId()).andStubReturn("member"); expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1)); expectPostRebalanceCatchup(SNAPSHOT); worker.startConnector(EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder)); + EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); + EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS); - worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.eq(herder)); + worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -201,9 +206,48 @@ public void testRebalance() { // and the new assignment started worker.startConnector(EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder)); + EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); + PowerMock.expectLastCall(); + EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); + EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS); + member.poll(EasyMock.anyInt()); + PowerMock.expectLastCall(); + + PowerMock.replayAll(); + + herder.tick(); + herder.tick(); + + PowerMock.verifyAll(); + } + + @Test + public void testRebalanceFailedConnector() throws Exception { + // Join group and get assignment + EasyMock.expect(member.memberId()).andStubReturn("member"); + expectRebalance(1, Arrays.asList(CONN1), Arrays.asList(TASK1)); + expectPostRebalanceCatchup(SNAPSHOT); + worker.startConnector(EasyMock.anyObject(), EasyMock.anyObject(), + EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); + EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS); + worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); + PowerMock.expectLastCall(); + member.poll(EasyMock.anyInt()); + PowerMock.expectLastCall(); + + expectRebalance(Arrays.asList(CONN1), Arrays.asList(TASK1), ConnectProtocol.Assignment.NO_ERROR, + 1, Arrays.asList(CONN1), Arrays.asList()); + + // and the new assignment started + worker.startConnector(EasyMock.anyObject(), EasyMock.anyObject(), + EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); + PowerMock.expectLastCall(); + EasyMock.expect(worker.isRunning(CONN1)).andReturn(false); + + // worker is not running, so we should see no call to connectorTaskConfigs() + member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -231,6 +275,8 @@ public void testHaltCleansUpWorker() { PowerMock.expectLastCall(); statusBackingStore.stop(); PowerMock.expectLastCall(); + worker.stop(); + PowerMock.expectLastCall(); PowerMock.replayAll(); @@ -295,14 +341,15 @@ public void testDestroyConnector() throws Exception { expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); expectPostRebalanceCatchup(SNAPSHOT); worker.startConnector(EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder)); + EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); + EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS); // And delete the connector member.wakeup(); PowerMock.expectLastCall(); - configStorage.putConnectorConfig(CONN1, null); + configStorage.removeConnectorConfig(CONN1); PowerMock.expectLastCall(); putConnectorCallback.onCompletion(null, new Herder.Created(false, null)); PowerMock.expectLastCall(); @@ -329,8 +376,9 @@ public void testRestartConnector() throws Exception { member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); worker.startConnector(EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder)); + EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); + EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); // now handle the connector restart member.wakeup(); @@ -345,8 +393,9 @@ public void testRestartConnector() throws Exception { worker.stopConnector(CONN1); PowerMock.expectLastCall(); worker.startConnector(EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder)); + EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); + EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); PowerMock.replayAll(); @@ -479,7 +528,7 @@ public void testRestartTask() throws Exception { expectPostRebalanceCatchup(SNAPSHOT); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.eq(herder)); + worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); // now handle the task restart @@ -494,7 +543,7 @@ public void testRestartTask() throws Exception { worker.stopAndAwaitTask(TASK0); PowerMock.expectLastCall(); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.eq(herder)); + worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); PowerMock.replayAll(); @@ -637,7 +686,8 @@ public void testConnectorConfigAdded() { expectRebalance(Collections.emptyList(), Collections.emptyList(), ConnectProtocol.Assignment.NO_ERROR, 1, Arrays.asList(CONN1), Collections.emptyList()); worker.startConnector(EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder)); + EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); + EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); PowerMock.expectLastCall(); EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS); member.poll(EasyMock.anyInt()); @@ -646,7 +696,7 @@ public void testConnectorConfigAdded() { PowerMock.replayAll(); herder.tick(); // join - connectorConfigCallback.onCompletion(null, CONN1); // read updated config + configUpdateListener.onConnectorConfigUpdate(CONN1); // read updated config herder.tick(); // apply config herder.tick(); // do rebalance @@ -654,7 +704,7 @@ public void testConnectorConfigAdded() { } @Test - public void testConnectorConfigUpdate() { + public void testConnectorConfigUpdate() throws Exception { // Connector config can be applied without any rebalance EasyMock.expect(member.memberId()).andStubReturn("member"); @@ -664,8 +714,9 @@ public void testConnectorConfigUpdate() { expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); expectPostRebalanceCatchup(SNAPSHOT); worker.startConnector(EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder)); + EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); + EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -678,8 +729,9 @@ public void testConnectorConfigUpdate() { worker.stopConnector(CONN1); PowerMock.expectLastCall(); worker.startConnector(EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder)); + EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); + EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -687,7 +739,7 @@ public void testConnectorConfigUpdate() { PowerMock.replayAll(); herder.tick(); // join - connectorConfigCallback.onCompletion(null, CONN1); // read updated config + configUpdateListener.onConnectorConfigUpdate(CONN1); // read updated config herder.tick(); // apply config PowerMock.verifyAll(); @@ -715,7 +767,7 @@ public void testTaskConfigAdded() { expectRebalance(Collections.emptyList(), Collections.emptyList(), ConnectProtocol.Assignment.NO_ERROR, 1, Collections.emptyList(), Arrays.asList(TASK0)); - worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.eq(herder)); + worker.startTask(EasyMock.eq(TASK0), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -723,7 +775,7 @@ public void testTaskConfigAdded() { PowerMock.replayAll(); herder.tick(); // join - taskConfigCallback.onCompletion(null, Arrays.asList(TASK0, TASK1, TASK2)); // read updated config + configUpdateListener.onTaskConfigUpdate(Arrays.asList(TASK0, TASK1, TASK2)); // read updated config herder.tick(); // apply config herder.tick(); // do rebalance @@ -738,9 +790,8 @@ public void testJoinLeaderCatchUpFails() throws Exception { ConnectProtocol.Assignment.CONFIG_MISMATCH, 1, Collections.emptyList(), Collections.emptyList()); // Reading to end of log times out - TestFuture readToEndFuture = new TestFuture<>(); - readToEndFuture.resolveOnGet(new TimeoutException()); - EasyMock.expect(configStorage.readToEnd()).andReturn(readToEndFuture); + configStorage.refresh(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class)); + EasyMock.expectLastCall().andThrow(new TimeoutException()); member.maybeLeaveGroup(); EasyMock.expectLastCall(); PowerMock.expectPrivate(herder, "backoff", DistributedConfig.WORKER_UNSYNC_BACKOFF_MS_DEFAULT); @@ -751,11 +802,12 @@ public void testJoinLeaderCatchUpFails() throws Exception { expectPostRebalanceCatchup(SNAPSHOT); worker.startConnector(EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder)); + EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS); - worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.eq(herder)); + worker.startTask(EasyMock.eq(TASK1), EasyMock.anyObject(), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); + EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); member.poll(EasyMock.anyInt()); PowerMock.expectLastCall(); @@ -816,8 +868,9 @@ public void testPutConnectorConfig() throws Exception { expectRebalance(1, Arrays.asList(CONN1), Collections.emptyList()); expectPostRebalanceCatchup(SNAPSHOT); worker.startConnector(EasyMock.anyObject(), EasyMock.anyObject(), - EasyMock.eq(herder)); + EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); + EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS); // list connectors, get connector info, get connector config, get task configs @@ -834,7 +887,7 @@ public void testPutConnectorConfig() throws Exception { @Override public Object answer() throws Throwable { // Simulate response to writing config + waiting until end of log to be read - connectorConfigCallback.onCompletion(null, CONN1); + configUpdateListener.onConnectorConfigUpdate(CONN1); return null; } }); @@ -845,8 +898,9 @@ public Object answer() throws Throwable { PowerMock.expectLastCall(); Capture capturedUpdatedConfig = EasyMock.newCapture(); worker.startConnector(EasyMock.capture(capturedUpdatedConfig), EasyMock.anyObject(), - EasyMock.eq(herder)); + EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); PowerMock.expectLastCall(); + EasyMock.expect(worker.isRunning(CONN1)).andReturn(true); EasyMock.expect(worker.connectorTaskConfigs(CONN1, MAX_TASKS, null)).andReturn(TASK_CONFIGS); member.poll(EasyMock.anyInt()); @@ -943,10 +997,9 @@ public Object answer() throws Throwable { PowerMock.expectLastCall(); } - private void expectPostRebalanceCatchup(final ClusterConfigState readToEndSnapshot) { - TestFuture readToEndFuture = new TestFuture<>(); - readToEndFuture.resolveOnGet((Void) null); - EasyMock.expect(configStorage.readToEnd()).andReturn(readToEndFuture); + private void expectPostRebalanceCatchup(final ClusterConfigState readToEndSnapshot) throws TimeoutException { + configStorage.refresh(EasyMock.anyLong(), EasyMock.anyObject(TimeUnit.class)); + EasyMock.expectLastCall(); EasyMock.expect(configStorage.snapshot()).andReturn(readToEndSnapshot); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java index bf33cb32c0ffe..f7423ec6f827e 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/distributed/WorkerCoordinatorTest.java @@ -31,7 +31,8 @@ import org.apache.kafka.common.requests.SyncGroupRequest; import org.apache.kafka.common.requests.SyncGroupResponse; import org.apache.kafka.common.utils.MockTime; -import org.apache.kafka.connect.storage.KafkaConfigStorage; +import org.apache.kafka.connect.runtime.TargetState; +import org.apache.kafka.connect.storage.KafkaConfigBackingStore; import org.apache.kafka.connect.util.ConnectorTaskId; import org.apache.kafka.test.TestUtils; import org.easymock.EasyMock; @@ -76,7 +77,7 @@ public class WorkerCoordinatorTest { private Metrics metrics; private ConsumerNetworkClient consumerClient; private MockRebalanceListener rebalanceListener; - @Mock private KafkaConfigStorage configStorage; + @Mock private KafkaConfigBackingStore configStorage; private WorkerCoordinator coordinator; private ClusterConfigState configState1; @@ -91,7 +92,7 @@ public void setup() { this.consumerClient = new ConsumerNetworkClient(client, metadata, time, 100, 1000); this.metrics = new Metrics(time); this.rebalanceListener = new MockRebalanceListener(); - this.configStorage = PowerMock.createMock(KafkaConfigStorage.class); + this.configStorage = PowerMock.createMock(KafkaConfigBackingStore.class); client.setNode(node); @@ -110,6 +111,7 @@ public void setup() { configState1 = new ClusterConfigState( 1L, Collections.singletonMap(connectorId, 1), Collections.singletonMap(connectorId, (Map) new HashMap()), + Collections.singletonMap(connectorId, TargetState.STARTED), Collections.singletonMap(taskId0, (Map) new HashMap()), Collections.emptySet() ); @@ -119,6 +121,9 @@ public void setup() { Map> configState2ConnectorConfigs = new HashMap<>(); configState2ConnectorConfigs.put(connectorId, new HashMap()); configState2ConnectorConfigs.put(connectorId2, new HashMap()); + Map targetStates = new HashMap<>(); + targetStates.put(connectorId, TargetState.STARTED); + targetStates.put(connectorId2, TargetState.STARTED); Map> configState2TaskConfigs = new HashMap<>(); configState2TaskConfigs.put(taskId0, new HashMap()); configState2TaskConfigs.put(taskId1, new HashMap()); @@ -126,6 +131,7 @@ public void setup() { configState2 = new ClusterConfigState( 2L, configState2ConnectorTaskCounts, configState2ConnectorConfigs, + targetStates, configState2TaskConfigs, Collections.emptySet() ); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java index 05a64a157e288..10e51946cd11f 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerderTest.java @@ -27,6 +27,7 @@ import org.apache.kafka.connect.runtime.ConnectorStatus; import org.apache.kafka.connect.runtime.Herder; import org.apache.kafka.connect.runtime.HerderConnectorContext; +import org.apache.kafka.connect.runtime.TargetState; import org.apache.kafka.connect.runtime.TaskConfig; import org.apache.kafka.connect.runtime.TaskStatus; import org.apache.kafka.connect.runtime.Worker; @@ -36,6 +37,7 @@ import org.apache.kafka.connect.sink.SinkTask; import org.apache.kafka.connect.source.SourceConnector; import org.apache.kafka.connect.source.SourceTask; +import org.apache.kafka.connect.storage.MemoryConfigBackingStore; import org.apache.kafka.connect.storage.StatusBackingStore; import org.apache.kafka.connect.util.Callback; import org.apache.kafka.connect.util.ConnectorTaskId; @@ -80,7 +82,7 @@ public class StandaloneHerderTest { @Before public void setup() { - herder = new StandaloneHerder(WORKER_ID, worker, statusBackingStore); + herder = new StandaloneHerder(worker, WORKER_ID, statusBackingStore, new MemoryConfigBackingStore()); } @Test @@ -163,7 +165,7 @@ public void testRestartConnector() throws Exception { EasyMock.expectLastCall(); worker.startConnector(EasyMock.eq(new ConnectorConfig(connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class))), - EasyMock.anyObject(HerderConnectorContext.class), EasyMock.eq(herder)); + EasyMock.anyObject(HerderConnectorContext.class), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); EasyMock.expectLastCall(); PowerMock.replayAll(); @@ -212,7 +214,7 @@ public void testRestartConnectorFailureOnStart() throws Exception { RuntimeException e = new RuntimeException(); worker.startConnector(EasyMock.eq(new ConnectorConfig(connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class))), - EasyMock.anyObject(HerderConnectorContext.class), EasyMock.eq(herder)); + EasyMock.anyObject(HerderConnectorContext.class), EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); EasyMock.expectLastCall().andThrow(e); PowerMock.replayAll(); @@ -240,7 +242,7 @@ public void testRestartTask() throws Exception { EasyMock.expectLastCall(); Map generatedTaskProps = taskConfig(BogusSourceTask.class, false); - worker.startTask(taskId, new TaskConfig(generatedTaskProps), herder); + worker.startTask(taskId, new TaskConfig(generatedTaskProps), herder, TargetState.STARTED); EasyMock.expectLastCall(); PowerMock.replayAll(); @@ -290,7 +292,7 @@ public void testRestartTaskFailureOnStart() throws Exception { RuntimeException e = new RuntimeException(); Map generatedTaskProps = taskConfig(BogusSourceTask.class, false); - worker.startTask(taskId, new TaskConfig(generatedTaskProps), herder); + worker.startTask(taskId, new TaskConfig(generatedTaskProps), herder, TargetState.STARTED); EasyMock.expectLastCall().andThrow(e); PowerMock.replayAll(); @@ -316,6 +318,11 @@ public void testCreateAndStop() throws Exception { // herder.stop() should stop any running connectors and tasks even if destroyConnector was not invoked expectStop(); + statusBackingStore.stop(); + EasyMock.expectLastCall(); + worker.stop(); + EasyMock.expectLastCall(); + PowerMock.replayAll(); herder.putConnectorConfig(CONNECTOR_NAME, connectorConfig(CONNECTOR_NAME, BogusSourceConnector.class), false, createCallback); @@ -334,7 +341,7 @@ public void testAccessors() throws Exception { Callback> taskConfigsCb = PowerMock.createMock(Callback.class); // Check accessors with empty worker - listConnectorsCb.onCompletion(null, Collections.EMPTY_LIST); + listConnectorsCb.onCompletion(null, Collections.EMPTY_SET); EasyMock.expectLastCall(); connectorInfoCb.onCompletion(EasyMock.anyObject(), EasyMock.isNull()); EasyMock.expectLastCall(); @@ -349,7 +356,7 @@ public void testAccessors() throws Exception { expectAdd(CONNECTOR_NAME, BogusSourceConnector.class, BogusSourceTask.class, false); // Validate accessors with 1 connector - listConnectorsCb.onCompletion(null, Arrays.asList(CONNECTOR_NAME)); + listConnectorsCb.onCompletion(null, Collections.singleton(CONNECTOR_NAME)); EasyMock.expectLastCall(); ConnectorInfo connInfo = new ConnectorInfo(CONNECTOR_NAME, connConfig, Arrays.asList(new ConnectorTaskId(CONNECTOR_NAME, 0))); connectorInfoCb.onCompletion(null, connInfo); @@ -399,8 +406,9 @@ public void testPutConnectorConfig() throws Exception { EasyMock.expectLastCall(); Capture capturedConfig = EasyMock.newCapture(); worker.startConnector(EasyMock.capture(capturedConfig), EasyMock.anyObject(), - EasyMock.eq(herder)); + EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); EasyMock.expectLastCall(); + EasyMock.expect(worker.isRunning(CONNECTOR_NAME)).andReturn(true); // Generate same task config, which should result in no additional action to restart tasks EasyMock.expect(worker.connectorTaskConfigs(CONNECTOR_NAME, DEFAULT_MAX_TASKS, TOPICS_LIST)) .andReturn(Collections.singletonList(taskConfig(BogusSourceTask.class, false))); @@ -411,7 +419,6 @@ public void testPutConnectorConfig() throws Exception { connectorConfigCb.onCompletion(null, newConnConfig); EasyMock.expectLastCall(); - PowerMock.replayAll(); herder.putConnectorConfig(CONNECTOR_NAME, connConfig, false, createCallback); @@ -444,8 +451,9 @@ private void expectAdd(String name, Map connectorProps = connectorConfig(name, connClass); worker.startConnector(EasyMock.eq(new ConnectorConfig(connectorProps)), EasyMock.anyObject(HerderConnectorContext.class), - EasyMock.eq(herder)); + EasyMock.eq(herder), EasyMock.eq(TargetState.STARTED)); EasyMock.expectLastCall(); + EasyMock.expect(worker.isRunning(name)).andReturn(true); ConnectorInfo connInfo = new ConnectorInfo(CONNECTOR_NAME, connectorProps, Arrays.asList(new ConnectorTaskId(CONNECTOR_NAME, 0))); createCallback.onCompletion(null, new Herder.Created<>(true, connInfo)); @@ -457,15 +465,15 @@ private void expectAdd(String name, EasyMock.expect(worker.connectorTaskConfigs(CONNECTOR_NAME, DEFAULT_MAX_TASKS, TOPICS_LIST)) .andReturn(Collections.singletonList(generatedTaskProps)); - worker.startTask(new ConnectorTaskId(CONNECTOR_NAME, 0), new TaskConfig(generatedTaskProps), herder); + worker.startTask(new ConnectorTaskId(CONNECTOR_NAME, 0), new TaskConfig(generatedTaskProps), herder, TargetState.STARTED); EasyMock.expectLastCall(); } private void expectStop() { ConnectorTaskId task = new ConnectorTaskId(CONNECTOR_NAME, 0); - worker.stopTasks(Collections.singleton(task)); + worker.stopTasks(Collections.singletonList(task)); EasyMock.expectLastCall(); - worker.awaitStopTasks(Collections.singleton(task)); + worker.awaitStopTasks(Collections.singletonList(task)); EasyMock.expectLastCall(); worker.stopConnector(CONNECTOR_NAME); EasyMock.expectLastCall(); @@ -475,7 +483,6 @@ private void expectDestroy() { expectStop(); } - private static HashMap connectorConfig(String name, Class connClass) { HashMap connectorProps = new HashMap<>(); connectorProps.put(ConnectorConfig.NAME_CONFIG, name); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigStorageTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java similarity index 89% rename from connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigStorageTest.java rename to connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java index 5e79a8d3d1b95..eaad34bd72269 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigStorageTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java @@ -60,10 +60,10 @@ import static org.junit.Assert.fail; @RunWith(PowerMockRunner.class) -@PrepareForTest(KafkaConfigStorage.class) +@PrepareForTest(KafkaConfigBackingStore.class) @PowerMockIgnore("javax.management.*") @SuppressWarnings("unchecked") -public class KafkaConfigStorageTest { +public class KafkaConfigBackingStoreTest { private static final String TOPIC = "connect-configs"; private static final Map DEFAULT_CONFIG_STORAGE_PROPS = new HashMap<>(); private static final DistributedConfig DEFAULT_DISTRIBUTED_CONFIG; @@ -84,6 +84,7 @@ public class KafkaConfigStorageTest { private static final List CONNECTOR_IDS = Arrays.asList("connector1", "connector2"); private static final List CONNECTOR_CONFIG_KEYS = Arrays.asList("connector-connector1", "connector-connector2"); private static final List COMMIT_TASKS_CONFIG_KEYS = Arrays.asList("commit-connector1", "commit-connector2"); + private static final List TARGET_STATE_KEYS = Arrays.asList("target-state-connector1", "target-state-connector2"); // Need a) connector with multiple tasks and b) multiple connectors private static final List TASK_IDS = Arrays.asList( @@ -100,17 +101,17 @@ public class KafkaConfigStorageTest { Collections.singletonMap("config-key-three", "config-value-three") ); private static final List CONNECTOR_CONFIG_STRUCTS = Arrays.asList( - new Struct(KafkaConfigStorage.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(0)), - new Struct(KafkaConfigStorage.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(1)), - new Struct(KafkaConfigStorage.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(2)) + new Struct(KafkaConfigBackingStore.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(0)), + new Struct(KafkaConfigBackingStore.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(1)), + new Struct(KafkaConfigBackingStore.CONNECTOR_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(2)) ); private static final List TASK_CONFIG_STRUCTS = Arrays.asList( - new Struct(KafkaConfigStorage.TASK_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(0)), - new Struct(KafkaConfigStorage.TASK_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(1)) + new Struct(KafkaConfigBackingStore.TASK_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(0)), + new Struct(KafkaConfigBackingStore.TASK_CONFIGURATION_V0).put("properties", SAMPLE_CONFIGS.get(1)) ); private static final Struct TASKS_COMMIT_STRUCT_TWO_TASK_CONNECTOR - = new Struct(KafkaConfigStorage.CONNECTOR_TASKS_COMMIT_V0).put("tasks", 2); + = new Struct(KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0).put("tasks", 2); // The exact format doesn't matter here since both conversions are mocked private static final List CONFIGS_SERIALIZED = Arrays.asList( @@ -122,12 +123,10 @@ public class KafkaConfigStorageTest { @Mock private Converter converter; @Mock - private Callback connectorReconfiguredCallback; - @Mock - private Callback> tasksReconfiguredCallback; + private ConfigBackingStore.UpdateListener configUpdateListener; @Mock KafkaBasedLog storeLog; - private KafkaConfigStorage configStorage; + private KafkaConfigBackingStore configStorage; private Capture capturedTopic = EasyMock.newCapture(); private Capture> capturedProducerProps = EasyMock.newCapture(); @@ -138,8 +137,8 @@ public class KafkaConfigStorageTest { @Before public void setUp() { - configStorage = PowerMock.createPartialMock(KafkaConfigStorage.class, new String[]{"createKafkaBasedLog"}, - converter, connectorReconfiguredCallback, tasksReconfiguredCallback); + configStorage = PowerMock.createPartialMock(KafkaConfigBackingStore.class, new String[]{"createKafkaBasedLog"}, converter); + configStorage.setUpdateListener(configUpdateListener); } @Test @@ -169,23 +168,27 @@ public void testPutConnectorConfig() throws Exception { expectStart(Collections.EMPTY_LIST, Collections.EMPTY_MAP); expectConvertWriteAndRead( - CONNECTOR_CONFIG_KEYS.get(0), KafkaConfigStorage.CONNECTOR_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(0), + CONNECTOR_CONFIG_KEYS.get(0), KafkaConfigBackingStore.CONNECTOR_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(0), "properties", SAMPLE_CONFIGS.get(0)); - connectorReconfiguredCallback.onCompletion(null, CONNECTOR_IDS.get(0)); + configUpdateListener.onConnectorConfigUpdate(CONNECTOR_IDS.get(0)); EasyMock.expectLastCall(); expectConvertWriteAndRead( - CONNECTOR_CONFIG_KEYS.get(1), KafkaConfigStorage.CONNECTOR_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(1), + CONNECTOR_CONFIG_KEYS.get(1), KafkaConfigBackingStore.CONNECTOR_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(1), "properties", SAMPLE_CONFIGS.get(1)); - connectorReconfiguredCallback.onCompletion(null, CONNECTOR_IDS.get(1)); + configUpdateListener.onConnectorConfigUpdate(CONNECTOR_IDS.get(1)); EasyMock.expectLastCall(); // Config deletion expectConvertWriteAndRead( - CONNECTOR_CONFIG_KEYS.get(1), KafkaConfigStorage.CONNECTOR_CONFIGURATION_V0, null, null, null); - connectorReconfiguredCallback.onCompletion(null, CONNECTOR_IDS.get(1)); + CONNECTOR_CONFIG_KEYS.get(1), KafkaConfigBackingStore.CONNECTOR_CONFIGURATION_V0, null, null, null); + configUpdateListener.onConnectorConfigRemove(CONNECTOR_IDS.get(1)); EasyMock.expectLastCall(); + // Target state deletion + storeLog.send(TARGET_STATE_KEYS.get(1), null); + PowerMock.expectLastCall(); + expectStop(); PowerMock.replayAll(); @@ -214,7 +217,7 @@ public void testPutConnectorConfig() throws Exception { assertEquals(SAMPLE_CONFIGS.get(1), configState.connectorConfig(CONNECTOR_IDS.get(1))); // Deletion should remove the second one we added - configStorage.putConnectorConfig(CONNECTOR_IDS.get(1), null); + configStorage.removeConnectorConfig(CONNECTOR_IDS.get(1)); configState = configStorage.snapshot(); assertEquals(3, configState.offset()); assertEquals(SAMPLE_CONFIGS.get(0), configState.connectorConfig(CONNECTOR_IDS.get(0))); @@ -233,17 +236,17 @@ public void testPutTaskConfigs() throws Exception { // Task configs should read to end, write to the log, read to end, write root, then read to end again expectReadToEnd(new LinkedHashMap()); expectConvertWriteRead( - TASK_CONFIG_KEYS.get(0), KafkaConfigStorage.TASK_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(0), + TASK_CONFIG_KEYS.get(0), KafkaConfigBackingStore.TASK_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(0), "properties", SAMPLE_CONFIGS.get(0)); expectConvertWriteRead( - TASK_CONFIG_KEYS.get(1), KafkaConfigStorage.TASK_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(1), + TASK_CONFIG_KEYS.get(1), KafkaConfigBackingStore.TASK_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(1), "properties", SAMPLE_CONFIGS.get(1)); expectReadToEnd(new LinkedHashMap()); expectConvertWriteRead( - COMMIT_TASKS_CONFIG_KEYS.get(0), KafkaConfigStorage.CONNECTOR_TASKS_COMMIT_V0, CONFIGS_SERIALIZED.get(2), + COMMIT_TASKS_CONFIG_KEYS.get(0), KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0, CONFIGS_SERIALIZED.get(2), "tasks", 2); // Starts with 0 tasks, after update has 2 // As soon as root is rewritten, we should see a callback notifying us that we reconfigured some tasks - tasksReconfiguredCallback.onCompletion(null, Arrays.asList(TASK_IDS.get(0), TASK_IDS.get(1))); + configUpdateListener.onTaskConfigUpdate(Arrays.asList(TASK_IDS.get(0), TASK_IDS.get(1))); EasyMock.expectLastCall(); // Records to be read by consumer as it reads to the end of the log @@ -275,7 +278,7 @@ public void testPutTaskConfigs() throws Exception { Map> taskConfigs = new HashMap<>(); taskConfigs.put(TASK_IDS.get(0), SAMPLE_CONFIGS.get(0)); taskConfigs.put(TASK_IDS.get(1), SAMPLE_CONFIGS.get(1)); - configStorage.putTaskConfigs(taskConfigs); + configStorage.putTaskConfigs("connector1", taskConfigs); // Validate root config by listing all connectors and tasks configState = configStorage.snapshot(); @@ -374,14 +377,14 @@ public void testPutTaskConfigsDoesNotResolveAllInconsistencies() throws Exceptio // Successful attempt to write new task config expectReadToEnd(new LinkedHashMap()); expectConvertWriteRead( - TASK_CONFIG_KEYS.get(0), KafkaConfigStorage.TASK_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(0), + TASK_CONFIG_KEYS.get(0), KafkaConfigBackingStore.TASK_CONFIGURATION_V0, CONFIGS_SERIALIZED.get(0), "properties", SAMPLE_CONFIGS.get(0)); expectReadToEnd(new LinkedHashMap()); expectConvertWriteRead( - COMMIT_TASKS_CONFIG_KEYS.get(0), KafkaConfigStorage.CONNECTOR_TASKS_COMMIT_V0, CONFIGS_SERIALIZED.get(2), + COMMIT_TASKS_CONFIG_KEYS.get(0), KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0, CONFIGS_SERIALIZED.get(2), "tasks", 1); // Updated to just 1 task // As soon as root is rewritten, we should see a callback notifying us that we reconfigured some tasks - tasksReconfiguredCallback.onCompletion(null, Arrays.asList(TASK_IDS.get(0))); + configUpdateListener.onTaskConfigUpdate(Arrays.asList(TASK_IDS.get(0))); EasyMock.expectLastCall(); // Records to be read by consumer as it reads to the end of the log LinkedHashMap serializedConfigs = new LinkedHashMap<>(); @@ -409,7 +412,7 @@ public void testPutTaskConfigsDoesNotResolveAllInconsistencies() throws Exceptio // First try sending an invalid set of configs (can't possibly represent a valid config set for the tasks) try { - configStorage.putTaskConfigs(Collections.singletonMap(TASK_IDS.get(1), SAMPLE_CONFIGS.get(2))); + configStorage.putTaskConfigs("connector1", Collections.singletonMap(TASK_IDS.get(1), SAMPLE_CONFIGS.get(2))); fail("Should have failed due to incomplete task set."); } catch (KafkaException e) { // expected @@ -417,7 +420,7 @@ public void testPutTaskConfigsDoesNotResolveAllInconsistencies() throws Exceptio // Next, issue a write that has everything that is needed and it should be accepted. Note that in this case // we are going to shrink the number of tasks to 1 - configStorage.putTaskConfigs(Collections.singletonMap(TASK_IDS.get(0), SAMPLE_CONFIGS.get(0))); + configStorage.putTaskConfigs("connector1", Collections.singletonMap(TASK_IDS.get(0), SAMPLE_CONFIGS.get(0))); // Validate updated config configState = configStorage.snapshot(); // This is only two more ahead of the last one because multiple calls fail, and so their configs are not written @@ -504,7 +507,7 @@ public Future answer() throws Throwable { private void expectConvertWriteAndRead(final String configKey, final Schema valueSchema, final byte[] serialized, - final String dataFieldName, final Object dataFieldValue) { + final String dataFieldName, final Object dataFieldValue) { expectConvertWriteRead(configKey, valueSchema, serialized, dataFieldName, dataFieldValue); LinkedHashMap recordsToRead = new LinkedHashMap<>(); recordsToRead.put(configKey, serialized); From 8a863ecee7e5dcdaf66a55b91040a7893ffdbf66 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Wed, 20 Apr 2016 17:08:00 -0700 Subject: [PATCH 157/206] KAFKA-3117: handle metadata updates during consumer rebalance Author: Jason Gustafson Reviewers: Guozhang Wang Closes #1247 from hachikuji/KAFKA-3117 --- .../org/apache/kafka/clients/Metadata.java | 14 ++++- .../internals/AbstractCoordinator.java | 4 +- .../internals/ConsumerCoordinator.java | 49 ++++++++++++---- .../internals/ConsumerNetworkClient.java | 2 +- .../internals/ConsumerCoordinatorTest.java | 56 +++++++++++++++++++ .../java/org/apache/kafka/test/TestUtils.java | 21 +++++-- 6 files changed, 125 insertions(+), 21 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/Metadata.java b/clients/src/main/java/org/apache/kafka/clients/Metadata.java index 73a9f333cc7a4..322ae0fbe2315 100644 --- a/clients/src/main/java/org/apache/kafka/clients/Metadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/Metadata.java @@ -107,6 +107,14 @@ public synchronized int requestUpdate() { return this.version; } + /** + * Check whether an update has been explicitly requested. + * @return true if an update was requested, false otherwise + */ + public synchronized boolean updateRequested() { + return this.needUpdate; + } + /** * Wait for metadata update until the current version is larger than the last version we know of */ @@ -203,10 +211,10 @@ public long refreshBackoff() { /** * Set state to indicate if metadata for all topics in Kafka cluster is required or not. - * @param needMetadaForAllTopics boolean indicating need for metadata of all topics in cluster. + * @param needMetadataForAllTopics boolean indicating need for metadata of all topics in cluster. */ - public synchronized void needMetadataForAllTopics(boolean needMetadaForAllTopics) { - this.needMetadataForAllTopics = needMetadaForAllTopics; + public synchronized void needMetadataForAllTopics(boolean needMetadataForAllTopics) { + this.needMetadataForAllTopics = needMetadataForAllTopics; } /** diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java index 496a1141e06b6..15185d7de03a1 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/AbstractCoordinator.java @@ -410,10 +410,10 @@ private RequestFuture sendSyncGroupRequest(SyncGroupRequest request) if (coordinatorUnknown()) return RequestFuture.coordinatorNotAvailable(); return client.send(coordinator, ApiKeys.SYNC_GROUP, request) - .compose(new SyncGroupRequestHandler()); + .compose(new SyncGroupResponseHandler()); } - private class SyncGroupRequestHandler extends CoordinatorResponseHandler { + private class SyncGroupResponseHandler extends CoordinatorResponseHandler { @Override public SyncGroupResponse parse(ClientResponse response) { diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java index 86b60d0ed7553..887f47c1cde4c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinator.java @@ -63,7 +63,6 @@ public final class ConsumerCoordinator extends AbstractCoordinator { private final List assignors; private final org.apache.kafka.clients.Metadata metadata; - private final MetadataSnapshot metadataSnapshot; private final ConsumerCoordinatorMetrics sensors; private final SubscriptionState subscriptions; private final OffsetCommitCallback defaultOffsetCommitCallback; @@ -72,6 +71,9 @@ public final class ConsumerCoordinator extends AbstractCoordinator { private final ConsumerInterceptors interceptors; private final boolean excludeInternalTopics; + private MetadataSnapshot metadataSnapshot; + private MetadataSnapshot assignmentSnapshot; + /** * Initialize the coordination manager. */ @@ -102,7 +104,7 @@ public ConsumerCoordinator(ConsumerNetworkClient client, this.metadata = metadata; this.metadata.requestUpdate(); - this.metadataSnapshot = new MetadataSnapshot(); + this.metadataSnapshot = new MetadataSnapshot(subscriptions, metadata.fetch()); this.subscriptions = subscriptions; this.defaultOffsetCommitCallback = defaultOffsetCommitCallback; this.autoCommitEnabled = autoCommitEnabled; @@ -159,8 +161,14 @@ public void onMetadataUpdate(Cluster cluster) { } // check if there are any changes to the metadata which should trigger a rebalance - if (metadataSnapshot.update(subscriptions, cluster) && subscriptions.partitionsAutoAssigned()) - subscriptions.needReassignment(); + if (subscriptions.partitionsAutoAssigned()) { + MetadataSnapshot snapshot = new MetadataSnapshot(subscriptions, cluster); + if (!snapshot.equals(metadataSnapshot)) { + metadataSnapshot = snapshot; + subscriptions.needReassignment(); + } + } + } }); } @@ -178,6 +186,13 @@ protected void onJoinComplete(int generation, String memberId, String assignmentStrategy, ByteBuffer assignmentBuffer) { + // if we were the assignor, then we need to make sure that there have been no metadata updates + // since the rebalance begin. Otherwise, we won't rebalance again until the next metadata change + if (assignmentSnapshot != null && !assignmentSnapshot.equals(metadataSnapshot)) { + subscriptions.needReassignment(); + return; + } + PartitionAssignor assignor = lookupAssignor(assignmentStrategy); if (assignor == null) throw new IllegalStateException("Coordinator selected invalid assignment protocol: " + assignmentStrategy); @@ -231,7 +246,11 @@ protected Map performAssignment(String leaderId, // which ensures that all metadata changes will eventually be seen this.subscriptions.groupSubscribe(allSubscribedTopics); metadata.setTopics(this.subscriptions.groupSubscription()); + + // update metadata (if needed) and keep track of the metadata used for assignment so that + // we can check after rebalance completion whether anything has changed client.ensureFreshMetadata(); + assignmentSnapshot = metadataSnapshot; log.debug("Performing assignment for group {} using strategy {} with subscriptions {}", groupId, assignor.name(), subscriptions); @@ -267,6 +286,7 @@ protected void onJoinPrepare(int generation, String memberId) { listener.getClass().getName(), groupId, e); } + assignmentSnapshot = null; subscriptions.needReassignment(); } @@ -669,19 +689,26 @@ public double measure(MetricConfig config, long now) { } private static class MetadataSnapshot { - private Map partitionsPerTopic = new HashMap<>(); + private final Map partitionsPerTopic; - public boolean update(SubscriptionState subscription, Cluster cluster) { + public MetadataSnapshot(SubscriptionState subscription, Cluster cluster) { Map partitionsPerTopic = new HashMap<>(); for (String topic : subscription.groupSubscription()) partitionsPerTopic.put(topic, cluster.partitionCountForTopic(topic)); + this.partitionsPerTopic = partitionsPerTopic; + } - if (!partitionsPerTopic.equals(this.partitionsPerTopic)) { - this.partitionsPerTopic = partitionsPerTopic; - return true; - } + @Override + public boolean equals(Object o) { + if (this == o) return true; + if (o == null || getClass() != o.getClass()) return false; + MetadataSnapshot that = (MetadataSnapshot) o; + return partitionsPerTopic != null ? partitionsPerTopic.equals(that.partitionsPerTopic) : that.partitionsPerTopic == null; + } - return false; + @Override + public int hashCode() { + return partitionsPerTopic != null ? partitionsPerTopic.hashCode() : 0; } } diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java index 4119954eaf596..d4c26568fdb01 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/ConsumerNetworkClient.java @@ -138,7 +138,7 @@ public void awaitMetadataUpdate() { * until it has completed). */ public void ensureFreshMetadata() { - if (this.metadata.timeToNextUpdate(time.milliseconds()) == 0) + if (this.metadata.updateRequested() || this.metadata.timeToNextUpdate(time.milliseconds()) == 0) awaitMetadataUpdate(); } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java index b864d692fde44..5a174db162419 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/ConsumerCoordinatorTest.java @@ -63,6 +63,7 @@ import java.util.Collection; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.concurrent.atomic.AtomicBoolean; @@ -575,6 +576,61 @@ public void testMetadataChangeTriggersRebalance() { assertTrue(subscriptions.partitionAssignmentNeeded()); } + + @Test + public void testUpdateMetadataDuringRebalance() { + final String topic1 = "topic1"; + final String topic2 = "topic2"; + TopicPartition tp1 = new TopicPartition(topic1, 0); + TopicPartition tp2 = new TopicPartition(topic2, 0); + final String consumerId = "leader"; + + List topics = Arrays.asList(topic1, topic2); + + subscriptions.subscribe(topics, rebalanceListener); + metadata.setTopics(topics); + subscriptions.needReassignment(); + + // we only have metadata for one topic initially + metadata.update(TestUtils.singletonCluster(topic1, 1), time.milliseconds()); + + client.prepareResponse(consumerMetadataResponse(node, Errors.NONE.code())); + coordinator.ensureCoordinatorKnown(); + + // prepare initial rebalance + Map> memberSubscriptions = Collections.singletonMap(consumerId, topics); + partitionAssignor.prepare(Collections.singletonMap(consumerId, Arrays.asList(tp1))); + + client.prepareResponse(joinGroupLeaderResponse(1, consumerId, memberSubscriptions, Errors.NONE.code())); + client.prepareResponse(new MockClient.RequestMatcher() { + @Override + public boolean matches(ClientRequest request) { + SyncGroupRequest sync = new SyncGroupRequest(request.request().body()); + if (sync.memberId().equals(consumerId) && + sync.generationId() == 1 && + sync.groupAssignment().containsKey(consumerId)) { + // trigger the metadata update including both topics after the sync group request has been sent + Map topicPartitionCounts = new HashMap<>(); + topicPartitionCounts.put(topic1, 1); + topicPartitionCounts.put(topic2, 1); + metadata.update(TestUtils.singletonCluster(topicPartitionCounts), time.milliseconds()); + return true; + } + return false; + } + }, syncGroupResponse(Arrays.asList(tp1), Errors.NONE.code())); + + // the metadata update should trigger a second rebalance + client.prepareResponse(joinGroupLeaderResponse(2, consumerId, memberSubscriptions, Errors.NONE.code())); + client.prepareResponse(syncGroupResponse(Arrays.asList(tp1, tp2), Errors.NONE.code())); + + coordinator.ensurePartitionAssignment(); + + assertFalse(subscriptions.partitionAssignmentNeeded()); + assertEquals(new HashSet<>(Arrays.asList(tp1, tp2)), subscriptions.assignedPartitions()); + } + + @Test public void testExcludeInternalTopicsConfigOption() { subscriptions.subscribe(Pattern.compile(".*"), rebalanceListener); diff --git a/clients/src/test/java/org/apache/kafka/test/TestUtils.java b/clients/src/test/java/org/apache/kafka/test/TestUtils.java index 7ffc54aa3c3eb..027221edeb71d 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestUtils.java @@ -23,6 +23,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.List; +import java.util.Map; import java.util.Random; import org.apache.kafka.common.Cluster; @@ -45,20 +46,32 @@ public class TestUtils { public static final Random SEEDED_RANDOM = new Random(192348092834L); public static final Random RANDOM = new Random(); + public static Cluster singletonCluster(Map topicPartitionCounts) { + return clusterWith(1, topicPartitionCounts); + } + public static Cluster singletonCluster(String topic, int partitions) { return clusterWith(1, topic, partitions); } - public static Cluster clusterWith(int nodes, String topic, int partitions) { + public static Cluster clusterWith(int nodes, Map topicPartitionCounts) { Node[] ns = new Node[nodes]; for (int i = 0; i < nodes; i++) ns[i] = new Node(i, "localhost", 1969); - List parts = new ArrayList(); - for (int i = 0; i < partitions; i++) - parts.add(new PartitionInfo(topic, i, ns[i % ns.length], ns, ns)); + List parts = new ArrayList<>(); + for (Map.Entry topicPartition : topicPartitionCounts.entrySet()) { + String topic = topicPartition.getKey(); + int partitions = topicPartition.getValue(); + 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, Collections.emptySet()); } + public static Cluster clusterWith(int nodes, String topic, int partitions) { + return clusterWith(nodes, Collections.singletonMap(topic, partitions)); + } + /** * Generate an array of random bytes * From f213625fc55dd0eb9087ba6482ce67fcb737172d Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Wed, 20 Apr 2016 17:50:00 -0700 Subject: [PATCH 158/206] MINOR: Fix comment in DistributedHerder ewencp Author: Liquan Pei Reviewers: Ewen Cheslack-Postava Closes #1243 from Ishiihara/docs --- .../kafka/connect/runtime/distributed/DistributedHerder.java | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java index 15126034fbd76..3aa6c33ca4c3b 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java @@ -866,8 +866,8 @@ private void reconfigureConnector(final String connName, final Callback cb configBackingStore.putTaskConfigs(connName, taskConfigListAsMap(connName, taskProps)); cb.onCompletion(null, null); } else { - // We cannot forward the request on the same thread because this reconfiguration can happen in as a - // result of . If we blocked + // We cannot forward the request on the same thread because this reconfiguration can happen as a result of connector + // addition or removal. If we blocked waiting for the response from leader, we may be kicked out of the worker group. forwardRequestExecutor.submit(new Runnable() { @Override public void run() { From 9d71489ff0e62fd1897c7a6f54671673db7b37b8 Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Wed, 20 Apr 2016 18:54:30 -0700 Subject: [PATCH 159/206] KAFKA-3548: Use root locale for case transformation of constant strings For enums and other constant strings, use locale independent case conversions to enable comparisons to work regardless of the default locale. Author: Rajini Sivaram Reviewers: Manikumar Reddy, Ismael Juma, Guozhang Wang, Gwen Shapira Closes #1220 from rajinisivaram/KAFKA-3548 --- checkstyle/checkstyle.xml | 7 +++++++ .../org/apache/kafka/clients/consumer/KafkaConsumer.java | 3 ++- .../apache/kafka/clients/consumer/internals/Fetcher.java | 3 ++- .../org/apache/kafka/clients/producer/KafkaProducer.java | 2 +- .../java/org/apache/kafka/common/config/ConfigDef.java | 9 +++++---- .../java/org/apache/kafka/common/metrics/stats/Rate.java | 3 ++- .../apache/kafka/common/protocol/SecurityProtocol.java | 3 ++- .../org/apache/kafka/test/MockConsumerInterceptor.java | 3 ++- .../main/java/org/apache/kafka/connect/data/Schema.java | 3 ++- core/src/main/scala/kafka/consumer/ConsumerConfig.scala | 3 ++- core/src/main/scala/kafka/log/LogConfig.scala | 5 +++-- core/src/main/scala/kafka/message/CompressionCodec.scala | 8 +++++--- core/src/main/scala/kafka/tools/ConsoleConsumer.scala | 4 ++-- core/src/main/scala/kafka/tools/ConsoleProducer.scala | 4 ++-- core/src/main/scala/kafka/utils/Log4jController.scala | 3 ++- core/src/main/scala/kafka/utils/Os.scala | 4 +++- .../integration/kafka/api/PlaintextConsumerTest.scala | 3 ++- .../java/kafka/examples/KafkaConsumerProducerDemo.java | 2 +- .../kafka/streams/examples/wordcount/WordCountDemo.java | 3 ++- .../examples/wordcount/WordCountProcessorDemo.java | 3 ++- .../kstream/internals/KStreamFlatMapValuesTest.java | 3 ++- .../streams/kstream/internals/KStreamForeachTest.java | 3 ++- .../streams/kstream/internals/KTableForeachTest.java | 3 ++- 23 files changed, 57 insertions(+), 30 deletions(-) diff --git a/checkstyle/checkstyle.xml b/checkstyle/checkstyle.xml index fb2042b93e482..3adc446d2070b 100644 --- a/checkstyle/checkstyle.xml +++ b/checkstyle/checkstyle.xml @@ -84,5 +84,12 @@ + + + + + + + diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index 557643102eeb0..d9b74e2a0aede 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -51,6 +51,7 @@ import java.util.HashSet; import java.util.LinkedHashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Properties; import java.util.Set; @@ -586,7 +587,7 @@ private KafkaConsumer(ConsumerConfig config, config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG), time); this.client = new ConsumerNetworkClient(netClient, metadata, time, retryBackoffMs, config.getInt(ConsumerConfig.REQUEST_TIMEOUT_MS_CONFIG)); - OffsetResetStrategy offsetResetStrategy = OffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase()); + OffsetResetStrategy offsetResetStrategy = OffsetResetStrategy.valueOf(config.getString(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG).toUpperCase(Locale.ROOT)); this.subscriptions = new SubscriptionState(offsetResetStrategy); List assignors = config.getConfiguredInstances( ConsumerConfig.PARTITION_ASSIGNMENT_STRATEGY_CONFIG, diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 9a26551a1e7ab..4985275624430 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -61,6 +61,7 @@ import java.util.Iterator; import java.util.LinkedList; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Set; @@ -291,7 +292,7 @@ else if (strategy == OffsetResetStrategy.LATEST) else throw new NoOffsetForPartitionException(partition); - log.debug("Resetting offset for partition {} to {} offset.", partition, strategy.name().toLowerCase()); + log.debug("Resetting offset for partition {} to {} offset.", partition, strategy.name().toLowerCase(Locale.ROOT)); long offset = listOffset(partition, timestamp); // we might lose the assignment while fetching the offset, so check it is still active diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index d60e28e9c91eb..2a60f612a87f7 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -338,7 +338,7 @@ private KafkaProducer(ProducerConfig config, Serializer keySerializer, Serial private static int parseAcks(String acksString) { try { - return acksString.trim().toLowerCase().equals("all") ? -1 : Integer.parseInt(acksString.trim()); + return acksString.trim().equalsIgnoreCase("all") ? -1 : Integer.parseInt(acksString.trim()); } catch (NumberFormatException e) { throw new ConfigException("Invalid configuration value for 'acks': " + acksString); } diff --git a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java index 1df55d98dc3fd..3a562ce9288e1 100644 --- a/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java +++ b/clients/src/main/java/org/apache/kafka/common/config/ConfigDef.java @@ -24,6 +24,7 @@ import java.util.Iterator; import java.util.LinkedList; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Set; @@ -891,7 +892,7 @@ public String toHtmlTable() { b.append(def.documentation); b.append(""); b.append(""); - b.append(def.type.toString().toLowerCase()); + b.append(def.type.toString().toLowerCase(Locale.ROOT)); b.append(""); b.append(""); if (def.hasDefault()) { @@ -908,7 +909,7 @@ else if (def.type == Type.STRING && def.defaultValue.toString().isEmpty()) b.append(def.validator != null ? def.validator.toString() : ""); b.append(""); b.append(""); - b.append(def.importance.toString().toLowerCase()); + b.append(def.importance.toString().toLowerCase(Locale.ROOT)); b.append(""); b.append("\n"); } @@ -937,7 +938,7 @@ public String toRst() { b.append("\n\n"); } b.append(" * Type: "); - b.append(def.type.toString().toLowerCase()); + b.append(def.type.toString().toLowerCase(Locale.ROOT)); b.append("\n"); if (def.defaultValue != null) { b.append(" * Default: "); @@ -951,7 +952,7 @@ public String toRst() { b.append("\n"); } b.append(" * Importance: "); - b.append(def.importance.toString().toLowerCase()); + b.append(def.importance.toString().toLowerCase(Locale.ROOT)); b.append("\n\n"); } return b.toString(); diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java index 9dfc457b0912e..971b7b648bb05 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Rate.java @@ -13,6 +13,7 @@ package org.apache.kafka.common.metrics.stats; import java.util.List; +import java.util.Locale; import java.util.concurrent.TimeUnit; import org.apache.kafka.common.metrics.MeasurableStat; @@ -48,7 +49,7 @@ public Rate(TimeUnit unit, SampledStat stat) { } public String unitName() { - return unit.name().substring(0, unit.name().length() - 2).toLowerCase(); + return unit.name().substring(0, unit.name().length() - 2).toLowerCase(Locale.ROOT); } @Override diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java index 905c67044ddb2..d5fbed71d2288 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/SecurityProtocol.java @@ -21,6 +21,7 @@ import java.util.EnumSet; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Set; @@ -85,7 +86,7 @@ public static SecurityProtocol forId(Short id) { /** Case insensitive lookup by protocol name */ public static SecurityProtocol forName(String name) { - return SecurityProtocol.valueOf(name.toUpperCase()); + return SecurityProtocol.valueOf(name.toUpperCase(Locale.ROOT)); } /** diff --git a/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java b/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java index 0c187cb79a8c7..cff12a3e4193d 100644 --- a/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java +++ b/clients/src/test/java/org/apache/kafka/test/MockConsumerInterceptor.java @@ -29,6 +29,7 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.concurrent.atomic.AtomicInteger; @@ -59,7 +60,7 @@ public ConsumerRecords onConsume(ConsumerRecords record.timestamp(), record.timestampType(), record.checksum(), record.serializedKeySize(), record.serializedValueSize(), - record.key(), record.value().toUpperCase())); + record.key(), record.value().toUpperCase(Locale.ROOT))); } recordMap.put(tp, lst); } diff --git a/connect/api/src/main/java/org/apache/kafka/connect/data/Schema.java b/connect/api/src/main/java/org/apache/kafka/connect/data/Schema.java index 3c0e40c01f96d..ae2eeb5447d19 100644 --- a/connect/api/src/main/java/org/apache/kafka/connect/data/Schema.java +++ b/connect/api/src/main/java/org/apache/kafka/connect/data/Schema.java @@ -18,6 +18,7 @@ package org.apache.kafka.connect.data; import java.util.List; +import java.util.Locale; import java.util.Map; /** @@ -48,7 +49,7 @@ enum Type { private String name; Type() { - this.name = this.name().toLowerCase(); + this.name = this.name().toLowerCase(Locale.ROOT); } public String getName() { diff --git a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala index 07e974ceb87fc..e1c792d01c110 100644 --- a/core/src/main/scala/kafka/consumer/ConsumerConfig.scala +++ b/core/src/main/scala/kafka/consumer/ConsumerConfig.scala @@ -21,6 +21,7 @@ import java.util.Properties import kafka.api.OffsetRequest import kafka.utils._ import kafka.common.{InvalidConfigException, Config} +import java.util.Locale object ConsumerConfig extends Config { val RefreshMetadataBackoffMs = 200 @@ -163,7 +164,7 @@ class ConsumerConfig private (val props: VerifiableProperties) extends ZKConfig( val offsetsCommitMaxRetries = props.getInt("offsets.commit.max.retries", OffsetsCommitMaxRetries) /** Specify whether offsets should be committed to "zookeeper" (default) or "kafka" */ - val offsetsStorage = props.getString("offsets.storage", OffsetsStorage).toLowerCase + val offsetsStorage = props.getString("offsets.storage", OffsetsStorage).toLowerCase(Locale.ROOT) /** If you are using "kafka" as offsets.storage, you can dual commit offsets to ZooKeeper (in addition to Kafka). This * is required during migration from zookeeper-based offset storage to kafka-based offset storage. With respect to any diff --git a/core/src/main/scala/kafka/log/LogConfig.scala b/core/src/main/scala/kafka/log/LogConfig.scala index 698464ebfcea6..d5e06fa13e6c8 100755 --- a/core/src/main/scala/kafka/log/LogConfig.scala +++ b/core/src/main/scala/kafka/log/LogConfig.scala @@ -27,6 +27,7 @@ import kafka.server.KafkaConfig import org.apache.kafka.common.config.{AbstractConfig, ConfigDef} import org.apache.kafka.common.record.TimestampType import org.apache.kafka.common.utils.Utils +import java.util.Locale object Defaults { val SegmentSize = kafka.server.Defaults.LogSegmentBytes @@ -70,10 +71,10 @@ case class LogConfig(props: java.util.Map[_, _]) extends AbstractConfig(LogConfi val fileDeleteDelayMs = getLong(LogConfig.FileDeleteDelayMsProp) val deleteRetentionMs = getLong(LogConfig.DeleteRetentionMsProp) val minCleanableRatio = getDouble(LogConfig.MinCleanableDirtyRatioProp) - val compact = getString(LogConfig.CleanupPolicyProp).toLowerCase != LogConfig.Delete + val compact = getString(LogConfig.CleanupPolicyProp).toLowerCase(Locale.ROOT) != LogConfig.Delete val uncleanLeaderElectionEnable = getBoolean(LogConfig.UncleanLeaderElectionEnableProp) val minInSyncReplicas = getInt(LogConfig.MinInSyncReplicasProp) - val compressionType = getString(LogConfig.CompressionTypeProp).toLowerCase + val compressionType = getString(LogConfig.CompressionTypeProp).toLowerCase(Locale.ROOT) val preallocate = getBoolean(LogConfig.PreAllocateEnableProp) val messageFormatVersion = ApiVersion(getString(LogConfig.MessageFormatVersionProp)) val messageTimestampType = TimestampType.forName(getString(LogConfig.MessageTimestampTypeProp)) diff --git a/core/src/main/scala/kafka/message/CompressionCodec.scala b/core/src/main/scala/kafka/message/CompressionCodec.scala index 4d7ce1774f0e5..a485271bcb72c 100644 --- a/core/src/main/scala/kafka/message/CompressionCodec.scala +++ b/core/src/main/scala/kafka/message/CompressionCodec.scala @@ -17,6 +17,8 @@ package kafka.message +import java.util.Locale + object CompressionCodec { def getCompressionCodec(codec: Int): CompressionCodec = { codec match { @@ -28,7 +30,7 @@ object CompressionCodec { } } def getCompressionCodec(name: String): CompressionCodec = { - name.toLowerCase match { + name.toLowerCase(Locale.ROOT) match { case NoCompressionCodec.name => NoCompressionCodec case GZIPCompressionCodec.name => GZIPCompressionCodec case SnappyCompressionCodec.name => SnappyCompressionCodec @@ -43,10 +45,10 @@ object BrokerCompressionCodec { val brokerCompressionCodecs = List(UncompressedCodec, SnappyCompressionCodec, LZ4CompressionCodec, GZIPCompressionCodec, ProducerCompressionCodec) val brokerCompressionOptions = brokerCompressionCodecs.map(codec => codec.name) - def isValid(compressionType: String): Boolean = brokerCompressionOptions.contains(compressionType.toLowerCase()) + def isValid(compressionType: String): Boolean = brokerCompressionOptions.contains(compressionType.toLowerCase(Locale.ROOT)) def getCompressionCodec(compressionType: String): CompressionCodec = { - compressionType.toLowerCase match { + compressionType.toLowerCase(Locale.ROOT) match { case UncompressedCodec.name => NoCompressionCodec case _ => CompressionCodec.getCompressionCodec(compressionType) } diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala index 50add72c977d6..e9a43f2bf422b 100755 --- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala @@ -349,9 +349,9 @@ class DefaultMessageFormatter extends MessageFormatter { override def init(props: Properties) { if (props.containsKey("print.timestamp")) - printTimestamp = props.getProperty("print.timestamp").trim.toLowerCase.equals("true") + printTimestamp = props.getProperty("print.timestamp").trim.equalsIgnoreCase("true") if (props.containsKey("print.key")) - printKey = props.getProperty("print.key").trim.toLowerCase.equals("true") + printKey = props.getProperty("print.key").trim.equalsIgnoreCase("true") if (props.containsKey("key.separator")) keySeparator = props.getProperty("key.separator").getBytes if (props.containsKey("line.separator")) diff --git a/core/src/main/scala/kafka/tools/ConsoleProducer.scala b/core/src/main/scala/kafka/tools/ConsoleProducer.scala index 0116a9666fd2c..e6476015f1901 100644 --- a/core/src/main/scala/kafka/tools/ConsoleProducer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleProducer.scala @@ -295,11 +295,11 @@ object ConsoleProducer { override def init(inputStream: InputStream, props: Properties) { topic = props.getProperty("topic") if (props.containsKey("parse.key")) - parseKey = props.getProperty("parse.key").trim.toLowerCase.equals("true") + parseKey = props.getProperty("parse.key").trim.equalsIgnoreCase("true") if (props.containsKey("key.separator")) keySeparator = props.getProperty("key.separator") if (props.containsKey("ignore.error")) - ignoreError = props.getProperty("ignore.error").trim.toLowerCase.equals("true") + ignoreError = props.getProperty("ignore.error").trim.equalsIgnoreCase("true") reader = new BufferedReader(new InputStreamReader(inputStream)) } diff --git a/core/src/main/scala/kafka/utils/Log4jController.scala b/core/src/main/scala/kafka/utils/Log4jController.scala index 673d84e33a48a..026fbaedce340 100755 --- a/core/src/main/scala/kafka/utils/Log4jController.scala +++ b/core/src/main/scala/kafka/utils/Log4jController.scala @@ -20,6 +20,7 @@ package kafka.utils import org.apache.log4j.{Logger, Level, LogManager} import java.util +import java.util.Locale object Log4jController { @@ -81,7 +82,7 @@ private class Log4jController extends Log4jControllerMBean { def setLogLevel(loggerName: String, level: String) = { val log = newLogger(loggerName) if (!loggerName.trim.isEmpty && !level.trim.isEmpty && log != null) { - log.setLevel(Level.toLevel(level.toUpperCase)) + log.setLevel(Level.toLevel(level.toUpperCase(Locale.ROOT))) true } else false diff --git a/core/src/main/scala/kafka/utils/Os.scala b/core/src/main/scala/kafka/utils/Os.scala index 6574f08d9182c..0100a0ab696d8 100644 --- a/core/src/main/scala/kafka/utils/Os.scala +++ b/core/src/main/scala/kafka/utils/Os.scala @@ -17,7 +17,9 @@ package kafka.utils +import java.util.Locale + object Os { - val name = System.getProperty("os.name").toLowerCase + val name = System.getProperty("os.name").toLowerCase(Locale.ROOT) val isWindows = name.startsWith("windows") } \ No newline at end of file diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 349f7ad507722..8dbb80be531c5 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -35,6 +35,7 @@ import org.junit.Test import scala.collection.JavaConverters._ import scala.collection.mutable.Buffer +import java.util.Locale /* We have some tests in this class instead of `BaseConsumerTest` in order to keep the build time under control. */ class PlaintextConsumerTest extends BaseConsumerTest { @@ -606,7 +607,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { for (i <- 0 until numRecords) { val record = records.get(i) assertEquals(s"key $i", new String(record.key())) - assertEquals(s"value $i$appendStr".toUpperCase, new String(record.value())) + assertEquals(s"value $i$appendStr".toUpperCase(Locale.ROOT), new String(record.value())) } // commit sync and verify onCommit is called diff --git a/examples/src/main/java/kafka/examples/KafkaConsumerProducerDemo.java b/examples/src/main/java/kafka/examples/KafkaConsumerProducerDemo.java index 414a6f7deeaed..c1541bfc6276b 100644 --- a/examples/src/main/java/kafka/examples/KafkaConsumerProducerDemo.java +++ b/examples/src/main/java/kafka/examples/KafkaConsumerProducerDemo.java @@ -18,7 +18,7 @@ public class KafkaConsumerProducerDemo { public static void main(String[] args) { - boolean isAsync = args.length == 0 || !args[0].trim().toLowerCase().equals("sync"); + boolean isAsync = args.length == 0 || !args[0].trim().equalsIgnoreCase("sync"); Producer producerThread = new Producer(KafkaProperties.TOPIC, isAsync); producerThread.start(); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java index c12977f8d43f1..5b52803f8079c 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java @@ -29,6 +29,7 @@ import org.apache.kafka.streams.kstream.ValueMapper; import java.util.Arrays; +import java.util.Locale; import java.util.Properties; /** @@ -63,7 +64,7 @@ public static void main(String[] args) throws Exception { .flatMapValues(new ValueMapper>() { @Override public Iterable apply(String value) { - return Arrays.asList(value.toLowerCase().split(" ")); + return Arrays.asList(value.toLowerCase(Locale.getDefault()).split(" ")); } }).map(new KeyValueMapper>() { @Override diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java index a5cddfd005e72..34c35b7c2fc2f 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountProcessorDemo.java @@ -30,6 +30,7 @@ import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.Stores; +import java.util.Locale; import java.util.Properties; /** @@ -63,7 +64,7 @@ public void init(ProcessorContext context) { @Override public void process(String dummy, String line) { - String[] words = line.toLowerCase().split(" "); + String[] words = line.toLowerCase(Locale.getDefault()).split(" "); for (String word : words) { Integer oldValue = this.kvStore.get(word); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java index a904cb15c6266..63f56364f57b1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java @@ -26,6 +26,7 @@ import org.junit.Test; import java.util.ArrayList; +import java.util.Locale; import static org.junit.Assert.assertEquals; @@ -42,7 +43,7 @@ public void testFlatMapValues() { @Override public Iterable apply(String value) { ArrayList result = new ArrayList(); - result.add(value.toLowerCase()); + result.add(value.toLowerCase(Locale.ROOT)); result.add(value); return result; } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamForeachTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamForeachTest.java index 65737790bcf39..d0a182d81595d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamForeachTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamForeachTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.test.KStreamTestDriver; import org.junit.Test; import java.util.List; +import java.util.Locale; import java.util.ArrayList; import java.util.Arrays; @@ -60,7 +61,7 @@ public void testForeach() { new ForeachAction() { @Override public void apply(Integer key, String value) { - actualRecords.add(new KeyValue<>(key * 2, value.toUpperCase())); + actualRecords.add(new KeyValue<>(key * 2, value.toUpperCase(Locale.ROOT))); } }; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableForeachTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableForeachTest.java index 4b612a5211489..27a51146070d2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableForeachTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableForeachTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.test.KStreamTestDriver; import org.junit.Test; import java.util.List; +import java.util.Locale; import java.util.ArrayList; import java.util.Arrays; @@ -60,7 +61,7 @@ public void testForeach() { new ForeachAction() { @Override public void apply(Integer key, String value) { - actualRecords.add(new KeyValue<>(key * 2, value.toUpperCase())); + actualRecords.add(new KeyValue<>(key * 2, value.toUpperCase(Locale.ROOT))); } }; From 5c547475d86aa336f8b3c4bb69faff39759d5df5 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Thu, 21 Apr 2016 13:42:17 -0700 Subject: [PATCH 160/206] KAFKA-3337: Extract selector as a separate groupBy operator for KTable aggregations Author: Matthias J. Sax Reviewers: Guozhang Wang Closes #1231 from mjsax/kafka-3337-extact-key-selector-from-agg --- .../kafka/streams/kstream/KGroupedTable.java | 82 +++++++++ .../apache/kafka/streams/kstream/KTable.java | 109 ++--------- .../kstream/internals/KGroupedTableImpl.java | 172 ++++++++++++++++++ .../streams/kstream/internals/KTableImpl.java | 162 +---------------- .../internals/KTableAggregateTest.java | 14 +- .../streams/smoketest/SmokeTestClient.java | 17 +- 6 files changed, 291 insertions(+), 265 deletions(-) create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java new file mode 100644 index 0000000000000..86c34b16bda69 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java @@ -0,0 +1,82 @@ +/** + * 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 org.apache.kafka.streams.kstream; + +import org.apache.kafka.common.annotation.InterfaceStability; +import org.apache.kafka.common.serialization.Serde; + +/** + * {@link KGroupedTable} is an abstraction of a grouped changelog stream from a primary-keyed table. + * + * @param Type of primary keys + * @param Type of value changes + */ +@InterfaceStability.Unstable +public interface KGroupedTable { + + /** + * Combine updating values of this stream by the selected key into a new instance of {@link KTable}. + * + * @param adder the instance of {@link Reducer} for addition + * @param subtractor the instance of {@link Reducer} for subtraction + * @param name the name of the resulted {@link KTable} + */ + KTable reduce(Reducer adder, + Reducer subtractor, + String name); + + /** + * Aggregate updating values of this stream by the selected key into a new instance of {@link KTable}. + * + * @param initializer the instance of {@link Initializer} + * @param adder the instance of {@link Aggregator} for addition + * @param substractor the instance of {@link Aggregator} for subtraction + * @param aggValueSerde value serdes for materializing the aggregated table, + * if not specified the default serdes defined in the configs will be used + * @param name the name of the resulted table + * @param the value type of the aggregated {@link KTable} + */ + KTable aggregate(Initializer initializer, + Aggregator adder, + Aggregator substractor, + Serde aggValueSerde, + String name); + + /** + * Aggregate updating values of this stream by the selected key into a new instance of {@link KTable} + * using default serializers and deserializers. + * + * @param initializer the instance of {@link Initializer} + * @param adder the instance of {@link Aggregator} for addition + * @param substractor the instance of {@link Aggregator} for subtraction + * @param name the name of the resulted {@link KTable} + * @param the value type of the aggregated {@link KTable} + */ + KTable aggregate(Initializer initializer, + Aggregator adder, + Aggregator substractor, + String name); + + /** + * Count number of records of this stream by the selected key into a new instance of {@link KTable}. + * + * @param name the name of the resulted {@link KTable} + */ + KTable count(String name); + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index 1e44cb5f38ae9..841427907b02c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -242,113 +242,26 @@ public interface KTable { KTable leftJoin(KTable other, ValueJoiner joiner); /** - * Combine updating values of this stream by the selected key into a new instance of {@link KTable}. - * - * @param adder the instance of {@link Reducer} for addition - * @param subtractor the instance of {@link Reducer} for subtraction - * @param selector the instance of {@link KeyValueMapper} that select the aggregate key - * @param keySerde key serdes for materializing the aggregated table, - * if not specified the default serdes defined in the configs will be used - * @param valueSerde value serdes for materializing the aggregated table, - * if not specified the default serdes defined in the configs will be used - * @param name the name of the resulted {@link KTable} - * @param the key type of the aggregated {@link KTable} - * @param the value type of the aggregated {@link KTable} - */ - KTable reduce(Reducer adder, - Reducer subtractor, - KeyValueMapper> selector, - Serde keySerde, - Serde valueSerde, - String name); - - /** - * Combine updating values of this stream by the selected key into a new instance of {@link KTable} - * using default serializers and deserializers. - * - * @param adder the instance of {@link Reducer} for addition - * @param subtractor the instance of {@link Reducer} for subtraction - * @param selector the instance of {@link KeyValueMapper} that select the aggregate key - * @param name the name of the resulted {@link KTable} - * @param the key type of the aggregated {@link KTable} - * @param the value type of the aggregated {@link KTable} - */ - KTable reduce(Reducer adder, - Reducer subtractor, - KeyValueMapper> selector, - String name); - - /** - * Aggregate updating values of this stream by the selected key into a new instance of {@link KTable}. - * - * @param initializer the instance of {@link Initializer} - * @param adder the instance of {@link Aggregator} for addition - * @param substractor the instance of {@link Aggregator} for subtraction - * @param selector the instance of {@link KeyValueMapper} that select the aggregate key - * @param keySerde key serdes for materializing this stream and the aggregated table, - * if not specified the default serdes defined in the configs will be used - * @param valueSerde value serdes for materializing this stream, - * if not specified the default serdes defined in the configs will be used - * @param aggValueSerde value serdes for materializing the aggregated table, - * if not specified the default serdes defined in the configs will be used - * @param name the name of the resulted table - * @param the key type of this {@link KTable} - * @param the value type of this {@link KTable} - * @param the value type of the aggregated {@link KTable} - */ - KTable aggregate(Initializer initializer, - Aggregator adder, - Aggregator substractor, - KeyValueMapper> selector, - Serde keySerde, - Serde valueSerde, - Serde aggValueSerde, - String name); - - /** - * Aggregate updating values of this stream by the selected key into a new instance of {@link KTable} - * using default serializers and deserializers. - * - * @param initializer the instance of {@link Initializer} - * @param adder the instance of {@link Aggregator} for addition - * @param substractor the instance of {@link Aggregator} for subtraction - * @param selector the instance of {@link KeyValueMapper} that select the aggregate key - * @param name the name of the resulted {@link KTable} - * @param the key type of the aggregated {@link KTable} - * @param the value type of the aggregated {@link KTable} - * @param the value type of the aggregated {@link KTable} - */ - KTable aggregate(Initializer initializer, - Aggregator adder, - Aggregator substractor, - KeyValueMapper> selector, - String name); - - /** - * Count number of records of this stream by the selected key into a new instance of {@link KTable}. - * - * @param selector the instance of {@link KeyValueMapper} that select the aggregate key + * Group the records of this {@link KTable} using the provided {@link KeyValueMapper}. + * + * @param selector select the grouping key and value to be aggregated * @param keySerde key serdes for materializing this stream, * if not specified the default serdes defined in the configs will be used * @param valueSerde value serdes for materializing this stream, * if not specified the default serdes defined in the configs will be used - * @param name the name of the resulted table - * @param the key type of the aggregated {@link KTable} + * @param the key type of the {@link KGroupedTable} + * @param the value type of the {@link KGroupedTable} */ - KTable count(KeyValueMapper selector, - Serde keySerde, - Serde valueSerde, - String name); + KGroupedTable groupBy(KeyValueMapper> selector, Serde keySerde, Serde valueSerde); /** - * Count number of records of this stream by the selected key into a new instance of {@link KTable} - * using default serializers and deserializers. + * Group the records of this {@link KTable} using the provided {@link KeyValueMapper} and default serializers and deserializers. * - * @param selector the instance of {@link KeyValueMapper} that select the aggregate key - * @param name the name of the resulted {@link KTable} - * @param the key type of the aggregated {@link KTable} + * @param selector select the grouping key and value to be aggregated + * @param the key type of the {@link KGroupedTable} + * @param the value type of the {@link KGroupedTable} */ - KTable count(KeyValueMapper selector, String name); + KGroupedTable groupBy(KeyValueMapper> selector); /** * Perform an action on each element of {@link KTable}. diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java new file mode 100644 index 0000000000000..d9b0f3d1763c6 --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java @@ -0,0 +1,172 @@ +/** + * 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 org.apache.kafka.streams.kstream.internals; + +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.streams.kstream.Aggregator; +import org.apache.kafka.streams.kstream.Initializer; +import org.apache.kafka.streams.kstream.KGroupedTable; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.Reducer; +import org.apache.kafka.streams.processor.ProcessorSupplier; +import org.apache.kafka.streams.processor.StateStoreSupplier; +import org.apache.kafka.streams.state.Stores; + +import java.util.Collections; + +/** + * The implementation class of {@link KGroupedTable}. + * + * @param the key type + * @param the value type + */ +public class KGroupedTableImpl extends AbstractStream implements KGroupedTable { + + private static final String AGGREGATE_NAME = "KTABLE-AGGREGATE-"; + + private static final String REDUCE_NAME = "KTABLE-REDUCE-"; + + private static final String REPARTITION_TOPIC_SUFFIX = "-repartition"; + + protected final Serde keySerde; + protected final Serde valSerde; + + private final String sourceName; + + public KGroupedTableImpl(KStreamBuilder topology, + String name, + String sourceName, + Serde keySerde, + Serde valSerde) { + super(topology, name, Collections.singleton(sourceName)); + this.sourceName = sourceName; + this.keySerde = keySerde; + this.valSerde = valSerde; + } + + @Override + public KTable aggregate(Initializer initializer, + Aggregator adder, + Aggregator subtractor, + Serde aggValueSerde, + String name) { + + String sinkName = topology.newName(KStreamImpl.SINK_NAME); + String sourceName = topology.newName(KStreamImpl.SOURCE_NAME); + String aggregateName = topology.newName(AGGREGATE_NAME); + + String topic = name + REPARTITION_TOPIC_SUFFIX; + + ChangedSerializer changedValueSerializer = new ChangedSerializer<>(valSerde.serializer()); + ChangedDeserializer changedValueDeserializer = new ChangedDeserializer<>(valSerde.deserializer()); + + ProcessorSupplier> aggregateSupplier = new KTableAggregate<>(name, initializer, adder, subtractor); + + StateStoreSupplier aggregateStore = Stores.create(name) + .withKeys(keySerde) + .withValues(aggValueSerde) + .persistent() + .build(); + + // send the aggregate key-value pairs to the intermediate topic for partitioning + topology.addInternalTopic(topic); + topology.addSink(sinkName, topic, keySerde.serializer(), changedValueSerializer, this.name); + + // read the intermediate topic + topology.addSource(sourceName, keySerde.deserializer(), changedValueDeserializer, topic); + + // aggregate the values with the aggregator and local store + topology.addProcessor(aggregateName, aggregateSupplier, sourceName); + topology.addStateStore(aggregateStore, aggregateName); + + // return the KTable representation with the intermediate topic as the sources + return new KTableImpl<>(topology, aggregateName, aggregateSupplier, Collections.singleton(sourceName)); + } + + @Override + public KTable aggregate(Initializer initializer, + Aggregator adder, + Aggregator substractor, + String name) { + + return aggregate(initializer, adder, substractor, null, name); + } + + @Override + public KTable count(String name) { + return this.aggregate( + new Initializer() { + @Override + public Long apply() { + return 0L; + } + }, + new Aggregator() { + @Override + public Long apply(K aggKey, V value, Long aggregate) { + return aggregate + 1L; + } + }, new Aggregator() { + @Override + public Long apply(K aggKey, V value, Long aggregate) { + return aggregate - 1L; + } + }, + Serdes.Long(), name); + } + + @Override + public KTable reduce(Reducer adder, + Reducer subtractor, + String name) { + + String sinkName = topology.newName(KStreamImpl.SINK_NAME); + String sourceName = topology.newName(KStreamImpl.SOURCE_NAME); + String reduceName = topology.newName(REDUCE_NAME); + + String topic = name + REPARTITION_TOPIC_SUFFIX; + + ChangedSerializer changedValueSerializer = new ChangedSerializer<>(valSerde.serializer()); + ChangedDeserializer changedValueDeserializer = new ChangedDeserializer<>(valSerde.deserializer()); + + ProcessorSupplier> aggregateSupplier = new KTableReduce<>(name, adder, subtractor); + + StateStoreSupplier aggregateStore = Stores.create(name) + .withKeys(keySerde) + .withValues(valSerde) + .persistent() + .build(); + + // send the aggregate key-value pairs to the intermediate topic for partitioning + topology.addInternalTopic(topic); + topology.addSink(sinkName, topic, keySerde.serializer(), changedValueSerializer, this.name); + + // read the intermediate topic + topology.addSource(sourceName, keySerde.deserializer(), changedValueDeserializer, topic); + + // aggregate the values with the aggregator and local store + topology.addProcessor(reduceName, aggregateSupplier, sourceName); + topology.addStateStore(aggregateStore, reduceName); + + // return the KTable representation with the intermediate topic as the sources + return new KTableImpl<>(topology, reduceName, aggregateSupplier, Collections.singleton(sourceName)); + } + +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java index 5c291f5f8e670..51d4cb46f64ed 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KTableImpl.java @@ -18,45 +18,38 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serde; -import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.errors.TopologyBuilderException; -import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.ForeachAction; -import org.apache.kafka.streams.kstream.Initializer; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.KGroupedTable; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Predicate; -import org.apache.kafka.streams.kstream.Reducer; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.streams.processor.ProcessorSupplier; import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.processor.StreamPartitioner; -import org.apache.kafka.streams.state.Stores; import java.io.FileNotFoundException; import java.io.FileOutputStream; import java.io.PrintStream; -import java.util.Collections; import java.util.Set; /** - * The implementation class of KTable + * The implementation class of {@link KTable}. * @param the key type * @param the source's (parent's) value type * @param the value type */ public class KTableImpl extends AbstractStream implements KTable { - private static final String REPARTITION_TOPIC_SUFFIX = "-repartition"; - - private static final String AGGREGATE_NAME = "KTABLE-AGGREGATE-"; - private static final String FILTER_NAME = "KTABLE-FILTER-"; + private static final String FOREACH_NAME = "KTABLE-FOREACH-"; + public static final String JOINTHIS_NAME = "KTABLE-JOINTHIS-"; public static final String JOINOTHER_NAME = "KTABLE-JOINOTHER-"; @@ -75,16 +68,12 @@ public class KTableImpl extends AbstractStream implements KTable processorSupplier; private final Serde keySerde; @@ -172,7 +161,6 @@ public void writeAsText(String filePath, Serde keySerde, Serde valSerde) { } } - @Override public KTable through(Serde keySerde, Serde valSerde, @@ -319,154 +307,24 @@ public KTable leftJoin(KTable other, ValueJoiner } @Override - public KTable aggregate(Initializer initializer, - Aggregator adder, - Aggregator subtractor, - KeyValueMapper> selector, - Serde keySerde, - Serde valueSerde, - Serde aggValueSerde, - String name) { + public KGroupedTable groupBy(KeyValueMapper> selector, + Serde keySerde, + Serde valueSerde) { String selectName = topology.newName(SELECT_NAME); - String sinkName = topology.newName(KStreamImpl.SINK_NAME); - String sourceName = topology.newName(KStreamImpl.SOURCE_NAME); - String aggregateName = topology.newName(AGGREGATE_NAME); - - String topic = name + REPARTITION_TOPIC_SUFFIX; - - ChangedSerializer changedValueSerializer = new ChangedSerializer<>(valueSerde.serializer()); - ChangedDeserializer changedValueDeserializer = new ChangedDeserializer<>(valueSerde.deserializer()); KTableProcessorSupplier> selectSupplier = new KTableRepartitionMap<>(this, selector); - ProcessorSupplier> aggregateSupplier = new KTableAggregate<>(name, initializer, adder, subtractor); - - StateStoreSupplier aggregateStore = Stores.create(name) - .withKeys(keySerde) - .withValues(aggValueSerde) - .persistent() - .build(); - // select the aggregate key and values (old and new), it would require parent to send old values topology.addProcessor(selectName, selectSupplier, this.name); this.enableSendingOldValues(); - // send the aggregate key-value pairs to the intermediate topic for partitioning - topology.addInternalTopic(topic); - topology.addSink(sinkName, topic, keySerde.serializer(), changedValueSerializer, selectName); - - // read the intermediate topic - topology.addSource(sourceName, keySerde.deserializer(), changedValueDeserializer, topic); - - // aggregate the values with the aggregator and local store - topology.addProcessor(aggregateName, aggregateSupplier, sourceName); - topology.addStateStore(aggregateStore, aggregateName); - - // return the KTable representation with the intermediate topic as the sources - return new KTableImpl<>(topology, aggregateName, aggregateSupplier, Collections.singleton(sourceName)); + return new KGroupedTableImpl<>(topology, selectName, this.name, keySerde, valueSerde); } @Override - public KTable aggregate(Initializer initializer, - Aggregator adder, - Aggregator substractor, - KeyValueMapper> selector, - String name) { - - return aggregate(initializer, adder, substractor, selector, null, null, null, name); - } - - @Override - public KTable count(final KeyValueMapper selector, - Serde keySerde, - Serde valueSerde, - String name) { - return this.aggregate( - new Initializer() { - @Override - public Long apply() { - return 0L; - } - }, - new Aggregator() { - @Override - public Long apply(K1 aggKey, V value, Long aggregate) { - return aggregate + 1L; - } - }, new Aggregator() { - @Override - public Long apply(K1 aggKey, V value, Long aggregate) { - return aggregate - 1L; - } - }, new KeyValueMapper>() { - @Override - public KeyValue apply(K key, V value) { - return new KeyValue<>(selector.apply(key, value), value); - } - }, - keySerde, valueSerde, Serdes.Long(), name); - } - - @Override - public KTable count(final KeyValueMapper selector, String name) { - return count(selector, null, null, name); - } - - @Override - public KTable reduce(Reducer adder, - Reducer subtractor, - KeyValueMapper> selector, - Serde keySerde, - Serde valueSerde, - String name) { - - String selectName = topology.newName(SELECT_NAME); - String sinkName = topology.newName(KStreamImpl.SINK_NAME); - String sourceName = topology.newName(KStreamImpl.SOURCE_NAME); - String reduceName = topology.newName(REDUCE_NAME); - - String topic = name + REPARTITION_TOPIC_SUFFIX; - - ChangedSerializer changedValueSerializer = new ChangedSerializer<>(valueSerde.serializer()); - ChangedDeserializer changedValueDeserializer = new ChangedDeserializer<>(valueSerde.deserializer()); - - KTableProcessorSupplier> selectSupplier = new KTableRepartitionMap<>(this, selector); - - ProcessorSupplier> aggregateSupplier = new KTableReduce<>(name, adder, subtractor); - - StateStoreSupplier aggregateStore = Stores.create(name) - .withKeys(keySerde) - .withValues(valueSerde) - .persistent() - .build(); - - // select the aggregate key and values (old and new), it would require parent to send old values - topology.addProcessor(selectName, selectSupplier, this.name); - this.enableSendingOldValues(); - - // send the aggregate key-value pairs to the intermediate topic for partitioning - topology.addInternalTopic(topic); - topology.addSink(sinkName, topic, keySerde.serializer(), changedValueSerializer, selectName); - - // read the intermediate topic - topology.addSource(sourceName, keySerde.deserializer(), changedValueDeserializer, topic); - - // aggregate the values with the aggregator and local store - topology.addProcessor(reduceName, aggregateSupplier, sourceName); - topology.addStateStore(aggregateStore, reduceName); - - // return the KTable representation with the intermediate topic as the sources - return new KTableImpl<>(topology, reduceName, aggregateSupplier, Collections.singleton(sourceName)); - } - - @Override - public KTable reduce(Reducer adder, - Reducer subtractor, - KeyValueMapper> selector, - String name) { - - return reduce(adder, subtractor, selector, null, null, name); + public KGroupedTable groupBy(KeyValueMapper> selector) { + return this.groupBy(selector, null, null); } @SuppressWarnings("unchecked") diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java index fc01e5e7f6004..1564e95682b8a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java @@ -72,12 +72,14 @@ public void testAggBasic() throws Exception { String topic1 = "topic1"; KTable table1 = builder.table(stringSerde, stringSerde, topic1); - KTable table2 = table1.aggregate(new StringInit(), new StringAdd(), new StringRemove(), - new NoOpKeyValueMapper(), - stringSerde, - stringSerde, - stringSerde, - "topic1-Canonized"); + KTable table2 = table1.groupBy(new NoOpKeyValueMapper(), + stringSerde, + stringSerde + ).aggregate(new StringInit(), + new StringAdd(), + new StringRemove(), + stringSerde, + "topic1-Canonized"); MockProcessorSupplier proc2 = new MockProcessorSupplier<>(); table2.toStream().process(proc2); diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java index 0a02824d1fadc..95e0fbfe3173a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java +++ b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java @@ -220,15 +220,14 @@ public KeyValue apply(Windowed key, Long value) { // test repartition Agg agg = new Agg(); - cntTable.aggregate( - agg.init(), - agg.adder(), - agg.remover(), - agg.selector(), - stringSerde, - longSerde, - longSerde, - "cntByCnt" + cntTable.groupBy(agg.selector(), + stringSerde, + longSerde + ).aggregate(agg.init(), + agg.adder(), + agg.remover(), + longSerde, + "cntByCnt" ).to(stringSerde, longSerde, "tagg"); return new KafkaStreams(builder, props); From 74e6dc842559d344241c70cb6607a69291e3a20d Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Thu, 21 Apr 2016 14:51:21 -0700 Subject: [PATCH 161/206] KAFKA-3589: set inner serializer for ChangedSerde upon initialization Author: Guozhang Wang Reviewers: Eno Thereska , Ismael Juma Closes #1246 from guozhangwang/K3589 --- .../internals/ChangedDeserializer.java | 10 ++- .../kstream/internals/ChangedSerializer.java | 10 ++- .../kstream/internals/KGroupedTableImpl.java | 76 ++++++++++--------- .../streams/processor/internals/SinkNode.java | 14 ++++ .../processor/internals/SourceNode.java | 18 ++++- .../internals/KTableAggregateTest.java | 39 ++-------- .../kstream/internals/KTableImplTest.java | 49 ++++++++++++ .../apache/kafka/test/KStreamTestDriver.java | 26 ++++++- .../org/apache/kafka/test/MockAggregator.java | 43 +++++++++++ ...yValueMapper.java => MockInitializer.java} | 16 ++-- .../apache/kafka/test/MockKeyValueMapper.java | 36 +++++++++ .../org/apache/kafka/test/MockReducer.java | 43 +++++++++++ 12 files changed, 302 insertions(+), 78 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/test/MockAggregator.java rename streams/src/test/java/org/apache/kafka/test/{NoOpKeyValueMapper.java => MockInitializer.java} (72%) create mode 100644 streams/src/test/java/org/apache/kafka/test/MockKeyValueMapper.java create mode 100644 streams/src/test/java/org/apache/kafka/test/MockReducer.java diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedDeserializer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedDeserializer.java index d4c4e2deb50f7..ce9be49a71998 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedDeserializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedDeserializer.java @@ -26,12 +26,20 @@ public class ChangedDeserializer implements Deserializer> { private static final int NEWFLAG_SIZE = 1; - private final Deserializer inner; + private Deserializer inner; public ChangedDeserializer(Deserializer inner) { this.inner = inner; } + public Deserializer inner() { + return inner; + } + + public void setInner(Deserializer inner) { + this.inner = inner; + } + @Override public void configure(Map configs, boolean isKey) { // do nothing diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java index 5dbbac9812f9a..12e06f8552e4b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/ChangedSerializer.java @@ -27,12 +27,20 @@ public class ChangedSerializer implements Serializer> { private static final int NEWFLAG_SIZE = 1; - private final Serializer inner; + private Serializer inner; public ChangedSerializer(Serializer inner) { this.inner = inner; } + public Serializer inner() { + return inner; + } + + public void setInner(Serializer inner) { + this.inner = inner; + } + @Override public void configure(Map configs, boolean isKey) { // do nothing diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java index d9b0f3d1763c6..f2e2eed580dcb 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java @@ -17,8 +17,10 @@ package org.apache.kafka.streams.kstream.internals; +import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.Initializer; import org.apache.kafka.streams.kstream.KGroupedTable; @@ -48,15 +50,12 @@ public class KGroupedTableImpl extends AbstractStream implements KGroup protected final Serde keySerde; protected final Serde valSerde; - private final String sourceName; - public KGroupedTableImpl(KStreamBuilder topology, String name, String sourceName, Serde keySerde, Serde valSerde) { super(topology, name, Collections.singleton(sourceName)); - this.sourceName = sourceName; this.keySerde = keySerde; this.valSerde = valSerde; } @@ -74,8 +73,13 @@ public KTable aggregate(Initializer initializer, String topic = name + REPARTITION_TOPIC_SUFFIX; - ChangedSerializer changedValueSerializer = new ChangedSerializer<>(valSerde.serializer()); - ChangedDeserializer changedValueDeserializer = new ChangedDeserializer<>(valSerde.deserializer()); + Serializer keySerializer = keySerde == null ? null : keySerde.serializer(); + Deserializer keyDeserializer = keySerde == null ? null : keySerde.deserializer(); + Serializer valueSerializer = valSerde == null ? null : valSerde.serializer(); + Deserializer valueDeserializer = valSerde == null ? null : valSerde.deserializer(); + + ChangedSerializer changedValueSerializer = new ChangedSerializer<>(valueSerializer); + ChangedDeserializer changedValueDeserializer = new ChangedDeserializer<>(valueDeserializer); ProcessorSupplier> aggregateSupplier = new KTableAggregate<>(name, initializer, adder, subtractor); @@ -87,10 +91,10 @@ public KTable aggregate(Initializer initializer, // send the aggregate key-value pairs to the intermediate topic for partitioning topology.addInternalTopic(topic); - topology.addSink(sinkName, topic, keySerde.serializer(), changedValueSerializer, this.name); + topology.addSink(sinkName, topic, keySerializer, changedValueSerializer, this.name); // read the intermediate topic - topology.addSource(sourceName, keySerde.deserializer(), changedValueDeserializer, topic); + topology.addSource(sourceName, keyDeserializer, changedValueDeserializer, topic); // aggregate the values with the aggregator and local store topology.addProcessor(aggregateName, aggregateSupplier, sourceName); @@ -109,29 +113,6 @@ public KTable aggregate(Initializer initializer, return aggregate(initializer, adder, substractor, null, name); } - @Override - public KTable count(String name) { - return this.aggregate( - new Initializer() { - @Override - public Long apply() { - return 0L; - } - }, - new Aggregator() { - @Override - public Long apply(K aggKey, V value, Long aggregate) { - return aggregate + 1L; - } - }, new Aggregator() { - @Override - public Long apply(K aggKey, V value, Long aggregate) { - return aggregate - 1L; - } - }, - Serdes.Long(), name); - } - @Override public KTable reduce(Reducer adder, Reducer subtractor, @@ -143,8 +124,13 @@ public KTable reduce(Reducer adder, String topic = name + REPARTITION_TOPIC_SUFFIX; - ChangedSerializer changedValueSerializer = new ChangedSerializer<>(valSerde.serializer()); - ChangedDeserializer changedValueDeserializer = new ChangedDeserializer<>(valSerde.deserializer()); + Serializer keySerializer = keySerde == null ? null : keySerde.serializer(); + Deserializer keyDeserializer = keySerde == null ? null : keySerde.deserializer(); + Serializer valueSerializer = valSerde == null ? null : valSerde.serializer(); + Deserializer valueDeserializer = valSerde == null ? null : valSerde.deserializer(); + + ChangedSerializer changedValueSerializer = new ChangedSerializer<>(valueSerializer); + ChangedDeserializer changedValueDeserializer = new ChangedDeserializer<>(valueDeserializer); ProcessorSupplier> aggregateSupplier = new KTableReduce<>(name, adder, subtractor); @@ -156,10 +142,10 @@ public KTable reduce(Reducer adder, // send the aggregate key-value pairs to the intermediate topic for partitioning topology.addInternalTopic(topic); - topology.addSink(sinkName, topic, keySerde.serializer(), changedValueSerializer, this.name); + topology.addSink(sinkName, topic, keySerializer, changedValueSerializer, this.name); // read the intermediate topic - topology.addSource(sourceName, keySerde.deserializer(), changedValueDeserializer, topic); + topology.addSource(sourceName, keyDeserializer, changedValueDeserializer, topic); // aggregate the values with the aggregator and local store topology.addProcessor(reduceName, aggregateSupplier, sourceName); @@ -169,4 +155,26 @@ public KTable reduce(Reducer adder, return new KTableImpl<>(topology, reduceName, aggregateSupplier, Collections.singleton(sourceName)); } + @Override + public KTable count(String name) { + return this.aggregate( + new Initializer() { + @Override + public Long apply() { + return 0L; + } + }, + new Aggregator() { + @Override + public Long apply(K aggKey, V value, Long aggregate) { + return aggregate + 1L; + } + }, new Aggregator() { + @Override + public Long apply(K aggKey, V value, Long aggregate) { + return aggregate - 1L; + } + }, + Serdes.Long(), name); + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java index e9c2760344dcf..37959168b17d2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SinkNode.java @@ -19,6 +19,7 @@ import org.apache.kafka.clients.producer.ProducerRecord; import org.apache.kafka.common.serialization.Serializer; +import org.apache.kafka.streams.kstream.internals.ChangedSerializer; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StreamPartitioner; @@ -52,8 +53,16 @@ public void addChild(ProcessorNode child) { @Override public void init(ProcessorContext context) { this.context = context; + + // if serializers are null, get the default ones from the context if (this.keySerializer == null) this.keySerializer = (Serializer) context.keySerde().serializer(); if (this.valSerializer == null) this.valSerializer = (Serializer) context.valueSerde().serializer(); + + // if value serializers are for {@code Change} values, set the inner serializer when necessary + if (this.valSerializer instanceof ChangedSerializer && + ((ChangedSerializer) this.valSerializer).inner() == null) + ((ChangedSerializer) this.valSerializer).setInner(context.valueSerde().serializer()); + } @Override @@ -67,4 +76,9 @@ public void process(K key, V value) { public void close() { // do nothing } + + // for test only + public Serializer valueSerializer() { + return valSerializer; + } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java index 1868c1b20b797..a55034494aa95 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/SourceNode.java @@ -18,6 +18,7 @@ package org.apache.kafka.streams.processor.internals; import org.apache.kafka.common.serialization.Deserializer; +import org.apache.kafka.streams.kstream.internals.ChangedDeserializer; import org.apache.kafka.streams.processor.ProcessorContext; public class SourceNode extends ProcessorNode { @@ -46,9 +47,16 @@ public V deserializeValue(String topic, byte[] data) { public void init(ProcessorContext context) { this.context = context; - // if serializers are null, get the default ones from the context - if (this.keyDeserializer == null) this.keyDeserializer = (Deserializer) context.keySerde().deserializer(); - if (this.valDeserializer == null) this.valDeserializer = (Deserializer) context.valueSerde().deserializer(); + // if deserializers are null, get the default ones from the context + if (this.keyDeserializer == null) + this.keyDeserializer = (Deserializer) context.keySerde().deserializer(); + if (this.valDeserializer == null) + this.valDeserializer = (Deserializer) context.valueSerde().deserializer(); + + // if value deserializers are for {@code Change} values, set the inner deserializer when necessary + if (this.valDeserializer instanceof ChangedDeserializer && + ((ChangedDeserializer) this.valDeserializer).inner() == null) + ((ChangedDeserializer) this.valDeserializer).setInner(context.valueSerde().deserializer()); } @Override @@ -61,4 +69,8 @@ public void close() { // do nothing } + // for test only + public Deserializer valueDeserializer() { + return valDeserializer; + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java index 1564e95682b8a..be0ec19656f5e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java @@ -20,13 +20,13 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.kstream.Aggregator; -import org.apache.kafka.streams.kstream.Initializer; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockAggregator; +import org.apache.kafka.test.MockInitializer; +import org.apache.kafka.test.MockKeyValueMapper; import org.apache.kafka.test.MockProcessorSupplier; -import org.apache.kafka.test.NoOpKeyValueMapper; import org.junit.Test; import java.io.File; @@ -38,31 +38,6 @@ public class KTableAggregateTest { final private Serde stringSerde = new Serdes.StringSerde(); - private class StringAdd implements Aggregator { - - @Override - public String apply(String aggKey, String value, String aggregate) { - return aggregate + "+" + value; - } - } - - private class StringRemove implements Aggregator { - - @Override - public String apply(String aggKey, String value, String aggregate) { - return aggregate + "-" + value; - } - } - - private class StringInit implements Initializer { - - @Override - public String apply() { - return "0"; - } - } - - @Test public void testAggBasic() throws Exception { final File baseDir = Files.createTempDirectory("test").toFile(); @@ -72,12 +47,12 @@ public void testAggBasic() throws Exception { String topic1 = "topic1"; KTable table1 = builder.table(stringSerde, stringSerde, topic1); - KTable table2 = table1.groupBy(new NoOpKeyValueMapper(), + KTable table2 = table1.groupBy(MockKeyValueMapper.NoOpKeyValueMapper(), stringSerde, stringSerde - ).aggregate(new StringInit(), - new StringAdd(), - new StringRemove(), + ).aggregate(MockInitializer.STRING_INIT, + MockAggregator.STRING_ADDER, + MockAggregator.STRING_REMOVER, stringSerde, "topic1-Canonized"); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java index 6f49b6a870577..8a13e9aaa6373 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java @@ -25,8 +25,14 @@ import org.apache.kafka.streams.kstream.Predicate; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.streams.processor.internals.SinkNode; +import org.apache.kafka.streams.processor.internals.SourceNode; import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockAggregator; +import org.apache.kafka.test.MockInitializer; +import org.apache.kafka.test.MockKeyValueMapper; import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockReducer; import org.junit.Test; import java.io.File; @@ -34,7 +40,9 @@ import java.nio.file.Files; import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; public class KTableImplTest { @@ -295,6 +303,47 @@ public String apply(String v1, Integer v2) { } finally { Utils.delete(stateDir); } + } + + @Test + public void testRepartition() throws IOException { + String topic1 = "topic1"; + + File stateDir = Files.createTempDirectory("test").toFile(); + try { + KStreamBuilder builder = new KStreamBuilder(); + + KTableImpl table1 = + (KTableImpl) builder.table(stringSerde, stringSerde, topic1); + KTableImpl table1Aggregated = (KTableImpl) table1 + .groupBy(MockKeyValueMapper.NoOpKeyValueMapper()) + .aggregate(MockInitializer.STRING_INIT, MockAggregator.STRING_ADDER, MockAggregator.STRING_REMOVER, "mock-result1"); + + + KTableImpl table1Reduced = (KTableImpl) table1 + .groupBy(MockKeyValueMapper.NoOpKeyValueMapper()) + .reduce(MockReducer.STRING_ADDER, MockReducer.STRING_REMOVER, "mock-result2"); + + KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, stringSerde, stringSerde); + driver.setTime(0L); + + // three state store should be created, one for source, one for aggregate and one for reduce + assertEquals(3, driver.allStateStores().size()); + + // contains the corresponding repartition source / sink nodes + assertTrue(driver.allProcessorNames().contains("KSTREAM-SINK-0000000003")); + assertTrue(driver.allProcessorNames().contains("KSTREAM-SOURCE-0000000004")); + assertTrue(driver.allProcessorNames().contains("KSTREAM-SINK-0000000007")); + assertTrue(driver.allProcessorNames().contains("KSTREAM-SOURCE-0000000008")); + + assertNotNull(((ChangedSerializer) ((SinkNode) driver.processor("KSTREAM-SINK-0000000003")).valueSerializer()).inner()); + assertNotNull(((ChangedDeserializer) ((SourceNode) driver.processor("KSTREAM-SOURCE-0000000004")).valueDeserializer()).inner()); + assertNotNull(((ChangedSerializer) ((SinkNode) driver.processor("KSTREAM-SINK-0000000007")).valueSerializer()).inner()); + assertNotNull(((ChangedDeserializer) ((SourceNode) driver.processor("KSTREAM-SOURCE-0000000008")).valueDeserializer()).inner()); + + } finally { + Utils.delete(stateDir); + } } } diff --git a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java index 2ee87303ee7ce..d7387948055e4 100644 --- a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java @@ -31,8 +31,10 @@ import org.apache.kafka.streams.processor.internals.RecordCollector; import java.io.File; +import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Set; public class KStreamTestDriver { @@ -151,11 +153,33 @@ public void forward(K key, V value, String childName) { } } + public Set allProcessorNames() { + Set names = new HashSet<>(); + + List nodes = topology.processors(); + + for (ProcessorNode node: nodes) { + names.add(node.name()); + } + + return names; + } + + public ProcessorNode processor(String name) { + List nodes = topology.processors(); + + for (ProcessorNode node: nodes) { + if (node.name().equals(name)) + return node; + } + + return null; + } + public Map allStateStores() { return context.allStateStores(); } - private class MockRecordCollector extends RecordCollector { public MockRecordCollector() { super(null); diff --git a/streams/src/test/java/org/apache/kafka/test/MockAggregator.java b/streams/src/test/java/org/apache/kafka/test/MockAggregator.java new file mode 100644 index 0000000000000..e8bb10b5836e8 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/MockAggregator.java @@ -0,0 +1,43 @@ +/** + * 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 org.apache.kafka.test; + +import org.apache.kafka.streams.kstream.Aggregator; + +public class MockAggregator { + + private static class StringAdd implements Aggregator { + + @Override + public String apply(String aggKey, String value, String aggregate) { + return aggregate + "+" + value; + } + } + + private static class StringRemove implements Aggregator { + + @Override + public String apply(String aggKey, String value, String aggregate) { + return aggregate + "-" + value; + } + } + + public final static Aggregator STRING_ADDER = new StringAdd(); + + public final static Aggregator STRING_REMOVER = new StringRemove(); +} diff --git a/streams/src/test/java/org/apache/kafka/test/NoOpKeyValueMapper.java b/streams/src/test/java/org/apache/kafka/test/MockInitializer.java similarity index 72% rename from streams/src/test/java/org/apache/kafka/test/NoOpKeyValueMapper.java rename to streams/src/test/java/org/apache/kafka/test/MockInitializer.java index 828b5ae6cbd3a..9bfe7f88eaeeb 100644 --- a/streams/src/test/java/org/apache/kafka/test/NoOpKeyValueMapper.java +++ b/streams/src/test/java/org/apache/kafka/test/MockInitializer.java @@ -17,13 +17,17 @@ package org.apache.kafka.test; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.Initializer; -public class NoOpKeyValueMapper implements KeyValueMapper> { +public class MockInitializer { - @Override - public KeyValue apply(K key, V value) { - return new KeyValue<>(key, value); + private static class StringInit implements Initializer { + + @Override + public String apply() { + return "0"; + } } + + public final static Initializer STRING_INIT = new StringInit(); } diff --git a/streams/src/test/java/org/apache/kafka/test/MockKeyValueMapper.java b/streams/src/test/java/org/apache/kafka/test/MockKeyValueMapper.java new file mode 100644 index 0000000000000..ae8c2fd80fec4 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/MockKeyValueMapper.java @@ -0,0 +1,36 @@ +/** + * 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 org.apache.kafka.test; + +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.KeyValueMapper; + +public class MockKeyValueMapper { + + private static class NoOpKeyValueMapper implements KeyValueMapper> { + + @Override + public KeyValue apply(K key, V value) { + return new KeyValue<>(key, value); + } + } + + public static KeyValueMapper> NoOpKeyValueMapper() { + return new NoOpKeyValueMapper<>(); + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/test/MockReducer.java b/streams/src/test/java/org/apache/kafka/test/MockReducer.java new file mode 100644 index 0000000000000..24a8fea8947e7 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/MockReducer.java @@ -0,0 +1,43 @@ +/** + * 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 org.apache.kafka.test; + +import org.apache.kafka.streams.kstream.Reducer; + +public class MockReducer { + + private static class StringAdd implements Reducer { + + @Override + public String apply(String value1, String value2) { + return value1 + "+" + value2; + } + } + + private static class StringRemove implements Reducer { + + @Override + public String apply(String value1, String value2) { + return value1 + "-" + value2; + } + } + + public final static Reducer STRING_ADDER = new StringAdd(); + + public final static Reducer STRING_REMOVER = new StringRemove(); +} \ No newline at end of file From 32740044972ad3bfc9539b5d76128dceddedc2ba Mon Sep 17 00:00:00 2001 From: Manikumar reddy O Date: Thu, 21 Apr 2016 15:13:25 -0700 Subject: [PATCH 162/206] KAFKA-3594; After calling MemoryRecords.close() method, hasRoomFor() method should return false This exception is occurring when producer is trying to append a record to a Re-enqueued record batch in the accumulator. We should not allow to add a record to Re-enqueued record batch. This is due a bug in MemoryRecords.java/hasRoomFor() method. After calling MemoryRecords.close() method, hasRoomFor() method should return false. Author: Manikumar reddy O Reviewers: Ismael Juma, Grant Henke, Guozhang Wang Closes #1249 from omkreddy/KAFKA-3594 --- .../org/apache/kafka/common/record/MemoryRecords.java | 5 ++++- .../apache/kafka/common/record/MemoryRecordsTest.java | 11 +++++++++++ 2 files changed, 15 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java index f37ef39c4c25e..7175953d66705 100644 --- a/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java +++ b/clients/src/main/java/org/apache/kafka/common/record/MemoryRecords.java @@ -115,7 +115,10 @@ public long append(long offset, long timestamp, byte[] key, byte[] value) { * to accept this single record. */ public boolean hasRoomFor(byte[] key, byte[] value) { - return this.writable && this.compressor.numRecordsWritten() == 0 ? + if (!this.writable) + return false; + + return this.compressor.numRecordsWritten() == 0 ? this.initialCapacity >= Records.LOG_OVERHEAD + Record.recordSize(key, value) : this.writeLimit >= this.compressor.estimatedBytesWritten() + Records.LOG_OVERHEAD + Record.recordSize(key, value); } diff --git a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java index ed64f63f10d91..b1117f1cf63b0 100644 --- a/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/record/MemoryRecordsTest.java @@ -71,6 +71,17 @@ public void testIterator() { } } + @Test + public void testHasRoomForMethod() { + MemoryRecords recs1 = MemoryRecords.emptyRecords(ByteBuffer.allocate(1024), compression); + recs1.append(0, new Record(0L, "a".getBytes(), "1".getBytes())); + + assertTrue(recs1.hasRoomFor("b".getBytes(), "2".getBytes())); + recs1.close(); + assertFalse(recs1.hasRoomFor("b".getBytes(), "2".getBytes())); + + } + @Parameterized.Parameters public static Collection data() { List values = new ArrayList(); From c7f9bd2a68ea7bb604c4dcf2a2f0b030fc019ca7 Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Thu, 21 Apr 2016 17:59:23 -0700 Subject: [PATCH 163/206] KAFKA-3606: Traverse CLASSPATH during herder start ewencp Can you take a quick look? Author: Liquan Pei Reviewers: Ewen Cheslack-Postava Closes #1252 from Ishiihara/pre-list-connectors --- .../kafka/connect/runtime/AbstractHerder.java | 49 ++++++++++++++----- 1 file changed, 36 insertions(+), 13 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java index a22f15c135e53..bd735895b9967 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java @@ -84,6 +84,9 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con private Map tempConnectors = new ConcurrentHashMap<>(); private static final List> SKIPPED_CONNECTORS = Arrays.>asList(VerifiableSourceConnector.class, VerifiableSinkConnector.class); private static List validConnectorPlugins; + private static final Object LOCK = new Object(); + private Thread classPathTraverser; + public AbstractHerder(Worker worker, String workerId, @@ -101,12 +104,20 @@ protected void startServices() { this.worker.start(); this.statusBackingStore.start(); this.configBackingStore.start(); + traverseClassPath(); } protected void stopServices() { this.statusBackingStore.stop(); this.configBackingStore.stop(); this.worker.stop(); + if (this.classPathTraverser != null) { + try { + this.classPathTraverser.join(); + } catch (InterruptedException e) { + // ignore as it can only happen during shutdown + } + } } @Override @@ -248,22 +259,24 @@ public ConfigInfos validateConfigs(String connType, Map connecto } public static List connectorPlugins() { - if (validConnectorPlugins != null) { - return validConnectorPlugins; - } + synchronized (LOCK) { + if (validConnectorPlugins != null) { + return validConnectorPlugins; + } - Reflections reflections = new Reflections(new ConfigurationBuilder().setUrls(ClasspathHelper.forJavaClassPath())); - Set> connectorClasses = reflections.getSubTypesOf(Connector.class); - connectorClasses.removeAll(SKIPPED_CONNECTORS); - List connectorPlugins = new LinkedList<>(); - for (Class connectorClass: connectorClasses) { - int mod = connectorClass.getModifiers(); - if (!Modifier.isAbstract(mod) && !Modifier.isInterface(mod)) { - connectorPlugins.add(new ConnectorPluginInfo(connectorClass.getCanonicalName())); + Reflections reflections = new Reflections(new ConfigurationBuilder().setUrls(ClasspathHelper.forJavaClassPath())); + Set> connectorClasses = reflections.getSubTypesOf(Connector.class); + connectorClasses.removeAll(SKIPPED_CONNECTORS); + List connectorPlugins = new LinkedList<>(); + for (Class connectorClass : connectorClasses) { + int mod = connectorClass.getModifiers(); + if (!Modifier.isAbstract(mod) && !Modifier.isInterface(mod)) { + connectorPlugins.add(new ConnectorPluginInfo(connectorClass.getCanonicalName())); + } } + validConnectorPlugins = connectorPlugins; + return connectorPlugins; } - validConnectorPlugins = connectorPlugins; - return connectorPlugins; } // public for testing @@ -354,4 +367,14 @@ private String trace(Throwable t) { return null; } } + + private void traverseClassPath() { + classPathTraverser = new Thread(new Runnable() { + @Override + public void run() { + connectorPlugins(); + } + }, "CLASSPATH traversal thread."); + classPathTraverser.start(); + } } From bc5051565171cf65b4ed7dd4d9ef269d66a1021a Mon Sep 17 00:00:00 2001 From: Ewen Cheslack-Postava Date: Fri, 22 Apr 2016 07:09:14 -0700 Subject: [PATCH 164/206] KAFKA-3605: Return error if connector config includes mismatching connector name. Author: Ewen Cheslack-Postava Reviewers: Jason Gustafson Closes #1253 from ewencp/kafka-3605-connector-name-mismatch --- .../runtime/distributed/DistributedHerder.java | 16 +++------------- .../rest/resources/ConnectorsResource.java | 9 +++++++++ .../rest/resources/ConnectorsResourceTest.java | 8 ++++++++ 3 files changed, 20 insertions(+), 13 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java index 3aa6c33ca4c3b..cbef1867744b4 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java @@ -435,16 +435,6 @@ public void onCompletion(Throwable error, ConnectorInfo result) { @Override public void putConnectorConfig(final String connName, final Map config, final boolean allowReplace, final Callback> callback) { - final Map connConfig; - if (config == null) { - connConfig = null; - } else if (!config.containsKey(ConnectorConfig.NAME_CONFIG)) { - connConfig = new HashMap<>(config); - connConfig.put(ConnectorConfig.NAME_CONFIG, connName); - } else { - connConfig = config; - } - log.trace("Submitting connector config write request {}", connName); addRequest( @@ -463,7 +453,7 @@ public Void call() throws Exception { return null; } - if (connConfig == null) { + if (config == null) { if (!exists) { callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null); } else { @@ -475,11 +465,11 @@ public Void call() throws Exception { } log.trace("Submitting connector config {} {} {}", connName, allowReplace, configState.connectors()); - configBackingStore.putConnectorConfig(connName, connConfig); + configBackingStore.putConnectorConfig(connName, config); // Note that we use the updated connector config despite the fact that we don't have an updated // snapshot yet. The existing task info should still be accurate. - ConnectorInfo info = new ConnectorInfo(connName, connConfig, configState.tasks(connName)); + ConnectorInfo info = new ConnectorInfo(connName, config, configState.tasks(connName)); callback.onCompletion(null, new Created<>(!exists, info)); return null; } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java index fc713cabdd2ad..2ec35f4898d07 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResource.java @@ -34,6 +34,7 @@ import org.slf4j.LoggerFactory; import javax.servlet.ServletContext; +import javax.ws.rs.BadRequestException; import javax.ws.rs.Consumes; import javax.ws.rs.DELETE; import javax.ws.rs.GET; @@ -129,6 +130,14 @@ public Response putConnectorConfig(final @PathParam("connector") String connecto final @QueryParam("forward") Boolean forward, final Map connectorConfig) throws Throwable { FutureCallback> cb = new FutureCallback<>(); + String includedName = connectorConfig.get(ConnectorConfig.NAME_CONFIG); + if (includedName != null) { + if (!includedName.equals(connector)) + throw new BadRequestException("Connector name configuration (" + includedName + ") doesn't match connector name in the URL (" + connector + ")"); + } else { + connectorConfig.put(ConnectorConfig.NAME_CONFIG, connector); + } + herder.putConnectorConfig(connector, connectorConfig, true, cb); Herder.Created createdInfo = completeOrForwardRequest(cb, "/connectors/" + connector + "/config", "PUT", connectorConfig, new TypeReference() { }, new CreatedConnectorInfoTranslator(), forward); diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java index fa7d997924c72..aa1b9a7f0339d 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorsResourceTest.java @@ -44,6 +44,7 @@ import org.powermock.core.classloader.annotations.PrepareForTest; import org.powermock.modules.junit4.PowerMockRunner; +import javax.ws.rs.BadRequestException; import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; @@ -290,6 +291,13 @@ public void testPutConnectorConfig() throws Throwable { PowerMock.verifyAll(); } + @Test(expected = BadRequestException.class) + public void testPutConnectorConfigNameMismatch() throws Throwable { + Map connConfig = new HashMap<>(CONNECTOR_CONFIG); + connConfig.put(ConnectorConfig.NAME_CONFIG, "mismatched-name"); + connectorsResource.putConnectorConfig(CONNECTOR_NAME, FORWARD, connConfig); + } + @Test public void testGetConnectorTaskConfigs() throws Throwable { final Capture>> cb = Capture.newInstance(); From 1b764c5e834c0d03f3c7107a58f21ad3bbb98ac3 Mon Sep 17 00:00:00 2001 From: Damian Guy Date: Mon, 25 Apr 2016 11:18:28 -0700 Subject: [PATCH 165/206] MINOR: add unit test for KGroupedTable.count Author: Damian Guy Reviewers: Guozhang Wang , Michael G. Noll Closes #1255 from dguy/kgroupedtable-count-test --- .../internals/KGroupedTableImplTest.java | 86 +++++++++++++++++++ 1 file changed, 86 insertions(+) create mode 100644 streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java new file mode 100644 index 0000000000000..9eeea20ebb6ba --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java @@ -0,0 +1,86 @@ +/** + * 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 org.apache.kafka.streams.kstream.internals; + +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockProcessorSupplier; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Files; +import java.util.Arrays; +import java.util.List; + +import static org.junit.Assert.assertEquals; + + +public class KGroupedTableImplTest { + + private File stateDir; + + @Before + public void setUp() throws IOException { + stateDir = Files.createTempDirectory("test").toFile(); + } + + @After + public void tearDown() throws IOException { + Utils.delete(stateDir); + } + + @SuppressWarnings("unchecked") + @Test + public void testGroupedCountOccurences() throws IOException { + final KStreamBuilder builder = new KStreamBuilder(); + final String input = "count-test-input"; + final MockProcessorSupplier processorSupplier = new MockProcessorSupplier<>(); + + builder.table(Serdes.String(), Serdes.String(), input) + .groupBy(new KeyValueMapper>() { + @Override + public KeyValue apply(final String key, final String value) { + return new KeyValue<>(value, value); + } + }, Serdes.String(), Serdes.String()) + .count("count") + .toStream() + .process(processorSupplier); + + + final KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir); + + + driver.process(input, "A", "green"); + driver.process(input, "B", "green"); + driver.process(input, "A", "blue"); + driver.process(input, "C", "yellow"); + driver.process(input, "D", "green"); + + final List expected = Arrays.asList("green:1", "green:2", "blue:1", "green:1", "yellow:1", "green:2"); + final List actual = processorSupplier.processed; + assertEquals(expected, actual); + } +} \ No newline at end of file From 996e29cfe8a9e5a45d4b778a84fb20479eeba469 Mon Sep 17 00:00:00 2001 From: Greg Fodor Date: Mon, 25 Apr 2016 13:45:51 -0700 Subject: [PATCH 166/206] KAFKA-3619: File handles are leaked on .lock files of ProcessorStateManager Kafka Streams seems to hold file handles on the `.lock` files for the state dirs, resulting in an explosion of filehandles over time. Running `lsof` shows the number of open filehandles on the `.lock` file increasing rapidly over time. In a separate test project, I reproduced the issue and determined that in order for the filehandle to be relinquished the `FileChannel` instance must be properly closed. Applying this patch seems to resolve the issue in my job. Author: Greg Fodor Reviewers: Guozhang Wang Closes #1267 from gfodor/bug/state-lock-filehandle-leak --- .../streams/processor/internals/ProcessorStateManager.java | 4 ++++ .../kafka/streams/processor/internals/StreamThread.java | 1 + .../processor/internals/ProcessorStateManagerTest.java | 5 ++++- 3 files changed, 9 insertions(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java index 003b988273fb5..0cdf44cbd0ca4 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/ProcessorStateManager.java @@ -134,6 +134,9 @@ private static FileLock lockStateDirectory(File stateDir, int retry) throws IOEx retry--; lock = lockStateDirectory(channel); } + if (lock == null) { + channel.close(); + } return lock; } @@ -368,6 +371,7 @@ public void close(Map ackedOffsets) throws IOException { } finally { // release the state directory directoryLock directoryLock.release(); + directoryLock.channel().close(); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index f02683e587170..eff90e88cfb01 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -519,6 +519,7 @@ protected void maybeClean() { if (directoryLock != null) { try { directoryLock.release(); + directoryLock.channel().close(); } catch (IOException e) { log.error("Failed to release the state directory lock"); } diff --git a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java index 84b59e639f79d..e3669e8c37d8a 100644 --- a/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/processor/internals/ProcessorStateManagerTest.java @@ -213,7 +213,10 @@ public void testLockStateDirectory() throws IOException { try { assertNotNull(lock); } finally { - if (lock != null) lock.release(); + if (lock != null) { + lock.release(); + lock.channel().close(); + } } } finally { Utils.delete(baseDir); From b3847f76b571deac5c7da7287a642c8b354a2a8c Mon Sep 17 00:00:00 2001 From: Ben Stopford Date: Tue, 26 Apr 2016 06:31:00 -0700 Subject: [PATCH 167/206] KAFKA-3270; Added some Happy Path Tests for the Reassign Partitions Command with help from enothereska :) Author: Ben Stopford Reviewers: Jun Rao , Eno Thereska , Ismael Juma Closes #956 from benstopford/KAFKA-3270-ReassignPartitionsCommand-Tests --- .../admin/ReassignPartitionsCommand.scala | 19 +-- core/src/main/scala/kafka/utils/ZkUtils.scala | 4 +- .../admin/ReassignPartitionsClusterTest.scala | 112 ++++++++++++++++++ 3 files changed, 126 insertions(+), 9 deletions(-) create mode 100644 core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala diff --git a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala index 446ab9f5374c8..1bf351a05e0a5 100755 --- a/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala +++ b/core/src/main/scala/kafka/admin/ReassignPartitionsCommand.scala @@ -39,7 +39,7 @@ object ReassignPartitionsCommand extends Logging { CommandLineUtils.checkRequiredArgs(opts.parser, opts.options, opts.zkConnectOpt) val zkConnect = opts.options.valueOf(opts.zkConnectOpt) - val zkUtils = ZkUtils(zkConnect, + val zkUtils = ZkUtils(zkConnect, 30000, 30000, JaasUtils.isZkSecurityEnabled()) @@ -93,8 +93,8 @@ object ReassignPartitionsCommand extends Logging { val topicsToMoveJsonString = Utils.readFileAsString(topicsToMoveJsonFile) val disableRackAware = opts.options.has(opts.disableRackAware) val (proposedAssignments, currentAssignments) = generateAssignment(zkUtils, brokerListToReassign, topicsToMoveJsonString, disableRackAware) - println("Current partition replica assignment\n\n%s".format(zkUtils.getPartitionReassignmentZkData(currentAssignments))) - println("Proposed partition reassignment configuration\n\n%s".format(zkUtils.getPartitionReassignmentZkData(proposedAssignments))) + println("Current partition replica assignment\n\n%s".format(zkUtils.formatAsReassignmentJson(currentAssignments))) + println("Proposed partition reassignment configuration\n\n%s".format(zkUtils.formatAsReassignmentJson(proposedAssignments))) } def generateAssignment(zkUtils: ZkUtils, brokerListToReassign: Seq[Int], topicsToMoveJsonString: String, disableRackAware: Boolean): (Map[TopicAndPartition, Seq[Int]], Map[TopicAndPartition, Seq[Int]]) = { @@ -125,9 +125,14 @@ object ReassignPartitionsCommand extends Logging { CommandLineUtils.printUsageAndDie(opts.parser, "If --execute option is used, command must include --reassignment-json-file that was output " + "during the --generate option") val reassignmentJsonFile = opts.options.valueOf(opts.reassignmentJsonFileOpt) val reassignmentJsonString = Utils.readFileAsString(reassignmentJsonFile) + executeAssignment(zkUtils, reassignmentJsonString) + } + + def executeAssignment(zkUtils: ZkUtils,reassignmentJsonString: String){ + val partitionsToBeReassigned = zkUtils.parsePartitionReassignmentDataWithoutDedup(reassignmentJsonString) if (partitionsToBeReassigned.isEmpty) - throw new AdminCommandFailedException("Partition reassignment data file %s is empty".format(reassignmentJsonFile)) + throw new AdminCommandFailedException("Partition reassignment data file is empty") val duplicateReassignedPartitions = CoreUtils.duplicates(partitionsToBeReassigned.map{ case(tp,replicas) => tp}) if (duplicateReassignedPartitions.nonEmpty) throw new AdminCommandFailedException("Partition reassignment contains duplicate topic partitions: %s".format(duplicateReassignedPartitions.mkString(","))) @@ -144,10 +149,10 @@ object ReassignPartitionsCommand extends Logging { // before starting assignment, output the current replica assignment to facilitate rollback val currentPartitionReplicaAssignment = zkUtils.getReplicaAssignmentForTopics(partitionsToBeReassigned.map(_._1.topic)) println("Current partition replica assignment\n\n%s\n\nSave this to use as the --reassignment-json-file option during rollback" - .format(zkUtils.getPartitionReassignmentZkData(currentPartitionReplicaAssignment))) + .format(zkUtils.formatAsReassignmentJson(currentPartitionReplicaAssignment))) // start the reassignment if(reassignPartitionsCommand.reassignPartitions()) - println("Successfully started reassignment of partitions %s".format(zkUtils.getPartitionReassignmentZkData(partitionsToBeReassigned.toMap))) + println("Successfully started reassignment of partitions %s".format(zkUtils.formatAsReassignmentJson(partitionsToBeReassigned.toMap))) else println("Failed to reassign partitions %s".format(partitionsToBeReassigned)) } @@ -228,7 +233,7 @@ class ReassignPartitionsCommand(zkUtils: ZkUtils, partitions: collection.Map[Top false } else { - val jsonReassignmentData = zkUtils.getPartitionReassignmentZkData(validPartitions) + val jsonReassignmentData = zkUtils.formatAsReassignmentJson(validPartitions) zkUtils.createPersistentPath(ZkUtils.ReassignPartitionsPath, jsonReassignmentData) true } diff --git a/core/src/main/scala/kafka/utils/ZkUtils.scala b/core/src/main/scala/kafka/utils/ZkUtils.scala index 155b3fdd94222..83ff51773ebba 100644 --- a/core/src/main/scala/kafka/utils/ZkUtils.scala +++ b/core/src/main/scala/kafka/utils/ZkUtils.scala @@ -710,7 +710,7 @@ class ZkUtils(val zkClient: ZkClient, topics } - def getPartitionReassignmentZkData(partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]]): String = { + def formatAsReassignmentJson(partitionsToBeReassigned: Map[TopicAndPartition, Seq[Int]]): String = { Json.encode(Map("version" -> 1, "partitions" -> partitionsToBeReassigned.map(e => Map("topic" -> e._1.topic, "partition" -> e._1.partition, "replicas" -> e._2)))) } @@ -722,7 +722,7 @@ class ZkUtils(val zkClient: ZkClient, deletePath(zkPath) info("No more partitions need to be reassigned. Deleting zk path %s".format(zkPath)) case _ => - val jsonData = getPartitionReassignmentZkData(partitionsToBeReassigned) + val jsonData = formatAsReassignmentJson(partitionsToBeReassigned) try { updatePersistentPath(zkPath, jsonData) debug("Updated partition reassignment path with %s".format(jsonData)) diff --git a/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala new file mode 100644 index 0000000000000..ac2c1aea7835b --- /dev/null +++ b/core/src/test/scala/unit/kafka/admin/ReassignPartitionsClusterTest.scala @@ -0,0 +1,112 @@ +/** + * 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 unit.kafka.admin + +import kafka.admin.ReassignPartitionsCommand +import kafka.server.{KafkaConfig, KafkaServer} +import kafka.utils.TestUtils._ +import kafka.utils.ZkUtils._ +import kafka.utils.{CoreUtils, Logging} +import kafka.zk.ZooKeeperTestHarness +import org.junit.{After, Before, Test} +import org.junit.Assert.assertEquals +import scala.collection.Seq + + +class ReassignPartitionsClusterTest extends ZooKeeperTestHarness with Logging { + val partitionId = 0 + var servers: Seq[KafkaServer] = null + val topicName = "my-topic" + + @Before + override def setUp() { + super.setUp() + } + + def startBrokers(brokerIds: Seq[Int]) { + servers = brokerIds.map(i => createBrokerConfig(i, zkConnect)) + .map(c => createServer(KafkaConfig.fromProps(c))) + } + + @After + override def tearDown() { + servers.foreach(_.shutdown()) + servers.foreach(server => CoreUtils.delete(server.config.logDirs)) + super.tearDown() + } + + @Test + def shouldMoveSinglePartition { + //Given a single replica on server 100 + startBrokers(Seq(100, 101)) + val partition = 0 + createTopic(zkUtils, topicName, Map(partition -> Seq(100)), servers = servers) + + //When we move the replica on 100 to broker 101 + ReassignPartitionsCommand.executeAssignment(zkUtils, s"""{"version":1,"partitions":[{"topic":"$topicName","partition":0,"replicas":[101]}]}""") + waitForReasignmentToComplete() + + //Then the replica should be on 101 + assertEquals(zkUtils.getPartitionAssignmentForTopics(Seq(topicName)).get(topicName).get(partition), Seq(101)) + } + + @Test + def shouldExpandCluster() { + //Given partitions on 2 of 3 brokers + val brokers = Array(100, 101, 102) + startBrokers(brokers) + createTopic(zkUtils, topicName, Map( + 0 -> Seq(100, 101), + 1 -> Seq(100, 101), + 2 -> Seq(100, 101) + ), servers = servers) + + //When rebalancing + val newAssignment = ReassignPartitionsCommand.generateAssignment(zkUtils, brokers, json(topicName), true)._1 + ReassignPartitionsCommand.executeAssignment(zkUtils, zkUtils.formatAsReassignmentJson(newAssignment)) + waitForReasignmentToComplete() + + //Then the replicas should span all three brokers + val actual = zkUtils.getPartitionAssignmentForTopics(Seq(topicName))(topicName) + assertEquals(actual.values.flatten.toSeq.distinct.sorted, Seq(100, 101, 102)) + } + + @Test + def shouldShrinkCluster() { + //Given partitions on 3 of 3 brokers + val brokers = Array(100, 101, 102) + startBrokers(brokers) + createTopic(zkUtils, topicName, Map( + 0 -> Seq(100, 101), + 1 -> Seq(101, 102), + 2 -> Seq(102, 100) + ), servers = servers) + + //When rebalancing + val newAssignment = ReassignPartitionsCommand.generateAssignment(zkUtils, Array(100, 101), json(topicName), true)._1 + ReassignPartitionsCommand.executeAssignment(zkUtils, zkUtils.formatAsReassignmentJson(newAssignment)) + waitForReasignmentToComplete() + + //Then replicas should only span the first two brokers + val actual = zkUtils.getPartitionAssignmentForTopics(Seq(topicName))(topicName) + assertEquals(actual.values.flatten.toSeq.distinct.sorted, Seq(100, 101)) + } + + def waitForReasignmentToComplete() { + waitUntilTrue(() => !zkUtils.pathExists(ReassignPartitionsPath), s"Znode $zkUtils.ReassignPartitionsPath wasn't deleted") + } + + def json(topic: String): String = { + s"""{"topics": [{"topic": "$topic"}],"version":1}""" + } +} From e7d04c2515be158d9a7f0ff0d7571bd41287d91a Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Tue, 26 Apr 2016 06:37:08 -0700 Subject: [PATCH 168/206] KAFKA-3602; Rename RecordAccumulator dequeFor() and fix usage Author: Jason Gustafson Reviewers: Grant Henke , Ashish Singh , Ismael Juma Closes #1254 from hachikuji/KAFKA-3602 --- .../producer/internals/RecordAccumulator.java | 16 ++++++++++------ .../kafka/clients/producer/internals/Sender.java | 2 +- .../internals/RecordAccumulatorTest.java | 14 +++++++------- 3 files changed, 18 insertions(+), 14 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java index d963981524433..1766609ace026 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/RecordAccumulator.java @@ -167,7 +167,7 @@ public RecordAppendResult append(TopicPartition tp, appendsInProgress.incrementAndGet(); try { // check if we have an in-progress batch - Deque dq = dequeFor(tp); + Deque dq = getOrCreateDeque(tp); synchronized (dq) { if (closed) throw new IllegalStateException("Cannot send after the producer is closed."); @@ -213,7 +213,7 @@ public RecordAppendResult append(TopicPartition tp, * Abort the batches that have been sitting in RecordAccumulator for more than the configured requestTimeout * due to metadata being unavailable */ - public List abortExpiredBatches(int requestTimeout, Cluster cluster, long now) { + public List abortExpiredBatches(int requestTimeout, long now) { List expiredBatches = new ArrayList(); int count = 0; for (Map.Entry> entry : this.batches.entrySet()) { @@ -259,7 +259,7 @@ public void reenqueue(RecordBatch batch, long now) { batch.lastAttemptMs = now; batch.lastAppendTime = now; batch.setRetry(); - Deque deque = dequeFor(batch.topicPartition); + Deque deque = getOrCreateDeque(batch.topicPartition); synchronized (deque) { deque.addFirst(batch); } @@ -369,7 +369,7 @@ public Map> drain(Cluster cluster, TopicPartition tp = new TopicPartition(part.topic(), part.partition()); // Only proceed if the partition has no in-flight batches. if (!muted.contains(tp)) { - Deque deque = dequeFor(new TopicPartition(part.topic(), part.partition())); + Deque deque = getDeque(new TopicPartition(part.topic(), part.partition())); if (deque != null) { synchronized (deque) { RecordBatch first = deque.peekFirst(); @@ -401,10 +401,14 @@ public Map> drain(Cluster cluster, return batches; } + private Deque getDeque(TopicPartition tp) { + return batches.get(tp); + } + /** * Get the deque for the given topic-partition, creating it if necessary. */ - private Deque dequeFor(TopicPartition tp) { + private Deque getOrCreateDeque(TopicPartition tp) { Deque d = this.batches.get(tp); if (d != null) return d; @@ -478,7 +482,7 @@ public void abortIncompleteBatches() { */ private void abortBatches() { for (RecordBatch batch : incomplete.all()) { - Deque dq = dequeFor(batch.topicPartition); + Deque dq = getDeque(batch.topicPartition); // Close the batch before aborting synchronized (dq) { batch.records.close(); diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java index db8918c2a49dd..29077b6bb6b3c 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/internals/Sender.java @@ -202,7 +202,7 @@ void run(long now) { } } - List expiredBatches = this.accumulator.abortExpiredBatches(this.requestTimeout, cluster, now); + List expiredBatches = this.accumulator.abortExpiredBatches(this.requestTimeout, now); // update sensors for (RecordBatch expiredBatch : expiredBatches) this.sensors.recordErrors(expiredBatch.topicPartition.topic(), expiredBatch.recordCount); diff --git a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java index 904aa73bd33cd..a39d2e82840ab 100644 --- a/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/producer/internals/RecordAccumulatorTest.java @@ -316,11 +316,11 @@ public void testExpiredBatches() throws InterruptedException { // Advance the clock to expire the batch. time.sleep(requestTimeout + 1); accum.mutePartition(tp1); - List expiredBatches = accum.abortExpiredBatches(requestTimeout, cluster, time.milliseconds()); + List expiredBatches = accum.abortExpiredBatches(requestTimeout, time.milliseconds()); assertEquals("The batch should not be expired when the partition is muted", 0, expiredBatches.size()); accum.unmutePartition(tp1); - expiredBatches = accum.abortExpiredBatches(requestTimeout, cluster, time.milliseconds()); + expiredBatches = accum.abortExpiredBatches(requestTimeout, time.milliseconds()); assertEquals("The batch should be expired", 1, expiredBatches.size()); assertEquals("No partitions should be ready.", 0, accum.ready(cluster, time.milliseconds()).readyNodes.size()); @@ -330,11 +330,11 @@ public void testExpiredBatches() throws InterruptedException { time.sleep(requestTimeout + 1); accum.mutePartition(tp1); - expiredBatches = accum.abortExpiredBatches(requestTimeout, cluster, time.milliseconds()); + expiredBatches = accum.abortExpiredBatches(requestTimeout, time.milliseconds()); assertEquals("The batch should not be expired when metadata is still available and partition is muted", 0, expiredBatches.size()); accum.unmutePartition(tp1); - expiredBatches = accum.abortExpiredBatches(requestTimeout, cluster, time.milliseconds()); + expiredBatches = accum.abortExpiredBatches(requestTimeout, time.milliseconds()); assertEquals("The batch should be expired when the partition is not muted", 1, expiredBatches.size()); assertEquals("No partitions should be ready.", 0, accum.ready(cluster, time.milliseconds()).readyNodes.size()); @@ -351,16 +351,16 @@ public void testExpiredBatches() throws InterruptedException { // test expiration. time.sleep(requestTimeout + retryBackoffMs); - expiredBatches = accum.abortExpiredBatches(requestTimeout, cluster, time.milliseconds()); + expiredBatches = accum.abortExpiredBatches(requestTimeout, time.milliseconds()); assertEquals("The batch should not be expired.", 0, expiredBatches.size()); time.sleep(1L); accum.mutePartition(tp1); - expiredBatches = accum.abortExpiredBatches(requestTimeout, cluster, time.milliseconds()); + expiredBatches = accum.abortExpiredBatches(requestTimeout, time.milliseconds()); assertEquals("The batch should not be expired when the partition is muted", 0, expiredBatches.size()); accum.unmutePartition(tp1); - expiredBatches = accum.abortExpiredBatches(requestTimeout, cluster, time.milliseconds()); + expiredBatches = accum.abortExpiredBatches(requestTimeout, time.milliseconds()); assertEquals("The batch should be expired when the partition is not muted.", 1, expiredBatches.size()); } From a02c8aaecfbd13838c2a062bac1455da352028fe Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 26 Apr 2016 07:32:21 -0700 Subject: [PATCH 169/206] KAFKA-3499: prevent array typed keys in KeyValueStore Author: Guozhang Wang Reviewers: Ismael Juma, Josh Gruenberg, Michael G. Noll, Ewen Cheslack-Postava Closes #1229 from guozhangwang/K3499 --- .../serialization/BytesDeserializer.java | 35 ++++ .../common/serialization/BytesSerializer.java | 36 ++++ .../kafka/common/serialization/Serdes.java | 39 +++- .../org/apache/kafka/common/utils/Bytes.java | 178 ++++++++++++++++++ .../kafka/streams/state/WindowStoreUtils.java | 10 +- .../InMemoryKeyValueStoreSupplier.java | 13 +- .../state/internals/MemoryLRUCache.java | 13 ++ .../state/internals/MeteredKeyValueStore.java | 8 +- .../state/internals/RawStoreChangeLogger.java | 56 ------ .../streams/state/internals/RocksDBStore.java | 73 ++++--- .../state/internals/RocksDBWindowStore.java | 48 ++--- .../state/internals/StoreChangeLogger.java | 10 + .../state/KeyValueStoreTestDriver.java | 15 +- .../internals/AbstractKeyValueStoreTest.java | 2 - .../internals/StoreChangeLoggerTest.java | 41 ---- 15 files changed, 386 insertions(+), 191 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/serialization/BytesDeserializer.java create mode 100644 clients/src/main/java/org/apache/kafka/common/serialization/BytesSerializer.java create mode 100644 clients/src/main/java/org/apache/kafka/common/utils/Bytes.java delete mode 100644 streams/src/main/java/org/apache/kafka/streams/state/internals/RawStoreChangeLogger.java diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/BytesDeserializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/BytesDeserializer.java new file mode 100644 index 0000000000000..ee6a57cb19e72 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/BytesDeserializer.java @@ -0,0 +1,35 @@ +/** + * 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 org.apache.kafka.common.serialization; + +import org.apache.kafka.common.utils.Bytes; + +import java.util.Map; + +public class BytesDeserializer implements Deserializer { + + public void configure(Map configs, boolean isKey) { + // nothing to do + } + + public Bytes deserialize(String topic, byte[] data) { + if (data == null) + return null; + + return new Bytes(data); + } + + public void close() { + // nothing to do + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/BytesSerializer.java b/clients/src/main/java/org/apache/kafka/common/serialization/BytesSerializer.java new file mode 100644 index 0000000000000..3d0444668d3a1 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/serialization/BytesSerializer.java @@ -0,0 +1,36 @@ +/** + * 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 org.apache.kafka.common.serialization; + +import org.apache.kafka.common.utils.Bytes; + +import java.util.Map; + +public class BytesSerializer implements Serializer { + + public void configure(Map configs, boolean isKey) { + // nothing to do + } + + public byte[] serialize(String topic, Bytes data) { + if (data == null) + return null; + + return data.get(); + } + + public void close() { + // nothing to do + } +} + diff --git a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java index f27f74fbc84de..d744522bcec50 100644 --- a/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java +++ b/clients/src/main/java/org/apache/kafka/common/serialization/Serdes.java @@ -13,6 +13,8 @@ package org.apache.kafka.common.serialization; +import org.apache.kafka.common.utils.Bytes; + import java.nio.ByteBuffer; /** @@ -80,6 +82,18 @@ public Deserializer deserializer() { } } + static public final class BytesSerde implements Serde { + @Override + public Serializer serializer() { + return new BytesSerializer(); + } + + @Override + public Deserializer deserializer() { + return new BytesDeserializer(); + } + } + static public final class ByteArraySerde implements Serde { @Override public Serializer serializer() { @@ -114,10 +128,14 @@ static public Serde serdeFrom(Class type) { return (Serde) ByteArray(); } - if (ByteBufferSerde.class.isAssignableFrom(type)) { + if (ByteBuffer.class.isAssignableFrom(type)) { return (Serde) ByteBuffer(); } + if (Bytes.class.isAssignableFrom(type)) { + return (Serde) Bytes(); + } + // TODO: we can also serializes objects of type T using generic Java serialization by default throw new IllegalArgumentException("Unknown class for built-in serializer"); } @@ -150,42 +168,49 @@ public Deserializer deserializer() { } /* - * A serde for nullable long type. + * A serde for nullable {@code Long} type. */ static public Serde Long() { return new LongSerde(); } /* - * A serde for nullable int type. + * A serde for nullable {@code Integer} type. */ static public Serde Integer() { return new IntegerSerde(); } /* - * A serde for nullable long type. + * A serde for nullable {@code Double} type. */ static public Serde Double() { return new DoubleSerde(); } /* - * A serde for nullable string type. + * A serde for nullable {@code String} type. */ static public Serde String() { return new StringSerde(); } /* - * A serde for nullable byte array type. + * A serde for nullable {@code ByteBuffer} type. */ static public Serde ByteBuffer() { return new ByteBufferSerde(); } /* - * A serde for nullable byte array type. + * A serde for nullable {@code Bytes} type. + */ + static public Serde Bytes() { + return new BytesSerde(); + } + + /* + * A serde for nullable {@code byte[]} type. */ static public Serde ByteArray() { return new ByteArraySerde(); diff --git a/clients/src/main/java/org/apache/kafka/common/utils/Bytes.java b/clients/src/main/java/org/apache/kafka/common/utils/Bytes.java new file mode 100644 index 0000000000000..78340e54daec4 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/utils/Bytes.java @@ -0,0 +1,178 @@ +/** + * 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 org.apache.kafka.common.utils; + +import java.util.Arrays; +import java.util.Comparator; + +/** + * Utility class that handles immutable byte arrays. + */ +public class Bytes implements Comparable { + + private static final char[] HEX_CHARS_UPPER = {'0', '1', '2', '3', '4', '5', '6', '7', '8', '9', 'A', 'B', 'C', 'D', 'E', 'F'}; + + private final byte[] bytes; + + // cache the hash code for the string, default to 0 + private int hashCode; + + public static Bytes wrap(byte[] bytes) { + return new Bytes(bytes); + } + + /** + * Create a Bytes using the byte array. + * + * @param bytes This array becomes the backing storage for the object. + */ + public Bytes(byte[] bytes) { + this.bytes = bytes; + + // initialize hash code to 0 + hashCode = 0; + } + + /** + * Get the data from the Bytes. + * @return The data is only valid between offset and offset+length. + */ + public byte[] get() { + return this.bytes; + } + + /** + * The hashcode is cached except for the case where it is computed as 0, in which + * case we compute the hashcode on every call. + * + * @return the hashcode + */ + @Override + public int hashCode() { + if (hashCode == 0) { + hashCode = Arrays.hashCode(bytes); + } + + return hashCode; + } + + @Override + public boolean equals(Object other) { + if (this == other) + return true; + + // we intentionally use the function to compute hashcode here + if (this.hashCode() != other.hashCode()) + return false; + + if (other instanceof Bytes) + return Arrays.equals(this.bytes, ((Bytes) other).get()); + + return false; + } + + @Override + public int compareTo(Bytes that) { + return BYTES_LEXICO_COMPARATOR.compare(this.bytes, that.bytes); + } + + @Override + public String toString() { + return Bytes.toString(bytes, 0, bytes.length); + } + + /** + * Write a printable representation of a byte array. Non-printable + * characters are hex escaped in the format \\x%02X, eg: + * \x00 \x05 etc. + * + * This function is brought from org.apache.hadoop.hbase.util.Bytes + * + * @param b array to write out + * @param off offset to start at + * @param len length to write + * @return string output + */ + private static String toString(final byte[] b, int off, int len) { + StringBuilder result = new StringBuilder(); + + if (b == null) + return result.toString(); + + // just in case we are passed a 'len' that is > buffer length... + if (off >= b.length) + return result.toString(); + + if (off + len > b.length) + len = b.length - off; + + for (int i = off; i < off + len; ++i) { + int ch = b[i] & 0xFF; + if (ch >= ' ' && ch <= '~' && ch != '\\') { + result.append((char) ch); + } else { + result.append("\\x"); + result.append(HEX_CHARS_UPPER[ch / 0x10]); + result.append(HEX_CHARS_UPPER[ch % 0x10]); + } + } + return result.toString(); + } + + /** + * A byte array comparator based on lexicograpic ordering. + */ + public final static Comparator BYTES_LEXICO_COMPARATOR = new LexicographicByteArrayComparator(); + + private interface ByteArrayComparator extends Comparator { + + int compare(final byte[] buffer1, int offset1, int length1, + final byte[] buffer2, int offset2, int length2); + } + + private static class LexicographicByteArrayComparator implements ByteArrayComparator { + + @Override + public int compare(byte[] buffer1, byte[] buffer2) { + return compare(buffer1, 0, buffer1.length, buffer2, 0, buffer2.length); + } + + public int compare(final byte[] buffer1, int offset1, int length1, + final byte[] buffer2, int offset2, int length2) { + + // short circuit equal case + if (buffer1 == buffer2 && + offset1 == offset2 && + length1 == length2) { + return 0; + } + + // similar to Arrays.compare() but considers offset and length + int end1 = offset1 + length1; + int end2 = offset2 + length2; + for (int i = offset1, j = offset2; i < end1 && j < end2; i++, j++) { + int a = buffer1[i] & 0xff; + int b = buffer2[j] & 0xff; + if (a != b) { + return a - b; + } + } + return length1 - length2; + } + } +} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java index fdf3269356127..2f99ad68df4dc 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java @@ -21,6 +21,7 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.utils.Bytes; import java.nio.ByteBuffer; @@ -30,13 +31,12 @@ public class WindowStoreUtils { private static final int TIMESTAMP_SIZE = 8; /** Inner byte array serde used for segments */ - public static final Serde INNER_SERDE = Serdes.ByteArray(); - - /** Inner byte array state serde used for segments */ - public static final StateSerdes INNER_SERDES = new StateSerdes<>("", INNER_SERDE, INNER_SERDE); + public static final Serde INNER_KEY_SERDE = Serdes.Bytes(); + public static final Serde INNER_VALUE_SERDE = Serdes.ByteArray(); + public static final StateSerdes INNER_SERDES = new StateSerdes<>("rocksDB-inner", INNER_KEY_SERDE, INNER_VALUE_SERDE); @SuppressWarnings("unchecked") - public static final KeyValueIterator[] NO_ITERATORS = (KeyValueIterator[]) new KeyValueIterator[0]; + public static final KeyValueIterator[] NO_ITERATORS = (KeyValueIterator[]) new KeyValueIterator[0]; public static byte[] toBinaryKey(K key, final long timestamp, final int seqnum, StateSerdes serdes) { byte[] serializedKey = serdes.rawKey(key); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java index 3a5819c108f53..a25153c973229 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/InMemoryKeyValueStoreSupplier.java @@ -35,6 +35,10 @@ /** * An in-memory key-value store based on a TreeMap. * + * Note that the use of array-typed keys is discouraged because they result in incorrect ordering behavior. + * If you intend to work on byte arrays as key, for example, you may want to wrap them with the {@code Bytes} class, + * i.e. use {@code RocksDBStore} rather than {@code RocksDBStore}. + * * @param The key type * @param The value type * @@ -63,7 +67,7 @@ public String name() { } public StateStore get() { - return new MeteredKeyValueStore<>(new MemoryStore(name, keySerde, valueSerde).enableLogging(), "in-memory-state", time); + return new MeteredKeyValueStore<>(new MemoryStore<>(name, keySerde, valueSerde).enableLogging(), "in-memory-state", time); } private static class MemoryStore implements KeyValueStore { @@ -76,6 +80,9 @@ public MemoryStore(String name, Serde keySerde, Serde valueSerde) { this.name = name; this.keySerde = keySerde; this.valueSerde = valueSerde; + + // TODO: when we have serde associated with class types, we can + // improve this situation by passing the comparator here. this.map = new TreeMap<>(); } @@ -131,12 +138,12 @@ public V delete(K key) { @Override public KeyValueIterator range(K from, K to) { - return new MemoryStoreIterator(this.map.subMap(from, true, to, false).entrySet().iterator()); + return new MemoryStoreIterator<>(this.map.subMap(from, true, to, false).entrySet().iterator()); } @Override public KeyValueIterator all() { - return new MemoryStoreIterator(this.map.entrySet().iterator()); + return new MemoryStoreIterator<>(this.map.entrySet().iterator()); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java index 76dd74439baef..d410e02bad08d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MemoryLRUCache.java @@ -29,6 +29,19 @@ import java.util.Map; import java.util.Set; +/** + * An in-memory LRU cache store based on HashSet and HashMap. + * + * * Note that the use of array-typed keys is discouraged because they result in incorrect ordering behavior. + * If you intend to work on byte arrays as key, for example, you may want to wrap them with the {@code Bytes} class, + * i.e. use {@code RocksDBStore} rather than {@code RocksDBStore}. + + * + * @param The key type + * @param The value type + * + * @see org.apache.kafka.streams.state.Stores#create(String) + */ public class MemoryLRUCache implements KeyValueStore { public interface EldestEntryRemovalListener { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java index 9808c0438e35a..5e5b54ae174bf 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/MeteredKeyValueStore.java @@ -137,9 +137,7 @@ public void putAll(List> entries) { public V delete(K key) { long startNs = time.nanoseconds(); try { - V value = this.inner.delete(key); - - return value; + return this.inner.delete(key); } finally { this.metrics.recordLatency(this.deleteTime, startNs, time.nanoseconds()); } @@ -147,12 +145,12 @@ public V delete(K key) { @Override public KeyValueIterator range(K from, K to) { - return new MeteredKeyValueIterator(this.inner.range(from, to), this.rangeTime); + return new MeteredKeyValueIterator<>(this.inner.range(from, to), this.rangeTime); } @Override public KeyValueIterator all() { - return new MeteredKeyValueIterator(this.inner.all(), this.allTime); + return new MeteredKeyValueIterator<>(this.inner.all(), this.allTime); } @Override diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RawStoreChangeLogger.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RawStoreChangeLogger.java deleted file mode 100644 index 4d99b59f431b3..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RawStoreChangeLogger.java +++ /dev/null @@ -1,56 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streams.state.internals; - -import org.apache.kafka.streams.processor.ProcessorContext; -import org.apache.kafka.streams.state.WindowStoreUtils; - -import java.util.Comparator; -import java.util.TreeSet; - -public class RawStoreChangeLogger extends StoreChangeLogger { - - private class ByteArrayComparator implements Comparator { - @Override - public int compare(byte[] left, byte[] right) { - for (int i = 0, j = 0; i < left.length && j < right.length; i++, j++) { - int a = left[i] & 0xff; - int b = right[j] & 0xff; - - if (a != b) - return a - b; - } - return left.length - right.length; - } - } - - public RawStoreChangeLogger(String storeName, ProcessorContext context) { - this(storeName, context, DEFAULT_WRITE_BATCH_SIZE, DEFAULT_WRITE_BATCH_SIZE); - } - - public RawStoreChangeLogger(String storeName, ProcessorContext context, int maxDirty, int maxRemoved) { - super(storeName, context, context.taskId().partition, WindowStoreUtils.INNER_SERDES, maxDirty, maxRemoved); - init(); - } - - @Override - public void init() { - this.dirty = new TreeSet<>(new ByteArrayComparator()); - this.removed = new TreeSet<>(new ByteArrayComparator()); - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index 944d40802f06f..3fef0efd18c4e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -18,6 +18,7 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.errors.ProcessorStateException; import org.apache.kafka.streams.processor.ProcessorContext; @@ -27,6 +28,7 @@ import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StateSerdes; +import org.apache.kafka.streams.state.WindowStoreUtils; import org.rocksdb.BlockBasedTableConfig; import org.rocksdb.CompactionStyle; import org.rocksdb.CompressionType; @@ -46,6 +48,18 @@ import java.util.NoSuchElementException; import java.util.Set; +/** + * A persistent key-value store based on RocksDB. + * + * Note that the use of array-typed keys is discouraged because they result in incorrect caching behavior. + * If you intend to work on byte arrays as key, for example, you may want to wrap them with the {@code Bytes} class, + * i.e. use {@code RocksDBStore} rather than {@code RocksDBStore}. + * + * @param The key type + * @param The value type + * + * @see org.apache.kafka.streams.state.Stores#create(String) + */ public class RocksDBStore implements KeyValueStore { private static final int TTL_NOT_USED = -1; @@ -80,8 +94,8 @@ public class RocksDBStore implements KeyValueStore { private Set cacheDirtyKeys; private MemoryLRUCache cache; - private StoreChangeLogger changeLogger; - private StoreChangeLogger.ValueGetter getter; + private StoreChangeLogger changeLogger; + private StoreChangeLogger.ValueGetter getter; public KeyValueStore enableLogging() { loggingEnabled = true; @@ -156,7 +170,7 @@ public void init(ProcessorContext context, StateStore root) { // open the DB dir openDB(context); - this.changeLogger = this.loggingEnabled ? new RawStoreChangeLogger(name, context) : null; + this.changeLogger = this.loggingEnabled ? new StoreChangeLogger<>(name, context, WindowStoreUtils.INNER_SERDES) : null; if (this.cacheSize > 0) { this.cache = new MemoryLRUCache(name, cacheSize) @@ -170,7 +184,6 @@ public void apply(K key, RocksDBCacheEntry entry) { } }); - this.cacheDirtyKeys = new HashSet<>(); } else { this.cache = null; @@ -179,10 +192,10 @@ public void apply(K key, RocksDBCacheEntry entry) { // value getter should always read directly from rocksDB // since it is only for values that are already flushed - this.getter = new StoreChangeLogger.ValueGetter() { + this.getter = new StoreChangeLogger.ValueGetter() { @Override - public byte[] get(byte[] key) { - return getInternal(key); + public byte[] get(Bytes key) { + return getInternal(key.get()); } }; @@ -258,7 +271,7 @@ public void put(K key, V value) { putInternal(rawKey, rawValue); if (loggingEnabled) { - changeLogger.add(rawKey); + changeLogger.add(Bytes.wrap(rawKey)); changeLogger.maybeLogChange(this.getter); } } @@ -325,7 +338,7 @@ public KeyValueIterator range(K from, K to) { if (cache != null) flushCache(); - return new RocksDBRangeIterator(db.newIterator(), serdes, from, to); + return new RocksDBRangeIterator<>(db.newIterator(), serdes, from, to); } @Override @@ -336,7 +349,7 @@ public KeyValueIterator all() { RocksIterator innerIter = db.newIterator(); innerIter.seekToFirst(); - return new RocksDbIterator(innerIter, serdes); + return new RocksDbIterator<>(innerIter, serdes); } private void flushCache() { @@ -348,14 +361,16 @@ private void flushCache() { for (K key : cacheDirtyKeys) { RocksDBCacheEntry entry = cache.get(key); - entry.isDirty = false; + if (entry != null) { + entry.isDirty = false; - byte[] rawKey = serdes.rawKey(key); + byte[] rawKey = serdes.rawKey(key); - if (entry.value != null) { - putBatch.add(new KeyValue<>(rawKey, serdes.rawValue(entry.value))); - } else { - deleteBatch.add(rawKey); + if (entry.value != null) { + putBatch.add(new KeyValue<>(rawKey, serdes.rawValue(entry.value))); + } else { + deleteBatch.add(rawKey); + } } } @@ -363,7 +378,7 @@ private void flushCache() { if (loggingEnabled) { for (KeyValue kv : putBatch) - changeLogger.add(kv.key); + changeLogger.add(Bytes.wrap(kv.key)); } // check all removed entries and remove them in rocksDB @@ -376,7 +391,7 @@ private void flushCache() { } if (loggingEnabled) { - changeLogger.delete(removedKey); + changeLogger.delete(Bytes.wrap(removedKey)); } } @@ -464,30 +479,14 @@ public void close() { } - private static class LexicographicComparator implements Comparator { - - @Override - public int compare(byte[] left, byte[] right) { - for (int i = 0, j = 0; i < left.length && j < right.length; i++, j++) { - int leftByte = left[i] & 0xff; - int rightByte = right[j] & 0xff; - if (leftByte != rightByte) { - return leftByte - rightByte; - } - } - return left.length - right.length; - } - } - private static class RocksDBRangeIterator extends RocksDbIterator { // RocksDB's JNI interface does not expose getters/setters that allow the // comparator to be pluggable, and the default is lexicographic, so it's // safe to just force lexicographic comparator here for now. - private final Comparator comparator = new LexicographicComparator(); - byte[] rawToKey; + private final Comparator comparator = Bytes.BYTES_LEXICO_COMPARATOR; + private byte[] rawToKey; - public RocksDBRangeIterator(RocksIterator iter, StateSerdes serdes, - K from, K to) { + public RocksDBRangeIterator(RocksIterator iter, StateSerdes serdes, K from, K to) { super(iter, serdes); iter.seek(serdes.rawKey(from)); this.rawToKey = serdes.rawKey(to); diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java index 5955d214b0f2a..4c964c63715d3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java @@ -20,6 +20,7 @@ package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.utils.Bytes; import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.processor.ProcessorContext; @@ -31,7 +32,6 @@ import org.apache.kafka.streams.state.WindowStoreIterator; import org.apache.kafka.streams.state.WindowStoreUtils; - import java.io.File; import java.text.SimpleDateFormat; import java.util.ArrayList; @@ -48,11 +48,12 @@ public class RocksDBWindowStore implements WindowStore { private static final long USE_CURRENT_TIMESTAMP = -1L; - private static class Segment extends RocksDBStore { + // use the Bytes wrapper for underlying rocksDB keys since they are used for hashing data structures + private static class Segment extends RocksDBStore { public final long id; Segment(String segmentName, String windowName, long id) { - super(segmentName, windowName, WindowStoreUtils.INNER_SERDE, WindowStoreUtils.INNER_SERDE); + super(segmentName, windowName, WindowStoreUtils.INNER_KEY_SERDE, WindowStoreUtils.INNER_VALUE_SERDE); this.id = id; } @@ -63,14 +64,14 @@ public void destroy() { private static class RocksDBWindowStoreIterator implements WindowStoreIterator { private final StateSerdes serdes; - private final KeyValueIterator[] iterators; + private final KeyValueIterator[] iterators; private int index = 0; RocksDBWindowStoreIterator(StateSerdes serdes) { this(serdes, WindowStoreUtils.NO_ITERATORS); } - RocksDBWindowStoreIterator(StateSerdes serdes, KeyValueIterator[] iterators) { + RocksDBWindowStoreIterator(StateSerdes serdes, KeyValueIterator[] iterators) { this.serdes = serdes; this.iterators = iterators; } @@ -94,9 +95,9 @@ public KeyValue next() { if (index >= iterators.length) throw new NoSuchElementException(); - KeyValue kv = iterators[index].next(); + KeyValue kv = iterators[index].next(); - return new KeyValue<>(WindowStoreUtils.timestampFromBinaryKey(kv.key), + return new KeyValue<>(WindowStoreUtils.timestampFromBinaryKey(kv.key.get()), serdes.valueFrom(kv.value)); } @@ -108,7 +109,7 @@ public void remove() { @Override public void close() { - for (KeyValueIterator iterator : iterators) { + for (KeyValueIterator iterator : iterators) { iterator.close(); } } @@ -121,7 +122,7 @@ public void close() { private final Serde keySerde; private final Serde valueSerde; private final SimpleDateFormat formatter; - private final StoreChangeLogger.ValueGetter getter; + private final StoreChangeLogger.ValueGetter getter; private ProcessorContext context; private int seqnum = 0; @@ -130,7 +131,7 @@ public void close() { private StateSerdes serdes; private boolean loggingEnabled = false; - private StoreChangeLogger changeLogger = null; + private StoreChangeLogger changeLogger = null; public RocksDBWindowStore(String name, long retentionPeriod, int numSegments, boolean retainDuplicates, Serde keySerde, Serde valueSerde) { this.name = name; @@ -144,9 +145,9 @@ public RocksDBWindowStore(String name, long retentionPeriod, int numSegments, bo this.retainDuplicates = retainDuplicates; - this.getter = new StoreChangeLogger.ValueGetter() { - public byte[] get(byte[] key) { - return getInternal(key); + this.getter = new StoreChangeLogger.ValueGetter() { + public byte[] get(Bytes key) { + return getInternal(key.get()); } }; @@ -178,13 +179,16 @@ public void init(ProcessorContext context, StateStore root) { openExistingSegments(); - this.changeLogger = this.loggingEnabled ? new RawStoreChangeLogger(name, context) : null; + this.changeLogger = this.loggingEnabled ? new StoreChangeLogger(name, context, WindowStoreUtils.INNER_SERDES) : null; // register and possibly restore the state from the logs context.register(root, loggingEnabled, new StateRestoreCallback() { @Override public void restore(byte[] key, byte[] value) { - putInternal(key, value); + // if the value is null, it means that this record has already been + // deleted while it was captured in the changelog, hence we do not need to put any more. + if (value != null) + putInternal(key, value); } }); @@ -249,7 +253,7 @@ public void put(K key, V value) { byte[] rawKey = putAndReturnInternalKey(key, value, USE_CURRENT_TIMESTAMP); if (rawKey != null && loggingEnabled) { - changeLogger.add(rawKey); + changeLogger.add(Bytes.wrap(rawKey)); changeLogger.maybeLogChange(this.getter); } } @@ -259,7 +263,7 @@ public void put(K key, V value, long timestamp) { byte[] rawKey = putAndReturnInternalKey(key, value, timestamp); if (rawKey != null && loggingEnabled) { - changeLogger.add(rawKey); + changeLogger.add(Bytes.wrap(rawKey)); changeLogger.maybeLogChange(this.getter); } } @@ -281,7 +285,7 @@ private byte[] putAndReturnInternalKey(K key, V value, long t) { if (retainDuplicates) seqnum = (seqnum + 1) & 0x7FFFFFFF; byte[] binaryKey = WindowStoreUtils.toBinaryKey(key, timestamp, seqnum, serdes); - segment.put(binaryKey, serdes.rawValue(value)); + segment.put(Bytes.wrap(binaryKey), serdes.rawValue(value)); return binaryKey; } else { return null; @@ -300,7 +304,7 @@ private void putInternal(byte[] binaryKey, byte[] binaryValue) { // If the record is within the retention period, put it in the store. Segment segment = getSegment(segmentId); if (segment != null) - segment.put(binaryKey, binaryValue); + segment.put(Bytes.wrap(binaryKey), binaryValue); } private byte[] getInternal(byte[] binaryKey) { @@ -308,7 +312,7 @@ private byte[] getInternal(byte[] binaryKey) { Segment segment = getSegment(segmentId); if (segment != null) { - return segment.get(binaryKey); + return segment.get(Bytes.wrap(binaryKey)); } else { return null; } @@ -323,12 +327,12 @@ public WindowStoreIterator fetch(K key, long timeFrom, long timeTo) { byte[] binaryFrom = WindowStoreUtils.toBinaryKey(key, timeFrom, 0, serdes); byte[] binaryTo = WindowStoreUtils.toBinaryKey(key, timeTo, Integer.MAX_VALUE, serdes); - ArrayList> iterators = new ArrayList<>(); + ArrayList> iterators = new ArrayList<>(); for (long segmentId = segFrom; segmentId <= segTo; segmentId++) { Segment segment = getSegment(segmentId); if (segment != null) - iterators.add(segment.range(binaryFrom, binaryTo)); + iterators.add(segment.range(Bytes.wrap(binaryFrom), Bytes.wrap(binaryTo))); } if (iterators.size() > 0) { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java index a4391179c1696..3f848fec3f021 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/StoreChangeLogger.java @@ -27,6 +27,16 @@ import java.util.HashSet; import java.util.Set; +/** + * Store change log collector that batches updates before sending to Kafka. + * + * Note that the use of array-typed keys is discouraged because they result in incorrect caching behavior. + * If you intend to work on byte arrays as key, for example, you may want to wrap them with the {@code Bytes} class, + * i.e. use {@code RocksDBStore} rather than {@code RocksDBStore}. + * + * @param + * @param + */ public class StoreChangeLogger { public interface ValueGetter { diff --git a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java index 0468f491f9d61..3a35d7542fcee 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/KeyValueStoreTestDriver.java @@ -198,19 +198,8 @@ protected KeyValueStoreTestDriver(final StateSerdes serdes) { public void send(ProducerRecord record, Serializer keySerializer, Serializer valueSerializer) { // for byte arrays we need to wrap it for comparison - K key; - if (record.key() instanceof byte[]) { - key = serdes.keyFrom((byte[]) record.key()); - } else { - key = (K) record.key(); - } - - V value; - if (record.key() instanceof byte[]) { - value = serdes.valueFrom((byte[]) record.value()); - } else { - value = (V) record.value(); - } + K key = serdes.keyFrom(keySerializer.serialize(record.topic(), record.key())); + V value = serdes.valueFrom(valueSerializer.serialize(record.topic(), record.value())); recordFlushed(key, value); } diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java index fb0efc9d25bfa..2bfe64425d045 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/AbstractKeyValueStoreTest.java @@ -192,8 +192,6 @@ public void testRestoreWithDefaultSerdes() { } } - - @Test public void testPutIfAbsent() { // Create the test driver ... diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java index 9a477dfc10a69..09f12fbcfe9ac 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/StoreChangeLoggerTest.java @@ -24,10 +24,7 @@ import java.util.Map; import org.apache.kafka.clients.producer.ProducerRecord; -import org.apache.kafka.common.serialization.IntegerDeserializer; -import org.apache.kafka.common.serialization.IntegerSerializer; import org.apache.kafka.common.serialization.Serializer; -import org.apache.kafka.common.serialization.StringSerializer; import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.processor.internals.RecordCollector; @@ -63,8 +60,6 @@ public void send(ProducerRecord record, Serializer keySeria private final StoreChangeLogger changeLogger = new StoreChangeLogger<>(topic, context, StateSerdes.withBuiltinTypes(topic, Integer.class, String.class), 3, 3); - private final StoreChangeLogger rawChangeLogger = new RawStoreChangeLogger(topic, context, 3, 3); - private final StoreChangeLogger.ValueGetter getter = new StoreChangeLogger.ValueGetter() { @Override public String get(Integer key) { @@ -72,16 +67,6 @@ public String get(Integer key) { } }; - private final StoreChangeLogger.ValueGetter rawGetter = new StoreChangeLogger.ValueGetter() { - private IntegerDeserializer deserializer = new IntegerDeserializer(); - private StringSerializer serializer = new StringSerializer(); - - @Override - public byte[] get(byte[] key) { - return serializer.serialize(topic, written.get(deserializer.deserialize(topic, key))); - } - }; - @Test public void testAddRemove() { written.put(0, "zero"); @@ -117,30 +102,4 @@ public void testAddRemove() { assertEquals("three", logged.get(3)); assertEquals("four", logged.get(4)); } - - @Test - public void testRaw() { - IntegerSerializer serializer = new IntegerSerializer(); - - written.put(0, "zero"); - rawChangeLogger.add(serializer.serialize(topic, 0)); - written.put(1, "one"); - rawChangeLogger.add(serializer.serialize(topic, 1)); - written.put(2, "two"); - rawChangeLogger.add(serializer.serialize(topic, 2)); - assertEquals(3, rawChangeLogger.numDirty()); - assertEquals(0, rawChangeLogger.numRemoved()); - - rawChangeLogger.delete(serializer.serialize(topic, 0)); - rawChangeLogger.delete(serializer.serialize(topic, 1)); - written.put(3, "three"); - rawChangeLogger.add(serializer.serialize(topic, 3)); - assertEquals(2, rawChangeLogger.numDirty()); - assertEquals(2, rawChangeLogger.numRemoved()); - - written.put(0, "zero-again"); - rawChangeLogger.add(serializer.serialize(topic, 0)); - assertEquals(3, rawChangeLogger.numDirty()); - assertEquals(1, rawChangeLogger.numRemoved()); - } } From 18dd1986bd354e8c7d75982112fad848780a6b33 Mon Sep 17 00:00:00 2001 From: Jeff Klukas Date: Tue, 26 Apr 2016 09:11:40 -0700 Subject: [PATCH 170/206] MINOR: Fix some copy-pasted Javadoc in StreamsConfig.java This contribution is my original work and I license the work to the Kafka project under the project's open source license. cc guozhangwang miguno ymatsuda Author: Jeff Klukas Reviewers: Michael G. Noll , Guozhang Wang Closes #1270 from jklukas/streams-doc-fix --- .../main/java/org/apache/kafka/streams/StreamsConfig.java | 6 +++--- 1 file changed, 3 insertions(+), 3 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java index 3e0f9550a5465..99eb58f0699fa 100644 --- a/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java +++ b/streams/src/main/java/org/apache/kafka/streams/StreamsConfig.java @@ -63,7 +63,7 @@ public class StreamsConfig extends AbstractConfig { public static final String NUM_STREAM_THREADS_CONFIG = "num.stream.threads"; private static final String NUM_STREAM_THREADS_DOC = "The number of threads to execute stream processing."; - /** num.stream.threads */ + /** num.standby.replicas */ public static final String NUM_STANDBY_REPLICAS_CONFIG = "num.standby.replicas"; private static final String NUM_STANDBY_REPLICAS_DOC = "The number of standby replicas for each task."; @@ -91,11 +91,11 @@ public class StreamsConfig extends AbstractConfig { public static final String REPLICATION_FACTOR_CONFIG = "replication.factor"; public static final String REPLICATION_FACTOR_DOC = "The replication factor for change log topics and repartition topics created by the stream processing application."; - /** replication.factor */ + /** key.serde */ public static final String KEY_SERDE_CLASS_CONFIG = "key.serde"; public static final String KEY_SERDE_CLASS_DOC = "Serializer / deserializer class for key that implements the Serde interface."; - /** replication.factor */ + /** value.serde */ public static final String VALUE_SERDE_CLASS_CONFIG = "value.serde"; public static final String VALUE_SERDE_CLASS_DOC = "Serializer / deserializer class for value that implements the Serde interface."; From 088ab3eaadb2389b52aedc049b6f1f0d4b5fb989 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Tue, 26 Apr 2016 09:59:37 -0700 Subject: [PATCH 171/206] KAFKA-3614: Consolidate duplicate code in KGroupedTableImpl Feel free to review guozhangwang enothereska mjsax . Author: Michael G. Noll Reviewers: Matthias J. Sax, Michael G. Noll, Eno Thereska Closes #1262 from miguno/KAFKA-3614 --- .../kstream/internals/KGroupedTableImpl.java | 73 ++++++------------- 1 file changed, 23 insertions(+), 50 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java index f2e2eed580dcb..f7fe4e53c0386 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImpl.java @@ -67,41 +67,8 @@ public KTable aggregate(Initializer initializer, Serde aggValueSerde, String name) { - String sinkName = topology.newName(KStreamImpl.SINK_NAME); - String sourceName = topology.newName(KStreamImpl.SOURCE_NAME); - String aggregateName = topology.newName(AGGREGATE_NAME); - - String topic = name + REPARTITION_TOPIC_SUFFIX; - - Serializer keySerializer = keySerde == null ? null : keySerde.serializer(); - Deserializer keyDeserializer = keySerde == null ? null : keySerde.deserializer(); - Serializer valueSerializer = valSerde == null ? null : valSerde.serializer(); - Deserializer valueDeserializer = valSerde == null ? null : valSerde.deserializer(); - - ChangedSerializer changedValueSerializer = new ChangedSerializer<>(valueSerializer); - ChangedDeserializer changedValueDeserializer = new ChangedDeserializer<>(valueDeserializer); - ProcessorSupplier> aggregateSupplier = new KTableAggregate<>(name, initializer, adder, subtractor); - - StateStoreSupplier aggregateStore = Stores.create(name) - .withKeys(keySerde) - .withValues(aggValueSerde) - .persistent() - .build(); - - // send the aggregate key-value pairs to the intermediate topic for partitioning - topology.addInternalTopic(topic); - topology.addSink(sinkName, topic, keySerializer, changedValueSerializer, this.name); - - // read the intermediate topic - topology.addSource(sourceName, keyDeserializer, changedValueDeserializer, topic); - - // aggregate the values with the aggregator and local store - topology.addProcessor(aggregateName, aggregateSupplier, sourceName); - topology.addStateStore(aggregateStore, aggregateName); - - // return the KTable representation with the intermediate topic as the sources - return new KTableImpl<>(topology, aggregateName, aggregateSupplier, Collections.singleton(sourceName)); + return doAggregate(aggregateSupplier, aggValueSerde, AGGREGATE_NAME, name); } @Override @@ -113,14 +80,13 @@ public KTable aggregate(Initializer initializer, return aggregate(initializer, adder, substractor, null, name); } - @Override - public KTable reduce(Reducer adder, - Reducer subtractor, - String name) { - + private KTable doAggregate(ProcessorSupplier> aggregateSupplier, + Serde aggValueSerde, + String functionName, + String name) { String sinkName = topology.newName(KStreamImpl.SINK_NAME); String sourceName = topology.newName(KStreamImpl.SOURCE_NAME); - String reduceName = topology.newName(REDUCE_NAME); + String funcName = topology.newName(functionName); String topic = name + REPARTITION_TOPIC_SUFFIX; @@ -132,13 +98,11 @@ public KTable reduce(Reducer adder, ChangedSerializer changedValueSerializer = new ChangedSerializer<>(valueSerializer); ChangedDeserializer changedValueDeserializer = new ChangedDeserializer<>(valueDeserializer); - ProcessorSupplier> aggregateSupplier = new KTableReduce<>(name, adder, subtractor); - StateStoreSupplier aggregateStore = Stores.create(name) - .withKeys(keySerde) - .withValues(valSerde) - .persistent() - .build(); + .withKeys(keySerde) + .withValues(aggValueSerde) + .persistent() + .build(); // send the aggregate key-value pairs to the intermediate topic for partitioning topology.addInternalTopic(topic); @@ -148,11 +112,19 @@ public KTable reduce(Reducer adder, topology.addSource(sourceName, keyDeserializer, changedValueDeserializer, topic); // aggregate the values with the aggregator and local store - topology.addProcessor(reduceName, aggregateSupplier, sourceName); - topology.addStateStore(aggregateStore, reduceName); + topology.addProcessor(funcName, aggregateSupplier, sourceName); + topology.addStateStore(aggregateStore, funcName); // return the KTable representation with the intermediate topic as the sources - return new KTableImpl<>(topology, reduceName, aggregateSupplier, Collections.singleton(sourceName)); + return new KTableImpl<>(topology, funcName, aggregateSupplier, Collections.singleton(sourceName)); + } + + @Override + public KTable reduce(Reducer adder, + Reducer subtractor, + String name) { + ProcessorSupplier> aggregateSupplier = new KTableReduce<>(name, adder, subtractor); + return doAggregate(aggregateSupplier, valSerde, REDUCE_NAME, name); } @Override @@ -177,4 +149,5 @@ public Long apply(K aggKey, V value, Long aggregate) { }, Serdes.Long(), name); } -} + +} \ No newline at end of file From 9d37b9f4b6ba228ff7f7b99c8a0921a971fc03a6 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Tue, 26 Apr 2016 10:53:49 -0700 Subject: [PATCH 172/206] KAKFA-3599: Move WindowStoreUtils to package "internals" Author: Matthias J. Sax Reviewers: Ismael Juma, Michael G. Noll, Guozhang Wang Closes #1266 from mjsax/kafka-3599-minorCodeCleanup --- .../apache/kafka/streams/state/internals/RocksDBStore.java | 1 - .../kafka/streams/state/internals/RocksDBWindowStore.java | 1 - .../kafka/streams/state/{ => internals}/WindowStoreUtils.java | 4 +++- .../kafka/streams/state/internals/RocksDBWindowStoreTest.java | 1 - 4 files changed, 3 insertions(+), 4 deletions(-) rename streams/src/main/java/org/apache/kafka/streams/state/{ => internals}/WindowStoreUtils.java (94%) diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java index 3fef0efd18c4e..37609a0d28b3a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBStore.java @@ -28,7 +28,6 @@ import org.apache.kafka.streams.state.KeyValueStore; import org.apache.kafka.streams.state.StateSerdes; -import org.apache.kafka.streams.state.WindowStoreUtils; import org.rocksdb.BlockBasedTableConfig; import org.rocksdb.CompactionStyle; import org.rocksdb.CompressionType; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java index 4c964c63715d3..803a08997f1b3 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/RocksDBWindowStore.java @@ -30,7 +30,6 @@ import org.apache.kafka.streams.state.StateSerdes; import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; -import org.apache.kafka.streams.state.WindowStoreUtils; import java.io.File; import java.text.SimpleDateFormat; diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreUtils.java similarity index 94% rename from streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java rename to streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreUtils.java index 2f99ad68df4dc..30693e7014e30 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStoreUtils.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/internals/WindowStoreUtils.java @@ -17,11 +17,13 @@ * under the License. */ -package org.apache.kafka.streams.state; +package org.apache.kafka.streams.state.internals; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Bytes; +import org.apache.kafka.streams.state.KeyValueIterator; +import org.apache.kafka.streams.state.StateSerdes; import java.nio.ByteBuffer; diff --git a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java index 502870b2a5448..e9888ada6be6c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/state/internals/RocksDBWindowStoreTest.java @@ -31,7 +31,6 @@ import org.apache.kafka.streams.state.StateSerdes; import org.apache.kafka.streams.state.WindowStore; import org.apache.kafka.streams.state.WindowStoreIterator; -import org.apache.kafka.streams.state.WindowStoreUtils; import org.apache.kafka.test.MockProcessorContext; import org.junit.Test; From f60a3fad38d22654e7560d56730e2dd74b5ae659 Mon Sep 17 00:00:00 2001 From: Dionysis Grigoropoulos Date: Tue, 26 Apr 2016 11:25:23 -0700 Subject: [PATCH 173/206] MINOR: Typo fixes in ReplicaFetchMaxBytesDoc Author: Dionysis Grigoropoulos Reviewers: Ismael Juma Closes #1228 from Erethon/trunk --- core/src/main/scala/kafka/server/KafkaConfig.scala | 2 +- docs/configuration.html | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index dc2a0a0814bf5..5c0d27a229981 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -456,7 +456,7 @@ object KafkaConfig { " the leader will remove the follower from isr" val ReplicaSocketTimeoutMsDoc = "The socket timeout for network requests. Its value should be at least replica.fetch.wait.max.ms" val ReplicaSocketReceiveBufferBytesDoc = "The socket receive buffer for network requests" - val ReplicaFetchMaxBytesDoc = "The number of byes of messages to attempt to fetch" + val ReplicaFetchMaxBytesDoc = "The number of bytes of messages to attempt to fetch" val ReplicaFetchWaitMaxMsDoc = "max wait time for each fetcher request issued by follower replicas. This value should always be less than the " + "replica.lag.time.max.ms at all times to prevent frequent shrinking of ISR for low throughput topics" val ReplicaFetchMinBytesDoc = "Minimum bytes expected for each fetch response. If not enough bytes, wait up to replicaMaxWaitTimeMs" diff --git a/docs/configuration.html b/docs/configuration.html index e5280a5744c9a..f9bd1e456db39 100644 --- a/docs/configuration.html +++ b/docs/configuration.html @@ -207,7 +207,7 @@

          3.3.1 Old Consumer Con fetch.message.max.bytes 1024 * 1024 - The number of byes of messages to attempt to fetch for each topic-partition in each fetch request. These bytes will be read into memory for each partition, so this helps control the memory used by the consumer. The fetch request size must be at least as large as the maximum message size the server allows or else it is possible for the producer to send messages larger than the consumer can fetch. + The number of bytes of messages to attempt to fetch for each topic-partition in each fetch request. These bytes will be read into memory for each partition, so this helps control the memory used by the consumer. The fetch request size must be at least as large as the maximum message size the server allows or else it is possible for the producer to send messages larger than the consumer can fetch. num.consumer.fetchers From 1a73629bb43bbc781e5a968a61f6079365bc75b7 Mon Sep 17 00:00:00 2001 From: Guozhang Wang Date: Tue, 26 Apr 2016 11:39:49 -0700 Subject: [PATCH 174/206] KAFKA-3607: Close KStreamTestDriver upon completing; follow-up fixes to be tracked in KAFKA-3623 Author: Guozhang Wang Reviewers: Eno Thereska, Michael G. Noll, Ismael Juma Closes #1258 from guozhangwang/K3607 --- .../java/org/apache/kafka/test/TestUtils.java | 37 +- .../scala/unit/kafka/utils/TestUtils.scala | 9 +- .../streams/kstream/KStreamBuilderTest.java | 13 +- .../internals/KGroupedTableImplTest.java | 11 +- .../kstream/internals/KStreamBranchTest.java | 13 +- .../kstream/internals/KStreamFilterTest.java | 15 +- .../kstream/internals/KStreamFlatMapTest.java | 13 +- .../internals/KStreamFlatMapValuesTest.java | 13 +- .../kstream/internals/KStreamForeachTest.java | 13 +- .../internals/KStreamKStreamJoinTest.java | 651 +++++++++--------- .../internals/KStreamKStreamLeftJoinTest.java | 341 +++++---- .../internals/KStreamKTableLeftJoinTest.java | 153 ++-- .../kstream/internals/KStreamMapTest.java | 13 +- .../internals/KStreamMapValuesTest.java | 13 +- .../internals/KStreamSelectKeyTest.java | 13 +- .../internals/KStreamTransformTest.java | 13 +- .../internals/KStreamTransformValuesTest.java | 13 +- .../internals/KStreamWindowAggregateTest.java | 455 ++++++------ .../internals/KTableAggregateTest.java | 92 +-- .../kstream/internals/KTableFilterTest.java | 327 +++++---- .../kstream/internals/KTableForeachTest.java | 13 +- .../kstream/internals/KTableImplTest.java | 451 ++++++------ .../internals/KTableKTableJoinTest.java | 394 +++++------ .../internals/KTableKTableLeftJoinTest.java | 397 +++++------ .../internals/KTableKTableOuterJoinTest.java | 426 ++++++------ .../kstream/internals/KTableMapKeysTest.java | 13 +- .../internals/KTableMapValuesTest.java | 381 +++++----- .../kstream/internals/KTableSourceTest.java | 172 ++--- .../KeyValuePrinterProcessorTest.java | 15 +- .../apache/kafka/test/KStreamTestDriver.java | 28 +- .../apache/kafka/test/MockKeyValueMapper.java | 2 +- .../apache/kafka/test/MockValueJoiner.java | 33 + 32 files changed, 2349 insertions(+), 2197 deletions(-) create mode 100644 streams/src/test/java/org/apache/kafka/test/MockValueJoiner.java diff --git a/clients/src/test/java/org/apache/kafka/test/TestUtils.java b/clients/src/test/java/org/apache/kafka/test/TestUtils.java index 027221edeb71d..1bfe578eb00bb 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestUtils.java @@ -20,6 +20,8 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Files; +import java.nio.file.Path; import java.util.ArrayList; import java.util.Collections; import java.util.List; @@ -29,6 +31,7 @@ import org.apache.kafka.common.Cluster; import org.apache.kafka.common.Node; import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.utils.Utils; /** @@ -97,12 +100,44 @@ public static String randomString(int len) { } /** - * Creates an empty file in the default temporary-file directory, using `kafka` as the prefix and `tmp` as the + * Create an empty file in the default temporary-file directory, using `kafka` as the prefix and `tmp` as the * suffix to generate its name. */ public static File tempFile() throws IOException { File file = File.createTempFile("kafka", ".tmp"); file.deleteOnExit(); + + return file; + } + + /** + * Create a temporary relative directory in the default temporary-file directory with the given prefix. + * + * @param prefix The prefix of the temporary directory, if null using "kafka-" as default prefix + */ + public static File tempDirectory(String prefix) throws IOException { + return tempDirectory(null, prefix); + } + + /** + * Create a temporary relative directory in the specified parent directory with the given prefix. + * + * @param parent The parent folder path name, if null using the default temporary-file directory + * @param prefix The prefix of the temporary directory, if null using "kafka-" as default prefix + */ + public static File tempDirectory(Path parent, String prefix) throws IOException { + final File file = parent == null ? + Files.createTempDirectory(prefix == null ? "kafka-" : prefix).toFile() : + Files.createTempDirectory(parent, prefix == null ? "kafka-" : prefix).toFile(); + file.deleteOnExit(); + + Runtime.getRuntime().addShutdownHook(new Thread() { + @Override + public void run() { + Utils.delete(file); + } + }); + return file; } diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index be7741df5cbaa..6bd6c63f7b751 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -95,15 +95,8 @@ object TestUtils extends Logging { def tempRelativeDir(parent: String): File = { val parentFile = new File(parent) parentFile.mkdirs() - val f = Files.createTempDirectory(parentFile.toPath, "kafka-").toFile - f.deleteOnExit() - Runtime.getRuntime().addShutdownHook(new Thread() { - override def run() = { - Utils.delete(f) - } - }) - f + org.apache.kafka.test.TestUtils.tempDirectory(parentFile.toPath, "kafka-"); } /** diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java index e75b595cc513d..cdf28db4a7a3e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/KStreamBuilderTest.java @@ -22,12 +22,23 @@ import org.apache.kafka.streams.errors.TopologyBuilderException; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorSupplier; +import org.junit.After; import org.junit.Test; import static org.junit.Assert.assertEquals; public class KStreamBuilderTest { + private KStreamTestDriver driver = null; + + @After + public void cleanup() { + if (driver != null) { + driver.close(); + } + driver = null; + } + @Test(expected = TopologyBuilderException.class) public void testFrom() { final KStreamBuilder builder = new KStreamBuilder(); @@ -66,7 +77,7 @@ public void testMerge() { MockProcessorSupplier processorSupplier = new MockProcessorSupplier<>(); merged.process(processorSupplier); - KStreamTestDriver driver = new KStreamTestDriver(builder); + driver = new KStreamTestDriver(builder); driver.setTime(0L); driver.process(topic1, "A", "aa"); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java index 9eeea20ebb6ba..fc0451aea949e 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KGroupedTableImplTest.java @@ -18,19 +18,17 @@ package org.apache.kafka.streams.kstream.internals; import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorSupplier; -import org.junit.After; +import org.apache.kafka.test.TestUtils; import org.junit.Before; import org.junit.Test; import java.io.File; import java.io.IOException; -import java.nio.file.Files; import java.util.Arrays; import java.util.List; @@ -43,12 +41,7 @@ public class KGroupedTableImplTest { @Before public void setUp() throws IOException { - stateDir = Files.createTempDirectory("test").toFile(); - } - - @After - public void tearDown() throws IOException { - Utils.delete(stateDir); + stateDir = TestUtils.tempDirectory("kafka-test"); } @SuppressWarnings("unchecked") diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java index e04a2738d9fb4..0650b95fef86c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamBranchTest.java @@ -23,6 +23,7 @@ import org.apache.kafka.streams.kstream.Predicate; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorSupplier; +import org.junit.After; import org.junit.Test; import java.lang.reflect.Array; @@ -33,6 +34,16 @@ public class KStreamBranchTest { private String topicName = "topic"; + private KStreamTestDriver driver = null; + + @After + public void cleanup() { + if (driver != null) { + driver.close(); + } + driver = null; + } + @SuppressWarnings("unchecked") @Test public void testKStreamBranch() { @@ -74,7 +85,7 @@ public boolean test(Integer key, String value) { branches[i].process(processors[i]); } - KStreamTestDriver driver = new KStreamTestDriver(builder); + driver = new KStreamTestDriver(builder); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java index 75465c85bb8c2..4be8513549198 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFilterTest.java @@ -24,6 +24,7 @@ import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorSupplier; +import org.junit.After; import org.junit.Test; import static org.junit.Assert.assertEquals; @@ -32,6 +33,16 @@ public class KStreamFilterTest { private String topicName = "topic"; + private KStreamTestDriver driver = null; + + @After + public void cleanup() { + if (driver != null) { + driver.close(); + } + driver = null; + } + private Predicate isMultipleOfThree = new Predicate() { @Override public boolean test(Integer key, String value) { @@ -51,7 +62,7 @@ public void testFilter() { stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName); stream.filter(isMultipleOfThree).process(processor); - KStreamTestDriver driver = new KStreamTestDriver(builder); + driver = new KStreamTestDriver(builder); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); } @@ -71,7 +82,7 @@ public void testFilterNot() { stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName); stream.filterNot(isMultipleOfThree).process(processor); - KStreamTestDriver driver = new KStreamTestDriver(builder); + driver = new KStreamTestDriver(builder); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java index bc85757ef450a..da57d4be8c562 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapTest.java @@ -24,6 +24,7 @@ import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorSupplier; +import org.junit.After; import org.junit.Test; import static org.junit.Assert.assertEquals; @@ -34,6 +35,16 @@ public class KStreamFlatMapTest { private String topicName = "topic"; + private KStreamTestDriver driver = null; + + @After + public void cleanup() { + if (driver != null) { + driver.close(); + } + driver = null; + } + @Test public void testFlatMap() { KStreamBuilder builder = new KStreamBuilder(); @@ -59,7 +70,7 @@ public Iterable> apply(Integer key, String value) { stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName); stream.flatMap(mapper).process(processor); - KStreamTestDriver driver = new KStreamTestDriver(builder); + driver = new KStreamTestDriver(builder); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java index 63f56364f57b1..9d1141bdab36c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamFlatMapValuesTest.java @@ -23,6 +23,7 @@ import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorSupplier; +import org.junit.After; import org.junit.Test; import java.util.ArrayList; @@ -34,6 +35,16 @@ public class KStreamFlatMapValuesTest { private String topicName = "topic"; + private KStreamTestDriver driver = null; + + @After + public void cleanup() { + if (driver != null) { + driver.close(); + } + driver = null; + } + @Test public void testFlatMapValues() { KStreamBuilder builder = new KStreamBuilder(); @@ -58,7 +69,7 @@ public Iterable apply(String value) { stream = builder.stream(Serdes.Integer(), Serdes.String(), topicName); stream.flatMapValues(mapper).process(processor); - KStreamTestDriver driver = new KStreamTestDriver(builder); + driver = new KStreamTestDriver(builder); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamForeachTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamForeachTest.java index d0a182d81595d..0bc5e7701a9d7 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamForeachTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamForeachTest.java @@ -24,6 +24,7 @@ import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.test.KStreamTestDriver; +import org.junit.After; import org.junit.Test; import java.util.List; import java.util.Locale; @@ -39,6 +40,16 @@ public class KStreamForeachTest { final private Serde intSerde = Serdes.Integer(); final private Serde stringSerde = Serdes.String(); + private KStreamTestDriver driver = null; + + @After + public void cleanup() { + if (driver != null) { + driver.close(); + } + driver = null; + } + @Test public void testForeach() { // Given @@ -71,7 +82,7 @@ public void apply(Integer key, String value) { stream.foreach(action); // Then - KStreamTestDriver driver = new KStreamTestDriver(builder); + driver = new KStreamTestDriver(builder); for (KeyValue record: inputRecords) { driver.process(topicName, record.key, record.value); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java index 19a941177e00d..6b0828a62aa78 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamJoinTest.java @@ -19,17 +19,19 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.JoinWindows; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; -import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockValueJoiner; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.io.File; -import java.nio.file.Files; +import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.HashSet; @@ -39,460 +41,447 @@ public class KStreamKStreamJoinTest { - private String topic1 = "topic1"; - private String topic2 = "topic2"; + final private String topic1 = "topic1"; + final private String topic2 = "topic2"; final private Serde intSerde = Serdes.Integer(); final private Serde stringSerde = Serdes.String(); - private ValueJoiner joiner = new ValueJoiner() { - @Override - public String apply(String value1, String value2) { - return value1 + "+" + value2; + private KStreamTestDriver driver = null; + private File stateDir = null; + + @After + public void tearDown() { + if (driver != null) { + driver.close(); } - }; + driver = null; + } + + @Before + public void setUp() throws IOException { + stateDir = TestUtils.tempDirectory("kafka-test"); + } @Test public void testJoin() throws Exception { - File baseDir = Files.createTempDirectory("test").toFile(); - try { - - KStreamBuilder builder = new KStreamBuilder(); - - final int[] expectedKeys = new int[]{0, 1, 2, 3}; - - KStream stream1; - KStream stream2; - KStream joined; - MockProcessorSupplier processor; + KStreamBuilder builder = new KStreamBuilder(); - processor = new MockProcessorSupplier<>(); - stream1 = builder.stream(intSerde, stringSerde, topic1); - stream2 = builder.stream(intSerde, stringSerde, topic2); - joined = stream1.join(stream2, joiner, JoinWindows.of("test").within(100), intSerde, stringSerde, stringSerde); - joined.process(processor); + final int[] expectedKeys = new int[]{0, 1, 2, 3}; - Collection> copartitionGroups = builder.copartitionGroups(); + KStream stream1; + KStream stream2; + KStream joined; + MockProcessorSupplier processor; - assertEquals(1, copartitionGroups.size()); - assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); + processor = new MockProcessorSupplier<>(); + stream1 = builder.stream(intSerde, stringSerde, topic1); + stream2 = builder.stream(intSerde, stringSerde, topic2); + joined = stream1.join(stream2, MockValueJoiner.STRING_JOINER, JoinWindows.of("test").within(100), intSerde, stringSerde, stringSerde); + joined.process(processor); - KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir); - driver.setTime(0L); + Collection> copartitionGroups = builder.copartitionGroups(); - // push two items to the primary stream. the other window is empty - // w1 = {} - // w2 = {} - // --> w1 = { 0:X1, 1:X1 } - // w2 = {} + assertEquals(1, copartitionGroups.size()); + assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); - for (int i = 0; i < 2; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + driver = new KStreamTestDriver(builder, stateDir); + driver.setTime(0L); - processor.checkAndClearProcessResult(); + // push two items to the primary stream. the other window is empty + // w1 = {} + // w2 = {} + // --> w1 = { 0:X1, 1:X1 } + // w2 = {} - // push two items to the other stream. this should produce two items. - // w1 = { 0:X0, 1:X1 } - // w2 = {} - // --> w1 = { 0:X1, 1:X1 } - // w2 = { 0:Y0, 1:Y1 } + for (int i = 0; i < 2; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); - } + processor.checkAndClearProcessResult(); - processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); + // push two items to the other stream. this should produce two items. + // w1 = { 0:X0, 1:X1 } + // w2 = {} + // --> w1 = { 0:X1, 1:X1 } + // w2 = { 0:Y0, 1:Y1 } - // push all four items to the primary stream. this should produce two items. - // w1 = { 0:X0, 1:X1 } - // w2 = { 0:Y0, 1:Y1 } - // --> w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 } - // w2 = { 0:Y0, 1:Y1 } + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); - processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); + // push all four items to the primary stream. this should produce two items. + // w1 = { 0:X0, 1:X1 } + // w2 = { 0:Y0, 1:Y1 } + // --> w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 } + // w2 = { 0:Y0, 1:Y1 } - // push all items to the other stream. this should produce six items. - // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 } - // w2 = { 0:Y0, 1:Y1 } - // --> w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 } - // w2 = { 0:Y0, 1:Y1, 0:YY0, 1:YY1, 2:YY2, 3:YY3 } + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); - processor.checkAndClearProcessResult("0:X0+YY0", "0:X0+YY0", "1:X1+YY1", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + // push all items to the other stream. this should produce six items. + // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 } + // w2 = { 0:Y0, 1:Y1 } + // --> w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 } + // w2 = { 0:Y0, 1:Y1, 0:YY0, 1:YY1, 2:YY2, 3:YY3 } - // push all four items to the primary stream. this should produce six items. - // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 } - // w2 = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 - // --> w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3, 0:XX0, 1:XX1, 2:XX2, 3:XX3 } - // w2 = { 0:Y0, 1:Y1, 0:YY0, 1:YY1, 2:YY2, 3:YY3 } + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:X0+YY0", "0:X0+YY0", "1:X1+YY1", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); - processor.checkAndClearProcessResult("0:XX0+Y0", "0:XX0+YY0", "1:XX1+Y1", "1:XX1+YY1", "2:XX2+YY2", "3:XX3+YY3"); + // push all four items to the primary stream. this should produce six items. + // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 } + // w2 = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 + // --> w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3, 0:XX0, 1:XX1, 2:XX2, 3:XX3 } + // w2 = { 0:Y0, 1:Y1, 0:YY0, 1:YY1, 2:YY2, 3:YY3 } - // push two items to the other stream. this should produce six item. - // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3, 0:XX0, 1:XX1, 2:XX2, 3:XX3 } - // w2 = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 - // --> w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3, 0:XX0, 1:XX1, 2:XX2, 3:XX3 } - // w2 = { 0:Y0, 1:Y1, 0:YY0, 1:YY1, 2:YY2, 3:YY3, 0:YYY0, 1:YYY1 } + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], "YYY" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:XX0+Y0", "0:XX0+YY0", "1:XX1+Y1", "1:XX1+YY1", "2:XX2+YY2", "3:XX3+YY3"); - processor.checkAndClearProcessResult("0:X0+YYY0", "0:X0+YYY0", "0:XX0+YYY0", "1:X1+YYY1", "1:X1+YYY1", "1:XX1+YYY1"); + // push two items to the other stream. this should produce six item. + // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3, 0:XX0, 1:XX1, 2:XX2, 3:XX3 } + // w2 = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 + // --> w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3, 0:XX0, 1:XX1, 2:XX2, 3:XX3 } + // w2 = { 0:Y0, 1:Y1, 0:YY0, 1:YY1, 2:YY2, 3:YY3, 0:YYY0, 1:YYY1 } - } finally { - Utils.delete(baseDir); + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], "YYY" + expectedKeys[i]); } + + processor.checkAndClearProcessResult("0:X0+YYY0", "0:X0+YYY0", "0:XX0+YYY0", "1:X1+YYY1", "1:X1+YYY1", "1:XX1+YYY1"); } @Test public void testOuterJoin() throws Exception { - File baseDir = Files.createTempDirectory("test").toFile(); - try { + KStreamBuilder builder = new KStreamBuilder(); - KStreamBuilder builder = new KStreamBuilder(); + final int[] expectedKeys = new int[]{0, 1, 2, 3}; - final int[] expectedKeys = new int[]{0, 1, 2, 3}; + KStream stream1; + KStream stream2; + KStream joined; + MockProcessorSupplier processor; - KStream stream1; - KStream stream2; - KStream joined; - MockProcessorSupplier processor; + processor = new MockProcessorSupplier<>(); + stream1 = builder.stream(intSerde, stringSerde, topic1); + stream2 = builder.stream(intSerde, stringSerde, topic2); + joined = stream1.outerJoin(stream2, MockValueJoiner.STRING_JOINER, JoinWindows.of("test").within(100), intSerde, stringSerde, stringSerde); + joined.process(processor); - processor = new MockProcessorSupplier<>(); - stream1 = builder.stream(intSerde, stringSerde, topic1); - stream2 = builder.stream(intSerde, stringSerde, topic2); - joined = stream1.outerJoin(stream2, joiner, JoinWindows.of("test").within(100), intSerde, stringSerde, stringSerde); - joined.process(processor); + Collection> copartitionGroups = builder.copartitionGroups(); - Collection> copartitionGroups = builder.copartitionGroups(); + assertEquals(1, copartitionGroups.size()); + assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); - assertEquals(1, copartitionGroups.size()); - assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); + driver = new KStreamTestDriver(builder, stateDir); + driver.setTime(0L); - KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir); - driver.setTime(0L); + // push two items to the primary stream. the other window is empty.this should produce two items + // w1 = {} + // w2 = {} + // --> w1 = { 0:X1, 1:X1 } + // w2 = {} - // push two items to the primary stream. the other window is empty.this should produce two items - // w1 = {} - // w2 = {} - // --> w1 = { 0:X1, 1:X1 } - // w2 = {} - - for (int i = 0; i < 2; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } - - processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); - - // push two items to the other stream. this should produce two items. - // w1 = { 0:X0, 1:X1 } - // w2 = {} - // --> w1 = { 0:X1, 1:X1 } - // w2 = { 0:Y0, 1:Y1 } + for (int i = 0; i < 2; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); - processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); + // push two items to the other stream. this should produce two items. + // w1 = { 0:X0, 1:X1 } + // w2 = {} + // --> w1 = { 0:X1, 1:X1 } + // w2 = { 0:Y0, 1:Y1 } - // push all four items to the primary stream. this should produce four items. - // w1 = { 0:X0, 1:X1 } - // w2 = { 0:Y0, 1:Y1 } - // --> w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 } - // w2 = { 0:Y0, 1:Y1 } + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); - processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null"); + // push all four items to the primary stream. this should produce four items. + // w1 = { 0:X0, 1:X1 } + // w2 = { 0:Y0, 1:Y1 } + // --> w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 } + // w2 = { 0:Y0, 1:Y1 } - // push all items to the other stream. this should produce six items. - // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 } - // w2 = { 0:Y0, 1:Y1 } - // --> w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 } - // w2 = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 } + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null"); - processor.checkAndClearProcessResult("0:X0+YY0", "0:X0+YY0", "1:X1+YY1", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + // push all items to the other stream. this should produce six items. + // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 } + // w2 = { 0:Y0, 1:Y1 } + // --> w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 } + // w2 = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 } - // push all four items to the primary stream. this should produce six items. - // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 } - // w2 = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 - // --> w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3, 0:XX0, 1:XX1, 2:XX2, 3:XX3 } - // w2 = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 } + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:X0+YY0", "0:X0+YY0", "1:X1+YY1", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); - processor.checkAndClearProcessResult("0:XX0+Y0", "0:XX0+YY0", "1:XX1+Y1", "1:XX1+YY1", "2:XX2+YY2", "3:XX3+YY3"); + // push all four items to the primary stream. this should produce six items. + // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3 } + // w2 = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 + // --> w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3, 0:XX0, 1:XX1, 2:XX2, 3:XX3 } + // w2 = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 } - // push two items to the other stream. this should produce six item. - // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3, 0:XX0, 1:XX1, 2:XX2, 3:XX3 } - // w2 = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 - // --> w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3, 0:XX0, 1:XX1, 2:XX2, 3:XX3 } - // w2 = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3, 0:YYY0, 1:YYY1 } + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], "YYY" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:XX0+Y0", "0:XX0+YY0", "1:XX1+Y1", "1:XX1+YY1", "2:XX2+YY2", "3:XX3+YY3"); - processor.checkAndClearProcessResult("0:X0+YYY0", "0:X0+YYY0", "0:XX0+YYY0", "1:X1+YYY1", "1:X1+YYY1", "1:XX1+YYY1"); + // push two items to the other stream. this should produce six item. + // w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3, 0:XX0, 1:XX1, 2:XX2, 3:XX3 } + // w2 = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 + // --> w1 = { 0:X1, 1:X1, 0:X0, 1:X1, 2:X2, 3:X3, 0:XX0, 1:XX1, 2:XX2, 3:XX3 } + // w2 = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3, 0:YYY0, 1:YYY1 } - } finally { - Utils.delete(baseDir); + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], "YYY" + expectedKeys[i]); } + + processor.checkAndClearProcessResult("0:X0+YYY0", "0:X0+YYY0", "0:XX0+YYY0", "1:X1+YYY1", "1:X1+YYY1", "1:XX1+YYY1"); } @Test public void testWindowing() throws Exception { - File baseDir = Files.createTempDirectory("test").toFile(); - try { - - long time = 0L; - - KStreamBuilder builder = new KStreamBuilder(); - - final int[] expectedKeys = new int[]{0, 1, 2, 3}; + long time = 0L; - KStream stream1; - KStream stream2; - KStream joined; - MockProcessorSupplier processor; + KStreamBuilder builder = new KStreamBuilder(); - processor = new MockProcessorSupplier<>(); - stream1 = builder.stream(intSerde, stringSerde, topic1); - stream2 = builder.stream(intSerde, stringSerde, topic2); - joined = stream1.join(stream2, joiner, JoinWindows.of("test").within(100), intSerde, stringSerde, stringSerde); - joined.process(processor); + final int[] expectedKeys = new int[]{0, 1, 2, 3}; - Collection> copartitionGroups = builder.copartitionGroups(); + KStream stream1; + KStream stream2; + KStream joined; + MockProcessorSupplier processor; - assertEquals(1, copartitionGroups.size()); - assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); + processor = new MockProcessorSupplier<>(); + stream1 = builder.stream(intSerde, stringSerde, topic1); + stream2 = builder.stream(intSerde, stringSerde, topic2); + joined = stream1.join(stream2, MockValueJoiner.STRING_JOINER, JoinWindows.of("test").within(100), intSerde, stringSerde, stringSerde); + joined.process(processor); - KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir); - driver.setTime(time); + Collection> copartitionGroups = builder.copartitionGroups(); - // push two items to the primary stream. the other window is empty. this should produce no items. - // w1 = {} - // w2 = {} - // --> w1 = { 0:X1, 1:X1 } - // w2 = {} + assertEquals(1, copartitionGroups.size()); + assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); - for (int i = 0; i < 2; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + driver = new KStreamTestDriver(builder, stateDir); + driver.setTime(time); - processor.checkAndClearProcessResult(); + // push two items to the primary stream. the other window is empty. this should produce no items. + // w1 = {} + // w2 = {} + // --> w1 = { 0:X1, 1:X1 } + // w2 = {} - // push two items to the other stream. this should produce two items. - // w1 = { 0:X0, 1:X1 } - // w2 = {} - // --> w1 = { 0:X1, 1:X1 } - // w2 = { 0:Y0, 1:Y1 } + for (int i = 0; i < 2; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); - } + processor.checkAndClearProcessResult(); - processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); + // push two items to the other stream. this should produce two items. + // w1 = { 0:X0, 1:X1 } + // w2 = {} + // --> w1 = { 0:X1, 1:X1 } + // w2 = { 0:Y0, 1:Y1 } - // clear logically - time = 1000L; + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.setTime(time + i); - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } - processor.checkAndClearProcessResult(); + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); - // gradually expires items in w1 - // w1 = { 0:X0, 1:X1, 2:X2, 3:X3 } + // clear logically + time = 1000L; - time = 1000 + 100L; - driver.setTime(time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.setTime(time + i); + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } + processor.checkAndClearProcessResult(); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + // gradually expires items in w1 + // w1 = { 0:X0, 1:X1, 2:X2, 3:X3 } - processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + time = 1000 + 100L; + driver.setTime(time); - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("2:X2+YY2", "3:X3+YY3"); + processor.checkAndClearProcessResult("1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("3:X3+YY3"); + processor.checkAndClearProcessResult("2:X2+YY2", "3:X3+YY3"); - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - processor.checkAndClearProcessResult(); + processor.checkAndClearProcessResult("3:X3+YY3"); - // go back to the time before expiration + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - time = 1000L - 100L - 1L; - driver.setTime(time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + processor.checkAndClearProcessResult(); - processor.checkAndClearProcessResult(); + // go back to the time before expiration - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + time = 1000L - 100L - 1L; + driver.setTime(time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("0:X0+YY0"); + processor.checkAndClearProcessResult(); - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1"); + processor.checkAndClearProcessResult("0:X0+YY0"); - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2"); + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1"); - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2"); - // clear (logically) - time = 2000L; + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.setTime(time + i); - driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); - processor.checkAndClearProcessResult(); + // clear (logically) + time = 2000L; - // gradually expires items in w2 - // w2 = { 0:Y0, 1:Y1, 2:Y2, 3:Y3 } + for (int i = 0; i < expectedKeys.length; i++) { + driver.setTime(time + i); + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); + } - time = 2000L + 100L; - driver.setTime(time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + processor.checkAndClearProcessResult(); - processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); + // gradually expires items in w2 + // w2 = { 0:Y0, 1:Y1, 2:Y2, 3:Y3 } - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + time = 2000L + 100L; + driver.setTime(time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); + processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("2:XX2+Y2", "3:XX3+Y3"); + processor.checkAndClearProcessResult("1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("3:XX3+Y3"); + processor.checkAndClearProcessResult("2:XX2+Y2", "3:XX3+Y3"); - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - processor.checkAndClearProcessResult(); + processor.checkAndClearProcessResult("3:XX3+Y3"); - // go back to the time before expiration + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - time = 2000L - 100L - 1L; - driver.setTime(time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + processor.checkAndClearProcessResult(); - processor.checkAndClearProcessResult(); + // go back to the time before expiration - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + time = 2000L - 100L - 1L; + driver.setTime(time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("0:XX0+Y0"); + processor.checkAndClearProcessResult(); - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1"); + processor.checkAndClearProcessResult("0:XX0+Y0"); - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2"); + processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1"); - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); + processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2"); - } finally { - Utils.delete(baseDir); + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - } + processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java index 65226d338f28c..65a4b542e583d 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKStreamLeftJoinTest.java @@ -19,17 +19,19 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.JoinWindows; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; -import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockValueJoiner; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.io.File; -import java.nio.file.Files; +import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.HashSet; @@ -39,245 +41,240 @@ public class KStreamKStreamLeftJoinTest { - private String topic1 = "topic1"; - private String topic2 = "topic2"; + final private String topic1 = "topic1"; + final private String topic2 = "topic2"; final private Serde intSerde = Serdes.Integer(); final private Serde stringSerde = Serdes.String(); - private ValueJoiner joiner = new ValueJoiner() { - @Override - public String apply(String value1, String value2) { - return value1 + "+" + value2; - } - }; + private KStreamTestDriver driver = null; + private File stateDir = null; - @Test - public void testLeftJoin() throws Exception { - File baseDir = Files.createTempDirectory("test").toFile(); - try { + @After + public void tearDown() { + if (driver != null) { + driver.close(); + } + driver = null; + } - KStreamBuilder builder = new KStreamBuilder(); + @Before + public void setUp() throws IOException { + stateDir = TestUtils.tempDirectory("kafka-test"); + } - final int[] expectedKeys = new int[]{0, 1, 2, 3}; - KStream stream1; - KStream stream2; - KStream joined; - MockProcessorSupplier processor; + @Test + public void testLeftJoin() throws Exception { + final KStreamBuilder builder = new KStreamBuilder(); - processor = new MockProcessorSupplier<>(); - stream1 = builder.stream(intSerde, stringSerde, topic1); - stream2 = builder.stream(intSerde, stringSerde, topic2); - joined = stream1.leftJoin(stream2, joiner, JoinWindows.of("test").within(100), intSerde, stringSerde); - joined.process(processor); + final int[] expectedKeys = new int[]{0, 1, 2, 3}; - Collection> copartitionGroups = builder.copartitionGroups(); + KStream stream1; + KStream stream2; + KStream joined; + MockProcessorSupplier processor; - assertEquals(1, copartitionGroups.size()); - assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); + processor = new MockProcessorSupplier<>(); + stream1 = builder.stream(intSerde, stringSerde, topic1); + stream2 = builder.stream(intSerde, stringSerde, topic2); + joined = stream1.leftJoin(stream2, MockValueJoiner.STRING_JOINER, JoinWindows.of("test").within(100), intSerde, stringSerde); + joined.process(processor); - KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir); - driver.setTime(0L); + Collection> copartitionGroups = builder.copartitionGroups(); - // push two items to the primary stream. the other window is empty - // w {} - // --> w = {} + assertEquals(1, copartitionGroups.size()); + assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); - for (int i = 0; i < 2; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + driver = new KStreamTestDriver(builder, stateDir); + driver.setTime(0L); - processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); + // push two items to the primary stream. the other window is empty + // w {} + // --> w = {} - // push two items to the other stream. this should produce two items. - // w {} - // --> w = { 0:Y0, 1:Y1 } + for (int i = 0; i < 2; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); - processor.checkAndClearProcessResult(); + // push two items to the other stream. this should produce two items. + // w {} + // --> w = { 0:Y0, 1:Y1 } - // push all four items to the primary stream. this should produce four items. - // w = { 0:Y0, 1:Y1 } - // --> w = { 0:Y0, 1:Y1 } + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + processor.checkAndClearProcessResult(); - processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null"); + // push all four items to the primary stream. this should produce four items. + // w = { 0:Y0, 1:Y1 } + // --> w = { 0:Y0, 1:Y1 } - // push all items to the other stream. this should produce no items. - // w = { 0:Y0, 1:Y1 } - // --> w = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 } + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null"); - processor.checkAndClearProcessResult(); + // push all items to the other stream. this should produce no items. + // w = { 0:Y0, 1:Y1 } + // --> w = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 } - // push all four items to the primary stream. this should produce four items. - // w = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 - // --> w = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 } + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + processor.checkAndClearProcessResult(); - processor.checkAndClearProcessResult("0:XX0+Y0", "0:XX0+YY0", "1:XX1+Y1", "1:XX1+YY1", "2:XX2+YY2", "3:XX3+YY3"); + // push all four items to the primary stream. this should produce four items. + // w = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 + // --> w = { 0:Y0, 1:Y1, 0:YY0, 0:YY0, 1:YY1, 2:YY2, 3:YY3 } - } finally { - Utils.delete(baseDir); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } + + processor.checkAndClearProcessResult("0:XX0+Y0", "0:XX0+YY0", "1:XX1+Y1", "1:XX1+YY1", "2:XX2+YY2", "3:XX3+YY3"); } @Test public void testWindowing() throws Exception { - File baseDir = Files.createTempDirectory("test").toFile(); - try { - - long time = 0L; - - KStreamBuilder builder = new KStreamBuilder(); + final KStreamBuilder builder = new KStreamBuilder(); - final int[] expectedKeys = new int[]{0, 1, 2, 3}; + final int[] expectedKeys = new int[]{0, 1, 2, 3}; - KStream stream1; - KStream stream2; - KStream joined; - MockProcessorSupplier processor; + long time = 0L; - processor = new MockProcessorSupplier<>(); - stream1 = builder.stream(intSerde, stringSerde, topic1); - stream2 = builder.stream(intSerde, stringSerde, topic2); - joined = stream1.leftJoin(stream2, joiner, JoinWindows.of("test").within(100), intSerde, stringSerde); - joined.process(processor); + KStream stream1; + KStream stream2; + KStream joined; + MockProcessorSupplier processor; - Collection> copartitionGroups = builder.copartitionGroups(); + processor = new MockProcessorSupplier<>(); + stream1 = builder.stream(intSerde, stringSerde, topic1); + stream2 = builder.stream(intSerde, stringSerde, topic2); + joined = stream1.leftJoin(stream2, MockValueJoiner.STRING_JOINER, JoinWindows.of("test").within(100), intSerde, stringSerde); + joined.process(processor); - assertEquals(1, copartitionGroups.size()); - assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); + Collection> copartitionGroups = builder.copartitionGroups(); - KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir); - driver.setTime(time); + assertEquals(1, copartitionGroups.size()); + assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); - // push two items to the primary stream. the other window is empty. this should produce two items - // w = {} - // --> w = {} + driver = new KStreamTestDriver(builder, stateDir); + driver.setTime(time); - for (int i = 0; i < 2; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + // push two items to the primary stream. the other window is empty. this should produce two items + // w = {} + // --> w = {} - processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); - - // push two items to the other stream. this should produce no items. - // w = {} - // --> w = { 0:Y0, 1:Y1 } + for (int i = 0; i < 2; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); - processor.checkAndClearProcessResult(); + // push two items to the other stream. this should produce no items. + // w = {} + // --> w = { 0:Y0, 1:Y1 } - // clear logically - time = 1000L; + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); + } - // push all items to the other stream. this should produce no items. - // w = {} - // --> w = { 0:Y0, 1:Y1, 2:Y2, 3:Y3 } - for (int i = 0; i < expectedKeys.length; i++) { - driver.setTime(time + i); - driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); - } + processor.checkAndClearProcessResult(); - processor.checkAndClearProcessResult(); + // clear logically + time = 1000L; - // gradually expire items in window. - // w = { 0:Y0, 1:Y1, 2:Y2, 3:Y3 } + // push all items to the other stream. this should produce no items. + // w = {} + // --> w = { 0:Y0, 1:Y1, 2:Y2, 3:Y3 } + for (int i = 0; i < expectedKeys.length; i++) { + driver.setTime(time + i); + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); + } - time = 1000L + 100L; - driver.setTime(time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + processor.checkAndClearProcessResult(); - processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); + // gradually expire items in window. + // w = { 0:Y0, 1:Y1, 2:Y2, 3:Y3 } - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + time = 1000L + 100L; + driver.setTime(time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); + processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+Y2", "3:XX3+Y3"); + processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+null", "3:XX3+Y3"); + processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+Y2", "3:XX3+Y3"); - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+null", "3:XX3+null"); + processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+null", "3:XX3+Y3"); - // go back to the time before expiration + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - time = 1000L - 100L - 1L; - driver.setTime(time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+null", "3:XX3+null"); - processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+null", "3:XX3+null"); + // go back to the time before expiration - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + time = 1000L - 100L - 1L; + driver.setTime(time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+null", "2:XX2+null", "3:XX3+null"); + processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+null", "3:XX3+null"); - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+null", "3:XX3+null"); + processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+null", "2:XX2+null", "3:XX3+null"); - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+null"); + processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+null", "3:XX3+null"); - driver.setTime(++time); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); + processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+null"); - } finally { - Utils.delete(baseDir); + driver.setTime(++time); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } - } + processor.checkAndClearProcessResult("0:XX0+Y0", "1:XX1+Y1", "2:XX2+Y2", "3:XX3+Y3"); + } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java index 3acb59a4a0c29..2c6108b368bfa 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamKTableLeftJoinTest.java @@ -20,19 +20,20 @@ import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; -import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.kstream.KeyValueMapper; -import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockKeyValueMapper; import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockValueJoiner; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.io.File; -import java.nio.file.Files; +import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.HashSet; @@ -42,111 +43,105 @@ public class KStreamKTableLeftJoinTest { - private String topic1 = "topic1"; - private String topic2 = "topic2"; + final private String topic1 = "topic1"; + final private String topic2 = "topic2"; - final private Serde intSerde = new Serdes.IntegerSerde(); - final private Serde stringSerde = new Serdes.StringSerde(); + final private Serde intSerde = Serdes.Integer(); + final private Serde stringSerde = Serdes.String(); - private ValueJoiner joiner = new ValueJoiner() { - @Override - public String apply(String value1, String value2) { - return value1 + "+" + value2; + private KStreamTestDriver driver = null; + private File stateDir = null; + + @After + public void tearDown() { + if (driver != null) { + driver.close(); } - }; + driver = null; + } - private KeyValueMapper> keyValueMapper = - new KeyValueMapper>() { - @Override - public KeyValue apply(Integer key, String value) { - return KeyValue.pair(key, value); - } - }; + @Before + public void setUp() throws IOException { + stateDir = TestUtils.tempDirectory("kafka-test"); + } @Test public void testJoin() throws Exception { - File baseDir = Files.createTempDirectory("test").toFile(); - try { - - KStreamBuilder builder = new KStreamBuilder(); - - final int[] expectedKeys = new int[]{0, 1, 2, 3}; - - KStream stream; - KTable table; - MockProcessorSupplier processor; + KStreamBuilder builder = new KStreamBuilder(); - processor = new MockProcessorSupplier<>(); - stream = builder.stream(intSerde, stringSerde, topic1); - table = builder.table(intSerde, stringSerde, topic2); - stream.leftJoin(table, joiner).process(processor); + final int[] expectedKeys = new int[]{0, 1, 2, 3}; - Collection> copartitionGroups = builder.copartitionGroups(); + KStream stream; + KTable table; + MockProcessorSupplier processor; - assertEquals(1, copartitionGroups.size()); - assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); + processor = new MockProcessorSupplier<>(); + stream = builder.stream(intSerde, stringSerde, topic1); + table = builder.table(intSerde, stringSerde, topic2); + stream.leftJoin(table, MockValueJoiner.STRING_JOINER).process(processor); - KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir); - driver.setTime(0L); + Collection> copartitionGroups = builder.copartitionGroups(); - // push two items to the primary stream. the other table is empty + assertEquals(1, copartitionGroups.size()); + assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); - for (int i = 0; i < 2; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + driver = new KStreamTestDriver(builder, stateDir); + driver.setTime(0L); - processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); + // push two items to the primary stream. the other table is empty - // push two items to the other stream. this should not produce any item. + for (int i = 0; i < 2; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); - processor.checkAndClearProcessResult(); + // push two items to the other stream. this should not produce any item. - // push all four items to the primary stream. this should produce four items. + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + processor.checkAndClearProcessResult(); - processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null"); + // push all four items to the primary stream. this should produce four items. - // push all items to the other stream. this should not produce any item - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - processor.checkAndClearProcessResult(); + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null"); - // push all four items to the primary stream. this should produce four items. + // push all items to the other stream. this should not produce any item + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + processor.checkAndClearProcessResult(); - processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + // push all four items to the primary stream. this should produce four items. - // push two items with null to the other stream as deletes. this should not produce any item. + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], null); - } + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); - processor.checkAndClearProcessResult(); + // push two items with null to the other stream as deletes. this should not produce any item. - // push all four items to the primary stream. this should produce four items. + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], null); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + processor.checkAndClearProcessResult(); - processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+YY2", "3:XX3+YY3"); + // push all four items to the primary stream. this should produce four items. - } finally { - Utils.delete(baseDir); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } + + processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+YY2", "3:XX3+YY3"); } @Test(expected = KafkaException.class) @@ -158,10 +153,10 @@ public void testNotJoinable() { MockProcessorSupplier processor; processor = new MockProcessorSupplier<>(); - stream = builder.stream(intSerde, stringSerde, topic1).map(keyValueMapper); + stream = builder.stream(intSerde, stringSerde, topic1).map(MockKeyValueMapper.NoOpKeyValueMapper()); table = builder.table(intSerde, stringSerde, topic2); - stream.leftJoin(table, joiner).process(processor); + stream.leftJoin(table, MockValueJoiner.STRING_JOINER).process(processor); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java index 68fa6565d50dc..00e5d70e5a177 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapTest.java @@ -25,6 +25,7 @@ import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorSupplier; +import org.junit.After; import org.junit.Test; import static org.junit.Assert.assertEquals; @@ -36,6 +37,16 @@ public class KStreamMapTest { final private Serde intSerde = Serdes.Integer(); final private Serde stringSerde = Serdes.String(); + private KStreamTestDriver driver = null; + + @After + public void cleanup() { + if (driver != null) { + driver.close(); + } + driver = null; + } + @Test public void testMap() { KStreamBuilder builder = new KStreamBuilder(); @@ -56,7 +67,7 @@ public KeyValue apply(Integer key, String value) { processor = new MockProcessorSupplier<>(); stream.map(mapper).process(processor); - KStreamTestDriver driver = new KStreamTestDriver(builder); + driver = new KStreamTestDriver(builder); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topicName, expectedKeys[i], "V" + expectedKeys[i]); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java index e671aabce392b..e48b677df0557 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamMapValuesTest.java @@ -24,6 +24,7 @@ import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorSupplier; +import org.junit.After; import org.junit.Test; import static org.junit.Assert.assertEquals; @@ -35,6 +36,16 @@ public class KStreamMapValuesTest { final private Serde intSerde = Serdes.Integer(); final private Serde stringSerde = Serdes.String(); + private KStreamTestDriver driver; + + @After + public void cleanup() { + if (driver != null) { + driver.close(); + } + driver = null; + } + @Test public void testFlatMapValues() { KStreamBuilder builder = new KStreamBuilder(); @@ -54,7 +65,7 @@ public Integer apply(String value) { stream = builder.stream(intSerde, stringSerde, topicName); stream.mapValues(mapper).process(processor); - KStreamTestDriver driver = new KStreamTestDriver(builder); + driver = new KStreamTestDriver(builder); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topicName, expectedKeys[i], Integer.toString(expectedKeys[i])); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java index 5f19b9eb3886b..1bd870e8d11bf 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamSelectKeyTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorSupplier; +import org.junit.After; import org.junit.Test; import java.util.HashMap; @@ -40,6 +41,16 @@ public class KStreamSelectKeyTest { final private Serde integerSerde = Serdes.Integer(); final private Serde stringSerde = Serdes.String(); + private KStreamTestDriver driver; + + @After + public void cleanup() { + if (driver != null) { + driver.close(); + } + driver = null; + } + @Test public void testSelectKey() { KStreamBuilder builder = new KStreamBuilder(); @@ -66,7 +77,7 @@ public String apply(String key, Integer value) { stream.selectKey(selector).process(processor); - KStreamTestDriver driver = new KStreamTestDriver(builder); + driver = new KStreamTestDriver(builder); for (int expectedValue : expectedValues) { driver.process(topicName, null, expectedValue); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java index a0a61f2d63b1f..e0bdfbced4ead 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformTest.java @@ -27,6 +27,7 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorSupplier; +import org.junit.After; import org.junit.Test; import static org.junit.Assert.assertEquals; @@ -37,6 +38,16 @@ public class KStreamTransformTest { final private Serde intSerde = Serdes.Integer(); + private KStreamTestDriver driver; + + @After + public void cleanup() { + if (driver != null) { + driver.close(); + } + driver = null; + } + @Test public void testTransform() { KStreamBuilder builder = new KStreamBuilder(); @@ -76,7 +87,7 @@ public void close() { KStream stream = builder.stream(intSerde, intSerde, topicName); stream.transform(transformerSupplier).process(processor); - KStreamTestDriver driver = new KStreamTestDriver(builder); + driver = new KStreamTestDriver(builder); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topicName, expectedKeys[i], expectedKeys[i] * 10); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java index f5f9698baac43..aebcc762a30b2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamTransformValuesTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.streams.processor.ProcessorContext; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorSupplier; +import org.junit.After; import org.junit.Test; import static org.junit.Assert.assertEquals; @@ -36,6 +37,16 @@ public class KStreamTransformValuesTest { final private Serde intSerde = Serdes.Integer(); + private KStreamTestDriver driver; + + @After + public void cleanup() { + if (driver != null) { + driver.close(); + } + driver = null; + } + @Test public void testTransform() { KStreamBuilder builder = new KStreamBuilder(); @@ -76,7 +87,7 @@ public void close() { stream = builder.stream(intSerde, intSerde, topicName); stream.transformValues(valueTransformerSupplier).process(processor); - KStreamTestDriver driver = new KStreamTestDriver(builder); + driver = new KStreamTestDriver(builder); for (int i = 0; i < expectedKeys.length; i++) { driver.process(topicName, expectedKeys[i], expectedKeys[i] * 10); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java index 3c7a1bdc60485..828103ac3dd5c 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java @@ -20,266 +20,257 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.kstream.Aggregator; import org.apache.kafka.streams.kstream.HoppingWindows; -import org.apache.kafka.streams.kstream.Initializer; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.test.KStreamTestDriver; +import org.apache.kafka.test.MockAggregator; +import org.apache.kafka.test.MockInitializer; import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.io.File; -import java.nio.file.Files; +import java.io.IOException; import static org.junit.Assert.assertEquals; public class KStreamWindowAggregateTest { - final private Serde strSerde = new Serdes.StringSerde(); + final private Serde strSerde = Serdes.String(); - private class StringAdd implements Aggregator { + private KStreamTestDriver driver = null; + private File stateDir = null; - @Override - public String apply(String aggKey, String value, String aggregate) { - return aggregate + "+" + value; + @After + public void tearDown() { + if (driver != null) { + driver.close(); } + driver = null; } - private class StringInit implements Initializer { - - @Override - public String apply() { - return "0"; - } + @Before + public void setUp() throws IOException { + stateDir = TestUtils.tempDirectory("kafka-test"); } @Test public void testAggBasic() throws Exception { - final File baseDir = Files.createTempDirectory("test").toFile(); - - try { - final KStreamBuilder builder = new KStreamBuilder(); - String topic1 = "topic1"; - - KStream stream1 = builder.stream(strSerde, strSerde, topic1); - KTable, String> table2 = stream1.aggregateByKey(new StringInit(), new StringAdd(), - HoppingWindows.of("topic1-Canonized").with(10L).every(5L), - strSerde, - strSerde); - - MockProcessorSupplier, String> proc2 = new MockProcessorSupplier<>(); - table2.toStream().process(proc2); - - KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir); - - driver.setTime(0L); - driver.process(topic1, "A", "1"); - driver.setTime(1L); - driver.process(topic1, "B", "2"); - driver.setTime(2L); - driver.process(topic1, "C", "3"); - driver.setTime(3L); - driver.process(topic1, "D", "4"); - driver.setTime(4L); - driver.process(topic1, "A", "1"); - - driver.setTime(5L); - driver.process(topic1, "A", "1"); - driver.setTime(6L); - driver.process(topic1, "B", "2"); - driver.setTime(7L); - driver.process(topic1, "D", "4"); - driver.setTime(8L); - driver.process(topic1, "B", "2"); - driver.setTime(9L); - driver.process(topic1, "C", "3"); - - driver.setTime(10L); - driver.process(topic1, "A", "1"); - driver.setTime(11L); - driver.process(topic1, "B", "2"); - driver.setTime(12L); - driver.process(topic1, "D", "4"); - driver.setTime(13L); - driver.process(topic1, "B", "2"); - driver.setTime(14L); - driver.process(topic1, "C", "3"); - - assertEquals(Utils.mkList( - "[A@0]:0+1", - "[B@0]:0+2", - "[C@0]:0+3", - "[D@0]:0+4", - "[A@0]:0+1+1", - - "[A@0]:0+1+1+1", "[A@5]:0+1", - "[B@0]:0+2+2", "[B@5]:0+2", - "[D@0]:0+4+4", "[D@5]:0+4", - "[B@0]:0+2+2+2", "[B@5]:0+2+2", - "[C@0]:0+3+3", "[C@5]:0+3", - - "[A@5]:0+1+1", "[A@10]:0+1", - "[B@5]:0+2+2+2", "[B@10]:0+2", - "[D@5]:0+4+4", "[D@10]:0+4", - "[B@5]:0+2+2+2+2", "[B@10]:0+2+2", - "[C@5]:0+3+3", "[C@10]:0+3"), proc2.processed); - - } finally { - Utils.delete(baseDir); - } + final KStreamBuilder builder = new KStreamBuilder(); + + String topic1 = "topic1"; + + KStream stream1 = builder.stream(strSerde, strSerde, topic1); + KTable, String> table2 = stream1.aggregateByKey(MockInitializer.STRING_INIT, MockAggregator.STRING_ADDER, + HoppingWindows.of("topic1-Canonized").with(10L).every(5L), + strSerde, + strSerde); + + MockProcessorSupplier, String> proc2 = new MockProcessorSupplier<>(); + table2.toStream().process(proc2); + + driver = new KStreamTestDriver(builder, stateDir); + + driver.setTime(0L); + driver.process(topic1, "A", "1"); + driver.setTime(1L); + driver.process(topic1, "B", "2"); + driver.setTime(2L); + driver.process(topic1, "C", "3"); + driver.setTime(3L); + driver.process(topic1, "D", "4"); + driver.setTime(4L); + driver.process(topic1, "A", "1"); + + driver.setTime(5L); + driver.process(topic1, "A", "1"); + driver.setTime(6L); + driver.process(topic1, "B", "2"); + driver.setTime(7L); + driver.process(topic1, "D", "4"); + driver.setTime(8L); + driver.process(topic1, "B", "2"); + driver.setTime(9L); + driver.process(topic1, "C", "3"); + + driver.setTime(10L); + driver.process(topic1, "A", "1"); + driver.setTime(11L); + driver.process(topic1, "B", "2"); + driver.setTime(12L); + driver.process(topic1, "D", "4"); + driver.setTime(13L); + driver.process(topic1, "B", "2"); + driver.setTime(14L); + driver.process(topic1, "C", "3"); + + assertEquals(Utils.mkList( + "[A@0]:0+1", + "[B@0]:0+2", + "[C@0]:0+3", + "[D@0]:0+4", + "[A@0]:0+1+1", + + "[A@0]:0+1+1+1", "[A@5]:0+1", + "[B@0]:0+2+2", "[B@5]:0+2", + "[D@0]:0+4+4", "[D@5]:0+4", + "[B@0]:0+2+2+2", "[B@5]:0+2+2", + "[C@0]:0+3+3", "[C@5]:0+3", + + "[A@5]:0+1+1", "[A@10]:0+1", + "[B@5]:0+2+2+2", "[B@10]:0+2", + "[D@5]:0+4+4", "[D@10]:0+4", + "[B@5]:0+2+2+2+2", "[B@10]:0+2+2", + "[C@5]:0+3+3", "[C@10]:0+3"), proc2.processed); } @Test public void testJoin() throws Exception { - final File baseDir = Files.createTempDirectory("test").toFile(); - - try { - final KStreamBuilder builder = new KStreamBuilder(); - String topic1 = "topic1"; - String topic2 = "topic2"; - - KStream stream1 = builder.stream(strSerde, strSerde, topic1); - KTable, String> table1 = stream1.aggregateByKey(new StringInit(), new StringAdd(), - HoppingWindows.of("topic1-Canonized").with(10L).every(5L), - strSerde, - strSerde); - - MockProcessorSupplier, String> proc1 = new MockProcessorSupplier<>(); - table1.toStream().process(proc1); - - KStream stream2 = builder.stream(strSerde, strSerde, topic2); - KTable, String> table2 = stream2.aggregateByKey(new StringInit(), new StringAdd(), - HoppingWindows.of("topic2-Canonized").with(10L).every(5L), - strSerde, - strSerde); - - MockProcessorSupplier, String> proc2 = new MockProcessorSupplier<>(); - table2.toStream().process(proc2); - - - MockProcessorSupplier, String> proc3 = new MockProcessorSupplier<>(); - table1.join(table2, new ValueJoiner() { - @Override - public String apply(String p1, String p2) { - return p1 + "%" + p2; - } - }).toStream().process(proc3); - - KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir); - - driver.setTime(0L); - driver.process(topic1, "A", "1"); - driver.setTime(1L); - driver.process(topic1, "B", "2"); - driver.setTime(2L); - driver.process(topic1, "C", "3"); - driver.setTime(3L); - driver.process(topic1, "D", "4"); - driver.setTime(4L); - driver.process(topic1, "A", "1"); - - proc1.checkAndClearProcessResult( - "[A@0]:0+1", - "[B@0]:0+2", - "[C@0]:0+3", - "[D@0]:0+4", - "[A@0]:0+1+1" - ); - proc2.checkAndClearProcessResult(); - proc3.checkAndClearProcessResult( - "[A@0]:null", - "[B@0]:null", - "[C@0]:null", - "[D@0]:null", - "[A@0]:null" - ); - - driver.setTime(5L); - driver.process(topic1, "A", "1"); - driver.setTime(6L); - driver.process(topic1, "B", "2"); - driver.setTime(7L); - driver.process(topic1, "D", "4"); - driver.setTime(8L); - driver.process(topic1, "B", "2"); - driver.setTime(9L); - driver.process(topic1, "C", "3"); - - proc1.checkAndClearProcessResult( - "[A@0]:0+1+1+1", "[A@5]:0+1", - "[B@0]:0+2+2", "[B@5]:0+2", - "[D@0]:0+4+4", "[D@5]:0+4", - "[B@0]:0+2+2+2", "[B@5]:0+2+2", - "[C@0]:0+3+3", "[C@5]:0+3" - ); - proc2.checkAndClearProcessResult(); - proc3.checkAndClearProcessResult( - "[A@0]:null", "[A@5]:null", - "[B@0]:null", "[B@5]:null", - "[D@0]:null", "[D@5]:null", - "[B@0]:null", "[B@5]:null", - "[C@0]:null", "[C@5]:null" - ); - - driver.setTime(0L); - driver.process(topic2, "A", "a"); - driver.setTime(1L); - driver.process(topic2, "B", "b"); - driver.setTime(2L); - driver.process(topic2, "C", "c"); - driver.setTime(3L); - driver.process(topic2, "D", "d"); - driver.setTime(4L); - driver.process(topic2, "A", "a"); - - proc1.checkAndClearProcessResult(); - proc2.checkAndClearProcessResult( - "[A@0]:0+a", - "[B@0]:0+b", - "[C@0]:0+c", - "[D@0]:0+d", - "[A@0]:0+a+a" - ); - proc3.checkAndClearProcessResult( - "[A@0]:0+1+1+1%0+a", - "[B@0]:0+2+2+2%0+b", - "[C@0]:0+3+3%0+c", - "[D@0]:0+4+4%0+d", - "[A@0]:0+1+1+1%0+a+a"); - - driver.setTime(5L); - driver.process(topic2, "A", "a"); - driver.setTime(6L); - driver.process(topic2, "B", "b"); - driver.setTime(7L); - driver.process(topic2, "D", "d"); - driver.setTime(8L); - driver.process(topic2, "B", "b"); - driver.setTime(9L); - driver.process(topic2, "C", "c"); - - proc1.checkAndClearProcessResult(); - proc2.checkAndClearProcessResult( - "[A@0]:0+a+a+a", "[A@5]:0+a", - "[B@0]:0+b+b", "[B@5]:0+b", - "[D@0]:0+d+d", "[D@5]:0+d", - "[B@0]:0+b+b+b", "[B@5]:0+b+b", - "[C@0]:0+c+c", "[C@5]:0+c" - ); - proc3.checkAndClearProcessResult( - "[A@0]:0+1+1+1%0+a+a+a", "[A@5]:0+1%0+a", - "[B@0]:0+2+2+2%0+b+b", "[B@5]:0+2+2%0+b", - "[D@0]:0+4+4%0+d+d", "[D@5]:0+4%0+d", - "[B@0]:0+2+2+2%0+b+b+b", "[B@5]:0+2+2%0+b+b", - "[C@0]:0+3+3%0+c+c", "[C@5]:0+3%0+c" - ); - - } finally { - Utils.delete(baseDir); - } + final KStreamBuilder builder = new KStreamBuilder(); + + String topic1 = "topic1"; + String topic2 = "topic2"; + + KStream stream1 = builder.stream(strSerde, strSerde, topic1); + KTable, String> table1 = stream1.aggregateByKey(MockInitializer.STRING_INIT, MockAggregator.STRING_ADDER, + HoppingWindows.of("topic1-Canonized").with(10L).every(5L), + strSerde, + strSerde); + + MockProcessorSupplier, String> proc1 = new MockProcessorSupplier<>(); + table1.toStream().process(proc1); + + KStream stream2 = builder.stream(strSerde, strSerde, topic2); + KTable, String> table2 = stream2.aggregateByKey(MockInitializer.STRING_INIT, MockAggregator.STRING_ADDER, + HoppingWindows.of("topic2-Canonized").with(10L).every(5L), + strSerde, + strSerde); + + MockProcessorSupplier, String> proc2 = new MockProcessorSupplier<>(); + table2.toStream().process(proc2); + + + MockProcessorSupplier, String> proc3 = new MockProcessorSupplier<>(); + table1.join(table2, new ValueJoiner() { + @Override + public String apply(String p1, String p2) { + return p1 + "%" + p2; + } + }).toStream().process(proc3); + + driver = new KStreamTestDriver(builder, stateDir); + + driver.setTime(0L); + driver.process(topic1, "A", "1"); + driver.setTime(1L); + driver.process(topic1, "B", "2"); + driver.setTime(2L); + driver.process(topic1, "C", "3"); + driver.setTime(3L); + driver.process(topic1, "D", "4"); + driver.setTime(4L); + driver.process(topic1, "A", "1"); + + proc1.checkAndClearProcessResult( + "[A@0]:0+1", + "[B@0]:0+2", + "[C@0]:0+3", + "[D@0]:0+4", + "[A@0]:0+1+1" + ); + proc2.checkAndClearProcessResult(); + proc3.checkAndClearProcessResult( + "[A@0]:null", + "[B@0]:null", + "[C@0]:null", + "[D@0]:null", + "[A@0]:null" + ); + + driver.setTime(5L); + driver.process(topic1, "A", "1"); + driver.setTime(6L); + driver.process(topic1, "B", "2"); + driver.setTime(7L); + driver.process(topic1, "D", "4"); + driver.setTime(8L); + driver.process(topic1, "B", "2"); + driver.setTime(9L); + driver.process(topic1, "C", "3"); + + proc1.checkAndClearProcessResult( + "[A@0]:0+1+1+1", "[A@5]:0+1", + "[B@0]:0+2+2", "[B@5]:0+2", + "[D@0]:0+4+4", "[D@5]:0+4", + "[B@0]:0+2+2+2", "[B@5]:0+2+2", + "[C@0]:0+3+3", "[C@5]:0+3" + ); + proc2.checkAndClearProcessResult(); + proc3.checkAndClearProcessResult( + "[A@0]:null", "[A@5]:null", + "[B@0]:null", "[B@5]:null", + "[D@0]:null", "[D@5]:null", + "[B@0]:null", "[B@5]:null", + "[C@0]:null", "[C@5]:null" + ); + + driver.setTime(0L); + driver.process(topic2, "A", "a"); + driver.setTime(1L); + driver.process(topic2, "B", "b"); + driver.setTime(2L); + driver.process(topic2, "C", "c"); + driver.setTime(3L); + driver.process(topic2, "D", "d"); + driver.setTime(4L); + driver.process(topic2, "A", "a"); + + proc1.checkAndClearProcessResult(); + proc2.checkAndClearProcessResult( + "[A@0]:0+a", + "[B@0]:0+b", + "[C@0]:0+c", + "[D@0]:0+d", + "[A@0]:0+a+a" + ); + proc3.checkAndClearProcessResult( + "[A@0]:0+1+1+1%0+a", + "[B@0]:0+2+2+2%0+b", + "[C@0]:0+3+3%0+c", + "[D@0]:0+4+4%0+d", + "[A@0]:0+1+1+1%0+a+a"); + + driver.setTime(5L); + driver.process(topic2, "A", "a"); + driver.setTime(6L); + driver.process(topic2, "B", "b"); + driver.setTime(7L); + driver.process(topic2, "D", "d"); + driver.setTime(8L); + driver.process(topic2, "B", "b"); + driver.setTime(9L); + driver.process(topic2, "C", "c"); + + proc1.checkAndClearProcessResult(); + proc2.checkAndClearProcessResult( + "[A@0]:0+a+a+a", "[A@5]:0+a", + "[B@0]:0+b+b", "[B@5]:0+b", + "[D@0]:0+d+d", "[D@5]:0+d", + "[B@0]:0+b+b+b", "[B@5]:0+b+b", + "[C@0]:0+c+c", "[C@5]:0+c" + ); + proc3.checkAndClearProcessResult( + "[A@0]:0+1+1+1%0+a+a+a", "[A@5]:0+1%0+a", + "[B@0]:0+2+2+2%0+b+b", "[B@5]:0+2+2%0+b", + "[D@0]:0+4+4%0+d+d", "[D@5]:0+4%0+d", + "[B@0]:0+2+2+2%0+b+b+b", "[B@5]:0+2+2%0+b+b", + "[C@0]:0+3+3%0+c+c", "[C@5]:0+3%0+c" + ); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java index be0ec19656f5e..a6144796490f8 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableAggregateTest.java @@ -27,61 +27,73 @@ import org.apache.kafka.test.MockInitializer; import org.apache.kafka.test.MockKeyValueMapper; import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.io.File; -import java.nio.file.Files; +import java.io.IOException; import static org.junit.Assert.assertEquals; public class KTableAggregateTest { - final private Serde stringSerde = new Serdes.StringSerde(); + final private Serde stringSerde = Serdes.String(); - @Test - public void testAggBasic() throws Exception { - final File baseDir = Files.createTempDirectory("test").toFile(); + private KStreamTestDriver driver = null; + private File stateDir = null; - try { - final KStreamBuilder builder = new KStreamBuilder(); - String topic1 = "topic1"; + @After + public void tearDown() { + if (driver != null) { + driver.close(); + } + driver = null; + } - KTable table1 = builder.table(stringSerde, stringSerde, topic1); - KTable table2 = table1.groupBy(MockKeyValueMapper.NoOpKeyValueMapper(), - stringSerde, - stringSerde - ).aggregate(MockInitializer.STRING_INIT, - MockAggregator.STRING_ADDER, - MockAggregator.STRING_REMOVER, - stringSerde, - "topic1-Canonized"); + @Before + public void setUp() throws IOException { + stateDir = TestUtils.tempDirectory("kafka-test"); + } - MockProcessorSupplier proc2 = new MockProcessorSupplier<>(); - table2.toStream().process(proc2); + @Test + public void testAggBasic() throws Exception { + final KStreamBuilder builder = new KStreamBuilder(); + String topic1 = "topic1"; - KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir); + KTable table1 = builder.table(stringSerde, stringSerde, topic1); + KTable table2 = table1.groupBy(MockKeyValueMapper.NoOpKeyValueMapper(), + stringSerde, + stringSerde + ).aggregate(MockInitializer.STRING_INIT, + MockAggregator.STRING_ADDER, + MockAggregator.STRING_REMOVER, + stringSerde, + "topic1-Canonized"); - driver.process(topic1, "A", "1"); - driver.process(topic1, "B", "2"); - driver.process(topic1, "A", "3"); - driver.process(topic1, "B", "4"); - driver.process(topic1, "C", "5"); - driver.process(topic1, "D", "6"); - driver.process(topic1, "B", "7"); - driver.process(topic1, "C", "8"); + MockProcessorSupplier proc2 = new MockProcessorSupplier<>(); + table2.toStream().process(proc2); - assertEquals(Utils.mkList( - "A:0+1", - "B:0+2", - "A:0+1+3", "A:0+1+3-1", - "B:0+2+4", "B:0+2+4-2", - "C:0+5", - "D:0+6", - "B:0+2+4-2+7", "B:0+2+4-2+7-4", - "C:0+5+8", "C:0+5+8-5"), proc2.processed); + driver = new KStreamTestDriver(builder, stateDir); - } finally { - Utils.delete(baseDir); - } + driver.process(topic1, "A", "1"); + driver.process(topic1, "B", "2"); + driver.process(topic1, "A", "3"); + driver.process(topic1, "B", "4"); + driver.process(topic1, "C", "5"); + driver.process(topic1, "D", "6"); + driver.process(topic1, "B", "7"); + driver.process(topic1, "C", "8"); + + assertEquals(Utils.mkList( + "A:0+1", + "B:0+2", + "A:0+1+3", "A:0+1+3-1", + "B:0+2+4", "B:0+2+4-2", + "C:0+5", + "D:0+6", + "B:0+2+4-2+7", "B:0+2+4-2+7-4", + "C:0+5+8", "C:0+5+8-5"), proc2.processed); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java index ee26058193c91..a3af133d3b0fc 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableFilterTest.java @@ -19,25 +19,42 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.Predicate; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.io.File; import java.io.IOException; -import java.nio.file.Files; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; public class KTableFilterTest { - final private Serde intSerde = new Serdes.IntegerSerde(); - final private Serde stringSerde = new Serdes.StringSerde(); + final private Serde intSerde = Serdes.Integer(); + final private Serde stringSerde = Serdes.String(); + + private KStreamTestDriver driver = null; + private File stateDir = null; + + @After + public void tearDown() { + if (driver != null) { + driver.close(); + } + driver = null; + } + + @Before + public void setUp() throws IOException { + stateDir = TestUtils.tempDirectory("kafka-test"); + } @Test public void testKTable() { @@ -65,7 +82,7 @@ public boolean test(String key, Integer value) { table2.toStream().process(proc2); table3.toStream().process(proc3); - KStreamTestDriver driver = new KStreamTestDriver(builder); + driver = new KStreamTestDriver(builder); driver.process(topic1, "A", 1); driver.process(topic1, "B", 2); @@ -80,199 +97,181 @@ public boolean test(String key, Integer value) { @Test public void testValueGetter() throws IOException { - File stateDir = Files.createTempDirectory("test").toFile(); - try { - final KStreamBuilder builder = new KStreamBuilder(); - - String topic1 = "topic1"; - - KTableImpl table1 = - (KTableImpl) builder.table(stringSerde, intSerde, topic1); - KTableImpl table2 = (KTableImpl) table1.filter( - new Predicate() { - @Override - public boolean test(String key, Integer value) { - return (value % 2) == 0; - } - }); - KTableImpl table3 = (KTableImpl) table1.filterNot( - new Predicate() { - @Override - public boolean test(String key, Integer value) { - return (value % 2) == 0; - } - }); - - KTableValueGetterSupplier getterSupplier2 = table2.valueGetterSupplier(); - KTableValueGetterSupplier getterSupplier3 = table3.valueGetterSupplier(); - - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); - - KTableValueGetter getter2 = getterSupplier2.get(); - KTableValueGetter getter3 = getterSupplier3.get(); - - getter2.init(driver.context()); - getter3.init(driver.context()); - - driver.process(topic1, "A", 1); - driver.process(topic1, "B", 1); - driver.process(topic1, "C", 1); - - assertNull(getter2.get("A")); - assertNull(getter2.get("B")); - assertNull(getter2.get("C")); - - assertEquals(1, (int) getter3.get("A")); - assertEquals(1, (int) getter3.get("B")); - assertEquals(1, (int) getter3.get("C")); - - driver.process(topic1, "A", 2); - driver.process(topic1, "B", 2); - - assertEquals(2, (int) getter2.get("A")); - assertEquals(2, (int) getter2.get("B")); - assertNull(getter2.get("C")); - - assertNull(getter3.get("A")); - assertNull(getter3.get("B")); - assertEquals(1, (int) getter3.get("C")); - - driver.process(topic1, "A", 3); - - assertNull(getter2.get("A")); - assertEquals(2, (int) getter2.get("B")); - assertNull(getter2.get("C")); - - assertEquals(3, (int) getter3.get("A")); - assertNull(getter3.get("B")); - assertEquals(1, (int) getter3.get("C")); - - driver.process(topic1, "A", null); - driver.process(topic1, "B", null); - - assertNull(getter2.get("A")); - assertNull(getter2.get("B")); - assertNull(getter2.get("C")); - - assertNull(getter3.get("A")); - assertNull(getter3.get("B")); - assertEquals(1, (int) getter3.get("C")); - - } finally { - Utils.delete(stateDir); - } + KStreamBuilder builder = new KStreamBuilder(); + + String topic1 = "topic1"; + + KTableImpl table1 = + (KTableImpl) builder.table(stringSerde, intSerde, topic1); + KTableImpl table2 = (KTableImpl) table1.filter( + new Predicate() { + @Override + public boolean test(String key, Integer value) { + return (value % 2) == 0; + } + }); + KTableImpl table3 = (KTableImpl) table1.filterNot( + new Predicate() { + @Override + public boolean test(String key, Integer value) { + return (value % 2) == 0; + } + }); + + KTableValueGetterSupplier getterSupplier2 = table2.valueGetterSupplier(); + KTableValueGetterSupplier getterSupplier3 = table3.valueGetterSupplier(); + + driver = new KStreamTestDriver(builder, stateDir, null, null); + + KTableValueGetter getter2 = getterSupplier2.get(); + KTableValueGetter getter3 = getterSupplier3.get(); + + getter2.init(driver.context()); + getter3.init(driver.context()); + + driver.process(topic1, "A", 1); + driver.process(topic1, "B", 1); + driver.process(topic1, "C", 1); + + assertNull(getter2.get("A")); + assertNull(getter2.get("B")); + assertNull(getter2.get("C")); + + assertEquals(1, (int) getter3.get("A")); + assertEquals(1, (int) getter3.get("B")); + assertEquals(1, (int) getter3.get("C")); + + driver.process(topic1, "A", 2); + driver.process(topic1, "B", 2); + + assertEquals(2, (int) getter2.get("A")); + assertEquals(2, (int) getter2.get("B")); + assertNull(getter2.get("C")); + + assertNull(getter3.get("A")); + assertNull(getter3.get("B")); + assertEquals(1, (int) getter3.get("C")); + + driver.process(topic1, "A", 3); + + assertNull(getter2.get("A")); + assertEquals(2, (int) getter2.get("B")); + assertNull(getter2.get("C")); + + assertEquals(3, (int) getter3.get("A")); + assertNull(getter3.get("B")); + assertEquals(1, (int) getter3.get("C")); + + driver.process(topic1, "A", null); + driver.process(topic1, "B", null); + + assertNull(getter2.get("A")); + assertNull(getter2.get("B")); + assertNull(getter2.get("C")); + + assertNull(getter3.get("A")); + assertNull(getter3.get("B")); + assertEquals(1, (int) getter3.get("C")); } @Test public void testNotSendingOldValue() throws IOException { - File stateDir = Files.createTempDirectory("test").toFile(); - try { - final KStreamBuilder builder = new KStreamBuilder(); - - String topic1 = "topic1"; - - KTableImpl table1 = - (KTableImpl) builder.table(stringSerde, intSerde, topic1); - KTableImpl table2 = (KTableImpl) table1.filter( - new Predicate() { - @Override - public boolean test(String key, Integer value) { - return (value % 2) == 0; - } - }); + KStreamBuilder builder = new KStreamBuilder(); - MockProcessorSupplier proc1 = new MockProcessorSupplier<>(); - MockProcessorSupplier proc2 = new MockProcessorSupplier<>(); + String topic1 = "topic1"; - builder.addProcessor("proc1", proc1, table1.name); - builder.addProcessor("proc2", proc2, table2.name); + KTableImpl table1 = + (KTableImpl) builder.table(stringSerde, intSerde, topic1); + KTableImpl table2 = (KTableImpl) table1.filter( + new Predicate() { + @Override + public boolean test(String key, Integer value) { + return (value % 2) == 0; + } + }); + + MockProcessorSupplier proc1 = new MockProcessorSupplier<>(); + MockProcessorSupplier proc2 = new MockProcessorSupplier<>(); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); + builder.addProcessor("proc1", proc1, table1.name); + builder.addProcessor("proc2", proc2, table2.name); - driver.process(topic1, "A", 1); - driver.process(topic1, "B", 1); - driver.process(topic1, "C", 1); + driver = new KStreamTestDriver(builder, stateDir, null, null); - proc1.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)"); - proc2.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)", "C:(null<-null)"); + driver.process(topic1, "A", 1); + driver.process(topic1, "B", 1); + driver.process(topic1, "C", 1); - driver.process(topic1, "A", 2); - driver.process(topic1, "B", 2); + proc1.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)"); + proc2.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)", "C:(null<-null)"); - proc1.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)"); - proc2.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)"); + driver.process(topic1, "A", 2); + driver.process(topic1, "B", 2); - driver.process(topic1, "A", 3); + proc1.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)"); + proc2.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)"); - proc1.checkAndClearProcessResult("A:(3<-null)"); - proc2.checkAndClearProcessResult("A:(null<-null)"); + driver.process(topic1, "A", 3); - driver.process(topic1, "A", null); - driver.process(topic1, "B", null); + proc1.checkAndClearProcessResult("A:(3<-null)"); + proc2.checkAndClearProcessResult("A:(null<-null)"); - proc1.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)"); - proc2.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)"); + driver.process(topic1, "A", null); + driver.process(topic1, "B", null); - } finally { - Utils.delete(stateDir); - } + proc1.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)"); + proc2.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)"); } @Test public void testSendingOldValue() throws IOException { - File stateDir = Files.createTempDirectory("test").toFile(); - try { - final KStreamBuilder builder = new KStreamBuilder(); - - String topic1 = "topic1"; + KStreamBuilder builder = new KStreamBuilder(); - KTableImpl table1 = - (KTableImpl) builder.table(stringSerde, intSerde, topic1); - KTableImpl table2 = (KTableImpl) table1.filter( - new Predicate() { - @Override - public boolean test(String key, Integer value) { - return (value % 2) == 0; - } - }); + String topic1 = "topic1"; - table2.enableSendingOldValues(); + KTableImpl table1 = + (KTableImpl) builder.table(stringSerde, intSerde, topic1); + KTableImpl table2 = (KTableImpl) table1.filter( + new Predicate() { + @Override + public boolean test(String key, Integer value) { + return (value % 2) == 0; + } + }); - MockProcessorSupplier proc1 = new MockProcessorSupplier<>(); - MockProcessorSupplier proc2 = new MockProcessorSupplier<>(); + table2.enableSendingOldValues(); - builder.addProcessor("proc1", proc1, table1.name); - builder.addProcessor("proc2", proc2, table2.name); + MockProcessorSupplier proc1 = new MockProcessorSupplier<>(); + MockProcessorSupplier proc2 = new MockProcessorSupplier<>(); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); + builder.addProcessor("proc1", proc1, table1.name); + builder.addProcessor("proc2", proc2, table2.name); - driver.process(topic1, "A", 1); - driver.process(topic1, "B", 1); - driver.process(topic1, "C", 1); + driver = new KStreamTestDriver(builder, stateDir, null, null); - proc1.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)"); - proc2.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)", "C:(null<-null)"); + driver.process(topic1, "A", 1); + driver.process(topic1, "B", 1); + driver.process(topic1, "C", 1); - driver.process(topic1, "A", 2); - driver.process(topic1, "B", 2); + proc1.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)"); + proc2.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)", "C:(null<-null)"); - proc1.checkAndClearProcessResult("A:(2<-1)", "B:(2<-1)"); - proc2.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)"); + driver.process(topic1, "A", 2); + driver.process(topic1, "B", 2); - driver.process(topic1, "A", 3); + proc1.checkAndClearProcessResult("A:(2<-1)", "B:(2<-1)"); + proc2.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)"); - proc1.checkAndClearProcessResult("A:(3<-2)"); - proc2.checkAndClearProcessResult("A:(null<-2)"); + driver.process(topic1, "A", 3); - driver.process(topic1, "A", null); - driver.process(topic1, "B", null); + proc1.checkAndClearProcessResult("A:(3<-2)"); + proc2.checkAndClearProcessResult("A:(null<-2)"); - proc1.checkAndClearProcessResult("A:(null<-3)", "B:(null<-2)"); - proc2.checkAndClearProcessResult("A:(null<-null)", "B:(null<-2)"); + driver.process(topic1, "A", null); + driver.process(topic1, "B", null); - } finally { - Utils.delete(stateDir); - } + proc1.checkAndClearProcessResult("A:(null<-3)", "B:(null<-2)"); + proc2.checkAndClearProcessResult("A:(null<-null)", "B:(null<-2)"); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableForeachTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableForeachTest.java index 27a51146070d2..af131c2aa7575 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableForeachTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableForeachTest.java @@ -24,6 +24,7 @@ import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.test.KStreamTestDriver; +import org.junit.After; import org.junit.Test; import java.util.List; import java.util.Locale; @@ -39,6 +40,16 @@ public class KTableForeachTest { final private Serde intSerde = Serdes.Integer(); final private Serde stringSerde = Serdes.String(); + private KStreamTestDriver driver; + + @After + public void cleanup() { + if (driver != null) { + driver.close(); + } + driver = null; + } + @Test public void testForeach() { // Given @@ -71,7 +82,7 @@ public void apply(Integer key, String value) { table.foreach(action); // Then - KStreamTestDriver driver = new KStreamTestDriver(builder); + driver = new KStreamTestDriver(builder); for (KeyValue record: inputRecords) { driver.process(topicName, record.key, record.value); } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java index 8a13e9aaa6373..ca3bbe181baf2 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableImplTest.java @@ -33,11 +33,13 @@ import org.apache.kafka.test.MockKeyValueMapper; import org.apache.kafka.test.MockProcessorSupplier; import org.apache.kafka.test.MockReducer; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.io.File; import java.io.IOException; -import java.nio.file.Files; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNotNull; @@ -46,7 +48,23 @@ public class KTableImplTest { - final private Serde stringSerde = new Serdes.StringSerde(); + final private Serde stringSerde = Serdes.String(); + + private KStreamTestDriver driver = null; + private File stateDir = null; + + @After + public void tearDown() { + if (driver != null) { + driver.close(); + } + driver = null; + } + + @Before + public void setUp() throws IOException { + stateDir = TestUtils.tempDirectory("kafka-test"); + } @Test public void testKTable() { @@ -85,7 +103,7 @@ public boolean test(String key, Integer value) { MockProcessorSupplier proc4 = new MockProcessorSupplier<>(); table4.toStream().process(proc4); - KStreamTestDriver driver = new KStreamTestDriver(builder); + driver = new KStreamTestDriver(builder); driver.process(topic1, "A", "01"); driver.process(topic1, "B", "02"); @@ -100,129 +118,157 @@ public boolean test(String key, Integer value) { @Test public void testValueGetter() throws IOException { - File stateDir = Files.createTempDirectory("test").toFile(); - try { - final KStreamBuilder builder = new KStreamBuilder(); - - String topic1 = "topic1"; - String topic2 = "topic2"; - - KTableImpl table1 = - (KTableImpl) builder.table(stringSerde, stringSerde, topic1); - KTableImpl table2 = (KTableImpl) table1.mapValues( - new ValueMapper() { - @Override - public Integer apply(String value) { - return new Integer(value); - } - }); - KTableImpl table3 = (KTableImpl) table2.filter( - new Predicate() { - @Override - public boolean test(String key, Integer value) { - return (value % 2) == 0; - } - }); - KTableImpl table4 = (KTableImpl) - table1.through(stringSerde, stringSerde, topic2); - - KTableValueGetterSupplier getterSupplier1 = table1.valueGetterSupplier(); - KTableValueGetterSupplier getterSupplier2 = table2.valueGetterSupplier(); - KTableValueGetterSupplier getterSupplier3 = table3.valueGetterSupplier(); - KTableValueGetterSupplier getterSupplier4 = table4.valueGetterSupplier(); - - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); - - // two state store should be created - assertEquals(2, driver.allStateStores().size()); - - KTableValueGetter getter1 = getterSupplier1.get(); - getter1.init(driver.context()); - KTableValueGetter getter2 = getterSupplier2.get(); - getter2.init(driver.context()); - KTableValueGetter getter3 = getterSupplier3.get(); - getter3.init(driver.context()); - KTableValueGetter getter4 = getterSupplier4.get(); - getter4.init(driver.context()); - - driver.process(topic1, "A", "01"); - driver.process(topic1, "B", "01"); - driver.process(topic1, "C", "01"); - - assertEquals("01", getter1.get("A")); - assertEquals("01", getter1.get("B")); - assertEquals("01", getter1.get("C")); - - assertEquals(new Integer(1), getter2.get("A")); - assertEquals(new Integer(1), getter2.get("B")); - assertEquals(new Integer(1), getter2.get("C")); - - assertNull(getter3.get("A")); - assertNull(getter3.get("B")); - assertNull(getter3.get("C")); - - assertEquals("01", getter4.get("A")); - assertEquals("01", getter4.get("B")); - assertEquals("01", getter4.get("C")); - - driver.process(topic1, "A", "02"); - driver.process(topic1, "B", "02"); - - assertEquals("02", getter1.get("A")); - assertEquals("02", getter1.get("B")); - assertEquals("01", getter1.get("C")); - - assertEquals(new Integer(2), getter2.get("A")); - assertEquals(new Integer(2), getter2.get("B")); - assertEquals(new Integer(1), getter2.get("C")); - - assertEquals(new Integer(2), getter3.get("A")); - assertEquals(new Integer(2), getter3.get("B")); - assertNull(getter3.get("C")); - - assertEquals("02", getter4.get("A")); - assertEquals("02", getter4.get("B")); - assertEquals("01", getter4.get("C")); - - driver.process(topic1, "A", "03"); - - assertEquals("03", getter1.get("A")); - assertEquals("02", getter1.get("B")); - assertEquals("01", getter1.get("C")); - - assertEquals(new Integer(3), getter2.get("A")); - assertEquals(new Integer(2), getter2.get("B")); - assertEquals(new Integer(1), getter2.get("C")); - - assertNull(getter3.get("A")); - assertEquals(new Integer(2), getter3.get("B")); - assertNull(getter3.get("C")); - - assertEquals("03", getter4.get("A")); - assertEquals("02", getter4.get("B")); - assertEquals("01", getter4.get("C")); - - driver.process(topic1, "A", null); - - assertNull(getter1.get("A")); - assertEquals("02", getter1.get("B")); - assertEquals("01", getter1.get("C")); - - assertNull(getter2.get("A")); - assertEquals(new Integer(2), getter2.get("B")); - assertEquals(new Integer(1), getter2.get("C")); - - assertNull(getter3.get("A")); - assertEquals(new Integer(2), getter3.get("B")); - assertNull(getter3.get("C")); - - assertNull(getter4.get("A")); - assertEquals("02", getter4.get("B")); - assertEquals("01", getter4.get("C")); + final KStreamBuilder builder = new KStreamBuilder(); - } finally { - Utils.delete(stateDir); - } + String topic1 = "topic1"; + String topic2 = "topic2"; + + KTableImpl table1 = + (KTableImpl) builder.table(stringSerde, stringSerde, topic1); + KTableImpl table2 = (KTableImpl) table1.mapValues( + new ValueMapper() { + @Override + public Integer apply(String value) { + return new Integer(value); + } + }); + KTableImpl table3 = (KTableImpl) table2.filter( + new Predicate() { + @Override + public boolean test(String key, Integer value) { + return (value % 2) == 0; + } + }); + KTableImpl table4 = (KTableImpl) + table1.through(stringSerde, stringSerde, topic2); + + KTableValueGetterSupplier getterSupplier1 = table1.valueGetterSupplier(); + KTableValueGetterSupplier getterSupplier2 = table2.valueGetterSupplier(); + KTableValueGetterSupplier getterSupplier3 = table3.valueGetterSupplier(); + KTableValueGetterSupplier getterSupplier4 = table4.valueGetterSupplier(); + + driver = new KStreamTestDriver(builder, stateDir, null, null); + + // two state store should be created + assertEquals(2, driver.allStateStores().size()); + + KTableValueGetter getter1 = getterSupplier1.get(); + getter1.init(driver.context()); + KTableValueGetter getter2 = getterSupplier2.get(); + getter2.init(driver.context()); + KTableValueGetter getter3 = getterSupplier3.get(); + getter3.init(driver.context()); + KTableValueGetter getter4 = getterSupplier4.get(); + getter4.init(driver.context()); + + driver.process(topic1, "A", "01"); + driver.process(topic1, "B", "01"); + driver.process(topic1, "C", "01"); + + assertEquals("01", getter1.get("A")); + assertEquals("01", getter1.get("B")); + assertEquals("01", getter1.get("C")); + + assertEquals(new Integer(1), getter2.get("A")); + assertEquals(new Integer(1), getter2.get("B")); + assertEquals(new Integer(1), getter2.get("C")); + + assertNull(getter3.get("A")); + assertNull(getter3.get("B")); + assertNull(getter3.get("C")); + + assertEquals("01", getter4.get("A")); + assertEquals("01", getter4.get("B")); + assertEquals("01", getter4.get("C")); + + driver.process(topic1, "A", "02"); + driver.process(topic1, "B", "02"); + + assertEquals("02", getter1.get("A")); + assertEquals("02", getter1.get("B")); + assertEquals("01", getter1.get("C")); + + assertEquals(new Integer(2), getter2.get("A")); + assertEquals(new Integer(2), getter2.get("B")); + assertEquals(new Integer(1), getter2.get("C")); + + assertEquals(new Integer(2), getter3.get("A")); + assertEquals(new Integer(2), getter3.get("B")); + assertNull(getter3.get("C")); + + assertEquals("02", getter4.get("A")); + assertEquals("02", getter4.get("B")); + assertEquals("01", getter4.get("C")); + + driver.process(topic1, "A", "03"); + + assertEquals("03", getter1.get("A")); + assertEquals("02", getter1.get("B")); + assertEquals("01", getter1.get("C")); + + assertEquals(new Integer(3), getter2.get("A")); + assertEquals(new Integer(2), getter2.get("B")); + assertEquals(new Integer(1), getter2.get("C")); + + assertNull(getter3.get("A")); + assertEquals(new Integer(2), getter3.get("B")); + assertNull(getter3.get("C")); + + assertEquals("03", getter4.get("A")); + assertEquals("02", getter4.get("B")); + assertEquals("01", getter4.get("C")); + + driver.process(topic1, "A", null); + + assertNull(getter1.get("A")); + assertEquals("02", getter1.get("B")); + assertEquals("01", getter1.get("C")); + + assertNull(getter2.get("A")); + assertEquals(new Integer(2), getter2.get("B")); + assertEquals(new Integer(1), getter2.get("C")); + + assertNull(getter3.get("A")); + assertEquals(new Integer(2), getter3.get("B")); + assertNull(getter3.get("C")); + + assertNull(getter4.get("A")); + assertEquals("02", getter4.get("B")); + assertEquals("01", getter4.get("C")); + } + + @Test + public void testStateStoreLazyEval() throws IOException { + String topic1 = "topic1"; + String topic2 = "topic2"; + + final KStreamBuilder builder = new KStreamBuilder(); + + KTableImpl table1 = + (KTableImpl) builder.table(stringSerde, stringSerde, topic1); + KTableImpl table2 = + (KTableImpl) builder.table(stringSerde, stringSerde, topic2); + + KTableImpl table1Mapped = (KTableImpl) table1.mapValues( + new ValueMapper() { + @Override + public Integer apply(String value) { + return new Integer(value); + } + }); + KTableImpl table1MappedFiltered = (KTableImpl) table1Mapped.filter( + new Predicate() { + @Override + public boolean test(String key, Integer value) { + return (value % 2) == 0; + } + }); + + driver = new KStreamTestDriver(builder, stateDir, null, null); + driver.setTime(0L); + + // no state store should be created + assertEquals(0, driver.allStateStores().size()); } @Test @@ -230,120 +276,75 @@ public void testStateStore() throws IOException { String topic1 = "topic1"; String topic2 = "topic2"; - File stateDir = Files.createTempDirectory("test").toFile(); - try { - KStreamBuilder builder = new KStreamBuilder(); - - KTableImpl table1 = - (KTableImpl) builder.table(stringSerde, stringSerde, topic1); - KTableImpl table2 = - (KTableImpl) builder.table(stringSerde, stringSerde, topic2); - - KTableImpl table1Mapped = (KTableImpl) table1.mapValues( - new ValueMapper() { - @Override - public Integer apply(String value) { - return new Integer(value); - } - }); - KTableImpl table1MappedFiltered = (KTableImpl) table1Mapped.filter( - new Predicate() { - @Override - public boolean test(String key, Integer value) { - return (value % 2) == 0; - } - }); - - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); - driver.setTime(0L); - - // no state store should be created - assertEquals(0, driver.allStateStores().size()); - - } finally { - Utils.delete(stateDir); - } + final KStreamBuilder builder = new KStreamBuilder(); - try { - KStreamBuilder builder = new KStreamBuilder(); - - KTableImpl table1 = - (KTableImpl) builder.table(stringSerde, stringSerde, topic1); - KTableImpl table2 = - (KTableImpl) builder.table(stringSerde, stringSerde, topic2); - - KTableImpl table1Mapped = (KTableImpl) table1.mapValues( - new ValueMapper() { - @Override - public Integer apply(String value) { - return new Integer(value); - } - }); - KTableImpl table1MappedFiltered = (KTableImpl) table1Mapped.filter( - new Predicate() { - @Override - public boolean test(String key, Integer value) { - return (value % 2) == 0; - } - }); - table2.join(table1MappedFiltered, - new ValueJoiner() { - @Override - public String apply(String v1, Integer v2) { - return v1 + v2; - } - }); - - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); - driver.setTime(0L); - - // two state store should be created - assertEquals(2, driver.allStateStores().size()); - - } finally { - Utils.delete(stateDir); - } + KTableImpl table1 = + (KTableImpl) builder.table(stringSerde, stringSerde, topic1); + KTableImpl table2 = + (KTableImpl) builder.table(stringSerde, stringSerde, topic2); + + KTableImpl table1Mapped = (KTableImpl) table1.mapValues( + new ValueMapper() { + @Override + public Integer apply(String value) { + return new Integer(value); + } + }); + KTableImpl table1MappedFiltered = (KTableImpl) table1Mapped.filter( + new Predicate() { + @Override + public boolean test(String key, Integer value) { + return (value % 2) == 0; + } + }); + table2.join(table1MappedFiltered, + new ValueJoiner() { + @Override + public String apply(String v1, Integer v2) { + return v1 + v2; + } + }); + + driver = new KStreamTestDriver(builder, stateDir, null, null); + driver.setTime(0L); + + // two state store should be created + assertEquals(2, driver.allStateStores().size()); } @Test public void testRepartition() throws IOException { String topic1 = "topic1"; - File stateDir = Files.createTempDirectory("test").toFile(); - try { - KStreamBuilder builder = new KStreamBuilder(); - - KTableImpl table1 = - (KTableImpl) builder.table(stringSerde, stringSerde, topic1); + final KStreamBuilder builder = new KStreamBuilder(); - KTableImpl table1Aggregated = (KTableImpl) table1 - .groupBy(MockKeyValueMapper.NoOpKeyValueMapper()) - .aggregate(MockInitializer.STRING_INIT, MockAggregator.STRING_ADDER, MockAggregator.STRING_REMOVER, "mock-result1"); + KTableImpl table1 = + (KTableImpl) builder.table(stringSerde, stringSerde, topic1); + KTableImpl table1Aggregated = (KTableImpl) table1 + .groupBy(MockKeyValueMapper.NoOpKeyValueMapper()) + .aggregate(MockInitializer.STRING_INIT, MockAggregator.STRING_ADDER, MockAggregator.STRING_REMOVER, "mock-result1"); - KTableImpl table1Reduced = (KTableImpl) table1 - .groupBy(MockKeyValueMapper.NoOpKeyValueMapper()) - .reduce(MockReducer.STRING_ADDER, MockReducer.STRING_REMOVER, "mock-result2"); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, stringSerde, stringSerde); - driver.setTime(0L); + KTableImpl table1Reduced = (KTableImpl) table1 + .groupBy(MockKeyValueMapper.NoOpKeyValueMapper()) + .reduce(MockReducer.STRING_ADDER, MockReducer.STRING_REMOVER, "mock-result2"); - // three state store should be created, one for source, one for aggregate and one for reduce - assertEquals(3, driver.allStateStores().size()); + driver = new KStreamTestDriver(builder, stateDir, stringSerde, stringSerde); + driver.setTime(0L); - // contains the corresponding repartition source / sink nodes - assertTrue(driver.allProcessorNames().contains("KSTREAM-SINK-0000000003")); - assertTrue(driver.allProcessorNames().contains("KSTREAM-SOURCE-0000000004")); - assertTrue(driver.allProcessorNames().contains("KSTREAM-SINK-0000000007")); - assertTrue(driver.allProcessorNames().contains("KSTREAM-SOURCE-0000000008")); + // three state store should be created, one for source, one for aggregate and one for reduce + assertEquals(3, driver.allStateStores().size()); - assertNotNull(((ChangedSerializer) ((SinkNode) driver.processor("KSTREAM-SINK-0000000003")).valueSerializer()).inner()); - assertNotNull(((ChangedDeserializer) ((SourceNode) driver.processor("KSTREAM-SOURCE-0000000004")).valueDeserializer()).inner()); - assertNotNull(((ChangedSerializer) ((SinkNode) driver.processor("KSTREAM-SINK-0000000007")).valueSerializer()).inner()); - assertNotNull(((ChangedDeserializer) ((SourceNode) driver.processor("KSTREAM-SOURCE-0000000008")).valueDeserializer()).inner()); + // contains the corresponding repartition source / sink nodes + assertTrue(driver.allProcessorNames().contains("KSTREAM-SINK-0000000003")); + assertTrue(driver.allProcessorNames().contains("KSTREAM-SOURCE-0000000004")); + assertTrue(driver.allProcessorNames().contains("KSTREAM-SINK-0000000007")); + assertTrue(driver.allProcessorNames().contains("KSTREAM-SOURCE-0000000008")); - } finally { - Utils.delete(stateDir); - } + assertNotNull(((ChangedSerializer) ((SinkNode) driver.processor("KSTREAM-SINK-0000000003")).valueSerializer()).inner()); + assertNotNull(((ChangedDeserializer) ((SourceNode) driver.processor("KSTREAM-SOURCE-0000000004")).valueDeserializer()).inner()); + assertNotNull(((ChangedSerializer) ((SinkNode) driver.processor("KSTREAM-SINK-0000000007")).valueSerializer()).inner()); + assertNotNull(((ChangedDeserializer) ((SourceNode) driver.processor("KSTREAM-SOURCE-0000000008")).valueDeserializer()).inner()); } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java index f6ebbe1a0831b..16015fecf8b86 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableJoinTest.java @@ -19,17 +19,19 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockValueJoiner; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.io.File; -import java.nio.file.Files; +import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.HashSet; @@ -42,309 +44,291 @@ public class KTableKTableJoinTest { - private String topic1 = "topic1"; - private String topic2 = "topic2"; + final private String topic1 = "topic1"; + final private String topic2 = "topic2"; - final private Serde intSerde = new Serdes.IntegerSerde(); - final private Serde stringSerde = new Serdes.StringSerde(); + final private Serde intSerde = Serdes.Integer(); + final private Serde stringSerde = Serdes.String(); - private ValueJoiner joiner = new ValueJoiner() { - @Override - public String apply(String value1, String value2) { - return value1 + "+" + value2; - } - }; + private KStreamTestDriver driver = null; + private File stateDir = null; - private static class JoinedKeyValue extends KeyValue { - public JoinedKeyValue(Integer key, String value) { - super(key, value); + @After + public void tearDown() { + if (driver != null) { + driver.close(); } + driver = null; + } + + @Before + public void setUp() throws IOException { + stateDir = TestUtils.tempDirectory("kafka-test"); } @Test public void testJoin() throws Exception { - File baseDir = Files.createTempDirectory("test").toFile(); - try { - - KStreamBuilder builder = new KStreamBuilder(); + final KStreamBuilder builder = new KStreamBuilder(); - final int[] expectedKeys = new int[]{0, 1, 2, 3}; + final int[] expectedKeys = new int[]{0, 1, 2, 3}; - KTable table1; - KTable table2; - KTable joined; - MockProcessorSupplier processor; + KTable table1; + KTable table2; + KTable joined; + MockProcessorSupplier processor; - processor = new MockProcessorSupplier<>(); - table1 = builder.table(intSerde, stringSerde, topic1); - table2 = builder.table(intSerde, stringSerde, topic2); - joined = table1.join(table2, joiner); - joined.toStream().process(processor); + processor = new MockProcessorSupplier<>(); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); + joined = table1.join(table2, MockValueJoiner.STRING_JOINER); + joined.toStream().process(processor); - Collection> copartitionGroups = builder.copartitionGroups(); + Collection> copartitionGroups = builder.copartitionGroups(); - assertEquals(1, copartitionGroups.size()); - assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); + assertEquals(1, copartitionGroups.size()); + assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); - KTableValueGetterSupplier getterSupplier = ((KTableImpl) joined).valueGetterSupplier(); + KTableValueGetterSupplier getterSupplier = ((KTableImpl) joined).valueGetterSupplier(); - KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir); - driver.setTime(0L); + driver = new KStreamTestDriver(builder, stateDir); + driver.setTime(0L); - KTableValueGetter getter = getterSupplier.get(); - getter.init(driver.context()); + KTableValueGetter getter = getterSupplier.get(); + getter.init(driver.context()); - // push two items to the primary stream. the other table is empty + // push two items to the primary stream. the other table is empty - for (int i = 0; i < 2; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } - - processor.checkAndClearProcessResult("0:null", "1:null"); - checkJoinedValues(getter, kv(0, null), kv(1, null), kv(2, null), kv(3, null)); + for (int i = 0; i < 2; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - // push two items to the other stream. this should produce two items. + processor.checkAndClearProcessResult("0:null", "1:null"); + checkJoinedValues(getter, kv(0, null), kv(1, null), kv(2, null), kv(3, null)); - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); - } + // push two items to the other stream. this should produce two items. - processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); - checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null)); + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); + } - // push all four items to the primary stream. this should produce four items. + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); + checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null)); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + // push all four items to the primary stream. this should produce four items. - processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:null", "3:null"); - checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null)); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - // push all items to the other stream. this should produce four items. - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:null", "3:null"); + checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null)); - processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); - checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); + // push all items to the other stream. this should produce four items. + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - // push all four items to the primary stream. this should produce four items. + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + // push all four items to the primary stream. this should produce four items. - processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); - checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - // push two items with null to the other stream as deletes. this should produce two item. + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], null); - } + // push two items with null to the other stream as deletes. this should produce two item. - processor.checkAndClearProcessResult("0:null", "1:null"); - checkJoinedValues(getter, kv(0, null), kv(1, null), kv(2, "X2+YY2"), kv(3, "X3+YY3")); + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], null); + } - // push all four items to the primary stream. this should produce four items. + processor.checkAndClearProcessResult("0:null", "1:null"); + checkJoinedValues(getter, kv(0, null), kv(1, null), kv(2, "X2+YY2"), kv(3, "X3+YY3")); - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + // push all four items to the primary stream. this should produce four items. - processor.checkAndClearProcessResult("0:null", "1:null", "2:XX2+YY2", "3:XX3+YY3"); - checkJoinedValues(getter, kv(0, null), kv(1, null), kv(2, "XX2+YY2"), kv(3, "XX3+YY3")); - - } finally { - Utils.delete(baseDir); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } + + processor.checkAndClearProcessResult("0:null", "1:null", "2:XX2+YY2", "3:XX3+YY3"); + checkJoinedValues(getter, kv(0, null), kv(1, null), kv(2, "XX2+YY2"), kv(3, "XX3+YY3")); } @Test public void testNotSendingOldValues() throws Exception { - File baseDir = Files.createTempDirectory("test").toFile(); - try { - - KStreamBuilder builder = new KStreamBuilder(); + final KStreamBuilder builder = new KStreamBuilder(); - final int[] expectedKeys = new int[]{0, 1, 2, 3}; + final int[] expectedKeys = new int[]{0, 1, 2, 3}; - KTable table1; - KTable table2; - KTable joined; - MockProcessorSupplier proc; + KTable table1; + KTable table2; + KTable joined; + MockProcessorSupplier proc; - table1 = builder.table(intSerde, stringSerde, topic1); - table2 = builder.table(intSerde, stringSerde, topic2); - joined = table1.join(table2, joiner); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); + joined = table1.join(table2, MockValueJoiner.STRING_JOINER); - proc = new MockProcessorSupplier<>(); - builder.addProcessor("proc", proc, ((KTableImpl) joined).name); + proc = new MockProcessorSupplier<>(); + builder.addProcessor("proc", proc, ((KTableImpl) joined).name); - KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir); - driver.setTime(0L); + driver = new KStreamTestDriver(builder, stateDir); + driver.setTime(0L); - assertFalse(((KTableImpl) table1).sendingOldValueEnabled()); - assertFalse(((KTableImpl) table2).sendingOldValueEnabled()); - assertFalse(((KTableImpl) joined).sendingOldValueEnabled()); + assertFalse(((KTableImpl) table1).sendingOldValueEnabled()); + assertFalse(((KTableImpl) table2).sendingOldValueEnabled()); + assertFalse(((KTableImpl) joined).sendingOldValueEnabled()); - // push two items to the primary stream. the other table is empty + // push two items to the primary stream. the other table is empty - for (int i = 0; i < 2; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } - - proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)"); - - // push two items to the other stream. this should produce two items. + for (int i = 0; i < 2; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)"); - proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)"); + // push two items to the other stream. this should produce two items. - // push all four items to the primary stream. this should produce four items. + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)"); - proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(null<-null)", "3:(null<-null)"); + // push all four items to the primary stream. this should produce four items. - // push all items to the other stream. this should produce four items. - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); + proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(null<-null)", "3:(null<-null)"); - // push all four items to the primary stream. this should produce four items. + // push all items to the other stream. this should produce four items. + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); - proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); + // push all four items to the primary stream. this should produce four items. - // push two items with null to the other stream as deletes. this should produce two item. + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], null); - } + proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); - proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)"); + // push two items with null to the other stream as deletes. this should produce two item. - // push all four items to the primary stream. this should produce four items. + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], null); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)"); - proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)", "2:(XX2+YY2<-null)", "3:(XX3+YY3<-null)"); + // push all four items to the primary stream. this should produce four items. - } finally { - Utils.delete(baseDir); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } + + proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)", "2:(XX2+YY2<-null)", "3:(XX3+YY3<-null)"); } @Test public void testSendingOldValues() throws Exception { - File baseDir = Files.createTempDirectory("test").toFile(); - try { - - KStreamBuilder builder = new KStreamBuilder(); + final KStreamBuilder builder = new KStreamBuilder(); - final int[] expectedKeys = new int[]{0, 1, 2, 3}; + final int[] expectedKeys = new int[]{0, 1, 2, 3}; - KTable table1; - KTable table2; - KTable joined; - MockProcessorSupplier proc; + KTable table1; + KTable table2; + KTable joined; + MockProcessorSupplier proc; - table1 = builder.table(intSerde, stringSerde, topic1); - table2 = builder.table(intSerde, stringSerde, topic2); - joined = table1.join(table2, joiner); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); + joined = table1.join(table2, MockValueJoiner.STRING_JOINER); - ((KTableImpl) joined).enableSendingOldValues(); + ((KTableImpl) joined).enableSendingOldValues(); - proc = new MockProcessorSupplier<>(); - builder.addProcessor("proc", proc, ((KTableImpl) joined).name); + proc = new MockProcessorSupplier<>(); + builder.addProcessor("proc", proc, ((KTableImpl) joined).name); - KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir); - driver.setTime(0L); + driver = new KStreamTestDriver(builder, stateDir); + driver.setTime(0L); - assertTrue(((KTableImpl) table1).sendingOldValueEnabled()); - assertTrue(((KTableImpl) table2).sendingOldValueEnabled()); - assertTrue(((KTableImpl) joined).sendingOldValueEnabled()); - - // push two items to the primary stream. the other table is empty - - for (int i = 0; i < 2; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + assertTrue(((KTableImpl) table1).sendingOldValueEnabled()); + assertTrue(((KTableImpl) table2).sendingOldValueEnabled()); + assertTrue(((KTableImpl) joined).sendingOldValueEnabled()); - proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)"); + // push two items to the primary stream. the other table is empty - // push two items to the other stream. this should produce two items. + for (int i = 0; i < 2; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)"); - proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)"); + // push two items to the other stream. this should produce two items. - // push all four items to the primary stream. this should produce four items. + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)"); - proc.checkAndClearProcessResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(null<-null)", "3:(null<-null)"); + // push all four items to the primary stream. this should produce four items. - // push all items to the other stream. this should produce four items. - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - proc.checkAndClearProcessResult("0:(X0+YY0<-X0+Y0)", "1:(X1+YY1<-X1+Y1)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); + proc.checkAndClearProcessResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(null<-null)", "3:(null<-null)"); - // push all four items to the primary stream. this should produce four items. + // push all items to the other stream. this should produce four items. + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(X0+YY0<-X0+Y0)", "1:(X1+YY1<-X1+Y1)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); - proc.checkAndClearProcessResult("0:(X0+YY0<-X0+YY0)", "1:(X1+YY1<-X1+YY1)", "2:(X2+YY2<-X2+YY2)", "3:(X3+YY3<-X3+YY3)"); + // push all four items to the primary stream. this should produce four items. - // push two items with null to the other stream as deletes. this should produce two item. + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], null); - } + proc.checkAndClearProcessResult("0:(X0+YY0<-X0+YY0)", "1:(X1+YY1<-X1+YY1)", "2:(X2+YY2<-X2+YY2)", "3:(X3+YY3<-X3+YY3)"); - proc.checkAndClearProcessResult("0:(null<-X0+YY0)", "1:(null<-X1+YY1)"); + // push two items with null to the other stream as deletes. this should produce two item. - // push all four items to the primary stream. this should produce four items. + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], null); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(null<-X0+YY0)", "1:(null<-X1+YY1)"); - proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)", "2:(XX2+YY2<-X2+YY2)", "3:(XX3+YY3<-X3+YY3)"); + // push all four items to the primary stream. this should produce four items. - } finally { - Utils.delete(baseDir); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } + + proc.checkAndClearProcessResult("0:(null<-null)", "1:(null<-null)", "2:(XX2+YY2<-X2+YY2)", "3:(XX3+YY3<-X3+YY3)"); } - private JoinedKeyValue kv(Integer key, String value) { - return new JoinedKeyValue(key, value); + private KeyValue kv(Integer key, String value) { + return new KeyValue<>(key, value); } - private void checkJoinedValues(KTableValueGetter getter, JoinedKeyValue... expected) { - for (JoinedKeyValue kv : expected) { + private void checkJoinedValues(KTableValueGetter getter, KeyValue... expected) { + for (KeyValue kv : expected) { String value = getter.get(kv.key); if (kv.value == null) { assertNull(value); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java index 449ea0501f439..5132ce30b3e6b 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableLeftJoinTest.java @@ -19,18 +19,19 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.kstream.KeyValueMapper; -import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockValueJoiner; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.io.File; -import java.nio.file.Files; +import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.HashSet; @@ -43,313 +44,287 @@ public class KTableKTableLeftJoinTest { - private String topic1 = "topic1"; - private String topic2 = "topic2"; + final private String topic1 = "topic1"; + final private String topic2 = "topic2"; - final private Serde intSerde = new Serdes.IntegerSerde(); - final private Serde stringSerde = new Serdes.StringSerde(); + final private Serde intSerde = Serdes.Integer(); + final private Serde stringSerde = Serdes.String(); - private ValueJoiner joiner = new ValueJoiner() { - @Override - public String apply(String value1, String value2) { - return value1 + "+" + value2; - } - }; - - private KeyValueMapper> keyValueMapper = - new KeyValueMapper>() { - @Override - public KeyValue apply(Integer key, String value) { - return KeyValue.pair(key, value); - } - }; + private KStreamTestDriver driver = null; + private File stateDir = null; - private static class JoinedKeyValue extends KeyValue { - public JoinedKeyValue(Integer key, String value) { - super(key, value); + @After + public void tearDown() { + if (driver != null) { + driver.close(); } + driver = null; + } + + @Before + public void setUp() throws IOException { + stateDir = TestUtils.tempDirectory("kafka-test"); } @Test public void testJoin() throws Exception { - File baseDir = Files.createTempDirectory("test").toFile(); - try { - - KStreamBuilder builder = new KStreamBuilder(); - - final int[] expectedKeys = new int[]{0, 1, 2, 3}; + final KStreamBuilder builder = new KStreamBuilder(); - KTable table1 = builder.table(intSerde, stringSerde, topic1); - KTable table2 = builder.table(intSerde, stringSerde, topic2); - KTable joined = table1.leftJoin(table2, joiner); - MockProcessorSupplier processor; - processor = new MockProcessorSupplier<>(); - joined.toStream().process(processor); + final int[] expectedKeys = new int[]{0, 1, 2, 3}; - Collection> copartitionGroups = builder.copartitionGroups(); + KTable table1 = builder.table(intSerde, stringSerde, topic1); + KTable table2 = builder.table(intSerde, stringSerde, topic2); + KTable joined = table1.leftJoin(table2, MockValueJoiner.STRING_JOINER); + MockProcessorSupplier processor; + processor = new MockProcessorSupplier<>(); + joined.toStream().process(processor); - assertEquals(1, copartitionGroups.size()); - assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); + Collection> copartitionGroups = builder.copartitionGroups(); - KTableValueGetterSupplier getterSupplier = ((KTableImpl) joined).valueGetterSupplier(); + assertEquals(1, copartitionGroups.size()); + assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); - KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir); - driver.setTime(0L); + KTableValueGetterSupplier getterSupplier = ((KTableImpl) joined).valueGetterSupplier(); - KTableValueGetter getter = getterSupplier.get(); - getter.init(driver.context()); + driver = new KStreamTestDriver(builder, stateDir); + driver.setTime(0L); - // push two items to the primary stream. the other table is empty + KTableValueGetter getter = getterSupplier.get(); + getter.init(driver.context()); - for (int i = 0; i < 2; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } - - processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); - checkJoinedValues(getter, kv(0, "X0+null"), kv(1, "X1+null"), kv(2, null), kv(3, null)); + // push two items to the primary stream. the other table is empty - // push two items to the other stream. this should produce two items. + for (int i = 0; i < 2; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); + checkJoinedValues(getter, kv(0, "X0+null"), kv(1, "X1+null"), kv(2, null), kv(3, null)); - processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); - checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null)); + // push two items to the other stream. this should produce two items. - // push all four items to the primary stream. this should produce four items. + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); + checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null)); - processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null"); - checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, "X2+null"), kv(3, "X3+null")); + // push all four items to the primary stream. this should produce four items. - // push all items to the other stream. this should produce four items. - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); - checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null"); + checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, "X2+null"), kv(3, "X3+null")); - // push all four items to the primary stream. this should produce four items. + // push all items to the other stream. this should produce four items. + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); - processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); - checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); + // push all four items to the primary stream. this should produce four items. - // push two items with null to the other stream as deletes. this should produce two item. + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], null); - } + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); - processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); - checkJoinedValues(getter, kv(0, "X0+null"), kv(1, "X1+null"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); + // push two items with null to the other stream as deletes. this should produce two item. - // push all four items to the primary stream. this should produce four items. + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], null); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); + checkJoinedValues(getter, kv(0, "X0+null"), kv(1, "X1+null"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); - processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+YY2", "3:XX3+YY3"); - checkJoinedValues(getter, kv(0, "XX0+null"), kv(1, "XX1+null"), kv(2, "XX2+YY2"), kv(3, "XX3+YY3")); + // push all four items to the primary stream. this should produce four items. - } finally { - Utils.delete(baseDir); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } + + processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+YY2", "3:XX3+YY3"); + checkJoinedValues(getter, kv(0, "XX0+null"), kv(1, "XX1+null"), kv(2, "XX2+YY2"), kv(3, "XX3+YY3")); } @Test public void testNotSendingOldValue() throws Exception { - File baseDir = Files.createTempDirectory("test").toFile(); - try { - - KStreamBuilder builder = new KStreamBuilder(); + final KStreamBuilder builder = new KStreamBuilder(); - final int[] expectedKeys = new int[]{0, 1, 2, 3}; + final int[] expectedKeys = new int[]{0, 1, 2, 3}; - KTable table1; - KTable table2; - KTable joined; - MockProcessorSupplier proc; + KTable table1; + KTable table2; + KTable joined; + MockProcessorSupplier proc; - table1 = builder.table(intSerde, stringSerde, topic1); - table2 = builder.table(intSerde, stringSerde, topic2); - joined = table1.leftJoin(table2, joiner); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); + joined = table1.leftJoin(table2, MockValueJoiner.STRING_JOINER); - proc = new MockProcessorSupplier<>(); - builder.addProcessor("proc", proc, ((KTableImpl) joined).name); + proc = new MockProcessorSupplier<>(); + builder.addProcessor("proc", proc, ((KTableImpl) joined).name); - KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir); - driver.setTime(0L); + driver = new KStreamTestDriver(builder, stateDir); + driver.setTime(0L); - assertFalse(((KTableImpl) table1).sendingOldValueEnabled()); - assertFalse(((KTableImpl) table2).sendingOldValueEnabled()); - assertFalse(((KTableImpl) joined).sendingOldValueEnabled()); + assertFalse(((KTableImpl) table1).sendingOldValueEnabled()); + assertFalse(((KTableImpl) table2).sendingOldValueEnabled()); + assertFalse(((KTableImpl) joined).sendingOldValueEnabled()); - // push two items to the primary stream. the other table is empty - - for (int i = 0; i < 2; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + // push two items to the primary stream. the other table is empty - proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)"); - - // push two items to the other stream. this should produce two items. + for (int i = 0; i < 2; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)"); - proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)"); + // push two items to the other stream. this should produce two items. - // push all four items to the primary stream. this should produce four items. + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)"); - proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(X2+null<-null)", "3:(X3+null<-null)"); + // push all four items to the primary stream. this should produce four items. - // push all items to the other stream. this should produce four items. - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); + proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(X2+null<-null)", "3:(X3+null<-null)"); - // push all four items to the primary stream. this should produce four items. + // push all items to the other stream. this should produce four items. + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); - proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); + // push all four items to the primary stream. this should produce four items. - // push two items with null to the other stream as deletes. this should produce two item. + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], null); - } + proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); - proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)"); + // push two items with null to the other stream as deletes. this should produce two item. - // push all four items to the primary stream. this should produce four items. + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], null); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)"); - proc.checkAndClearProcessResult("0:(XX0+null<-null)", "1:(XX1+null<-null)", "2:(XX2+YY2<-null)", "3:(XX3+YY3<-null)"); + // push all four items to the primary stream. this should produce four items. - } finally { - Utils.delete(baseDir); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } + + proc.checkAndClearProcessResult("0:(XX0+null<-null)", "1:(XX1+null<-null)", "2:(XX2+YY2<-null)", "3:(XX3+YY3<-null)"); } @Test public void testSendingOldValue() throws Exception { - File baseDir = Files.createTempDirectory("test").toFile(); - try { - - KStreamBuilder builder = new KStreamBuilder(); - - final int[] expectedKeys = new int[]{0, 1, 2, 3}; - - KTable table1; - KTable table2; - KTable joined; - MockProcessorSupplier proc; + final KStreamBuilder builder = new KStreamBuilder(); - table1 = builder.table(intSerde, stringSerde, topic1); - table2 = builder.table(intSerde, stringSerde, topic2); - joined = table1.leftJoin(table2, joiner); + final int[] expectedKeys = new int[]{0, 1, 2, 3}; - ((KTableImpl) joined).enableSendingOldValues(); + KTable table1; + KTable table2; + KTable joined; + MockProcessorSupplier proc; - proc = new MockProcessorSupplier<>(); - builder.addProcessor("proc", proc, ((KTableImpl) joined).name); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); + joined = table1.leftJoin(table2, MockValueJoiner.STRING_JOINER); - KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir); - driver.setTime(0L); + ((KTableImpl) joined).enableSendingOldValues(); - assertTrue(((KTableImpl) table1).sendingOldValueEnabled()); - assertTrue(((KTableImpl) table2).sendingOldValueEnabled()); - assertTrue(((KTableImpl) joined).sendingOldValueEnabled()); + proc = new MockProcessorSupplier<>(); + builder.addProcessor("proc", proc, ((KTableImpl) joined).name); - // push two items to the primary stream. the other table is empty + driver = new KStreamTestDriver(builder, stateDir); + driver.setTime(0L); - for (int i = 0; i < 2; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + assertTrue(((KTableImpl) table1).sendingOldValueEnabled()); + assertTrue(((KTableImpl) table2).sendingOldValueEnabled()); + assertTrue(((KTableImpl) joined).sendingOldValueEnabled()); - proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)"); + // push two items to the primary stream. the other table is empty - // push two items to the other stream. this should produce two items. + for (int i = 0; i < 2; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)"); - proc.checkAndClearProcessResult("0:(X0+Y0<-X0+null)", "1:(X1+Y1<-X1+null)"); + // push two items to the other stream. this should produce two items. - // push all four items to the primary stream. this should produce four items. + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(X0+Y0<-X0+null)", "1:(X1+Y1<-X1+null)"); - proc.checkAndClearProcessResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(X2+null<-null)", "3:(X3+null<-null)"); + // push all four items to the primary stream. this should produce four items. - // push all items to the other stream. this should produce four items. - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - proc.checkAndClearProcessResult("0:(X0+YY0<-X0+Y0)", "1:(X1+YY1<-X1+Y1)", "2:(X2+YY2<-X2+null)", "3:(X3+YY3<-X3+null)"); + proc.checkAndClearProcessResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(X2+null<-null)", "3:(X3+null<-null)"); - // push all four items to the primary stream. this should produce four items. + // push all items to the other stream. this should produce four items. + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(X0+YY0<-X0+Y0)", "1:(X1+YY1<-X1+Y1)", "2:(X2+YY2<-X2+null)", "3:(X3+YY3<-X3+null)"); - proc.checkAndClearProcessResult("0:(X0+YY0<-X0+YY0)", "1:(X1+YY1<-X1+YY1)", "2:(X2+YY2<-X2+YY2)", "3:(X3+YY3<-X3+YY3)"); + // push all four items to the primary stream. this should produce four items. - // push two items with null to the other stream as deletes. this should produce two item. + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], null); - } + proc.checkAndClearProcessResult("0:(X0+YY0<-X0+YY0)", "1:(X1+YY1<-X1+YY1)", "2:(X2+YY2<-X2+YY2)", "3:(X3+YY3<-X3+YY3)"); - proc.checkAndClearProcessResult("0:(X0+null<-X0+YY0)", "1:(X1+null<-X1+YY1)"); + // push two items with null to the other stream as deletes. this should produce two item. - // push all four items to the primary stream. this should produce four items. + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], null); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(X0+null<-X0+YY0)", "1:(X1+null<-X1+YY1)"); - proc.checkAndClearProcessResult("0:(XX0+null<-X0+null)", "1:(XX1+null<-X1+null)", "2:(XX2+YY2<-X2+YY2)", "3:(XX3+YY3<-X3+YY3)"); + // push all four items to the primary stream. this should produce four items. - } finally { - Utils.delete(baseDir); + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); } + + proc.checkAndClearProcessResult("0:(XX0+null<-X0+null)", "1:(XX1+null<-X1+null)", "2:(XX2+YY2<-X2+YY2)", "3:(XX3+YY3<-X3+YY3)"); } - private JoinedKeyValue kv(Integer key, String value) { - return new JoinedKeyValue(key, value); + private KeyValue kv(Integer key, String value) { + return new KeyValue<>(key, value); } - private void checkJoinedValues(KTableValueGetter getter, JoinedKeyValue... expected) { - for (JoinedKeyValue kv : expected) { + private void checkJoinedValues(KTableValueGetter getter, KeyValue... expected) { + for (KeyValue kv : expected) { String value = getter.get(kv.key); if (kv.value == null) { assertNull(value); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java index ea7476ae2e1b3..312455682f816 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableKTableOuterJoinTest.java @@ -19,17 +19,19 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; -import org.apache.kafka.common.utils.Utils; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.MockValueJoiner; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.io.File; -import java.nio.file.Files; +import java.io.IOException; import java.util.Arrays; import java.util.Collection; import java.util.HashSet; @@ -42,334 +44,316 @@ public class KTableKTableOuterJoinTest { - private String topic1 = "topic1"; - private String topic2 = "topic2"; + final private String topic1 = "topic1"; + final private String topic2 = "topic2"; - final private Serde intSerde = new Serdes.IntegerSerde(); - final private Serde stringSerde = new Serdes.StringSerde(); + final private Serde intSerde = Serdes.Integer(); + final private Serde stringSerde = Serdes.String(); - private ValueJoiner joiner = new ValueJoiner() { - @Override - public String apply(String value1, String value2) { - return value1 + "+" + value2; - } - }; + private KStreamTestDriver driver = null; + private File stateDir = null; - private static class JoinedKeyValue extends KeyValue { - public JoinedKeyValue(Integer key, String value) { - super(key, value); + @After + public void tearDown() { + if (driver != null) { + driver.close(); } + driver = null; + } + + @Before + public void setUp() throws IOException { + stateDir = TestUtils.tempDirectory("kafka-test"); } @Test public void testJoin() throws Exception { - File baseDir = Files.createTempDirectory("test").toFile(); - try { - - KStreamBuilder builder = new KStreamBuilder(); + KStreamBuilder builder = new KStreamBuilder(); - final int[] expectedKeys = new int[]{0, 1, 2, 3}; + final int[] expectedKeys = new int[]{0, 1, 2, 3}; - KTable table1; - KTable table2; - KTable joined; - MockProcessorSupplier processor; + KTable table1; + KTable table2; + KTable joined; + MockProcessorSupplier processor; - processor = new MockProcessorSupplier<>(); - table1 = builder.table(intSerde, stringSerde, topic1); - table2 = builder.table(intSerde, stringSerde, topic2); - joined = table1.outerJoin(table2, joiner); - joined.toStream().process(processor); + processor = new MockProcessorSupplier<>(); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); + joined = table1.outerJoin(table2, MockValueJoiner.STRING_JOINER); + joined.toStream().process(processor); - Collection> copartitionGroups = builder.copartitionGroups(); + Collection> copartitionGroups = builder.copartitionGroups(); - assertEquals(1, copartitionGroups.size()); - assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); + assertEquals(1, copartitionGroups.size()); + assertEquals(new HashSet<>(Arrays.asList(topic1, topic2)), copartitionGroups.iterator().next()); - KTableValueGetterSupplier getterSupplier = ((KTableImpl) joined).valueGetterSupplier(); + KTableValueGetterSupplier getterSupplier = ((KTableImpl) joined).valueGetterSupplier(); - KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir); - driver.setTime(0L); + driver = new KStreamTestDriver(builder, stateDir); + driver.setTime(0L); - KTableValueGetter getter = getterSupplier.get(); - getter.init(driver.context()); + KTableValueGetter getter = getterSupplier.get(); + getter.init(driver.context()); - // push two items to the primary stream. the other table is empty + // push two items to the primary stream. the other table is empty - for (int i = 0; i < 2; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } - - processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); - checkJoinedValues(getter, kv(0, "X0+null"), kv(1, "X1+null"), kv(2, null), kv(3, null)); + for (int i = 0; i < 2; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - // push two items to the other stream. this should produce two items. + processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); + checkJoinedValues(getter, kv(0, "X0+null"), kv(1, "X1+null"), kv(2, null), kv(3, null)); - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); - } + // push two items to the other stream. this should produce two items. - processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); - checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null)); - - // push all four items to the primary stream. this should produce four items. + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1"); + checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, null), kv(3, null)); - processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null"); - checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, "X2+null"), kv(3, "X3+null")); + // push all four items to the primary stream. this should produce four items. - // push all items to the other stream. this should produce four items. - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); - checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); + processor.checkAndClearProcessResult("0:X0+Y0", "1:X1+Y1", "2:X2+null", "3:X3+null"); + checkJoinedValues(getter, kv(0, "X0+Y0"), kv(1, "X1+Y1"), kv(2, "X2+null"), kv(3, "X3+null")); - // push all four items to the primary stream. this should produce four items. + // push all items to the other stream. this should produce four items. + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); - processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); - checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); + // push all four items to the primary stream. this should produce four items. - // push two items with null to the other stream as deletes. this should produce two item. + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], null); - } + processor.checkAndClearProcessResult("0:X0+YY0", "1:X1+YY1", "2:X2+YY2", "3:X3+YY3"); + checkJoinedValues(getter, kv(0, "X0+YY0"), kv(1, "X1+YY1"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); - processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); - checkJoinedValues(getter, kv(0, "X0+null"), kv(1, "X1+null"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); + // push two items with null to the other stream as deletes. this should produce two item. - // push all four items to the primary stream. this should produce four items. + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], null); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + processor.checkAndClearProcessResult("0:X0+null", "1:X1+null"); + checkJoinedValues(getter, kv(0, "X0+null"), kv(1, "X1+null"), kv(2, "X2+YY2"), kv(3, "X3+YY3")); - processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+YY2", "3:XX3+YY3"); - checkJoinedValues(getter, kv(0, "XX0+null"), kv(1, "XX1+null"), kv(2, "XX2+YY2"), kv(3, "XX3+YY3")); + // push all four items to the primary stream. this should produce four items. - // push middle two items to the primary stream with null. this should produce two items. + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - for (int i = 1; i < 3; i++) { - driver.process(topic1, expectedKeys[i], null); - } + processor.checkAndClearProcessResult("0:XX0+null", "1:XX1+null", "2:XX2+YY2", "3:XX3+YY3"); + checkJoinedValues(getter, kv(0, "XX0+null"), kv(1, "XX1+null"), kv(2, "XX2+YY2"), kv(3, "XX3+YY3")); - processor.checkAndClearProcessResult("1:null", "2:null+YY2"); - checkJoinedValues(getter, kv(0, "XX0+null"), kv(1, null), kv(2, "null+YY2"), kv(3, "XX3+YY3")); + // push middle two items to the primary stream with null. this should produce two items. - } finally { - Utils.delete(baseDir); + for (int i = 1; i < 3; i++) { + driver.process(topic1, expectedKeys[i], null); } + + processor.checkAndClearProcessResult("1:null", "2:null+YY2"); + checkJoinedValues(getter, kv(0, "XX0+null"), kv(1, null), kv(2, "null+YY2"), kv(3, "XX3+YY3")); } @Test public void testNotSendingOldValue() throws Exception { - File baseDir = Files.createTempDirectory("test").toFile(); - try { - - KStreamBuilder builder = new KStreamBuilder(); + final KStreamBuilder builder = new KStreamBuilder(); - final int[] expectedKeys = new int[]{0, 1, 2, 3}; + final int[] expectedKeys = new int[]{0, 1, 2, 3}; - KTable table1; - KTable table2; - KTable joined; - MockProcessorSupplier proc; + KTable table1; + KTable table2; + KTable joined; + MockProcessorSupplier proc; - table1 = builder.table(intSerde, stringSerde, topic1); - table2 = builder.table(intSerde, stringSerde, topic2); - joined = table1.outerJoin(table2, joiner); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); + joined = table1.outerJoin(table2, MockValueJoiner.STRING_JOINER); - proc = new MockProcessorSupplier<>(); - builder.addProcessor("proc", proc, ((KTableImpl) joined).name); + proc = new MockProcessorSupplier<>(); + builder.addProcessor("proc", proc, ((KTableImpl) joined).name); - KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir); - driver.setTime(0L); + driver = new KStreamTestDriver(builder, stateDir); + driver.setTime(0L); - assertFalse(((KTableImpl) table1).sendingOldValueEnabled()); - assertFalse(((KTableImpl) table2).sendingOldValueEnabled()); - assertFalse(((KTableImpl) joined).sendingOldValueEnabled()); + assertFalse(((KTableImpl) table1).sendingOldValueEnabled()); + assertFalse(((KTableImpl) table2).sendingOldValueEnabled()); + assertFalse(((KTableImpl) joined).sendingOldValueEnabled()); - // push two items to the primary stream. the other table is empty + // push two items to the primary stream. the other table is empty - for (int i = 0; i < 2; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } - - proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)"); + for (int i = 0; i < 2; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - // push two items to the other stream. this should produce two items. + proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)"); - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); - } + // push two items to the other stream. this should produce two items. - proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)"); - - // push all four items to the primary stream. this should produce four items. + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)"); - proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(X2+null<-null)", "3:(X3+null<-null)"); + // push all four items to the primary stream. this should produce four items. - // push all items to the other stream. this should produce four items. - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); + proc.checkAndClearProcessResult("0:(X0+Y0<-null)", "1:(X1+Y1<-null)", "2:(X2+null<-null)", "3:(X3+null<-null)"); - // push all four items to the primary stream. this should produce four items. + // push all items to the other stream. this should produce four items. + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); - proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); + // push all four items to the primary stream. this should produce four items. - // push two items with null to the other stream as deletes. this should produce two item. + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], null); - } + proc.checkAndClearProcessResult("0:(X0+YY0<-null)", "1:(X1+YY1<-null)", "2:(X2+YY2<-null)", "3:(X3+YY3<-null)"); - proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)"); + // push two items with null to the other stream as deletes. this should produce two item. - // push all four items to the primary stream. this should produce four items. + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], null); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)"); - proc.checkAndClearProcessResult("0:(XX0+null<-null)", "1:(XX1+null<-null)", "2:(XX2+YY2<-null)", "3:(XX3+YY3<-null)"); + // push all four items to the primary stream. this should produce four items. - // push middle two items to the primary stream with null. this should produce two items. + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - for (int i = 1; i < 3; i++) { - driver.process(topic1, expectedKeys[i], null); - } + proc.checkAndClearProcessResult("0:(XX0+null<-null)", "1:(XX1+null<-null)", "2:(XX2+YY2<-null)", "3:(XX3+YY3<-null)"); - proc.checkAndClearProcessResult("1:(null<-null)", "2:(null+YY2<-null)"); + // push middle two items to the primary stream with null. this should produce two items. - } finally { - Utils.delete(baseDir); + for (int i = 1; i < 3; i++) { + driver.process(topic1, expectedKeys[i], null); } + + proc.checkAndClearProcessResult("1:(null<-null)", "2:(null+YY2<-null)"); } @Test public void testSendingOldValue() throws Exception { - File baseDir = Files.createTempDirectory("test").toFile(); - try { - - KStreamBuilder builder = new KStreamBuilder(); + final KStreamBuilder builder = new KStreamBuilder(); - final int[] expectedKeys = new int[]{0, 1, 2, 3}; + final int[] expectedKeys = new int[]{0, 1, 2, 3}; - KTable table1; - KTable table2; - KTable joined; - MockProcessorSupplier proc; + KTable table1; + KTable table2; + KTable joined; + MockProcessorSupplier proc; - table1 = builder.table(intSerde, stringSerde, topic1); - table2 = builder.table(intSerde, stringSerde, topic2); - joined = table1.outerJoin(table2, joiner); + table1 = builder.table(intSerde, stringSerde, topic1); + table2 = builder.table(intSerde, stringSerde, topic2); + joined = table1.outerJoin(table2, MockValueJoiner.STRING_JOINER); - ((KTableImpl) joined).enableSendingOldValues(); + ((KTableImpl) joined).enableSendingOldValues(); - proc = new MockProcessorSupplier<>(); - builder.addProcessor("proc", proc, ((KTableImpl) joined).name); + proc = new MockProcessorSupplier<>(); + builder.addProcessor("proc", proc, ((KTableImpl) joined).name); - KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir); - driver.setTime(0L); + driver = new KStreamTestDriver(builder, stateDir); + driver.setTime(0L); - assertTrue(((KTableImpl) table1).sendingOldValueEnabled()); - assertTrue(((KTableImpl) table2).sendingOldValueEnabled()); - assertTrue(((KTableImpl) joined).sendingOldValueEnabled()); - - // push two items to the primary stream. the other table is empty - - for (int i = 0; i < 2; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + assertTrue(((KTableImpl) table1).sendingOldValueEnabled()); + assertTrue(((KTableImpl) table2).sendingOldValueEnabled()); + assertTrue(((KTableImpl) joined).sendingOldValueEnabled()); - proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)"); + // push two items to the primary stream. the other table is empty - // push two items to the other stream. this should produce two items. + for (int i = 0; i < 2; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(X0+null<-null)", "1:(X1+null<-null)"); - proc.checkAndClearProcessResult("0:(X0+Y0<-X0+null)", "1:(X1+Y1<-X1+null)"); + // push two items to the other stream. this should produce two items. - // push all four items to the primary stream. this should produce four items. + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], "Y" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(X0+Y0<-X0+null)", "1:(X1+Y1<-X1+null)"); - proc.checkAndClearProcessResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(X2+null<-null)", "3:(X3+null<-null)"); + // push all four items to the primary stream. this should produce four items. - // push all items to the other stream. this should produce four items. - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); - } + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - proc.checkAndClearProcessResult("0:(X0+YY0<-X0+Y0)", "1:(X1+YY1<-X1+Y1)", "2:(X2+YY2<-X2+null)", "3:(X3+YY3<-X3+null)"); + proc.checkAndClearProcessResult("0:(X0+Y0<-X0+Y0)", "1:(X1+Y1<-X1+Y1)", "2:(X2+null<-null)", "3:(X3+null<-null)"); - // push all four items to the primary stream. this should produce four items. + // push all items to the other stream. this should produce four items. + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic2, expectedKeys[i], "YY" + expectedKeys[i]); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(X0+YY0<-X0+Y0)", "1:(X1+YY1<-X1+Y1)", "2:(X2+YY2<-X2+null)", "3:(X3+YY3<-X3+null)"); - proc.checkAndClearProcessResult("0:(X0+YY0<-X0+YY0)", "1:(X1+YY1<-X1+YY1)", "2:(X2+YY2<-X2+YY2)", "3:(X3+YY3<-X3+YY3)"); + // push all four items to the primary stream. this should produce four items. - // push two items with null to the other stream as deletes. this should produce two item. + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "X" + expectedKeys[i]); + } - for (int i = 0; i < 2; i++) { - driver.process(topic2, expectedKeys[i], null); - } + proc.checkAndClearProcessResult("0:(X0+YY0<-X0+YY0)", "1:(X1+YY1<-X1+YY1)", "2:(X2+YY2<-X2+YY2)", "3:(X3+YY3<-X3+YY3)"); - proc.checkAndClearProcessResult("0:(X0+null<-X0+YY0)", "1:(X1+null<-X1+YY1)"); + // push two items with null to the other stream as deletes. this should produce two item. - // push all four items to the primary stream. this should produce four items. + for (int i = 0; i < 2; i++) { + driver.process(topic2, expectedKeys[i], null); + } - for (int i = 0; i < expectedKeys.length; i++) { - driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); - } + proc.checkAndClearProcessResult("0:(X0+null<-X0+YY0)", "1:(X1+null<-X1+YY1)"); - proc.checkAndClearProcessResult("0:(XX0+null<-X0+null)", "1:(XX1+null<-X1+null)", "2:(XX2+YY2<-X2+YY2)", "3:(XX3+YY3<-X3+YY3)"); + // push all four items to the primary stream. this should produce four items. - // push middle two items to the primary stream with null. this should produce two items. + for (int i = 0; i < expectedKeys.length; i++) { + driver.process(topic1, expectedKeys[i], "XX" + expectedKeys[i]); + } - for (int i = 1; i < 3; i++) { - driver.process(topic1, expectedKeys[i], null); - } + proc.checkAndClearProcessResult("0:(XX0+null<-X0+null)", "1:(XX1+null<-X1+null)", "2:(XX2+YY2<-X2+YY2)", "3:(XX3+YY3<-X3+YY3)"); - proc.checkAndClearProcessResult("1:(null<-XX1+null)", "2:(null+YY2<-XX2+YY2)"); + // push middle two items to the primary stream with null. this should produce two items. - } finally { - Utils.delete(baseDir); + for (int i = 1; i < 3; i++) { + driver.process(topic1, expectedKeys[i], null); } + + proc.checkAndClearProcessResult("1:(null<-XX1+null)", "2:(null+YY2<-XX2+YY2)"); } - private JoinedKeyValue kv(Integer key, String value) { - return new JoinedKeyValue(key, value); + private KeyValue kv(Integer key, String value) { + return new KeyValue<>(key, value); } - private void checkJoinedValues(KTableValueGetter getter, JoinedKeyValue... expected) { - for (JoinedKeyValue kv : expected) { + private void checkJoinedValues(KTableValueGetter getter, KeyValue... expected) { + for (KeyValue kv : expected) { String value = getter.get(kv.key); if (kv.value == null) { assertNull(value); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java index ce1b9d62dd248..cf74017e79d98 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapKeysTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorSupplier; +import org.junit.After; import org.junit.Test; import java.util.HashMap; @@ -38,6 +39,16 @@ public class KTableMapKeysTest { final private Serde stringSerde = new Serdes.StringSerde(); final private Serde integerSerde = new Serdes.IntegerSerde(); + private KStreamTestDriver driver = null; + + @After + public void cleanup() { + if (driver != null) { + driver.close(); + } + driver = null; + } + @Test public void testMapKeysConvertingToStream() { final KStreamBuilder builder = new KStreamBuilder(); @@ -70,7 +81,7 @@ public String apply(Integer key, String value) { convertedStream.process(processor); - KStreamTestDriver driver = new KStreamTestDriver(builder); + driver = new KStreamTestDriver(builder); for (int i = 0; i < originalKeys.length; i++) { driver.process(topic1, originalKeys[i], values[i]); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java index 9cafe8b3bb0c4..efb17fc838855 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableMapValuesTest.java @@ -26,11 +26,13 @@ import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.io.File; import java.io.IOException; -import java.nio.file.Files; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; @@ -39,7 +41,23 @@ public class KTableMapValuesTest { - final private Serde stringSerde = new Serdes.StringSerde(); + final private Serde stringSerde = Serdes.String(); + + private KStreamTestDriver driver = null; + private File stateDir = null; + + @After + public void tearDown() { + if (driver != null) { + driver.close(); + } + driver = null; + } + + @Before + public void setUp() throws IOException { + stateDir = TestUtils.tempDirectory("kafka-test"); + } @Test public void testKTable() { @@ -58,7 +76,7 @@ public Integer apply(String value) { MockProcessorSupplier proc2 = new MockProcessorSupplier<>(); table2.toStream().process(proc2); - KStreamTestDriver driver = new KStreamTestDriver(builder); + driver = new KStreamTestDriver(builder); driver.process(topic1, "A", "01"); driver.process(topic1, "B", "02"); @@ -70,230 +88,211 @@ public Integer apply(String value) { @Test public void testValueGetter() throws IOException { - File stateDir = Files.createTempDirectory("test").toFile(); - try { - final KStreamBuilder builder = new KStreamBuilder(); - - String topic1 = "topic1"; - String topic2 = "topic2"; - - KTableImpl table1 = - (KTableImpl) builder.table(stringSerde, stringSerde, topic1); - KTableImpl table2 = (KTableImpl) table1.mapValues( - new ValueMapper() { - @Override - public Integer apply(String value) { - return new Integer(value); - } - }); - KTableImpl table3 = (KTableImpl) table2.filter( - new Predicate() { - @Override - public boolean test(String key, Integer value) { - return (value % 2) == 0; - } - }); - KTableImpl table4 = (KTableImpl) - table1.through(stringSerde, stringSerde, topic2); - - KTableValueGetterSupplier getterSupplier1 = table1.valueGetterSupplier(); - KTableValueGetterSupplier getterSupplier2 = table2.valueGetterSupplier(); - KTableValueGetterSupplier getterSupplier3 = table3.valueGetterSupplier(); - KTableValueGetterSupplier getterSupplier4 = table4.valueGetterSupplier(); - - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); - - KTableValueGetter getter1 = getterSupplier1.get(); - getter1.init(driver.context()); - KTableValueGetter getter2 = getterSupplier2.get(); - getter2.init(driver.context()); - KTableValueGetter getter3 = getterSupplier3.get(); - getter3.init(driver.context()); - KTableValueGetter getter4 = getterSupplier4.get(); - getter4.init(driver.context()); - - driver.process(topic1, "A", "01"); - driver.process(topic1, "B", "01"); - driver.process(topic1, "C", "01"); - - assertEquals("01", getter1.get("A")); - assertEquals("01", getter1.get("B")); - assertEquals("01", getter1.get("C")); - - assertEquals(new Integer(1), getter2.get("A")); - assertEquals(new Integer(1), getter2.get("B")); - assertEquals(new Integer(1), getter2.get("C")); - - assertNull(getter3.get("A")); - assertNull(getter3.get("B")); - assertNull(getter3.get("C")); - - assertEquals("01", getter4.get("A")); - assertEquals("01", getter4.get("B")); - assertEquals("01", getter4.get("C")); - - driver.process(topic1, "A", "02"); - driver.process(topic1, "B", "02"); - - assertEquals("02", getter1.get("A")); - assertEquals("02", getter1.get("B")); - assertEquals("01", getter1.get("C")); - - assertEquals(new Integer(2), getter2.get("A")); - assertEquals(new Integer(2), getter2.get("B")); - assertEquals(new Integer(1), getter2.get("C")); - - assertEquals(new Integer(2), getter3.get("A")); - assertEquals(new Integer(2), getter3.get("B")); - assertNull(getter3.get("C")); - - assertEquals("02", getter4.get("A")); - assertEquals("02", getter4.get("B")); - assertEquals("01", getter4.get("C")); - - driver.process(topic1, "A", "03"); - - assertEquals("03", getter1.get("A")); - assertEquals("02", getter1.get("B")); - assertEquals("01", getter1.get("C")); - - assertEquals(new Integer(3), getter2.get("A")); - assertEquals(new Integer(2), getter2.get("B")); - assertEquals(new Integer(1), getter2.get("C")); - - assertNull(getter3.get("A")); - assertEquals(new Integer(2), getter3.get("B")); - assertNull(getter3.get("C")); - - assertEquals("03", getter4.get("A")); - assertEquals("02", getter4.get("B")); - assertEquals("01", getter4.get("C")); - - driver.process(topic1, "A", null); - - assertNull(getter1.get("A")); - assertEquals("02", getter1.get("B")); - assertEquals("01", getter1.get("C")); - - assertNull(getter2.get("A")); - assertEquals(new Integer(2), getter2.get("B")); - assertEquals(new Integer(1), getter2.get("C")); - - assertNull(getter3.get("A")); - assertEquals(new Integer(2), getter3.get("B")); - assertNull(getter3.get("C")); - - assertNull(getter4.get("A")); - assertEquals("02", getter4.get("B")); - assertEquals("01", getter4.get("C")); - - } finally { - Utils.delete(stateDir); - } + KStreamBuilder builder = new KStreamBuilder(); + + String topic1 = "topic1"; + String topic2 = "topic2"; + + KTableImpl table1 = + (KTableImpl) builder.table(stringSerde, stringSerde, topic1); + KTableImpl table2 = (KTableImpl) table1.mapValues( + new ValueMapper() { + @Override + public Integer apply(String value) { + return new Integer(value); + } + }); + KTableImpl table3 = (KTableImpl) table2.filter( + new Predicate() { + @Override + public boolean test(String key, Integer value) { + return (value % 2) == 0; + } + }); + KTableImpl table4 = (KTableImpl) + table1.through(stringSerde, stringSerde, topic2); + + KTableValueGetterSupplier getterSupplier1 = table1.valueGetterSupplier(); + KTableValueGetterSupplier getterSupplier2 = table2.valueGetterSupplier(); + KTableValueGetterSupplier getterSupplier3 = table3.valueGetterSupplier(); + KTableValueGetterSupplier getterSupplier4 = table4.valueGetterSupplier(); + + driver = new KStreamTestDriver(builder, stateDir, null, null); + + KTableValueGetter getter1 = getterSupplier1.get(); + getter1.init(driver.context()); + KTableValueGetter getter2 = getterSupplier2.get(); + getter2.init(driver.context()); + KTableValueGetter getter3 = getterSupplier3.get(); + getter3.init(driver.context()); + KTableValueGetter getter4 = getterSupplier4.get(); + getter4.init(driver.context()); + + driver.process(topic1, "A", "01"); + driver.process(topic1, "B", "01"); + driver.process(topic1, "C", "01"); + + assertEquals("01", getter1.get("A")); + assertEquals("01", getter1.get("B")); + assertEquals("01", getter1.get("C")); + + assertEquals(new Integer(1), getter2.get("A")); + assertEquals(new Integer(1), getter2.get("B")); + assertEquals(new Integer(1), getter2.get("C")); + + assertNull(getter3.get("A")); + assertNull(getter3.get("B")); + assertNull(getter3.get("C")); + + assertEquals("01", getter4.get("A")); + assertEquals("01", getter4.get("B")); + assertEquals("01", getter4.get("C")); + + driver.process(topic1, "A", "02"); + driver.process(topic1, "B", "02"); + + assertEquals("02", getter1.get("A")); + assertEquals("02", getter1.get("B")); + assertEquals("01", getter1.get("C")); + + assertEquals(new Integer(2), getter2.get("A")); + assertEquals(new Integer(2), getter2.get("B")); + assertEquals(new Integer(1), getter2.get("C")); + + assertEquals(new Integer(2), getter3.get("A")); + assertEquals(new Integer(2), getter3.get("B")); + assertNull(getter3.get("C")); + + assertEquals("02", getter4.get("A")); + assertEquals("02", getter4.get("B")); + assertEquals("01", getter4.get("C")); + + driver.process(topic1, "A", "03"); + + assertEquals("03", getter1.get("A")); + assertEquals("02", getter1.get("B")); + assertEquals("01", getter1.get("C")); + + assertEquals(new Integer(3), getter2.get("A")); + assertEquals(new Integer(2), getter2.get("B")); + assertEquals(new Integer(1), getter2.get("C")); + + assertNull(getter3.get("A")); + assertEquals(new Integer(2), getter3.get("B")); + assertNull(getter3.get("C")); + + assertEquals("03", getter4.get("A")); + assertEquals("02", getter4.get("B")); + assertEquals("01", getter4.get("C")); + + driver.process(topic1, "A", null); + + assertNull(getter1.get("A")); + assertEquals("02", getter1.get("B")); + assertEquals("01", getter1.get("C")); + + assertNull(getter2.get("A")); + assertEquals(new Integer(2), getter2.get("B")); + assertEquals(new Integer(1), getter2.get("C")); + + assertNull(getter3.get("A")); + assertEquals(new Integer(2), getter3.get("B")); + assertNull(getter3.get("C")); + + assertNull(getter4.get("A")); + assertEquals("02", getter4.get("B")); + assertEquals("01", getter4.get("C")); } @Test public void testNotSendingOldValue() throws IOException { - File stateDir = Files.createTempDirectory("test").toFile(); - try { - final KStreamBuilder builder = new KStreamBuilder(); - - String topic1 = "topic1"; + KStreamBuilder builder = new KStreamBuilder(); - KTableImpl table1 = - (KTableImpl) builder.table(stringSerde, stringSerde, topic1); - KTableImpl table2 = (KTableImpl) table1.mapValues( - new ValueMapper() { - @Override - public Integer apply(String value) { - return new Integer(value); - } - }); + String topic1 = "topic1"; - MockProcessorSupplier proc = new MockProcessorSupplier<>(); + KTableImpl table1 = + (KTableImpl) builder.table(stringSerde, stringSerde, topic1); + KTableImpl table2 = (KTableImpl) table1.mapValues( + new ValueMapper() { + @Override + public Integer apply(String value) { + return new Integer(value); + } + }); - builder.addProcessor("proc", proc, table2.name); + MockProcessorSupplier proc = new MockProcessorSupplier<>(); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); + builder.addProcessor("proc", proc, table2.name); - assertFalse(table1.sendingOldValueEnabled()); - assertFalse(table2.sendingOldValueEnabled()); + driver = new KStreamTestDriver(builder, stateDir, null, null); - driver.process(topic1, "A", "01"); - driver.process(topic1, "B", "01"); - driver.process(topic1, "C", "01"); + assertFalse(table1.sendingOldValueEnabled()); + assertFalse(table2.sendingOldValueEnabled()); - proc.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)"); + driver.process(topic1, "A", "01"); + driver.process(topic1, "B", "01"); + driver.process(topic1, "C", "01"); - driver.process(topic1, "A", "02"); - driver.process(topic1, "B", "02"); + proc.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)"); - proc.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)"); + driver.process(topic1, "A", "02"); + driver.process(topic1, "B", "02"); - driver.process(topic1, "A", "03"); + proc.checkAndClearProcessResult("A:(2<-null)", "B:(2<-null)"); - proc.checkAndClearProcessResult("A:(3<-null)"); + driver.process(topic1, "A", "03"); - driver.process(topic1, "A", null); + proc.checkAndClearProcessResult("A:(3<-null)"); - proc.checkAndClearProcessResult("A:(null<-null)"); + driver.process(topic1, "A", null); - } finally { - Utils.delete(stateDir); - } + proc.checkAndClearProcessResult("A:(null<-null)"); } @Test public void testSendingOldValue() throws IOException { - File stateDir = Files.createTempDirectory("test").toFile(); - try { - final KStreamBuilder builder = new KStreamBuilder(); - - String topic1 = "topic1"; + KStreamBuilder builder = new KStreamBuilder(); - KTableImpl table1 = - (KTableImpl) builder.table(stringSerde, stringSerde, topic1); - KTableImpl table2 = (KTableImpl) table1.mapValues( - new ValueMapper() { - @Override - public Integer apply(String value) { - return new Integer(value); - } - }); + String topic1 = "topic1"; - table2.enableSendingOldValues(); + KTableImpl table1 = + (KTableImpl) builder.table(stringSerde, stringSerde, topic1); + KTableImpl table2 = (KTableImpl) table1.mapValues( + new ValueMapper() { + @Override + public Integer apply(String value) { + return new Integer(value); + } + }); - MockProcessorSupplier proc = new MockProcessorSupplier<>(); + table2.enableSendingOldValues(); - builder.addProcessor("proc", proc, table2.name); + MockProcessorSupplier proc = new MockProcessorSupplier<>(); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); + builder.addProcessor("proc", proc, table2.name); - assertTrue(table1.sendingOldValueEnabled()); - assertTrue(table2.sendingOldValueEnabled()); + driver = new KStreamTestDriver(builder, stateDir, null, null); - driver.process(topic1, "A", "01"); - driver.process(topic1, "B", "01"); - driver.process(topic1, "C", "01"); + assertTrue(table1.sendingOldValueEnabled()); + assertTrue(table2.sendingOldValueEnabled()); - proc.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)"); + driver.process(topic1, "A", "01"); + driver.process(topic1, "B", "01"); + driver.process(topic1, "C", "01"); - driver.process(topic1, "A", "02"); - driver.process(topic1, "B", "02"); + proc.checkAndClearProcessResult("A:(1<-null)", "B:(1<-null)", "C:(1<-null)"); - proc.checkAndClearProcessResult("A:(2<-1)", "B:(2<-1)"); + driver.process(topic1, "A", "02"); + driver.process(topic1, "B", "02"); - driver.process(topic1, "A", "03"); + proc.checkAndClearProcessResult("A:(2<-1)", "B:(2<-1)"); - proc.checkAndClearProcessResult("A:(3<-2)"); + driver.process(topic1, "A", "03"); - driver.process(topic1, "A", null); + proc.checkAndClearProcessResult("A:(3<-2)"); - proc.checkAndClearProcessResult("A:(null<-3)"); + driver.process(topic1, "A", null); - } finally { - Utils.delete(stateDir); - } + proc.checkAndClearProcessResult("A:(null<-3)"); } - } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java index 7c158e2bb6b20..aaa6cc7d188ce 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KTableSourceTest.java @@ -24,11 +24,13 @@ import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.test.KStreamTestDriver; import org.apache.kafka.test.MockProcessorSupplier; +import org.apache.kafka.test.TestUtils; +import org.junit.After; +import org.junit.Before; import org.junit.Test; import java.io.File; import java.io.IOException; -import java.nio.file.Files; import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; @@ -36,7 +38,23 @@ public class KTableSourceTest { - final private Serde stringSerde = new Serdes.StringSerde(); + final private Serde stringSerde = Serdes.String(); + + private KStreamTestDriver driver = null; + private File stateDir = null; + + @After + public void tearDown() { + if (driver != null) { + driver.close(); + } + driver = null; + } + + @Before + public void setUp() throws IOException { + stateDir = TestUtils.tempDirectory("kafka-test"); + } @Test public void testKTable() { @@ -49,7 +67,7 @@ public void testKTable() { MockProcessorSupplier proc1 = new MockProcessorSupplier<>(); table1.toStream().process(proc1); - KStreamTestDriver driver = new KStreamTestDriver(builder); + driver = new KStreamTestDriver(builder); driver.process(topic1, "A", 1); driver.process(topic1, "B", 2); @@ -63,138 +81,120 @@ public void testKTable() { @Test public void testValueGetter() throws IOException { - File stateDir = Files.createTempDirectory("test").toFile(); - try { - final KStreamBuilder builder = new KStreamBuilder(); + final KStreamBuilder builder = new KStreamBuilder(); - String topic1 = "topic1"; + String topic1 = "topic1"; - KTableImpl table1 = (KTableImpl) builder.table(stringSerde, stringSerde, topic1); + KTableImpl table1 = (KTableImpl) builder.table(stringSerde, stringSerde, topic1); - KTableValueGetterSupplier getterSupplier1 = table1.valueGetterSupplier(); + KTableValueGetterSupplier getterSupplier1 = table1.valueGetterSupplier(); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); + driver = new KStreamTestDriver(builder, stateDir, null, null); - KTableValueGetter getter1 = getterSupplier1.get(); - getter1.init(driver.context()); + KTableValueGetter getter1 = getterSupplier1.get(); + getter1.init(driver.context()); - driver.process(topic1, "A", "01"); - driver.process(topic1, "B", "01"); - driver.process(topic1, "C", "01"); + driver.process(topic1, "A", "01"); + driver.process(topic1, "B", "01"); + driver.process(topic1, "C", "01"); - assertEquals("01", getter1.get("A")); - assertEquals("01", getter1.get("B")); - assertEquals("01", getter1.get("C")); + assertEquals("01", getter1.get("A")); + assertEquals("01", getter1.get("B")); + assertEquals("01", getter1.get("C")); - driver.process(topic1, "A", "02"); - driver.process(topic1, "B", "02"); + driver.process(topic1, "A", "02"); + driver.process(topic1, "B", "02"); - assertEquals("02", getter1.get("A")); - assertEquals("02", getter1.get("B")); - assertEquals("01", getter1.get("C")); + assertEquals("02", getter1.get("A")); + assertEquals("02", getter1.get("B")); + assertEquals("01", getter1.get("C")); - driver.process(topic1, "A", "03"); + driver.process(topic1, "A", "03"); - assertEquals("03", getter1.get("A")); - assertEquals("02", getter1.get("B")); - assertEquals("01", getter1.get("C")); + assertEquals("03", getter1.get("A")); + assertEquals("02", getter1.get("B")); + assertEquals("01", getter1.get("C")); - driver.process(topic1, "A", null); - driver.process(topic1, "B", null); + driver.process(topic1, "A", null); + driver.process(topic1, "B", null); - assertNull(getter1.get("A")); - assertNull(getter1.get("B")); - assertEquals("01", getter1.get("C")); + assertNull(getter1.get("A")); + assertNull(getter1.get("B")); + assertEquals("01", getter1.get("C")); - } finally { - Utils.delete(stateDir); - } } @Test public void testNotSedingOldValue() throws IOException { - File stateDir = Files.createTempDirectory("test").toFile(); - try { - final KStreamBuilder builder = new KStreamBuilder(); - - String topic1 = "topic1"; + final KStreamBuilder builder = new KStreamBuilder(); - KTableImpl table1 = (KTableImpl) builder.table(stringSerde, stringSerde, topic1); + String topic1 = "topic1"; - MockProcessorSupplier proc1 = new MockProcessorSupplier<>(); + KTableImpl table1 = (KTableImpl) builder.table(stringSerde, stringSerde, topic1); - builder.addProcessor("proc1", proc1, table1.name); + MockProcessorSupplier proc1 = new MockProcessorSupplier<>(); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); + builder.addProcessor("proc1", proc1, table1.name); - driver.process(topic1, "A", "01"); - driver.process(topic1, "B", "01"); - driver.process(topic1, "C", "01"); + driver = new KStreamTestDriver(builder, stateDir, null, null); - proc1.checkAndClearProcessResult("A:(01<-null)", "B:(01<-null)", "C:(01<-null)"); + driver.process(topic1, "A", "01"); + driver.process(topic1, "B", "01"); + driver.process(topic1, "C", "01"); - driver.process(topic1, "A", "02"); - driver.process(topic1, "B", "02"); + proc1.checkAndClearProcessResult("A:(01<-null)", "B:(01<-null)", "C:(01<-null)"); - proc1.checkAndClearProcessResult("A:(02<-null)", "B:(02<-null)"); + driver.process(topic1, "A", "02"); + driver.process(topic1, "B", "02"); - driver.process(topic1, "A", "03"); + proc1.checkAndClearProcessResult("A:(02<-null)", "B:(02<-null)"); - proc1.checkAndClearProcessResult("A:(03<-null)"); + driver.process(topic1, "A", "03"); - driver.process(topic1, "A", null); - driver.process(topic1, "B", null); + proc1.checkAndClearProcessResult("A:(03<-null)"); - proc1.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)"); + driver.process(topic1, "A", null); + driver.process(topic1, "B", null); - } finally { - Utils.delete(stateDir); - } + proc1.checkAndClearProcessResult("A:(null<-null)", "B:(null<-null)"); } @Test public void testSedingOldValue() throws IOException { - File stateDir = Files.createTempDirectory("test").toFile(); - try { - final KStreamBuilder builder = new KStreamBuilder(); - - String topic1 = "topic1"; + final KStreamBuilder builder = new KStreamBuilder(); - KTableImpl table1 = (KTableImpl) builder.table(stringSerde, stringSerde, topic1); + String topic1 = "topic1"; - table1.enableSendingOldValues(); + KTableImpl table1 = (KTableImpl) builder.table(stringSerde, stringSerde, topic1); - assertTrue(table1.sendingOldValueEnabled()); + table1.enableSendingOldValues(); - MockProcessorSupplier proc1 = new MockProcessorSupplier<>(); + assertTrue(table1.sendingOldValueEnabled()); - builder.addProcessor("proc1", proc1, table1.name); + MockProcessorSupplier proc1 = new MockProcessorSupplier<>(); - KStreamTestDriver driver = new KStreamTestDriver(builder, stateDir, null, null); + builder.addProcessor("proc1", proc1, table1.name); - driver.process(topic1, "A", "01"); - driver.process(topic1, "B", "01"); - driver.process(topic1, "C", "01"); + driver = new KStreamTestDriver(builder, stateDir, null, null); - proc1.checkAndClearProcessResult("A:(01<-null)", "B:(01<-null)", "C:(01<-null)"); + driver.process(topic1, "A", "01"); + driver.process(topic1, "B", "01"); + driver.process(topic1, "C", "01"); - driver.process(topic1, "A", "02"); - driver.process(topic1, "B", "02"); + proc1.checkAndClearProcessResult("A:(01<-null)", "B:(01<-null)", "C:(01<-null)"); - proc1.checkAndClearProcessResult("A:(02<-01)", "B:(02<-01)"); + driver.process(topic1, "A", "02"); + driver.process(topic1, "B", "02"); - driver.process(topic1, "A", "03"); + proc1.checkAndClearProcessResult("A:(02<-01)", "B:(02<-01)"); - proc1.checkAndClearProcessResult("A:(03<-02)"); + driver.process(topic1, "A", "03"); - driver.process(topic1, "A", null); - driver.process(topic1, "B", null); + proc1.checkAndClearProcessResult("A:(03<-02)"); - proc1.checkAndClearProcessResult("A:(null<-03)", "B:(null<-02)"); + driver.process(topic1, "A", null); + driver.process(topic1, "B", null); - } finally { - Utils.delete(stateDir); - } + proc1.checkAndClearProcessResult("A:(null<-03)", "B:(null<-02)"); } - } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinterProcessorTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinterProcessorTest.java index 22948abcc6f41..c8707af64a764 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinterProcessorTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KeyValuePrinterProcessorTest.java @@ -26,6 +26,7 @@ import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.test.KStreamTestDriver; +import org.junit.After; import org.junit.Test; import java.io.ByteArrayOutputStream; @@ -39,11 +40,19 @@ public class KeyValuePrinterProcessorTest { private String topicName = "topic"; private Serde stringSerde = Serdes.String(); - private Serde bytesSerde = Serdes.ByteArray(); private ByteArrayOutputStream baos = new ByteArrayOutputStream(); private KStreamBuilder builder = new KStreamBuilder(); private PrintStream printStream = new PrintStream(baos); + private KStreamTestDriver driver = null; + + @After + public void cleanup() { + if (driver != null) { + driver.close(); + } + driver = null; + } @Test public void testPrintKeyValueDefaultSerde() throws Exception { @@ -57,7 +66,7 @@ public void testPrintKeyValueDefaultSerde() throws Exception { KStream stream = builder.stream(stringSerde, stringSerde, topicName); stream.process(keyValuePrinter); - KStreamTestDriver driver = new KStreamTestDriver(builder); + driver = new KStreamTestDriver(builder); for (int i = 0; i < suppliedKeys.length; i++) { driver.process(topicName, suppliedKeys[i], suppliedValues[i]); } @@ -79,7 +88,7 @@ public void testPrintKeyValueWithProvidedSerde() throws Exception { stream.process(keyValuePrinter); - KStreamTestDriver driver = new KStreamTestDriver(builder); + driver = new KStreamTestDriver(builder); String suppliedKey = null; byte[] suppliedValue = "{\"name\":\"print\", \"label\":\"test\"}".getBytes(Charset.forName("UTF-8")); diff --git a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java index d7387948055e4..73168048f5e17 100644 --- a/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java +++ b/streams/src/test/java/org/apache/kafka/test/KStreamTestDriver.java @@ -27,6 +27,7 @@ import org.apache.kafka.streams.processor.StateStoreSupplier; import org.apache.kafka.streams.processor.StreamPartitioner; import org.apache.kafka.streams.processor.internals.ProcessorNode; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; import org.apache.kafka.streams.processor.internals.ProcessorTopology; import org.apache.kafka.streams.processor.internals.RecordCollector; @@ -82,6 +83,12 @@ public ProcessorContext context() { public void process(String topicName, Object key, Object value) { currNode = topology.source(topicName); + + // if currNode is null, check if this topic is a changelog topic; + // if yes, skip + if (topicName.endsWith(ProcessorStateManager.STATE_CHANGELOG_TOPIC_SUFFIX)) + return; + try { forward(key, value); } finally { @@ -108,10 +115,6 @@ public void setTime(long timestamp) { context.setTime(timestamp); } - public StateStore getStateStore(String name) { - return context.getStateStore(name); - } - @SuppressWarnings("unchecked") public void forward(K key, V value) { ProcessorNode thisNode = currNode; @@ -153,6 +156,23 @@ public void forward(K key, V value, String childName) { } } + public void close() { + // close all processors + for (ProcessorNode node : topology.processors()) { + currNode = node; + try { + node.close(); + } finally { + currNode = null; + } + } + + // close all state stores + for (StateStore store : context.allStateStores().values()) { + store.close(); + } + } + public Set allProcessorNames() { Set names = new HashSet<>(); diff --git a/streams/src/test/java/org/apache/kafka/test/MockKeyValueMapper.java b/streams/src/test/java/org/apache/kafka/test/MockKeyValueMapper.java index ae8c2fd80fec4..769ee71f2eda8 100644 --- a/streams/src/test/java/org/apache/kafka/test/MockKeyValueMapper.java +++ b/streams/src/test/java/org/apache/kafka/test/MockKeyValueMapper.java @@ -26,7 +26,7 @@ private static class NoOpKeyValueMapper implements KeyValueMapper apply(K key, V value) { - return new KeyValue<>(key, value); + return KeyValue.pair(key, value); } } diff --git a/streams/src/test/java/org/apache/kafka/test/MockValueJoiner.java b/streams/src/test/java/org/apache/kafka/test/MockValueJoiner.java new file mode 100644 index 0000000000000..4d441665e84cf --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/test/MockValueJoiner.java @@ -0,0 +1,33 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kafka.test; + +import org.apache.kafka.streams.kstream.ValueJoiner; + +public class MockValueJoiner { + + private static class StringJoin implements ValueJoiner { + + @Override + public String apply(String value1, String value2) { + return value1 + "+" + value2; + } + }; + + public final static ValueJoiner STRING_JOINER = new StringJoin(); +} \ No newline at end of file From fbac525c08d54a26248eb8f9a5f7d3d053f99bd4 Mon Sep 17 00:00:00 2001 From: Stig Date: Tue, 26 Apr 2016 11:46:03 -0700 Subject: [PATCH 175/206] MINOR: Fix zk path in KafkaHealthCheck comment Author: Stig Reviewers: Ismael Juma Closes #1182 from srdo/comment-fix --- core/src/main/scala/kafka/server/KafkaHealthcheck.scala | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala index 2598e6dbee604..4e3fc29f9dc41 100644 --- a/core/src/main/scala/kafka/server/KafkaHealthcheck.scala +++ b/core/src/main/scala/kafka/server/KafkaHealthcheck.scala @@ -30,7 +30,7 @@ import org.apache.zookeeper.Watcher.Event.KeeperState /** * This class registers the broker in zookeeper to allow * other brokers and consumers to detect failures. It uses an ephemeral znode with the path: - * /brokers/[0...N] --> advertisedHost:advertisedPort + * /brokers/ids/[0...N] --> advertisedHost:advertisedPort * * Right now our definition of health is fairly naive. If we register in zk we are healthy, otherwise * we are dead. From 2790d0cb1fc6ade216f2c1bf2667a0ebb53d85e3 Mon Sep 17 00:00:00 2001 From: Rafael Winterhalter Date: Tue, 26 Apr 2016 11:52:33 -0700 Subject: [PATCH 176/206] MINOR: Changed port of bootstrap server to default Kafka is typically running on port 9092. The example named a different port what makes it difficult to run a bootstrap example without any further configuration. Author: Rafael Winterhalter Reviewers: Ismael Juma Closes #1074 from raphw/trunk --- .../java/org/apache/kafka/clients/producer/KafkaProducer.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java index 2a60f612a87f7..5b05272ab2213 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/KafkaProducer.java @@ -73,7 +73,7 @@ *
            * {@code
            * Properties props = new Properties();
          - * props.put("bootstrap.servers", "localhost:4242");
          + * props.put("bootstrap.servers", "localhost:9092");
            * props.put("acks", "all");
            * props.put("retries", 0);
            * props.put("batch.size", 16384);
          
          From cd427c9b905c161a6b135504dc515515de6a6438 Mon Sep 17 00:00:00 2001
          From: Ismael Juma 
          Date: Tue, 26 Apr 2016 15:14:33 -0700
          Subject: [PATCH 177/206] MINOR: Upgrade to Gradle 2.13
          
          There are a few improvements in 2.12 and 2.13. I am particularly interested in the performance improvements:
          * 2.12: "This release brings support for compile only dependencies, improved build script compilation speed and even better IDE support."
          * 2.13: "We've achieved performance improvements during Gradle's configuration and execution phase, where we have measured up to 25% improvements to build time in our performance tests. No changes to your build script are necessary to start taking advantage of these improvements."
          
          Author: Ismael Juma 
          
          Reviewers: Gwen Shapira
          
          Closes #1271 from ijuma/gradle-2.13
          ---
           build.gradle |  2 +-
           gradlew      | 46 +++++++++++++++++++++++++---------------------
           gradlew.bat  |  6 +++---
           3 files changed, 29 insertions(+), 25 deletions(-)
          
          diff --git a/build.gradle b/build.gradle
          index da0152b990f97..dfa7c40835b7a 100644
          --- a/build.gradle
          +++ b/build.gradle
          @@ -59,7 +59,7 @@ allprojects {
           }
           
           ext {
          -  gradleVersion = "2.11"
          +  gradleVersion = "2.13"
             buildVersionFileName = "kafka-version.properties"
           
             userMaxForks = project.hasProperty('maxParallelForks') ? maxParallelForks.toInteger() : null
          diff --git a/gradlew b/gradlew
          index 9d82f78915133..27309d92314c5 100755
          --- a/gradlew
          +++ b/gradlew
          @@ -6,12 +6,30 @@
           ##
           ##############################################################################
           
          -# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script.
          -DEFAULT_JVM_OPTS=""
          +# Attempt to set APP_HOME
          +# Resolve links: $0 may be a link
          +PRG="$0"
          +# Need this for relative symlinks.
          +while [ -h "$PRG" ] ; do
          +    ls=`ls -ld "$PRG"`
          +    link=`expr "$ls" : '.*-> \(.*\)$'`
          +    if expr "$link" : '/.*' > /dev/null; then
          +        PRG="$link"
          +    else
          +        PRG=`dirname "$PRG"`"/$link"
          +    fi
          +done
          +SAVED="`pwd`"
          +cd "`dirname \"$PRG\"`/" >/dev/null
          +APP_HOME="`pwd -P`"
          +cd "$SAVED" >/dev/null
           
           APP_NAME="Gradle"
           APP_BASE_NAME=`basename "$0"`
           
          +# Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script.
          +DEFAULT_JVM_OPTS=""
          +
           # Use the maximum available, or set MAX_FD != -1 to use that value.
           MAX_FD="maximum"
           
          @@ -30,6 +48,7 @@ die ( ) {
           cygwin=false
           msys=false
           darwin=false
          +nonstop=false
           case "`uname`" in
             CYGWIN* )
               cygwin=true
          @@ -40,26 +59,11 @@ case "`uname`" in
             MINGW* )
               msys=true
               ;;
          +  NONSTOP* )
          +    nonstop=true
          +    ;;
           esac
           
          -# Attempt to set APP_HOME
          -# Resolve links: $0 may be a link
          -PRG="$0"
          -# Need this for relative symlinks.
          -while [ -h "$PRG" ] ; do
          -    ls=`ls -ld "$PRG"`
          -    link=`expr "$ls" : '.*-> \(.*\)$'`
          -    if expr "$link" : '/.*' > /dev/null; then
          -        PRG="$link"
          -    else
          -        PRG=`dirname "$PRG"`"/$link"
          -    fi
          -done
          -SAVED="`pwd`"
          -cd "`dirname \"$PRG\"`/" >/dev/null
          -APP_HOME="`pwd -P`"
          -cd "$SAVED" >/dev/null
          -
           CLASSPATH=$APP_HOME/gradle/wrapper/gradle-wrapper.jar
           
           # Determine the Java command to use to start the JVM.
          @@ -85,7 +89,7 @@ location of your Java installation."
           fi
           
           # Increase the maximum file descriptors if we can.
          -if [ "$cygwin" = "false" -a "$darwin" = "false" ] ; then
          +if [ "$cygwin" = "false" -a "$darwin" = "false" -a "$nonstop" = "false" ] ; then
               MAX_FD_LIMIT=`ulimit -H -n`
               if [ $? -eq 0 ] ; then
                   if [ "$MAX_FD" = "maximum" -o "$MAX_FD" = "max" ] ; then
          diff --git a/gradlew.bat b/gradlew.bat
          index 72d362dafd896..f6d5974e72fda 100644
          --- a/gradlew.bat
          +++ b/gradlew.bat
          @@ -8,14 +8,14 @@
           @rem Set local scope for the variables with windows NT shell
           if "%OS%"=="Windows_NT" setlocal
           
          -@rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script.
          -set DEFAULT_JVM_OPTS=
          -
           set DIRNAME=%~dp0
           if "%DIRNAME%" == "" set DIRNAME=.
           set APP_BASE_NAME=%~n0
           set APP_HOME=%DIRNAME%
           
          +@rem Add default JVM options here. You can also use JAVA_OPTS and GRADLE_OPTS to pass JVM options to this script.
          +set DEFAULT_JVM_OPTS=
          +
           @rem Find java.exe
           if defined JAVA_HOME goto findJavaFromJavaHome
           
          
          From 5b375d7bf9b26aaeed06bac2dc5de3f8214cbad4 Mon Sep 17 00:00:00 2001
          From: Rajini Sivaram 
          Date: Tue, 26 Apr 2016 16:56:42 -0700
          Subject: [PATCH 178/206] KAFKA-3149; Extend SASL implementation to support
           more mechanisms
          
          Code changes corresponding to KIP-43 to enable review of the KIP.
          
          Author: Rajini Sivaram 
          
          Reviewers: Jun Rao , Ismael Juma 
          
          Closes #812 from rajinisivaram/KAFKA-3149
          ---
           checkstyle/import-control.xml                 |   7 +
           .../org/apache/kafka/clients/ClientUtils.java |   4 +-
           .../apache/kafka/clients/NetworkClient.java   |  20 +-
           .../kafka/common/config/SaslConfigs.java      |  15 +-
           .../errors/AuthenticationException.java       |  27 ++
           .../errors/IllegalSaslStateException.java     |  27 ++
           .../UnsupportedSaslMechanismException.java    |  27 ++
           .../kafka/common/network/ChannelBuilders.java |  14 +-
           .../common/network/SaslChannelBuilder.java    |  36 ++-
           .../apache/kafka/common/protocol/ApiKeys.java |   3 +-
           .../apache/kafka/common/protocol/Errors.java  |   8 +-
           .../kafka/common/protocol/Protocol.java       |  13 +
           .../common/requests/AbstractRequest.java      |   2 +
           .../kafka/common/requests/ResponseSend.java   |   2 +-
           .../common/requests/SaslHandshakeRequest.java |  83 ++++++
           .../requests/SaslHandshakeResponse.java       |  85 ++++++
           .../security/auth/AuthCallbackHandler.java    |  46 ++++
           .../kafka/common/security/auth/Login.java     |  57 +++++
           .../security/authenticator/AbstractLogin.java | 108 ++++++++
           .../security/authenticator/DefaultLogin.java  |  32 +++
           .../LoginManager.java                         |  44 +---
           .../SaslClientAuthenticator.java              | 242 ++++++++++++------
           .../SaslClientCallbackHandler.java            |  94 +++++++
           .../SaslServerAuthenticator.java              | 195 ++++++++++++--
           .../SaslServerCallbackHandler.java            |  22 +-
           .../{Login.java => KerberosLogin.java}        | 107 ++++----
           .../security/plain/PlainLoginModule.java      |  66 +++++
           .../security/plain/PlainSaslServer.java       | 170 ++++++++++++
           .../plain/PlainSaslServerProvider.java        |  38 +++
           .../common/requests/RequestResponseTest.java  |  11 +-
           .../org/apache/kafka/test/TestSslUtils.java   |   3 +
           .../scala/kafka/common/ErrorMapping.scala     |   2 +
           .../controller/ControllerChannelManager.scala |  10 +-
           .../scala/kafka/network/SocketServer.scala    |   2 +-
           .../main/scala/kafka/server/KafkaApis.scala   |   8 +
           .../main/scala/kafka/server/KafkaConfig.scala |  18 +-
           .../main/scala/kafka/server/KafkaServer.scala |   9 +-
           .../kafka/server/ReplicaFetcherThread.scala   |  10 +-
           .../kafka/api/BaseConsumerTest.scala          |   9 +-
           .../kafka/api/BaseProducerSendTest.scala      |   4 +-
           .../kafka/api/EndToEndAuthorizationTest.scala |   4 +-
           .../kafka/api/IntegrationTestHarness.scala    |   8 +-
           .../kafka/api/PlaintextConsumerTest.scala     |   2 +-
           .../api/SaslMultiMechanismConsumerTest.scala  |  86 +++++++
           .../api/SaslPlainPlaintextConsumerTest.scala  |  27 ++
           .../integration/kafka/api/SaslSetup.scala     |  36 +--
           .../kafka/api/SaslTestHarness.scala           |  22 +-
           .../integration/KafkaServerTestHarness.scala  |   4 +-
           .../unit/kafka/server/KafkaConfigTest.scala   |   2 +
           .../unit/kafka/utils/JaasTestUtils.scala      | 122 ++++++---
           .../scala/unit/kafka/utils/TestUtils.scala    |  40 ++-
           51 files changed, 1740 insertions(+), 293 deletions(-)
           create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/AuthenticationException.java
           create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/IllegalSaslStateException.java
           create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/UnsupportedSaslMechanismException.java
           create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java
           create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java
           create mode 100644 clients/src/main/java/org/apache/kafka/common/security/auth/AuthCallbackHandler.java
           create mode 100644 clients/src/main/java/org/apache/kafka/common/security/auth/Login.java
           create mode 100644 clients/src/main/java/org/apache/kafka/common/security/authenticator/AbstractLogin.java
           create mode 100644 clients/src/main/java/org/apache/kafka/common/security/authenticator/DefaultLogin.java
           rename clients/src/main/java/org/apache/kafka/common/security/{kerberos => authenticator}/LoginManager.java (69%)
           create mode 100644 clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java
           rename clients/src/main/java/org/apache/kafka/common/security/kerberos/{Login.java => KerberosLogin.java} (86%)
           create mode 100644 clients/src/main/java/org/apache/kafka/common/security/plain/PlainLoginModule.java
           create mode 100644 clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServer.java
           create mode 100644 clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServerProvider.java
           create mode 100644 core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala
           create mode 100644 core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala
          
          diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml
          index 051c8d13ef67c..e94698c385f0b 100644
          --- a/checkstyle/import-control.xml
          +++ b/checkstyle/import-control.xml
          @@ -69,6 +69,13 @@
                 
                 
                 
          +      
          +        
          +        
          +        
          +        
          +        
          +      
               
           
               
          diff --git a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
          index 0201257f41dc3..ad9c5d0a29364 100644
          --- a/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
          +++ b/clients/src/main/java/org/apache/kafka/clients/ClientUtils.java
          @@ -25,6 +25,7 @@
           import org.apache.kafka.common.protocol.SecurityProtocol;
           import org.apache.kafka.common.network.ChannelBuilder;
           import org.apache.kafka.common.config.ConfigException;
          +import org.apache.kafka.common.config.SaslConfigs;
           import org.slf4j.Logger;
           import org.slf4j.LoggerFactory;
           
          @@ -76,7 +77,8 @@ public static ChannelBuilder createChannelBuilder(Map configs) {
                   SecurityProtocol securityProtocol = SecurityProtocol.forName((String) configs.get(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG));
                   if (!SecurityProtocol.nonTestingValues().contains(securityProtocol))
                       throw new ConfigException("Invalid SecurityProtocol " + securityProtocol);
          -        return ChannelBuilders.create(securityProtocol, Mode.CLIENT, LoginType.CLIENT, configs);
          +        String clientSaslMechanism = (String) configs.get(SaslConfigs.SASL_MECHANISM);
          +        return ChannelBuilders.create(securityProtocol, Mode.CLIENT, LoginType.CLIENT, configs, clientSaslMechanism, true);
               }
           
           }
          diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
          index d2eaace98b833..cc5dc6f1f8df4 100644
          --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
          +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java
          @@ -33,6 +33,7 @@
           
           import java.io.IOException;
           import java.net.InetSocketAddress;
          +import java.nio.ByteBuffer;
           import java.util.ArrayList;
           import java.util.HashSet;
           import java.util.List;
          @@ -373,6 +374,16 @@ public Node leastLoadedNode(long now) {
                   return found;
               }
           
          +    public static Struct parseResponse(ByteBuffer responseBuffer, RequestHeader requestHeader) {
          +        ResponseHeader responseHeader = ResponseHeader.parse(responseBuffer);
          +        // Always expect the response version id to be the same as the request version id
          +        short apiKey = requestHeader.apiKey();
          +        short apiVer = requestHeader.apiVersion();
          +        Struct responseBody = ProtoUtils.responseSchema(apiKey, apiVer).read(responseBuffer);
          +        correlate(requestHeader, responseHeader);
          +        return responseBody;
          +    }
          +
               /**
                * Post process disconnection of a node
                *
          @@ -437,12 +448,7 @@ private void handleCompletedReceives(List responses, long now) {
                   for (NetworkReceive receive : this.selector.completedReceives()) {
                       String source = receive.source();
                       ClientRequest req = inFlightRequests.completeNext(source);
          -            ResponseHeader header = ResponseHeader.parse(receive.payload());
          -            // Always expect the response version id to be the same as the request version id
          -            short apiKey = req.request().header().apiKey();
          -            short apiVer = req.request().header().apiVersion();
          -            Struct body = ProtoUtils.responseSchema(apiKey, apiVer).read(receive.payload());
          -            correlate(req.request().header(), header);
          +            Struct body = parseResponse(receive.payload(), req.request().header());
                       if (!metadataUpdater.maybeHandleCompletedReceive(req, now, body))
                           responses.add(new ClientResponse(req, now, false, body));
                   }
          @@ -477,7 +483,7 @@ private void handleConnections() {
               /**
                * Validate that the response corresponds to the request we expect or else explode
                */
          -    private void correlate(RequestHeader requestHeader, ResponseHeader responseHeader) {
          +    private static 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() + ")");
          diff --git a/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java
          index d61838f30b824..d3aa0d63cd882 100644
          --- a/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java
          +++ b/clients/src/main/java/org/apache/kafka/common/config/SaslConfigs.java
          @@ -20,6 +20,17 @@ public class SaslConfigs {
               /*
                * NOTE: DO NOT CHANGE EITHER CONFIG NAMES AS THESE ARE PART OF THE PUBLIC API AND CHANGE WILL BREAK USER CODE.
                */
          +    /** SASL mechanism configuration - standard mechanism names are listed here. */
          +    public static final String SASL_MECHANISM = "sasl.mechanism";
          +    public static final String SASL_MECHANISM_DOC = "SASL mechanism used for client connections. This may be any mechanism for which a security provider is available. GSSAPI is the default mechanism.";
          +    public static final String GSSAPI_MECHANISM = "GSSAPI";
          +    public static final String DEFAULT_SASL_MECHANISM = GSSAPI_MECHANISM;
          +
          +    public static final String SASL_ENABLED_MECHANISMS = "sasl.enabled.mechanisms";
          +    public static final String SASL_ENABLED_MECHANISMS_DOC = "The list of SASL mechanisms enabled in the Kafka server. "
          +        + "The list may contain any mechanism for which a security provider is available. "
          +        + "Only GSSAPI is enabled by default.";
          +    public static final List DEFAULT_SASL_ENABLED_MECHANISMS = Collections.singletonList(GSSAPI_MECHANISM);
           
               public static final String SASL_KERBEROS_SERVICE_NAME = "sasl.kerberos.service.name";
               public static final String SASL_KERBEROS_SERVICE_NAME_DOC = "The Kerberos principal name that Kafka runs as. "
          @@ -54,7 +65,7 @@ public static void addClientSaslSupport(ConfigDef config) {
                           .define(SaslConfigs.SASL_KERBEROS_KINIT_CMD, ConfigDef.Type.STRING, SaslConfigs.DEFAULT_KERBEROS_KINIT_CMD, ConfigDef.Importance.LOW, SaslConfigs.SASL_KERBEROS_KINIT_CMD_DOC)
                           .define(SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR, ConfigDef.Type.DOUBLE, SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_WINDOW_FACTOR, ConfigDef.Importance.LOW, SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_DOC)
                           .define(SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER, ConfigDef.Type.DOUBLE, SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_JITTER, ConfigDef.Importance.LOW, SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER_DOC)
          -                .define(SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN, ConfigDef.Type.LONG, SaslConfigs.DEFAULT_KERBEROS_MIN_TIME_BEFORE_RELOGIN, ConfigDef.Importance.LOW, SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_DOC);
          +                .define(SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN, ConfigDef.Type.LONG, SaslConfigs.DEFAULT_KERBEROS_MIN_TIME_BEFORE_RELOGIN, ConfigDef.Importance.LOW, SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN_DOC)
          +                .define(SaslConfigs.SASL_MECHANISM, ConfigDef.Type.STRING, SaslConfigs.DEFAULT_SASL_MECHANISM, ConfigDef.Importance.MEDIUM, SaslConfigs.SASL_MECHANISM_DOC);
               }
          -
           }
          diff --git a/clients/src/main/java/org/apache/kafka/common/errors/AuthenticationException.java b/clients/src/main/java/org/apache/kafka/common/errors/AuthenticationException.java
          new file mode 100644
          index 0000000000000..7b60e11cd0d4d
          --- /dev/null
          +++ b/clients/src/main/java/org/apache/kafka/common/errors/AuthenticationException.java
          @@ -0,0 +1,27 @@
          +/**
          + * 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 org.apache.kafka.common.errors;
          +
          +public class AuthenticationException extends ApiException {
          +
          +    private static final long serialVersionUID = 1L;
          +
          +    public AuthenticationException(String message) {
          +        super(message);
          +    }
          +
          +    public AuthenticationException(String message, Throwable cause) {
          +        super(message, cause);
          +    }
          +
          +}
          diff --git a/clients/src/main/java/org/apache/kafka/common/errors/IllegalSaslStateException.java b/clients/src/main/java/org/apache/kafka/common/errors/IllegalSaslStateException.java
          new file mode 100644
          index 0000000000000..7fd008c1bf7dd
          --- /dev/null
          +++ b/clients/src/main/java/org/apache/kafka/common/errors/IllegalSaslStateException.java
          @@ -0,0 +1,27 @@
          +/**
          + * 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 org.apache.kafka.common.errors;
          +
          +public class IllegalSaslStateException extends AuthenticationException {
          +
          +    private static final long serialVersionUID = 1L;
          +
          +    public IllegalSaslStateException(String message) {
          +        super(message);
          +    }
          +
          +    public IllegalSaslStateException(String message, Throwable cause) {
          +        super(message, cause);
          +    }
          +
          +}
          diff --git a/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedSaslMechanismException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedSaslMechanismException.java
          new file mode 100644
          index 0000000000000..289a09f78dae7
          --- /dev/null
          +++ b/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedSaslMechanismException.java
          @@ -0,0 +1,27 @@
          +/**
          + * 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 org.apache.kafka.common.errors;
          +
          +public class UnsupportedSaslMechanismException extends AuthenticationException {
          +
          +    private static final long serialVersionUID = 1L;
          +
          +    public UnsupportedSaslMechanismException(String message) {
          +        super(message);
          +    }
          +
          +    public UnsupportedSaslMechanismException(String message, Throwable cause) {
          +        super(message, cause);
          +    }
          +
          +}
          diff --git a/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java b/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java
          index 669f269f91c2a..2d6ba8a98667e 100644
          --- a/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java
          +++ b/clients/src/main/java/org/apache/kafka/common/network/ChannelBuilders.java
          @@ -31,10 +31,18 @@ private ChannelBuilders() { }
                *             it is ignored otherwise
                * @param loginType the loginType, it must be non-null if `securityProtocol` is SASL_*; it is ignored otherwise
                * @param configs client/server configs
          +     * @param clientSaslMechanism SASL mechanism if mode is CLIENT, ignored otherwise
          +     * @param saslHandshakeRequestEnable flag to enable Sasl handshake requests; disabled only for SASL
          +     *             inter-broker connections with inter-broker protocol version < 0.10
                * @return the configured `ChannelBuilder`
                * @throws IllegalArgumentException if `mode` invariants described above is not maintained
                */
          -    public static ChannelBuilder create(SecurityProtocol securityProtocol, Mode mode, LoginType loginType, Map configs) {
          +    public static ChannelBuilder create(SecurityProtocol securityProtocol,
          +                                        Mode mode,
          +                                        LoginType loginType,
          +                                        Map configs,
          +                                        String clientSaslMechanism,
          +                                        boolean saslHandshakeRequestEnable) {
                   ChannelBuilder channelBuilder;
           
                   switch (securityProtocol) {
          @@ -47,7 +55,9 @@ public static ChannelBuilder create(SecurityProtocol securityProtocol, Mode mode
                           requireNonNullMode(mode, securityProtocol);
                           if (loginType == null)
                               throw new IllegalArgumentException("`loginType` must be non-null if `securityProtocol` is `" + securityProtocol + "`");
          -                channelBuilder = new SaslChannelBuilder(mode, loginType, securityProtocol);
          +                if (mode == Mode.CLIENT && clientSaslMechanism == null)
          +                    throw new IllegalArgumentException("`clientSaslMechanism` must be non-null in client mode if `securityProtocol` is `" + securityProtocol + "`");
          +                channelBuilder = new SaslChannelBuilder(mode, loginType, securityProtocol, clientSaslMechanism, saslHandshakeRequestEnable);
                           break;
                       case PLAINTEXT:
                       case TRACE:
          diff --git a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java
          index 0cd5bfe9005de..a0464bc611eca 100644
          --- a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java
          +++ b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java
          @@ -21,13 +21,12 @@
           import org.apache.kafka.common.config.SaslConfigs;
           import org.apache.kafka.common.security.JaasUtils;
           import org.apache.kafka.common.security.kerberos.KerberosShortNamer;
          -import org.apache.kafka.common.security.kerberos.LoginManager;
          +import org.apache.kafka.common.security.authenticator.LoginManager;
           import org.apache.kafka.common.security.authenticator.SaslClientAuthenticator;
           import org.apache.kafka.common.security.authenticator.SaslServerAuthenticator;
           import org.apache.kafka.common.security.ssl.SslFactory;
           import org.apache.kafka.common.protocol.SecurityProtocol;
           import org.apache.kafka.common.KafkaException;
          -
           import org.slf4j.Logger;
           import org.slf4j.LoggerFactory;
           
          @@ -35,24 +34,34 @@ public class SaslChannelBuilder implements ChannelBuilder {
               private static final Logger log = LoggerFactory.getLogger(SaslChannelBuilder.class);
           
               private final SecurityProtocol securityProtocol;
          +    private final String clientSaslMechanism;
               private final Mode mode;
               private final LoginType loginType;
          +    private final boolean handshakeRequestEnable;
           
               private LoginManager loginManager;
               private SslFactory sslFactory;
               private Map configs;
               private KerberosShortNamer kerberosShortNamer;
           
          -    public SaslChannelBuilder(Mode mode, LoginType loginType, SecurityProtocol securityProtocol) {
          +    public SaslChannelBuilder(Mode mode, LoginType loginType, SecurityProtocol securityProtocol, String clientSaslMechanism, boolean handshakeRequestEnable) {
                   this.mode = mode;
                   this.loginType = loginType;
                   this.securityProtocol = securityProtocol;
          +        this.handshakeRequestEnable = handshakeRequestEnable;
          +        this.clientSaslMechanism = clientSaslMechanism;
               }
           
               public void configure(Map configs) throws KafkaException {
                   try {
                       this.configs = configs;
          -            this.loginManager = LoginManager.acquireLoginManager(loginType, configs);
          +            boolean hasKerberos;
          +            if (mode == Mode.SERVER) {
          +                List enabledMechanisms = (List) this.configs.get(SaslConfigs.SASL_ENABLED_MECHANISMS);
          +                hasKerberos = enabledMechanisms == null || enabledMechanisms.contains(SaslConfigs.GSSAPI_MECHANISM);
          +            } else {
          +                hasKerberos = clientSaslMechanism.equals(SaslConfigs.GSSAPI_MECHANISM);
          +            }
           
                       String defaultRealm;
                       try {
          @@ -61,10 +70,13 @@ public void configure(Map configs) throws KafkaException {
                           defaultRealm = "";
                       }
           
          -            @SuppressWarnings("unchecked")
          -            List principalToLocalRules = (List) configs.get(SaslConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES);
          -            if (principalToLocalRules != null)
          -                kerberosShortNamer = KerberosShortNamer.fromUnparsedRules(defaultRealm, principalToLocalRules);
          +            if (hasKerberos) {
          +                @SuppressWarnings("unchecked")
          +                List principalToLocalRules = (List) configs.get(SaslConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES);
          +                if (principalToLocalRules != null)
          +                    kerberosShortNamer = KerberosShortNamer.fromUnparsedRules(defaultRealm, principalToLocalRules);
          +            }
          +            this.loginManager = LoginManager.acquireLoginManager(loginType, hasKerberos, configs);
           
                       if (this.securityProtocol == SecurityProtocol.SASL_SSL) {
                           // Disable SSL client authentication as we are using SASL authentication
          @@ -82,10 +94,11 @@ public KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize
                       TransportLayer transportLayer = buildTransportLayer(id, key, socketChannel);
                       Authenticator authenticator;
                       if (mode == Mode.SERVER)
          -                authenticator = new SaslServerAuthenticator(id, loginManager.subject(), kerberosShortNamer, maxReceiveSize);
          +                authenticator = new SaslServerAuthenticator(id, loginManager.subject(), kerberosShortNamer,
          +                        socketChannel.socket().getLocalAddress().getHostName(), maxReceiveSize);
                       else
                           authenticator = new SaslClientAuthenticator(id, loginManager.subject(), loginManager.serviceName(),
          -                        socketChannel.socket().getInetAddress().getHostName());
          +                        socketChannel.socket().getInetAddress().getHostName(), clientSaslMechanism, handshakeRequestEnable);
                       // Both authenticators don't use `PrincipalBuilder`, so we pass `null` for now. Reconsider if this changes.
                       authenticator.configure(transportLayer, null, this.configs);
                       return new KafkaChannel(id, transportLayer, authenticator, maxReceiveSize);
          @@ -96,7 +109,8 @@ public KafkaChannel buildChannel(String id, SelectionKey key, int maxReceiveSize
               }
           
               public void close()  {
          -        this.loginManager.release();
          +        if (this.loginManager != null)
          +            this.loginManager.release();
               }
           
               protected TransportLayer buildTransportLayer(String id, SelectionKey key, SocketChannel socketChannel) throws IOException {
          diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
          index e8fd3d3215e55..512a1211b819a 100644
          --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
          +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java
          @@ -36,7 +36,8 @@ public enum ApiKeys {
               LEAVE_GROUP(13, "LeaveGroup"),
               SYNC_GROUP(14, "SyncGroup"),
               DESCRIBE_GROUPS(15, "DescribeGroups"),
          -    LIST_GROUPS(16, "ListGroups");
          +    LIST_GROUPS(16, "ListGroups"),
          +    SASL_HANDSHAKE(17, "SaslHandshake");
           
               private static final ApiKeys[] ID_TO_TYPE;
               private static final int MIN_API_KEY = 0;
          diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
          index 0f33516b934c2..901339903a1e2 100644
          --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
          +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java
          @@ -28,6 +28,7 @@
           import org.apache.kafka.common.errors.GroupCoordinatorNotAvailableException;
           import org.apache.kafka.common.errors.GroupLoadInProgressException;
           import org.apache.kafka.common.errors.IllegalGenerationException;
          +import org.apache.kafka.common.errors.IllegalSaslStateException;
           import org.apache.kafka.common.errors.InconsistentGroupProtocolException;
           import org.apache.kafka.common.errors.InvalidCommitOffsetSizeException;
           import org.apache.kafka.common.errors.InvalidFetchSizeException;
          @@ -49,6 +50,7 @@
           import org.apache.kafka.common.errors.RecordTooLargeException;
           import org.apache.kafka.common.errors.ReplicaNotAvailableException;
           import org.apache.kafka.common.errors.RetriableException;
          +import org.apache.kafka.common.errors.UnsupportedSaslMechanismException;
           import org.apache.kafka.common.errors.TimeoutException;
           import org.apache.kafka.common.errors.TopicAuthorizationException;
           import org.apache.kafka.common.errors.UnknownMemberIdException;
          @@ -130,7 +132,11 @@ public enum Errors {
               CLUSTER_AUTHORIZATION_FAILED(31,
                       new ClusterAuthorizationException("Cluster authorization failed.")),
               INVALID_TIMESTAMP(32,
          -            new InvalidTimestampException("The timestamp of the message is out of acceptable range."));
          +            new InvalidTimestampException("The timestamp of the message is out of acceptable range.")),
          +    UNSUPPORTED_SASL_MECHANISM(33,
          +            new UnsupportedSaslMechanismException("The broker does not support the requested SASL mechanism.")),
          +    ILLEGAL_SASL_STATE(34,
          +            new IllegalSaslStateException("Request is not valid given the current SASL state."));
           
               private static final Logger log = LoggerFactory.getLogger(Errors.class);
           
          diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
          index 248b7ecc51c98..bf76557307cf8 100644
          --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
          +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java
          @@ -718,6 +718,17 @@ public class Protocol {
               public static final Schema[] UPDATE_METADATA_REQUEST = new Schema[] {UPDATE_METADATA_REQUEST_V0, UPDATE_METADATA_REQUEST_V1, UPDATE_METADATA_REQUEST_V2};
               public static final Schema[] UPDATE_METADATA_RESPONSE = new Schema[] {UPDATE_METADATA_RESPONSE_V0, UPDATE_METADATA_RESPONSE_V1, UPDATE_METADATA_RESPONSE_V2};
           
          +    /* SASL handshake api */
          +    public static final Schema SASL_HANDSHAKE_REQUEST_V0 = new Schema(
          +            new Field("mechanism", STRING, "SASL Mechanism chosen by the client."));
          +
          +    public static final Schema SASL_HANDSHAKE_RESPONSE_V0 = new Schema(
          +            new Field("error_code", INT16),
          +            new Field("enabled_mechanisms", new ArrayOf(Type.STRING), "Array of mechanisms enabled in the server."));
          +
          +    public static final Schema[] SASL_HANDSHAKE_REQUEST = new Schema[] {SASL_HANDSHAKE_REQUEST_V0};
          +    public static final Schema[] SASL_HANDSHAKE_RESPONSE = new Schema[] {SASL_HANDSHAKE_RESPONSE_V0};
          +
               /* an array of all requests and responses with all schema versions; a null value in the inner array means that the
                * particular version is not supported */
               public static final Schema[][] REQUESTS = new Schema[ApiKeys.MAX_API_KEY + 1][];
          @@ -744,6 +755,7 @@ public class Protocol {
                   REQUESTS[ApiKeys.SYNC_GROUP.id] = SYNC_GROUP_REQUEST;
                   REQUESTS[ApiKeys.DESCRIBE_GROUPS.id] = DESCRIBE_GROUPS_REQUEST;
                   REQUESTS[ApiKeys.LIST_GROUPS.id] = LIST_GROUPS_REQUEST;
          +        REQUESTS[ApiKeys.SASL_HANDSHAKE.id] = SASL_HANDSHAKE_REQUEST;
           
                   RESPONSES[ApiKeys.PRODUCE.id] = PRODUCE_RESPONSE;
                   RESPONSES[ApiKeys.FETCH.id] = FETCH_RESPONSE;
          @@ -762,6 +774,7 @@ public class Protocol {
                   RESPONSES[ApiKeys.SYNC_GROUP.id] = SYNC_GROUP_RESPONSE;
                   RESPONSES[ApiKeys.DESCRIBE_GROUPS.id] = DESCRIBE_GROUPS_RESPONSE;
                   RESPONSES[ApiKeys.LIST_GROUPS.id] = LIST_GROUPS_RESPONSE;
          +        RESPONSES[ApiKeys.SASL_HANDSHAKE.id] = SASL_HANDSHAKE_RESPONSE;
           
                   /* set the maximum version of each api */
                   for (ApiKeys api : ApiKeys.values())
          diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
          index 5a40b7fe2149a..89c2ce15bb279 100644
          --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
          +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java
          @@ -72,6 +72,8 @@ public static AbstractRequest getRequest(int requestId, int versionId, ByteBuffe
                           return DescribeGroupsRequest.parse(buffer, versionId);
                       case LIST_GROUPS:
                           return ListGroupsRequest.parse(buffer, versionId);
          +            case SASL_HANDSHAKE:
          +                return SaslHandshakeRequest.parse(buffer, versionId);
                       default:
                           throw new AssertionError(String.format("ApiKey %s is not currently handled in `getRequest`, the " +
                                   "code should be updated to do so.", apiKey));
          diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ResponseSend.java b/clients/src/main/java/org/apache/kafka/common/requests/ResponseSend.java
          index 12b06d16e0380..9494de7caaa78 100644
          --- a/clients/src/main/java/org/apache/kafka/common/requests/ResponseSend.java
          +++ b/clients/src/main/java/org/apache/kafka/common/requests/ResponseSend.java
          @@ -31,7 +31,7 @@ public ResponseSend(String destination, ResponseHeader header, AbstractRequestRe
                   this(destination, header, response.toStruct());
               }
           
          -    private static ByteBuffer serialize(ResponseHeader header, Struct body) {
          +    public static ByteBuffer serialize(ResponseHeader header, Struct body) {
                   ByteBuffer buffer = ByteBuffer.allocate(header.sizeOf() + body.sizeOf());
                   header.writeTo(buffer);
                   body.writeTo(buffer);
          diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java
          new file mode 100644
          index 0000000000000..bddc9f092c06c
          --- /dev/null
          +++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeRequest.java
          @@ -0,0 +1,83 @@
          +/**
          + * 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 org.apache.kafka.common.requests;
          +
          +import java.nio.ByteBuffer;
          +import java.util.Collections;
          +import java.util.List;
          +
          +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.Schema;
          +import org.apache.kafka.common.protocol.types.Struct;
          +
          +
          +/**
          + * Request from SASL client containing client SASL mechanism.
          + * 

          + * For interoperability with Kafka 0.9.0.x, the mechanism flow may be omitted when using GSSAPI. Hence + * this request should not conflict with the first GSSAPI client packet. For GSSAPI, the first context + * establishment packet starts with byte 0x60 (APPLICATION-0 tag) followed by a variable-length encoded size. + * This handshake request starts with a request header two-byte API key set to 17, followed by a mechanism name, + * making it easy to distinguish from a GSSAPI packet. + */ +public class SaslHandshakeRequest extends AbstractRequest { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.SASL_HANDSHAKE.id); + public static final String MECHANISM_KEY_NAME = "mechanism"; + + private final String mechanism; + + public SaslHandshakeRequest(String mechanism) { + super(new Struct(CURRENT_SCHEMA)); + struct.set(MECHANISM_KEY_NAME, mechanism); + this.mechanism = mechanism; + } + + public SaslHandshakeRequest(Struct struct) { + super(struct); + mechanism = struct.getString(MECHANISM_KEY_NAME); + } + + public String mechanism() { + return mechanism; + } + + @Override + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { + switch (versionId) { + case 0: + List enabledMechanisms = Collections.emptyList(); + return new SaslHandshakeResponse(Errors.forException(e).code(), enabledMechanisms); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.SASL_HANDSHAKE.id))); + } + } + + public static SaslHandshakeRequest parse(ByteBuffer buffer, int versionId) { + return new SaslHandshakeRequest(ProtoUtils.parseRequest(ApiKeys.SASL_HANDSHAKE.id, versionId, buffer)); + } + + public static SaslHandshakeRequest parse(ByteBuffer buffer) { + return new SaslHandshakeRequest(CURRENT_SCHEMA.read(buffer)); + } +} + diff --git a/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java new file mode 100644 index 0000000000000..c0fc4954c1607 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/SaslHandshakeResponse.java @@ -0,0 +1,85 @@ +/** + * 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 org.apache.kafka.common.requests; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.List; + +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.ProtoUtils; +import org.apache.kafka.common.protocol.types.Schema; +import org.apache.kafka.common.protocol.types.Struct; + + +/** + * Response from SASL server which indicates if the client-chosen mechanism is enabled in the server. + * For error responses, the list of enabled mechanisms is included in the response. + */ +public class SaslHandshakeResponse extends AbstractRequestResponse { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.SASL_HANDSHAKE.id); + + private static final String ERROR_CODE_KEY_NAME = "error_code"; + private static final String ENABLED_MECHANISMS_KEY_NAME = "enabled_mechanisms"; + + /** + * Possible error codes: + * UNSUPPORTED_SASL_MECHANISM(33): Client mechanism not enabled in server + * ILLEGAL_SASL_STATE(34) : Invalid request during SASL handshake + */ + private final short errorCode; + private final List enabledMechanisms; + + public SaslHandshakeResponse(short errorCode, Collection enabledMechanisms) { + super(new Struct(CURRENT_SCHEMA)); + struct.set(ERROR_CODE_KEY_NAME, errorCode); + struct.set(ENABLED_MECHANISMS_KEY_NAME, enabledMechanisms.toArray()); + this.errorCode = errorCode; + this.enabledMechanisms = new ArrayList<>(enabledMechanisms); + } + + public SaslHandshakeResponse(Struct struct) { + super(struct); + errorCode = struct.getShort(ERROR_CODE_KEY_NAME); + Object[] mechanisms = struct.getArray(ENABLED_MECHANISMS_KEY_NAME); + ArrayList enabledMechanisms = new ArrayList<>(); + for (Object mechanism : mechanisms) + enabledMechanisms.add((String) mechanism); + this.enabledMechanisms = enabledMechanisms; + } + + public short errorCode() { + return errorCode; + } + + public List enabledMechanisms() { + return enabledMechanisms; + } + + public static SaslHandshakeResponse parse(ByteBuffer buffer) { + return new SaslHandshakeResponse(CURRENT_SCHEMA.read(buffer)); + } + + public static SaslHandshakeResponse parse(ByteBuffer buffer, int version) { + return new SaslHandshakeResponse(ProtoUtils.parseResponse(ApiKeys.SASL_HANDSHAKE.id, version, buffer)); + } +} + diff --git a/clients/src/main/java/org/apache/kafka/common/security/auth/AuthCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/auth/AuthCallbackHandler.java new file mode 100644 index 0000000000000..ed2c087fdfc82 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/auth/AuthCallbackHandler.java @@ -0,0 +1,46 @@ +/** + * 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 org.apache.kafka.common.security.auth; + +import java.util.Map; + +import org.apache.kafka.common.network.Mode; + +import javax.security.auth.Subject; +import javax.security.auth.callback.CallbackHandler; + +/* + * Callback handler for SASL-based authentication + */ +public interface AuthCallbackHandler extends CallbackHandler { + + /** + * Configures this callback handler. + * + * @param configs Configuration + * @param mode The mode that indicates if this is a client or server connection + * @param subject Subject from login context + * @param saslMechanism Negotiated SASL mechanism + */ + void configure(Map configs, Mode mode, Subject subject, String saslMechanism); + + /** + * Closes this instance. + */ + void close(); +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/auth/Login.java b/clients/src/main/java/org/apache/kafka/common/security/auth/Login.java new file mode 100644 index 0000000000000..1ac779d7dfd66 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/auth/Login.java @@ -0,0 +1,57 @@ +/** + * 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 org.apache.kafka.common.security.auth; + +import java.util.Map; + +import javax.security.auth.Subject; +import javax.security.auth.login.LoginContext; +import javax.security.auth.login.LoginException; + +/** + * Login interface for authentication. + */ +public interface Login { + + /** + * Configures this login instance. + */ + void configure(Map configs, String loginContextName); + + /** + * Performs login for each login module specified for the login context of this instance. + */ + LoginContext login() throws LoginException; + + /** + * Returns the authenticated subject of this login context. + */ + Subject subject(); + + /** + * Returns the service name to be used for SASL. + */ + String serviceName(); + + /** + * Closes this instance. + */ + void close(); +} + diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/AbstractLogin.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/AbstractLogin.java new file mode 100644 index 0000000000000..2fe43abaa28ae --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/AbstractLogin.java @@ -0,0 +1,108 @@ +/** + * 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 org.apache.kafka.common.security.authenticator; + +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.Configuration; +import javax.security.auth.login.LoginContext; +import javax.security.auth.login.LoginException; +import javax.security.sasl.RealmCallback; +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.auth.Subject; + +import org.apache.kafka.common.security.JaasUtils; +import org.apache.kafka.common.security.auth.Login; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.Map; + +/** + * Base login class that implements methods common to typical SASL mechanisms. + */ +public abstract class AbstractLogin implements Login { + private static final Logger log = LoggerFactory.getLogger(AbstractLogin.class); + + private String loginContextName; + private LoginContext loginContext; + + + @Override + public void configure(Map configs, String loginContextName) { + this.loginContextName = loginContextName; + } + + @Override + public LoginContext login() throws LoginException { + String jaasConfigFile = System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM); + if (jaasConfigFile == null) { + log.debug("System property '" + JaasUtils.JAVA_LOGIN_CONFIG_PARAM + "' is not set, using default JAAS configuration."); + } + AppConfigurationEntry[] configEntries = Configuration.getConfiguration().getAppConfigurationEntry(loginContextName); + if (configEntries == null) { + String errorMessage = "Could not find a '" + loginContextName + "' entry in the JAAS configuration. System property '" + + JaasUtils.JAVA_LOGIN_CONFIG_PARAM + "' is " + (jaasConfigFile == null ? "not set" : jaasConfigFile); + throw new IllegalArgumentException(errorMessage); + } + + loginContext = new LoginContext(loginContextName, new LoginCallbackHandler()); + loginContext.login(); + log.info("Successfully logged in."); + return loginContext; + } + + @Override + public Subject subject() { + return loginContext.getSubject(); + } + + /** + * Callback handler for creating login context. Login callback handlers + * should support the callbacks required for the login modules used by + * the KafkaServer and KafkaClient contexts. Kafka does not support + * callback handlers which require additional user input. + * + */ + public static class LoginCallbackHandler implements CallbackHandler { + + @Override + public void handle(Callback[] callbacks) throws UnsupportedCallbackException { + for (Callback callback : callbacks) { + if (callback instanceof NameCallback) { + NameCallback nc = (NameCallback) callback; + nc.setName(nc.getDefaultName()); + } else if (callback instanceof PasswordCallback) { + String errorMessage = "Could not login: the client is being asked for a password, but the Kafka" + + " client code does not currently support obtaining a password from the user."; + throw new UnsupportedCallbackException(callback, errorMessage); + } else if (callback instanceof RealmCallback) { + RealmCallback rc = (RealmCallback) callback; + rc.setText(rc.getDefaultText()); + } else { + throw new UnsupportedCallbackException(callback, "Unrecognized SASL Login callback"); + } + } + } + } +} + diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/DefaultLogin.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/DefaultLogin.java new file mode 100644 index 0000000000000..0a405bc3f2be3 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/DefaultLogin.java @@ -0,0 +1,32 @@ +/** + * 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 org.apache.kafka.common.security.authenticator; + +public class DefaultLogin extends AbstractLogin { + + @Override + public String serviceName() { + return "kafka"; + } + + @Override + public void close() { + } +} + diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/LoginManager.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java similarity index 69% rename from clients/src/main/java/org/apache/kafka/common/security/kerberos/LoginManager.java rename to clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java index e163ba8cf70c6..9aec9a7af7411 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/kerberos/LoginManager.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/LoginManager.java @@ -16,51 +16,34 @@ * limitations under the License. */ -package org.apache.kafka.common.security.kerberos; +package org.apache.kafka.common.security.authenticator; import javax.security.auth.Subject; import javax.security.auth.login.LoginException; + import java.io.IOException; import java.util.ArrayList; import java.util.EnumMap; import java.util.Map; -import org.apache.kafka.common.config.SaslConfigs; import org.apache.kafka.common.network.LoginType; -import org.apache.kafka.common.security.JaasUtils; +import org.apache.kafka.common.security.auth.Login; +import org.apache.kafka.common.security.kerberos.KerberosLogin; public class LoginManager { private static final EnumMap CACHED_INSTANCES = new EnumMap<>(LoginType.class); private final Login login; - private final String serviceName; private final LoginType loginType; private int refCount; - private LoginManager(LoginType loginType, Map configs) throws IOException, LoginException { + private LoginManager(LoginType loginType, boolean hasKerberos, Map configs) throws IOException, LoginException { this.loginType = loginType; String loginContext = loginType.contextName(); - login = new Login(loginContext, configs); - this.serviceName = getServiceName(loginContext, configs); - login.startThreadIfNeeded(); - } - - private static String getServiceName(String loginContext, Map configs) throws IOException { - String jaasServiceName = JaasUtils.jaasConfig(loginContext, JaasUtils.SERVICE_NAME); - String configServiceName = (String) configs.get(SaslConfigs.SASL_KERBEROS_SERVICE_NAME); - if (jaasServiceName != null && configServiceName != null && !jaasServiceName.equals(configServiceName)) { - String message = "Conflicting serviceName values found in JAAS and Kafka configs " + - "value in JAAS file " + jaasServiceName + ", value in Kafka config " + configServiceName; - throw new IllegalArgumentException(message); - } - - if (jaasServiceName != null) - return jaasServiceName; - if (configServiceName != null) - return configServiceName; - - throw new IllegalArgumentException("No serviceName defined in either JAAS or Kafka config"); + login = hasKerberos ? new KerberosLogin() : new DefaultLogin(); + login.configure(configs, loginContext); + login.login(); } /** @@ -78,11 +61,11 @@ private static String getServiceName(String loginContext, Map configs * (i.e. consumer and producer) * @param configs configuration as key/value pairs */ - public static final LoginManager acquireLoginManager(LoginType loginType, Map configs) throws IOException, LoginException { + public static final LoginManager acquireLoginManager(LoginType loginType, boolean hasKerberos, Map configs) throws IOException, LoginException { synchronized (LoginManager.class) { LoginManager loginManager = CACHED_INSTANCES.get(loginType); if (loginManager == null) { - loginManager = new LoginManager(loginType, configs); + loginManager = new LoginManager(loginType, hasKerberos, configs); CACHED_INSTANCES.put(loginType, loginManager); } return loginManager.acquire(); @@ -94,7 +77,7 @@ public Subject subject() { } public String serviceName() { - return serviceName; + return login.serviceName(); } private LoginManager acquire() { @@ -111,7 +94,7 @@ public void release() { throw new IllegalStateException("release called on LoginManager with refCount == 0"); else if (refCount == 1) { CACHED_INSTANCES.remove(loginType); - login.shutdown(); + login.close(); } --refCount; } @@ -122,9 +105,8 @@ public static void closeAll() { synchronized (LoginManager.class) { for (LoginType loginType : new ArrayList<>(CACHED_INSTANCES.keySet())) { LoginManager loginManager = CACHED_INSTANCES.remove(loginType); - loginManager.login.shutdown(); + loginManager.login.close(); } } } - } diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java index 370e7296f7f2c..ba201dcd67144 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientAuthenticator.java @@ -23,38 +23,44 @@ import java.nio.channels.SelectionKey; import java.util.Arrays; import java.util.Map; - import java.security.Principal; import java.security.PrivilegedActionException; import java.security.PrivilegedExceptionAction; import javax.security.auth.Subject; -import javax.security.auth.callback.Callback; -import javax.security.auth.callback.CallbackHandler; -import javax.security.auth.callback.NameCallback; -import javax.security.auth.callback.PasswordCallback; -import javax.security.auth.callback.UnsupportedCallbackException; -import javax.security.sasl.AuthorizeCallback; -import javax.security.sasl.RealmCallback; import javax.security.sasl.Sasl; import javax.security.sasl.SaslClient; import javax.security.sasl.SaslException; +import org.apache.kafka.common.errors.AuthenticationException; +import org.apache.kafka.common.errors.IllegalSaslStateException; +import org.apache.kafka.common.errors.UnsupportedSaslMechanismException; import org.apache.kafka.common.network.Authenticator; +import org.apache.kafka.common.network.Mode; import org.apache.kafka.common.network.NetworkSend; import org.apache.kafka.common.network.NetworkReceive; import org.apache.kafka.common.network.TransportLayer; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.SchemaException; +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.SaslHandshakeRequest; +import org.apache.kafka.common.requests.SaslHandshakeResponse; +import org.apache.kafka.common.security.auth.AuthCallbackHandler; import org.apache.kafka.common.security.auth.KafkaPrincipal; import org.apache.kafka.common.security.auth.PrincipalBuilder; +import org.apache.kafka.clients.CommonClientConfigs; +import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.common.KafkaException; - import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class SaslClientAuthenticator implements Authenticator { public enum SaslState { - INITIAL, INTERMEDIATE, COMPLETE, FAILED + SEND_HANDSHAKE_REQUEST, RECEIVE_HANDSHAKE_RESPONSE, INITIAL, INTERMEDIATE, COMPLETE, FAILED } private static final Logger LOG = LoggerFactory.getLogger(SaslClientAuthenticator.class); @@ -63,33 +69,57 @@ public enum SaslState { private final String servicePrincipal; private final String host; private final String node; + private final String mechanism; + private final boolean handshakeRequestEnable; // assigned in `configure` private SaslClient saslClient; + private Map configs; private String clientPrincipalName; + private AuthCallbackHandler callbackHandler; private TransportLayer transportLayer; // buffers used in `authenticate` private NetworkReceive netInBuffer; private NetworkSend netOutBuffer; - private SaslState saslState = SaslState.INITIAL; + // Current SASL state + private SaslState saslState; + // Next SASL state to be set when outgoing writes associated with the current SASL state complete + private SaslState pendingSaslState; + // Correlation ID for the next request + private int correlationId; + // Request header for which response from the server is pending + private RequestHeader currentRequestHeader; - public SaslClientAuthenticator(String node, Subject subject, String servicePrincipal, String host) throws IOException { + public SaslClientAuthenticator(String node, Subject subject, String servicePrincipal, String host, String mechanism, boolean handshakeRequestEnable) throws IOException { this.node = node; this.subject = subject; this.host = host; this.servicePrincipal = servicePrincipal; + this.mechanism = mechanism; + this.handshakeRequestEnable = handshakeRequestEnable; + this.correlationId = -1; } public void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder, Map configs) throws KafkaException { try { this.transportLayer = transportLayer; + this.configs = configs; + + setSaslState(handshakeRequestEnable ? SaslState.SEND_HANDSHAKE_REQUEST : SaslState.INITIAL); // determine client principal from subject. - Principal clientPrincipal = subject.getPrincipals().iterator().next(); - this.clientPrincipalName = clientPrincipal.getName(); - this.saslClient = createSaslClient(); + if (!subject.getPrincipals().isEmpty()) { + Principal clientPrincipal = subject.getPrincipals().iterator().next(); + this.clientPrincipalName = clientPrincipal.getName(); + } else { + clientPrincipalName = null; + } + callbackHandler = new SaslClientCallbackHandler(); + callbackHandler.configure(configs, Mode.CLIENT, subject, mechanism); + + saslClient = createSaslClient(); } catch (Exception e) { throw new KafkaException("Failed to configure SaslClientAuthenticator", e); } @@ -99,15 +129,14 @@ private SaslClient createSaslClient() { try { return Subject.doAs(subject, new PrivilegedExceptionAction() { public SaslClient run() throws SaslException { - String[] mechs = {"GSSAPI"}; + String[] mechs = {mechanism}; LOG.debug("Creating SaslClient: client={};service={};serviceHostname={};mechs={}", clientPrincipalName, servicePrincipal, host, Arrays.toString(mechs)); - return Sasl.createSaslClient(mechs, clientPrincipalName, servicePrincipal, host, null, - new ClientCallbackHandler()); + return Sasl.createSaslClient(mechs, clientPrincipalName, servicePrincipal, host, configs, callbackHandler); } }); } catch (PrivilegedActionException e) { - throw new KafkaException("Failed to create SaslClient", e.getCause()); + throw new KafkaException("Failed to create SaslClient with mechanism " + mechanism, e.getCause()); } } @@ -123,22 +152,39 @@ public void authenticate() throws IOException { return; switch (saslState) { + case SEND_HANDSHAKE_REQUEST: + String clientId = (String) configs.get(CommonClientConfigs.CLIENT_ID_CONFIG); + currentRequestHeader = new RequestHeader(ApiKeys.SASL_HANDSHAKE.id, clientId, correlationId++); + SaslHandshakeRequest handshakeRequest = new SaslHandshakeRequest(mechanism); + send(RequestSend.serialize(currentRequestHeader, handshakeRequest.toStruct())); + setSaslState(SaslState.RECEIVE_HANDSHAKE_RESPONSE); + break; + case RECEIVE_HANDSHAKE_RESPONSE: + byte[] responseBytes = receiveResponseOrToken(); + if (responseBytes == null) + break; + else { + try { + handleKafkaResponse(currentRequestHeader, responseBytes); + currentRequestHeader = null; + } catch (Exception e) { + setSaslState(SaslState.FAILED); + throw e; + } + setSaslState(SaslState.INITIAL); + // Fall through and start SASL authentication using the configured client mechanism + } case INITIAL: - sendSaslToken(new byte[0]); - saslState = SaslState.INTERMEDIATE; + sendSaslToken(new byte[0], true); + setSaslState(SaslState.INTERMEDIATE); break; case INTERMEDIATE: - if (netInBuffer == null) netInBuffer = new NetworkReceive(node); - netInBuffer.readFrom(transportLayer); - if (netInBuffer.complete()) { - netInBuffer.payload().rewind(); - byte[] serverToken = new byte[netInBuffer.payload().remaining()]; - netInBuffer.payload().get(serverToken, 0, serverToken.length); - netInBuffer = null; // reset the networkReceive as we read all the data. - sendSaslToken(serverToken); + byte[] serverToken = receiveResponseOrToken(); + if (serverToken != null) { + sendSaslToken(serverToken, false); } if (saslClient.isComplete()) { - saslState = SaslState.COMPLETE; + setSaslState(SaslState.COMPLETE); transportLayer.removeInterestOps(SelectionKey.OP_WRITE); } break; @@ -149,30 +195,58 @@ public void authenticate() throws IOException { } } - private void sendSaslToken(byte[] serverToken) throws IOException { + private void setSaslState(SaslState saslState) { + if (netOutBuffer != null && !netOutBuffer.completed()) + pendingSaslState = saslState; + else { + this.pendingSaslState = null; + this.saslState = saslState; + LOG.debug("Set SASL client state to {}", saslState); + } + } + + private void sendSaslToken(byte[] serverToken, boolean isInitial) throws IOException { if (!saslClient.isComplete()) { - try { - byte[] saslToken = createSaslToken(serverToken); - if (saslToken != null) { - netOutBuffer = new NetworkSend(node, ByteBuffer.wrap(saslToken)); - flushNetOutBufferAndUpdateInterestOps(); - } - } catch (IOException e) { - saslState = SaslState.FAILED; - throw e; - } + byte[] saslToken = createSaslToken(serverToken, isInitial); + if (saslToken != null) + send(ByteBuffer.wrap(saslToken)); + } + } + + private void send(ByteBuffer buffer) throws IOException { + try { + netOutBuffer = new NetworkSend(node, buffer); + flushNetOutBufferAndUpdateInterestOps(); + } catch (IOException e) { + setSaslState(SaslState.FAILED); + throw e; } } private boolean flushNetOutBufferAndUpdateInterestOps() throws IOException { boolean flushedCompletely = flushNetOutBuffer(); - if (flushedCompletely) + if (flushedCompletely) { transportLayer.removeInterestOps(SelectionKey.OP_WRITE); - else + if (pendingSaslState != null) + setSaslState(pendingSaslState); + } else transportLayer.addInterestOps(SelectionKey.OP_WRITE); return flushedCompletely; } + private byte[] receiveResponseOrToken() throws IOException { + if (netInBuffer == null) netInBuffer = new NetworkReceive(node); + netInBuffer.readFrom(transportLayer); + byte[] serverPacket = null; + if (netInBuffer.complete()) { + netInBuffer.payload().rewind(); + serverPacket = new byte[netInBuffer.payload().remaining()]; + netInBuffer.payload().get(serverPacket, 0, serverPacket.length); + netInBuffer = null; // reset the networkReceive as we read all the data. + } + return serverPacket; + } + public Principal principal() { return new KafkaPrincipal(KafkaPrincipal.USER_TYPE, clientPrincipalName); } @@ -182,19 +256,25 @@ public boolean complete() { } public void close() throws IOException { - saslClient.dispose(); + if (saslClient != null) + saslClient.dispose(); + if (callbackHandler != null) + callbackHandler.close(); } - private byte[] createSaslToken(final byte[] saslToken) throws SaslException { + private byte[] createSaslToken(final byte[] saslToken, boolean isInitial) throws SaslException { if (saslToken == null) throw new SaslException("Error authenticating with the Kafka Broker: received a `null` saslToken."); try { - return Subject.doAs(subject, new PrivilegedExceptionAction() { - public byte[] run() throws SaslException { - return saslClient.evaluateChallenge(saslToken); - } - }); + if (isInitial && !saslClient.hasInitialResponse()) + return saslToken; + else + return Subject.doAs(subject, new PrivilegedExceptionAction() { + public byte[] run() throws SaslException { + return saslClient.evaluateChallenge(saslToken); + } + }); } catch (PrivilegedActionException e) { String error = "An error: (" + e + ") occurred when evaluating SASL token received from the Kafka Broker."; // Try to provide hints to use about what went wrong so they can fix their configuration. @@ -221,35 +301,39 @@ private boolean flushNetOutBuffer() throws IOException { return netOutBuffer.completed(); } - public static class ClientCallbackHandler implements CallbackHandler { - - public void handle(Callback[] callbacks) throws UnsupportedCallbackException { - for (Callback callback : callbacks) { - if (callback instanceof NameCallback) { - NameCallback nc = (NameCallback) callback; - nc.setName(nc.getDefaultName()); - } else if (callback instanceof PasswordCallback) { - // Call `setPassword` once we support obtaining a password from the user and update message below - throw new UnsupportedCallbackException(callback, "Could not login: the client is being asked for a password, but the Kafka" + - " client code does not currently support obtaining a password from the user." + - " Make sure -Djava.security.auth.login.config property passed to JVM and" + - " the client is configured to use a ticket cache (using" + - " the JAAS configuration setting 'useTicketCache=true)'. Make sure you are using" + - " FQDN of the Kafka broker you are trying to connect to."); - } else if (callback instanceof RealmCallback) { - RealmCallback rc = (RealmCallback) callback; - rc.setText(rc.getDefaultText()); - } else if (callback instanceof AuthorizeCallback) { - AuthorizeCallback ac = (AuthorizeCallback) callback; - String authId = ac.getAuthenticationID(); - String authzId = ac.getAuthorizationID(); - ac.setAuthorized(authId.equals(authzId)); - if (ac.isAuthorized()) - ac.setAuthorizedID(authzId); - } else { - throw new UnsupportedCallbackException(callback, "Unrecognized SASL ClientCallback"); - } - } + private void handleKafkaResponse(RequestHeader requestHeader, byte[] responseBytes) { + Struct struct; + ApiKeys apiKey; + try { + struct = NetworkClient.parseResponse(ByteBuffer.wrap(responseBytes), requestHeader); + apiKey = ApiKeys.forId(requestHeader.apiKey()); + } catch (SchemaException | IllegalArgumentException e) { + LOG.debug("Invalid SASL mechanism response, server may be expecting only GSSAPI tokens"); + throw new AuthenticationException("Invalid SASL mechanism response", e); + } + switch (apiKey) { + case SASL_HANDSHAKE: + handleSaslHandshakeResponse(new SaslHandshakeResponse(struct)); + break; + default: + throw new IllegalStateException("Unexpected API key during handshake: " + apiKey); + } + } + + private void handleSaslHandshakeResponse(SaslHandshakeResponse response) { + Errors error = Errors.forCode(response.errorCode()); + switch (error) { + case NONE: + break; + case UNSUPPORTED_SASL_MECHANISM: + throw new UnsupportedSaslMechanismException(String.format("Client SASL mechanism '%s' not enabled in the server, enabled mechanisms are %s", + mechanism, response.enabledMechanisms())); + case ILLEGAL_SASL_STATE: + throw new IllegalSaslStateException(String.format("Unexpected handshake request with client mechanism %s, enabled mechanisms are %s", + mechanism, response.enabledMechanisms())); + default: + throw new AuthenticationException(String.format("Unknown error code %d, client mechanism is %s, enabled mechanisms are %s", + response.errorCode(), mechanism, response.enabledMechanisms())); } } } diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java new file mode 100644 index 0000000000000..8e0b8dbfa3d27 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslClientCallbackHandler.java @@ -0,0 +1,94 @@ +/** + * 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 org.apache.kafka.common.security.authenticator; + +import java.util.Map; + +import javax.security.auth.Subject; +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; + +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.network.Mode; +import org.apache.kafka.common.security.auth.AuthCallbackHandler; + +/** + * Callback handler for Sasl clients. The callbacks required for the SASL mechanism + * configured for the client should be supported by this callback handler. See + * Java SASL API + * for the list of SASL callback handlers required for each SASL mechanism. + */ +public class SaslClientCallbackHandler implements AuthCallbackHandler { + + private boolean isKerberos; + private Subject subject; + + @Override + public void configure(Map configs, Mode mode, Subject subject, String mechanism) { + this.isKerberos = mechanism.equals(SaslConfigs.GSSAPI_MECHANISM); + this.subject = subject; + } + + @Override + public void handle(Callback[] callbacks) throws UnsupportedCallbackException { + for (Callback callback : callbacks) { + if (callback instanceof NameCallback) { + NameCallback nc = (NameCallback) callback; + if (!isKerberos && subject != null && !subject.getPublicCredentials(String.class).isEmpty()) { + nc.setName(subject.getPublicCredentials(String.class).iterator().next()); + } else + nc.setName(nc.getDefaultName()); + } else if (callback instanceof PasswordCallback) { + if (!isKerberos && subject != null && !subject.getPrivateCredentials(String.class).isEmpty()) { + char [] password = subject.getPrivateCredentials(String.class).iterator().next().toCharArray(); + ((PasswordCallback) callback).setPassword(password); + } else { + String errorMessage = "Could not login: the client is being asked for a password, but the Kafka" + + " client code does not currently support obtaining a password from the user."; + if (isKerberos) { + errorMessage += " Make sure -Djava.security.auth.login.config property passed to JVM and" + + " the client is configured to use a ticket cache (using" + + " the JAAS configuration setting 'useTicketCache=true)'. Make sure you are using" + + " FQDN of the Kafka broker you are trying to connect to."; + } + throw new UnsupportedCallbackException(callback, errorMessage); + } + } else if (callback instanceof RealmCallback) { + RealmCallback rc = (RealmCallback) callback; + rc.setText(rc.getDefaultText()); + } else if (callback instanceof AuthorizeCallback) { + AuthorizeCallback ac = (AuthorizeCallback) callback; + String authId = ac.getAuthenticationID(); + String authzId = ac.getAuthorizationID(); + ac.setAuthorized(authId.equals(authzId)); + if (ac.isAuthorized()) + ac.setAuthorizedID(authzId); + } else { + throw new UnsupportedCallbackException(callback, "Unrecognized SASL ClientCallback"); + } + } + } + + @Override + public void close() { + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java index 1f925f99a860b..89c6e6c0c0a7d 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java @@ -18,11 +18,12 @@ package org.apache.kafka.common.security.authenticator; import java.io.IOException; +import java.util.HashSet; +import java.util.List; import java.util.Map; - +import java.util.Set; import java.nio.ByteBuffer; import java.nio.channels.SelectionKey; - import java.security.Principal; import java.security.PrivilegedActionException; import java.security.PrivilegedExceptionAction; @@ -44,51 +45,102 @@ import org.ietf.jgss.Oid; import org.slf4j.Logger; import org.slf4j.LoggerFactory; - +import org.apache.kafka.common.security.auth.AuthCallbackHandler; import org.apache.kafka.common.security.auth.KafkaPrincipal; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.errors.AuthenticationException; +import org.apache.kafka.common.errors.IllegalSaslStateException; +import org.apache.kafka.common.errors.UnsupportedSaslMechanismException; import org.apache.kafka.common.network.Authenticator; +import org.apache.kafka.common.network.Mode; import org.apache.kafka.common.network.NetworkSend; import org.apache.kafka.common.network.NetworkReceive; import org.apache.kafka.common.network.TransportLayer; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.types.SchemaException; +import org.apache.kafka.common.requests.AbstractRequest; +import org.apache.kafka.common.requests.AbstractRequestResponse; +import org.apache.kafka.common.requests.RequestHeader; +import org.apache.kafka.common.requests.ResponseHeader; +import org.apache.kafka.common.requests.ResponseSend; +import org.apache.kafka.common.requests.SaslHandshakeRequest; +import org.apache.kafka.common.requests.SaslHandshakeResponse; import org.apache.kafka.common.security.auth.PrincipalBuilder; public class SaslServerAuthenticator implements Authenticator { private static final Logger LOG = LoggerFactory.getLogger(SaslServerAuthenticator.class); - private final SaslServer saslServer; - private final Subject subject; + public enum SaslState { + HANDSHAKE_REQUEST, AUTHENTICATE, COMPLETE, FAILED + } + private final String node; + private final Subject subject; private final KerberosShortNamer kerberosNamer; private final int maxReceiveSize; + private final String host; + + // Current SASL state + private SaslState saslState = SaslState.HANDSHAKE_REQUEST; + // Next SASL state to be set when outgoing writes associated with the current SASL state complete + private SaslState pendingSaslState = null; + private SaslServer saslServer; + private String saslMechanism; + private AuthCallbackHandler callbackHandler; // assigned in `configure` private TransportLayer transportLayer; + private Set enabledMechanisms; + private Map configs; // buffers used in `authenticate` private NetworkReceive netInBuffer; private NetworkSend netOutBuffer; - public SaslServerAuthenticator(String node, final Subject subject, KerberosShortNamer kerberosNameParser, int maxReceiveSize) throws IOException { + public SaslServerAuthenticator(String node, final Subject subject, KerberosShortNamer kerberosNameParser, String host, int maxReceiveSize) throws IOException { if (subject == null) throw new IllegalArgumentException("subject cannot be null"); - if (subject.getPrincipals().isEmpty()) - throw new IllegalArgumentException("subject must have at least one principal"); this.node = node; this.subject = subject; this.kerberosNamer = kerberosNameParser; this.maxReceiveSize = maxReceiveSize; - saslServer = createSaslServer(); + this.host = host; } public void configure(TransportLayer transportLayer, PrincipalBuilder principalBuilder, Map configs) { this.transportLayer = transportLayer; + this.configs = configs; + List enabledMechanisms = (List) this.configs.get(SaslConfigs.SASL_ENABLED_MECHANISMS); + if (enabledMechanisms == null || enabledMechanisms.isEmpty()) + throw new IllegalArgumentException("No SASL mechanisms are enabled"); + this.enabledMechanisms = new HashSet<>(enabledMechanisms); } - private SaslServer createSaslServer() throws IOException { + private void createSaslServer(String mechanism) throws IOException { + this.saslMechanism = mechanism; + callbackHandler = new SaslServerCallbackHandler(Configuration.getConfiguration(), kerberosNamer); + callbackHandler.configure(configs, Mode.SERVER, subject, saslMechanism); + if (mechanism.equals(SaslConfigs.GSSAPI_MECHANISM)) { + if (subject.getPrincipals().isEmpty()) + throw new IllegalArgumentException("subject must have at least one principal"); + saslServer = createSaslKerberosServer(callbackHandler, configs); + } else { + try { + saslServer = Subject.doAs(subject, new PrivilegedExceptionAction() { + public SaslServer run() throws SaslException { + return Sasl.createSaslServer(saslMechanism, "kafka", host, configs, callbackHandler); + } + }); + } catch (PrivilegedActionException e) { + throw new SaslException("Kafka Server failed to create a SaslServer to interact with a client during session authentication", e.getCause()); + } + } + } + + private SaslServer createSaslKerberosServer(final AuthCallbackHandler saslServerCallbackHandler, final Map configs) throws IOException { // server is using a JAAS-authenticated subject: determine service principal name and hostname from kafka server's subject. - final SaslServerCallbackHandler saslServerCallbackHandler = new SaslServerCallbackHandler( - Configuration.getConfiguration(), kerberosNamer); final Principal servicePrincipal = subject.getPrincipals().iterator().next(); KerberosName kerberosName; try { @@ -99,9 +151,7 @@ private SaslServer createSaslServer() throws IOException { final String servicePrincipalName = kerberosName.serviceName(); final String serviceHostname = kerberosName.hostName(); - final String mech = "GSSAPI"; - - LOG.debug("Creating SaslServer for {} with mechanism {}", kerberosName, mech); + LOG.debug("Creating SaslServer for {} with mechanism {}", kerberosName, saslMechanism); // As described in http://docs.oracle.com/javase/8/docs/technotes/guides/security/jgss/jgss-features.html: // "To enable Java GSS to delegate to the native GSS library and its list of native mechanisms, @@ -127,7 +177,7 @@ private SaslServer createSaslServer() throws IOException { try { return Subject.doAs(subject, new PrivilegedExceptionAction() { public SaslServer run() throws SaslException { - return Sasl.createSaslServer(mech, servicePrincipalName, serviceHostname, null, saslServerCallbackHandler); + return Sasl.createSaslServer(saslMechanism, servicePrincipalName, serviceHostname, configs, saslServerCallbackHandler); } }); } catch (PrivilegedActionException e) { @@ -146,8 +196,8 @@ public void authenticate() throws IOException { if (netOutBuffer != null && !flushNetOutBufferAndUpdateInterestOps()) return; - if (saslServer.isComplete()) { - transportLayer.removeInterestOps(SelectionKey.OP_WRITE); + if (saslServer != null && saslServer.isComplete()) { + setSaslState(SaslState.COMPLETE); return; } @@ -161,12 +211,28 @@ public void authenticate() throws IOException { netInBuffer.payload().get(clientToken, 0, clientToken.length); netInBuffer = null; // reset the networkReceive as we read all the data. try { - byte[] response = saslServer.evaluateResponse(clientToken); - if (response != null) { - netOutBuffer = new NetworkSend(node, ByteBuffer.wrap(response)); - flushNetOutBufferAndUpdateInterestOps(); + switch (saslState) { + case HANDSHAKE_REQUEST: + if (handleKafkaRequest(clientToken)) + break; + // For default GSSAPI, fall through to authenticate using the client token as the first GSSAPI packet. + // This is required for interoperability with 0.9.0.x clients which do not send handshake request + case AUTHENTICATE: + byte[] response = saslServer.evaluateResponse(clientToken); + if (response != null) { + netOutBuffer = new NetworkSend(node, ByteBuffer.wrap(response)); + flushNetOutBufferAndUpdateInterestOps(); + } + // When the authentication exchange is complete and no more tokens are expected from the client, + // update SASL state. Current SASL state will be updated when outgoing writes to the client complete. + if (saslServer.isComplete()) + setSaslState(SaslState.COMPLETE); + break; + default: + break; } } catch (Exception e) { + setSaslState(SaslState.FAILED); throw new IOException(e); } } @@ -177,18 +243,33 @@ public Principal principal() { } public boolean complete() { - return saslServer.isComplete(); + return saslState == SaslState.COMPLETE; } public void close() throws IOException { - saslServer.dispose(); + if (saslServer != null) + saslServer.dispose(); + if (callbackHandler != null) + callbackHandler.close(); + } + + private void setSaslState(SaslState saslState) { + if (netOutBuffer != null && !netOutBuffer.completed()) + pendingSaslState = saslState; + else { + this.pendingSaslState = null; + this.saslState = saslState; + LOG.debug("Set SASL server state to {}", saslState); + } } private boolean flushNetOutBufferAndUpdateInterestOps() throws IOException { boolean flushedCompletely = flushNetOutBuffer(); - if (flushedCompletely) + if (flushedCompletely) { transportLayer.removeInterestOps(SelectionKey.OP_WRITE); - else + if (pendingSaslState != null) + setSaslState(pendingSaslState); + } else transportLayer.addInterestOps(SelectionKey.OP_WRITE); return flushedCompletely; } @@ -198,4 +279,66 @@ private boolean flushNetOutBuffer() throws IOException { netOutBuffer.writeTo(transportLayer); return netOutBuffer.completed(); } + + private boolean handleKafkaRequest(byte[] requestBytes) throws IOException, AuthenticationException { + boolean isKafkaRequest = false; + String clientMechanism = null; + try { + ByteBuffer requestBuffer = ByteBuffer.wrap(requestBytes); + RequestHeader requestHeader = RequestHeader.parse(requestBuffer); + AbstractRequest request = AbstractRequest.getRequest(requestHeader.apiKey(), requestHeader.apiVersion(), requestBuffer); + isKafkaRequest = true; + + ApiKeys apiKey = ApiKeys.forId(requestHeader.apiKey()); + switch (apiKey) { + case SASL_HANDSHAKE: + clientMechanism = handleHandshakeRequest(requestHeader, (SaslHandshakeRequest) request); + break; + default: + throw new IllegalSaslStateException("Unexpected Kafka request of type " + apiKey + " during SASL handshake."); + } + } catch (SchemaException | IllegalArgumentException e) { + // SchemaException is thrown if the request is not in Kafka format. IIlegalArgumentException is thrown + // if the API key is invalid. For compatibility with 0.9.0.x where the first packet is a GSSAPI token + // starting with 0x60, revert to GSSAPI for both these exceptions. + if (LOG.isDebugEnabled()) { + StringBuilder tokenBuilder = new StringBuilder(); + for (byte b : requestBytes) { + tokenBuilder.append(String.format("%02x", b)); + if (tokenBuilder.length() >= 20) + break; + } + LOG.debug("Received client packet of length {} starting with bytes 0x{}, process as GSSAPI packet", requestBytes.length, tokenBuilder); + } + if (enabledMechanisms.contains(SaslConfigs.GSSAPI_MECHANISM)) { + LOG.debug("First client packet is not a SASL mechanism request, using default mechanism GSSAPI"); + clientMechanism = SaslConfigs.GSSAPI_MECHANISM; + } else + throw new UnsupportedSaslMechanismException("Exception handling first SASL packet from client, GSSAPI is not supported by server", e); + } + if (clientMechanism != null) { + createSaslServer(clientMechanism); + setSaslState(SaslState.AUTHENTICATE); + } + return isKafkaRequest; + } + + private String handleHandshakeRequest(RequestHeader requestHeader, SaslHandshakeRequest handshakeRequest) throws IOException, UnsupportedSaslMechanismException { + String clientMechanism = handshakeRequest.mechanism(); + if (enabledMechanisms.contains(clientMechanism)) { + LOG.debug("Using SASL mechanism '{}' provided by client", clientMechanism); + sendKafkaResponse(requestHeader, new SaslHandshakeResponse((short) 0, enabledMechanisms)); + return clientMechanism; + } else { + LOG.debug("SASL mechanism '{}' requested by client is not supported", clientMechanism); + sendKafkaResponse(requestHeader, new SaslHandshakeResponse(Errors.UNSUPPORTED_SASL_MECHANISM.code(), enabledMechanisms)); + throw new UnsupportedSaslMechanismException("Unsupported SASL mechanism " + clientMechanism); + } + } + + private void sendKafkaResponse(RequestHeader requestHeader, AbstractRequestResponse response) throws IOException { + ResponseHeader responseHeader = new ResponseHeader(requestHeader.correlationId()); + netOutBuffer = new NetworkSend(node, ResponseSend.serialize(responseHeader, response.toStruct())); + flushNetOutBufferAndUpdateInterestOps(); + } } diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerCallbackHandler.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerCallbackHandler.java index 1de4a2e085c97..c23e390241ef1 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerCallbackHandler.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerCallbackHandler.java @@ -19,12 +19,15 @@ package org.apache.kafka.common.security.authenticator; import java.io.IOException; +import java.util.Map; +import org.apache.kafka.common.security.auth.AuthCallbackHandler; import org.apache.kafka.common.security.kerberos.KerberosShortNamer; import org.slf4j.Logger; import org.slf4j.LoggerFactory; + +import javax.security.auth.Subject; import javax.security.auth.callback.Callback; -import javax.security.auth.callback.CallbackHandler; import javax.security.auth.callback.UnsupportedCallbackException; import javax.security.auth.login.AppConfigurationEntry; import javax.security.auth.login.Configuration; @@ -32,9 +35,16 @@ import javax.security.sasl.RealmCallback; import org.apache.kafka.common.security.kerberos.KerberosName; +import org.apache.kafka.common.network.Mode; import org.apache.kafka.common.security.JaasUtils; -public class SaslServerCallbackHandler implements CallbackHandler { +/** + * Callback handler for Sasl servers. The callbacks required for all the SASL + * mechanisms enabled in the server should be supported by this callback handler. See + * Java SASL API + * for the list of SASL callback handlers required for each SASL mechanism. + */ +public class SaslServerCallbackHandler implements AuthCallbackHandler { private static final Logger LOG = LoggerFactory.getLogger(SaslServerCallbackHandler.class); private final KerberosShortNamer kerberosShortNamer; @@ -45,6 +55,11 @@ public SaslServerCallbackHandler(Configuration configuration, KerberosShortNamer this.kerberosShortNamer = kerberosNameParser; } + @Override + public void configure(Map configs, Mode mode, Subject subject, String saslMechanism) { + } + + @Override public void handle(Callback[] callbacks) throws UnsupportedCallbackException { for (Callback callback : callbacks) { if (callback instanceof RealmCallback) { @@ -78,4 +93,7 @@ private void handleAuthorizeCallback(AuthorizeCallback ac) { } } + @Override + public void close() { + } } diff --git a/clients/src/main/java/org/apache/kafka/common/security/kerberos/Login.java b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java similarity index 86% rename from clients/src/main/java/org/apache/kafka/common/security/kerberos/Login.java rename to clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java index 2e1a056a80ff9..58becdf67fd77 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/kerberos/Login.java +++ b/clients/src/main/java/org/apache/kafka/common/security/kerberos/KerberosLogin.java @@ -23,21 +23,21 @@ import javax.security.auth.login.Configuration; import javax.security.auth.login.LoginContext; import javax.security.auth.login.LoginException; -import javax.security.auth.callback.CallbackHandler; import javax.security.auth.kerberos.KerberosTicket; import javax.security.auth.Subject; -import org.apache.kafka.common.security.authenticator.SaslClientAuthenticator.ClientCallbackHandler; +import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.security.JaasUtils; +import org.apache.kafka.common.security.authenticator.AbstractLogin; import org.apache.kafka.common.config.SaslConfigs; import org.apache.kafka.common.utils.Shell; import org.apache.kafka.common.utils.Time; import org.apache.kafka.common.utils.SystemTime; - import org.apache.kafka.common.utils.Utils; import org.slf4j.Logger; import org.slf4j.LoggerFactory; +import java.io.IOException; import java.util.Date; import java.util.Random; import java.util.Set; @@ -47,40 +47,40 @@ * This class is responsible for refreshing Kerberos credentials for * logins for both Kafka client and server. */ -public class Login { - private static final Logger log = LoggerFactory.getLogger(Login.class); +public class KerberosLogin extends AbstractLogin { + private static final Logger log = LoggerFactory.getLogger(KerberosLogin.class); private static final Random RNG = new Random(); - private final Thread t; - private final boolean isKrbTicket; - private final boolean isUsingTicketCache; - - private final String loginContextName; - private final String principal; private final Time time = new SystemTime(); - private final CallbackHandler callbackHandler = new ClientCallbackHandler(); + private Thread t; + private boolean isKrbTicket; + private boolean isUsingTicketCache; + + private String loginContextName; + private String principal; // LoginThread will sleep until 80% of time from last refresh to // ticket's expiry has been reached, at which time it will wake // and try to renew the ticket. - private final double ticketRenewWindowFactor; + private double ticketRenewWindowFactor; /** * Percentage of random jitter added to the renewal time */ - private final double ticketRenewJitter; + private double ticketRenewJitter; // Regardless of ticketRenewWindowFactor setting above and the ticket expiry time, // thread will not sleep between refresh attempts any less than 1 minute (60*1000 milliseconds = 1 minute). // Change the '1' to e.g. 5, to change this to 5 minutes. - private final long minTimeBeforeRelogin; + private long minTimeBeforeRelogin; - private final String kinitCmd; + private String kinitCmd; private volatile Subject subject; - private LoginContext login; + private LoginContext loginContext; + private String serviceName; private long lastLogin; /** @@ -93,16 +93,22 @@ public class Login { * @throws javax.security.auth.login.LoginException * Thrown if authentication fails. */ - public Login(final String loginContextName, Map configs) throws LoginException { + public void configure(Map configs, final String loginContextName) { + super.configure(configs, loginContextName); this.loginContextName = loginContextName; this.ticketRenewWindowFactor = (Double) configs.get(SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR); this.ticketRenewJitter = (Double) configs.get(SaslConfigs.SASL_KERBEROS_TICKET_RENEW_JITTER); this.minTimeBeforeRelogin = (Long) configs.get(SaslConfigs.SASL_KERBEROS_MIN_TIME_BEFORE_RELOGIN); this.kinitCmd = (String) configs.get(SaslConfigs.SASL_KERBEROS_KINIT_CMD); + this.serviceName = getServiceName(configs, loginContextName); + } + + @Override + public LoginContext login() throws LoginException { this.lastLogin = currentElapsedTime(); - login = login(loginContextName); - subject = login.getSubject(); + loginContext = super.login(); + subject = loginContext.getSubject(); isKrbTicket = !subject.getPrivateCredentials(KerberosTicket.class).isEmpty(); AppConfigurationEntry[] entries = Configuration.getConfiguration().getAppConfigurationEntry(loginContextName); @@ -127,7 +133,7 @@ public Login(final String loginContextName, Map configs) throws Login log.debug("It is not a Kerberos ticket"); t = null; // if no TGT, do not bother with ticket management. - return; + return loginContext; } log.debug("It is a Kerberos ticket"); @@ -259,16 +265,12 @@ public void run() { } } }, true); + t.start(); + return loginContext; } - public void startThreadIfNeeded() { - // thread object 't' will be null if a refresh thread is not needed. - if (t != null) { - t.start(); - } - } - - public void shutdown() { + @Override + public void close() { if ((t != null) && (t.isAlive())) { t.interrupt(); try { @@ -279,28 +281,39 @@ public void shutdown() { } } + @Override public Subject subject() { return subject; } - private synchronized LoginContext login(final String loginContextName) throws LoginException { - String jaasConfigFile = System.getProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM); - if (jaasConfigFile == null) { - log.debug("System property '" + JaasUtils.JAVA_LOGIN_CONFIG_PARAM + "' is not set, using default JAAS configuration."); + @Override + public String serviceName() { + return serviceName; + } + + private String getServiceName(Map configs, String loginContext) { + String jaasServiceName; + try { + jaasServiceName = JaasUtils.jaasConfig(loginContext, JaasUtils.SERVICE_NAME); + } catch (IOException e) { + throw new KafkaException("Jaas configuration not found", e); } - AppConfigurationEntry[] configEntries = Configuration.getConfiguration().getAppConfigurationEntry(loginContextName); - if (configEntries == null) { - String errorMessage = "Could not find a '" + loginContextName + "' entry in the JAAS configuration. System property '" + - JaasUtils.JAVA_LOGIN_CONFIG_PARAM + "' is " + (jaasConfigFile == null ? "not set" : jaasConfigFile); - throw new IllegalArgumentException(errorMessage); + String configServiceName = (String) configs.get(SaslConfigs.SASL_KERBEROS_SERVICE_NAME); + if (jaasServiceName != null && configServiceName != null && !jaasServiceName.equals(configServiceName)) { + String message = "Conflicting serviceName values found in JAAS and Kafka configs " + + "value in JAAS file " + jaasServiceName + ", value in Kafka config " + configServiceName; + throw new IllegalArgumentException(message); } - LoginContext loginContext = new LoginContext(loginContextName, callbackHandler); - loginContext.login(); - log.info("Successfully logged in."); - return loginContext; + if (jaasServiceName != null) + return jaasServiceName; + if (configServiceName != null) + return configServiceName; + + throw new IllegalArgumentException("No serviceName defined in either JAAS or Kafka config"); } + private long getRefreshTime(KerberosTicket tgt) { long start = tgt.getStartTime().getTime(); long expires = tgt.getEndTime().getTime(); @@ -346,25 +359,25 @@ private synchronized void reLogin() throws LoginException { if (!isKrbTicket) { return; } - if (login == null) { + if (loginContext == null) { throw new LoginException("Login must be done first"); } if (!hasSufficientTimeElapsed()) { return; } log.info("Initiating logout for {}", principal); - synchronized (Login.class) { + synchronized (KerberosLogin.class) { // register most recent relogin attempt lastLogin = currentElapsedTime(); //clear up the kerberos state. But the tokens are not cleared! As per //the Java kerberos login module code, only the kerberos credentials //are cleared - login.logout(); + loginContext.logout(); //login and also update the subject field of this instance to //have the new credentials (pass it to the LoginContext constructor) - login = new LoginContext(loginContextName, subject); + loginContext = new LoginContext(loginContextName, subject); log.info("Initiating re-login for {}", principal); - login.login(); + loginContext.login(); } } diff --git a/clients/src/main/java/org/apache/kafka/common/security/plain/PlainLoginModule.java b/clients/src/main/java/org/apache/kafka/common/security/plain/PlainLoginModule.java new file mode 100644 index 0000000000000..f06fbf62fb387 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/plain/PlainLoginModule.java @@ -0,0 +1,66 @@ +/** + * 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 org.apache.kafka.common.security.plain; + +import java.util.Map; + +import javax.security.auth.Subject; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.login.LoginException; +import javax.security.auth.spi.LoginModule; + +public class PlainLoginModule implements LoginModule { + + private static final String USERNAME_CONFIG = "username"; + private static final String PASSWORD_CONFIG = "password"; + + static { + PlainSaslServerProvider.initialize(); + } + + @Override + public void initialize(Subject subject, CallbackHandler callbackHandler, Map sharedState, Map options) { + String username = (String) options.get(USERNAME_CONFIG); + if (username != null) + subject.getPublicCredentials().add(username); + String password = (String) options.get(PASSWORD_CONFIG); + if (password != null) + subject.getPrivateCredentials().add(password); + } + + @Override + public boolean login() throws LoginException { + return true; + } + + @Override + public boolean logout() throws LoginException { + return true; + } + + @Override + public boolean commit() throws LoginException { + return true; + } + + @Override + public boolean abort() throws LoginException { + return false; + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServer.java b/clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServer.java new file mode 100644 index 0000000000000..5c6fd78c08953 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServer.java @@ -0,0 +1,170 @@ +/** + * 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 org.apache.kafka.common.security.plain; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.util.Arrays; +import java.util.Map; + +import javax.security.auth.callback.CallbackHandler; +import javax.security.sasl.Sasl; +import javax.security.sasl.SaslException; +import javax.security.sasl.SaslServer; +import javax.security.sasl.SaslServerFactory; + +import org.apache.kafka.common.network.LoginType; +import org.apache.kafka.common.security.JaasUtils; + +/** + * Simple SaslServer implementation for SASL/PLAIN. In order to make this implementation + * fully pluggable, authentication of username/password is fully contained within the + * server implementation. + *

          + * Valid users with passwords are specified in the Jaas configuration file. Each user + * is specified with user_ as key and as value. This is consistent + * with Zookeeper Digest-MD5 implementation. + *

          + * To avoid storing clear passwords on disk or to integrate with external authentication + * servers in production systems, this module can be replaced with a different implementation. + * + */ +public class PlainSaslServer implements SaslServer { + + public static final String PLAIN_MECHANISM = "PLAIN"; + private static final String JAAS_USER_PREFIX = "user_"; + + private boolean complete; + private String authorizationID; + + public PlainSaslServer(CallbackHandler callbackHandler) { + } + + @Override + public byte[] evaluateResponse(byte[] response) throws SaslException { + /* + * Message format (from https://tools.ietf.org/html/rfc4616): + * + * message = [authzid] UTF8NUL authcid UTF8NUL passwd + * authcid = 1*SAFE ; MUST accept up to 255 octets + * authzid = 1*SAFE ; MUST accept up to 255 octets + * passwd = 1*SAFE ; MUST accept up to 255 octets + * UTF8NUL = %x00 ; UTF-8 encoded NUL character + * + * SAFE = UTF1 / UTF2 / UTF3 / UTF4 + * ;; any UTF-8 encoded Unicode character except NUL + */ + + String[] tokens; + try { + tokens = new String(response, "UTF-8").split("\u0000"); + } catch (UnsupportedEncodingException e) { + throw new SaslException("UTF-8 encoding not supported", e); + } + if (tokens.length != 3) + throw new SaslException("Invalid SASL/PLAIN response: expected 3 tokens, got " + tokens.length); + authorizationID = tokens[0]; + String username = tokens[1]; + String password = tokens[2]; + + if (username.isEmpty()) { + throw new SaslException("Authentication failed: username not specified"); + } + if (password.isEmpty()) { + throw new SaslException("Authentication failed: password not specified"); + } + if (authorizationID.isEmpty()) + authorizationID = username; + + try { + String expectedPassword = JaasUtils.jaasConfig(LoginType.SERVER.contextName(), JAAS_USER_PREFIX + username); + if (!password.equals(expectedPassword)) { + throw new SaslException("Authentication failed: Invalid username or password"); + } + } catch (IOException e) { + throw new SaslException("Authentication failed: Invalid JAAS configuration", e); + } + complete = true; + return new byte[0]; + } + + @Override + public String getAuthorizationID() { + if (!complete) + throw new IllegalStateException("Authentication exchange has not completed"); + return authorizationID; + } + + @Override + public String getMechanismName() { + return PLAIN_MECHANISM; + } + + @Override + public Object getNegotiatedProperty(String propName) { + if (!complete) + throw new IllegalStateException("Authentication exchange has not completed"); + return null; + } + + @Override + public boolean isComplete() { + return complete; + } + + @Override + public byte[] unwrap(byte[] incoming, int offset, int len) throws SaslException { + if (!complete) + throw new IllegalStateException("Authentication exchange has not completed"); + return Arrays.copyOfRange(incoming, offset, offset + len); + } + + @Override + public byte[] wrap(byte[] outgoing, int offset, int len) throws SaslException { + if (!complete) + throw new IllegalStateException("Authentication exchange has not completed"); + return Arrays.copyOfRange(outgoing, offset, offset + len); + } + + @Override + public void dispose() throws SaslException { + } + + public static class PlainSaslServerFactory implements SaslServerFactory { + + @Override + public SaslServer createSaslServer(String mechanism, String protocol, String serverName, Map props, CallbackHandler cbh) + throws SaslException { + + if (!PLAIN_MECHANISM.equals(mechanism)) { + throw new SaslException(String.format("Mechanism \'%s\' is not supported. Only PLAIN is supported.", mechanism)); + } + return new PlainSaslServer(cbh); + } + + @Override + public String[] getMechanismNames(Map props) { + String noPlainText = (String) props.get(Sasl.POLICY_NOPLAINTEXT); + if ("true".equals(noPlainText)) + return new String[]{}; + else + return new String[]{PLAIN_MECHANISM}; + } + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServerProvider.java b/clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServerProvider.java new file mode 100644 index 0000000000000..c3db1f577411b --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/security/plain/PlainSaslServerProvider.java @@ -0,0 +1,38 @@ +/** + * 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 org.apache.kafka.common.security.plain; + +import java.security.Provider; +import java.security.Security; + +import org.apache.kafka.common.security.plain.PlainSaslServer.PlainSaslServerFactory; + +public class PlainSaslServerProvider extends Provider { + + private static final long serialVersionUID = 1L; + + protected PlainSaslServerProvider() { + super("Simple SASL/PLAIN Server Provider", 1.0, "Simple SASL/PLAIN Server Provider for Kafka"); + super.put("SaslServerFactory." + PlainSaslServer.PLAIN_MECHANISM, PlainSaslServerFactory.class.getName()); + } + + public static void initialize() { + Security.addProvider(new PlainSaslServerProvider()); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 9def5577a5bc3..92f3101e78468 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -91,7 +91,10 @@ public void testSerialization() throws Exception { createUpdateMetadataResponse(), createLeaderAndIsrRequest(), createLeaderAndIsrRequest().getErrorResponse(0, new UnknownServerException()), - createLeaderAndIsrResponse() + createLeaderAndIsrResponse(), + createSaslHandshakeRequest(), + createSaslHandshakeRequest().getErrorResponse(0, new UnknownServerException()), + createSaslHandshakeResponse() ); for (AbstractRequestResponse req : requestResponseList) @@ -425,5 +428,11 @@ private AbstractRequestResponse createUpdateMetadataResponse() { return new UpdateMetadataResponse(Errors.NONE.code()); } + private AbstractRequest createSaslHandshakeRequest() { + return new SaslHandshakeRequest("PLAIN"); + } + private AbstractRequestResponse createSaslHandshakeResponse() { + return new SaslHandshakeResponse(Errors.NONE.code(), Collections.singletonList("GSSAPI")); + } } diff --git a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java index fbe2630e4c28f..91e921fcee359 100644 --- a/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java +++ b/clients/src/test/java/org/apache/kafka/test/TestSslUtils.java @@ -210,6 +210,7 @@ public static Map createSslConfig(boolean useClientCert, boolea X509Certificate cCert = generateCertificate("CN=" + host + ", O=A client", cKP, 30, "SHA1withRSA"); createKeyStore(keyStoreFile.getPath(), password, "client", cKP.getPrivate(), cCert); certs.put(certAlias, cCert); + keyStoreFile.deleteOnExit(); } else if (mode == Mode.SERVER) { keyStoreFile = File.createTempFile("serverKS", ".jks"); KeyPair sKP = generateKeyPair("RSA"); @@ -217,10 +218,12 @@ public static Map createSslConfig(boolean useClientCert, boolea "SHA1withRSA"); createKeyStore(keyStoreFile.getPath(), password, password, "server", sKP.getPrivate(), sCert); certs.put(certAlias, sCert); + keyStoreFile.deleteOnExit(); } if (trustStore) { createTrustStore(trustStoreFile.getPath(), trustStorePassword, certs); + trustStoreFile.deleteOnExit(); } return createSslConfig(mode, keyStoreFile, password, password, trustStoreFile, trustStorePassword); diff --git a/core/src/main/scala/kafka/common/ErrorMapping.scala b/core/src/main/scala/kafka/common/ErrorMapping.scala index 9708c4e5394ec..91a1d757559c8 100644 --- a/core/src/main/scala/kafka/common/ErrorMapping.scala +++ b/core/src/main/scala/kafka/common/ErrorMapping.scala @@ -63,6 +63,8 @@ object ErrorMapping { val GroupAuthorizationCode: Short = 30 val ClusterAuthorizationCode: Short = 31 // 32: INVALID_TIMESTAMP + // 33: UNSUPPORTED_SASL_MECHANISM + // 34: ILLEGAL_SASL_STATE private val exceptionToCode = Map[Class[Throwable], Short]( diff --git a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala index e9731fd4e091c..018946ed9b4f1 100755 --- a/core/src/main/scala/kafka/controller/ControllerChannelManager.scala +++ b/core/src/main/scala/kafka/controller/ControllerChannelManager.scala @@ -89,6 +89,14 @@ class ControllerChannelManager(controllerContext: ControllerContext, config: Kaf val brokerEndPoint = broker.getBrokerEndPoint(config.interBrokerSecurityProtocol) val brokerNode = new Node(broker.id, brokerEndPoint.host, brokerEndPoint.port) val networkClient = { + val channelBuilder = ChannelBuilders.create( + config.interBrokerSecurityProtocol, + Mode.CLIENT, + LoginType.SERVER, + config.values, + config.saslMechanismInterBrokerProtocol, + config.saslInterBrokerHandshakeRequestEnable + ) val selector = new Selector( NetworkReceive.UNLIMITED, config.connectionsMaxIdleMs, @@ -97,7 +105,7 @@ class ControllerChannelManager(controllerContext: ControllerContext, config: Kaf "controller-channel", Map("broker-id" -> broker.id.toString).asJava, false, - ChannelBuilders.create(config.interBrokerSecurityProtocol, Mode.CLIENT, LoginType.SERVER, config.values) + channelBuilder ) new NetworkClient( selector, diff --git a/core/src/main/scala/kafka/network/SocketServer.scala b/core/src/main/scala/kafka/network/SocketServer.scala index f1ec2ef6c4515..b757abd6ec7ad 100644 --- a/core/src/main/scala/kafka/network/SocketServer.scala +++ b/core/src/main/scala/kafka/network/SocketServer.scala @@ -399,7 +399,7 @@ private[kafka] class Processor(val id: Int, "socket-server", metricTags, false, - ChannelBuilders.create(protocol, Mode.SERVER, LoginType.SERVER, channelConfigs)) + ChannelBuilders.create(protocol, Mode.SERVER, LoginType.SERVER, channelConfigs, null, true)) override def run() { startupComplete() diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 4f77d30273427..9afefa5c6de2a 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -49,6 +49,7 @@ import org.apache.kafka.common.internals.TopicConstants import scala.collection._ import scala.collection.JavaConverters._ +import org.apache.kafka.common.requests.SaslHandshakeResponse /** * Logic to handle the various Kafka requests @@ -93,6 +94,7 @@ class KafkaApis(val requestChannel: RequestChannel, case ApiKeys.SYNC_GROUP => handleSyncGroupRequest(request) case ApiKeys.DESCRIBE_GROUPS => handleDescribeGroupRequest(request) case ApiKeys.LIST_GROUPS => handleListGroupsRequest(request) + case ApiKeys.SASL_HANDSHAKE => handleSaslHandshakeRequest(request) case requestId => throw new KafkaException("Unknown api code " + requestId) } } catch { @@ -993,6 +995,12 @@ class KafkaApis(val requestChannel: RequestChannel, } } + def handleSaslHandshakeRequest(request: RequestChannel.Request) { + val respHeader = new ResponseHeader(request.header.correlationId) + val response = new SaslHandshakeResponse(Errors.ILLEGAL_SASL_STATE.code, config.saslEnabledMechanisms) + requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, respHeader, response))) + } + def close() { quotaManagers.foreach { case (apiKey, quotaManager) => quotaManager.shutdown() diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 5c0d27a229981..5e28bd7d16c07 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -19,7 +19,7 @@ package kafka.server import java.util.Properties -import kafka.api.ApiVersion +import kafka.api.{ApiVersion, KAFKA_0_10_0_IV0} import kafka.cluster.EndPoint import kafka.consumer.ConsumerConfig import kafka.coordinator.OffsetConfig @@ -175,6 +175,8 @@ object Defaults { val SslClientAuth = SslClientAuthNone /** ********* Sasl configuration ***********/ + val SaslMechanismInterBrokerProtocol = SaslConfigs.DEFAULT_SASL_MECHANISM + val SaslEnabledMechanisms = SaslConfigs.DEFAULT_SASL_ENABLED_MECHANISMS val SaslKerberosKinitCmd = SaslConfigs.DEFAULT_KERBEROS_KINIT_CMD val SaslKerberosTicketRenewWindowFactor = SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_WINDOW_FACTOR val SaslKerberosTicketRenewJitter = SaslConfigs.DEFAULT_KERBEROS_TICKET_RENEW_JITTER @@ -337,6 +339,8 @@ object KafkaConfig { val SslClientAuthProp = SslConfigs.SSL_CLIENT_AUTH_CONFIG /** ********* SASL Configuration ****************/ + val SaslMechanismInterBrokerProtocolProp = "sasl.mechanism.inter.broker.protocol" + val SaslEnabledMechanismsProp = SaslConfigs.SASL_ENABLED_MECHANISMS val SaslKerberosServiceNameProp = SaslConfigs.SASL_KERBEROS_SERVICE_NAME val SaslKerberosKinitCmdProp = SaslConfigs.SASL_KERBEROS_KINIT_CMD val SaslKerberosTicketRenewWindowFactorProp = SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR @@ -533,6 +537,8 @@ object KafkaConfig { val SslClientAuthDoc = SslConfigs.SSL_CLIENT_AUTH_DOC /** ********* Sasl Configuration ****************/ + val SaslMechanismInterBrokerProtocolDoc = "SASL mechanism used for inter-broker communication. Default is GSSAPI." + val SaslEnabledMechanismsDoc = SaslConfigs.SASL_ENABLED_MECHANISMS_DOC val SaslKerberosServiceNameDoc = SaslConfigs.SASL_KERBEROS_SERVICE_NAME_DOC val SaslKerberosKinitCmdDoc = SaslConfigs.SASL_KERBEROS_KINIT_CMD_DOC val SaslKerberosTicketRenewWindowFactorDoc = SaslConfigs.SASL_KERBEROS_TICKET_RENEW_WINDOW_FACTOR_DOC @@ -704,6 +710,8 @@ object KafkaConfig { .define(SslCipherSuitesProp, LIST, null, MEDIUM, SslCipherSuitesDoc) /** ********* Sasl Configuration ****************/ + .define(SaslMechanismInterBrokerProtocolProp, STRING, Defaults.SaslMechanismInterBrokerProtocol, MEDIUM, SaslMechanismInterBrokerProtocolDoc) + .define(SaslEnabledMechanismsProp, LIST, Defaults.SaslEnabledMechanisms, MEDIUM, SaslEnabledMechanismsDoc) .define(SaslKerberosServiceNameProp, STRING, null, MEDIUM, SaslKerberosServiceNameDoc) .define(SaslKerberosKinitCmdProp, STRING, Defaults.SaslKerberosKinitCmd, MEDIUM, SaslKerberosKinitCmdDoc) .define(SaslKerberosTicketRenewWindowFactorProp, DOUBLE, Defaults.SaslKerberosTicketRenewWindowFactor, MEDIUM, SaslKerberosTicketRenewWindowFactorDoc) @@ -894,12 +902,15 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean) extends Abstra val sslCipher = getList(KafkaConfig.SslCipherSuitesProp) /** ********* Sasl Configuration **************/ + val saslMechanismInterBrokerProtocol = getString(KafkaConfig.SaslMechanismInterBrokerProtocolProp) + val saslEnabledMechanisms = getList(KafkaConfig.SaslEnabledMechanismsProp) val saslKerberosServiceName = getString(KafkaConfig.SaslKerberosServiceNameProp) val saslKerberosKinitCmd = getString(KafkaConfig.SaslKerberosKinitCmdProp) val saslKerberosTicketRenewWindowFactor = getDouble(KafkaConfig.SaslKerberosTicketRenewWindowFactorProp) val saslKerberosTicketRenewJitter = getDouble(KafkaConfig.SaslKerberosTicketRenewJitterProp) val saslKerberosMinTimeBeforeRelogin = getLong(KafkaConfig.SaslKerberosMinTimeBeforeReloginProp) val saslKerberosPrincipalToLocalRules = getList(KafkaConfig.SaslKerberosPrincipalToLocalRulesProp) + val saslInterBrokerHandshakeRequestEnable = interBrokerProtocolVersion >= KAFKA_0_10_0_IV0 /** ********* Quota Configuration **************/ val producerQuotaBytesPerSecondDefault = getLong(KafkaConfig.ProducerQuotaBytesPerSecondDefaultProp) @@ -1009,5 +1020,10 @@ class KafkaConfig(val props: java.util.Map[_, _], doLog: Boolean) extends Abstra ) require(interBrokerProtocolVersion >= logMessageFormatVersion, s"log.message.format.version $logMessageFormatVersionString cannot be used when inter.broker.protocol.version is set to $interBrokerProtocolVersionString") + val interBrokerUsesSasl = interBrokerSecurityProtocol == SecurityProtocol.SASL_PLAINTEXT || interBrokerSecurityProtocol == SecurityProtocol.SASL_SSL + require(!interBrokerUsesSasl || saslInterBrokerHandshakeRequestEnable || saslMechanismInterBrokerProtocol == SaslConfigs.GSSAPI_MECHANISM, + s"Only GSSAPI mechanism is supported for inter-broker communication with SASL when inter.broker.protocol.version is set to $interBrokerProtocolVersionString") + require(!interBrokerUsesSasl || saslEnabledMechanisms.contains(saslMechanismInterBrokerProtocol), + s"${KafkaConfig.SaslMechanismInterBrokerProtocolProp} must be included in ${KafkaConfig.SaslEnabledMechanismsProp} when SASL is used for inter-broker communication") } } diff --git a/core/src/main/scala/kafka/server/KafkaServer.scala b/core/src/main/scala/kafka/server/KafkaServer.scala index f998d82104d36..36b52fdc4b798 100755 --- a/core/src/main/scala/kafka/server/KafkaServer.scala +++ b/core/src/main/scala/kafka/server/KafkaServer.scala @@ -323,6 +323,13 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr def networkClientControlledShutdown(retries: Int): Boolean = { val metadataUpdater = new ManualMetadataUpdater() val networkClient = { + val channelBuilder = ChannelBuilders.create( + config.interBrokerSecurityProtocol, + Mode.CLIENT, + LoginType.SERVER, + config.values, + config.saslMechanismInterBrokerProtocol, + config.saslInterBrokerHandshakeRequestEnable) val selector = new Selector( NetworkReceive.UNLIMITED, config.connectionsMaxIdleMs, @@ -331,7 +338,7 @@ class KafkaServer(val config: KafkaConfig, time: Time = SystemTime, threadNamePr "kafka-server-controlled-shutdown", Map.empty.asJava, false, - ChannelBuilders.create(config.interBrokerSecurityProtocol, Mode.CLIENT, LoginType.SERVER, config.values) + channelBuilder ) new NetworkClient( selector, diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 26838cac96dbe..84f2e123f9c9f 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -73,6 +73,14 @@ class ReplicaFetcherThread(name: String, // as the metrics tag to avoid metric name conflicts with // more than one fetcher thread to the same broker private val networkClient = { + val channelBuilder = ChannelBuilders.create( + brokerConfig.interBrokerSecurityProtocol, + Mode.CLIENT, + LoginType.SERVER, + brokerConfig.values, + brokerConfig.saslMechanismInterBrokerProtocol, + brokerConfig.saslInterBrokerHandshakeRequestEnable + ) val selector = new Selector( NetworkReceive.UNLIMITED, brokerConfig.connectionsMaxIdleMs, @@ -81,7 +89,7 @@ class ReplicaFetcherThread(name: String, "replica-fetcher", Map("broker-id" -> sourceBroker.id.toString, "fetcher-id" -> fetcherId.toString).asJava, false, - ChannelBuilders.create(brokerConfig.interBrokerSecurityProtocol, Mode.CLIENT, LoginType.SERVER, brokerConfig.values) + channelBuilder ) new NetworkClient( selector, diff --git a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala index 56dae7617973e..23fcfa61570a1 100644 --- a/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseConsumerTest.scala @@ -26,6 +26,7 @@ import org.junit.{Before, Test} import scala.collection.JavaConverters._ import scala.collection.mutable.Buffer import org.apache.kafka.common.internals.TopicConstants +import org.apache.kafka.clients.producer.KafkaProducer /** * Integration tests for the new consumer that cover basic usage as well as server failures @@ -268,10 +269,14 @@ abstract class BaseConsumerTest extends IntegrationTestHarness with Logging { } protected def sendRecords(numRecords: Int, tp: TopicPartition) { + sendRecords(this.producers(0), numRecords, tp) + } + + protected def sendRecords(producer: KafkaProducer[Array[Byte], Array[Byte]], numRecords: Int, tp: TopicPartition) { (0 until numRecords).foreach { i => - this.producers(0).send(new ProducerRecord(tp.topic(), tp.partition(), i.toLong, s"key $i".getBytes, s"value $i".getBytes)) + producer.send(new ProducerRecord(tp.topic(), tp.partition(), i.toLong, s"key $i".getBytes, s"value $i".getBytes)) } - this.producers(0).flush() + producer.flush() } protected def consumeAndVerifyRecords(consumer: Consumer[Array[Byte], Array[Byte]], diff --git a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala index 49ce748bac1d1..15eeb63dd4c81 100644 --- a/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala +++ b/core/src/test/scala/integration/kafka/api/BaseProducerSendTest.scala @@ -41,7 +41,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { val numServers = 2 overridingProps.put(KafkaConfig.NumPartitionsProp, 4.toString) TestUtils.createBrokerConfigs(numServers, zkConnect, false, interBrokerSecurityProtocol = Some(securityProtocol), - trustStoreFile = trustStoreFile).map(KafkaConfig.fromProps(_, overridingProps)) + trustStoreFile = trustStoreFile, saslProperties = saslProperties).map(KafkaConfig.fromProps(_, overridingProps)) } private var consumer1: SimpleConsumer = null @@ -72,7 +72,7 @@ abstract class BaseProducerSendTest extends KafkaServerTestHarness { private def createProducer(brokerList: String, retries: Int = 0, lingerMs: Long = 0, props: Option[Properties] = None): KafkaProducer[Array[Byte],Array[Byte]] = { val producer = TestUtils.createNewProducer(brokerList, securityProtocol = securityProtocol, trustStoreFile = trustStoreFile, - retries = retries, lingerMs = lingerMs, props = props) + saslProperties = saslProperties, retries = retries, lingerMs = lingerMs, props = props) producers += producer producer } diff --git a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala index e2314b391f5c7..870caca2b9b7c 100644 --- a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala @@ -141,9 +141,9 @@ trait EndToEndAuthorizationTest extends IntegrationTestHarness with SaslSetup { override def setUp { securityProtocol match { case SecurityProtocol.SSL => - startSasl(ZkSasl) + startSasl(ZkSasl, null, null) case _ => - startSasl(Both) + startSasl(Both, List("GSSAPI"), List("GSSAPI")) } super.setUp AclCommand.main(topicBrokerReadAclArgs) diff --git a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala index d0680b8309bc6..de05c9c6ea4b5 100644 --- a/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/IntegrationTestHarness.scala @@ -46,15 +46,15 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { override def generateConfigs() = { val cfgs = TestUtils.createBrokerConfigs(serverCount, zkConnect, interBrokerSecurityProtocol = Some(securityProtocol), - trustStoreFile = trustStoreFile) + trustStoreFile = trustStoreFile, saslProperties = saslProperties) cfgs.foreach(_.putAll(serverConfig)) cfgs.map(KafkaConfig.fromProps) } @Before override def setUp() { - val producerSecurityProps = TestUtils.producerSecurityConfigs(securityProtocol, trustStoreFile) - val consumerSecurityProps = TestUtils.consumerSecurityConfigs(securityProtocol, trustStoreFile) + val producerSecurityProps = TestUtils.producerSecurityConfigs(securityProtocol, trustStoreFile, saslProperties) + val consumerSecurityProps = TestUtils.consumerSecurityConfigs(securityProtocol, trustStoreFile, saslProperties) super.setUp() producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArraySerializer]) producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, classOf[org.apache.kafka.common.serialization.ByteArraySerializer]) @@ -66,11 +66,13 @@ trait IntegrationTestHarness extends KafkaServerTestHarness { producers += TestUtils.createNewProducer(brokerList, securityProtocol = this.securityProtocol, trustStoreFile = this.trustStoreFile, + saslProperties = this.saslProperties, props = Some(producerConfig)) for (i <- 0 until consumerCount) { consumers += TestUtils.createNewConsumer(brokerList, securityProtocol = this.securityProtocol, trustStoreFile = this.trustStoreFile, + saslProperties = this.saslProperties, props = Some(consumerConfig)) } diff --git a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala index 8dbb80be531c5..b22ccde9ed313 100644 --- a/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/PlaintextConsumerTest.scala @@ -355,7 +355,7 @@ class PlaintextConsumerTest extends BaseConsumerTest { producerProps.setProperty(ProducerConfig.COMPRESSION_TYPE_CONFIG, CompressionType.GZIP.name) producerProps.setProperty(ProducerConfig.LINGER_MS_CONFIG, Long.MaxValue.toString) val producer = TestUtils.createNewProducer(brokerList, securityProtocol = securityProtocol, trustStoreFile = trustStoreFile, - retries = 0, lingerMs = Long.MaxValue, props = Some(producerProps)) + saslProperties = saslProperties, retries = 0, lingerMs = Long.MaxValue, props = Some(producerProps)) (0 until numRecords).foreach { i => producer.send(new ProducerRecord(tp.topic, tp.partition, i.toLong, s"key $i".getBytes, s"value $i".getBytes)) } diff --git a/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala new file mode 100644 index 0000000000000..d203245ace72e --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala @@ -0,0 +1,86 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package kafka.api + +import java.io.File +import org.apache.kafka.common.protocol.SecurityProtocol +import kafka.server.KafkaConfig +import org.junit.Test +import kafka.utils.TestUtils +import scala.collection.JavaConverters._ + +class SaslMultiMechanismConsumerTest extends BaseConsumerTest with SaslTestHarness { + override protected val zkSaslEnabled = true + override protected val kafkaClientSaslMechanism = "PLAIN" + override protected val kafkaServerSaslMechanisms = List("GSSAPI", "PLAIN") + override protected def allKafkaClientSaslMechanisms = List("PLAIN", "GSSAPI") + this.serverConfig.setProperty(KafkaConfig.ZkEnableSecureAclsProp, "true") + override protected def securityProtocol = SecurityProtocol.SASL_SSL + override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks")) + override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, kafkaServerSaslMechanisms)) + + @Test + def testMultipleBrokerMechanisms() { + + val plainSaslProducer = producers(0) + val plainSaslConsumer = consumers(0) + + val gssapiSaslProperties = kafkaSaslProperties("GSSAPI", kafkaServerSaslMechanisms) + val gssapiSaslProducer = TestUtils.createNewProducer(brokerList, + securityProtocol = this.securityProtocol, + trustStoreFile = this.trustStoreFile, + saslProperties = Some(gssapiSaslProperties)) + producers += gssapiSaslProducer + val gssapiSaslConsumer = TestUtils.createNewConsumer(brokerList, + securityProtocol = this.securityProtocol, + trustStoreFile = this.trustStoreFile, + saslProperties = Some(gssapiSaslProperties)) + consumers += gssapiSaslConsumer + val numRecords = 1000 + var startingOffset = 0 + + // Test SASL/PLAIN producer and consumer + sendRecords(plainSaslProducer, numRecords, tp) + plainSaslConsumer.assign(List(tp).asJava) + plainSaslConsumer.seek(tp, 0) + consumeAndVerifyRecords(consumer = plainSaslConsumer, numRecords = numRecords, startingOffset = startingOffset) + val plainCommitCallback = new CountConsumerCommitCallback() + plainSaslConsumer.commitAsync(plainCommitCallback) + awaitCommitCallback(plainSaslConsumer, plainCommitCallback) + startingOffset += numRecords + + // Test SASL/GSSAPI producer and consumer + sendRecords(gssapiSaslProducer, numRecords, tp) + gssapiSaslConsumer.assign(List(tp).asJava) + gssapiSaslConsumer.seek(tp, startingOffset) + consumeAndVerifyRecords(consumer = gssapiSaslConsumer, numRecords = numRecords, startingOffset = startingOffset) + val gssapiCommitCallback = new CountConsumerCommitCallback() + gssapiSaslConsumer.commitAsync(gssapiCommitCallback) + awaitCommitCallback(gssapiSaslConsumer, gssapiCommitCallback) + startingOffset += numRecords + + // Test SASL/PLAIN producer and SASL/GSSAPI consumer + sendRecords(plainSaslProducer, numRecords, tp) + gssapiSaslConsumer.assign(List(tp).asJava) + gssapiSaslConsumer.seek(tp, startingOffset) + consumeAndVerifyRecords(consumer = gssapiSaslConsumer, numRecords = numRecords, startingOffset = startingOffset) + startingOffset += numRecords + + // Test SASL/GSSAPI producer and SASL/PLAIN consumer + sendRecords(gssapiSaslProducer, numRecords, tp) + plainSaslConsumer.assign(List(tp).asJava) + plainSaslConsumer.seek(tp, startingOffset) + consumeAndVerifyRecords(consumer = plainSaslConsumer, numRecords = numRecords, startingOffset = startingOffset) + + } +} diff --git a/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala new file mode 100644 index 0000000000000..687cfc39033c1 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala @@ -0,0 +1,27 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package kafka.api + +import java.io.File +import org.apache.kafka.common.protocol.SecurityProtocol +import kafka.server.KafkaConfig + +class SaslPlainPlaintextConsumerTest extends BaseConsumerTest with SaslTestHarness { + override protected val zkSaslEnabled = true + override protected val kafkaClientSaslMechanism = "PLAIN" + override protected val kafkaServerSaslMechanisms = List(kafkaClientSaslMechanism) + this.serverConfig.setProperty(KafkaConfig.ZkEnableSecureAclsProp, "true") + override protected def securityProtocol = SecurityProtocol.SASL_PLAINTEXT + override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks")) + override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, kafkaServerSaslMechanisms)) +} diff --git a/core/src/test/scala/integration/kafka/api/SaslSetup.scala b/core/src/test/scala/integration/kafka/api/SaslSetup.scala index 967cae1ea5634..acc86e3a692bb 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSetup.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSetup.scala @@ -19,11 +19,10 @@ package kafka.api import java.io.File import javax.security.auth.login.Configuration - import kafka.security.minikdc.MiniKdc import kafka.utils.{JaasTestUtils, TestUtils} import org.apache.kafka.common.security.JaasUtils -import org.apache.kafka.common.security.kerberos.LoginManager +import org.apache.kafka.common.security.authenticator.LoginManager /* * Implements an enumeration for the modes enabled here: @@ -40,35 +39,42 @@ case object Both extends SaslSetupMode trait SaslSetup { private val workDir = TestUtils.tempDir() private val kdcConf = MiniKdc.createConfig - private val kdc = new MiniKdc(kdcConf, workDir) + private var kdc: MiniKdc = null - def startSasl(mode: SaslSetupMode = Both) { + def startSasl(mode: SaslSetupMode = Both, kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanisms: List[String]) { // Important if tests leak consumers, producers or brokers LoginManager.closeAll() - val (serverKeytabFile, clientKeytabFile) = createKeytabsAndSetConfiguration(mode) - kdc.start() - kdc.createPrincipal(serverKeytabFile, "kafka/localhost") - kdc.createPrincipal(clientKeytabFile, "client") + val hasKerberos = mode != ZkSasl && (kafkaClientSaslMechanisms.contains("GSSAPI") || kafkaServerSaslMechanisms.contains("GSSAPI")) + if (hasKerberos) { + val serverKeytabFile = TestUtils.tempFile() + val clientKeytabFile = TestUtils.tempFile() + setJaasConfiguration(mode, kafkaServerSaslMechanisms, kafkaClientSaslMechanisms, Some(serverKeytabFile), Some(clientKeytabFile)) + kdc = new MiniKdc(kdcConf, workDir) + kdc.start() + kdc.createPrincipal(serverKeytabFile, "kafka/localhost") + kdc.createPrincipal(clientKeytabFile, "client") + } else { + setJaasConfiguration(mode, kafkaServerSaslMechanisms, kafkaClientSaslMechanisms) + } if (mode == Both || mode == ZkSasl) System.setProperty("zookeeper.authProvider.1", "org.apache.zookeeper.server.auth.SASLAuthenticationProvider") } - protected def createKeytabsAndSetConfiguration(mode: SaslSetupMode): (File, File) = { - val serverKeytabFile = TestUtils.tempFile() - val clientKeytabFile = TestUtils.tempFile() + protected def setJaasConfiguration(mode: SaslSetupMode, kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanisms: List[String], + serverKeytabFile: Option[File] = None, clientKeytabFile: Option[File] = None) { val jaasFile = mode match { case ZkSasl => JaasTestUtils.writeZkFile() - case KafkaSasl => JaasTestUtils.writeKafkaFile(serverKeytabFile, clientKeytabFile) - case Both => JaasTestUtils.writeZkAndKafkaFiles(serverKeytabFile, clientKeytabFile) + case KafkaSasl => JaasTestUtils.writeKafkaFile(kafkaServerSaslMechanisms, kafkaClientSaslMechanisms, serverKeytabFile, clientKeytabFile) + case Both => JaasTestUtils.writeZkAndKafkaFiles(kafkaServerSaslMechanisms, kafkaClientSaslMechanisms, serverKeytabFile, clientKeytabFile) } // This will cause a reload of the Configuration singleton when `getConfiguration` is called Configuration.setConfiguration(null) System.setProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM, jaasFile) - (serverKeytabFile, clientKeytabFile) } def closeSasl() { - kdc.stop() + if (kdc != null) + kdc.stop() // Important if tests leak consumers, producers or brokers LoginManager.closeAll() System.clearProperty(JaasUtils.JAVA_LOGIN_CONFIG_PARAM) diff --git a/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala index b4ae74f0e7cf5..55319195443d8 100644 --- a/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala @@ -13,17 +13,27 @@ package kafka.api import kafka.zk.ZooKeeperTestHarness +import kafka.server.KafkaConfig import org.junit.{After, Before} +import java.util.Properties +import scala.collection.JavaConverters._ +import org.apache.kafka.common.config.SaslConfigs trait SaslTestHarness extends ZooKeeperTestHarness with SaslSetup { protected val zkSaslEnabled: Boolean + protected val kafkaClientSaslMechanism = "GSSAPI" + protected val kafkaServerSaslMechanisms = List(kafkaClientSaslMechanism) + + // Override this list to enable client login modules for multiple mechanisms for testing + // of multi-mechanism brokers with clients using different mechanisms in a single JVM + protected def allKafkaClientSaslMechanisms = List(kafkaClientSaslMechanism) @Before override def setUp() { if (zkSaslEnabled) - startSasl(Both) + startSasl(Both, kafkaServerSaslMechanisms, allKafkaClientSaslMechanisms) else - startSasl(KafkaSasl) + startSasl(KafkaSasl, kafkaServerSaslMechanisms, allKafkaClientSaslMechanisms) super.setUp } @@ -33,4 +43,12 @@ trait SaslTestHarness extends ZooKeeperTestHarness with SaslSetup { closeSasl() } + def kafkaSaslProperties(kafkaClientSaslMechanism: String, kafkaServerSaslMechanisms: List[String]) = { + val props = new Properties + props.put(SaslConfigs.SASL_MECHANISM, kafkaClientSaslMechanism) + props.put(KafkaConfig.SaslMechanismInterBrokerProtocolProp, kafkaClientSaslMechanism) + props.put(SaslConfigs.SASL_ENABLED_MECHANISMS, kafkaServerSaslMechanisms.asJava) + props + } + } diff --git a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala index 2ca64f2fc1143..8e8ae8b3266c4 100755 --- a/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala +++ b/core/src/test/scala/unit/kafka/integration/KafkaServerTestHarness.scala @@ -19,7 +19,6 @@ package kafka.integration import java.io.File import java.util.Arrays - import kafka.common.KafkaException import kafka.server._ import kafka.utils.{CoreUtils, TestUtils} @@ -27,8 +26,8 @@ import kafka.zk.ZooKeeperTestHarness import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.kafka.common.security.auth.KafkaPrincipal import org.junit.{After, Before} - import scala.collection.mutable.Buffer +import java.util.Properties /** * A test harness that brings up some number of broker nodes @@ -57,6 +56,7 @@ trait KafkaServerTestHarness extends ZooKeeperTestHarness { protected def securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT protected def trustStoreFile: Option[File] = None + protected def saslProperties: Option[Properties] = None @Before override def setUp() { diff --git a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala index fa240d2efd1b8..f8476cd303be1 100755 --- a/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala +++ b/core/src/test/scala/unit/kafka/server/KafkaConfigTest.scala @@ -550,6 +550,8 @@ class KafkaConfigTest { case KafkaConfig.SslCipherSuitesProp => // ignore string //Sasl Configs + case KafkaConfig.SaslMechanismInterBrokerProtocolProp => // ignore + case KafkaConfig.SaslEnabledMechanismsProp => case KafkaConfig.SaslKerberosServiceNameProp => // ignore string case KafkaConfig.SaslKerberosKinitCmdProp => case KafkaConfig.SaslKerberosTicketRenewWindowFactorProp => diff --git a/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala b/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala index a14cd3f94cc6b..7c4b951e574e9 100644 --- a/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/JaasTestUtils.scala @@ -20,16 +20,14 @@ import java.io.{File, BufferedWriter, FileWriter} object JaasTestUtils { - case class Krb5LoginModule(contextName: String, - useKeyTab: Boolean, + case class Krb5LoginModule(useKeyTab: Boolean, storeKey: Boolean, keyTab: String, principal: String, debug: Boolean, serviceName: Option[String]) { - def toJaasSection: JaasSection = { - JaasSection( - contextName, + def toJaasModule: JaasModule = { + JaasModule( "com.sun.security.auth.module.Krb5LoginModule", debug = debug, entries = Map( @@ -42,15 +40,38 @@ object JaasTestUtils { } } - case class JaasSection(contextName: String, - moduleName: String, - debug: Boolean, - entries: Map[String, String]) { + case class PlainLoginModule(username: String, + password: String, + debug: Boolean = false, + validUsers: Map[String, String] = Map.empty) { + def toJaasModule: JaasModule = { + JaasModule( + "org.apache.kafka.common.security.plain.PlainLoginModule", + debug = debug, + entries = Map( + "username" -> username, + "password" -> password + ) ++ validUsers.map { case (user, pass) => (s"user_$user"-> pass)} + ) + } + } + + case class JaasModule(moduleName: String, + debug: Boolean, + entries: Map[String, String]) { override def toString: String = { - s"""|$contextName { - | $moduleName required + s"""$moduleName required | debug=$debug | ${entries.map { case (k, v) => s"""$k="$v"""" }.mkString("", "\n| ", ";")} + |""" + } + } + + class JaasSection(contextName: String, + jaasModule: Seq[JaasModule]) { + override def toString: String = { + s"""|$contextName { + | ${jaasModule.mkString("\n ")} |}; |""".stripMargin } @@ -67,6 +88,11 @@ object JaasTestUtils { private val KafkaServerPrincipal = "kafka/localhost@EXAMPLE.COM" private val KafkaClientContextName = "KafkaClient" private val KafkaClientPrincipal = "client@EXAMPLE.COM" + + private val KafkaPlainUser = "testuser" + private val KafkaPlainPassword = "testuser-secret" + private val KafkaPlainAdmin = "admin" + private val KafkaPlainAdminPassword = "admin-secret" def writeZkFile(): String = { val jaasFile = TestUtils.tempFile() @@ -74,43 +100,65 @@ object JaasTestUtils { jaasFile.getCanonicalPath } - def writeKafkaFile(serverKeyTabLocation: File, clientKeyTabLocation: File): String = { + def writeKafkaFile(kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanisms: List[String], serverKeyTabLocation: Option[File], clientKeyTabLocation: Option[File]): String = { val jaasFile = TestUtils.tempFile() - writeToFile(jaasFile, kafkaSections(serverKeyTabLocation, clientKeyTabLocation)) + val kafkaSections = Seq(kafkaServerSection(kafkaServerSaslMechanisms, serverKeyTabLocation), kafkaClientSection(kafkaClientSaslMechanisms, clientKeyTabLocation)) + writeToFile(jaasFile, kafkaSections) jaasFile.getCanonicalPath } - def writeZkAndKafkaFiles(serverKeyTabLocation: File, clientKeyTabLocation: File): String = { + def writeZkAndKafkaFiles(kafkaServerSaslMechanisms: List[String], kafkaClientSaslMechanisms: List[String], serverKeyTabLocation: Option[File], clientKeyTabLocation: Option[File]): String = { val jaasFile = TestUtils.tempFile() - writeToFile(jaasFile, kafkaSections(serverKeyTabLocation, clientKeyTabLocation) ++ zkSections) + val kafkaSections = Seq(kafkaServerSection(kafkaServerSaslMechanisms, serverKeyTabLocation), kafkaClientSection(kafkaClientSaslMechanisms, clientKeyTabLocation)) + writeToFile(jaasFile, kafkaSections ++ zkSections) jaasFile.getCanonicalPath } private def zkSections: Seq[JaasSection] = Seq( - JaasSection(ZkServerContextName, ZkModule, false, Map("user_super" -> ZkUserSuperPasswd, s"user_$ZkUser" -> ZkUserPassword)), - JaasSection(ZkClientContextName, ZkModule, false, Map("username" -> ZkUser, "password" -> ZkUserPassword)) + new JaasSection(ZkServerContextName, Seq(JaasModule(ZkModule, false, Map("user_super" -> ZkUserSuperPasswd, s"user_$ZkUser" -> ZkUserPassword)))), + new JaasSection(ZkClientContextName, Seq(JaasModule(ZkModule, false, Map("username" -> ZkUser, "password" -> ZkUserPassword)))) ) - private def kafkaSections(serverKeytabLocation: File, clientKeytabLocation: File): Seq[JaasSection] = { - Seq( - Krb5LoginModule( - KafkaServerContextName, - useKeyTab = true, - storeKey = true, - keyTab = serverKeytabLocation.getAbsolutePath, - principal = KafkaServerPrincipal, - debug = true, - serviceName = Some("kafka")), - Krb5LoginModule( - KafkaClientContextName, - useKeyTab = true, - storeKey = true, - keyTab = clientKeytabLocation.getAbsolutePath, - principal = KafkaClientPrincipal, - debug = true, - serviceName = Some("kafka") - ) - ).map(_.toJaasSection) + private def kafkaServerSection(mechanisms: List[String], keytabLocation: Option[File]): JaasSection = { + val modules = mechanisms.map { + case "GSSAPI" => + Krb5LoginModule( + useKeyTab = true, + storeKey = true, + keyTab = keytabLocation.getOrElse(throw new IllegalArgumentException("Keytab location not specified for GSSAPI")).getAbsolutePath, + principal = KafkaServerPrincipal, + debug = true, + serviceName = Some("kafka")).toJaasModule + case "PLAIN" => + PlainLoginModule( + KafkaPlainAdmin, + KafkaPlainAdminPassword, + debug = false, + Map(KafkaPlainAdmin -> KafkaPlainAdminPassword, KafkaPlainUser -> KafkaPlainPassword)).toJaasModule + case mechanism => throw new IllegalArgumentException("Unsupported server mechanism " + mechanism) + } + new JaasSection(KafkaServerContextName, modules) + } + + private def kafkaClientSection(mechanisms: List[String], keytabLocation: Option[File]): JaasSection = { + val modules = mechanisms.map { + case "GSSAPI" => + Krb5LoginModule( + useKeyTab = true, + storeKey = true, + keyTab = keytabLocation.getOrElse(throw new IllegalArgumentException("Keytab location not specified for GSSAPI")).getAbsolutePath, + principal = KafkaClientPrincipal, + debug = true, + serviceName = Some("kafka") + ).toJaasModule + case "PLAIN" => + PlainLoginModule( + KafkaPlainUser, + KafkaPlainPassword + ).toJaasModule + case mechanism => throw new IllegalArgumentException("Unsupported client mechanism " + mechanism) + } + new JaasSection(KafkaClientContextName, modules) } private def jaasSectionsToString(jaasSections: Seq[JaasSection]): String = diff --git a/core/src/test/scala/unit/kafka/utils/TestUtils.scala b/core/src/test/scala/unit/kafka/utils/TestUtils.scala index 6bd6c63f7b751..7df87fc31c76e 100755 --- a/core/src/test/scala/unit/kafka/utils/TestUtils.scala +++ b/core/src/test/scala/unit/kafka/utils/TestUtils.scala @@ -26,12 +26,10 @@ import java.util.{Collections, Properties, Random} import java.security.cert.X509Certificate import javax.net.ssl.X509TrustManager import charset.Charset - import kafka.security.auth.{Acl, Authorizer, Resource} import org.apache.kafka.common.protocol.SecurityProtocol import org.apache.kafka.common.utils.Utils._ import org.apache.kafka.test.TestSslUtils - import scala.collection.mutable.{ArrayBuffer, ListBuffer} import kafka.server._ import kafka.producer._ @@ -146,6 +144,7 @@ object TestUtils extends Logging { enableDeleteTopic: Boolean = false, interBrokerSecurityProtocol: Option[SecurityProtocol] = None, trustStoreFile: Option[File] = None, + saslProperties: Option[Properties] = None, enablePlaintext: Boolean = true, enableSsl: Boolean = false, enableSaslPlaintext: Boolean = false, @@ -153,7 +152,7 @@ object TestUtils extends Logging { rackInfo: Map[Int, String] = Map()): Seq[Properties] = { (0 until numConfigs).map { node => createBrokerConfig(node, zkConnect, enableControlledShutdown, enableDeleteTopic, RandomPort, - interBrokerSecurityProtocol, trustStoreFile, enablePlaintext = enablePlaintext, enableSsl = enableSsl, + interBrokerSecurityProtocol, trustStoreFile, saslProperties, enablePlaintext = enablePlaintext, enableSsl = enableSsl, enableSaslPlaintext = enableSaslPlaintext, enableSaslSsl = enableSaslSsl, rack = rackInfo.get(node)) } } @@ -173,6 +172,7 @@ object TestUtils extends Logging { port: Int = RandomPort, interBrokerSecurityProtocol: Option[SecurityProtocol] = None, trustStoreFile: Option[File] = None, + saslProperties: Option[Properties] = None, enablePlaintext: Boolean = true, enableSaslPlaintext: Boolean = false, saslPlaintextPort: Int = RandomPort, enableSsl: Boolean = false, sslPort: Int = RandomPort, @@ -211,6 +211,9 @@ object TestUtils extends Logging { if (protocolAndPorts.exists { case (protocol, _) => usesSslTransportLayer(protocol) }) props.putAll(sslConfigs(Mode.SERVER, false, trustStoreFile, s"server$nodeId")) + if (protocolAndPorts.exists { case (protocol, _) => usesSaslTransportLayer(protocol) }) + props.putAll(saslConfigs(saslProperties)) + interBrokerSecurityProtocol.foreach { protocol => props.put(KafkaConfig.InterBrokerSecurityProtocolProp, protocol.name) } @@ -440,16 +443,19 @@ object TestUtils extends Logging { private def securityConfigs(mode: Mode, securityProtocol: SecurityProtocol, trustStoreFile: Option[File], - certAlias: String): Properties = { + certAlias: String, + saslProperties: Option[Properties]): Properties = { val props = new Properties if (usesSslTransportLayer(securityProtocol)) props.putAll(sslConfigs(mode, securityProtocol == SecurityProtocol.SSL, trustStoreFile, certAlias)) + if (usesSaslTransportLayer(securityProtocol)) + props.putAll(saslConfigs(saslProperties)) props.put(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG, securityProtocol.name) props } - def producerSecurityConfigs(securityProtocol: SecurityProtocol, trustStoreFile: Option[File]): Properties = - securityConfigs(Mode.CLIENT, securityProtocol, trustStoreFile, "producer") + def producerSecurityConfigs(securityProtocol: SecurityProtocol, trustStoreFile: Option[File], saslProperties: Option[Properties]): Properties = + securityConfigs(Mode.CLIENT, securityProtocol, trustStoreFile, "producer", saslProperties) /** * Create a (new) producer with a few pre-configured properties. @@ -463,6 +469,7 @@ object TestUtils extends Logging { requestTimeoutMs: Long = 10 * 1024L, securityProtocol: SecurityProtocol = SecurityProtocol.PLAINTEXT, trustStoreFile: Option[File] = None, + saslProperties: Option[Properties] = None, keySerializer: Serializer[K] = new ByteArraySerializer, valueSerializer: Serializer[V] = new ByteArraySerializer, props: Option[Properties] = None): KafkaProducer[K, V] = { @@ -493,7 +500,7 @@ object TestUtils extends Logging { * SSL client auth fails. */ if (!producerProps.containsKey(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)) - producerProps.putAll(producerSecurityConfigs(securityProtocol, trustStoreFile)) + producerProps.putAll(producerSecurityConfigs(securityProtocol, trustStoreFile, saslProperties)) new KafkaProducer[K, V](producerProps, keySerializer, valueSerializer) } @@ -503,8 +510,13 @@ object TestUtils extends Logging { case _ => false } - def consumerSecurityConfigs(securityProtocol: SecurityProtocol, trustStoreFile: Option[File]): Properties = - securityConfigs(Mode.CLIENT, securityProtocol, trustStoreFile, "consumer") + private def usesSaslTransportLayer(securityProtocol: SecurityProtocol): Boolean = securityProtocol match { + case SecurityProtocol.SASL_PLAINTEXT | SecurityProtocol.SASL_SSL => true + case _ => false + } + + def consumerSecurityConfigs(securityProtocol: SecurityProtocol, trustStoreFile: Option[File], saslProperties: Option[Properties]): Properties = + securityConfigs(Mode.CLIENT, securityProtocol, trustStoreFile, "consumer", saslProperties) /** * Create a new consumer with a few pre-configured properties. @@ -517,6 +529,7 @@ object TestUtils extends Logging { sessionTimeout: Int = 30000, securityProtocol: SecurityProtocol, trustStoreFile: Option[File] = None, + saslProperties: Option[Properties] = None, props: Option[Properties] = None) : KafkaConsumer[Array[Byte],Array[Byte]] = { import org.apache.kafka.clients.consumer.ConsumerConfig @@ -545,7 +558,7 @@ object TestUtils extends Logging { * SSL client auth fails. */ if(!consumerProps.containsKey(CommonClientConfigs.SECURITY_PROTOCOL_CONFIG)) - consumerProps.putAll(consumerSecurityConfigs(securityProtocol, trustStoreFile)) + consumerProps.putAll(consumerSecurityConfigs(securityProtocol, trustStoreFile, saslProperties)) new KafkaConsumer[Array[Byte],Array[Byte]](consumerProps) } @@ -1058,6 +1071,13 @@ object TestUtils extends Logging { sslProps } + def saslConfigs(saslProperties: Option[Properties]): Properties = { + saslProperties match { + case Some(properties) => properties + case None => new Properties + } + } + // a X509TrustManager to trust self-signed certs for unit tests. def trustAllCerts: X509TrustManager = { val trustManager = new X509TrustManager() { From 33d745e2dcfa7a9cac90af5594903330ad774cd2 Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Tue, 26 Apr 2016 17:03:18 -0700 Subject: [PATCH 179/206] =?UTF-8?q?KAFKA-3306:=20Change=20metadata=20respo?= =?UTF-8?q?nse=20to=20include=20required=20additional=20fi=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …elds - Adds boolean type to the protocol - Allows protocol arrays to be null (optionally) - Adds support to ask for no topics in the metadata request - Adds new fields to the Metadata response protocol - Adds server code to handle new fields - Support no-topic metadata requests - Track controller id in the metadata cache - Check if a topic is considered internal - Included rack information if present - Include all replicas and ISRs, even if node is down - Adds test code to test new functionality independent of the client Author: Grant Henke Reviewers: Gwen Shapira, Ismael Juma, Ashish Singh Closes #1095 from granthenke/metadata-changes --- .../apache/kafka/clients/NetworkClient.java | 25 +-- .../kafka/clients/consumer/KafkaConsumer.java | 3 +- .../clients/consumer/internals/Fetcher.java | 17 +- .../java/org/apache/kafka/common/Node.java | 32 +++- .../kafka/common/protocol/Protocol.java | 44 ++++- .../kafka/common/protocol/types/ArrayOf.java | 33 +++- .../kafka/common/protocol/types/Struct.java | 34 ++-- .../kafka/common/protocol/types/Type.java | 43 ++++- .../common/requests/MetadataRequest.java | 40 ++++- .../common/requests/MetadataResponse.java | 101 ++++++++++- .../consumer/internals/FetcherTest.java | 11 +- .../types/ProtocolSerializationTest.java | 13 +- .../common/requests/RequestResponseTest.java | 21 ++- .../main/scala/kafka/admin/AdminUtils.scala | 4 +- .../main/scala/kafka/admin/TopicCommand.scala | 2 +- .../src/main/scala/kafka/cluster/Broker.scala | 4 +- core/src/main/scala/kafka/common/Topic.scala | 5 +- .../main/scala/kafka/server/KafkaApis.scala | 65 ++++--- .../scala/kafka/server/MetadataCache.scala | 39 ++-- .../scala/kafka/server/ReplicaManager.scala | 4 +- .../unit/kafka/server/BaseRequestTest.scala | 106 +++++++++++ .../unit/kafka/server/MetadataCacheTest.scala | 46 ++++- .../kafka/server/MetadataRequestTest.scala | 168 ++++++++++++++++++ 23 files changed, 732 insertions(+), 128 deletions(-) create mode 100644 core/src/test/scala/unit/kafka/server/BaseRequestTest.scala create mode 100644 core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala diff --git a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java index cc5dc6f1f8df4..b134631bd8b11 100644 --- a/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java +++ b/clients/src/main/java/org/apache/kafka/clients/NetworkClient.java @@ -35,11 +35,9 @@ import java.net.InetSocketAddress; import java.nio.ByteBuffer; import java.util.ArrayList; -import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Random; -import java.util.Set; /** * A network client for asynchronous request/response network i/o. This is an internal class used to implement the @@ -53,7 +51,7 @@ public class NetworkClient implements KafkaClient { /* the selector used to perform network i/o */ private final Selectable selector; - + private final MetadataUpdater metadataUpdater; private final Random randOffset; @@ -78,7 +76,7 @@ public class NetworkClient implements KafkaClient { /* max time in ms for the producer to wait for acknowledgement from server*/ private final int requestTimeoutMs; - + private final Time time; public NetworkClient(Selectable selector, @@ -114,7 +112,7 @@ private NetworkClient(MetadataUpdater metadataUpdater, int maxInFlightRequestsPerConnection, long reconnectBackoffMs, int socketSendBuffer, - int socketReceiveBuffer, + int socketReceiveBuffer, int requestTimeoutMs, Time time) { @@ -370,7 +368,7 @@ public Node leastLoadedNode(long now) { found = node; } } - + return found; } @@ -546,7 +544,7 @@ public long maybeUpdate(long now) { // if there is no node available to connect, back off refreshing metadata long metadataTimeout = Math.max(Math.max(timeToNextMetadataUpdate, timeToNextReconnectAttempt), waitForMetadataFetch); - + if (metadataTimeout == 0) { // Beware that the behavior of this method and the computation of timeouts for poll() are // highly dependent on the behavior of leastLoadedNode. @@ -614,8 +612,7 @@ private void handleResponse(RequestHeader header, Struct body, long now) { /** * Create a metadata request for the given topics */ - private ClientRequest request(long now, String node, Set topics) { - MetadataRequest metadata = new MetadataRequest(new ArrayList<>(topics)); + private ClientRequest request(long now, String node, MetadataRequest metadata) { RequestSend send = new RequestSend(node, nextRequestHeader(ApiKeys.METADATA), metadata.toStruct()); return new ClientRequest(now, true, send, null, true); } @@ -633,11 +630,15 @@ private void maybeUpdate(long now, Node node) { String nodeConnectionId = node.idString(); if (canSendRequest(nodeConnectionId)) { - Set topics = metadata.needMetadataForAllTopics() ? new HashSet() : metadata.topics(); this.metadataFetchInProgress = true; - ClientRequest metadataRequest = request(now, nodeConnectionId, topics); + MetadataRequest metadataRequest; + if (metadata.needMetadataForAllTopics()) + metadataRequest = MetadataRequest.allTopics(); + else + metadataRequest = new MetadataRequest(new ArrayList<>(metadata.topics())); + ClientRequest clientRequest = request(now, nodeConnectionId, metadataRequest); log.debug("Sending metadata request {} to node {}", metadataRequest, node.id()); - doSend(metadataRequest, now); + doSend(clientRequest, now); } else if (connectionStates.canConnect(nodeConnectionId, now)) { // we don't have a connection to this node right now, make one log.debug("Initialize connection to node {} for sending metadata request", node.id()); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index d9b74e2a0aede..ad44d16250093 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -35,6 +35,7 @@ import org.apache.kafka.common.metrics.MetricsReporter; import org.apache.kafka.common.network.ChannelBuilder; import org.apache.kafka.common.network.Selector; +import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.serialization.Deserializer; import org.apache.kafka.common.utils.AppInfoParser; import org.apache.kafka.common.utils.SystemTime; @@ -1190,7 +1191,7 @@ public List partitionsFor(String topic) { if (parts != null) return parts; - Map> topicMetadata = fetcher.getTopicMetadata(Collections.singletonList(topic), requestTimeoutMs); + Map> topicMetadata = fetcher.getTopicMetadata(new MetadataRequest(Collections.singletonList(topic)), requestTimeoutMs); return topicMetadata.get(topic); } finally { release(); diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java index 4985275624430..f6d338747668b 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/internals/Fetcher.java @@ -183,25 +183,26 @@ public void updateFetchPositions(Set partitions) { * @return The map of topics with their partition information */ public Map> getAllTopicMetadata(long timeout) { - return getTopicMetadata(null, timeout); + return getTopicMetadata(MetadataRequest.allTopics(), timeout); } /** * Get metadata for all topics present in Kafka cluster * - * @param topics The list of topics to fetch or null to fetch all + * @param request The MetadataRequest to send * @param timeout time for which getting topic metadata is attempted * @return The map of topics with their partition information */ - public Map> getTopicMetadata(List topics, long timeout) { - if (topics != null && topics.isEmpty()) + public Map> getTopicMetadata(MetadataRequest request, long timeout) { + // Save the round trip if no topics are requested. + if (!request.isAllTopics() && request.topics().isEmpty()) return Collections.emptyMap(); long start = time.milliseconds(); long remaining = timeout; do { - RequestFuture future = sendMetadataRequest(topics); + RequestFuture future = sendMetadataRequest(request); client.poll(future, remaining); if (future.failed() && !future.isRetriable()) @@ -266,14 +267,12 @@ else if (error.exception() instanceof RetriableException) * Send Metadata Request to least loaded node in Kafka cluster asynchronously * @return A future that indicates result of sent metadata request */ - private RequestFuture sendMetadataRequest(List topics) { - if (topics == null) - topics = Collections.emptyList(); + private RequestFuture sendMetadataRequest(MetadataRequest request) { final Node node = client.leastLoadedNode(); if (node == null) return RequestFuture.noBrokersAvailable(); else - return client.send(node, ApiKeys.METADATA, new MetadataRequest(topics)); + return client.send(node, ApiKeys.METADATA, request); } /** diff --git a/clients/src/main/java/org/apache/kafka/common/Node.java b/clients/src/main/java/org/apache/kafka/common/Node.java index 6c3fd0bab93b7..f569ddd576aae 100644 --- a/clients/src/main/java/org/apache/kafka/common/Node.java +++ b/clients/src/main/java/org/apache/kafka/common/Node.java @@ -3,9 +3,9 @@ * 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. @@ -23,13 +23,19 @@ public class Node { private final String idString; private final String host; private final int port; + private final String rack; public Node(int id, String host, int port) { + this(id, host, port, null); + } + + public Node(int id, String host, int port, String rack) { super(); this.id = id; this.idString = Integer.toString(id); this.host = host; this.port = port; + this.rack = rack; } public static Node noNode() { @@ -74,6 +80,20 @@ public int port() { return port; } + /** + * True if this node has a defined rack + */ + public boolean hasRack() { + return rack != null; + } + + /** + * The rack for this node + */ + public String rack() { + return rack; + } + @Override public int hashCode() { final int prime = 31; @@ -81,6 +101,7 @@ public int hashCode() { result = prime * result + ((host == null) ? 0 : host.hashCode()); result = prime * result + id; result = prime * result + port; + result = prime * result + ((rack == null) ? 0 : rack.hashCode()); return result; } @@ -102,12 +123,17 @@ public boolean equals(Object obj) { return false; if (port != other.port) return false; + if (rack == null) { + if (other.rack != null) + return false; + } else if (!rack.equals(other.rack)) + return false; return true; } @Override public String toString() { - return host + ":" + port + " (id: " + idString + ")"; + return host + ":" + port + " (id: " + idString + " rack: " + rack + ")"; } } diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index bf76557307cf8..d32209566677e 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -26,6 +26,7 @@ import java.util.Map; import java.util.Set; +import static org.apache.kafka.common.protocol.types.Type.BOOLEAN; 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; @@ -56,10 +57,13 @@ public class Protocol { new ArrayOf(STRING), "An array of topics to fetch metadata for. If no topics are specified fetch metadata for all topics.")); - public static final Schema BROKER = new Schema(new Field("node_id", INT32, "The broker id."), + public static final Schema METADATA_REQUEST_V1 = new Schema(new Field("topics", + ArrayOf.nullable(STRING), + "An array of topics to fetch metadata for. If the topics array is null fetch metadata for all topics.")); + + public static final Schema METADATA_BROKER_V0 = new Schema(new Field("node_id", INT32, "The broker id."), new Field("host", STRING, "The hostname of the broker."), - new Field("port", - INT32, + new Field("port", INT32, "The port on which the broker accepts requests.")); public static final Schema PARTITION_METADATA_V0 = new Schema(new Field("partition_error_code", @@ -87,13 +91,34 @@ public class Protocol { "Metadata for each partition of the topic.")); public static final Schema METADATA_RESPONSE_V0 = new Schema(new Field("brokers", - new ArrayOf(BROKER), + new ArrayOf(METADATA_BROKER_V0), "Host and port information for all brokers."), new Field("topic_metadata", new ArrayOf(TOPIC_METADATA_V0))); - public static final Schema[] METADATA_REQUEST = new Schema[] {METADATA_REQUEST_V0}; - public static final Schema[] METADATA_RESPONSE = new Schema[] {METADATA_RESPONSE_V0}; + public static final Schema METADATA_BROKER_V1 = 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."), + new Field("rack", NULLABLE_STRING, "The rack of the broker.")); + + public static final Schema PARTITION_METADATA_V1 = PARTITION_METADATA_V0; + + public static final Schema TOPIC_METADATA_V1 = 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("is_internal", BOOLEAN, + "Indicates if the topic is considered a Kafka internal topic"), + new Field("partition_metadata", new ArrayOf(PARTITION_METADATA_V1), + "Metadata for each partition of the topic.")); + + public static final Schema METADATA_RESPONSE_V1 = new Schema(new Field("brokers", new ArrayOf(METADATA_BROKER_V1), + "Host and port information for all brokers."), + new Field("controller_id", INT32, + "The broker id of the controller broker."), + new Field("topic_metadata", new ArrayOf(TOPIC_METADATA_V1))); + + public static final Schema[] METADATA_REQUEST = new Schema[] {METADATA_REQUEST_V0, METADATA_REQUEST_V1}; + public static final Schema[] METADATA_RESPONSE = new Schema[] {METADATA_RESPONSE_V0, METADATA_RESPONSE_V1}; /* Produce api */ @@ -496,9 +521,14 @@ public class Protocol { STRING, "The unique group id.")); + public static final Schema GROUP_COORDINATOR_BROKER_V0 = 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 final Schema GROUP_COORDINATOR_RESPONSE_V0 = new Schema(new Field("error_code", INT16), new Field("coordinator", - BROKER, + GROUP_COORDINATOR_BROKER_V0, "Host and port information for the coordinator for a consumer group.")); public static final Schema[] GROUP_COORDINATOR_REQUEST = new Schema[] {GROUP_COORDINATOR_REQUEST_V0}; diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java index a08f876abdb36..207f108abccf9 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/ArrayOf.java @@ -24,13 +24,33 @@ public class ArrayOf extends Type { private final Type type; + private final boolean nullable; public ArrayOf(Type type) { + this(type, false); + } + + public static ArrayOf nullable(Type type) { + return new ArrayOf(type, true); + } + + private ArrayOf(Type type, boolean nullable) { this.type = type; + this.nullable = nullable; + } + + @Override + public boolean isNullable() { + return nullable; } @Override public void write(ByteBuffer buffer, Object o) { + if (o == null) { + buffer.putInt(-1); + return; + } + Object[] objs = (Object[]) o; int size = objs.length; buffer.putInt(size); @@ -41,8 +61,11 @@ public void write(ByteBuffer buffer, Object o) { @Override public Object read(ByteBuffer buffer) { int size = buffer.getInt(); - if (size < 0) + if (size < 0 && isNullable()) + return null; + else if (size < 0) throw new SchemaException("Array size " + size + " cannot be negative"); + if (size > buffer.remaining()) throw new SchemaException("Error reading array of size " + size + ", only " + buffer.remaining() + " bytes available"); Object[] objs = new Object[size]; @@ -53,8 +76,11 @@ public Object read(ByteBuffer buffer) { @Override public int sizeOf(Object o) { - Object[] objs = (Object[]) o; int size = 4; + if (o == null) + return size; + + Object[] objs = (Object[]) o; for (int i = 0; i < objs.length; i++) size += type.sizeOf(objs[i]); return size; @@ -72,6 +98,9 @@ public String toString() { @Override public Object[] validate(Object item) { try { + if (isNullable() && item == null) + return null; + Object[] array = (Object[]) item; for (int i = 0; i < array.length; i++) type.validate(array[i]); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java index 79f0638c11e21..7eee09f0de425 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java @@ -3,9 +3,9 @@ * 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. @@ -41,7 +41,7 @@ public Schema 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. */ @@ -59,7 +59,7 @@ else if (field.type.isNullable()) /** * 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. * @throws SchemaException if the field has no value and has no default. @@ -71,7 +71,7 @@ public Object get(Field 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 * @throws SchemaException If no such field exists @@ -148,6 +148,14 @@ public String getString(String name) { return (String) get(name); } + public Boolean getBoolean(Field field) { + return (Boolean) get(field); + } + + public Boolean getBoolean(String name) { + return (Boolean) get(name); + } + public ByteBuffer getBytes(Field field) { Object result = get(field); if (result instanceof byte[]) @@ -164,7 +172,7 @@ public ByteBuffer getBytes(String name) { /** * Set the given field to the specified value - * + * * @param field The field * @param value The value * @throws SchemaException If the validation of the field failed @@ -177,7 +185,7 @@ public Struct set(Field field, Object value) { /** * Set the field specified by the given name to the value - * + * * @param name The name of the field * @param value The value to set * @throws SchemaException If the field is not known @@ -194,7 +202,7 @@ public Struct set(String name, Object value) { * Create a struct for the schema of a container type (struct or array). Note that for array type, this method * assumes that the type is an array of schema and creates a struct of that schema. Arrays of other types can't be * instantiated with this method. - * + * * @param field The field to create an instance of * @return The struct * @throws SchemaException If the given field is not a container type @@ -213,7 +221,7 @@ public Struct instance(Field field) { /** * 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 * @throws SchemaException If the given field is not a container type @@ -307,9 +315,11 @@ public int hashCode() { for (int i = 0; i < this.values.length; i++) { Field f = this.schema.get(i); if (f.type() instanceof ArrayOf) { - Object[] arrayObject = (Object []) this.get(f); - for (Object arrayItem: arrayObject) - result = prime * result + arrayItem.hashCode(); + if (this.get(f) != null) { + Object[] arrayObject = (Object []) this.get(f); + for (Object arrayItem: arrayObject) + result = prime * result + arrayItem.hashCode(); + } } else { Object field = this.get(f); if (field != null) { diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java index 92c1f7c67b17e..43b4a377a2781 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Type.java @@ -59,6 +59,47 @@ public boolean isNullable() { return false; } + /** + * The Boolean type represents a boolean value in a byte by using + * the value of 0 to represent false, and 1 to represent true. + * + * If for some reason a value that is not 0 or 1 is read, + * then any non-zero value will return true. + */ + public static final Type BOOLEAN = new Type() { + @Override + public void write(ByteBuffer buffer, Object o) { + if ((Boolean) o) + buffer.put((byte) 1); + else + buffer.put((byte) 0); + } + + @Override + public Object read(ByteBuffer buffer) { + byte value = buffer.get(); + return value != 0; + } + + @Override + public int sizeOf(Object o) { + return 1; + } + + @Override + public String toString() { + return "BOOLEAN"; + } + + @Override + public Boolean validate(Object item) { + if (item instanceof Boolean) + return (Boolean) item; + else + throw new SchemaException(item + " is not a Boolean."); + } + }; + public static final Type INT8 = new Type() { @Override public void write(ByteBuffer buffer, Object o) { @@ -196,7 +237,7 @@ public Object read(ByteBuffer buffer) { throw new SchemaException("String length " + length + " cannot be negative"); if (length > buffer.remaining()) throw new SchemaException("Error reading string of length " + length + ", only " + buffer.remaining() + " bytes available"); - + byte[] bytes = new byte[length]; buffer.get(bytes); return Utils.utf8(bytes); diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java index 92d8c6dc785a9..f0cb8fc0c0ade 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataRequest.java @@ -25,24 +25,41 @@ import java.util.List; public class MetadataRequest extends AbstractRequest { - + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.METADATA.id); private static final String TOPICS_KEY_NAME = "topics"; + private static final MetadataRequest ALL_TOPICS_REQUEST = new MetadataRequest((List) null); // Unusual cast to work around constructor ambiguity + private final List topics; + public static MetadataRequest allTopics() { + return ALL_TOPICS_REQUEST; + } + + /** + * In v0 null is not allowed and and empty list indicates requesting all topics. + * In v1 null indicates requesting all topics, and an empty list indicates requesting no topics. + */ public MetadataRequest(List topics) { super(new Struct(CURRENT_SCHEMA)); - struct.set(TOPICS_KEY_NAME, topics.toArray()); + if (topics == null) + struct.set(TOPICS_KEY_NAME, null); + else + struct.set(TOPICS_KEY_NAME, topics.toArray()); this.topics = topics; } public MetadataRequest(Struct struct) { super(struct); Object[] topicArray = struct.getArray(TOPICS_KEY_NAME); - topics = new ArrayList<>(); - for (Object topicObj: topicArray) { - topics.add((String) topicObj); + if (topicArray != null) { + topics = new ArrayList<>(); + for (Object topicObj: topicArray) { + topics.add((String) topicObj); + } + } else { + topics = null; } } @@ -52,18 +69,25 @@ public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { Errors error = Errors.forException(e); List partitions = Collections.emptyList(); - for (String topic : topics) - topicMetadatas.add(new MetadataResponse.TopicMetadata(error, topic, partitions)); + if (topics != null) { + for (String topic : topics) + topicMetadatas.add(new MetadataResponse.TopicMetadata(error, topic, false, partitions)); + } switch (versionId) { case 0: - return new MetadataResponse(Collections.emptyList(), topicMetadatas); + case 1: + return new MetadataResponse(Collections.emptyList(), MetadataResponse.NO_CONTROLLER_ID, topicMetadatas, versionId); default: throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.METADATA.id))); } } + public boolean isAllTopics() { + return topics == null; + } + public List topics() { return topics; } diff --git a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java index 13e0d8f7c5648..09a5beea843d0 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/MetadataResponse.java @@ -18,7 +18,6 @@ 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.Schema; import org.apache.kafka.common.protocol.types.Struct; import java.nio.ByteBuffer; @@ -32,7 +31,7 @@ public class MetadataResponse extends AbstractRequestResponse { - private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.METADATA.id); + private static final short CURRENT_VERSION = ProtoUtils.latestVersion(ApiKeys.METADATA.id); private static final String BROKERS_KEY_NAME = "brokers"; private static final String TOPIC_METADATA_KEY_NAME = "topic_metadata"; @@ -40,6 +39,10 @@ public class MetadataResponse extends AbstractRequestResponse { private static final String NODE_ID_KEY_NAME = "node_id"; private static final String HOST_KEY_NAME = "host"; private static final String PORT_KEY_NAME = "port"; + private static final String RACK_KEY_NAME = "rack"; + + private static final String CONTROLLER_ID_KEY_NAME = "controller_id"; + public static final int NO_CONTROLLER_ID = -1; // topic level field names private static final String TOPIC_ERROR_CODE_KEY_NAME = "topic_error_code"; @@ -54,6 +57,7 @@ public class MetadataResponse extends AbstractRequestResponse { */ private static final String TOPIC_KEY_NAME = "topic"; + private static final String IS_INTERNAL_KEY_NAME = "is_internal"; private static final String PARTITION_METADATA_KEY_NAME = "partition_metadata"; // partition level field names @@ -72,13 +76,24 @@ public class MetadataResponse extends AbstractRequestResponse { private static final String ISR_KEY_NAME = "isr"; private final Collection brokers; + private final Node controller; private final List topicMetadata; + /** + * Constructor for the latest version + */ + public MetadataResponse(List brokers, int controllerId, List topicMetadata) { + this(brokers, controllerId, topicMetadata, CURRENT_VERSION); + } - public MetadataResponse(List brokers, List topicMetadata) { - super(new Struct(CURRENT_SCHEMA)); + /** + * Constructor for a specific version + */ + public MetadataResponse(List brokers, int controllerId, List topicMetadata, int version) { + super(new Struct(ProtoUtils.responseSchema(ApiKeys.METADATA.id, version))); this.brokers = brokers; + this.controller = getControllerNode(controllerId, brokers); this.topicMetadata = topicMetadata; List brokerArray = new ArrayList<>(); @@ -87,15 +102,25 @@ public MetadataResponse(List brokers, List topicMetadata) { broker.set(NODE_ID_KEY_NAME, node.id()); broker.set(HOST_KEY_NAME, node.host()); broker.set(PORT_KEY_NAME, node.port()); + // This field only exists in v1+ + if (broker.hasField(RACK_KEY_NAME)) + broker.set(RACK_KEY_NAME, node.rack()); brokerArray.add(broker); } struct.set(BROKERS_KEY_NAME, brokerArray.toArray()); + // This field only exists in v1+ + if (struct.hasField(CONTROLLER_ID_KEY_NAME)) + struct.set(CONTROLLER_ID_KEY_NAME, controllerId); + List topicMetadataArray = new ArrayList<>(topicMetadata.size()); for (TopicMetadata metadata : topicMetadata) { Struct topicData = struct.instance(TOPIC_METADATA_KEY_NAME); topicData.set(TOPIC_KEY_NAME, metadata.topic); topicData.set(TOPIC_ERROR_CODE_KEY_NAME, metadata.error.code()); + // This field only exists in v1+ + if (topicData.hasField(IS_INTERNAL_KEY_NAME)) + topicData.set(IS_INTERNAL_KEY_NAME, metadata.isInternal()); List partitionMetadataArray = new ArrayList<>(metadata.partitionMetadata.size()); for (PartitionMetadata partitionMetadata : metadata.partitionMetadata()) { @@ -130,15 +155,28 @@ public MetadataResponse(Struct struct) { int nodeId = broker.getInt(NODE_ID_KEY_NAME); String host = broker.getString(HOST_KEY_NAME); int port = broker.getInt(PORT_KEY_NAME); - brokers.put(nodeId, new Node(nodeId, host, port)); + // This field only exists in v1+ + // When we can't know if a rack exists in a v0 response we default to null + String rack = broker.hasField(RACK_KEY_NAME) ? broker.getString(RACK_KEY_NAME) : null; + brokers.put(nodeId, new Node(nodeId, host, port, rack)); } + // This field only exists in v1+ + // When we can't know the controller id in a v0 response we default to NO_CONTROLLER_ID + int controllerId = NO_CONTROLLER_ID; + if (struct.hasField(CONTROLLER_ID_KEY_NAME)) + controllerId = struct.getInt(CONTROLLER_ID_KEY_NAME); + List topicMetadata = new ArrayList<>(); Object[] topicInfos = (Object[]) struct.get(TOPIC_METADATA_KEY_NAME); for (int i = 0; i < topicInfos.length; i++) { Struct topicInfo = (Struct) topicInfos[i]; Errors topicError = Errors.forCode(topicInfo.getShort(TOPIC_ERROR_CODE_KEY_NAME)); String topic = topicInfo.getString(TOPIC_KEY_NAME); + // This field only exists in v1+ + // When we can't know if a topic is internal or not in a v0 response we default to false + boolean isInternal = topicInfo.hasField(IS_INTERNAL_KEY_NAME) ? topicInfo.getBoolean(IS_INTERNAL_KEY_NAME) : false; + List partitionMetadata = new ArrayList<>(); Object[] partitionInfos = (Object[]) topicInfo.get(PARTITION_METADATA_KEY_NAME); @@ -149,23 +187,41 @@ public MetadataResponse(Struct struct) { int leader = partitionInfo.getInt(LEADER_KEY_NAME); Node leaderNode = leader == -1 ? null : brokers.get(leader); Object[] replicas = (Object[]) partitionInfo.get(REPLICAS_KEY_NAME); + List replicaNodes = new ArrayList<>(replicas.length); for (Object replicaNodeId : replicas) - replicaNodes.add(brokers.get(replicaNodeId)); + if (brokers.containsKey(replicaNodeId)) + replicaNodes.add(brokers.get(replicaNodeId)); + else + replicaNodes.add(new Node((int) replicaNodeId, "", -1)); + Object[] isr = (Object[]) partitionInfo.get(ISR_KEY_NAME); List isrNodes = new ArrayList<>(isr.length); for (Object isrNode : isr) - isrNodes.add(brokers.get(isrNode)); + if (brokers.containsKey(isrNode)) + isrNodes.add(brokers.get(isrNode)); + else + isrNodes.add(new Node((int) isrNode, "", -1)); + partitionMetadata.add(new PartitionMetadata(partitionError, partition, leaderNode, replicaNodes, isrNodes)); } - topicMetadata.add(new TopicMetadata(topicError, topic, partitionMetadata)); + topicMetadata.add(new TopicMetadata(topicError, topic, isInternal, partitionMetadata)); } this.brokers = brokers.values(); + this.controller = getControllerNode(controllerId, brokers.values()); this.topicMetadata = topicMetadata; } + private Node getControllerNode(int controllerId, Collection brokers) { + for (Node broker : brokers) { + if (broker.id() == controllerId) + return broker; + } + return null; + } + /** * Get a map of the topics which had metadata errors * @return the map @@ -211,20 +267,43 @@ public Collection brokers() { return brokers; } + /** + * Get all topic metadata returned in the metadata response + * @return the topicMetadata + */ + public Collection topicMetadata() { + return topicMetadata; + } + + /** + * The controller node returned in metadata response + * @return the controller node or null if it doesn't exist + */ + public Node controller() { + return controller; + } + public static MetadataResponse parse(ByteBuffer buffer) { - return new MetadataResponse(CURRENT_SCHEMA.read(buffer)); + return parse(buffer, CURRENT_VERSION); + } + + public static MetadataResponse parse(ByteBuffer buffer, int version) { + return new MetadataResponse(ProtoUtils.responseSchema(ApiKeys.METADATA.id, version).read(buffer)); } public static class TopicMetadata { private final Errors error; private final String topic; + private final boolean isInternal; private final List partitionMetadata; public TopicMetadata(Errors error, String topic, + boolean isInternal, List partitionMetadata) { this.error = error; this.topic = topic; + this.isInternal = isInternal; this.partitionMetadata = partitionMetadata; } @@ -236,6 +315,10 @@ public String topic() { return topic; } + public boolean isInternal() { + return isInternal; + } + public List partitionMetadata() { return partitionMetadata; } diff --git a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java index 9002e819b4f6e..49bff1031a21f 100644 --- a/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java +++ b/clients/src/test/java/org/apache/kafka/clients/consumer/internals/FetcherTest.java @@ -44,6 +44,7 @@ import org.apache.kafka.common.requests.FetchResponse; import org.apache.kafka.common.requests.ListOffsetRequest; import org.apache.kafka.common.requests.ListOffsetResponse; +import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.MetadataResponse; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.kafka.common.utils.MockTime; @@ -478,14 +479,14 @@ public void testGetAllTopicsUnauthorized() { @Test(expected = InvalidTopicException.class) public void testGetTopicMetadataInvalidTopic() { client.prepareResponse(newMetadataResponse(topicName, Errors.INVALID_TOPIC_EXCEPTION).toStruct()); - fetcher.getTopicMetadata(Collections.singletonList(topicName), 5000L); + fetcher.getTopicMetadata(new MetadataRequest(Collections.singletonList(topicName)), 5000L); } @Test public void testGetTopicMetadataUnknownTopic() { client.prepareResponse(newMetadataResponse(topicName, Errors.UNKNOWN_TOPIC_OR_PARTITION).toStruct()); - Map> topicMetadata = fetcher.getTopicMetadata(Collections.singletonList(topicName), 5000L); + Map> topicMetadata = fetcher.getTopicMetadata(new MetadataRequest(Collections.singletonList(topicName)), 5000L); assertNull(topicMetadata.get(topicName)); } @@ -494,7 +495,7 @@ public void testGetTopicMetadataLeaderNotAvailable() { client.prepareResponse(newMetadataResponse(topicName, Errors.LEADER_NOT_AVAILABLE).toStruct()); client.prepareResponse(newMetadataResponse(topicName, Errors.NONE).toStruct()); - Map> topicMetadata = fetcher.getTopicMetadata(Collections.singletonList(topicName), 5000L); + Map> topicMetadata = fetcher.getTopicMetadata(new MetadataRequest(Collections.singletonList(topicName)), 5000L); assertTrue(topicMetadata.containsKey(topicName)); } @@ -570,8 +571,8 @@ private MetadataResponse newMetadataResponse(String topic, Errors error) { } } - MetadataResponse.TopicMetadata topicMetadata = new MetadataResponse.TopicMetadata(error, topic, partitionsMetadata); - return new MetadataResponse(cluster.nodes(), Arrays.asList(topicMetadata)); + MetadataResponse.TopicMetadata topicMetadata = new MetadataResponse.TopicMetadata(error, topic, false, partitionsMetadata); + return new MetadataResponse(cluster.nodes(), MetadataResponse.NO_CONTROLLER_ID, Arrays.asList(topicMetadata)); } private Fetcher createFetcher(int maxPollRecords, diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java index 5c342776b48f6..e91b2fb591d4e 100644 --- a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java @@ -33,7 +33,8 @@ public class ProtocolSerializationTest { @Before public void setup() { - this.schema = new Schema(new Field("int8", Type.INT8), + this.schema = new Schema(new Field("boolean", Type.BOOLEAN), + new Field("int8", Type.INT8), new Field("int16", Type.INT16), new Field("int32", Type.INT32), new Field("int64", Type.INT64), @@ -42,8 +43,10 @@ public void setup() { new Field("bytes", Type.BYTES), new Field("nullable_bytes", Type.NULLABLE_BYTES), new Field("array", new ArrayOf(Type.INT32)), + new Field("null_array", ArrayOf.nullable(Type.INT32)), new Field("struct", new Schema(new Field("field", new ArrayOf(Type.INT32))))); - this.struct = new Struct(this.schema).set("int8", (byte) 1) + this.struct = new Struct(this.schema).set("boolean", true) + .set("int8", (byte) 1) .set("int16", (short) 1) .set("int32", 1) .set("int64", 1L) @@ -51,12 +54,15 @@ public void setup() { .set("nullable_string", null) .set("bytes", ByteBuffer.wrap("1".getBytes())) .set("nullable_bytes", null) - .set("array", new Object[] {1}); + .set("array", new Object[] {1}) + .set("null_array", null); this.struct.set("struct", this.struct.instance("struct").set("field", new Object[] {1, 2, 3})); } @Test public void testSimple() { + check(Type.BOOLEAN, false); + check(Type.BOOLEAN, true); check(Type.INT8, (byte) -111); check(Type.INT16, (short) -11111); check(Type.INT32, -11111111); @@ -75,6 +81,7 @@ public void testSimple() { 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"}); + check(ArrayOf.nullable(Type.STRING), null); } @Test diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 92f3101e78468..0018f531f405f 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -70,9 +70,10 @@ public void testSerialization() throws Exception { createListOffsetRequest(), createListOffsetRequest().getErrorResponse(0, new UnknownServerException()), createListOffsetResponse(), - createMetadataRequest(), - createMetadataRequest().getErrorResponse(0, new UnknownServerException()), - createMetadataResponse(), + MetadataRequest.allTopics(), + createMetadataRequest(Arrays.asList("topic1")), + createMetadataRequest(Arrays.asList("topic1")).getErrorResponse(1, new UnknownServerException()), + createMetadataResponse(1), createOffsetCommitRequest(2), createOffsetCommitRequest(2).getErrorResponse(2, new UnknownServerException()), createOffsetCommitResponse(), @@ -100,6 +101,8 @@ public void testSerialization() throws Exception { for (AbstractRequestResponse req : requestResponseList) checkSerialization(req, null); + createMetadataResponse(0); + createMetadataRequest(Arrays.asList("topic1")).getErrorResponse(0, new UnknownServerException()); checkSerialization(createFetchRequest().getErrorResponse(0, new UnknownServerException()), 0); checkSerialization(createOffsetCommitRequest(0), 0); checkSerialization(createOffsetCommitRequest(0).getErrorResponse(0, new UnknownServerException()), 0); @@ -281,22 +284,22 @@ private AbstractRequestResponse createListOffsetResponse() { return new ListOffsetResponse(responseData); } - private AbstractRequest createMetadataRequest() { - return new MetadataRequest(Arrays.asList("topic1")); + private AbstractRequest createMetadataRequest(List topics) { + return new MetadataRequest(topics); } - private AbstractRequestResponse createMetadataResponse() { + private AbstractRequestResponse createMetadataResponse(int version) { Node node = new Node(1, "host1", 1001); List replicas = Arrays.asList(node); List isr = Arrays.asList(node); List allTopicMetadata = new ArrayList<>(); - allTopicMetadata.add(new MetadataResponse.TopicMetadata(Errors.NONE, "topic1", + allTopicMetadata.add(new MetadataResponse.TopicMetadata(Errors.NONE, "__consumer_offsets", true, Arrays.asList(new MetadataResponse.PartitionMetadata(Errors.NONE, 1, node, replicas, isr)))); - allTopicMetadata.add(new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, "topic2", + allTopicMetadata.add(new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, "topic2", false, Collections.emptyList())); - return new MetadataResponse(Arrays.asList(node), allTopicMetadata); + return new MetadataResponse(Arrays.asList(node), MetadataResponse.NO_CONTROLLER_ID, allTopicMetadata, version); } private AbstractRequest createOffsetCommitRequest(int version) { diff --git a/core/src/main/scala/kafka/admin/AdminUtils.scala b/core/src/main/scala/kafka/admin/AdminUtils.scala index 24174bea95461..a8a282e2c13a1 100644 --- a/core/src/main/scala/kafka/admin/AdminUtils.scala +++ b/core/src/main/scala/kafka/admin/AdminUtils.scala @@ -605,10 +605,10 @@ object AdminUtils extends Logging { new MetadataResponse.PartitionMetadata(Errors.forException(e), partition, leaderInfo, replicaInfo.asJava, isrInfo.asJava) } } - new MetadataResponse.TopicMetadata(Errors.NONE, topic, partitionMetadata.toList.asJava) + new MetadataResponse.TopicMetadata(Errors.NONE, topic, Topic.isInternal(topic), partitionMetadata.asJava) } else { // topic doesn't exist, send appropriate error code - new MetadataResponse.TopicMetadata(Errors.UNKNOWN_TOPIC_OR_PARTITION, topic, java.util.Collections.emptyList()) + new MetadataResponse.TopicMetadata(Errors.UNKNOWN_TOPIC_OR_PARTITION, topic, Topic.isInternal(topic), java.util.Collections.emptyList()) } } diff --git a/core/src/main/scala/kafka/admin/TopicCommand.scala b/core/src/main/scala/kafka/admin/TopicCommand.scala index 232db4ac1d5f8..9f1014f286b7e 100755 --- a/core/src/main/scala/kafka/admin/TopicCommand.scala +++ b/core/src/main/scala/kafka/admin/TopicCommand.scala @@ -170,7 +170,7 @@ object TopicCommand extends Logging { } topics.foreach { topic => try { - if (TopicConstants.INTERNAL_TOPICS.contains(topic)) { + if (Topic.isInternal(topic)) { throw new AdminOperationException("Topic %s is a kafka internal topic and is not allowed to be marked for deletion.".format(topic)) } else { zkUtils.createPersistentPath(getDeleteTopicPath(topic)) diff --git a/core/src/main/scala/kafka/cluster/Broker.scala b/core/src/main/scala/kafka/cluster/Broker.scala index 77b85e01b4020..61290c1fe6730 100755 --- a/core/src/main/scala/kafka/cluster/Broker.scala +++ b/core/src/main/scala/kafka/cluster/Broker.scala @@ -5,7 +5,7 @@ * 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 @@ -123,7 +123,7 @@ case class Broker(id: Int, endPoints: collection.Map[SecurityProtocol, EndPoint] def getNode(protocolType: SecurityProtocol): Node = { val endpoint = endPoints.getOrElse(protocolType, throw new BrokerEndPointNotAvailableException(s"End point with security protocol $protocolType not found for broker $id")) - new Node(id, endpoint.host, endpoint.port) + new Node(id, endpoint.host, endpoint.port, rack.orNull) } def getBrokerEndPoint(protocolType: SecurityProtocol): BrokerEndPoint = { diff --git a/core/src/main/scala/kafka/common/Topic.scala b/core/src/main/scala/kafka/common/Topic.scala index 6067712762afe..054c5eb38ea0e 100644 --- a/core/src/main/scala/kafka/common/Topic.scala +++ b/core/src/main/scala/kafka/common/Topic.scala @@ -18,7 +18,7 @@ package kafka.common import util.matching.Regex -import kafka.coordinator.GroupCoordinator +import org.apache.kafka.common.internals.TopicConstants.INTERNAL_TOPICS object Topic { val legalChars = "[a-zA-Z0-9\\._\\-]" @@ -62,4 +62,7 @@ object Topic { topicA.replace('.', '_') == topicB.replace('.', '_') } + def isInternal(topic: String): Boolean = + INTERNAL_TOPICS.contains(topic) + } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 9afefa5c6de2a..406b1bdb70fef 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -24,6 +24,7 @@ import java.util.Properties import kafka.admin.{RackAwareMode, AdminUtils} import kafka.api._ import kafka.cluster.Partition +import kafka.common import kafka.common._ import kafka.controller.KafkaController import kafka.coordinator.{GroupCoordinator, JoinGroupResult} @@ -631,12 +632,15 @@ class KafkaApis(val requestChannel: RequestChannel, AdminUtils.createTopic(zkUtils, topic, numPartitions, replicationFactor, properties, RackAwareMode.Safe) info("Auto creation of topic %s with %d partitions and replication factor %d is successful" .format(topic, numPartitions, replicationFactor)) - new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, topic, java.util.Collections.emptyList()) + new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, topic, common.Topic.isInternal(topic), + java.util.Collections.emptyList()) } catch { case e: TopicExistsException => // let it go, possibly another broker created this topic - new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, topic, java.util.Collections.emptyList()) + new MetadataResponse.TopicMetadata(Errors.LEADER_NOT_AVAILABLE, topic, common.Topic.isInternal(topic), + java.util.Collections.emptyList()) case itex: InvalidTopicException => - new MetadataResponse.TopicMetadata(Errors.INVALID_TOPIC_EXCEPTION, topic, java.util.Collections.emptyList()) + new MetadataResponse.TopicMetadata(Errors.INVALID_TOPIC_EXCEPTION, topic, common.Topic.isInternal(topic), + java.util.Collections.emptyList()) } } @@ -656,8 +660,8 @@ class KafkaApis(val requestChannel: RequestChannel, topicMetadata.headOption.getOrElse(createGroupMetadataTopic()) } - private def getTopicMetadata(topics: Set[String], securityProtocol: SecurityProtocol): Seq[MetadataResponse.TopicMetadata] = { - val topicResponses = metadataCache.getTopicMetadata(topics, securityProtocol) + private def getTopicMetadata(topics: Set[String], securityProtocol: SecurityProtocol, errorUnavailableEndpoints: Boolean): Seq[MetadataResponse.TopicMetadata] = { + val topicResponses = metadataCache.getTopicMetadata(topics, securityProtocol, errorUnavailableEndpoints) if (topics.isEmpty || topicResponses.size == topics.size) { topicResponses } else { @@ -668,7 +672,8 @@ class KafkaApis(val requestChannel: RequestChannel, } else if (config.autoCreateTopicsEnable) { createTopic(topic, config.numPartitions, config.defaultReplicationFactor) } else { - new MetadataResponse.TopicMetadata(Errors.UNKNOWN_TOPIC_OR_PARTITION, topic, java.util.Collections.emptyList()) + new MetadataResponse.TopicMetadata(Errors.UNKNOWN_TOPIC_OR_PARTITION, topic, common.Topic.isInternal(topic), + java.util.Collections.emptyList()) } } topicResponses ++ responsesForNonExistentTopics @@ -680,16 +685,24 @@ class KafkaApis(val requestChannel: RequestChannel, */ def handleTopicMetadataRequest(request: RequestChannel.Request) { val metadataRequest = request.body.asInstanceOf[MetadataRequest] + val requestVersion = request.header.apiVersion() - val topics = metadataRequest.topics.asScala.toSet - var (authorizedTopics, unauthorizedTopics) = if (metadataRequest.topics.isEmpty) { - //if topics is empty -> fetch all topics metadata but filter out the topic response that are not authorized - val authorized = metadataCache.getAllTopics() - .filter(topic => authorize(request.session, Describe, new Resource(Topic, topic))) - (authorized, mutable.Set[String]()) - } else { + val topics = + // Handle old metadata request logic. Version 0 has no way to specify "no topics". + if (requestVersion == 0) { + if (metadataRequest.topics() == null || metadataRequest.topics().isEmpty) + metadataCache.getAllTopics() + else + metadataRequest.topics.asScala.toSet + } else { + if (metadataRequest.isAllTopics) + metadataCache.getAllTopics() + else + metadataRequest.topics.asScala.toSet + } + + var (authorizedTopics, unauthorizedTopics) = topics.partition(topic => authorize(request.session, Describe, new Resource(Topic, topic))) - } if (authorizedTopics.nonEmpty) { val nonExistingTopics = metadataCache.getNonExistingTopics(authorizedTopics) @@ -704,22 +717,32 @@ class KafkaApis(val requestChannel: RequestChannel, } val unauthorizedTopicMetadata = unauthorizedTopics.map(topic => - new MetadataResponse.TopicMetadata(Errors.TOPIC_AUTHORIZATION_FAILED, topic, java.util.Collections.emptyList())) + new MetadataResponse.TopicMetadata(Errors.TOPIC_AUTHORIZATION_FAILED, topic, common.Topic.isInternal(topic), + java.util.Collections.emptyList())) + + // In version 0, we returned an error when brokers with replicas were unavailable, + // while in higher versions we simply don't include the broker in the returned broker list + val errorUnavailableEndpoints = requestVersion == 0 + val topicMetadata = + if (authorizedTopics.isEmpty) + Seq.empty[MetadataResponse.TopicMetadata] + else + getTopicMetadata(authorizedTopics, request.securityProtocol, errorUnavailableEndpoints) - val topicMetadata = if (authorizedTopics.isEmpty) - Seq.empty[MetadataResponse.TopicMetadata] - else - getTopicMetadata(authorizedTopics, request.securityProtocol) + val completeTopicMetadata = topicMetadata ++ unauthorizedTopicMetadata val brokers = metadataCache.getAliveBrokers - trace("Sending topic metadata %s and brokers %s for correlation id %d to client %s".format(topicMetadata.mkString(","), + trace("Sending topic metadata %s and brokers %s for correlation id %d to client %s".format(completeTopicMetadata.mkString(","), brokers.mkString(","), request.header.correlationId, request.header.clientId)) val responseHeader = new ResponseHeader(request.header.correlationId) + val responseBody = new MetadataResponse( brokers.map(_.getNode(request.securityProtocol)).asJava, - (topicMetadata ++ unauthorizedTopicMetadata).asJava + metadataCache.getControllerId.getOrElse(MetadataResponse.NO_CONTROLLER_ID), + completeTopicMetadata.asJava, + requestVersion ) requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, responseHeader, responseBody))) } diff --git a/core/src/main/scala/kafka/server/MetadataCache.scala b/core/src/main/scala/kafka/server/MetadataCache.scala index 06fae42116e54..b387f2efab3d5 100755 --- a/core/src/main/scala/kafka/server/MetadataCache.scala +++ b/core/src/main/scala/kafka/server/MetadataCache.scala @@ -24,11 +24,11 @@ import scala.collection.{Seq, Set, mutable} import scala.collection.JavaConverters._ import kafka.cluster.{Broker, EndPoint} import kafka.api._ -import kafka.common.{BrokerEndPointNotAvailableException, TopicAndPartition} +import kafka.common.{BrokerEndPointNotAvailableException, Topic, TopicAndPartition} import kafka.controller.{KafkaController, LeaderIsrAndControllerEpoch} import kafka.utils.CoreUtils._ import kafka.utils.Logging -import org.apache.kafka.common.{Node, TopicPartition} +import org.apache.kafka.common.Node import org.apache.kafka.common.protocol.{Errors, SecurityProtocol} import org.apache.kafka.common.requests.UpdateMetadataRequest.PartitionState import org.apache.kafka.common.requests.{MetadataResponse, UpdateMetadataRequest} @@ -40,16 +40,24 @@ import org.apache.kafka.common.requests.{MetadataResponse, UpdateMetadataRequest private[server] class MetadataCache(brokerId: Int) extends Logging { private val stateChangeLogger = KafkaController.stateChangeLogger private val cache = mutable.Map[String, mutable.Map[Int, PartitionStateInfo]]() + private var controllerId: Option[Int] = None private val aliveBrokers = mutable.Map[Int, Broker]() private val aliveNodes = mutable.Map[Int, collection.Map[SecurityProtocol, Node]]() private val partitionMetadataLock = new ReentrantReadWriteLock() this.logIdent = s"[Kafka Metadata Cache on broker $brokerId] " - private def getAliveEndpoints(brokers: Iterable[Int], protocol: SecurityProtocol): Seq[Node] = { + // This method is the main hotspot when it comes to the performance of metadata requests, + // we should be careful about adding additional logic here. + // filterUnavailableEndpoints exists to support v0 MetadataResponses + private def getEndpoints(brokers: Iterable[Int], protocol: SecurityProtocol, filterUnavailableEndpoints: Boolean): Seq[Node] = { val result = new mutable.ArrayBuffer[Node](math.min(aliveBrokers.size, brokers.size)) brokers.foreach { brokerId => - getAliveEndpoint(brokerId, protocol).foreach(result +=) + val endpoint = getAliveEndpoint(brokerId, protocol) match { + case None => if (!filterUnavailableEndpoints) Some(new Node(brokerId, "", -1)) else None + case Some(node) => Some(node) + } + endpoint.foreach(result +=) } result } @@ -60,7 +68,8 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { throw new BrokerEndPointNotAvailableException(s"Broker `$brokerId` does not support security protocol `$protocol`")) } - private def getPartitionMetadata(topic: String, protocol: SecurityProtocol): Option[Iterable[MetadataResponse.PartitionMetadata]] = { + // errorUnavailableEndpoints exists to support v0 MetadataResponses + private def getPartitionMetadata(topic: String, protocol: SecurityProtocol, errorUnavailableEndpoints: Boolean): Option[Iterable[MetadataResponse.PartitionMetadata]] = { cache.get(topic).map { partitions => partitions.map { case (partitionId, partitionState) => val topicPartition = TopicAndPartition(topic, partitionId) @@ -69,7 +78,7 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { val maybeLeader = getAliveEndpoint(leaderAndIsr.leader, protocol) val replicas = partitionState.allReplicas - val replicaInfo = getAliveEndpoints(replicas, protocol) + val replicaInfo = getEndpoints(replicas, protocol, errorUnavailableEndpoints) maybeLeader match { case None => @@ -79,7 +88,7 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { case Some(leader) => val isr = leaderAndIsr.isr - val isrInfo = getAliveEndpoints(isr, protocol) + val isrInfo = getEndpoints(isr, protocol, errorUnavailableEndpoints) if (replicaInfo.size < replicas.size) { debug(s"Error while fetching metadata for $topicPartition: replica information not available for " + @@ -101,12 +110,12 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { } } - def getTopicMetadata(topics: Set[String], protocol: SecurityProtocol): Seq[MetadataResponse.TopicMetadata] = { + // errorUnavailableEndpoints exists to support v0 MetadataResponses + def getTopicMetadata(topics: Set[String], protocol: SecurityProtocol, errorUnavailableEndpoints: Boolean = false): Seq[MetadataResponse.TopicMetadata] = { inReadLock(partitionMetadataLock) { - val topicsRequested = if (topics.isEmpty) cache.keySet else topics - topicsRequested.toSeq.flatMap { topic => - getPartitionMetadata(topic, protocol).map { partitionMetadata => - new MetadataResponse.TopicMetadata(Errors.NONE, topic, partitionMetadata.toBuffer.asJava) + topics.toSeq.flatMap { topic => + getPartitionMetadata(topic, protocol, errorUnavailableEndpoints).map { partitionMetadata => + new MetadataResponse.TopicMetadata(Errors.NONE, topic, Topic.isInternal(topic), partitionMetadata.toBuffer.asJava) } } } @@ -151,8 +160,14 @@ private[server] class MetadataCache(brokerId: Int) extends Logging { } } + def getControllerId: Option[Int] = controllerId + def updateCache(correlationId: Int, updateMetadataRequest: UpdateMetadataRequest) { inWriteLock(partitionMetadataLock) { + controllerId = updateMetadataRequest.controllerId match { + case id if id < 0 => None + case id => Some(id) + } aliveNodes.clear() aliveBrokers.clear() updateMetadataRequest.liveBrokers.asScala.foreach { broker => diff --git a/core/src/main/scala/kafka/server/ReplicaManager.scala b/core/src/main/scala/kafka/server/ReplicaManager.scala index 9bbd29eaad307..888912bef911d 100644 --- a/core/src/main/scala/kafka/server/ReplicaManager.scala +++ b/core/src/main/scala/kafka/server/ReplicaManager.scala @@ -28,6 +28,7 @@ import kafka.log.{LogAppendInfo, LogManager} import kafka.message.{ByteBufferMessageSet, InvalidMessageException, Message, MessageSet} import kafka.metrics.KafkaMetricsGroup import kafka.utils._ +import org.I0Itec.zkclient.IZkChildListener import org.apache.kafka.common.errors.{OffsetOutOfRangeException, RecordBatchTooLargeException, ReplicaNotAvailableException, RecordTooLargeException, InvalidTopicException, ControllerMovedException, NotLeaderForPartitionException, CorruptRecordException, UnknownTopicOrPartitionException, InvalidTimestampException} @@ -39,7 +40,6 @@ import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.utils.{Time => JTime} import scala.collection._ import scala.collection.JavaConverters._ -import org.apache.kafka.common.internals.TopicConstants /* * Result metadata of a log append operation on the log @@ -394,7 +394,7 @@ class ReplicaManager(val config: KafkaConfig, BrokerTopicStats.getBrokerAllTopicsStats().totalProduceRequestRate.mark() // reject appending to internal topics if it is not allowed - if (TopicConstants.INTERNAL_TOPICS.contains(topicPartition.topic) && !internalTopicsAllowed) { + if (Topic.isInternal(topicPartition.topic) && !internalTopicsAllowed) { (topicPartition, LogAppendResult( LogAppendInfo.UnknownLogAppendInfo, Some(new InvalidTopicException("Cannot append to internal topic %s".format(topicPartition.topic))))) diff --git a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala new file mode 100644 index 0000000000000..3d05c1d34629e --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala @@ -0,0 +1,106 @@ +/** + * 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.server + +import java.io.{DataInputStream, DataOutputStream} +import java.net.Socket +import java.nio.ByteBuffer +import java.util.Properties + +import kafka.integration.KafkaServerTestHarness +import kafka.network.SocketServer +import kafka.utils._ +import org.apache.kafka.common.protocol.{ApiKeys, SecurityProtocol} +import org.apache.kafka.common.requests.{AbstractRequest, RequestHeader, ResponseHeader} +import org.junit.Before + +abstract class BaseRequestTest extends KafkaServerTestHarness { + val numBrokers = 3 + private var correlationId = 0 + + // Override properties by mutating the passed Properties object + def propertyOverrides(properties: Properties): Unit + + def generateConfigs() = { + val props = TestUtils.createBrokerConfigs(numBrokers, zkConnect, enableControlledShutdown = false) + props.foreach(propertyOverrides) + props.map(KafkaConfig.fromProps) + } + + @Before + override def setUp() { + super.setUp() + TestUtils.waitUntilTrue(() => servers.head.metadataCache.getAliveBrokers.size == numBrokers, "Wait for cache to update") + } + + def socketServer = { + servers.find { server => + val state = server.brokerState.currentState + state != NotRunning.state && state != BrokerShuttingDown.state + }.map(_.socketServer).getOrElse(throw new IllegalStateException("No live broker is available")) + } + + private def connect(s: SocketServer = socketServer, protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): Socket = { + new Socket("localhost", s.boundPort(protocol)) + } + + private def sendRequest(socket: Socket, request: Array[Byte]) { + val outgoing = new DataOutputStream(socket.getOutputStream) + outgoing.writeInt(request.length) + outgoing.write(request) + outgoing.flush() + } + + private def receiveResponse(socket: Socket): Array[Byte] = { + val incoming = new DataInputStream(socket.getInputStream) + val len = incoming.readInt() + val response = new Array[Byte](len) + incoming.readFully(response) + response + } + + private def requestAndReceive(request: Array[Byte]): Array[Byte] = { + val plainSocket = connect() + try { + sendRequest(plainSocket, request) + receiveResponse(plainSocket) + } finally { + plainSocket.close() + } + } + + /** + * Serializes and send the request to the given api. A ByteBuffer containing the response is returned. + */ + def send(request: AbstractRequest, apiKey: ApiKeys, version: Short): ByteBuffer = { + correlationId += 1 + val serializedBytes = { + val header = new RequestHeader(apiKey.id, version, "", correlationId) + val byteBuffer = ByteBuffer.allocate(header.sizeOf() + request.sizeOf) + header.writeTo(byteBuffer) + request.writeTo(byteBuffer) + byteBuffer.array() + } + + val response = requestAndReceive(serializedBytes) + + val responseBuffer = ByteBuffer.wrap(response) + ResponseHeader.parse(responseBuffer) // Parse the header to ensure its valid and move the buffer forward + responseBuffer + } +} diff --git a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala index 017faeabaa617..770513c5cfe63 100644 --- a/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala +++ b/core/src/test/scala/unit/kafka/server/MetadataCacheTest.scala @@ -158,7 +158,8 @@ class MetadataCacheTest { val updateMetadataRequest = new UpdateMetadataRequest(controllerId, controllerEpoch, partitionStates.asJava, brokers.asJava) cache.updateCache(15, updateMetadataRequest) - val topicMetadatas = cache.getTopicMetadata(Set(topic), SecurityProtocol.PLAINTEXT) + // Validate errorUnavailableEndpoints = false + val topicMetadatas = cache.getTopicMetadata(Set(topic), SecurityProtocol.PLAINTEXT, errorUnavailableEndpoints = false) assertEquals(1, topicMetadatas.size) val topicMetadata = topicMetadatas.head @@ -169,9 +170,25 @@ class MetadataCacheTest { val partitionMetadata = partitionMetadatas.get(0) assertEquals(0, partitionMetadata.partition) - assertEquals(Errors.REPLICA_NOT_AVAILABLE, partitionMetadata.error) - assertEquals(Set(0), partitionMetadata.replicas.asScala.map(_.id).toSet) + assertEquals(Errors.NONE, partitionMetadata.error) + assertEquals(Set(0, 1), partitionMetadata.replicas.asScala.map(_.id).toSet) assertEquals(Set(0), partitionMetadata.isr.asScala.map(_.id).toSet) + + // Validate errorUnavailableEndpoints = true + val topicMetadatasWithError = cache.getTopicMetadata(Set(topic), SecurityProtocol.PLAINTEXT, errorUnavailableEndpoints = true) + assertEquals(1, topicMetadatasWithError.size) + + val topicMetadataWithError = topicMetadatasWithError.head + assertEquals(Errors.NONE, topicMetadataWithError.error) + + val partitionMetadatasWithError = topicMetadataWithError.partitionMetadata + assertEquals(1, partitionMetadatasWithError.size) + + val partitionMetadataWithError = partitionMetadatasWithError.get(0) + assertEquals(0, partitionMetadataWithError.partition) + assertEquals(Errors.REPLICA_NOT_AVAILABLE, partitionMetadataWithError.error) + assertEquals(Set(0), partitionMetadataWithError.replicas.asScala.map(_.id).toSet) + assertEquals(Set(0), partitionMetadataWithError.isr.asScala.map(_.id).toSet) } @Test @@ -197,7 +214,8 @@ class MetadataCacheTest { val updateMetadataRequest = new UpdateMetadataRequest(controllerId, controllerEpoch, partitionStates.asJava, brokers.asJava) cache.updateCache(15, updateMetadataRequest) - val topicMetadatas = cache.getTopicMetadata(Set(topic), SecurityProtocol.PLAINTEXT) + // Validate errorUnavailableEndpoints = false + val topicMetadatas = cache.getTopicMetadata(Set(topic), SecurityProtocol.PLAINTEXT, errorUnavailableEndpoints = false) assertEquals(1, topicMetadatas.size) val topicMetadata = topicMetadatas.head @@ -208,9 +226,25 @@ class MetadataCacheTest { val partitionMetadata = partitionMetadatas.get(0) assertEquals(0, partitionMetadata.partition) - assertEquals(Errors.REPLICA_NOT_AVAILABLE, partitionMetadata.error) + assertEquals(Errors.NONE, partitionMetadata.error) assertEquals(Set(0), partitionMetadata.replicas.asScala.map(_.id).toSet) - assertEquals(Set(0), partitionMetadata.isr.asScala.map(_.id).toSet) + assertEquals(Set(0, 1), partitionMetadata.isr.asScala.map(_.id).toSet) + + // Validate errorUnavailableEndpoints = true + val topicMetadatasWithError = cache.getTopicMetadata(Set(topic), SecurityProtocol.PLAINTEXT, errorUnavailableEndpoints = true) + assertEquals(1, topicMetadatasWithError.size) + + val topicMetadataWithError = topicMetadatasWithError.head + assertEquals(Errors.NONE, topicMetadataWithError.error) + + val partitionMetadatasWithError = topicMetadataWithError.partitionMetadata + assertEquals(1, partitionMetadatasWithError.size) + + val partitionMetadataWithError = partitionMetadatasWithError.get(0) + assertEquals(0, partitionMetadataWithError.partition) + assertEquals(Errors.REPLICA_NOT_AVAILABLE, partitionMetadataWithError.error) + assertEquals(Set(0), partitionMetadataWithError.replicas.asScala.map(_.id).toSet) + assertEquals(Set(0), partitionMetadataWithError.isr.asScala.map(_.id).toSet) } @Test diff --git a/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala new file mode 100644 index 0000000000000..3d4b40c6a9251 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/MetadataRequestTest.scala @@ -0,0 +1,168 @@ +/** + * 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.server + +import java.util.Properties + +import kafka.utils.TestUtils +import org.apache.kafka.common.internals.TopicConstants +import org.apache.kafka.common.protocol.{ApiKeys, Errors} +import org.apache.kafka.common.requests.{MetadataRequest, MetadataResponse} +import org.junit.Assert._ +import org.junit.Test + +import scala.collection.JavaConverters._ + +class MetadataRequestTest extends BaseRequestTest { + + override def propertyOverrides(properties: Properties) { + properties.setProperty(KafkaConfig.RackProp, s"rack/${properties.getProperty(KafkaConfig.BrokerIdProp)}") + } + + @Test + def testControllerId() { + val controllerServer = servers.find(_.kafkaController.isActive()).get + val controllerId = controllerServer.config.brokerId + val metadataResponse = sendMetadataRequest(MetadataRequest.allTopics(), 1) + + assertEquals("Controller id should match the active controller", + controllerId, metadataResponse.controller.id) + + // Fail over the controller + controllerServer.shutdown() + controllerServer.startup() + + val controllerServer2 = servers.find(_.kafkaController.isActive()).get + val controllerId2 = controllerServer2.config.brokerId + assertNotEquals("Controller id should switch to a new broker", controllerId, controllerId2) + TestUtils.waitUntilTrue(() => { + val metadataResponse2 = sendMetadataRequest(MetadataRequest.allTopics(), 1) + controllerServer2.apis.brokerId == metadataResponse2.controller.id + }, "Controller id should match the active controller after failover", 5000) + } + + @Test + def testRack() { + val metadataResponse = sendMetadataRequest(MetadataRequest.allTopics(), 1) + // Validate rack matches what's set in generateConfigs() above + metadataResponse.brokers.asScala.foreach { broker => + assertEquals("Rack information should match config", s"rack/${broker.id}", broker.rack) + } + } + + @Test + def testIsInternal() { + val internalTopic = TopicConstants.GROUP_METADATA_TOPIC_NAME + val notInternalTopic = "notInternal" + // create the topics + TestUtils.createTopic(zkUtils, internalTopic, 3, 2, servers) + TestUtils.createTopic(zkUtils, notInternalTopic, 3, 2, servers) + + val metadataResponse = sendMetadataRequest(MetadataRequest.allTopics(), 1) + assertTrue("Response should have no errors", metadataResponse.errors.isEmpty) + + val topicMetadata = metadataResponse.topicMetadata.asScala + val internalTopicMetadata = topicMetadata.find(_.topic == internalTopic).get + val notInternalTopicMetadata = topicMetadata.find(_.topic == notInternalTopic).get + + assertTrue("internalTopic should show isInternal", internalTopicMetadata.isInternal) + assertFalse("notInternalTopic topic not should show isInternal", notInternalTopicMetadata.isInternal) + } + + @Test + def testNoTopicsRequest() { + // create some topics + TestUtils.createTopic(zkUtils, "t1", 3, 2, servers) + TestUtils.createTopic(zkUtils, "t2", 3, 2, servers) + + // v0, Doesn't support a "no topics" request + // v1, Empty list represents "no topics" + val metadataResponse = sendMetadataRequest(new MetadataRequest(List[String]().asJava), 1) + assertTrue("Response should have no errors", metadataResponse.errors.isEmpty) + assertTrue("Response should have no topics", metadataResponse.topicMetadata.isEmpty) + } + + @Test + def testAllTopicsRequest() { + // create some topics + TestUtils.createTopic(zkUtils, "t1", 3, 2, servers) + TestUtils.createTopic(zkUtils, "t2", 3, 2, servers) + + // v0, Empty list represents all topics + val metadataResponseV0 = sendMetadataRequest(new MetadataRequest(List[String]().asJava), 0) + assertTrue("V0 Response should have no errors", metadataResponseV0.errors.isEmpty) + assertEquals("V0 Response should have 2 (all) topics", 2, metadataResponseV0.topicMetadata.size()) + + // v1, Null represents all topics + val metadataResponseV1 = sendMetadataRequest(MetadataRequest.allTopics(), 1) + assertTrue("V1 Response should have no errors", metadataResponseV1.errors.isEmpty) + assertEquals("V1 Response should have 2 (all) topics", 2, metadataResponseV1.topicMetadata.size()) + } + + @Test + def testReplicaDownResponse() { + val replicaDownTopic = "replicaDown" + val replicaCount = 3 + + // create a topic with 3 replicas + TestUtils.createTopic(zkUtils, replicaDownTopic, 1, replicaCount, servers) + + // Kill a replica node that is not the leader + val metadataResponse = sendMetadataRequest(new MetadataRequest(List(replicaDownTopic).asJava), 1) + val partitionMetadata = metadataResponse.topicMetadata.asScala.head.partitionMetadata.asScala.head + val downNode = servers.find { server => + val serverId = server.apis.brokerId + val leaderId = partitionMetadata.leader.id + val replicaIds = partitionMetadata.replicas.asScala.map(_.id) + serverId != leaderId && replicaIds.contains(serverId) + }.get + downNode.shutdown() + + TestUtils.waitUntilTrue(() => { + val response = sendMetadataRequest(new MetadataRequest(List(replicaDownTopic).asJava), 1) + val metadata = response.topicMetadata.asScala.head.partitionMetadata.asScala.head + val replica = metadata.replicas.asScala.find(_.id == downNode.apis.brokerId).get + replica.host == "" & replica.port == -1 + }, "Replica was not found down", 5000) + + // Validate version 0 still filters unavailable replicas and contains error + val v0MetadataResponse = sendMetadataRequest(new MetadataRequest(List(replicaDownTopic).asJava), 0) + val v0BrokerIds = v0MetadataResponse.brokers().asScala.map(_.id).toSeq + assertTrue("Response should have no errors", v0MetadataResponse.errors.isEmpty) + assertFalse(s"The downed broker should not be in the brokers list", v0BrokerIds.contains(downNode)) + assertTrue("Response should have one topic", v0MetadataResponse.topicMetadata.size == 1) + val v0PartitionMetadata = v0MetadataResponse.topicMetadata.asScala.head.partitionMetadata.asScala.head + assertTrue("PartitionMetadata should have an error", v0PartitionMetadata.error == Errors.REPLICA_NOT_AVAILABLE) + assertTrue(s"Response should have ${replicaCount - 1} replicas", v0PartitionMetadata.replicas.size == replicaCount - 1) + + // Validate version 1 returns unavailable replicas with no error + val v1MetadataResponse = sendMetadataRequest(new MetadataRequest(List(replicaDownTopic).asJava), 1) + val v1BrokerIds = v1MetadataResponse.brokers().asScala.map(_.id).toSeq + assertTrue("Response should have no errors", v1MetadataResponse.errors.isEmpty) + assertFalse(s"The downed broker should not be in the brokers list", v1BrokerIds.contains(downNode)) + assertEquals("Response should have one topic", 1, v1MetadataResponse.topicMetadata.size) + val v1PartitionMetadata = v1MetadataResponse.topicMetadata.asScala.head.partitionMetadata.asScala.head + assertEquals("PartitionMetadata should have no errors", Errors.NONE, v1PartitionMetadata.error) + assertEquals(s"Response should have $replicaCount replicas", replicaCount, v1PartitionMetadata.replicas.size) + } + + private def sendMetadataRequest(request: MetadataRequest, version: Short): MetadataResponse = { + val response = send(request, ApiKeys.METADATA, version) + MetadataResponse.parse(response, version) + } +} From 77fa0b116f0cc8aba37884a1a5d487f8683f7845 Mon Sep 17 00:00:00 2001 From: Manikumar reddy O Date: Wed, 27 Apr 2016 00:03:00 -0700 Subject: [PATCH 180/206] KAFKA-3406; Update CommonClientConfigs.RETRY_BACKOFF_MS_DOC doc string Author: Manikumar reddy O Reviewers: Sriharsha Chintalapani , Ismael Juma Closes #1230 from omkreddy/KAFKA-3406 --- .../main/java/org/apache/kafka/clients/CommonClientConfigs.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java index 298e1d8c541be..2f1fe93a5ac08 100644 --- a/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java +++ b/clients/src/main/java/org/apache/kafka/clients/CommonClientConfigs.java @@ -50,7 +50,7 @@ public class CommonClientConfigs { public static final String RECONNECT_BACKOFF_MS_DOC = "The amount of time to wait before attempting to reconnect to a given host. This avoids repeatedly connecting to a host in a tight loop. This backoff applies to all requests sent by the consumer to the broker."; public static final String RETRY_BACKOFF_MS_CONFIG = "retry.backoff.ms"; - public static final String RETRY_BACKOFF_MS_DOC = "The amount of time to wait before attempting to retry a failed fetch request to a given topic partition. This avoids repeated fetching-and-failing in a tight loop."; + public static final String RETRY_BACKOFF_MS_DOC = "The amount of time to wait before attempting to retry a failed request to a given topic partition. This avoids repeatedly sending requests in a tight loop under some failure scenarios."; public static final String METRICS_SAMPLE_WINDOW_MS_CONFIG = "metrics.sample.window.ms"; public static final String METRICS_SAMPLE_WINDOW_MS_DOC = "The window of time a metrics sample is computed over."; From 2885bc33daaf75477bf39a92d1d1da02c0e03eaa Mon Sep 17 00:00:00 2001 From: Manikumar reddy O Date: Wed, 27 Apr 2016 06:53:16 -0700 Subject: [PATCH 181/206] KAFKA-3580; Improve error logging in ReplicaFetchThread Author: Manikumar reddy O Reviewers: Ismael Juma Closes #1237 from omkreddy/KAFKA-3580 --- .../scala/kafka/server/ReplicaFetcherThread.scala | 12 ++++++------ 1 file changed, 6 insertions(+), 6 deletions(-) diff --git a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala index 84f2e123f9c9f..d58f120738406 100644 --- a/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala +++ b/core/src/main/scala/kafka/server/ReplicaFetcherThread.scala @@ -115,10 +115,10 @@ class ReplicaFetcherThread(name: String, val TopicAndPartition(topic, partitionId) = topicAndPartition val replica = replicaMgr.getReplica(topic, partitionId).get val messageSet = partitionData.toByteBufferMessageSet - warnIfMessageOversized(messageSet) + warnIfMessageOversized(messageSet, topicAndPartition) if (fetchOffset != replica.logEndOffset.messageOffset) - throw new RuntimeException("Offset mismatch: fetched offset = %d, log end offset = %d.".format(fetchOffset, replica.logEndOffset.messageOffset)) + throw new RuntimeException("Offset mismatch for partition %s: fetched offset = %d, log end offset = %d.".format(topicAndPartition, fetchOffset, replica.logEndOffset.messageOffset)) if (logger.isTraceEnabled) trace("Follower %d has replica log end offset %d for partition %s. Received %d messages and leader hw %d" .format(replica.brokerId, replica.logEndOffset.messageOffset, topicAndPartition, messageSet.sizeInBytes, partitionData.highWatermark)) @@ -136,15 +136,15 @@ class ReplicaFetcherThread(name: String, .format(replica.brokerId, topic, partitionId, followerHighWatermark)) } catch { case e: KafkaStorageException => - fatal("Disk error while replicating data.", e) + fatal(s"Disk error while replicating data for $topicAndPartition", e) Runtime.getRuntime.halt(1) } } - def warnIfMessageOversized(messageSet: ByteBufferMessageSet): Unit = { + def warnIfMessageOversized(messageSet: ByteBufferMessageSet, topicAndPartition: TopicAndPartition): Unit = { if (messageSet.sizeInBytes > 0 && messageSet.validBytes <= 0) - error("Replication is failing due to a message that is greater than replica.fetch.max.bytes. This " + - "generally occurs when the max.message.bytes has been overridden to exceed this value and a suitably large " + + error(s"Replication is failing due to a message that is greater than replica.fetch.max.bytes for partition $topicAndPartition. " + + "This generally occurs when the max.message.bytes has been overridden to exceed this value and a suitably large " + "message has also been sent. To fix this problem increase replica.fetch.max.bytes in your broker config to be " + "equal or larger than your settings for max.message.bytes, both at a broker and topic level.") } From 4c76b5fa6a72412efa5936c284800148c2c69c24 Mon Sep 17 00:00:00 2001 From: Damian Guy Date: Wed, 27 Apr 2016 10:50:20 -0700 Subject: [PATCH 182/206] KAFKA-3629; KStreamImpl.to(...) throws NPE when the value SerDe is null guozhangwang Author: Damian Guy Reviewers: Matthias J. Sax, Guozhang Wang Closes #1272 from dguy/kstreamimpl-to-npe and squashes the following commits: 49d48fb [Damian Guy] actually commit the fix 07ce589 [Damian Guy] fix npe in KStreamImpl.to(..) 74d396d [Damian Guy] fix npe in KStreamImpl.to(..) --- .../kafka/streams/kstream/internals/KStreamImpl.java | 2 +- .../kafka/streams/kstream/internals/KStreamImplTest.java | 7 +++++++ 2 files changed, 8 insertions(+), 1 deletion(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java index a84b4aa980d5b..91bcef94eb35d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamImpl.java @@ -298,7 +298,7 @@ public void to(Serde keySerde, Serde valSerde, StreamPartitioner par String name = topology.newName(SINK_NAME); Serializer keySerializer = keySerde == null ? null : keySerde.serializer(); - Serializer valSerializer = keySerde == null ? null : valSerde.serializer(); + Serializer valSerializer = valSerde == null ? null : valSerde.serializer(); if (partitioner == null && keySerializer != null && keySerializer instanceof WindowedSerializer) { WindowedSerializer windowedSerializer = (WindowedSerializer) keySerializer; diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java index b5c3d47a80b10..3d45d1dcc8a29 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamImplTest.java @@ -133,4 +133,11 @@ public Integer apply(Integer value1, Integer value2) { 1, // process builder.build("X", null).processors().size()); } + + @Test + public void testToWithNullValueSerdeDoesntNPE() { + final KStreamBuilder builder = new KStreamBuilder(); + final KStream inputStream = builder.stream(stringSerde, stringSerde, "input"); + inputStream.to(stringSerde, null, "output"); + } } From 8407dac6ee409d832c95533e6f1d5578511232ae Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Wed, 27 Apr 2016 11:28:32 -0700 Subject: [PATCH 183/206] KAFKA-3307; Add ApiVersions Request/Response and server side handling. The patch does the following. 1. Adds ApiVersionsRequest/Response. 2. Adds UNSUPPORTED_VERSION error and UnsupportedVersionException. 3. Adds broker side handling of ApiVersionsRequest. Author: Ashish Singh Reviewers: Gwen Shapira, Ismael Juma, Magnus Edenhill Closes #986 from SinghAsDev/KAFKA-3307 --- .../errors/UnsupportedVersionException.java | 25 ++++ .../apache/kafka/common/protocol/ApiKeys.java | 5 +- .../apache/kafka/common/protocol/Errors.java | 5 +- .../kafka/common/protocol/Protocol.java | 41 ++++++- .../common/requests/AbstractRequest.java | 4 +- .../common/requests/ApiVersionsRequest.java | 55 +++++++++ .../common/requests/ApiVersionsResponse.java | 116 ++++++++++++++++++ .../common/requests/RequestResponseTest.java | 16 ++- .../main/scala/kafka/server/KafkaApis.scala | 77 ++++++++---- .../unit/kafka/server/ApiVersionsTest.scala | 51 ++++++++ 10 files changed, 357 insertions(+), 38 deletions(-) create mode 100644 clients/src/main/java/org/apache/kafka/common/errors/UnsupportedVersionException.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java create mode 100644 clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java create mode 100644 core/src/test/scala/unit/kafka/server/ApiVersionsTest.scala diff --git a/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedVersionException.java b/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedVersionException.java new file mode 100644 index 0000000000000..3679be4fc31be --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/errors/UnsupportedVersionException.java @@ -0,0 +1,25 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.common.errors; + +public class UnsupportedVersionException extends ApiException { + private static final long serialVersionUID = 1L; + + public UnsupportedVersionException(String message, Throwable cause) { + super(message, cause); + } + + public UnsupportedVersionException(String message) { + super(message); + } +} diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java index 512a1211b819a..aeb0b453b2b06 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/ApiKeys.java @@ -37,7 +37,8 @@ public enum ApiKeys { SYNC_GROUP(14, "SyncGroup"), DESCRIBE_GROUPS(15, "DescribeGroups"), LIST_GROUPS(16, "ListGroups"), - SASL_HANDSHAKE(17, "SaslHandshake"); + SASL_HANDSHAKE(17, "SaslHandshake"), + API_VERSIONS(18, "ApiVersions"); private static final ApiKeys[] ID_TO_TYPE; private static final int MIN_API_KEY = 0; @@ -97,4 +98,4 @@ public static void main(String[] args) { System.out.println(toHtml()); } -} +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java index 901339903a1e2..64a709e5eeaaa 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Errors.java @@ -53,6 +53,7 @@ import org.apache.kafka.common.errors.UnsupportedSaslMechanismException; import org.apache.kafka.common.errors.TimeoutException; import org.apache.kafka.common.errors.TopicAuthorizationException; +import org.apache.kafka.common.errors.UnsupportedVersionException; import org.apache.kafka.common.errors.UnknownMemberIdException; import org.apache.kafka.common.errors.UnknownServerException; import org.apache.kafka.common.errors.UnknownTopicOrPartitionException; @@ -136,7 +137,9 @@ public enum Errors { UNSUPPORTED_SASL_MECHANISM(33, new UnsupportedSaslMechanismException("The broker does not support the requested SASL mechanism.")), ILLEGAL_SASL_STATE(34, - new IllegalSaslStateException("Request is not valid given the current SASL state.")); + new IllegalSaslStateException("Request is not valid given the current SASL state.")), + UNSUPPORTED_VERSION(35, + new UnsupportedVersionException("The version of API is not supported.")); private static final Logger log = LoggerFactory.getLogger(Errors.class); diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index d32209566677e..99cdbf9f57988 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -759,10 +759,24 @@ public class Protocol { public static final Schema[] SASL_HANDSHAKE_REQUEST = new Schema[] {SASL_HANDSHAKE_REQUEST_V0}; public static final Schema[] SASL_HANDSHAKE_RESPONSE = new Schema[] {SASL_HANDSHAKE_RESPONSE_V0}; + /* ApiVersion api */ + public static final Schema API_VERSIONS_REQUEST_V0 = new Schema(); + + public static final Schema API_VERSIONS_V0 = new Schema(new Field("api_key", INT16, "API key."), + new Field("min_version", INT16, "Minimum supported version."), + new Field("max_version", INT16, "Maximum supported version.")); + + public static final Schema API_VERSIONS_RESPONSE_V0 = new Schema(new Field("error_code", INT16, "Error code."), + new Field("api_versions", new ArrayOf(API_VERSIONS_V0), "API versions supported by the broker.")); + + public static final Schema[] API_VERSIONS_REQUEST = new Schema[]{API_VERSIONS_REQUEST_V0}; + public static final Schema[] API_VERSIONS_RESPONSE = new Schema[]{API_VERSIONS_RESPONSE_V0}; + /* an array of all requests and responses with all schema versions; a null value in the inner array means that the * particular version is not supported */ public static final Schema[][] REQUESTS = new Schema[ApiKeys.MAX_API_KEY + 1][]; public static final Schema[][] RESPONSES = new Schema[ApiKeys.MAX_API_KEY + 1][]; + public static final short[] MIN_VERSIONS = new short[ApiKeys.MAX_API_KEY + 1]; /* the latest version of each api */ public static final short[] CURR_VERSION = new short[ApiKeys.MAX_API_KEY + 1]; @@ -786,6 +800,7 @@ public class Protocol { REQUESTS[ApiKeys.DESCRIBE_GROUPS.id] = DESCRIBE_GROUPS_REQUEST; REQUESTS[ApiKeys.LIST_GROUPS.id] = LIST_GROUPS_REQUEST; REQUESTS[ApiKeys.SASL_HANDSHAKE.id] = SASL_HANDSHAKE_REQUEST; + REQUESTS[ApiKeys.API_VERSIONS.id] = API_VERSIONS_REQUEST; RESPONSES[ApiKeys.PRODUCE.id] = PRODUCE_RESPONSE; RESPONSES[ApiKeys.FETCH.id] = FETCH_RESPONSE; @@ -805,16 +820,32 @@ public class Protocol { RESPONSES[ApiKeys.DESCRIBE_GROUPS.id] = DESCRIBE_GROUPS_RESPONSE; RESPONSES[ApiKeys.LIST_GROUPS.id] = LIST_GROUPS_RESPONSE; RESPONSES[ApiKeys.SASL_HANDSHAKE.id] = SASL_HANDSHAKE_RESPONSE; + RESPONSES[ApiKeys.API_VERSIONS.id] = API_VERSIONS_RESPONSE; - /* set the maximum version of each api */ - for (ApiKeys api : ApiKeys.values()) + /* set the minimum and maximum version of each api */ + for (ApiKeys api : ApiKeys.values()) { CURR_VERSION[api.id] = (short) (REQUESTS[api.id].length - 1); + for (int i = 0; i < REQUESTS[api.id].length; ++i) + if (REQUESTS[api.id][i] != null) { + MIN_VERSIONS[api.id] = (short) i; + break; + } + } - /* sanity check that we have the same number of request and response versions for each api */ - for (ApiKeys api : ApiKeys.values()) + /* sanity check that: + * - we have the same number of request and response versions for each api + * - we have a consistent set 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."); + + for (int i = 0; i < REQUESTS[api.id].length; ++i) + if ((REQUESTS[api.id][i] == null && RESPONSES[api.id][i] != null) || + (REQUESTS[api.id][i] != null && RESPONSES[api.id][i] == null)) + throw new IllegalStateException("Request and response for version " + i + " of API " + + api.id + " are defined inconsistently. One is null while the other is not null."); + } } private static String indentString(int size) { @@ -977,4 +1008,4 @@ public static void main(String[] args) { System.out.println(toHtml()); } -} +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java index 89c2ce15bb279..ab61c6634271e 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/AbstractRequest.java @@ -74,9 +74,11 @@ public static AbstractRequest getRequest(int requestId, int versionId, ByteBuffe return ListGroupsRequest.parse(buffer, versionId); case SASL_HANDSHAKE: return SaslHandshakeRequest.parse(buffer, versionId); + case API_VERSIONS: + return ApiVersionsRequest.parse(buffer, versionId); default: throw new AssertionError(String.format("ApiKey %s is not currently handled in `getRequest`, the " + "code should be updated to do so.", apiKey)); } } -} +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java new file mode 100644 index 0000000000000..b78c759e64462 --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsRequest.java @@ -0,0 +1,55 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more contributor license agreements. See the NOTICE + * file distributed with this work for additional information regarding copyright ownership. The ASF licenses this file + * to You under the Apache License, Version 2.0 (the "License"); you may not use this file except in compliance with the + * License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software distributed under the License is distributed on + * an "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the License for the + * specific language governing permissions and limitations under the License. + */ +package org.apache.kafka.common.requests; + +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.Schema; +import org.apache.kafka.common.protocol.types.Struct; + +import java.nio.ByteBuffer; +import java.util.Collections; + +public class ApiVersionsRequest extends AbstractRequest { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentRequestSchema(ApiKeys.API_VERSIONS.id); + + public ApiVersionsRequest() { + super(new Struct(CURRENT_SCHEMA)); + } + + public ApiVersionsRequest(Struct struct) { + super(struct); + } + + @Override + public AbstractRequestResponse getErrorResponse(int versionId, Throwable e) { + switch (versionId) { + case 0: + short errorCode = Errors.forException(e).code(); + return new ApiVersionsResponse(errorCode, Collections.emptyList()); + default: + throw new IllegalArgumentException(String.format("Version %d is not valid. Valid versions for %s are 0 to %d", + versionId, this.getClass().getSimpleName(), ProtoUtils.latestVersion(ApiKeys.API_VERSIONS.id))); + } + } + + public static ApiVersionsRequest parse(ByteBuffer buffer, int versionId) { + return new ApiVersionsRequest(ProtoUtils.parseRequest(ApiKeys.API_VERSIONS.id, versionId, buffer)); + } + + public static ApiVersionsRequest parse(ByteBuffer buffer) { + return new ApiVersionsRequest(CURRENT_SCHEMA.read(buffer)); + } +} \ No newline at end of file diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java new file mode 100644 index 0000000000000..36881a3e090cb --- /dev/null +++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java @@ -0,0 +1,116 @@ +/** + * 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 org.apache.kafka.common.requests; + +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.Schema; +import org.apache.kafka.common.protocol.types.Struct; + +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +public class ApiVersionsResponse extends AbstractRequestResponse { + + private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.API_VERSIONS.id); + + public static final String ERROR_CODE_KEY_NAME = "error_code"; + public static final String API_VERSIONS_KEY_NAME = "api_versions"; + public static final String API_KEY_NAME = "api_key"; + public static final String MIN_VERSION_KEY_NAME = "min_version"; + public static final String MAX_VERSION_KEY_NAME = "max_version"; + + /** + * Possible error codes: + * + * UNSUPPORTED_VERSION (33) + */ + private final short errorCode; + private final Map apiKeyToApiVersion; + + public static final class ApiVersion { + public final short apiKey; + public final short minVersion; + public final short maxVersion; + + public ApiVersion(short apiKey, short minVersion, short maxVersion) { + this.apiKey = apiKey; + this.minVersion = minVersion; + this.maxVersion = maxVersion; + } + } + + public ApiVersionsResponse(short errorCode, List apiVersions) { + super(new Struct(CURRENT_SCHEMA)); + struct.set(ERROR_CODE_KEY_NAME, errorCode); + List apiVersionList = new ArrayList<>(); + for (ApiVersion apiVersion : apiVersions) { + Struct apiVersionStruct = struct.instance(API_VERSIONS_KEY_NAME); + apiVersionStruct.set(API_KEY_NAME, apiVersion.apiKey); + apiVersionStruct.set(MIN_VERSION_KEY_NAME, apiVersion.minVersion); + apiVersionStruct.set(MAX_VERSION_KEY_NAME, apiVersion.maxVersion); + apiVersionList.add(apiVersionStruct); + } + struct.set(API_VERSIONS_KEY_NAME, apiVersionList.toArray()); + this.errorCode = errorCode; + this.apiKeyToApiVersion = buildApiKeyToApiVersion(apiVersions); + } + + public ApiVersionsResponse(Struct struct) { + super(struct); + this.errorCode = struct.getShort(ERROR_CODE_KEY_NAME); + List tempApiVersions = new ArrayList<>(); + for (Object apiVersionsObj : struct.getArray(API_VERSIONS_KEY_NAME)) { + Struct apiVersionStruct = (Struct) apiVersionsObj; + short apiKey = apiVersionStruct.getShort(API_KEY_NAME); + short minVersion = apiVersionStruct.getShort(MIN_VERSION_KEY_NAME); + short maxVersion = apiVersionStruct.getShort(MAX_VERSION_KEY_NAME); + tempApiVersions.add(new ApiVersion(apiKey, minVersion, maxVersion)); + } + this.apiKeyToApiVersion = buildApiKeyToApiVersion(tempApiVersions); + } + + public Collection apiVersions() { + return apiKeyToApiVersion.values(); + } + + public ApiVersion apiVersion(short apiKey) { + return apiKeyToApiVersion.get(apiKey); + } + + public short errorCode() { + return errorCode; + } + + public static ApiVersionsResponse parse(ByteBuffer buffer) { + return new ApiVersionsResponse(CURRENT_SCHEMA.read(buffer)); + } + + public static ApiVersionsResponse fromError(Errors error) { + return new ApiVersionsResponse(error.code(), Collections.emptyList()); + } + + private Map buildApiKeyToApiVersion(List apiVersions) { + Map tempApiIdToApiVersion = new HashMap<>(); + for (ApiVersion apiVersion: apiVersions) { + tempApiIdToApiVersion.put(apiVersion.apiKey, apiVersion); + } + return tempApiIdToApiVersion; + } +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 0018f531f405f..345de3f957ae0 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -95,7 +95,10 @@ public void testSerialization() throws Exception { createLeaderAndIsrResponse(), createSaslHandshakeRequest(), createSaslHandshakeRequest().getErrorResponse(0, new UnknownServerException()), - createSaslHandshakeResponse() + createSaslHandshakeResponse(), + createApiVersionRequest(), + createApiVersionRequest().getErrorResponse(0, new UnknownServerException()), + createApiVersionResponse() ); for (AbstractRequestResponse req : requestResponseList) @@ -438,4 +441,13 @@ private AbstractRequest createSaslHandshakeRequest() { private AbstractRequestResponse createSaslHandshakeResponse() { return new SaslHandshakeResponse(Errors.NONE.code(), Collections.singletonList("GSSAPI")); } -} + + private AbstractRequest createApiVersionRequest() { + return new ApiVersionsRequest(); + } + + private AbstractRequestResponse createApiVersionResponse() { + List apiVersions = Arrays.asList(new ApiVersionsResponse.ApiVersion((short) 0, (short) 0, (short) 2)); + return new ApiVersionsResponse(Errors.NONE.code(), apiVersions); + } +} \ No newline at end of file diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 406b1bdb70fef..67d46fc092706 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -21,7 +21,7 @@ import java.nio.ByteBuffer import java.lang.{Long => JLong, Short => JShort} import java.util.Properties -import kafka.admin.{RackAwareMode, AdminUtils} +import kafka.admin.{AdminUtils, RackAwareMode} import kafka.api._ import kafka.cluster.Partition import kafka.common @@ -31,27 +31,32 @@ import kafka.coordinator.{GroupCoordinator, JoinGroupResult} import kafka.log._ import kafka.message.{ByteBufferMessageSet, Message, MessageSet} import kafka.network._ -import kafka.network.RequestChannel.{Session, Response} -import kafka.security.auth.{Authorizer, ClusterAction, Group, Create, Describe, Operation, Read, Resource, Topic, Write} +import kafka.network.RequestChannel.{Response, Session} +import kafka.security.auth.{Authorizer, ClusterAction, Create, Describe, Group, Operation, Read, Resource, Topic, Write} import kafka.utils.{Logging, SystemTime, ZKGroupTopicDirs, ZkUtils} -import org.apache.kafka.common.errors.{InvalidTopicException, NotLeaderForPartitionException, UnknownTopicOrPartitionException, -ClusterAuthorizationException} +import org.apache.kafka.common.errors.{ClusterAuthorizationException, InvalidTopicException, NotLeaderForPartitionException, UnknownTopicOrPartitionException} import org.apache.kafka.common.metrics.Metrics -import org.apache.kafka.common.protocol.{ApiKeys, Errors, SecurityProtocol} -import org.apache.kafka.common.requests.{ListOffsetRequest, ListOffsetResponse, GroupCoordinatorRequest, GroupCoordinatorResponse, ListGroupsResponse, -DescribeGroupsRequest, DescribeGroupsResponse, HeartbeatRequest, HeartbeatResponse, JoinGroupRequest, JoinGroupResponse, -LeaveGroupRequest, LeaveGroupResponse, ResponseHeader, ResponseSend, SyncGroupRequest, SyncGroupResponse, LeaderAndIsrRequest, LeaderAndIsrResponse, -StopReplicaRequest, StopReplicaResponse, ProduceRequest, ProduceResponse, UpdateMetadataRequest, UpdateMetadataResponse, -MetadataRequest, MetadataResponse, OffsetCommitRequest, OffsetCommitResponse, OffsetFetchRequest, OffsetFetchResponse} +import org.apache.kafka.common.protocol.{ApiKeys, Errors, Protocol, SecurityProtocol} +import org.apache.kafka.common.requests.{ApiVersionsResponse, DescribeGroupsRequest, DescribeGroupsResponse, GroupCoordinatorRequest, GroupCoordinatorResponse, HeartbeatRequest, HeartbeatResponse, JoinGroupRequest, JoinGroupResponse, LeaderAndIsrRequest, LeaderAndIsrResponse, LeaveGroupRequest, LeaveGroupResponse, ListGroupsResponse, ListOffsetRequest, ListOffsetResponse, MetadataRequest, MetadataResponse, OffsetCommitRequest, OffsetCommitResponse, OffsetFetchRequest, OffsetFetchResponse, ProduceRequest, ProduceResponse, ResponseHeader, ResponseSend, StopReplicaRequest, StopReplicaResponse, SyncGroupRequest, SyncGroupResponse, UpdateMetadataRequest, UpdateMetadataResponse} import org.apache.kafka.common.requests.ProduceResponse.PartitionResponse import org.apache.kafka.common.utils.Utils -import org.apache.kafka.common.{TopicPartition, Node} +import org.apache.kafka.common.{Node, TopicPartition} import org.apache.kafka.common.internals.TopicConstants import scala.collection._ import scala.collection.JavaConverters._ import org.apache.kafka.common.requests.SaslHandshakeResponse +object KafkaApis { + val apiVersionsResponse = new ApiVersionsResponse(Errors.NONE.code, buildApiKeysToApiVersions.values.toList.asJava) + + private def buildApiKeysToApiVersions: Map[Short, ApiVersionsResponse.ApiVersion] = { + ApiKeys.values.map(apiKey => + apiKey.id -> new ApiVersionsResponse.ApiVersion(apiKey.id, Protocol.MIN_VERSIONS(apiKey.id), Protocol.CURR_VERSION(apiKey.id))).toMap + } +} + + /** * Logic to handle the various Kafka requests */ @@ -74,7 +79,7 @@ class KafkaApis(val requestChannel: RequestChannel, * Top-level method that handles all requests and multiplexes to the right api */ def handle(request: RequestChannel.Request) { - try{ + try { trace("Handling request:%s from connection %s;securityProtocol:%s,principal:%s". format(request.requestObj, request.connectionId, request.securityProtocol, request.session.principal)) ApiKeys.forId(request.requestId) match { @@ -96,6 +101,7 @@ class KafkaApis(val requestChannel: RequestChannel, case ApiKeys.DESCRIBE_GROUPS => handleDescribeGroupRequest(request) case ApiKeys.LIST_GROUPS => handleListGroupsRequest(request) case ApiKeys.SASL_HANDSHAKE => handleSaslHandshakeRequest(request) + case ApiKeys.API_VERSIONS => handleApiVersionsRequest(request) case requestId => throw new KafkaException("Unknown api code " + requestId) } } catch { @@ -143,7 +149,7 @@ class KafkaApis(val requestChannel: RequestChannel, } val responseHeader = new ResponseHeader(correlationId) - val leaderAndIsrResponse= + val leaderAndIsrResponse = if (authorize(request.session, ClusterAction, Resource.ClusterResource)) { val result = replicaManager.becomeLeaderOrFollower(correlationId, leaderAndIsrRequest, metadataCache, onLeadershipChange) new LeaderAndIsrResponse(result.errorCode, result.responseMap.mapValues(new JShort(_)).asJava) @@ -234,7 +240,7 @@ class KafkaApis(val requestChannel: RequestChannel, } val filteredRequestInfo = offsetCommitRequest.offsetData.asScala.toMap -- invalidRequestsInfo.keys - val (authorizedRequestInfo, unauthorizedRequestInfo) = filteredRequestInfo.partition { + val (authorizedRequestInfo, unauthorizedRequestInfo) = filteredRequestInfo.partition { case (topicPartition, offsetMetadata) => authorize(request.session, Read, new Resource(Topic, topicPartition.topic)) } @@ -251,7 +257,7 @@ class KafkaApis(val requestChannel: RequestChannel, val combinedCommitStatus = mergedCommitStatus.mapValues(new JShort(_)) ++ invalidRequestsInfo.map(_._1 -> new JShort(Errors.UNKNOWN_TOPIC_OR_PARTITION.code)) val responseHeader = new ResponseHeader(header.correlationId) - val responseBody = new OffsetCommitResponse(combinedCommitStatus.asJava) + val responseBody = new OffsetCommitResponse(combinedCommitStatus.asJava) requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, responseHeader, responseBody))) } @@ -376,7 +382,7 @@ class KafkaApis(val requestChannel: RequestChannel, val respHeader = new ResponseHeader(request.header.correlationId) val respBody = request.header.apiVersion match { case 0 => new ProduceResponse(mergedResponseStatus.asJava) - case version@ (1 | 2) => new ProduceResponse(mergedResponseStatus.asJava, delayTimeMs, version) + case version@(1 | 2) => new ProduceResponse(mergedResponseStatus.asJava, delayTimeMs, version) // This case shouldn't happen unless a new version of ProducerRequest is added without // updating this part of the code to handle it properly. case version => throw new IllegalArgumentException(s"Version `$version` of ProduceRequest is not handled. Code must be updated.") @@ -426,7 +432,7 @@ class KafkaApis(val requestChannel: RequestChannel, def handleFetchRequest(request: RequestChannel.Request) { val fetchRequest = request.requestObj.asInstanceOf[FetchRequest] - val (authorizedRequestInfo, unauthorizedRequestInfo) = fetchRequest.requestInfo.partition { + val (authorizedRequestInfo, unauthorizedRequestInfo) = fetchRequest.requestInfo.partition { case (topicAndPartition, _) => authorize(request.session, Read, new Resource(Topic, topicAndPartition.topic)) } @@ -552,14 +558,14 @@ class KafkaApis(val requestChannel: RequestChannel, case utpe: UnknownTopicOrPartitionException => debug("Offset request with correlation id %d from client %s on partition %s failed due to %s".format( correlationId, clientId, topicPartition, utpe.getMessage)) - (topicPartition, new ListOffsetResponse.PartitionData(Errors.forException(utpe).code, List[JLong]().asJava)) + (topicPartition, new ListOffsetResponse.PartitionData(Errors.forException(utpe).code, List[JLong]().asJava)) case nle: NotLeaderForPartitionException => debug("Offset request with correlation id %d from client %s on partition %s failed due to %s".format( correlationId, clientId, topicPartition,nle.getMessage)) - (topicPartition, new ListOffsetResponse.PartitionData(Errors.forException(nle).code, List[JLong]().asJava)) + (topicPartition, new ListOffsetResponse.PartitionData(Errors.forException(nle).code, List[JLong]().asJava)) case e: Throwable => error("Error while responding to offset request", e) - (topicPartition, new ListOffsetResponse.PartitionData(Errors.forException(e).code, List[JLong]().asJava)) + (topicPartition, new ListOffsetResponse.PartitionData(Errors.forException(e).code, List[JLong]().asJava)) } }) @@ -591,7 +597,7 @@ class KafkaApis(val requestChannel: RequestChannel, else offsetTimeArray = new Array[(Long, Long)](segsArray.length) - for(i <- 0 until segsArray.length) + for (i <- 0 until segsArray.length) offsetTimeArray(i) = (segsArray(i).baseOffset, segsArray(i).lastModified) if (segsArray.last.size > 0) offsetTimeArray(segsArray.length) = (log.logEndOffset, SystemTime.milliseconds) @@ -610,18 +616,18 @@ class KafkaApis(val requestChannel: RequestChannel, if (offsetTimeArray(startIndex)._2 <= timestamp) isFound = true else - startIndex -=1 + startIndex -= 1 } } val retSize = maxNumOffsets.min(startIndex + 1) val ret = new Array[Long](retSize) - for(j <- 0 until retSize) { + for (j <- 0 until retSize) { ret(j) = offsetTimeArray(startIndex)._1 startIndex -= 1 } // ensure that the returned seq is in descending order of offsets - ret.toSeq.sortBy(- _) + ret.toSeq.sortBy(-_) } private def createTopic(topic: String, @@ -871,7 +877,7 @@ class KafkaApis(val requestChannel: RequestChannel, ListGroupsResponse.fromError(Errors.CLUSTER_AUTHORIZATION_FAILED) } else { val (error, groups) = coordinator.handleListGroups() - val allGroups = groups.map{ group => new ListGroupsResponse.Group(group.groupId, group.protocolType) } + val allGroups = groups.map { group => new ListGroupsResponse.Group(group.groupId, group.protocolType) } new ListGroupsResponse(error.code, allGroups.asJava) } requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, responseHeader, responseBody))) @@ -1024,6 +1030,23 @@ class KafkaApis(val requestChannel: RequestChannel, requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, respHeader, response))) } + def handleApiVersionsRequest(request: RequestChannel.Request) { + // Note that broker returns its full list of supported ApiKeys and versions regardless of current + // authentication state (e.g., before SASL authentication on an SASL listener, do note that no + // Kafka protocol requests may take place on a SSL listener before the SSL handshake is finished). + // If this is considered to leak information about the broker version a workaround is to use SSL + // with client authentication which is performed at an earlier stage of the connection where the + // ApiVersionRequest is not available. + val responseHeader = new ResponseHeader(request.header.correlationId) + val isApiVersionsRequestVersionSupported = request.header.apiVersion <= Protocol.CURR_VERSION(ApiKeys.API_VERSIONS.id) && + request.header.apiVersion >= Protocol.MIN_VERSIONS(ApiKeys.API_VERSIONS.id) + val responseBody = if (isApiVersionsRequestVersionSupported) + KafkaApis.apiVersionsResponse + else + ApiVersionsResponse.fromError(Errors.UNSUPPORTED_VERSION) + requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, responseHeader, responseBody))) + } + def close() { quotaManagers.foreach { case (apiKey, quotaManager) => quotaManager.shutdown() @@ -1035,4 +1058,4 @@ class KafkaApis(val requestChannel: RequestChannel, if (!authorize(request.session, ClusterAction, Resource.ClusterResource)) throw new ClusterAuthorizationException(s"Request $request is not authorized.") } -} +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsTest.scala new file mode 100644 index 0000000000000..4429f26e72bf8 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ApiVersionsTest.scala @@ -0,0 +1,51 @@ +/** + * 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 unit.kafka.server + +import kafka.server.KafkaApis +import org.apache.kafka.common.protocol.{Protocol, ApiKeys} +import org.junit.Assert._ +import org.junit.Test + +class ApiVersionsTest { + + @Test + def testApiVersions { + val apiVersions = KafkaApis.apiVersionsResponse.apiVersions + assertEquals("API versions for all API keys must be maintained.", apiVersions.size, ApiKeys.values().length) + + for (key <- ApiKeys.values) { + val version = KafkaApis.apiVersionsResponse.apiVersion(key.id) + assertNotNull(s"Could not find ApiVersion for API ${key.name}", version) + assertEquals(s"Incorrect min version for Api ${key.name}.", version.minVersion, Protocol.MIN_VERSIONS(key.id)) + assertEquals(s"Incorrect max version for Api ${key.name}.", version.maxVersion, Protocol.CURR_VERSION(key.id)) + + // Check if versions less than min version are indeed set as null, i.e., deprecated. + for (i <- 0 until version.minVersion) { + assertNull(s"Request version $i for API ${version.apiKey} must be null.", Protocol.REQUESTS(version.apiKey)(i)) + assertNull(s"Response version $i for API ${version.apiKey} must be null.", Protocol.RESPONSES(version.apiKey)(i)) + } + + // Check if versions between min and max versions are non null, i.e., valid. + for (i <- version.minVersion.toInt to version.maxVersion) { + assertNotNull(s"Request version $i for API ${version.apiKey} must not be null.", Protocol.REQUESTS(version.apiKey)(i)) + assertNotNull(s"Response version $i for API ${version.apiKey} must not be null.", Protocol.RESPONSES(version.apiKey)(i)) + } + } + } +} \ No newline at end of file From 94aee2143ed5290d27cdd4072c6ae9bb70a6ba30 Mon Sep 17 00:00:00 2001 From: Eno Thereska Date: Wed, 27 Apr 2016 16:55:51 -0700 Subject: [PATCH 184/206] KAFKA-3612: Added structure for integration tests Author: Eno Thereska Reviewers: Ismael Juma, Damian Guy, Michael G. Noll, Guozhang Wang Closes #1260 from enothereska/KAFKA-3612-integration-tests --- build.gradle | 2 + checkstyle/import-control.xml | 11 + .../InternalTopicIntegrationTest.java | 169 ++++++++++++++++ .../utils/EmbeddedSingleNodeKafkaCluster.java | 128 ++++++++++++ .../utils/IntegrationTestUtils.java | 157 +++++++++++++++ .../integration/utils/KafkaEmbedded.java | 189 ++++++++++++++++++ 6 files changed, 656 insertions(+) create mode 100644 streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedSingleNodeKafkaCluster.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java diff --git a/build.gradle b/build.gradle index dfa7c40835b7a..06c41d5a9b457 100644 --- a/build.gradle +++ b/build.gradle @@ -669,6 +669,8 @@ project(':streams') { compile libs.jacksonDatabind // this dependency should be removed after KIP-4 testCompile project(':clients').sourceSets.test.output + testCompile project(':core') + testCompile project(':core').sourceSets.test.output testCompile libs.junit } diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index e94698c385f0b..39d4ca32bc679 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -138,6 +138,17 @@ + + + + + + + + + + + diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java new file mode 100644 index 0000000000000..2a3e7670e2878 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/integration/InternalTopicIntegrationTest.java @@ -0,0 +1,169 @@ +/** + * 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 org.apache.kafka.streams.integration; + + +import org.I0Itec.zkclient.ZkClient; +import org.I0Itec.zkclient.ZkConnection; +import org.apache.kafka.clients.producer.ProducerConfig; +import org.apache.kafka.common.serialization.Serde; +import org.apache.kafka.common.serialization.Serdes; +import org.apache.kafka.common.serialization.StringSerializer; +import org.apache.kafka.streams.KafkaStreams; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsConfig; +import org.apache.kafka.streams.kstream.KStream; +import org.apache.kafka.streams.kstream.KStreamBuilder; +import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.ValueMapper; +import org.apache.kafka.streams.processor.internals.ProcessorStateManager; +import org.junit.BeforeClass; +import org.junit.ClassRule; +import org.junit.Test; +import static org.junit.Assert.assertEquals; +import java.util.Arrays; +import java.util.List; +import java.util.Locale; +import java.util.Properties; + +import org.apache.kafka.streams.integration.utils.EmbeddedSingleNodeKafkaCluster; +import org.apache.kafka.streams.integration.utils.IntegrationTestUtils; +import kafka.admin.AdminUtils; +import kafka.log.LogConfig; +import kafka.utils.ZKStringSerializer$; +import kafka.utils.ZkUtils; +import scala.Tuple2; +import scala.collection.Iterator; +import scala.collection.Map; + +/** + * Tests related to internal topics in streams + */ +public class InternalTopicIntegrationTest { + @ClassRule + public static EmbeddedSingleNodeKafkaCluster cluster = new EmbeddedSingleNodeKafkaCluster(); + private static final String DEFAULT_INPUT_TOPIC = "inputTopic"; + private static final String DEFAULT_OUTPUT_TOPIC = "outputTopic"; + private static final int DEFAULT_ZK_SESSION_TIMEOUT_MS = 10 * 1000; + private static final int DEFAULT_ZK_CONNECTION_TIMEOUT_MS = 8 * 1000; + + @BeforeClass + public static void startKafkaCluster() throws Exception { + cluster.createTopic(DEFAULT_INPUT_TOPIC); + cluster.createTopic(DEFAULT_OUTPUT_TOPIC); + } + + /** + * Validates that any state changelog topics are compacted + * @return true if topics have a valid config, false otherwise + */ + private boolean isUsingCompactionForStateChangelogTopics() { + boolean valid = true; + + // Note: You must initialize the ZkClient with ZKStringSerializer. If you don't, then + // createTopic() will only seem to work (it will return without error). The topic will exist in + // only ZooKeeper and will be returned when listing topics, but Kafka itself does not create the + // topic. + ZkClient zkClient = new ZkClient( + cluster.zKConnectString(), + DEFAULT_ZK_SESSION_TIMEOUT_MS, + DEFAULT_ZK_CONNECTION_TIMEOUT_MS, + ZKStringSerializer$.MODULE$); + boolean isSecure = false; + ZkUtils zkUtils = new ZkUtils(zkClient, new ZkConnection(cluster.zKConnectString()), isSecure); + + Map topicConfigs = AdminUtils.fetchAllTopicConfigs(zkUtils); + Iterator it = topicConfigs.iterator(); + while (it.hasNext()) { + Tuple2 topicConfig = (Tuple2) it.next(); + String topic = topicConfig._1; + Properties prop = topicConfig._2; + + // state changelogs should be compacted + if (topic.endsWith(ProcessorStateManager.STATE_CHANGELOG_TOPIC_SUFFIX)) { + if (!prop.containsKey(LogConfig.CleanupPolicyProp()) || + !prop.getProperty(LogConfig.CleanupPolicyProp()).equals(LogConfig.Compact())) { + valid = false; + break; + } + } + } + zkClient.close(); + return valid; + } + + @Test + public void shouldCompactTopicsForStateChangelogs() throws Exception { + List inputValues = Arrays.asList("hello", "world", "world", "hello world"); + + // + // Step 1: Configure and start a simple word count topology + // + final Serde stringSerde = Serdes.String(); + final Serde longSerde = Serdes.Long(); + + Properties streamsConfiguration = new Properties(); + streamsConfiguration.put(StreamsConfig.APPLICATION_ID_CONFIG, "compact-topics-integration-test"); + streamsConfiguration.put(StreamsConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); + streamsConfiguration.put(StreamsConfig.ZOOKEEPER_CONNECT_CONFIG, cluster.zKConnectString()); + streamsConfiguration.put(StreamsConfig.KEY_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); + streamsConfiguration.put(StreamsConfig.VALUE_SERDE_CLASS_CONFIG, Serdes.String().getClass().getName()); + streamsConfiguration.put(StreamsConfig.STATE_DIR_CONFIG, "/tmp/kafka-streams"); + + KStreamBuilder builder = new KStreamBuilder(); + + KStream textLines = builder.stream(DEFAULT_INPUT_TOPIC); + + KStream wordCounts = textLines + .flatMapValues(new ValueMapper>() { + @Override + public Iterable apply(String value) { + return Arrays.asList(value.toLowerCase(Locale.getDefault()).split("\\W+")); + } + }).map(new KeyValueMapper>() { + @Override + public KeyValue apply(String key, String value) { + return new KeyValue(value, value); + } + }).countByKey("Counts").toStream(); + + wordCounts.to(stringSerde, longSerde, DEFAULT_OUTPUT_TOPIC); + + // Remove any state from previous test runs + IntegrationTestUtils.purgeLocalStreamsState(streamsConfiguration); + + KafkaStreams streams = new KafkaStreams(builder, streamsConfiguration); + streams.start(); + + // + // Step 2: Produce some input data to the input topic. + // + Properties producerConfig = new Properties(); + producerConfig.put(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG, cluster.bootstrapServers()); + producerConfig.put(ProducerConfig.ACKS_CONFIG, "all"); + producerConfig.put(ProducerConfig.RETRIES_CONFIG, 0); + producerConfig.put(ProducerConfig.KEY_SERIALIZER_CLASS_CONFIG, StringSerializer.class); + producerConfig.put(ProducerConfig.VALUE_SERIALIZER_CLASS_CONFIG, StringSerializer.class); + IntegrationTestUtils.produceValuesSynchronously(DEFAULT_INPUT_TOPIC, inputValues, producerConfig); + + // + // Step 3: Verify the state changelog topics are compact + // + streams.close(); + assertEquals(isUsingCompactionForStateChangelogTopics(), true); + } +} diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedSingleNodeKafkaCluster.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedSingleNodeKafkaCluster.java new file mode 100644 index 0000000000000..34753ae2e672f --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/EmbeddedSingleNodeKafkaCluster.java @@ -0,0 +1,128 @@ +/** + * 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 org.apache.kafka.streams.integration.utils; + +import kafka.server.KafkaConfig$; +import kafka.zk.EmbeddedZookeeper; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Properties; +import org.junit.rules.ExternalResource; + +/** + * Runs an in-memory, "embedded" Kafka cluster with 1 ZooKeeper instance and 1 Kafka broker. + */ +public class EmbeddedSingleNodeKafkaCluster extends ExternalResource { + + private static final Logger log = LoggerFactory.getLogger(EmbeddedSingleNodeKafkaCluster.class); + private static final int DEFAULT_BROKER_PORT = 0; // 0 results in a random port being selected + private EmbeddedZookeeper zookeeper = null; + private KafkaEmbedded broker = null; + + /** + * Creates and starts a Kafka cluster. + */ + public void start() throws IOException, InterruptedException { + Properties brokerConfig = new Properties(); + + log.debug("Initiating embedded Kafka cluster startup"); + log.debug("Starting a ZooKeeper instance"); + zookeeper = new EmbeddedZookeeper(); + log.debug("ZooKeeper instance is running at {}", zKConnectString()); + brokerConfig.put(KafkaConfig$.MODULE$.ZkConnectProp(), zKConnectString()); + brokerConfig.put(KafkaConfig$.MODULE$.PortProp(), DEFAULT_BROKER_PORT); + + log.debug("Starting a Kafka instance on port {} ...", brokerConfig.getProperty(KafkaConfig$.MODULE$.PortProp())); + broker = new KafkaEmbedded(brokerConfig); + + log.debug("Kafka instance is running at {}, connected to ZooKeeper at {}", + broker.brokerList(), broker.zookeeperConnect()); + } + + /** + * Stop the Kafka cluster. + */ + public void stop() { + broker.stop(); + zookeeper.shutdown(); + } + + /** + * The ZooKeeper connection string aka `zookeeper.connect` in `hostnameOrIp:port` format. + * Example: `127.0.0.1:2181`. + * + * You can use this to e.g. tell Kafka brokers how to connect to this instance. + */ + public String zKConnectString() { + return "localhost:" + zookeeper.port(); + } + + /** + * This cluster's `bootstrap.servers` value. Example: `127.0.0.1:9092`. + * + * You can use this to tell Kafka producers how to connect to this cluster. + */ + public String bootstrapServers() { + return broker.brokerList(); + } + + protected void before() throws Throwable { + start(); + } + + protected void after() { + stop(); + } + + /** + * Create a Kafka topic with 1 partition and a replication factor of 1. + * + * @param topic The name of the topic. + */ + public void createTopic(String topic) { + createTopic(topic, 1, 1, new Properties()); + } + + /** + * Create a Kafka topic with the given parameters. + * + * @param topic The name of the topic. + * @param partitions The number of partitions for this topic. + * @param replication The replication factor for (the partitions of) this topic. + */ + public void createTopic(String topic, int partitions, int replication) { + createTopic(topic, partitions, replication, new Properties()); + } + + /** + * Create a Kafka topic with the given parameters. + * + * @param topic The name of the topic. + * @param partitions The number of partitions for this topic. + * @param replication The replication factor for (partitions of) this topic. + * @param topicConfig Additional topic-level configuration settings. + */ + public void createTopic(String topic, + int partitions, + int replication, + Properties topicConfig) { + broker.createTopic(topic, partitions, replication, topicConfig); + } +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java new file mode 100644 index 0000000000000..89fe0c4ef943e --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/IntegrationTestUtils.java @@ -0,0 +1,157 @@ +/** + * 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 org.apache.kafka.streams.integration.utils; + +import org.apache.kafka.clients.consumer.ConsumerRecord; +import org.apache.kafka.clients.consumer.ConsumerRecords; +import org.apache.kafka.clients.consumer.KafkaConsumer; +import org.apache.kafka.clients.producer.KafkaProducer; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.kafka.clients.producer.RecordMetadata; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.streams.KeyValue; +import org.apache.kafka.streams.StreamsConfig; + +import java.io.File; +import java.io.IOException; +import java.nio.file.Paths; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Properties; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; + +/** + * Utility functions to make integration testing more convenient. + */ +public class IntegrationTestUtils { + + private static final int UNLIMITED_MESSAGES = -1; + + /** + * Returns up to `maxMessages` message-values from the topic. + * + * @param topic Kafka topic to read messages from + * @param consumerConfig Kafka consumer configuration + * @param maxMessages Maximum number of messages to read via the consumer. + * @return The values retrieved via the consumer. + */ + public static List readValues(String topic, Properties consumerConfig, int maxMessages) { + List returnList = new ArrayList<>(); + List> kvs = readKeyValues(topic, consumerConfig, maxMessages); + for (KeyValue kv : kvs) { + returnList.add(kv.value); + } + return returnList; + } + + /** + * Returns as many messages as possible from the topic until a (currently hardcoded) timeout is + * reached. + * + * @param topic Kafka topic to read messages from + * @param consumerConfig Kafka consumer configuration + * @return The KeyValue elements retrieved via the consumer. + */ + public static List> readKeyValues(String topic, Properties consumerConfig) { + return readKeyValues(topic, consumerConfig, UNLIMITED_MESSAGES); + } + + /** + * Returns up to `maxMessages` by reading via the provided consumer (the topic(s) to read from + * are already configured in the consumer). + * + * @param topic Kafka topic to read messages from + * @param consumerConfig Kafka consumer configuration + * @param maxMessages Maximum number of messages to read via the consumer + * @return The KeyValue elements retrieved via the consumer + */ + public static List> readKeyValues(String topic, Properties consumerConfig, int maxMessages) { + KafkaConsumer consumer = new KafkaConsumer<>(consumerConfig); + consumer.subscribe(Collections.singletonList(topic)); + int pollIntervalMs = 100; + int maxTotalPollTimeMs = 2000; + int totalPollTimeMs = 0; + List> consumedValues = new ArrayList<>(); + while (totalPollTimeMs < maxTotalPollTimeMs && continueConsuming(consumedValues.size(), maxMessages)) { + totalPollTimeMs += pollIntervalMs; + ConsumerRecords records = consumer.poll(pollIntervalMs); + for (ConsumerRecord record : records) { + consumedValues.add(new KeyValue<>(record.key(), record.value())); + } + } + consumer.close(); + return consumedValues; + } + + private static boolean continueConsuming(int messagesConsumed, int maxMessages) { + return maxMessages <= 0 || messagesConsumed < maxMessages; + } + + /** + * Removes local state stores. Useful to reset state in-between integration test runs. + * + * @param streamsConfiguration Streams configuration settings + */ + public static void purgeLocalStreamsState(Properties streamsConfiguration) throws IOException { + String path = streamsConfiguration.getProperty(StreamsConfig.STATE_DIR_CONFIG); + if (path != null) { + File node = Paths.get(path).normalize().toFile(); + // Only purge state when it's under /tmp. This is a safety net to prevent accidentally + // deleting important local directory trees. + if (node.getAbsolutePath().startsWith("/tmp")) { + Utils.delete(new File(node.getAbsolutePath())); + } + } + } + + /** + * @param topic Kafka topic to write the data records to + * @param records Data records to write to Kafka + * @param producerConfig Kafka producer configuration + * @param Key type of the data records + * @param Value type of the data records + */ + public static void produceKeyValuesSynchronously( + String topic, Collection> records, Properties producerConfig) + throws ExecutionException, InterruptedException { + Producer producer = new KafkaProducer<>(producerConfig); + for (KeyValue record : records) { + Future f = producer.send( + new ProducerRecord<>(topic, record.key, record.value)); + f.get(); + } + producer.flush(); + producer.close(); + } + + public static void produceValuesSynchronously( + String topic, Collection records, Properties producerConfig) + throws ExecutionException, InterruptedException { + Collection> keyedRecords = new ArrayList<>(); + for (V value : records) { + KeyValue kv = new KeyValue<>(null, value); + keyedRecords.add(kv); + } + produceKeyValuesSynchronously(topic, keyedRecords, producerConfig); + } + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java b/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java new file mode 100644 index 0000000000000..348b46b5c7112 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/integration/utils/KafkaEmbedded.java @@ -0,0 +1,189 @@ +/** + * 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 org.apache.kafka.streams.integration.utils; + + +import org.apache.kafka.common.protocol.SecurityProtocol; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Properties; + +import org.I0Itec.zkclient.ZkClient; +import org.I0Itec.zkclient.ZkConnection; + +import java.io.File; +import java.util.Collections; +import java.util.List; + +import kafka.admin.AdminUtils; +import kafka.admin.RackAwareMode; +import kafka.server.KafkaConfig; +import kafka.server.KafkaConfig$; +import kafka.server.KafkaServer; +import kafka.utils.CoreUtils; +import kafka.utils.SystemTime$; +import kafka.utils.TestUtils; +import kafka.utils.ZKStringSerializer$; +import kafka.utils.ZkUtils; +import org.junit.rules.TemporaryFolder; +/** + * Runs an in-memory, "embedded" instance of a Kafka broker, which listens at `127.0.0.1:9092` by + * default. + * + * Requires a running ZooKeeper instance to connect to. + */ +public class KafkaEmbedded { + + private static final Logger log = LoggerFactory.getLogger(KafkaEmbedded.class); + + private static final String DEFAULT_ZK_CONNECT = "127.0.0.1:2181"; + private static final int DEFAULT_ZK_SESSION_TIMEOUT_MS = 10 * 1000; + private static final int DEFAULT_ZK_CONNECTION_TIMEOUT_MS = 8 * 1000; + private final Properties effectiveConfig; + private final File logDir; + public final TemporaryFolder tmpFolder; + private final KafkaServer kafka; + + /** + * Creates and starts an embedded Kafka broker. + * @param config Broker configuration settings. Used to modify, for example, on which port the + * broker should listen to. Note that you cannot change the `log.dirs` setting + * currently. + */ + public KafkaEmbedded(Properties config) throws IOException { + tmpFolder = new TemporaryFolder(); + tmpFolder.create(); + logDir = tmpFolder.newFolder(); + effectiveConfig = effectiveConfigFrom(config); + boolean loggingEnabled = true; + KafkaConfig kafkaConfig = new KafkaConfig(effectiveConfig, loggingEnabled); + log.debug("Starting embedded Kafka broker (with log.dirs={} and ZK ensemble at {}) ...", + logDir, zookeeperConnect()); + kafka = TestUtils.createServer(kafkaConfig, SystemTime$.MODULE$); + log.debug("Startup of embedded Kafka broker at {} completed (with ZK ensemble at {}) ...", + brokerList(), zookeeperConnect()); + } + + + /** + * Creates the configuration for starting the Kafka broker by merging default values with + * overwrites. + * @param initialConfig Broker configuration settings that override the default config. + * @return + * @throws IOException + */ + private Properties effectiveConfigFrom(Properties initialConfig) throws IOException { + Properties effectiveConfig = new Properties(); + effectiveConfig.put(KafkaConfig$.MODULE$.BrokerIdProp(), 0); + effectiveConfig.put(KafkaConfig$.MODULE$.HostNameProp(), "127.0.0.1"); + effectiveConfig.put(KafkaConfig$.MODULE$.PortProp(), "9092"); + effectiveConfig.put(KafkaConfig$.MODULE$.NumPartitionsProp(), 1); + effectiveConfig.put(KafkaConfig$.MODULE$.AutoCreateTopicsEnableProp(), true); + effectiveConfig.put(KafkaConfig$.MODULE$.MessageMaxBytesProp(), 1000000); + effectiveConfig.put(KafkaConfig$.MODULE$.ControlledShutdownEnableProp(), true); + + effectiveConfig.putAll(initialConfig); + effectiveConfig.setProperty(KafkaConfig$.MODULE$.LogDirProp(), logDir.getAbsolutePath()); + return effectiveConfig; + } + + /** + * This broker's `metadata.broker.list` value. Example: `127.0.0.1:9092`. + * + * You can use this to tell Kafka producers and consumers how to connect to this instance. + */ + public String brokerList() { + return kafka.config().hostName() + ":" + kafka.boundPort(SecurityProtocol.PLAINTEXT); + } + + + /** + * The ZooKeeper connection string aka `zookeeper.connect`. + */ + public String zookeeperConnect() { + return effectiveConfig.getProperty("zookeeper.connect", DEFAULT_ZK_CONNECT); + } + + /** + * Stop the broker. + */ + public void stop() { + log.debug("Shutting down embedded Kafka broker at {} (with ZK ensemble at {}) ...", + brokerList(), zookeeperConnect()); + kafka.shutdown(); + kafka.awaitShutdown(); + log.debug("Removing logs.dir at {} ...", logDir); + List logDirs = Collections.singletonList(logDir.getAbsolutePath()); + CoreUtils.delete(scala.collection.JavaConversions.asScalaBuffer(logDirs).seq()); + tmpFolder.delete(); + log.debug("Shutdown of embedded Kafka broker at {} completed (with ZK ensemble at {}) ...", + brokerList(), zookeeperConnect()); + } + + /** + * Create a Kafka topic with 1 partition and a replication factor of 1. + * + * @param topic The name of the topic. + */ + public void createTopic(String topic) { + createTopic(topic, 1, 1, new Properties()); + } + + /** + * Create a Kafka topic with the given parameters. + * + * @param topic The name of the topic. + * @param partitions The number of partitions for this topic. + * @param replication The replication factor for (the partitions of) this topic. + */ + public void createTopic(String topic, int partitions, int replication) { + createTopic(topic, partitions, replication, new Properties()); + } + + /** + * Create a Kafka topic with the given parameters. + * + * @param topic The name of the topic. + * @param partitions The number of partitions for this topic. + * @param replication The replication factor for (partitions of) this topic. + * @param topicConfig Additional topic-level configuration settings. + */ + public void createTopic(String topic, + int partitions, + int replication, + Properties topicConfig) { + log.debug("Creating topic { name: {}, partitions: {}, replication: {}, config: {} }", + topic, partitions, replication, topicConfig); + + // Note: You must initialize the ZkClient with ZKStringSerializer. If you don't, then + // createTopic() will only seem to work (it will return without error). The topic will exist in + // only ZooKeeper and will be returned when listing topics, but Kafka itself does not create the + // topic. + ZkClient zkClient = new ZkClient( + zookeeperConnect(), + DEFAULT_ZK_SESSION_TIMEOUT_MS, + DEFAULT_ZK_CONNECTION_TIMEOUT_MS, + ZKStringSerializer$.MODULE$); + boolean isSecure = false; + ZkUtils zkUtils = new ZkUtils(zkClient, new ZkConnection(zookeeperConnect()), isSecure); + AdminUtils.createTopic(zkUtils, topic, partitions, replication, topicConfig, RackAwareMode.Enforced$.MODULE$); + zkClient.close(); + } +} \ No newline at end of file From 669be7fadc808a8436ba28d033b70d24641c75ea Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Wed, 27 Apr 2016 17:15:37 -0700 Subject: [PATCH 185/206] KAFKA-3631; Fix Struct.toString for nullable arrayOf Author: Grant Henke Reviewers: Ismael Juma Closes #1279 from granthenke/struct-fix --- .../org/apache/kafka/common/protocol/types/Struct.java | 2 +- .../common/protocol/types/ProtocolSerializationTest.java | 8 ++++++++ 2 files changed, 9 insertions(+), 1 deletion(-) diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java index 7eee09f0de425..212d701aaaa84 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/types/Struct.java @@ -290,7 +290,7 @@ public String toString() { Field f = this.schema.get(i); b.append(f.name); b.append('='); - if (f.type() instanceof ArrayOf) { + if (f.type() instanceof ArrayOf && this.values[i] != null) { Object[] arrayValue = (Object[]) this.values[i]; b.append('['); for (int j = 0; j < arrayValue.length; j++) { diff --git a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java index e91b2fb591d4e..1633e89acd564 100644 --- a/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java +++ b/clients/src/test/java/org/apache/kafka/common/protocol/types/ProtocolSerializationTest.java @@ -18,6 +18,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotNull; import static org.junit.Assert.fail; import java.nio.ByteBuffer; @@ -231,6 +232,13 @@ public void testReadNegativeBytesSize() { } } + @Test + public void testToString() { + String structStr = this.struct.toString(); + assertNotNull("Struct string should not be null.", structStr); + assertFalse("Struct string should not be empty.", structStr.isEmpty()); + } + private Object roundtrip(Type type, Object obj) { ByteBuffer buffer = ByteBuffer.allocate(type.sizeOf(obj)); type.write(buffer, obj); From 57831a56a581128e164b4372c5e5eb4b55038d4a Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Wed, 27 Apr 2016 17:31:31 -0700 Subject: [PATCH 186/206] KAFKA-3621; Add tests for ApiVersionRequest/Response Author: Ashish Singh Reviewers: Ismael Juma Closes #1275 from SinghAsDev/KAFKA-3621 --- .../kafka/server/ApiVersionsRequestTest.scala | 50 +++++++++++++++++++ .../unit/kafka/server/BaseRequestTest.scala | 8 +-- 2 files changed, 55 insertions(+), 3 deletions(-) create mode 100644 core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala new file mode 100644 index 0000000000000..ed599300c3f04 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala @@ -0,0 +1,50 @@ +/** + * 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.server + +import org.apache.kafka.common.protocol.ApiKeys +import org.apache.kafka.common.requests.ApiVersionsResponse.ApiVersion +import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse} +import org.junit.Assert._ +import org.junit.Test + +import scala.collection.JavaConversions._ + +class ApiVersionsRequestTest extends BaseRequestTest { + + override def numBrokers: Int = 1 + + @Test + def testApiVersionsRequest() { + val apiVersionsResponse = sendApiVersionsRequest(new ApiVersionsRequest, 0) + + assertEquals("API keys in ApiVersionsResponse must match API keys supported by broker.", ApiKeys.values.length, apiVersionsResponse.apiVersions.size) + for (expectedApiVersion: ApiVersion <- KafkaApis.apiVersionsResponse.apiVersions) { + val actualApiVersion = apiVersionsResponse.apiVersion(expectedApiVersion.apiKey) + assertNotNull(s"API key ${actualApiVersion.apiKey} is supported by broker, but not received in ApiVersionsResponse.", actualApiVersion) + assertEquals("API key must be supported by the broker.", expectedApiVersion.apiKey, actualApiVersion.apiKey) + assertEquals(s"Received unexpected min version for API key ${actualApiVersion.apiKey}.", expectedApiVersion.minVersion, actualApiVersion.minVersion) + assertEquals(s"Received unexpected max version for API key ${actualApiVersion.apiKey}.", expectedApiVersion.maxVersion, actualApiVersion.maxVersion) + } + } + + private def sendApiVersionsRequest(request: ApiVersionsRequest, version: Short): ApiVersionsResponse = { + val response = send(request, ApiKeys.API_VERSIONS, version) + ApiVersionsResponse.parse(response) + } +} \ No newline at end of file diff --git a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala index 3d05c1d34629e..d92ccea81e19d 100644 --- a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala @@ -30,11 +30,13 @@ import org.apache.kafka.common.requests.{AbstractRequest, RequestHeader, Respons import org.junit.Before abstract class BaseRequestTest extends KafkaServerTestHarness { - val numBrokers = 3 private var correlationId = 0 - // Override properties by mutating the passed Properties object - def propertyOverrides(properties: Properties): Unit + // If required, set number of brokers + protected def numBrokers: Int = 3 + + // If required, override properties by mutating the passed Properties object + protected def propertyOverrides(properties: Properties) {} def generateConfigs() = { val props = TestUtils.createBrokerConfigs(numBrokers, zkConnect, enableControlledShutdown = false) From 316389d6adfb1398e30ca2ce5d586ea94d3f3110 Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Thu, 28 Apr 2016 11:59:02 -0700 Subject: [PATCH 187/206] KAFKA-3611: Remove warnings when using reflections ewencp granders Can you take a look? Thanks! Author: Liquan Pei Reviewers: Ewen Cheslack-Postava Closes #1259 from Ishiihara/fix-warning --- bin/kafka-run-class.sh | 6 +- checkstyle/import-control.xml | 1 + .../kafka/connect/runtime/AbstractHerder.java | 10 ++- .../kafka/connect/util/ReflectionsUtil.java | 90 +++++++++++++++++++ .../ConnectorPluginsResourceTest.java | 3 +- 5 files changed, 104 insertions(+), 6 deletions(-) create mode 100644 connect/runtime/src/main/java/org/apache/kafka/connect/util/ReflectionsUtil.java diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index f45d8d4af17d5..88d43be3188d9 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -34,7 +34,11 @@ fi shopt -s nullglob for dir in $base_dir/core/build/dependant-libs-${SCALA_VERSION}*; do - CLASSPATH=$CLASSPATH:$dir/* + if [ -z $CLASSPATH ] ; then + CLASSPATH=$dir/* + else + CLASSPATH=$CLASSPATH:$dir/* + fi done for file in $base_dir/examples/build/libs//kafka-examples*.jar; diff --git a/checkstyle/import-control.xml b/checkstyle/import-control.xml index 39d4ca32bc679..7a45515e7bcf0 100644 --- a/checkstyle/import-control.xml +++ b/checkstyle/import-control.xml @@ -221,6 +221,7 @@ + diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java index bd735895b9967..83f56e2cb7a0e 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/AbstractHerder.java @@ -34,6 +34,7 @@ import org.apache.kafka.connect.tools.VerifiableSinkConnector; import org.apache.kafka.connect.tools.VerifiableSourceConnector; import org.apache.kafka.connect.util.ConnectorTaskId; +import org.apache.kafka.connect.util.ReflectionsUtil; import org.reflections.Reflections; import org.reflections.util.ClasspathHelper; import org.reflections.util.ConfigurationBuilder; @@ -82,11 +83,10 @@ public abstract class AbstractHerder implements Herder, TaskStatus.Listener, Con protected final ConfigBackingStore configBackingStore; private Map tempConnectors = new ConcurrentHashMap<>(); - private static final List> SKIPPED_CONNECTORS = Arrays.>asList(VerifiableSourceConnector.class, VerifiableSinkConnector.class); private static List validConnectorPlugins; private static final Object LOCK = new Object(); private Thread classPathTraverser; - + private static final List> EXCLUDES = Arrays.>asList(VerifiableSourceConnector.class, VerifiableSinkConnector.class); public AbstractHerder(Worker worker, String workerId, @@ -263,10 +263,12 @@ public static List connectorPlugins() { if (validConnectorPlugins != null) { return validConnectorPlugins; } + ReflectionsUtil.registerUrlTypes(); + ConfigurationBuilder builder = new ConfigurationBuilder().setUrls(ClasspathHelper.forJavaClassPath()); + Reflections reflections = new Reflections(builder); - Reflections reflections = new Reflections(new ConfigurationBuilder().setUrls(ClasspathHelper.forJavaClassPath())); Set> connectorClasses = reflections.getSubTypesOf(Connector.class); - connectorClasses.removeAll(SKIPPED_CONNECTORS); + connectorClasses.removeAll(EXCLUDES); List connectorPlugins = new LinkedList<>(); for (Class connectorClass : connectorClasses) { int mod = connectorClass.getModifiers(); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/util/ReflectionsUtil.java b/connect/runtime/src/main/java/org/apache/kafka/connect/util/ReflectionsUtil.java new file mode 100644 index 0000000000000..fc3a0ddbfe658 --- /dev/null +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/util/ReflectionsUtil.java @@ -0,0 +1,90 @@ +/** + * 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 org.apache.kafka.connect.util; + +import org.reflections.vfs.Vfs; +import org.reflections.vfs.Vfs.Dir; +import org.reflections.vfs.Vfs.File; +import org.reflections.vfs.Vfs.UrlType; + +import java.net.URL; +import java.util.Arrays; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; + +/** + * CLASSPATH on OSX contains .mar and .jnilib file extensions. Vfs used by Reflections does not recognize + * urls with those extensions and log WARNs when scan them. Those WARNs can be eliminated by registering + * URL types before using reflection. + */ +public class ReflectionsUtil { + + private static final String FILE_PROTOCOL = "file"; + private static final List ENDINGS = Arrays.asList(".mar", ".jnilib", "*"); + + public static void registerUrlTypes() { + final List urlTypes = new LinkedList<>(); + urlTypes.add(new EmptyUrlType(ENDINGS)); + urlTypes.addAll(Arrays.asList(Vfs.DefaultUrlTypes.values())); + Vfs.setDefaultURLTypes(urlTypes); + } + + private static class EmptyUrlType implements UrlType { + + private final List endings; + + private EmptyUrlType(final List endings) { + this.endings = endings; + } + + public boolean matches(URL url) { + final String protocol = url.getProtocol(); + final String externalForm = url.toExternalForm(); + if (!protocol.equals(FILE_PROTOCOL)) { + return false; + } + for (String ending : endings) { + if (externalForm.endsWith(ending)) { + return true; + } + } + return false; + } + + public Dir createDir(final URL url) throws Exception { + return emptyVfsDir(url); + } + + private static Dir emptyVfsDir(final URL url) { + return new Dir() { + @Override + public String getPath() { + return url.toExternalForm(); + } + + @Override + public Iterable getFiles() { + return Collections.emptyList(); + } + + @Override + public void close() { + + } + }; + } + } +} \ No newline at end of file diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java index 732db3d725674..241d331bd2287 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/resources/ConnectorPluginsResourceTest.java @@ -21,8 +21,8 @@ import org.apache.kafka.common.config.Config; import org.apache.kafka.common.config.ConfigDef; -import org.apache.kafka.common.config.ConfigDef.Recommender; import org.apache.kafka.common.config.ConfigDef.Importance; +import org.apache.kafka.common.config.ConfigDef.Recommender; import org.apache.kafka.common.config.ConfigDef.Type; import org.apache.kafka.common.config.ConfigDef.Width; import org.apache.kafka.connect.connector.Connector; @@ -149,6 +149,7 @@ public void testListConnectorPlugins() { assertTrue(connectorPlugins.contains(new ConnectorPluginInfo(ConnectorPluginsResourceTestConnector.class.getCanonicalName()))); } + /* Name here needs to be unique as we are testing the aliasing mechanism */ public static class ConnectorPluginsResourceTestConnector extends Connector { From 3a496f480d002a4512273477eda9d92731e600c3 Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Thu, 28 Apr 2016 13:39:22 -0700 Subject: [PATCH 188/206] KAFKA-3617; Unit tests for SASL authenticator Unit tests for SASL authenticator, tests for SASL/PLAIN and multiple mechanisms, authorization test for SASL/PLAIN Author: Rajini Sivaram Reviewers: Ismael Juma Closes #1273 from rajinisivaram/KAFKA-3617 --- .../common/network/SaslChannelBuilder.java | 13 +- .../kafka/common/security/JaasUtils.java | 2 +- .../kafka/common/network/CertStores.java | 46 ++ .../common/network/NetworkTestUtils.java | 80 ++++ .../kafka/common/network/NioEchoServer.java | 145 +++++++ .../common/network/SslTransportLayerTest.java | 293 +++---------- .../authenticator/SaslAuthenticatorTest.java | 408 ++++++++++++++++++ .../authenticator/TestDigestLoginModule.java | 109 +++++ .../authenticator/TestJaasConfig.java | 89 ++++ .../kafka/api/EndToEndAuthorizationTest.scala | 5 +- .../api/SaslMultiMechanismConsumerTest.scala | 4 +- .../api/SaslPlainPlaintextConsumerTest.scala | 2 +- ...aslPlainSslEndToEndAuthorizationTest.scala | 28 ++ .../integration/kafka/api/SaslSetup.scala | 13 + .../kafka/api/SaslTestHarness.scala | 13 - 15 files changed, 982 insertions(+), 268 deletions(-) create mode 100644 clients/src/test/java/org/apache/kafka/common/network/CertStores.java create mode 100644 clients/src/test/java/org/apache/kafka/common/network/NetworkTestUtils.java create mode 100644 clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java create mode 100644 clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java create mode 100644 clients/src/test/java/org/apache/kafka/common/security/authenticator/TestDigestLoginModule.java create mode 100644 clients/src/test/java/org/apache/kafka/common/security/authenticator/TestJaasConfig.java create mode 100644 core/src/test/scala/integration/kafka/api/SaslPlainSslEndToEndAuthorizationTest.scala diff --git a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java index a0464bc611eca..5c907edd596d6 100644 --- a/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java +++ b/clients/src/main/java/org/apache/kafka/common/network/SaslChannelBuilder.java @@ -63,14 +63,13 @@ public void configure(Map configs) throws KafkaException { hasKerberos = clientSaslMechanism.equals(SaslConfigs.GSSAPI_MECHANISM); } - String defaultRealm; - try { - defaultRealm = JaasUtils.defaultRealm(); - } catch (Exception ke) { - defaultRealm = ""; - } - if (hasKerberos) { + String defaultRealm; + try { + defaultRealm = JaasUtils.defaultKerberosRealm(); + } catch (Exception ke) { + defaultRealm = ""; + } @SuppressWarnings("unchecked") List principalToLocalRules = (List) configs.get(SaslConfigs.SASL_KERBEROS_PRINCIPAL_TO_LOCAL_RULES); if (principalToLocalRules != null) diff --git a/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java b/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java index ff5e0082f1771..63bbafc17ea8b 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java +++ b/clients/src/main/java/org/apache/kafka/common/security/JaasUtils.java @@ -58,7 +58,7 @@ public static String jaasConfig(String loginContextName, String key) throws IOEx return null; } - public static String defaultRealm() + public static String defaultKerberosRealm() throws ClassNotFoundException, NoSuchMethodException, IllegalArgumentException, IllegalAccessException, InvocationTargetException { diff --git a/clients/src/test/java/org/apache/kafka/common/network/CertStores.java b/clients/src/test/java/org/apache/kafka/common/network/CertStores.java new file mode 100644 index 0000000000000..6f108b5545645 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/network/CertStores.java @@ -0,0 +1,46 @@ +/** + * 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 org.apache.kafka.common.network; + +import java.io.File; +import java.util.HashMap; +import java.util.Map; + +import org.apache.kafka.common.config.SslConfigs; +import org.apache.kafka.test.TestSslUtils; + +public class CertStores { + + private final Map sslConfig; + + public CertStores(boolean server, String host) throws Exception { + String name = server ? "server" : "client"; + Mode mode = server ? Mode.SERVER : Mode.CLIENT; + File truststoreFile = File.createTempFile(name + "TS", ".jks"); + sslConfig = TestSslUtils.createSslConfig(!server, true, mode, truststoreFile, name, host); + if (server) + sslConfig.put(SslConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, Class.forName(SslConfigs.DEFAULT_PRINCIPAL_BUILDER_CLASS)); + } + + public Map getTrustingConfig(CertStores truststoreConfig) { + Map config = new HashMap<>(sslConfig); + config.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, truststoreConfig.sslConfig.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG)); + config.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, truststoreConfig.sslConfig.get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG)); + config.put(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, truststoreConfig.sslConfig.get(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG)); + return config; + } + + public Map getUntrustingConfig() { + return sslConfig; + } +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/common/network/NetworkTestUtils.java b/clients/src/test/java/org/apache/kafka/common/network/NetworkTestUtils.java new file mode 100644 index 0000000000000..53ba954b934b4 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/network/NetworkTestUtils.java @@ -0,0 +1,80 @@ +/** + * 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 org.apache.kafka.common.network; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.protocol.SecurityProtocol; +import org.apache.kafka.common.utils.MockTime; +import org.apache.kafka.common.utils.Utils; +import org.apache.kafka.test.TestUtils; + +/** + * Common utility functions used by transport layer and authenticator tests. + */ +public class NetworkTestUtils { + + public static NioEchoServer createEchoServer(SecurityProtocol securityProtocol, Map serverConfigs) throws Exception { + NioEchoServer server = new NioEchoServer(securityProtocol, serverConfigs, "localhost"); + server.start(); + return server; + } + + public static Selector createSelector(ChannelBuilder channelBuilder) { + return new Selector(5000, new Metrics(), new MockTime(), "MetricGroup", channelBuilder); + } + + public static void checkClientConnection(Selector selector, String node, int minMessageSize, int messageCount) throws Exception { + + String prefix = TestUtils.randomString(minMessageSize); + int requests = 0; + int responses = 0; + // wait for handshake to finish + while (!selector.isChannelReady(node)) { + selector.poll(1000L); + } + selector.send(new NetworkSend(node, ByteBuffer.wrap((prefix + "-0").getBytes()))); + requests++; + while (responses < messageCount) { + selector.poll(0L); + assertEquals("No disconnects should have occurred.", 0, selector.disconnected().size()); + + for (NetworkReceive receive : selector.completedReceives()) { + assertEquals(prefix + "-" + responses, new String(Utils.toArray(receive.payload()))); + responses++; + } + + for (int i = 0; i < selector.completedSends().size() && requests < messageCount && selector.isChannelReady(node); i++, requests++) { + selector.send(new NetworkSend(node, ByteBuffer.wrap((prefix + "-" + requests).getBytes()))); + } + } + } + + public static void waitForChannelClose(Selector selector, String node) throws IOException { + boolean closed = false; + for (int i = 0; i < 30; i++) { + selector.poll(1000L); + if (selector.channel(node) == null) { + closed = true; + break; + } + } + assertTrue(closed); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java b/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java new file mode 100644 index 0000000000000..e99a3999658b1 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/network/NioEchoServer.java @@ -0,0 +1,145 @@ +/** + * 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 org.apache.kafka.common.network; + +import java.io.IOException; +import java.net.InetSocketAddress; +import java.nio.channels.SelectionKey; +import java.nio.channels.ServerSocketChannel; +import java.nio.channels.SocketChannel; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentLinkedQueue; + +import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.protocol.SecurityProtocol; +import org.apache.kafka.common.utils.MockTime; + +/** + * Non-blocking EchoServer implementation that uses ChannelBuilder to create channels + * with the configured security protocol. + * + */ +public class NioEchoServer extends Thread { + private final int port; + private final ServerSocketChannel serverSocketChannel; + private final List newChannels; + private final List socketChannels; + private final AcceptorThread acceptorThread; + private final Selector selector; + private final ConcurrentLinkedQueue inflightSends = new ConcurrentLinkedQueue(); + + public NioEchoServer(SecurityProtocol securityProtocol, Map configs, String serverHost) throws Exception { + serverSocketChannel = ServerSocketChannel.open(); + serverSocketChannel.configureBlocking(false); + serverSocketChannel.socket().bind(new InetSocketAddress(serverHost, 0)); + this.port = serverSocketChannel.socket().getLocalPort(); + this.socketChannels = Collections.synchronizedList(new ArrayList()); + this.newChannels = Collections.synchronizedList(new ArrayList()); + ChannelBuilder channelBuilder = ChannelBuilders.create(securityProtocol, Mode.SERVER, LoginType.SERVER, configs, null, true); + this.selector = new Selector(5000, new Metrics(), new MockTime(), "MetricGroup", channelBuilder); + setName("echoserver"); + setDaemon(true); + acceptorThread = new AcceptorThread(); + } + + public int port() { + return port; + } + + @Override + public void run() { + try { + acceptorThread.start(); + while (serverSocketChannel.isOpen()) { + selector.poll(1000); + for (SocketChannel socketChannel : newChannels) { + String id = id(socketChannel); + selector.register(id, socketChannel); + socketChannels.add(socketChannel); + } + newChannels.clear(); + while (true) { + NetworkSend send = inflightSends.peek(); + if (send != null && !selector.channel(send.destination()).hasSend()) { + send = inflightSends.poll(); + selector.send(send); + } else + break; + } + List completedReceives = selector.completedReceives(); + for (NetworkReceive rcv : completedReceives) { + NetworkSend send = new NetworkSend(rcv.source(), rcv.payload()); + if (!selector.channel(send.destination()).hasSend()) + selector.send(send); + else + inflightSends.add(send); + } + } + } catch (IOException e) { + // ignore + } + } + + private String id(SocketChannel channel) { + return channel.socket().getLocalAddress().getHostAddress() + ":" + channel.socket().getLocalPort() + "-" + + channel.socket().getInetAddress().getHostAddress() + ":" + channel.socket().getPort(); + } + + public void closeConnections() throws IOException { + for (SocketChannel channel : socketChannels) + channel.close(); + socketChannels.clear(); + } + + public void close() throws IOException, InterruptedException { + this.serverSocketChannel.close(); + closeConnections(); + acceptorThread.interrupt(); + acceptorThread.join(); + interrupt(); + join(); + } + + private class AcceptorThread extends Thread { + public AcceptorThread() throws IOException { + setName("acceptor"); + } + public void run() { + try { + java.nio.channels.Selector acceptSelector = java.nio.channels.Selector.open(); + serverSocketChannel.register(acceptSelector, SelectionKey.OP_ACCEPT); + while (serverSocketChannel.isOpen()) { + if (acceptSelector.select(1000) > 0) { + Iterator it = acceptSelector.selectedKeys().iterator(); + while (it.hasNext()) { + SelectionKey key = it.next(); + if (key.isAcceptable()) { + SocketChannel socketChannel = ((ServerSocketChannel) key.channel()).accept(); + socketChannel.configureBlocking(false); + newChannels.add(socketChannel); + selector.wakeup(); + } + it.remove(); + } + } + } + } catch (IOException e) { + // ignore + } + } + } +} \ No newline at end of file diff --git a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java index d3302c87a25ad..4e96411c362a7 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java +++ b/clients/src/test/java/org/apache/kafka/common/network/SslTransportLayerTest.java @@ -12,25 +12,15 @@ */ package org.apache.kafka.common.network; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; -import java.util.ArrayList; import java.util.Arrays; -import java.util.Collections; -import java.util.HashMap; -import java.util.Iterator; -import java.util.List; import java.util.Map; -import java.util.concurrent.ConcurrentLinkedQueue; import java.io.IOException; -import java.io.File; import java.net.InetAddress; import java.net.InetSocketAddress; import java.nio.ByteBuffer; import java.nio.channels.SelectionKey; -import java.nio.channels.ServerSocketChannel; import java.nio.channels.SocketChannel; import javax.net.ssl.SSLContext; @@ -40,11 +30,9 @@ import org.apache.kafka.common.config.SslConfigs; import org.apache.kafka.common.security.ssl.SslFactory; import org.apache.kafka.common.metrics.Metrics; +import org.apache.kafka.common.protocol.SecurityProtocol; import org.apache.kafka.common.utils.MockTime; import org.apache.kafka.common.config.types.Password; -import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.test.TestSslUtils; -import org.apache.kafka.test.TestUtils; import org.junit.After; import org.junit.Before; import org.junit.Test; @@ -56,7 +44,7 @@ public class SslTransportLayerTest { private static final int BUFFER_SIZE = 4 * 1024; - private SslEchoServer server; + private NioEchoServer server; private Selector selector; private ChannelBuilder channelBuilder; private CertStores serverCertStores; @@ -91,13 +79,13 @@ public void teardown() throws Exception { @Test public void testValidEndpointIdentification() throws Exception { String node = "0"; - createEchoServer(sslServerConfigs); + server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs); sslClientConfigs.put(SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, "HTTPS"); createSelector(sslClientConfigs); - InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port()); selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); - testClientConnection(node, 100, 10); + NetworkTestUtils.checkClientConnection(selector, node, 100, 10); } /** @@ -113,12 +101,12 @@ public void testInvalidEndpointIdentification() throws Exception { sslServerConfigs = serverCertStores.getTrustingConfig(clientCertStores); sslClientConfigs = clientCertStores.getTrustingConfig(serverCertStores); sslClientConfigs.put(SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG, "HTTPS"); - createEchoServer(sslServerConfigs); + server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs); createSelector(sslClientConfigs); - InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port()); selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); - waitForChannelClose(node); + NetworkTestUtils.waitForChannelClose(selector, node); } /** @@ -129,14 +117,14 @@ public void testInvalidEndpointIdentification() throws Exception { public void testEndpointIdentificationDisabled() throws Exception { String node = "0"; String serverHost = InetAddress.getLocalHost().getHostAddress(); - server = new SslEchoServer(sslServerConfigs, serverHost); + server = new NioEchoServer(SecurityProtocol.SSL, sslServerConfigs, serverHost); server.start(); sslClientConfigs.remove(SslConfigs.SSL_ENDPOINT_IDENTIFICATION_ALGORITHM_CONFIG); createSelector(sslClientConfigs); - InetSocketAddress addr = new InetSocketAddress(serverHost, server.port); + InetSocketAddress addr = new InetSocketAddress(serverHost, server.port()); selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); - testClientConnection(node, 100, 10); + NetworkTestUtils.checkClientConnection(selector, node, 100, 10); } /** @@ -147,12 +135,12 @@ public void testEndpointIdentificationDisabled() throws Exception { public void testClientAuthenticationRequiredValidProvided() throws Exception { String node = "0"; sslServerConfigs.put(SslConfigs.SSL_CLIENT_AUTH_CONFIG, "required"); - createEchoServer(sslServerConfigs); + server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs); createSelector(sslClientConfigs); - InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port()); selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); - testClientConnection(node, 100, 10); + NetworkTestUtils.checkClientConnection(selector, node, 100, 10); } /** @@ -164,12 +152,12 @@ public void testClientAuthenticationRequiredUntrustedProvided() throws Exception String node = "0"; sslServerConfigs = serverCertStores.getUntrustingConfig(); sslServerConfigs.put(SslConfigs.SSL_CLIENT_AUTH_CONFIG, "required"); - createEchoServer(sslServerConfigs); + server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs); createSelector(sslClientConfigs); - InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port()); selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); - waitForChannelClose(node); + NetworkTestUtils.waitForChannelClose(selector, node); } /** @@ -180,16 +168,16 @@ public void testClientAuthenticationRequiredUntrustedProvided() throws Exception public void testClientAuthenticationRequiredNotProvided() throws Exception { String node = "0"; sslServerConfigs.put(SslConfigs.SSL_CLIENT_AUTH_CONFIG, "required"); - createEchoServer(sslServerConfigs); + server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs); sslClientConfigs.remove(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG); sslClientConfigs.remove(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG); sslClientConfigs.remove(SslConfigs.SSL_KEY_PASSWORD_CONFIG); createSelector(sslClientConfigs); - InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port()); selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); - waitForChannelClose(node); + NetworkTestUtils.waitForChannelClose(selector, node); } /** @@ -201,12 +189,12 @@ public void testClientAuthenticationDisabledUntrustedProvided() throws Exception String node = "0"; sslServerConfigs = serverCertStores.getUntrustingConfig(); sslServerConfigs.put(SslConfigs.SSL_CLIENT_AUTH_CONFIG, "none"); - createEchoServer(sslServerConfigs); + server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs); createSelector(sslClientConfigs); - InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port()); selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); - testClientConnection(node, 100, 10); + NetworkTestUtils.checkClientConnection(selector, node, 100, 10); } /** @@ -217,16 +205,16 @@ public void testClientAuthenticationDisabledUntrustedProvided() throws Exception public void testClientAuthenticationDisabledNotProvided() throws Exception { String node = "0"; sslServerConfigs.put(SslConfigs.SSL_CLIENT_AUTH_CONFIG, "none"); - createEchoServer(sslServerConfigs); + server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs); sslClientConfigs.remove(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG); sslClientConfigs.remove(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG); sslClientConfigs.remove(SslConfigs.SSL_KEY_PASSWORD_CONFIG); createSelector(sslClientConfigs); - InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port()); selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); - testClientConnection(node, 100, 10); + NetworkTestUtils.checkClientConnection(selector, node, 100, 10); } /** @@ -237,12 +225,12 @@ public void testClientAuthenticationDisabledNotProvided() throws Exception { public void testClientAuthenticationRequestedValidProvided() throws Exception { String node = "0"; sslServerConfigs.put(SslConfigs.SSL_CLIENT_AUTH_CONFIG, "requested"); - createEchoServer(sslServerConfigs); + server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs); createSelector(sslClientConfigs); - InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port()); selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); - testClientConnection(node, 100, 10); + NetworkTestUtils.checkClientConnection(selector, node, 100, 10); } /** @@ -253,16 +241,16 @@ public void testClientAuthenticationRequestedValidProvided() throws Exception { public void testClientAuthenticationRequestedNotProvided() throws Exception { String node = "0"; sslServerConfigs.put(SslConfigs.SSL_CLIENT_AUTH_CONFIG, "requested"); - createEchoServer(sslServerConfigs); + server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs); sslClientConfigs.remove(SslConfigs.SSL_KEYSTORE_LOCATION_CONFIG); sslClientConfigs.remove(SslConfigs.SSL_KEYSTORE_PASSWORD_CONFIG); sslClientConfigs.remove(SslConfigs.SSL_KEY_PASSWORD_CONFIG); createSelector(sslClientConfigs); - InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port()); selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); - testClientConnection(node, 100, 10); + NetworkTestUtils.checkClientConnection(selector, node, 100, 10); } /** @@ -303,12 +291,12 @@ public void testInvalidKeystorePassword() throws Exception { public void testInvalidKeyPassword() throws Exception { String node = "0"; sslServerConfigs.put(SslConfigs.SSL_KEY_PASSWORD_CONFIG, new Password("invalid")); - createEchoServer(sslServerConfigs); + server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs); createSelector(sslClientConfigs); - InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port()); selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); - waitForChannelClose(node); + NetworkTestUtils.waitForChannelClose(selector, node); } /** @@ -318,14 +306,14 @@ public void testInvalidKeyPassword() throws Exception { public void testUnsupportedTLSVersion() throws Exception { String node = "0"; sslServerConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.2")); - createEchoServer(sslServerConfigs); + server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs); sslClientConfigs.put(SslConfigs.SSL_ENABLED_PROTOCOLS_CONFIG, Arrays.asList("TLSv1.1")); createSelector(sslClientConfigs); - InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port()); selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); - waitForChannelClose(node); + NetworkTestUtils.waitForChannelClose(selector, node); } /** @@ -336,14 +324,14 @@ public void testUnsupportedCiphers() throws Exception { String node = "0"; String[] cipherSuites = SSLContext.getDefault().getDefaultSSLParameters().getCipherSuites(); sslServerConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuites[0])); - createEchoServer(sslServerConfigs); + server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs); sslClientConfigs.put(SslConfigs.SSL_CIPHER_SUITES_CONFIG, Arrays.asList(cipherSuites[1])); createSelector(sslClientConfigs); - InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port()); selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); - waitForChannelClose(node); + NetworkTestUtils.waitForChannelClose(selector, node); } /** @@ -352,12 +340,12 @@ public void testUnsupportedCiphers() throws Exception { @Test public void testNetReadBufferResize() throws Exception { String node = "0"; - createEchoServer(sslServerConfigs); + server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs); createSelector(sslClientConfigs, 10, null, null); - InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port()); selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); - testClientConnection(node, 64000, 10); + NetworkTestUtils.checkClientConnection(selector, node, 64000, 10); } /** @@ -366,12 +354,12 @@ public void testNetReadBufferResize() throws Exception { @Test public void testNetWriteBufferResize() throws Exception { String node = "0"; - createEchoServer(sslServerConfigs); + server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs); createSelector(sslClientConfigs, null, 10, null); - InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port()); selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); - testClientConnection(node, 64000, 10); + NetworkTestUtils.checkClientConnection(selector, node, 64000, 10); } /** @@ -380,55 +368,12 @@ public void testNetWriteBufferResize() throws Exception { @Test public void testApplicationBufferResize() throws Exception { String node = "0"; - createEchoServer(sslServerConfigs); + server = NetworkTestUtils.createEchoServer(SecurityProtocol.SSL, sslServerConfigs); createSelector(sslClientConfigs, null, null, 10); - InetSocketAddress addr = new InetSocketAddress("localhost", server.port); + InetSocketAddress addr = new InetSocketAddress("localhost", server.port()); selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); - testClientConnection(node, 64000, 10); - } - - private void testClientConnection(String node, int minMessageSize, int messageCount) throws Exception { - - String prefix = TestUtils.randomString(minMessageSize); - int requests = 0; - int responses = 0; - // wait for handshake to finish - while (!selector.isChannelReady(node)) { - selector.poll(1000L); - } - selector.send(new NetworkSend(node, ByteBuffer.wrap((prefix + "-0").getBytes()))); - requests++; - while (responses < messageCount) { - selector.poll(0L); - assertEquals("No disconnects should have occurred.", 0, selector.disconnected().size()); - - for (NetworkReceive receive : selector.completedReceives()) { - assertEquals(prefix + "-" + responses, new String(Utils.toArray(receive.payload()))); - responses++; - } - - for (int i = 0; i < selector.completedSends().size() && requests < messageCount && selector.isChannelReady(node); i++, requests++) { - selector.send(new NetworkSend(node, ByteBuffer.wrap((prefix + "-" + requests).getBytes()))); - } - } - } - - private void waitForChannelClose(String node) throws IOException { - boolean closed = false; - for (int i = 0; i < 30; i++) { - selector.poll(1000L); - if (selector.channel(node) == null) { - closed = true; - break; - } - } - assertTrue(closed); - } - - private void createEchoServer(Map sslServerConfigs) throws Exception { - server = new SslEchoServer(sslServerConfigs, "localhost"); - server.start(); + NetworkTestUtils.checkClientConnection(selector, node, 64000, 10); } private void createSelector(Map sslClientConfigs) { @@ -455,32 +400,6 @@ protected SslTransportLayer buildTransportLayer(SslFactory sslFactory, String id this.selector = new Selector(5000, new Metrics(), new MockTime(), "MetricGroup", channelBuilder); } - private static class CertStores { - - Map sslConfig; - - CertStores(boolean server, String host) throws Exception { - String name = server ? "server" : "client"; - Mode mode = server ? Mode.SERVER : Mode.CLIENT; - File truststoreFile = File.createTempFile(name + "TS", ".jks"); - sslConfig = TestSslUtils.createSslConfig(!server, true, mode, truststoreFile, name, host); - if (server) - sslConfig.put(SslConfigs.PRINCIPAL_BUILDER_CLASS_CONFIG, Class.forName(SslConfigs.DEFAULT_PRINCIPAL_BUILDER_CLASS)); - } - - private Map getTrustingConfig(CertStores truststoreConfig) { - Map config = new HashMap<>(sslConfig); - config.put(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG, truststoreConfig.sslConfig.get(SslConfigs.SSL_TRUSTSTORE_LOCATION_CONFIG)); - config.put(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG, truststoreConfig.sslConfig.get(SslConfigs.SSL_TRUSTSTORE_PASSWORD_CONFIG)); - config.put(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG, truststoreConfig.sslConfig.get(SslConfigs.SSL_TRUSTSTORE_TYPE_CONFIG)); - return config; - } - - private Map getUntrustingConfig() { - return sslConfig; - } - } - /** * SSLTransportLayer with overrides for packet and application buffer size to test buffer resize * code path. The overridden buffer size starts with a small value and increases in size when the buffer @@ -537,117 +456,5 @@ int updateAndGet(int actualSize, boolean update) { } } } - - // Non-blocking EchoServer implementation that uses SSLTransportLayer - private class SslEchoServer extends Thread { - private final int port; - private final ServerSocketChannel serverSocketChannel; - private final List newChannels; - private final List socketChannels; - private final AcceptorThread acceptorThread; - private SslFactory sslFactory; - private final Selector selector; - private final ConcurrentLinkedQueue inflightSends = new ConcurrentLinkedQueue(); - - public SslEchoServer(Map configs, String serverHost) throws Exception { - this.sslFactory = new SslFactory(Mode.SERVER); - this.sslFactory.configure(configs); - serverSocketChannel = ServerSocketChannel.open(); - serverSocketChannel.configureBlocking(false); - serverSocketChannel.socket().bind(new InetSocketAddress(serverHost, 0)); - this.port = serverSocketChannel.socket().getLocalPort(); - this.socketChannels = Collections.synchronizedList(new ArrayList()); - this.newChannels = Collections.synchronizedList(new ArrayList()); - SslChannelBuilder channelBuilder = new SslChannelBuilder(Mode.SERVER); - channelBuilder.configure(sslServerConfigs); - this.selector = new Selector(5000, new Metrics(), new MockTime(), "MetricGroup", channelBuilder); - setName("echoserver"); - setDaemon(true); - acceptorThread = new AcceptorThread(); - } - - @Override - public void run() { - try { - acceptorThread.start(); - while (serverSocketChannel.isOpen()) { - selector.poll(1000); - for (SocketChannel socketChannel : newChannels) { - String id = id(socketChannel); - selector.register(id, socketChannel); - socketChannels.add(socketChannel); - } - newChannels.clear(); - while (true) { - NetworkSend send = inflightSends.peek(); - if (send != null && !selector.channel(send.destination()).hasSend()) { - send = inflightSends.poll(); - selector.send(send); - } else - break; - } - List completedReceives = selector.completedReceives(); - for (NetworkReceive rcv : completedReceives) { - NetworkSend send = new NetworkSend(rcv.source(), rcv.payload()); - if (!selector.channel(send.destination()).hasSend()) - selector.send(send); - else - inflightSends.add(send); - } - } - } catch (IOException e) { - // ignore - } - } - - private String id(SocketChannel channel) { - return channel.socket().getLocalAddress().getHostAddress() + ":" + channel.socket().getLocalPort() + "-" + - channel.socket().getInetAddress().getHostAddress() + ":" + channel.socket().getPort(); - } - - public void closeConnections() throws IOException { - for (SocketChannel channel : socketChannels) - channel.close(); - socketChannels.clear(); - } - - public void close() throws IOException, InterruptedException { - this.serverSocketChannel.close(); - closeConnections(); - acceptorThread.interrupt(); - acceptorThread.join(); - interrupt(); - join(); - } - - private class AcceptorThread extends Thread { - public AcceptorThread() throws IOException { - setName("acceptor"); - } - public void run() { - try { - - java.nio.channels.Selector acceptSelector = java.nio.channels.Selector.open(); - serverSocketChannel.register(acceptSelector, SelectionKey.OP_ACCEPT); - while (serverSocketChannel.isOpen()) { - if (acceptSelector.select(1000) > 0) { - Iterator it = acceptSelector.selectedKeys().iterator(); - while (it.hasNext()) { - SelectionKey key = it.next(); - if (key.isAcceptable()) { - SocketChannel socketChannel = ((ServerSocketChannel) key.channel()).accept(); - socketChannel.configureBlocking(false); - newChannels.add(socketChannel); - selector.wakeup(); - } - } - } - } - } catch (IOException e) { - // ignore - } - } - } - } } diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java new file mode 100644 index 0000000000000..0a4928b176ecc --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java @@ -0,0 +1,408 @@ +/** + * 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 org.apache.kafka.common.security.authenticator; + +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.net.InetSocketAddress; +import java.nio.ByteBuffer; + +import static org.junit.Assert.fail; + +import org.apache.kafka.common.KafkaException; +import org.apache.kafka.common.config.SaslConfigs; +import org.apache.kafka.common.network.CertStores; +import org.apache.kafka.common.network.ChannelBuilder; +import org.apache.kafka.common.network.ChannelBuilders; +import org.apache.kafka.common.network.LoginType; +import org.apache.kafka.common.network.Mode; +import org.apache.kafka.common.network.NetworkSend; +import org.apache.kafka.common.network.NetworkTestUtils; +import org.apache.kafka.common.network.NioEchoServer; +import org.apache.kafka.common.network.Selector; +import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.SecurityProtocol; +import org.apache.kafka.common.requests.MetadataRequest; +import org.apache.kafka.common.requests.RequestHeader; +import org.apache.kafka.common.requests.RequestSend; +import org.apache.kafka.common.requests.SaslHandshakeRequest; +import org.apache.kafka.common.security.JaasUtils; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +/** + * Tests for the Sasl authenticator. These use a test harness that runs a simple socket server that echos back responses. + */ +public class SaslAuthenticatorTest { + + private static final int BUFFER_SIZE = 4 * 1024; + + private NioEchoServer server; + private Selector selector; + private ChannelBuilder channelBuilder; + private CertStores serverCertStores; + private CertStores clientCertStores; + private Map saslClientConfigs; + private Map saslServerConfigs; + + @Before + public void setup() throws Exception { + serverCertStores = new CertStores(true, "localhost"); + clientCertStores = new CertStores(false, "localhost"); + saslServerConfigs = serverCertStores.getTrustingConfig(clientCertStores); + saslClientConfigs = clientCertStores.getTrustingConfig(serverCertStores); + } + + @After + public void teardown() throws Exception { + if (server != null) + this.server.close(); + if (selector != null) + this.selector.close(); + } + + /** + * Tests good path SASL/PLAIN client and server channels using SSL transport layer. + */ + @Test + public void testValidSaslPlainOverSsl() throws Exception { + String node = "0"; + SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL; + configureMechanisms("PLAIN", Arrays.asList("PLAIN")); + + server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs); + createAndCheckClientConnection(securityProtocol, node); + } + + /** + * Tests good path SASL/PLAIN client and server channels using PLAINTEXT transport layer. + */ + @Test + public void testValidSaslPlainOverPlaintext() throws Exception { + String node = "0"; + SecurityProtocol securityProtocol = SecurityProtocol.SASL_PLAINTEXT; + configureMechanisms("PLAIN", Arrays.asList("PLAIN")); + + server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs); + createAndCheckClientConnection(securityProtocol, node); + } + + /** + * Tests that SASL/PLAIN clients with invalid password fail authentication. + */ + @Test + public void testInvalidPasswordSaslPlain() throws Exception { + String node = "0"; + SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL; + TestJaasConfig jaasConfig = configureMechanisms("PLAIN", Arrays.asList("PLAIN")); + jaasConfig.setPlainClientOptions(TestJaasConfig.USERNAME, "invalidpassword"); + + server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs); + createClientConnection(securityProtocol, node); + NetworkTestUtils.waitForChannelClose(selector, node); + } + + /** + * Tests that SASL/PLAIN clients with invalid username fail authentication. + */ + @Test + public void testInvalidUsernameSaslPlain() throws Exception { + String node = "0"; + SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL; + TestJaasConfig jaasConfig = configureMechanisms("PLAIN", Arrays.asList("PLAIN")); + jaasConfig.setPlainClientOptions("invaliduser", TestJaasConfig.PASSWORD); + + server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs); + createClientConnection(securityProtocol, node); + NetworkTestUtils.waitForChannelClose(selector, node); + } + + /** + * Tests that SASL/PLAIN clients without valid username fail authentication. + */ + @Test + public void testMissingUsernameSaslPlain() throws Exception { + String node = "0"; + TestJaasConfig jaasConfig = configureMechanisms("PLAIN", Arrays.asList("PLAIN")); + jaasConfig.setPlainClientOptions(null, "mypassword"); + + SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL; + server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs); + createSelector(securityProtocol, saslClientConfigs); + InetSocketAddress addr = new InetSocketAddress("127.0.0.1", server.port()); + try { + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + fail("SASL/PLAIN channel created without username"); + } catch (KafkaException e) { + // Expected exception + } + } + + /** + * Tests that SASL/PLAIN clients with missing password in JAAS configuration fail authentication. + */ + @Test + public void testMissingPasswordSaslPlain() throws Exception { + String node = "0"; + TestJaasConfig jaasConfig = configureMechanisms("PLAIN", Arrays.asList("PLAIN")); + jaasConfig.setPlainClientOptions("myuser", null); + + SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL; + server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs); + createSelector(securityProtocol, saslClientConfigs); + InetSocketAddress addr = new InetSocketAddress("127.0.0.1", server.port()); + try { + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + fail("SASL/PLAIN channel created without password"); + } catch (KafkaException e) { + // Expected exception + } + } + + /** + * Tests that mechanisms that are not supported in Kafka can be plugged in without modifying + * Kafka code if Sasl client and server providers are available. + */ + @Test + public void testMechanismPluggability() throws Exception { + String node = "0"; + SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL; + configureMechanisms("DIGEST-MD5", Arrays.asList("DIGEST-MD5")); + + server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs); + createAndCheckClientConnection(securityProtocol, node); + } + + /** + * Tests that servers supporting multiple SASL mechanisms work with clients using + * any of the enabled mechanisms. + */ + @Test + public void testMultipleServerMechanisms() throws Exception { + SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL; + configureMechanisms("DIGEST-MD5", Arrays.asList("DIGEST-MD5", "PLAIN")); + server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs); + + String node1 = "1"; + saslClientConfigs.put(SaslConfigs.SASL_MECHANISM, "PLAIN"); + createAndCheckClientConnection(securityProtocol, node1); + + String node2 = "2"; + saslClientConfigs.put(SaslConfigs.SASL_MECHANISM, "DIGEST-MD5"); + createSelector(securityProtocol, saslClientConfigs); + InetSocketAddress addr = new InetSocketAddress("127.0.0.1", server.port()); + selector.connect(node2, addr, BUFFER_SIZE, BUFFER_SIZE); + NetworkTestUtils.checkClientConnection(selector, node2, 100, 10); + } + + /** + * Tests that any invalid data during Kafka SASL handshake request flow + * or the actual SASL authentication flow result in authentication failure + * and do not cause any failures in the server. + */ + @Test + public void testInvalidSaslPacket() throws Exception { + SecurityProtocol securityProtocol = SecurityProtocol.SASL_PLAINTEXT; + configureMechanisms("PLAIN", Arrays.asList("PLAIN")); + server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs); + + // Send invalid SASL packet after valid handshake request + String node1 = "invalid1"; + createClientConnection(SecurityProtocol.PLAINTEXT, node1); + sendHandshakeRequest(node1); + Random random = new Random(); + byte[] bytes = new byte[1024]; + random.nextBytes(bytes); + selector.send(new NetworkSend(node1, ByteBuffer.wrap(bytes))); + NetworkTestUtils.waitForChannelClose(selector, node1); + selector.close(); + + // Test good connection still works + createAndCheckClientConnection(securityProtocol, "good1"); + + // Send invalid SASL packet before handshake request + String node2 = "invalid2"; + createClientConnection(SecurityProtocol.PLAINTEXT, node2); + random.nextBytes(bytes); + selector.send(new NetworkSend(node2, ByteBuffer.wrap(bytes))); + NetworkTestUtils.waitForChannelClose(selector, node2); + selector.close(); + + // Test good connection still works + createAndCheckClientConnection(securityProtocol, "good2"); + } + + /** + * Tests that packets that are too big during Kafka SASL handshake request flow + * or the actual SASL authentication flow result in authentication failure + * and do not cause any failures in the server. + */ + @Test + public void testPacketSizeTooBig() throws Exception { + SecurityProtocol securityProtocol = SecurityProtocol.SASL_PLAINTEXT; + configureMechanisms("PLAIN", Arrays.asList("PLAIN")); + server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs); + + // Send SASL packet with large size after valid handshake request + String node1 = "invalid1"; + createClientConnection(SecurityProtocol.PLAINTEXT, node1); + sendHandshakeRequest(node1); + ByteBuffer buffer = ByteBuffer.allocate(1024); + buffer.putInt(Integer.MAX_VALUE); + buffer.put(new byte[buffer.capacity() - 4]); + buffer.rewind(); + selector.send(new NetworkSend(node1, buffer)); + NetworkTestUtils.waitForChannelClose(selector, node1); + selector.close(); + + // Test good connection still works + createAndCheckClientConnection(securityProtocol, "good1"); + + // Send packet with large size before handshake request + String node2 = "invalid2"; + createClientConnection(SecurityProtocol.PLAINTEXT, node2); + buffer.clear(); + buffer.putInt(Integer.MAX_VALUE); + buffer.put(new byte[buffer.capacity() - 4]); + buffer.rewind(); + selector.send(new NetworkSend(node2, buffer)); + NetworkTestUtils.waitForChannelClose(selector, node2); + selector.close(); + + // Test good connection still works + createAndCheckClientConnection(securityProtocol, "good2"); + } + + /** + * Tests that Kafka requests that are forbidden until successful authentication result + * in authentication failure and do not cause any failures in the server. + */ + @Test + public void testDisallowedKafkaRequestsBeforeAuthentication() throws Exception { + SecurityProtocol securityProtocol = SecurityProtocol.SASL_PLAINTEXT; + configureMechanisms("PLAIN", Arrays.asList("PLAIN")); + server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs); + + // Send metadata request before Kafka SASL handshake request + String node1 = "invalid1"; + createClientConnection(SecurityProtocol.PLAINTEXT, node1); + RequestHeader metadataRequestHeader1 = new RequestHeader(ApiKeys.METADATA.id, "someclient", 1); + MetadataRequest metadataRequest1 = new MetadataRequest(Collections.singletonList("sometopic")); + selector.send(new NetworkSend(node1, RequestSend.serialize(metadataRequestHeader1, metadataRequest1.toStruct()))); + NetworkTestUtils.waitForChannelClose(selector, node1); + selector.close(); + + // Test good connection still works + createAndCheckClientConnection(securityProtocol, "good1"); + + // Send metadata request after Kafka SASL handshake request + String node2 = "invalid2"; + createClientConnection(SecurityProtocol.PLAINTEXT, node2); + sendHandshakeRequest(node2); + RequestHeader metadataRequestHeader2 = new RequestHeader(ApiKeys.METADATA.id, "someclient", 2); + MetadataRequest metadataRequest2 = new MetadataRequest(Collections.singletonList("sometopic")); + selector.send(new NetworkSend(node2, RequestSend.serialize(metadataRequestHeader2, metadataRequest2.toStruct()))); + NetworkTestUtils.waitForChannelClose(selector, node2); + selector.close(); + + // Test good connection still works + createAndCheckClientConnection(securityProtocol, "good2"); + } + + /** + * Tests that connections cannot be created if the login module class is unavailable. + */ + @Test + public void testInvalidLoginModule() throws Exception { + TestJaasConfig jaasConfig = configureMechanisms("PLAIN", Arrays.asList("PLAIN")); + jaasConfig.createOrUpdateEntry(JaasUtils.LOGIN_CONTEXT_CLIENT, "InvalidLoginModule", TestJaasConfig.defaultClientOptions()); + + SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL; + server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs); + try { + createSelector(securityProtocol, saslClientConfigs); + fail("SASL/PLAIN channel created without valid login module"); + } catch (KafkaException e) { + // Expected exception + } + } + + /** + * Tests that mechanisms with default implementation in Kafka may be disabled in + * the Kafka server by removing from the enabled mechanism list. + */ + @Test + public void testDisabledMechanism() throws Exception { + String node = "0"; + SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL; + configureMechanisms("PLAIN", Arrays.asList("DIGEST-MD5")); + + server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs); + createClientConnection(securityProtocol, node); + NetworkTestUtils.waitForChannelClose(selector, node); + } + + /** + * Tests that clients using invalid SASL mechanisms fail authentication. + */ + @Test + public void testInvalidMechanism() throws Exception { + String node = "0"; + SecurityProtocol securityProtocol = SecurityProtocol.SASL_SSL; + configureMechanisms("PLAIN", Arrays.asList("PLAIN")); + saslClientConfigs.put(SaslConfigs.SASL_MECHANISM, "INVALID"); + + server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs); + createClientConnection(securityProtocol, node); + NetworkTestUtils.waitForChannelClose(selector, node); + } + + private TestJaasConfig configureMechanisms(String clientMechanism, List serverMechanisms) { + saslClientConfigs.put(SaslConfigs.SASL_MECHANISM, clientMechanism); + saslServerConfigs.put(SaslConfigs.SASL_ENABLED_MECHANISMS, serverMechanisms); + return TestJaasConfig.createConfiguration(clientMechanism, serverMechanisms); + } + + private void createSelector(SecurityProtocol securityProtocol, Map clientConfigs) { + String saslMechanism = (String) saslClientConfigs.get(SaslConfigs.SASL_MECHANISM); + this.channelBuilder = ChannelBuilders.create(securityProtocol, Mode.CLIENT, LoginType.CLIENT, clientConfigs, saslMechanism, true); + this.selector = NetworkTestUtils.createSelector(channelBuilder); + } + + private void createClientConnection(SecurityProtocol securityProtocol, String node) throws Exception { + createSelector(securityProtocol, saslClientConfigs); + InetSocketAddress addr = new InetSocketAddress("127.0.0.1", server.port()); + selector.connect(node, addr, BUFFER_SIZE, BUFFER_SIZE); + } + + private void createAndCheckClientConnection(SecurityProtocol securityProtocol, String node) throws Exception { + createClientConnection(securityProtocol, node); + NetworkTestUtils.checkClientConnection(selector, node, 100, 10); + selector.close(); + selector = null; + } + + private void sendHandshakeRequest(String node) throws Exception { + RequestHeader header = new RequestHeader(ApiKeys.SASL_HANDSHAKE.id, "someclient", 1); + SaslHandshakeRequest handshakeRequest = new SaslHandshakeRequest("PLAIN"); + selector.send(new NetworkSend(node, RequestSend.serialize(header, handshakeRequest.toStruct()))); + int waitSeconds = 10; + do { + selector.poll(1000); + } while (selector.completedSends().isEmpty() && waitSeconds-- > 0); + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/TestDigestLoginModule.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/TestDigestLoginModule.java new file mode 100644 index 0000000000000..2923a5a618b66 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/TestDigestLoginModule.java @@ -0,0 +1,109 @@ +/** + * 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 org.apache.kafka.common.security.authenticator; + +import java.io.IOException; +import java.security.Provider; +import java.security.Security; +import java.util.Arrays; +import java.util.Enumeration; +import java.util.HashMap; +import java.util.Map; + +import javax.security.auth.callback.Callback; +import javax.security.auth.callback.CallbackHandler; +import javax.security.auth.callback.NameCallback; +import javax.security.auth.callback.PasswordCallback; +import javax.security.auth.callback.UnsupportedCallbackException; +import javax.security.sasl.AuthorizeCallback; +import javax.security.sasl.RealmCallback; +import javax.security.sasl.Sasl; +import javax.security.sasl.SaslException; +import javax.security.sasl.SaslServer; +import javax.security.sasl.SaslServerFactory; + +import org.apache.kafka.common.security.plain.PlainLoginModule; + +/** + * Digest-MD5 login module for multi-mechanism tests. Since callback handlers are not configurable in Kafka + * yet, this replaces the standard Digest-MD5 SASL server provider with one that invokes the test callback handler. + * This login module uses the same format as PlainLoginModule and hence simply reuses the same methods. + * + */ +public class TestDigestLoginModule extends PlainLoginModule { + + private static final SaslServerFactory STANDARD_DIGEST_SASL_SERVER_FACTORY; + static { + SaslServerFactory digestSaslServerFactory = null; + Enumeration factories = Sasl.getSaslServerFactories(); + Map emptyProps = new HashMap<>(); + while (factories.hasMoreElements()) { + SaslServerFactory factory = factories.nextElement(); + if (Arrays.asList(factory.getMechanismNames(emptyProps)).contains("DIGEST-MD5")) { + digestSaslServerFactory = factory; + break; + } + } + STANDARD_DIGEST_SASL_SERVER_FACTORY = digestSaslServerFactory; + Security.insertProviderAt(new DigestSaslServerProvider(), 1); + } + + public static class DigestServerCallbackHandler implements CallbackHandler { + + @Override + public void handle(Callback[] callbacks) throws IOException, UnsupportedCallbackException { + for (Callback callback : callbacks) { + if (callback instanceof NameCallback) { + NameCallback nameCallback = (NameCallback) callback; + nameCallback.setName(nameCallback.getDefaultName()); + } else if (callback instanceof PasswordCallback) { + PasswordCallback passwordCallback = (PasswordCallback) callback; + passwordCallback.setPassword(TestJaasConfig.PASSWORD.toCharArray()); + } else if (callback instanceof RealmCallback) { + RealmCallback realmCallback = (RealmCallback) callback; + realmCallback.setText(realmCallback.getDefaultText()); + } else if (callback instanceof AuthorizeCallback) { + AuthorizeCallback authCallback = (AuthorizeCallback) callback; + if (TestJaasConfig.USERNAME.equals(authCallback.getAuthenticationID())) { + authCallback.setAuthorized(true); + authCallback.setAuthorizedID(authCallback.getAuthenticationID()); + } + } + } + } + } + + public static class DigestSaslServerFactory implements SaslServerFactory { + + @Override + public SaslServer createSaslServer(String mechanism, String protocol, String serverName, Map props, CallbackHandler cbh) + throws SaslException { + return STANDARD_DIGEST_SASL_SERVER_FACTORY.createSaslServer(mechanism, protocol, serverName, props, new DigestServerCallbackHandler()); + } + + @Override + public String[] getMechanismNames(Map props) { + return new String[] {"DIGEST-MD5"}; + } + } + + public static class DigestSaslServerProvider extends Provider { + + private static final long serialVersionUID = 1L; + + protected DigestSaslServerProvider() { + super("Test SASL/Digest-MD5 Server Provider", 1.0, "Test SASL/Digest-MD5 Server Provider for Kafka"); + super.put("SaslServerFactory.DIGEST-MD5", TestDigestLoginModule.DigestSaslServerFactory.class.getName()); + } + } +} diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/TestJaasConfig.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/TestJaasConfig.java new file mode 100644 index 0000000000000..2291cc188dc12 --- /dev/null +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/TestJaasConfig.java @@ -0,0 +1,89 @@ +/** + * 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 org.apache.kafka.common.security.authenticator; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; + +import javax.security.auth.login.AppConfigurationEntry; +import javax.security.auth.login.Configuration; +import javax.security.auth.login.AppConfigurationEntry.LoginModuleControlFlag; + +import org.apache.kafka.common.security.JaasUtils; +import org.apache.kafka.common.security.plain.PlainLoginModule; + +public class TestJaasConfig extends Configuration { + + static final String USERNAME = "myuser"; + static final String PASSWORD = "mypassword"; + + private Map entryMap = new HashMap<>(); + + public static TestJaasConfig createConfiguration(String clientMechanism, List serverMechanisms) { + TestJaasConfig config = new TestJaasConfig(); + config.createOrUpdateEntry(JaasUtils.LOGIN_CONTEXT_CLIENT, loginModule(clientMechanism), defaultClientOptions()); + for (String mechanism : serverMechanisms) { + config.createOrUpdateEntry(JaasUtils.LOGIN_CONTEXT_SERVER, loginModule(mechanism), defaultServerOptions()); + } + Configuration.setConfiguration(config); + return config; + } + + public void setPlainClientOptions(String clientUsername, String clientPassword) { + Map options = new HashMap<>(); + if (clientUsername != null) + options.put("username", clientUsername); + if (clientPassword != null) + options.put("password", clientPassword); + createOrUpdateEntry(JaasUtils.LOGIN_CONTEXT_CLIENT, PlainLoginModule.class.getName(), options); + } + + public void createOrUpdateEntry(String name, String loginModule, Map options) { + AppConfigurationEntry entry = new AppConfigurationEntry(loginModule, LoginModuleControlFlag.REQUIRED, options); + entryMap.put(name, new AppConfigurationEntry[] {entry}); + } + + @Override + public AppConfigurationEntry[] getAppConfigurationEntry(String name) { + return entryMap.get(name); + } + + private static String loginModule(String mechanism) { + String loginModule; + switch (mechanism) { + case "PLAIN": + loginModule = PlainLoginModule.class.getName(); + break; + case "DIGEST-MD5": + loginModule = TestDigestLoginModule.class.getName(); + break; + default: + throw new IllegalArgumentException("Unsupported mechanism " + mechanism); + } + return loginModule; + } + + public static Map defaultClientOptions() { + Map options = new HashMap<>(); + options.put("username", USERNAME); + options.put("password", PASSWORD); + return options; + } + + public static Map defaultServerOptions() { + Map options = new HashMap<>(); + options.put("user_" + USERNAME, PASSWORD); + return options; + } +} \ No newline at end of file diff --git a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala index 870caca2b9b7c..fec96cda0bddb 100644 --- a/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala +++ b/core/src/test/scala/integration/kafka/api/EndToEndAuthorizationTest.scala @@ -78,6 +78,9 @@ trait EndToEndAuthorizationTest extends IntegrationTestHarness with SaslSetup { val kafkaPrincipal: String override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks")) + protected def kafkaClientSaslMechanism = "GSSAPI" + protected def kafkaServerSaslMechanisms = List("GSSAPI") + override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, Some(kafkaServerSaslMechanisms))) val topicResource = new Resource(Topic, topic) val groupResource = new Resource(Group, group) @@ -143,7 +146,7 @@ trait EndToEndAuthorizationTest extends IntegrationTestHarness with SaslSetup { case SecurityProtocol.SSL => startSasl(ZkSasl, null, null) case _ => - startSasl(Both, List("GSSAPI"), List("GSSAPI")) + startSasl(Both, List(kafkaClientSaslMechanism), kafkaServerSaslMechanisms) } super.setUp AclCommand.main(topicBrokerReadAclArgs) diff --git a/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala index d203245ace72e..fc79c60edfb2d 100644 --- a/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslMultiMechanismConsumerTest.scala @@ -27,7 +27,7 @@ class SaslMultiMechanismConsumerTest extends BaseConsumerTest with SaslTestHarne this.serverConfig.setProperty(KafkaConfig.ZkEnableSecureAclsProp, "true") override protected def securityProtocol = SecurityProtocol.SASL_SSL override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks")) - override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, kafkaServerSaslMechanisms)) + override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, Some(kafkaServerSaslMechanisms))) @Test def testMultipleBrokerMechanisms() { @@ -35,7 +35,7 @@ class SaslMultiMechanismConsumerTest extends BaseConsumerTest with SaslTestHarne val plainSaslProducer = producers(0) val plainSaslConsumer = consumers(0) - val gssapiSaslProperties = kafkaSaslProperties("GSSAPI", kafkaServerSaslMechanisms) + val gssapiSaslProperties = kafkaSaslProperties("GSSAPI") val gssapiSaslProducer = TestUtils.createNewProducer(brokerList, securityProtocol = this.securityProtocol, trustStoreFile = this.trustStoreFile, diff --git a/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala b/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala index 687cfc39033c1..bdca577c0e878 100644 --- a/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala +++ b/core/src/test/scala/integration/kafka/api/SaslPlainPlaintextConsumerTest.scala @@ -23,5 +23,5 @@ class SaslPlainPlaintextConsumerTest extends BaseConsumerTest with SaslTestHarne this.serverConfig.setProperty(KafkaConfig.ZkEnableSecureAclsProp, "true") override protected def securityProtocol = SecurityProtocol.SASL_PLAINTEXT override protected lazy val trustStoreFile = Some(File.createTempFile("truststore", ".jks")) - override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, kafkaServerSaslMechanisms)) + override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, Some(kafkaServerSaslMechanisms))) } diff --git a/core/src/test/scala/integration/kafka/api/SaslPlainSslEndToEndAuthorizationTest.scala b/core/src/test/scala/integration/kafka/api/SaslPlainSslEndToEndAuthorizationTest.scala new file mode 100644 index 0000000000000..63636c013f6a2 --- /dev/null +++ b/core/src/test/scala/integration/kafka/api/SaslPlainSslEndToEndAuthorizationTest.scala @@ -0,0 +1,28 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package kafka.api + +import kafka.server.KafkaConfig +import org.apache.kafka.common.protocol.SecurityProtocol + +class SaslPlainSslEndToEndAuthorizationTest extends EndToEndAuthorizationTest { + override protected def securityProtocol = SecurityProtocol.SASL_SSL + override protected def kafkaClientSaslMechanism = "PLAIN" + override protected def kafkaServerSaslMechanisms = List("PLAIN") + override val clientPrincipal = "testuser" + override val kafkaPrincipal = "admin" +} diff --git a/core/src/test/scala/integration/kafka/api/SaslSetup.scala b/core/src/test/scala/integration/kafka/api/SaslSetup.scala index acc86e3a692bb..765f191639826 100644 --- a/core/src/test/scala/integration/kafka/api/SaslSetup.scala +++ b/core/src/test/scala/integration/kafka/api/SaslSetup.scala @@ -18,11 +18,14 @@ package kafka.api import java.io.File +import java.util.Properties import javax.security.auth.login.Configuration import kafka.security.minikdc.MiniKdc +import kafka.server.KafkaConfig import kafka.utils.{JaasTestUtils, TestUtils} import org.apache.kafka.common.security.JaasUtils import org.apache.kafka.common.security.authenticator.LoginManager +import org.apache.kafka.common.config.SaslConfigs /* * Implements an enumeration for the modes enabled here: @@ -81,4 +84,14 @@ trait SaslSetup { System.clearProperty("zookeeper.authProvider.1") Configuration.setConfiguration(null) } + + def kafkaSaslProperties(clientSaslMechanism: String, serverSaslMechanisms: Option[Seq[String]] = None) = { + val props = new Properties + props.put(SaslConfigs.SASL_MECHANISM, clientSaslMechanism) + serverSaslMechanisms.foreach { serverMechanisms => + props.put(KafkaConfig.SaslMechanismInterBrokerProtocolProp, clientSaslMechanism) + props.put(SaslConfigs.SASL_ENABLED_MECHANISMS, serverMechanisms.mkString(",")) + } + props + } } diff --git a/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala index 55319195443d8..8fd3eb4422e0e 100644 --- a/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala +++ b/core/src/test/scala/integration/kafka/api/SaslTestHarness.scala @@ -13,11 +13,7 @@ package kafka.api import kafka.zk.ZooKeeperTestHarness -import kafka.server.KafkaConfig import org.junit.{After, Before} -import java.util.Properties -import scala.collection.JavaConverters._ -import org.apache.kafka.common.config.SaslConfigs trait SaslTestHarness extends ZooKeeperTestHarness with SaslSetup { protected val zkSaslEnabled: Boolean @@ -42,13 +38,4 @@ trait SaslTestHarness extends ZooKeeperTestHarness with SaslSetup { super.tearDown closeSasl() } - - def kafkaSaslProperties(kafkaClientSaslMechanism: String, kafkaServerSaslMechanisms: List[String]) = { - val props = new Properties - props.put(SaslConfigs.SASL_MECHANISM, kafkaClientSaslMechanism) - props.put(KafkaConfig.SaslMechanismInterBrokerProtocolProp, kafkaClientSaslMechanism) - props.put(SaslConfigs.SASL_ENABLED_MECHANISMS, kafkaServerSaslMechanisms.asJava) - props - } - } From fdded2da6e0f57431fc19fd1705bbb6373c1c863 Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Thu, 28 Apr 2016 14:08:10 -0700 Subject: [PATCH 189/206] KAFKA-3636; Change default max session timeout to 5 minutes Author: Jason Gustafson Reviewers: Guozhang Wang , Onur Karaman , Ismael Juma Closes #1284 from hachikuji/KAFKA-3636 --- .../main/scala/kafka/server/KafkaConfig.scala | 18 +++++++++--------- 1 file changed, 9 insertions(+), 9 deletions(-) diff --git a/core/src/main/scala/kafka/server/KafkaConfig.scala b/core/src/main/scala/kafka/server/KafkaConfig.scala index 5e28bd7d16c07..c5da55d4acac6 100755 --- a/core/src/main/scala/kafka/server/KafkaConfig.scala +++ b/core/src/main/scala/kafka/server/KafkaConfig.scala @@ -130,9 +130,9 @@ object Defaults { val ControlledShutdownRetryBackoffMs = 5000 val ControlledShutdownEnable = true - /** ********* Consumer coordinator configuration ***********/ - val ConsumerMinSessionTimeoutMs = 6000 - val ConsumerMaxSessionTimeoutMs = 30000 + /** ********* Group coordinator configuration ***********/ + val GroupMinSessionTimeoutMs = 6000 + val GroupMaxSessionTimeoutMs = 300000 /** ********* Offset management configuration ***********/ val OffsetMetadataMaxSize = OffsetConfig.DefaultMaxMetadataSize @@ -371,7 +371,7 @@ object KafkaConfig { val AuthorizerClassNameDoc = "The authorizer class that should be used for authorization" /** ********* Socket Server Configuration ***********/ val PortDoc = "DEPRECATED: only used when `listeners` is not set. " + - "Use `listeners` instead. \n" + + "Use `listeners` instead. \n" + "the port to listen and accept connections on" val HostNameDoc = "DEPRECATED: only used when `listeners` is not set. " + "Use `listeners` instead. \n" + @@ -484,8 +484,8 @@ object KafkaConfig { val ControlledShutdownRetryBackoffMsDoc = "Before each retry, the system needs time to recover from the state that caused the previous failure (Controller fail over, replica lag etc). This config determines the amount of time to wait before retrying." val ControlledShutdownEnableDoc = "Enable controlled shutdown of the server" /** ********* Consumer coordinator configuration ***********/ - val ConsumerMinSessionTimeoutMsDoc = "The minimum allowed session timeout for registered consumers. Shorter timeouts leader to quicker failure detection at the cost of more frequent consumer heartbeating, which can overwhelm broker resources." - val ConsumerMaxSessionTimeoutMsDoc = "The maximum allowed session timeout for registered consumers. Longer timeouts give consumers more time to process messages in between heartbeats at the cost of a longer time to detect failures." + val GroupMinSessionTimeoutMsDoc = "The minimum allowed session timeout for registered consumers. Shorter timeouts leader to quicker failure detection at the cost of more frequent consumer heartbeating, which can overwhelm broker resources." + val GroupMaxSessionTimeoutMsDoc = "The maximum allowed session timeout for registered consumers. Longer timeouts give consumers more time to process messages in between heartbeats at the cost of a longer time to detect failures." /** ********* Offset management configuration ***********/ val OffsetMetadataMaxSizeDoc = "The maximum size for a metadata entry associated with an offset commit" val OffsetsLoadBufferSizeDoc = "Batch size for reading from the offsets segments when loading offsets into the cache." @@ -661,9 +661,9 @@ object KafkaConfig { .define(ControlledShutdownRetryBackoffMsProp, LONG, Defaults.ControlledShutdownRetryBackoffMs, MEDIUM, ControlledShutdownRetryBackoffMsDoc) .define(ControlledShutdownEnableProp, BOOLEAN, Defaults.ControlledShutdownEnable, MEDIUM, ControlledShutdownEnableDoc) - /** ********* Consumer coordinator configuration ***********/ - .define(GroupMinSessionTimeoutMsProp, INT, Defaults.ConsumerMinSessionTimeoutMs, MEDIUM, ConsumerMinSessionTimeoutMsDoc) - .define(GroupMaxSessionTimeoutMsProp, INT, Defaults.ConsumerMaxSessionTimeoutMs, MEDIUM, ConsumerMaxSessionTimeoutMsDoc) + /** ********* Group coordinator configuration ***********/ + .define(GroupMinSessionTimeoutMsProp, INT, Defaults.GroupMinSessionTimeoutMs, MEDIUM, GroupMinSessionTimeoutMsDoc) + .define(GroupMaxSessionTimeoutMsProp, INT, Defaults.GroupMaxSessionTimeoutMs, MEDIUM, GroupMaxSessionTimeoutMsDoc) /** ********* Offset management configuration ***********/ .define(OffsetMetadataMaxSizeProp, INT, Defaults.OffsetMetadataMaxSize, HIGH, OffsetMetadataMaxSizeDoc) From 346df72736d03acb4a6f5681e705ce3e330900a8 Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Thu, 28 Apr 2016 14:46:47 -0700 Subject: [PATCH 190/206] MINOR: Avoid compiler warnings when registering metrics in KafkaServer Author: Liquan Pei Reviewers: Ismael Juma Closes #1280 from Ishiihara/minor-warning-fix --- .../kafka/controller/KafkaController.scala | 19 ++++++++++++++----- 1 file changed, 14 insertions(+), 5 deletions(-) diff --git a/core/src/main/scala/kafka/controller/KafkaController.scala b/core/src/main/scala/kafka/controller/KafkaController.scala index 103f6cf575d43..6c503a57a23f0 100755 --- a/core/src/main/scala/kafka/controller/KafkaController.scala +++ b/core/src/main/scala/kafka/controller/KafkaController.scala @@ -23,15 +23,16 @@ import org.apache.kafka.common.protocol.ApiKeys import org.apache.kafka.common.requests.{AbstractRequest, AbstractRequestResponse} import scala.collection._ -import com.yammer.metrics.core.Gauge +import com.yammer.metrics.core.{Gauge, Meter} import java.util.concurrent.TimeUnit + import kafka.admin.AdminUtils import kafka.admin.PreferredReplicaLeaderElectionCommand import kafka.api._ import kafka.cluster.Broker import kafka.common._ import kafka.log.LogConfig -import kafka.metrics.{KafkaTimer, KafkaMetricsGroup} +import kafka.metrics.{KafkaMetricsGroup, KafkaTimer} import kafka.utils.ZkUtils._ import kafka.utils._ import kafka.utils.CoreUtils._ @@ -39,8 +40,9 @@ import org.apache.zookeeper.Watcher.Event.KeeperState import org.apache.kafka.common.metrics.Metrics import org.apache.kafka.common.utils.Time import org.I0Itec.zkclient.{IZkChildListener, IZkDataListener, IZkStateListener, ZkClient, ZkConnection} -import org.I0Itec.zkclient.exception.{ZkNodeExistsException, ZkNoNodeException} +import org.I0Itec.zkclient.exception.{ZkNoNodeException, ZkNodeExistsException} import java.util.concurrent.locks.ReentrantLock + import kafka.server._ import kafka.common.TopicAndPartition @@ -1468,6 +1470,13 @@ case class LeaderIsrAndControllerEpoch(leaderAndIsr: LeaderAndIsr, controllerEpo } object ControllerStats extends KafkaMetricsGroup { - val uncleanLeaderElectionRate = newMeter("UncleanLeaderElectionsPerSec", "elections", TimeUnit.SECONDS) - val leaderElectionTimer = new KafkaTimer(newTimer("LeaderElectionRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS)) + + private val _uncleanLeaderElectionRate = newMeter("UncleanLeaderElectionsPerSec", "elections", TimeUnit.SECONDS) + private val _leaderElectionTimer = new KafkaTimer(newTimer("LeaderElectionRateAndTimeMs", TimeUnit.MILLISECONDS, TimeUnit.SECONDS)) + + // KafkaServer needs to initialize controller metrics during startup. We perform initialization + // through method calls to avoid Scala compiler warnings. + def uncleanLeaderElectionRate: Meter = _uncleanLeaderElectionRate + + def leaderElectionTimer: KafkaTimer = _leaderElectionTimer } From 0ada3b1fc215bb8efdf5c7ae27eb52b29e0fbbdc Mon Sep 17 00:00:00 2001 From: Ashish Singh Date: Thu, 28 Apr 2016 15:49:01 -0700 Subject: [PATCH 191/206] KAFKA-3382: Add system test for ReplicationVerificationTool Author: Ashish Singh Reviewers: Geoff Anderson , Ewen Cheslack-Postava Closes #1160 from SinghAsDev/KAFKA-3382 --- .../services/replica_verification_tool.py | 81 +++++++++++++++++ .../kafkatest/services/verifiable_producer.py | 8 +- .../tests/tools/replica_verification_test.py | 88 +++++++++++++++++++ 3 files changed, 176 insertions(+), 1 deletion(-) create mode 100644 tests/kafkatest/services/replica_verification_tool.py create mode 100644 tests/kafkatest/tests/tools/replica_verification_test.py diff --git a/tests/kafkatest/services/replica_verification_tool.py b/tests/kafkatest/services/replica_verification_tool.py new file mode 100644 index 0000000000000..f6374fbcc259e --- /dev/null +++ b/tests/kafkatest/services/replica_verification_tool.py @@ -0,0 +1,81 @@ +# 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. + +from ducktape.services.background_thread import BackgroundThreadService + +from kafkatest.services.kafka.directory import kafka_dir +from kafkatest.services.security.security_config import SecurityConfig + +import re + +class ReplicaVerificationTool(BackgroundThreadService): + + logs = { + "producer_log": { + "path": "/mnt/replica_verification_tool.log", + "collect_default": False} + } + + def __init__(self, context, num_nodes, kafka, topic, report_interval_ms, security_protocol="PLAINTEXT"): + super(ReplicaVerificationTool, self).__init__(context, num_nodes) + + self.kafka = kafka + self.topic = topic + self.report_interval_ms = report_interval_ms + self.security_protocol = security_protocol + self.security_config = SecurityConfig(security_protocol) + self.partition_lag = {} + + def _worker(self, idx, node): + cmd = self.start_cmd(node) + self.logger.debug("ReplicaVerificationTool %d command: %s" % (idx, cmd)) + self.security_config.setup_node(node) + for line in node.account.ssh_capture(cmd): + self.logger.debug("Parsing line:{}".format(line)) + + parsed = re.search('.*max lag is (.+?) for partition \[(.+?)\] at', line) + if parsed: + lag = int(parsed.group(1)) + topic_partition = parsed.group(2) + self.logger.debug("Setting max lag for {} as {}".format(topic_partition, lag)) + self.partition_lag[topic_partition] = lag + + def get_lag_for_partition(self, topic, partition): + """ + Get latest lag for given topic-partition + + Args: + topic: a topic + partition: a partition of the topic + """ + topic_partition = topic + ',' + str(partition) + lag = self.partition_lag[topic_partition] + self.logger.debug("Retuning lag for {} as {}".format(topic_partition, lag)) + return lag + + def start_cmd(self, node): + cmd = "/opt/%s/bin/" % kafka_dir(node) + cmd += "kafka-run-class.sh kafka.tools.ReplicaVerificationTool" + cmd += " --broker-list %s --topic-white-list %s --time -2 --report-interval-ms %s" % (self.kafka.bootstrap_servers(self.security_protocol), self.topic, self.report_interval_ms) + + cmd += " 2>> /mnt/replica_verification_tool.log | tee -a /mnt/replica_verification_tool.log &" + return cmd + + def stop_node(self, node): + node.account.kill_process("java", clean_shutdown=True, allow_fail=True) + + def clean_node(self, node): + node.account.kill_process("java", clean_shutdown=False, allow_fail=True) + node.account.ssh("rm -rf /mnt/replica_verification_tool.log", allow_fail=False) \ No newline at end of file diff --git a/tests/kafkatest/services/verifiable_producer.py b/tests/kafkatest/services/verifiable_producer.py index 414da84ad9e64..500410f134f10 100644 --- a/tests/kafkatest/services/verifiable_producer.py +++ b/tests/kafkatest/services/verifiable_producer.py @@ -43,7 +43,7 @@ class VerifiableProducer(BackgroundThreadService): } def __init__(self, context, num_nodes, kafka, topic, max_messages=-1, throughput=100000, - message_validator=is_int, compression_types=None, version=TRUNK): + message_validator=is_int, compression_types=None, version=TRUNK, acks=None): """ :param max_messages is a number of messages to be produced per producer :param message_validator checks for an expected format of messages produced. There are @@ -71,6 +71,7 @@ def __init__(self, context, num_nodes, kafka, topic, max_messages=-1, throughput self.acked_values = [] self.not_acked_values = [] self.produced_count = {} + self.acks = acks @property @@ -96,6 +97,9 @@ def _worker(self, idx, node): # Create and upload config file producer_prop_file = self.prop_file(node) + if self.acks is not None: + self.logger.info("VerifiableProducer (index = %d) will use acks = %s", idx, self.acks) + producer_prop_file += "\nacks=%s\n" % self.acks self.logger.info("verifiable_producer.properties:") self.logger.info(producer_prop_file) node.account.create_file(VerifiableProducer.CONFIG_FILE, producer_prop_file) @@ -156,6 +160,8 @@ def start_cmd(self, node, idx): cmd += " --throughput %s" % str(self.throughput) if self.message_validator == is_int_with_prefix: cmd += " --value-prefix %s" % str(idx) + if self.acks is not None: + cmd += " --acks %s\n" % str(self.acks) cmd += " --producer.config %s" % VerifiableProducer.CONFIG_FILE cmd += " 2>> %s | tee -a %s &" % (VerifiableProducer.STDOUT_CAPTURE, VerifiableProducer.STDOUT_CAPTURE) diff --git a/tests/kafkatest/tests/tools/replica_verification_test.py b/tests/kafkatest/tests/tools/replica_verification_test.py new file mode 100644 index 0000000000000..1b625e94db1f2 --- /dev/null +++ b/tests/kafkatest/tests/tools/replica_verification_test.py @@ -0,0 +1,88 @@ +# 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. + + +from ducktape.utils.util import wait_until +from ducktape.tests.test import Test +from kafkatest.services.verifiable_producer import VerifiableProducer + +from kafkatest.services.zookeeper import ZookeeperService +from kafkatest.services.kafka import KafkaService +from kafkatest.services.replica_verification_tool import ReplicaVerificationTool + +TOPIC = "topic-replica-verification" +REPORT_INTERVAL_MS = 1000 + +class ReplicaVerificationToolTest(Test): + """ + Tests ReplicaVerificationTool + """ + def __init__(self, test_context): + super(ReplicaVerificationToolTest, self).__init__(test_context) + self.num_zk = 1 + self.num_brokers = 2 + self.messages_received_count = 0 + self.topics = { + TOPIC: {'partitions': 1, 'replication-factor': 2} + } + + self.zk = ZookeeperService(test_context, self.num_zk) + self.kafka = None + self.producer = None + self.replica_verifier = None + + def setUp(self): + self.zk.start() + + def start_kafka(self, security_protocol, interbroker_security_protocol): + self.kafka = KafkaService( + self.test_context, self.num_brokers, + self.zk, security_protocol=security_protocol, + interbroker_security_protocol=interbroker_security_protocol, topics=self.topics) + self.kafka.start() + + def start_replica_verification_tool(self, security_protocol): + self.replica_verifier = ReplicaVerificationTool(self.test_context, 1, self.kafka, TOPIC, report_interval_ms=REPORT_INTERVAL_MS, security_protocol=security_protocol) + self.replica_verifier.start() + + def start_producer(self, max_messages, acks, timeout): + # This will produce to kafka cluster + self.producer = VerifiableProducer(self.test_context, num_nodes=1, kafka=self.kafka, topic=TOPIC, throughput=1000, acks=acks, max_messages=max_messages) + current_acked = self.producer.num_acked + self.logger.info("current_acked = %s" % current_acked) + self.producer.start() + wait_until(lambda: acks == 0 or self.producer.num_acked >= current_acked + max_messages, timeout_sec=timeout, + err_msg="Timeout awaiting messages to be produced and acked") + + def stop_producer(self): + self.producer.stop() + + def test_replica_lags(self, security_protocol='PLAINTEXT'): + """ + Tests ReplicaVerificationTool + :return: None + """ + self.start_kafka(security_protocol, security_protocol) + self.start_replica_verification_tool(security_protocol) + self.start_producer(max_messages=10, acks=-1, timeout=15) + # Verify that there is no lag in replicas and is correctly reported by ReplicaVerificationTool + wait_until(lambda: self.replica_verifier.get_lag_for_partition(TOPIC, 0) == 0, timeout_sec=10, + err_msg="Timed out waiting to reach zero replica lags.") + self.stop_producer() + + self.start_producer(max_messages=1000, acks=0, timeout=5) + # Verify that there is lag in replicas and is correctly reported by ReplicaVerificationTool + wait_until(lambda: self.replica_verifier.get_lag_for_partition(TOPIC, 0) > 0, timeout_sec=10, + err_msg="Timed out waiting to reach non-zero number of replica lags.") \ No newline at end of file From 68433dcfdc0ae078ee4e7d278c286a9b7c1b3e76 Mon Sep 17 00:00:00 2001 From: "Michael G. Noll" Date: Fri, 29 Apr 2016 07:44:03 -0700 Subject: [PATCH 192/206] KAFKA-3613: Consolidate TumblingWindows and HoppingWindows into TimeWindows This PR includes the same code as https://github.com/apache/kafka/pull/1261 but is rebased on latest trunk. Author: Michael G. Noll Reviewers: Matthias J. Sax, Guozhang Wang Closes #1277 from miguno/KAFKA-3613-v2 --- .../examples/pageview/PageViewTypedDemo.java | 4 +- .../pageview/PageViewUntypedDemo.java | 4 +- .../examples/wordcount/WordCountDemo.java | 2 +- .../kafka/streams/kstream/HoppingWindows.java | 95 ---- .../kafka/streams/kstream/JoinWindows.java | 27 +- .../kafka/streams/kstream/TimeWindows.java | 125 +++++ .../streams/kstream/TumblingWindows.java | 74 --- .../streams/kstream/UnlimitedWindows.java | 31 +- .../apache/kafka/streams/kstream/Window.java | 14 +- .../apache/kafka/streams/kstream/Windows.java | 6 +- .../{HoppingWindow.java => TimeWindow.java} | 12 +- .../kstream/internals/TumblingWindow.java | 38 -- .../kstream/internals/UnlimitedWindow.java | 8 +- .../streams/kstream/TimeWindowsTest.java | 123 +++++ .../streams/kstream/UnlimitedWindowsTest.java | 80 ++++ .../internals/KStreamWindowAggregateTest.java | 429 +++++++++--------- .../WindowedStreamPartitionerTest.java | 4 +- .../kstream/internals/WindowsTest.java | 70 --- .../streams/smoketest/SmokeTestClient.java | 4 +- 19 files changed, 615 insertions(+), 535 deletions(-) delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java create mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/TumblingWindows.java rename streams/src/main/java/org/apache/kafka/streams/kstream/internals/{HoppingWindow.java => TimeWindow.java} (75%) delete mode 100644 streams/src/main/java/org/apache/kafka/streams/kstream/internals/TumblingWindow.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java create mode 100644 streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java delete mode 100644 streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowsTest.java diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java index 4124b32c301d8..39ec41fab9c82 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java @@ -24,11 +24,11 @@ import org.apache.kafka.common.serialization.Serializer; import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; -import org.apache.kafka.streams.kstream.HoppingWindows; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.streams.StreamsConfig; @@ -160,7 +160,7 @@ public KeyValue apply(String user, PageViewByRegion vi return new KeyValue<>(viewRegion.region, viewRegion); } }) - .countByKey(HoppingWindows.of("GeoPageViewsWindow").with(7 * 24 * 60 * 60 * 1000), Serdes.String()) + .countByKey(TimeWindows.of("GeoPageViewsWindow", 7 * 24 * 60 * 60 * 1000L).advanceBy(1000), Serdes.String()) // TODO: we can merge ths toStream().map(...) with a single toStream(...) .toStream() .map(new KeyValueMapper, Long, KeyValue>() { diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java index e61842ffe3273..9a41b9ea4d83e 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java @@ -30,11 +30,11 @@ import org.apache.kafka.streams.KafkaStreams; import org.apache.kafka.streams.KeyValue; import org.apache.kafka.streams.StreamsConfig; -import org.apache.kafka.streams.kstream.HoppingWindows; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.KeyValueMapper; +import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.ValueMapper; import org.apache.kafka.streams.kstream.Windowed; @@ -99,7 +99,7 @@ public KeyValue apply(String user, JsonNode viewRegion) { return new KeyValue<>(viewRegion.get("region").textValue(), viewRegion); } }) - .countByKey(HoppingWindows.of("GeoPageViewsWindow").with(7 * 24 * 60 * 60 * 1000), Serdes.String()) + .countByKey(TimeWindows.of("GeoPageViewsWindow", 7 * 24 * 60 * 60 * 1000L).advanceBy(1000), Serdes.String()) // TODO: we can merge ths toStream().map(...) with a single toStream(...) .toStream() .map(new KeyValueMapper, Long, KeyValue>() { diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java index 5b52803f8079c..12395f99dcf5d 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/wordcount/WordCountDemo.java @@ -69,7 +69,7 @@ public Iterable apply(String value) { }).map(new KeyValueMapper>() { @Override public KeyValue apply(String key, String value) { - return new KeyValue(value, value); + return new KeyValue<>(value, value); } }) .countByKey("Counts"); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java deleted file mode 100644 index aa866e4274fc4..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/HoppingWindows.java +++ /dev/null @@ -1,95 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streams.kstream; - -import org.apache.kafka.streams.kstream.internals.HoppingWindow; - -import java.util.HashMap; -import java.util.Map; - -/** - * The hopping window specifications used for aggregations. - */ -public class HoppingWindows extends Windows { - - private static final long DEFAULT_SIZE_MS = 1000L; - - public final long size; - - public final long period; - - private HoppingWindows(String name, long size, long period) { - super(name); - - this.size = size; - this.period = period; - } - - /** - * Returns a half-interval hopping window definition with the window size in milliseconds - * of the form [ N * default_size, N * default_size + default_size ) - */ - public static HoppingWindows of(String name) { - return new HoppingWindows(name, DEFAULT_SIZE_MS, DEFAULT_SIZE_MS); - } - - /** - * Returns a new hopping window definition with the original size but reassign the window - * period in milliseconds of the form [ N * period, N * period + size ) - */ - public HoppingWindows with(long size) { - return new HoppingWindows(this.name, size, this.period); - } - - /** - * Returns a new hopping window definition with the original size but reassign the window - * period in milliseconds of the form [ N * period, N * period + size ) - */ - public HoppingWindows every(long period) { - return new HoppingWindows(this.name, this.size, period); - } - - @Override - public Map windowsFor(long timestamp) { - long enclosed = (size - 1) / period; - - long windowStart = Math.max(0, timestamp - timestamp % period - enclosed * period); - - Map windows = new HashMap<>(); - while (windowStart <= timestamp) { - // add the window - HoppingWindow window = new HoppingWindow(windowStart, windowStart + this.size); - windows.put(windowStart, window); - - // advance the step period - windowStart += this.period; - } - - return windows; - } - - @Override - public boolean equalTo(Windows other) { - if (!other.getClass().equals(HoppingWindows.class)) - return false; - - HoppingWindows otherWindows = (HoppingWindows) other; - - return this.size == otherWindows.size && this.period == otherWindows.period; - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java index 24dbdd33b2547..a74984a29b130 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java @@ -17,15 +17,14 @@ package org.apache.kafka.streams.kstream; - -import org.apache.kafka.streams.kstream.internals.TumblingWindow; +import org.apache.kafka.streams.kstream.internals.TimeWindow; import java.util.Map; /** * The window specifications used for joins. */ -public class JoinWindows extends Windows { +public class JoinWindows extends Windows { public final long before; public final long after; @@ -74,19 +73,29 @@ public JoinWindows after(long timeDifference) { } @Override - public Map windowsFor(long timestamp) { + public Map windowsFor(long timestamp) { // this function should never be called throw new UnsupportedOperationException("windowsFor() is not supported in JoinWindows"); } @Override - public boolean equalTo(Windows other) { - if (!other.getClass().equals(JoinWindows.class)) + public final boolean equals(Object o) { + if (o == this) { + return true; + } + if (!(o instanceof JoinWindows)) { return false; + } - JoinWindows otherWindows = (JoinWindows) other; + JoinWindows other = (JoinWindows) o; + return this.before == other.before && this.after == other.after; + } - return this.before == otherWindows.before && this.after == otherWindows.after; + @Override + public int hashCode() { + int result = (int) (before ^ (before >>> 32)); + result = 31 * result + (int) (after ^ (after >>> 32)); + return result; } -} +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java new file mode 100644 index 0000000000000..fa3a9d822aece --- /dev/null +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java @@ -0,0 +1,125 @@ +/** + * 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 org.apache.kafka.streams.kstream; + +import org.apache.kafka.streams.kstream.internals.TimeWindow; + +import java.util.HashMap; +import java.util.Map; + +/** + * The time-based window specifications used for aggregations. + */ +public class TimeWindows extends Windows { + + /** + * The size of the window, i.e. how long a window lasts. + * The window size's effective time unit is determined by the semantics of the topology's + * configured {@link org.apache.kafka.streams.processor.TimestampExtractor}. + */ + public final long size; + + /** + * The size of the window's advance interval, i.e. by how much a window moves forward relative + * to the previous one. The interval's effective time unit is determined by the semantics of + * the topology's configured {@link org.apache.kafka.streams.processor.TimestampExtractor}. + */ + public final long advance; + + private TimeWindows(String name, long size, long advance) { + super(name); + if (size <= 0) { + throw new IllegalArgumentException("window size must be > 0 (you provided " + size + ")"); + } + this.size = size; + if (!(0 < advance && advance <= size)) { + throw new IllegalArgumentException( + String.format("advance interval (%d) must lie within interval (0, %d]", advance, size)); + } + this.advance = advance; + } + + /** + * Returns a window definition with the given window size, and with the advance interval being + * equal to the window size. Think: [N * size, N * size + size), with N denoting the N-th + * window. + * + * This provides the semantics of tumbling windows, which are fixed-sized, gap-less, + * non-overlapping windows. Tumbling windows are a specialization of hopping windows. + * + * @param name The name of the window. Must not be null or empty. + * @param size The size of the window, with the requirement that size > 0. + * The window size's effective time unit is determined by the semantics of the + * topology's configured {@link org.apache.kafka.streams.processor.TimestampExtractor}. + * @return a new window definition + */ + public static TimeWindows of(String name, long size) { + return new TimeWindows(name, size, size); + } + + /** + * Returns a window definition with the original size, but advance ("hop") the window by the given + * interval, which specifies by how much a window moves forward relative to the previous one. + * Think: [N * advanceInterval, N * advanceInterval + size), with N denoting the N-th window. + * + * This provides the semantics of hopping windows, which are fixed-sized, overlapping windows. + * + * @param interval The advance interval ("hop") of the window, with the requirement that + * 0 < interval ≤ size. The interval's effective time unit is + * determined by the semantics of the topology's configured + * {@link org.apache.kafka.streams.processor.TimestampExtractor}. + * @return a new window definition + */ + public TimeWindows advanceBy(long interval) { + return new TimeWindows(this.name, this.size, interval); + } + + @Override + public Map windowsFor(long timestamp) { + long enclosed = (size - 1) / advance; + long windowStart = Math.max(0, timestamp - timestamp % advance - enclosed * advance); + + Map windows = new HashMap<>(); + while (windowStart <= timestamp) { + TimeWindow window = new TimeWindow(windowStart, windowStart + this.size); + windows.put(windowStart, window); + windowStart += this.advance; + } + return windows; + } + + @Override + public final boolean equals(Object o) { + if (o == this) { + return true; + } + if (!(o instanceof TimeWindows)) { + return false; + } + TimeWindows other = (TimeWindows) o; + return this.size == other.size && this.advance == other.advance; + } + + @Override + public int hashCode() { + int result = (int) (size ^ (size >>> 32)); + result = 31 * result + (int) (advance ^ (advance >>> 32)); + return result; + } + +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/TumblingWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/TumblingWindows.java deleted file mode 100644 index cadedba550ba9..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/TumblingWindows.java +++ /dev/null @@ -1,74 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streams.kstream; - -import org.apache.kafka.streams.kstream.internals.TumblingWindow; - -import java.util.HashMap; -import java.util.Map; - -/** - * The tumbling window specifications used for aggregations. - */ -public class TumblingWindows extends Windows { - - private static final long DEFAULT_SIZE_MS = 1000L; - - public final long size; - - private TumblingWindows(String name, long size) { - super(name); - - this.size = size; - } - - /** - * Returns a half-interval sliding window definition with the default window size - */ - public static TumblingWindows of(String name) { - return new TumblingWindows(name, DEFAULT_SIZE_MS); - } - - /** - * Returns a half-interval sliding window definition with the window size in milliseconds - */ - public TumblingWindows with(long size) { - return new TumblingWindows(this.name, size); - } - - @Override - public Map windowsFor(long timestamp) { - long windowStart = timestamp - timestamp % size; - - // we cannot use Collections.singleMap since it does not support remove() call - Map windows = new HashMap<>(); - windows.put(windowStart, new TumblingWindow(windowStart, windowStart + size)); - - return windows; - } - - @Override - public boolean equalTo(Windows other) { - if (!other.getClass().equals(TumblingWindows.class)) - return false; - - TumblingWindows otherWindows = (TumblingWindows) other; - - return this.size == otherWindows.size; - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java index 7cadfb4ff1d25..bea3b57b134e6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java @@ -34,6 +34,9 @@ public class UnlimitedWindows extends Windows { private UnlimitedWindows(String name, long start) { super(name); + if (start < 0) { + throw new IllegalArgumentException("start must be > 0 (you provided " + start + ")"); + } this.start = start; } @@ -52,21 +55,31 @@ public UnlimitedWindows startOn(long start) { public Map windowsFor(long timestamp) { // always return the single unlimited window - // we cannot use Collections.singleMap since it does not support remove() call + // we cannot use Collections.singleMap since it does not support remove() Map windows = new HashMap<>(); - windows.put(start, new UnlimitedWindow(start)); - - + if (timestamp >= start) { + windows.put(start, new UnlimitedWindow(start)); + } return windows; } @Override - public boolean equalTo(Windows other) { - if (!other.getClass().equals(UnlimitedWindows.class)) + public final boolean equals(Object o) { + if (o == this) { + return true; + } + + if (!(o instanceof UnlimitedWindows)) { return false; + } - UnlimitedWindows otherWindows = (UnlimitedWindows) other; + UnlimitedWindows other = (UnlimitedWindows) o; + return this.start == other.start; + } - return this.start == otherWindows.start; + @Override + public int hashCode() { + return (int) (start ^ (start >>> 32)); } -} + +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java index f2965dc07d372..784d5c309c659 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java @@ -48,21 +48,18 @@ public boolean overlap(Window other) { return this.start() < other.end() || other.start() < this.end(); } - public boolean equalsTo(Window other) { - return this.start() == other.start() && this.end() == other.end(); - } - @Override public boolean equals(Object obj) { - if (obj == this) + if (obj == this) { return true; + } - if (!(obj instanceof Window)) + if (getClass() != obj.getClass()) { return false; + } Window other = (Window) obj; - - return this.equalsTo(other) && this.start == other.start && this.end == other.end; + return this.start == other.start && this.end == other.end; } @Override @@ -70,4 +67,5 @@ public int hashCode() { long n = (this.start << 32) | this.end; return (int) (n % 0xFFFFFFFFL); } + } \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java index e7dc23ec9a739..1406de62e217d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java @@ -45,6 +45,9 @@ public abstract class Windows { public int segments; protected Windows(String name) { + if (name == null || name.isEmpty()) { + throw new IllegalArgumentException("name must not be null or empty"); + } this.name = name; this.segments = DEFAULT_NUM_SEGMENTS; this.emitDurationMs = DEFAULT_EMIT_DURATION; @@ -95,7 +98,6 @@ protected String newName(String prefix) { return prefix + String.format("%010d", NAME_INDEX.getAndIncrement()); } - public abstract boolean equalTo(Windows other); - public abstract Map windowsFor(long timestamp); + } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/HoppingWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindow.java similarity index 75% rename from streams/src/main/java/org/apache/kafka/streams/kstream/internals/HoppingWindow.java rename to streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindow.java index 8b0b2fbe26c4f..5dfb9eb601d85 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/HoppingWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TimeWindow.java @@ -19,19 +19,15 @@ import org.apache.kafka.streams.kstream.Window; -public class HoppingWindow extends Window { +public class TimeWindow extends Window { - public HoppingWindow(long start, long end) { + public TimeWindow(long start, long end) { super(start, end); } @Override public boolean overlap(Window other) { - return super.overlap(other) && other.getClass().equals(HoppingWindow.class); + return getClass() == other.getClass() && super.overlap(other); } - @Override - public boolean equalsTo(Window other) { - return super.equalsTo(other) && other.getClass().equals(HoppingWindow.class); - } -} +} \ No newline at end of file diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TumblingWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TumblingWindow.java deleted file mode 100644 index a02d4b90937fd..0000000000000 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/TumblingWindow.java +++ /dev/null @@ -1,38 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one or more - * contributor license agreements. See the NOTICE file distributed with - * this work for additional information regarding copyright ownership. - * The ASF licenses this file to You under the Apache License, Version 2.0 - * (the "License"); you may not use this file except in compliance with - * the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.kafka.streams.kstream.internals; - - -import org.apache.kafka.streams.kstream.Window; - -public class TumblingWindow extends Window { - - public TumblingWindow(long start, long end) { - super(start, end); - } - - @Override - public boolean overlap(Window other) { - return super.overlap(other) && other.getClass().equals(TumblingWindow.class); - } - - @Override - public boolean equalsTo(Window other) { - return super.equalsTo(other) && other.getClass().equals(TumblingWindow.class); - } -} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindow.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindow.java index 8ac8f70e4c13b..4b93f9b731d19 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindow.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/UnlimitedWindow.java @@ -27,11 +27,7 @@ public UnlimitedWindow(long start) { @Override public boolean overlap(Window other) { - return super.overlap(other) && other.getClass().equals(UnlimitedWindow.class); + return getClass() == other.getClass() && super.overlap(other); } - @Override - public boolean equalsTo(Window other) { - return super.equalsTo(other) && other.getClass().equals(UnlimitedWindow.class); - } -} +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java new file mode 100644 index 0000000000000..e9ff235d48c80 --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/TimeWindowsTest.java @@ -0,0 +1,123 @@ +/** + * 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 org.apache.kafka.streams.kstream; + +import org.apache.kafka.streams.kstream.internals.TimeWindow; +import org.junit.Test; + +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class TimeWindowsTest { + + private static String anyName = "window"; + private static long anySize = 123L; + + @Test + public void shouldHaveSaneEqualsAndHashCode() { + TimeWindows w1 = TimeWindows.of("w1", anySize); + TimeWindows w2 = TimeWindows.of("w2", w1.size); + + // Reflexive + assertTrue(w1.equals(w1)); + assertTrue(w1.hashCode() == w1.hashCode()); + + // Symmetric + assertTrue(w1.equals(w2)); + assertTrue(w1.hashCode() == w2.hashCode()); + assertTrue(w2.hashCode() == w1.hashCode()); + + // Transitive + TimeWindows w3 = TimeWindows.of("w3", w2.size); + assertTrue(w2.equals(w3)); + assertTrue(w2.hashCode() == w3.hashCode()); + assertTrue(w1.equals(w3)); + assertTrue(w1.hashCode() == w3.hashCode()); + + // Inequality scenarios + assertFalse("must be false for null", w1.equals(null)); + assertFalse("must be false for different window types", w1.equals(UnlimitedWindows.of("irrelevant"))); + assertFalse("must be false for different types", w1.equals(new Object())); + + TimeWindows differentWindowSize = TimeWindows.of("differentWindowSize", w1.size + 1); + assertFalse("must be false when window sizes are different", w1.equals(differentWindowSize)); + + TimeWindows differentAdvanceInterval = w1.advanceBy(w1.advance - 1); + assertFalse("must be false when advance intervals are different", w1.equals(differentAdvanceInterval)); + } + + @Test(expected = IllegalArgumentException.class) + public void nameMustNotBeEmpty() { + TimeWindows.of("", anySize); + } + + @Test(expected = IllegalArgumentException.class) + public void nameMustNotBeNull() { + TimeWindows.of(null, anySize); + } + + @Test(expected = IllegalArgumentException.class) + public void windowSizeMustNotBeNegative() { + TimeWindows.of(anyName, -1); + } + + @Test(expected = IllegalArgumentException.class) + public void windowSizeMustNotBeZero() { + TimeWindows.of(anyName, 0); + } + + @Test(expected = IllegalArgumentException.class) + public void advanceIntervalMustNotBeNegative() { + TimeWindows.of(anyName, anySize).advanceBy(-1); + } + + @Test(expected = IllegalArgumentException.class) + public void advanceIntervalMustNotBeZero() { + TimeWindows.of(anyName, anySize).advanceBy(0); + } + + @Test(expected = IllegalArgumentException.class) + public void advanceIntervalMustNotBeLargerThanWindowSize() { + long size = anySize; + TimeWindows.of(anyName, size).advanceBy(size + 1); + } + + @Test + public void windowsForHoppingWindows() { + TimeWindows windows = TimeWindows.of(anyName, 12L).advanceBy(5L); + Map matched = windows.windowsFor(21L); + assertEquals(12L / 5L + 1, matched.size()); + assertEquals(new TimeWindow(10L, 22L), matched.get(10L)); + assertEquals(new TimeWindow(15L, 27L), matched.get(15L)); + assertEquals(new TimeWindow(20L, 32L), matched.get(20L)); + } + + @Test + public void windowsForTumblingWindows() { + TimeWindows windows = TimeWindows.of(anyName, 12L); + Map matched = windows.windowsFor(21L); + assertEquals(1, matched.size()); + assertEquals(new TimeWindow(12L, 24L), matched.get(12L)); + } + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java new file mode 100644 index 0000000000000..da5f159df153b --- /dev/null +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/UnlimitedWindowsTest.java @@ -0,0 +1,80 @@ +/** + * 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 org.apache.kafka.streams.kstream; + +import org.apache.kafka.streams.kstream.internals.UnlimitedWindow; +import org.junit.Test; + +import java.util.Map; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class UnlimitedWindowsTest { + + private static String anyName = "window"; + private static long anyStartTime = 10L; + + @Test(expected = IllegalArgumentException.class) + public void nameMustNotBeEmpty() { + UnlimitedWindows.of(""); + } + + @Test(expected = IllegalArgumentException.class) + public void nameMustNotBeNull() { + UnlimitedWindows.of(null); + } + + @Test(expected = IllegalArgumentException.class) + public void startTimeMustNotBeNegative() { + UnlimitedWindows.of(anyName).startOn(-1); + } + + @Test + public void startTimeCanBeZero() { + UnlimitedWindows.of(anyName).startOn(0); + } + + @Test + public void shouldIncludeRecordsThatHappenedOnWindowStart() { + UnlimitedWindows w = UnlimitedWindows.of(anyName).startOn(anyStartTime); + Map matchedWindows = w.windowsFor(w.start); + assertEquals(1, matchedWindows.size()); + assertEquals(new UnlimitedWindow(anyStartTime), matchedWindows.get(anyStartTime)); + } + + @Test + public void shouldIncludeRecordsThatHappenedAfterWindowStart() { + UnlimitedWindows w = UnlimitedWindows.of(anyName).startOn(anyStartTime); + long timestamp = w.start + 1; + Map matchedWindows = w.windowsFor(timestamp); + assertEquals(1, matchedWindows.size()); + assertEquals(new UnlimitedWindow(anyStartTime), matchedWindows.get(anyStartTime)); + } + + @Test + public void shouldExcludeRecordsThatHappenedBeforeWindowStart() { + UnlimitedWindows w = UnlimitedWindows.of(anyName).startOn(anyStartTime); + long timestamp = w.start - 1; + Map matchedWindows = w.windowsFor(timestamp); + assertTrue(matchedWindows.isEmpty()); + } + +} \ No newline at end of file diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java index 828103ac3dd5c..f4fe3a6471682 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregateTest.java @@ -20,10 +20,10 @@ import org.apache.kafka.common.serialization.Serde; import org.apache.kafka.common.serialization.Serdes; import org.apache.kafka.common.utils.Utils; -import org.apache.kafka.streams.kstream.HoppingWindows; import org.apache.kafka.streams.kstream.KStream; import org.apache.kafka.streams.kstream.KStreamBuilder; import org.apache.kafka.streams.kstream.KTable; +import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.Windowed; import org.apache.kafka.test.KStreamTestDriver; @@ -37,6 +37,7 @@ import java.io.File; import java.io.IOException; +import java.nio.file.Files; import static org.junit.Assert.assertEquals; @@ -62,215 +63,229 @@ public void setUp() throws IOException { @Test public void testAggBasic() throws Exception { - final KStreamBuilder builder = new KStreamBuilder(); - - String topic1 = "topic1"; - - KStream stream1 = builder.stream(strSerde, strSerde, topic1); - KTable, String> table2 = stream1.aggregateByKey(MockInitializer.STRING_INIT, MockAggregator.STRING_ADDER, - HoppingWindows.of("topic1-Canonized").with(10L).every(5L), - strSerde, - strSerde); - - MockProcessorSupplier, String> proc2 = new MockProcessorSupplier<>(); - table2.toStream().process(proc2); - - driver = new KStreamTestDriver(builder, stateDir); - - driver.setTime(0L); - driver.process(topic1, "A", "1"); - driver.setTime(1L); - driver.process(topic1, "B", "2"); - driver.setTime(2L); - driver.process(topic1, "C", "3"); - driver.setTime(3L); - driver.process(topic1, "D", "4"); - driver.setTime(4L); - driver.process(topic1, "A", "1"); - - driver.setTime(5L); - driver.process(topic1, "A", "1"); - driver.setTime(6L); - driver.process(topic1, "B", "2"); - driver.setTime(7L); - driver.process(topic1, "D", "4"); - driver.setTime(8L); - driver.process(topic1, "B", "2"); - driver.setTime(9L); - driver.process(topic1, "C", "3"); - - driver.setTime(10L); - driver.process(topic1, "A", "1"); - driver.setTime(11L); - driver.process(topic1, "B", "2"); - driver.setTime(12L); - driver.process(topic1, "D", "4"); - driver.setTime(13L); - driver.process(topic1, "B", "2"); - driver.setTime(14L); - driver.process(topic1, "C", "3"); - - assertEquals(Utils.mkList( - "[A@0]:0+1", - "[B@0]:0+2", - "[C@0]:0+3", - "[D@0]:0+4", - "[A@0]:0+1+1", - - "[A@0]:0+1+1+1", "[A@5]:0+1", - "[B@0]:0+2+2", "[B@5]:0+2", - "[D@0]:0+4+4", "[D@5]:0+4", - "[B@0]:0+2+2+2", "[B@5]:0+2+2", - "[C@0]:0+3+3", "[C@5]:0+3", - - "[A@5]:0+1+1", "[A@10]:0+1", - "[B@5]:0+2+2+2", "[B@10]:0+2", - "[D@5]:0+4+4", "[D@10]:0+4", - "[B@5]:0+2+2+2+2", "[B@10]:0+2+2", - "[C@5]:0+3+3", "[C@10]:0+3"), proc2.processed); + final File baseDir = Files.createTempDirectory("test").toFile(); + + try { + final KStreamBuilder builder = new KStreamBuilder(); + String topic1 = "topic1"; + + KStream stream1 = builder.stream(strSerde, strSerde, topic1); + KTable, String> table2 = + stream1.aggregateByKey(MockInitializer.STRING_INIT, MockAggregator.STRING_ADDER, + TimeWindows.of("topic1-Canonized", 10).advanceBy(5), + strSerde, + strSerde); + + MockProcessorSupplier, String> proc2 = new MockProcessorSupplier<>(); + table2.toStream().process(proc2); + + KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir); + + driver.setTime(0L); + driver.process(topic1, "A", "1"); + driver.setTime(1L); + driver.process(topic1, "B", "2"); + driver.setTime(2L); + driver.process(topic1, "C", "3"); + driver.setTime(3L); + driver.process(topic1, "D", "4"); + driver.setTime(4L); + driver.process(topic1, "A", "1"); + + driver.setTime(5L); + driver.process(topic1, "A", "1"); + driver.setTime(6L); + driver.process(topic1, "B", "2"); + driver.setTime(7L); + driver.process(topic1, "D", "4"); + driver.setTime(8L); + driver.process(topic1, "B", "2"); + driver.setTime(9L); + driver.process(topic1, "C", "3"); + + driver.setTime(10L); + driver.process(topic1, "A", "1"); + driver.setTime(11L); + driver.process(topic1, "B", "2"); + driver.setTime(12L); + driver.process(topic1, "D", "4"); + driver.setTime(13L); + driver.process(topic1, "B", "2"); + driver.setTime(14L); + driver.process(topic1, "C", "3"); + + assertEquals(Utils.mkList( + "[A@0]:0+1", + "[B@0]:0+2", + "[C@0]:0+3", + "[D@0]:0+4", + "[A@0]:0+1+1", + + "[A@0]:0+1+1+1", "[A@5]:0+1", + "[B@0]:0+2+2", "[B@5]:0+2", + "[D@0]:0+4+4", "[D@5]:0+4", + "[B@0]:0+2+2+2", "[B@5]:0+2+2", + "[C@0]:0+3+3", "[C@5]:0+3", + + "[A@5]:0+1+1", "[A@10]:0+1", + "[B@5]:0+2+2+2", "[B@10]:0+2", + "[D@5]:0+4+4", "[D@10]:0+4", + "[B@5]:0+2+2+2+2", "[B@10]:0+2+2", + "[C@5]:0+3+3", "[C@10]:0+3"), proc2.processed); + + } finally { + Utils.delete(baseDir); + } } @Test public void testJoin() throws Exception { - final KStreamBuilder builder = new KStreamBuilder(); - - String topic1 = "topic1"; - String topic2 = "topic2"; - - KStream stream1 = builder.stream(strSerde, strSerde, topic1); - KTable, String> table1 = stream1.aggregateByKey(MockInitializer.STRING_INIT, MockAggregator.STRING_ADDER, - HoppingWindows.of("topic1-Canonized").with(10L).every(5L), - strSerde, - strSerde); - - MockProcessorSupplier, String> proc1 = new MockProcessorSupplier<>(); - table1.toStream().process(proc1); - - KStream stream2 = builder.stream(strSerde, strSerde, topic2); - KTable, String> table2 = stream2.aggregateByKey(MockInitializer.STRING_INIT, MockAggregator.STRING_ADDER, - HoppingWindows.of("topic2-Canonized").with(10L).every(5L), - strSerde, - strSerde); - - MockProcessorSupplier, String> proc2 = new MockProcessorSupplier<>(); - table2.toStream().process(proc2); - - - MockProcessorSupplier, String> proc3 = new MockProcessorSupplier<>(); - table1.join(table2, new ValueJoiner() { - @Override - public String apply(String p1, String p2) { - return p1 + "%" + p2; - } - }).toStream().process(proc3); - - driver = new KStreamTestDriver(builder, stateDir); - - driver.setTime(0L); - driver.process(topic1, "A", "1"); - driver.setTime(1L); - driver.process(topic1, "B", "2"); - driver.setTime(2L); - driver.process(topic1, "C", "3"); - driver.setTime(3L); - driver.process(topic1, "D", "4"); - driver.setTime(4L); - driver.process(topic1, "A", "1"); - - proc1.checkAndClearProcessResult( - "[A@0]:0+1", - "[B@0]:0+2", - "[C@0]:0+3", - "[D@0]:0+4", - "[A@0]:0+1+1" - ); - proc2.checkAndClearProcessResult(); - proc3.checkAndClearProcessResult( - "[A@0]:null", - "[B@0]:null", - "[C@0]:null", - "[D@0]:null", - "[A@0]:null" - ); - - driver.setTime(5L); - driver.process(topic1, "A", "1"); - driver.setTime(6L); - driver.process(topic1, "B", "2"); - driver.setTime(7L); - driver.process(topic1, "D", "4"); - driver.setTime(8L); - driver.process(topic1, "B", "2"); - driver.setTime(9L); - driver.process(topic1, "C", "3"); - - proc1.checkAndClearProcessResult( - "[A@0]:0+1+1+1", "[A@5]:0+1", - "[B@0]:0+2+2", "[B@5]:0+2", - "[D@0]:0+4+4", "[D@5]:0+4", - "[B@0]:0+2+2+2", "[B@5]:0+2+2", - "[C@0]:0+3+3", "[C@5]:0+3" - ); - proc2.checkAndClearProcessResult(); - proc3.checkAndClearProcessResult( - "[A@0]:null", "[A@5]:null", - "[B@0]:null", "[B@5]:null", - "[D@0]:null", "[D@5]:null", - "[B@0]:null", "[B@5]:null", - "[C@0]:null", "[C@5]:null" - ); - - driver.setTime(0L); - driver.process(topic2, "A", "a"); - driver.setTime(1L); - driver.process(topic2, "B", "b"); - driver.setTime(2L); - driver.process(topic2, "C", "c"); - driver.setTime(3L); - driver.process(topic2, "D", "d"); - driver.setTime(4L); - driver.process(topic2, "A", "a"); - - proc1.checkAndClearProcessResult(); - proc2.checkAndClearProcessResult( - "[A@0]:0+a", - "[B@0]:0+b", - "[C@0]:0+c", - "[D@0]:0+d", - "[A@0]:0+a+a" - ); - proc3.checkAndClearProcessResult( - "[A@0]:0+1+1+1%0+a", - "[B@0]:0+2+2+2%0+b", - "[C@0]:0+3+3%0+c", - "[D@0]:0+4+4%0+d", - "[A@0]:0+1+1+1%0+a+a"); - - driver.setTime(5L); - driver.process(topic2, "A", "a"); - driver.setTime(6L); - driver.process(topic2, "B", "b"); - driver.setTime(7L); - driver.process(topic2, "D", "d"); - driver.setTime(8L); - driver.process(topic2, "B", "b"); - driver.setTime(9L); - driver.process(topic2, "C", "c"); - - proc1.checkAndClearProcessResult(); - proc2.checkAndClearProcessResult( - "[A@0]:0+a+a+a", "[A@5]:0+a", - "[B@0]:0+b+b", "[B@5]:0+b", - "[D@0]:0+d+d", "[D@5]:0+d", - "[B@0]:0+b+b+b", "[B@5]:0+b+b", - "[C@0]:0+c+c", "[C@5]:0+c" - ); - proc3.checkAndClearProcessResult( - "[A@0]:0+1+1+1%0+a+a+a", "[A@5]:0+1%0+a", - "[B@0]:0+2+2+2%0+b+b", "[B@5]:0+2+2%0+b", - "[D@0]:0+4+4%0+d+d", "[D@5]:0+4%0+d", - "[B@0]:0+2+2+2%0+b+b+b", "[B@5]:0+2+2%0+b+b", - "[C@0]:0+3+3%0+c+c", "[C@5]:0+3%0+c" - ); + final File baseDir = Files.createTempDirectory("test").toFile(); + + try { + final KStreamBuilder builder = new KStreamBuilder(); + String topic1 = "topic1"; + String topic2 = "topic2"; + + KStream stream1 = builder.stream(strSerde, strSerde, topic1); + KTable, String> table1 = + stream1.aggregateByKey(MockInitializer.STRING_INIT, MockAggregator.STRING_ADDER, + TimeWindows.of("topic1-Canonized", 10).advanceBy(5), + strSerde, + strSerde); + + MockProcessorSupplier, String> proc1 = new MockProcessorSupplier<>(); + table1.toStream().process(proc1); + + KStream stream2 = builder.stream(strSerde, strSerde, topic2); + KTable, String> table2 = + stream2.aggregateByKey(MockInitializer.STRING_INIT, MockAggregator.STRING_ADDER, + TimeWindows.of("topic2-Canonized", 10).advanceBy(5), + strSerde, + strSerde); + + MockProcessorSupplier, String> proc2 = new MockProcessorSupplier<>(); + table2.toStream().process(proc2); + + + MockProcessorSupplier, String> proc3 = new MockProcessorSupplier<>(); + table1.join(table2, new ValueJoiner() { + @Override + public String apply(String p1, String p2) { + return p1 + "%" + p2; + } + }).toStream().process(proc3); + + KStreamTestDriver driver = new KStreamTestDriver(builder, baseDir); + + driver.setTime(0L); + driver.process(topic1, "A", "1"); + driver.setTime(1L); + driver.process(topic1, "B", "2"); + driver.setTime(2L); + driver.process(topic1, "C", "3"); + driver.setTime(3L); + driver.process(topic1, "D", "4"); + driver.setTime(4L); + driver.process(topic1, "A", "1"); + + proc1.checkAndClearProcessResult( + "[A@0]:0+1", + "[B@0]:0+2", + "[C@0]:0+3", + "[D@0]:0+4", + "[A@0]:0+1+1" + ); + proc2.checkAndClearProcessResult(); + proc3.checkAndClearProcessResult( + "[A@0]:null", + "[B@0]:null", + "[C@0]:null", + "[D@0]:null", + "[A@0]:null" + ); + + driver.setTime(5L); + driver.process(topic1, "A", "1"); + driver.setTime(6L); + driver.process(topic1, "B", "2"); + driver.setTime(7L); + driver.process(topic1, "D", "4"); + driver.setTime(8L); + driver.process(topic1, "B", "2"); + driver.setTime(9L); + driver.process(topic1, "C", "3"); + + proc1.checkAndClearProcessResult( + "[A@0]:0+1+1+1", "[A@5]:0+1", + "[B@0]:0+2+2", "[B@5]:0+2", + "[D@0]:0+4+4", "[D@5]:0+4", + "[B@0]:0+2+2+2", "[B@5]:0+2+2", + "[C@0]:0+3+3", "[C@5]:0+3" + ); + proc2.checkAndClearProcessResult(); + proc3.checkAndClearProcessResult( + "[A@0]:null", "[A@5]:null", + "[B@0]:null", "[B@5]:null", + "[D@0]:null", "[D@5]:null", + "[B@0]:null", "[B@5]:null", + "[C@0]:null", "[C@5]:null" + ); + + driver.setTime(0L); + driver.process(topic2, "A", "a"); + driver.setTime(1L); + driver.process(topic2, "B", "b"); + driver.setTime(2L); + driver.process(topic2, "C", "c"); + driver.setTime(3L); + driver.process(topic2, "D", "d"); + driver.setTime(4L); + driver.process(topic2, "A", "a"); + + proc1.checkAndClearProcessResult(); + proc2.checkAndClearProcessResult( + "[A@0]:0+a", + "[B@0]:0+b", + "[C@0]:0+c", + "[D@0]:0+d", + "[A@0]:0+a+a" + ); + proc3.checkAndClearProcessResult( + "[A@0]:0+1+1+1%0+a", + "[B@0]:0+2+2+2%0+b", + "[C@0]:0+3+3%0+c", + "[D@0]:0+4+4%0+d", + "[A@0]:0+1+1+1%0+a+a"); + + driver.setTime(5L); + driver.process(topic2, "A", "a"); + driver.setTime(6L); + driver.process(topic2, "B", "b"); + driver.setTime(7L); + driver.process(topic2, "D", "d"); + driver.setTime(8L); + driver.process(topic2, "B", "b"); + driver.setTime(9L); + driver.process(topic2, "C", "c"); + + proc1.checkAndClearProcessResult(); + proc2.checkAndClearProcessResult( + "[A@0]:0+a+a+a", "[A@5]:0+a", + "[B@0]:0+b+b", "[B@5]:0+b", + "[D@0]:0+d+d", "[D@5]:0+d", + "[B@0]:0+b+b+b", "[B@5]:0+b+b", + "[C@0]:0+c+c", "[C@5]:0+c" + ); + proc3.checkAndClearProcessResult( + "[A@0]:0+1+1+1%0+a+a+a", "[A@5]:0+1%0+a", + "[B@0]:0+2+2+2%0+b+b", "[B@5]:0+2+2%0+b", + "[D@0]:0+4+4%0+d+d", "[D@5]:0+4%0+d", + "[B@0]:0+2+2+2%0+b+b+b", "[B@5]:0+2+2%0+b+b", + "[C@0]:0+3+3%0+c+c", "[C@5]:0+3%0+c" + ); + } finally { + Utils.delete(baseDir); + } } } diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java index 7c6d5ec0fd2a2..b31b20d622223 100644 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitionerTest.java @@ -49,7 +49,7 @@ public class WindowedStreamPartitionerTest { new PartitionInfo(topicName, 5, Node.noNode(), new Node[0], new Node[0]) ); - private Cluster cluster = new Cluster(Arrays.asList(Node.noNode()), infos, Collections.emptySet()); + private Cluster cluster = new Cluster(Collections.singletonList(Node.noNode()), infos, Collections.emptySet()); @Test public void testCopartitioning() { @@ -71,7 +71,7 @@ public void testCopartitioning() { Integer expected = defaultPartitioner.partition("topic", key, keyBytes, value, valueBytes, cluster); for (int w = 0; w < 10; w++) { - HoppingWindow window = new HoppingWindow(10 * w, 20 * w); + TimeWindow window = new TimeWindow(10 * w, 20 * w); Windowed windowedKey = new Windowed<>(key, window); Integer actual = streamPartitioner.partition(windowedKey, value, infos.size()); diff --git a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowsTest.java b/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowsTest.java deleted file mode 100644 index f9b6ba5656548..0000000000000 --- a/streams/src/test/java/org/apache/kafka/streams/kstream/internals/WindowsTest.java +++ /dev/null @@ -1,70 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - *

          - * http://www.apache.org/licenses/LICENSE-2.0 - *

          - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.kafka.streams.kstream.internals; - -import org.apache.kafka.streams.kstream.HoppingWindows; -import org.apache.kafka.streams.kstream.TumblingWindows; -import org.apache.kafka.streams.kstream.UnlimitedWindows; -import org.junit.Test; - -import java.util.Map; - -import static org.junit.Assert.assertEquals; - -public class WindowsTest { - - @Test - public void hoppingWindows() { - - HoppingWindows windows = HoppingWindows.of("test").with(12L).every(5L); - - Map matched = windows.windowsFor(21L); - - assertEquals(3, matched.size()); - - assertEquals(new HoppingWindow(10L, 22L), matched.get(10L)); - assertEquals(new HoppingWindow(15L, 27L), matched.get(15L)); - assertEquals(new HoppingWindow(20L, 32L), matched.get(20L)); - } - - @Test - public void tumblineWindows() { - - TumblingWindows windows = TumblingWindows.of("test").with(12L); - - Map matched = windows.windowsFor(21L); - - assertEquals(1, matched.size()); - - assertEquals(new TumblingWindow(12L, 24L), matched.get(12L)); - } - - @Test - public void unlimitedWindows() { - - UnlimitedWindows windows = UnlimitedWindows.of("test").startOn(10L); - - Map matched = windows.windowsFor(21L); - - assertEquals(1, matched.size()); - - assertEquals(new UnlimitedWindow(10L), matched.get(10L)); - } -} diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java index 95e0fbfe3173a..733c1ea33e6e5 100644 --- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java +++ b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java @@ -28,7 +28,7 @@ import org.apache.kafka.streams.kstream.KTable; import org.apache.kafka.streams.kstream.KeyValueMapper; import org.apache.kafka.streams.kstream.Predicate; -import org.apache.kafka.streams.kstream.TumblingWindows; +import org.apache.kafka.streams.kstream.TimeWindows; import org.apache.kafka.streams.kstream.UnlimitedWindows; import org.apache.kafka.streams.kstream.ValueJoiner; import org.apache.kafka.streams.kstream.Windowed; @@ -207,7 +207,7 @@ public Double apply(Long value1, Long value2) { // windowed count data.countByKey( - TumblingWindows.of("tumbling-win-cnt").with(WINDOW_SIZE), + TimeWindows.of("tumbling-win-cnt", WINDOW_SIZE), stringSerde ).toStream().map( new KeyValueMapper, Long, KeyValue>() { From 4ab4e4af814fb791fe6e8c2bd3381da8ca80b0b5 Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Fri, 29 Apr 2016 08:49:16 -0700 Subject: [PATCH 193/206] KAFKA-3598: Improve JavaDoc of public API Author: Matthias J. Sax Reviewers: Michael G. Noll, Guozhang Wang Closes #1250 from mjsax/JavaDoc-publicAPI --- .../examples/pageview/PageViewTypedDemo.java | 4 +- .../pageview/PageViewUntypedDemo.java | 2 +- .../apache/kafka/streams/KafkaStreams.java | 10 ++- .../org/apache/kafka/streams/KeyValue.java | 17 ++++ .../kafka/streams/kstream/Aggregator.java | 10 ++- .../kafka/streams/kstream/ForeachAction.java | 10 ++- .../kafka/streams/kstream/Initializer.java | 7 +- .../kafka/streams/kstream/JoinWindows.java | 21 ++--- .../apache/kafka/streams/kstream/KStream.java | 10 +-- .../kafka/streams/kstream/KStreamBuilder.java | 15 ++-- .../apache/kafka/streams/kstream/KTable.java | 19 ++--- .../kafka/streams/kstream/KeyValueMapper.java | 9 ++- .../kafka/streams/kstream/Predicate.java | 9 ++- .../apache/kafka/streams/kstream/Reducer.java | 9 ++- .../kafka/streams/kstream/TimeWindows.java | 18 +++-- .../kafka/streams/kstream/Transformer.java | 8 +- .../streams/kstream/TransformerSupplier.java | 7 +- .../streams/kstream/UnlimitedWindows.java | 9 ++- .../kafka/streams/kstream/ValueJoiner.java | 9 ++- .../kafka/streams/kstream/ValueMapper.java | 8 +- .../streams/kstream/ValueTransformer.java | 8 +- .../kstream/ValueTransformerSupplier.java | 7 +- .../apache/kafka/streams/kstream/Window.java | 18 ++++- .../kafka/streams/kstream/Windowed.java | 30 ++++--- .../apache/kafka/streams/kstream/Windows.java | 37 ++++----- .../internals/KStreamWindowAggregate.java | 2 +- .../internals/KStreamWindowReduce.java | 2 +- .../kstream/internals/WindowedSerializer.java | 4 +- .../internals/WindowedStreamPartitioner.java | 6 +- .../ConsumerRecordTimestampExtractor.java | 3 +- .../kafka/streams/processor/Processor.java | 9 ++- .../streams/processor/ProcessorSupplier.java | 5 ++ .../streams/processor/StateStoreSupplier.java | 10 +++ .../streams/processor/StreamPartitioner.java | 20 ++--- .../kafka/streams/processor/TaskId.java | 6 +- .../streams/processor/TimestampExtractor.java | 8 +- .../streams/processor/TopologyBuilder.java | 64 +++++++-------- .../WallclockTimestampExtractor.java | 9 ++- .../processor/internals/StreamThread.java | 2 +- .../kafka/streams/state/StateSerdes.java | 79 +++++++++++++++++-- .../kafka/streams/state/WindowStore.java | 2 +- .../streams/smoketest/SmokeTestClient.java | 2 +- .../streams/smoketest/SmokeTestUtil.java | 2 +- 43 files changed, 383 insertions(+), 163 deletions(-) diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java index 39ec41fab9c82..e53b037478436 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewTypedDemo.java @@ -168,10 +168,10 @@ public KeyValue apply(String user, PageViewByRegion vi public KeyValue apply(Windowed key, Long value) { WindowedPageViewByRegion wViewByRegion = new WindowedPageViewByRegion(); wViewByRegion.windowStart = key.window().start(); - wViewByRegion.region = key.value(); + wViewByRegion.region = key.key(); RegionCount rCount = new RegionCount(); - rCount.region = key.value(); + rCount.region = key.key(); rCount.count = value; return new KeyValue<>(wViewByRegion, rCount); diff --git a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java index 9a41b9ea4d83e..8a0af6c66f1c4 100644 --- a/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java +++ b/streams/examples/src/main/java/org/apache/kafka/streams/examples/pageview/PageViewUntypedDemo.java @@ -107,7 +107,7 @@ public KeyValue apply(String user, JsonNode viewRegion) { public KeyValue apply(Windowed key, Long value) { ObjectNode keyNode = JsonNodeFactory.instance.objectNode(); keyNode.put("window-start", key.window().start()) - .put("region", key.value()); + .put("region", key.key()); ObjectNode valueNode = JsonNodeFactory.instance.objectNode(); valueNode.put("count", value); diff --git a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java index 4d1306d228b74..45024f2240100 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java +++ b/streams/src/main/java/org/apache/kafka/streams/KafkaStreams.java @@ -97,6 +97,12 @@ public class KafkaStreams { // usage only and should not be exposed to users at all. private final UUID processId; + /** + * Construct the stream instance. + * + * @param builder the processor topology builder specifying the computational logic + * @param props properties for the {@link StreamsConfig} + */ public KafkaStreams(TopologyBuilder builder, Properties props) { this(builder, new StreamsConfig(props)); } @@ -104,8 +110,8 @@ public KafkaStreams(TopologyBuilder builder, Properties props) { /** * Construct the stream instance. * - * @param builder The processor topology builder specifying the computational logic - * @param config The stream configs + * @param builder the processor topology builder specifying the computational logic + * @param config the stream configs */ public KafkaStreams(TopologyBuilder builder, StreamsConfig config) { // create the metrics diff --git a/streams/src/main/java/org/apache/kafka/streams/KeyValue.java b/streams/src/main/java/org/apache/kafka/streams/KeyValue.java index ca86fc402469b..58f2083b8457b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KeyValue.java +++ b/streams/src/main/java/org/apache/kafka/streams/KeyValue.java @@ -29,14 +29,31 @@ */ public class KeyValue { + /** The key of the key-value pair. */ public final K key; + /** The value of the key-value pair. */ public final V value; + /** + * Create a new key-value pair. + * + * @param key the key + * @param value the value + */ public KeyValue(K key, V value) { this.key = key; this.value = value; } + /** + * Create a new key-value pair. + * + * @param key the key + * @param value the value + * @param the type of the key + * @param the type of the value + * @return a new key value pair + */ public static KeyValue pair(K key, V value) { return new KeyValue<>(key, value); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java index 9ec9f966442a9..989d89f1a2a5e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Aggregator.java @@ -18,7 +18,7 @@ package org.apache.kafka.streams.kstream; /** - * The Aggregator interface for aggregating values of the given key. + * The {@link Aggregator} interface for aggregating values of the given key. * * @param key type * @param original value type @@ -26,5 +26,13 @@ */ public interface Aggregator { + /** + * Compute a new aggregate from the key and value of a record and the current aggregate of the same key. + * + * @param aggKey the key of the record + * @param value the value of the record + * @param aggregate the current aggregate value + * @return the new aggregate value + */ T apply(K aggKey, V value, T aggregate); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java index 83064e8034efd..b3e3169792d40 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ForeachAction.java @@ -18,9 +18,8 @@ package org.apache.kafka.streams.kstream; - /** - * The ForeachAction interface for performing an action on a key-value pair. + * The {@link ForeachAction} interface for performing an action on a key-value pair. * Note that this action is stateless. If stateful processing is required, consider * using {@link KStream#transform(TransformerSupplier, String...)} or * {@link KStream#process(ProcessorSupplier, String...)} instead. @@ -29,6 +28,13 @@ * @param original value type */ public interface ForeachAction { + + /** + * Perform an action for each record of a stream. + * + * @param key the key of the record + * @param value the value of the record + */ void apply(K key, V value); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Initializer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Initializer.java index 67c1c21638408..39bc40dbaddca 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Initializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Initializer.java @@ -18,11 +18,16 @@ package org.apache.kafka.streams.kstream; /** - * The Initializer interface for creating an initial value in aggregations. + * The {@link Initializer} interface for creating an initial value in aggregations. * * @param aggregate value type */ public interface Initializer { + /** + * Return the initial value for an aggregation. + * + * @return the initial value for an aggregation + */ T apply(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java index a74984a29b130..a6d5603f6dd3a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java @@ -26,7 +26,9 @@ */ public class JoinWindows extends Windows { + /** Maximum time difference for tuples that are before the join tuple. */ public final long before; + /** Maximum time difference for tuples that are after the join tuple. */ public final long after; private JoinWindows(String name, long before, long after) { @@ -41,40 +43,41 @@ public static JoinWindows of(String name) { } /** - * Specifies that records of the same key are joinable if their timestamp stamps are within - * timeDifference. + * Specifies that records of the same key are joinable if their timestamps are within {@code timeDifference}. * - * @param timeDifference join window interval in milliseconds + * @param timeDifference join window interval */ public JoinWindows within(long timeDifference) { return new JoinWindows(this.name, timeDifference, timeDifference); } /** - * Specifies that records of the same key are joinable if their timestamp stamps are within + * Specifies that records of the same key are joinable if their timestamps are within * the join window interval, and if the timestamp of a record from the secondary stream is * earlier than or equal to the timestamp of a record from the first stream. * - * @param timeDifference join window interval in milliseconds + * @param timeDifference join window interval */ public JoinWindows before(long timeDifference) { return new JoinWindows(this.name, timeDifference, this.after); } /** - * Specifies that records of the same key are joinable if their timestamp stamps are within + * Specifies that records of the same key are joinable if their timestamps are within * the join window interval, and if the timestamp of a record from the secondary stream * is later than or equal to the timestamp of a record from the first stream. * - * @param timeDifference join window interval in milliseconds + * @param timeDifference join window interval */ public JoinWindows after(long timeDifference) { return new JoinWindows(this.name, this.before, timeDifference); } + /** + * Not supported by {@link JoinWindows}. Throws {@link UnsupportedOperationException}. + */ @Override public Map windowsFor(long timestamp) { - // this function should never be called throw new UnsupportedOperationException("windowsFor() is not supported in JoinWindows"); } @@ -98,4 +101,4 @@ public int hashCode() { return result; } -} \ No newline at end of file +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index 7e3562c558d34..6df2deb6e71af 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -24,7 +24,7 @@ import org.apache.kafka.streams.processor.StreamPartitioner; /** - * KStream is an abstraction of a record stream of key-value pairs. + * {@link KStream} is an abstraction of a record stream of key-value pairs. * * @param Type of keys * @param Type of values @@ -510,7 +510,7 @@ KTable aggregateByKey(Initializer initializer, String name); /** - * Count number of messages of this stream by key on a window basis into a new instance of windowed {@link KTable}. + * Count number of records of this stream by key on a window basis into a new instance of windowed {@link KTable}. * * @param windows the specification of the aggregation {@link Windows} * @param keySerde key serdes for materializing the counting table, @@ -519,7 +519,7 @@ KTable aggregateByKey(Initializer initializer, KTable, Long> countByKey(Windows windows, Serde keySerde); /** - * Count number of messages of this stream by key on a window basis into a new instance of windowed {@link KTable} + * Count number of records of this stream by key on a window basis into a new instance of windowed {@link KTable} * with default serializers and deserializers. * * @param windows the specification of the aggregation {@link Windows} @@ -527,7 +527,7 @@ KTable aggregateByKey(Initializer initializer, KTable, Long> countByKey(Windows windows); /** - * Count number of messages of this stream by key into a new instance of ever-updating {@link KTable}. + * Count number of records of this stream by key into a new instance of ever-updating {@link KTable}. * * @param keySerde key serdes for materializing the counting table, * if not specified the default serdes defined in the configs will be used @@ -536,7 +536,7 @@ KTable aggregateByKey(Initializer initializer, KTable countByKey(Serde keySerde, String name); /** - * Count number of messages of this stream by key into a new instance of ever-updating {@link KTable} + * Count number of records of this stream by key into a new instance of ever-updating {@link KTable} * with default serializers and deserializers. * * @param name the name of the resulted {@link KTable} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java index 6b770b4006e4f..159876c4ab91e 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java @@ -28,19 +28,22 @@ import java.util.concurrent.atomic.AtomicInteger; /** - * KStreamBuilder is a subclass of {@link TopologyBuilder} that provides the Kafka Streams DSL + * {@link KStreamBuilder} is a subclass of {@link TopologyBuilder} that provides the Kafka Streams DSL * for users to specify computational logic and translates the given logic to a {@link org.apache.kafka.streams.processor.internals.ProcessorTopology}. */ public class KStreamBuilder extends TopologyBuilder { private final AtomicInteger index = new AtomicInteger(0); + /** + * Create a new {@link KStreamBuilder} instance. + */ public KStreamBuilder() { super(); } /** - * Creates a {@link KStream} instance from the specified topics. + * Create a {@link KStream} instance from the specified topics. * The default deserializers specified in the config are used. * * @param topics the topic names; must contain at least one topic name @@ -50,7 +53,7 @@ public KStream stream(String... topics) { } /** - * Creates a {@link KStream} instance for the specified topics. + * Create a {@link KStream} instance for the specified topics. * * @param keySerde key serde used to read this source {@link KStream}, * if not specified the default serde defined in the configs will be used @@ -67,7 +70,7 @@ public KStream stream(Serde keySerde, Serde valSerde, String. } /** - * Creates a {@link KTable} instance for the specified topic. + * Create a {@link KTable} instance for the specified topic. * The default deserializers specified in the config are used. * * @param topic the topic name; cannot be null @@ -77,7 +80,7 @@ public KTable table(String topic) { } /** - * Creates a {@link KTable} instance for the specified topic. + * Create a {@link KTable} instance for the specified topic. * * @param keySerde key serde used to send key-value pairs, * if not specified the default key serde defined in the configuration will be used @@ -98,7 +101,7 @@ public KTable table(Serde keySerde, Serde valSerde, String to } /** - * Creates a new instance of {@link KStream} by merging the given streams + * Create a new instance of {@link KStream} by merging the given streams. * * @param streams the instances of {@link KStream} to be merged */ diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index 841427907b02c..4ff9b481a9e70 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -23,7 +23,7 @@ import org.apache.kafka.streams.processor.StreamPartitioner; /** - * KTable is an abstraction of a changelog stream from a primary-keyed table. + * {@link KTable} is an abstraction of a changelog stream from a primary-keyed table. * * @param Type of primary keys * @param Type of value changes @@ -39,7 +39,7 @@ public interface KTable { KTable filter(Predicate predicate); /** - * Create a new instance of {@link KTable} that consists all elements of this stream which do not satisfy a predicate + * Create a new instance of {@link KTable} that consists all elements of this stream which do not satisfy a predicate. * * @param predicate the instance of {@link Predicate} */ @@ -55,7 +55,7 @@ public interface KTable { /** - * Print the elements of this stream to System.out + * Print the elements of this stream to {@code System.out} * * Implementors will need to override toString for keys and values that are not of * type String, Integer etc to get meaningful information. @@ -63,7 +63,7 @@ public interface KTable { void print(); /** - * Print the elements of this stream to System.out + * Print the elements of this stream to {@code System.out} * @param keySerde key serde used to send key-value pairs, * if not specified the default serde defined in the configs will be used * @param valSerde value serde used to send key-value pairs, @@ -75,15 +75,16 @@ public interface KTable { void print(Serde keySerde, Serde valSerde); /** - * Write the elements of this stream to a file at the given path. + * Write the elements of this stream to a file at the given path using default serializers and deserializers. * @param filePath name of file to write to * - * Implementors will need to override toString for keys and values that are not of - * type String, Integer etc to get meaningful information. + * Implementors will need to override {@code toString} for keys and values that are not of + * type {@link String}, {@link Integer} etc. to get meaningful information. */ void writeAsText(String filePath); /** + * Write the elements of this stream to a file at the given path. * * @param filePath name of file to write to * @param keySerde key serde used to send key-value pairs, @@ -91,8 +92,8 @@ public interface KTable { * @param valSerde value serde used to send key-value pairs, * if not specified the default serde defined in the configs will be used * - * Implementors will need to override toString for keys and values that are not of - * type String, Integer etc to get meaningful information. + * Implementors will need to override {@code toString} for keys and values that are not of + * type {@link String}, {@link Integer} etc. to get meaningful information. */ void writeAsText(String filePath, Serde keySerde, Serde valSerde); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java index a4aed91f92c4c..b36ed63586a54 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KeyValueMapper.java @@ -18,7 +18,7 @@ package org.apache.kafka.streams.kstream; /** - * The KeyValueMapper interface for mapping a key-value pair to a new value (could be another key-value pair). + * The {@link KeyValueMapper} interface for mapping a key-value pair to a new value (could be another key-value pair). * * @param original key type * @param original value type @@ -26,5 +26,12 @@ */ public interface KeyValueMapper { + /** + * Map a record with the given key and value to a new value. + * + * @param key the key of the record + * @param value the value of the record + * @return the new value + */ R apply(K key, V value); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java index c90554b192cc6..2df2d5fb7d92a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Predicate.java @@ -18,12 +18,19 @@ package org.apache.kafka.streams.kstream; /** - * The Predicate interface represents a predicate (boolean-valued function) of a key-value pair. + * The {@link Predicate} interface represents a predicate (boolean-valued function) of a key-value pair. * * @param key type * @param value type */ public interface Predicate { + /** + * Test if the record with the given key and value satisfies the predicate. + * + * @param key the key of the record + * @param value the value of the record + * @return return {@code true} if the key-value pair satisfies the predicate—{@code false} otherwise + */ boolean test(K key, V value); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Reducer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Reducer.java index 551a6720e66c4..e7cfa0e5fa082 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Reducer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Reducer.java @@ -18,11 +18,18 @@ package org.apache.kafka.streams.kstream; /** - * The Reducer interface for combining two values of the same type into a new value. + * The {@link Reducer} interface for combining two values of the same type into a new value. * * @param value type */ public interface Reducer { + /** + * Aggregate the two given values into a single one. + * + * @param value1 the first value for the aggregation + * @param value2 the second value for the aggregation + * @return the aggregated value + */ V apply(V value1, V value2); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java index fa3a9d822aece..e4ce88332ef3d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/TimeWindows.java @@ -24,6 +24,14 @@ /** * The time-based window specifications used for aggregations. + *

          + * The semantics of a time-based window are: Every T1 (advance) time-units, compute the aggregate total for T2 (size) time-units. + *

            + *
          • If {@code advance < size} a hopping windows is defined:
            + * it discretize a stream into overlapping windows, which implies that a record maybe contained in one and or more "adjacent" windows.
          • + *
          • If {@code advance == size} a tumbling window is defined:
            + * it discretize a stream into non-overlapping windows, which implies that a record is only ever contained in one and only one tumbling window.
          • + *
          */ public class TimeWindows extends Windows { @@ -36,7 +44,7 @@ public class TimeWindows extends Windows { /** * The size of the window's advance interval, i.e. by how much a window moves forward relative - * to the previous one. The interval's effective time unit is determined by the semantics of + * to the previous one. The interval's effective time unit is determined by the semantics of * the topology's configured {@link org.apache.kafka.streams.processor.TimestampExtractor}. */ public final long advance; @@ -56,13 +64,13 @@ private TimeWindows(String name, long size, long advance) { /** * Returns a window definition with the given window size, and with the advance interval being - * equal to the window size. Think: [N * size, N * size + size), with N denoting the N-th + * equal to the window size. Think: [N * size, N * size + size), with N denoting the N-th * window. * * This provides the semantics of tumbling windows, which are fixed-sized, gap-less, - * non-overlapping windows. Tumbling windows are a specialization of hopping windows. + * non-overlapping windows. Tumbling windows are a specialization of hopping windows. * - * @param name The name of the window. Must not be null or empty. + * @param name The name of the window. Must not be null or empty. * @param size The size of the window, with the requirement that size > 0. * The window size's effective time unit is determined by the semantics of the * topology's configured {@link org.apache.kafka.streams.processor.TimestampExtractor}. @@ -80,7 +88,7 @@ public static TimeWindows of(String name, long size) { * This provides the semantics of hopping windows, which are fixed-sized, overlapping windows. * * @param interval The advance interval ("hop") of the window, with the requirement that - * 0 < interval ≤ size. The interval's effective time unit is + * 0 < interval ≤ size. The interval's effective time unit is * determined by the semantics of the topology's configured * {@link org.apache.kafka.streams.processor.TimestampExtractor}. * @return a new window definition diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java index 5197e94769f8f..239854beecf6b 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Transformer.java @@ -20,7 +20,7 @@ import org.apache.kafka.streams.processor.ProcessorContext; /** - * A stateful Transformer interface for transform a key-value pair into a new value. + * A stateful {@link Transformer} interface for transform a key-value pair into a new value. * * @param key type * @param value type @@ -40,10 +40,10 @@ public interface Transformer { void init(ProcessorContext context); /** - * Transform the message with the given key and value. + * Transform the record with the given key and value. * - * @param key the key for the message - * @param value the value for the message + * @param key the key for the record + * @param value the value for the record * @return new value; if null no key-value pair will be forwarded to down stream */ R transform(K key, V value); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java index fc7ba60d08ef1..0341702dafe7d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/TransformerSupplier.java @@ -18,9 +18,14 @@ package org.apache.kafka.streams.kstream; /** - * A transformer supplier which can create one or more {@link Transformer} instances. + * A {@link TransformerSupplier} interface which can create one or more {@link Transformer} instances. */ public interface TransformerSupplier { + /** + * Return a new {@link Transformer} instance. + * + * @return a new {@link Transformer} instance + */ Transformer get(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java index bea3b57b134e6..f45f8c5845532 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/UnlimitedWindows.java @@ -29,6 +29,7 @@ public class UnlimitedWindows extends Windows { private static final long DEFAULT_START_TIMESTAMP = 0L; + /** The start timestamp of the window. */ public final long start; private UnlimitedWindows(String name, long start) { @@ -41,12 +42,18 @@ private UnlimitedWindows(String name, long start) { } /** - * Returns an unlimited window definition + * Return an unlimited window starting at timestamp zero. */ public static UnlimitedWindows of(String name) { return new UnlimitedWindows(name, DEFAULT_START_TIMESTAMP); } + /** + * Return a new unlimited window for the specified start timestamp. + * + * @param start the window start time + * @return a new unlimited window that starts at {@code start} + */ public UnlimitedWindows startOn(long start) { return new UnlimitedWindows(this.name, start); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java index 5f00a1a7ff203..8d4a8e760d340 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueJoiner.java @@ -18,7 +18,7 @@ package org.apache.kafka.streams.kstream; /** - * The ValueJoiner interface for joining two values and return a the joined new value. + * The {@link ValueJoiner} interface for joining two values into a new value. * * @param first value type * @param second value type @@ -26,5 +26,12 @@ */ public interface ValueJoiner { + /** + * Return a joined value consisting of {@code value1} and {@code value2}. + * + * @param value1 the first value for joining + * @param value2 the second value for joining + * @return the joined value + */ R apply(V1 value1, V2 value2); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java index 6e62a55774f40..e168e37f4ef2a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueMapper.java @@ -18,12 +18,18 @@ package org.apache.kafka.streams.kstream; /** - * The KeyValueMapper interface for mapping an original value to a new value (could be another key-value pair). + * The {@link ValueMapper} interface for mapping an original value to a new value (could be another key-value pair). * * @param original value type * @param mapped value type */ public interface ValueMapper { + /** + * Map the given value to a new value. + * + * @param value the value to be mapped + * @return the new value + */ V2 apply(V1 value); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java index 63214fde2238b..f92d9a155d0ef 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformer.java @@ -20,7 +20,7 @@ import org.apache.kafka.streams.processor.ProcessorContext; /** - * A stateful Value Transformer interface for transform a value into a new value. + * A stateful {@link ValueTransformer} interface to transform a value into a new value. * * @param value type * @param return type @@ -31,7 +31,7 @@ public interface ValueTransformer { * Initialize this transformer with the given context. The framework ensures this is called once per processor when the topology * that contains it is initialized. *

          - * If this tranformer is to be {@link #punctuate(long) called periodically} by the framework, then this method should + * If this transformer is to be {@link #punctuate(long) called periodically} by the framework, then this method should * {@link ProcessorContext#schedule(long) schedule itself} with the provided context. * * @param context the context; may not be null @@ -39,9 +39,9 @@ public interface ValueTransformer { void init(ProcessorContext context); /** - * Transform the message with the given key and value. + * Transform the record with the given key and value. * - * @param value the value for the message + * @param value the value for the record * @return new value */ R transform(V value); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerSupplier.java b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerSupplier.java index 6bc86bc17443a..ecd454abac15d 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/ValueTransformerSupplier.java @@ -18,9 +18,14 @@ package org.apache.kafka.streams.kstream; /** - * A value transformer supplier which can create one or more {@link ValueTransformer} instances. + * A {@link ValueTransformerSupplier} interface which can create one or more {@link ValueTransformer} instances. */ public interface ValueTransformerSupplier { + /** + * Return a new {@link ValueTransformer} instance. + * + * @return a new {@link ValueTransformer} instance. + */ ValueTransformer get(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java index 784d5c309c659..e1ea9a0f2a19f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Window.java @@ -25,25 +25,37 @@ public abstract class Window { private long start; private long end; + /** + * Create a new window for the given start time (inclusive) and end time (exclusive). + * + * @param start the start timestamp of the window (inclusive) + * @param end the end timestamp of the window (exclusive) + */ public Window(long start, long end) { this.start = start; this.end = end; } /** - * Returns the start timestamp of this window, inclusive + * Return the start timestamp of this window, inclusive */ public long start() { return start; } /** - * Returns the end timestamp of this window, exclusive + * Return the end timestamp of this window, exclusive */ public long end() { return end; } + /** + * Check if the given window overlaps with this window. + * + * @param other another window + * @return {@code true} if {@code other} overlaps with this window—{@code false} otherwise + */ public boolean overlap(Window other) { return this.start() < other.end() || other.start() < this.end(); } @@ -68,4 +80,4 @@ public int hashCode() { return (int) (n % 0xFFFFFFFFL); } -} \ No newline at end of file +} diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java index 369128277733b..feaf6a34305a0 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Windowed.java @@ -22,30 +22,40 @@ * i.e. {@link KStream#aggregateByKey(Initializer, Aggregator, Windows, org.apache.kafka.common.serialization.Serde, * org.apache.kafka.common.serialization.Serde)} * - * @param Type of the key + * @param Type of the key */ -public class Windowed { +public class Windowed { - private T value; + private K key; private Window window; - public Windowed(T value, Window window) { - this.value = value; + public Windowed(K key, Window window) { + this.key = key; this.window = window; } - public T value() { - return value; + /** + * Return the key of the window. + * + * @return the key of the window + */ + public K key() { + return key; } + /** + * Return the window containing the values associated with this key. + * + * @return the window containing the values + */ public Window window() { return window; } @Override public String toString() { - return "[" + value + "@" + window.start() + "]"; + return "[" + key + "@" + window.start() + "]"; } @Override @@ -58,12 +68,12 @@ public boolean equals(Object obj) { Windowed that = (Windowed) obj; - return this.window.equals(that.window) && this.value.equals(that.value); + return this.window.equals(that.window) && this.key.equals(that.key); } @Override public int hashCode() { - long n = ((long) window.hashCode() << 32) | value.hashCode(); + long n = ((long) window.hashCode() << 32) | key.hashCode(); return (int) (n % 0xFFFFFFFFL); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java index 1406de62e217d..06cacb4fa1288 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/Windows.java @@ -30,16 +30,12 @@ public abstract class Windows { private static final int DEFAULT_NUM_SEGMENTS = 3; - private static final long DEFAULT_EMIT_DURATION = 1000L; - private static final long DEFAULT_MAINTAIN_DURATION = 24 * 60 * 60 * 1000L; // one day private static final AtomicInteger NAME_INDEX = new AtomicInteger(0); protected String name; - private long emitDurationMs; - private long maintainDurationMs; public int segments; @@ -50,7 +46,6 @@ protected Windows(String name) { } this.name = name; this.segments = DEFAULT_NUM_SEGMENTS; - this.emitDurationMs = DEFAULT_EMIT_DURATION; this.maintainDurationMs = DEFAULT_MAINTAIN_DURATION; } @@ -58,17 +53,10 @@ public String name() { return name; } - /** - * Set the window emit duration in milliseconds of system time. - */ - public Windows emit(long durationMs) { - this.emitDurationMs = durationMs; - - return this; - } - /** * Set the window maintain duration in milliseconds of system time. + * + * @return itself */ public Windows until(long durationMs) { this.maintainDurationMs = durationMs; @@ -79,6 +67,8 @@ public Windows until(long durationMs) { /** * Specify the number of segments to be used for rolling the window store, * this function is not exposed to users but can be called by developers that extend this JoinWindows specs. + * + * @return itself */ protected Windows segments(int segments) { this.segments = segments; @@ -86,18 +76,21 @@ protected Windows segments(int segments) { return this; } - public long emitEveryMs() { - return this.emitDurationMs; - } - + /** + * Return the window maintain duration in milliseconds of system time. + * + * @return the window maintain duration in milliseconds of system time + */ public long maintainMs() { return this.maintainDurationMs; } - protected String newName(String prefix) { - return prefix + String.format("%010d", NAME_INDEX.getAndIncrement()); - } - + /** + * Creates all windows that contain the provided timestamp. + * + * @param timestamp the timestamp window should get created for + * @return a map of {@code windowStartTimestamp -> Window} entries + */ public abstract Map windowsFor(long timestamp); } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java index f36cc8c8558fd..b4272f89a827a 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowAggregate.java @@ -163,7 +163,7 @@ public void init(ProcessorContext context) { @SuppressWarnings("unchecked") @Override public T get(Windowed windowedKey) { - K key = windowedKey.value(); + K key = windowedKey.key(); W window = (W) windowedKey.window(); // this iterator should contain at most one element diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduce.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduce.java index 6c05ce30008eb..3ed1499f658a8 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduce.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/KStreamWindowReduce.java @@ -157,7 +157,7 @@ public void init(ProcessorContext context) { @SuppressWarnings("unchecked") @Override public V get(Windowed windowedKey) { - K key = windowedKey.value(); + K key = windowedKey.key(); W window = (W) windowedKey.window(); // this iterator should only contain one element diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedSerializer.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedSerializer.java index 0afcad131d11b..2e19816b1fb04 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedSerializer.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedSerializer.java @@ -40,7 +40,7 @@ public void configure(Map configs, boolean isKey) { @Override public byte[] serialize(String topic, Windowed data) { - byte[] serializedKey = inner.serialize(topic, data.value()); + byte[] serializedKey = inner.serialize(topic, data.key()); ByteBuffer buf = ByteBuffer.allocate(serializedKey.length + TIMESTAMP_SIZE); buf.put(serializedKey); @@ -55,7 +55,7 @@ public void close() { } public byte[] serializeBaseKey(String topic, Windowed data) { - return inner.serialize(topic, data.value()); + return inner.serialize(topic, data.key()); } } diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitioner.java b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitioner.java index 10e69cc776fbb..1e3086447e509 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitioner.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/internals/WindowedStreamPartitioner.java @@ -29,12 +29,12 @@ public WindowedStreamPartitioner(WindowedSerializer serializer) { } /** - * WindowedStreamPartitioner determines the partition number for a message with the given windowed key and value + * WindowedStreamPartitioner determines the partition number for a record with the given windowed key and value * and the current number of partitions. The partition number id determined by the original key of the windowed key * using the same logic as DefaultPartitioner so that the topic is partitioned by the original key. * - * @param windowedKey the key of the message - * @param value the value of the message + * @param windowedKey the key of the record + * @param value the value of the record * @param numPartitions the total number of partitions * @return an integer between 0 and {@code numPartitions-1}, or {@code null} if the default partitioning logic should be used */ diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ConsumerRecordTimestampExtractor.java b/streams/src/main/java/org/apache/kafka/streams/processor/ConsumerRecordTimestampExtractor.java index 61b1c9817a211..0d3424e6ff239 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ConsumerRecordTimestampExtractor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ConsumerRecordTimestampExtractor.java @@ -27,7 +27,8 @@ * via this timestamp extractor. * * If CreateTime is used to define the built-in timestamps, using this extractor effectively provide - * event-time semantics. + * event-time semantics. If LogAppendTime is used to define the built-in timestamps, using + * this extractor effectively provides ingestion-time semantics. * * If you need processing-time semantics, use {@link WallclockTimestampExtractor}. */ diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java index fbd72f01b05fe..92fcf1257bf4f 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/Processor.java @@ -37,10 +37,10 @@ public interface Processor { void init(ProcessorContext context); /** - * Process the message with the given key and value. + * Process the record with the given key and value. * - * @param key the key for the message - * @param value the value for the message + * @param key the key for the record + * @param value the value for the record */ void process(K key, V value); @@ -53,7 +53,8 @@ public interface Processor { void punctuate(long timestamp); /** - * Close this processor and clean up any resources. + * Close this processor and clean up any resources. Be aware that {@link #close()} is called after an internal cleanup. + * Thus, it is not possible to write anything to Kafka as underlying clients are already closed. */ void close(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorSupplier.java b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorSupplier.java index 65618999b1e31..7976e169f77f5 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/ProcessorSupplier.java @@ -29,5 +29,10 @@ */ public interface ProcessorSupplier { + /** + * Return a new {@link Processor} instance. + * + * @return a new {@link Processor} instance + */ Processor get(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/StateStoreSupplier.java b/streams/src/main/java/org/apache/kafka/streams/processor/StateStoreSupplier.java index 993500d015352..f2ae020de15ad 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/StateStoreSupplier.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/StateStoreSupplier.java @@ -22,7 +22,17 @@ */ public interface StateStoreSupplier { + /** + * Return the name of this state store supplier. + * + * @return the name of this state store supplier + */ String name(); + /** + * Return a new {@link StateStore} instance. + * + * @return a new {@link StateStore} instance + */ StateStore get(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/StreamPartitioner.java b/streams/src/main/java/org/apache/kafka/streams/processor/StreamPartitioner.java index f14d9d943f516..fbb037849bc0c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/StreamPartitioner.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/StreamPartitioner.java @@ -17,21 +17,21 @@ package org.apache.kafka.streams.processor; /** - * Determine how messages are distributed among the partitions in a Kafka topic. If not specified, the underlying producer's + * Determine how records are distributed among the partitions in a Kafka topic. If not specified, the underlying producer's * {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} will be used to determine the partition. *

          * Kafka topics are divided into one or more partitions. Since each partition must fit on the servers that host it, so * using multiple partitions allows the topic to scale beyond a size that will fit on a single machine. Partitions also enable you - * to use multiple instances of your topology to process in parallel all of the messages on the topology's source topics. + * to use multiple instances of your topology to process in parallel all of the records on the topology's source topics. *

          * When a topology is instantiated, each of its sources are assigned a subset of that topic's partitions. That means that only - * those processors in that topology instance will consume the messages from those partitions. In many cases, Kafka Streams will + * those processors in that topology instance will consume the records from those partitions. In many cases, Kafka Streams will * automatically manage these instances, and adjust when new topology instances are added or removed. *

          - * Some topologies, though, need more control over which messages appear in each partition. For example, some topologies that have - * stateful processors may want all messages within a range of keys to always be delivered to and handled by the same topology instance. - * An upstream topology producing messages to that topic can use a custom stream partitioner to precisely and consistently - * determine to which partition each message should be written. + * Some topologies, though, need more control over which records appear in each partition. For example, some topologies that have + * stateful processors may want all records within a range of keys to always be delivered to and handled by the same topology instance. + * An upstream topology producing records to that topic can use a custom stream partitioner to precisely and consistently + * determine to which partition each record should be written. *

          * To do this, create a StreamPartitioner implementation, and when you build your topology specify that custom partitioner * when {@link TopologyBuilder#addSink(String, String, org.apache.kafka.common.serialization.Serializer, org.apache.kafka.common.serialization.Serializer, StreamPartitioner, String...) adding a sink} @@ -48,10 +48,10 @@ public interface StreamPartitioner { /** - * Determine the partition number for a message with the given key and value and the current number of partitions. + * Determine the partition number for a record with the given key and value and the current number of partitions. * - * @param key the key of the message - * @param value the value of the message + * @param key the key of the record + * @param value the value of the record * @param numPartitions the total number of partitions * @return an integer between 0 and {@code numPartitions-1}, or {@code null} if the default partitioning logic should be used */ diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java b/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java index fa7c73c112ad5..7fc00d1f5fff1 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TaskId.java @@ -25,11 +25,13 @@ import java.nio.ByteBuffer; /** - * The task id representation composed as topic group id plus the assigned partition id. + * The task ID representation composed as topic group ID plus the assigned partition ID. */ public class TaskId implements Comparable { + /** The ID of the topic group. */ public final int topicGroupId; + /** The ID of the partition. */ public final int partition; public TaskId(int topicGroupId, int partition) { @@ -42,7 +44,7 @@ public String toString() { } /** - * @throws TaskIdFormatException if the string is not a valid TaskId + * @throws TaskIdFormatException if the string is not a valid {@link TaskId} */ public static TaskId parse(String string) { int index = string.indexOf('_'); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java b/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java index 224d58009368b..c872fa15ccaa6 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TimestampExtractor.java @@ -26,10 +26,12 @@ public interface TimestampExtractor { /** - * Extracts a timestamp from a message + * Extracts a timestamp from a record. + *

          + * Typically, the timestamp represents the milliseconds since midnight, January 1, 1970 UTC. * - * @param record ConsumerRecord - * @return timestamp + * @param record a data record + * @return the timestamp of the record */ long extract(ConsumerRecord record); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java index 487d5fe8684f6..5425149528390 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/TopologyBuilder.java @@ -43,11 +43,11 @@ /** * A component that is used to build a {@link ProcessorTopology}. A topology contains an acyclic graph of sources, processors, * and sinks. A {@link SourceNode source} is a node in the graph that consumes one or more Kafka topics and forwards them to - * its child nodes. A {@link Processor processor} is a node in the graph that receives input messages from upstream nodes, - * processes that message, and optionally forwarding new messages to one or all of its children. Finally, a {@link SinkNode sink} - * is a node in the graph that receives messages from upstream nodes and writes them to a Kafka topic. This builder allows you + * its child nodes. A {@link Processor processor} is a node in the graph that receives input records from upstream nodes, + * processes that records, and optionally forwarding new records to one or all of its children. Finally, a {@link SinkNode sink} + * is a node in the graph that receives records from upstream nodes and writes them to a Kafka topic. This builder allows you * to construct an acyclic graph of these nodes, and the builder is then passed into a new {@link org.apache.kafka.streams.KafkaStreams} - * instance that will then {@link org.apache.kafka.streams.KafkaStreams#start() begin consuming, processing, and producing messages}. + * instance that will then {@link org.apache.kafka.streams.KafkaStreams#start() begin consuming, processing, and producing records}. */ public class TopologyBuilder { @@ -193,7 +193,7 @@ public int hashCode() { public TopologyBuilder() {} /** - * Add a new source that consumes the named topics and forwards the messages to child processor and/or sink nodes. + * Add a new source that consumes the named topics and forwards the records to child processor and/or sink nodes. * The source will use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key deserializer} and * {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value deserializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration}. @@ -208,15 +208,15 @@ public final TopologyBuilder addSource(String name, String... topics) { } /** - * Add a new source that consumes the named topics and forwards the messages to child processor and/or sink nodes. + * Add a new source that consumes the named topics and forwards the records to child processor and/or sink nodes. * The source will use the specified key and value deserializers. * * @param name the unique name of the source used to reference this node when * {@link #addProcessor(String, ProcessorSupplier, String...) adding processor children}. - * @param keyDeserializer the {@link Deserializer key deserializer} used when consuming messages; may be null if the source + * @param keyDeserializer the {@link Deserializer key deserializer} used when consuming records; may be null if the source * should use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key deserializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration} - * @param valDeserializer the {@link Deserializer value deserializer} used when consuming messages; may be null if the source + * @param valDeserializer the {@link Deserializer value deserializer} used when consuming records; may be null if the source * should use the {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value deserializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration} * @param topics the name of one or more Kafka topics that this source is to consume @@ -242,14 +242,14 @@ public final TopologyBuilder addSource(String name, Deserializer keyDeserializer } /** - * Add a new sink that forwards messages from upstream parent processor and/or source nodes to the named Kafka topic. + * Add a new sink that forwards records from upstream parent processor and/or source nodes to the named Kafka topic. * The sink will use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key serializer} and * {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value serializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration}. * * @param name the unique name of the sink - * @param topic the name of the Kafka topic to which this sink should write its messages - * @param parentNames the name of one or more source or processor nodes whose output message this sink should consume + * @param topic the name of the Kafka topic to which this sink should write its records + * @param parentNames the name of one or more source or processor nodes whose output records this sink should consume * and write to its topic * @return this builder instance so methods can be chained together; never null * @see #addSink(String, String, StreamPartitioner, String...) @@ -261,22 +261,22 @@ public final TopologyBuilder addSink(String name, String topic, String... parent } /** - * Add a new sink that forwards messages from upstream parent processor and/or source nodes to the named Kafka topic, using + * Add a new sink that forwards records from upstream parent processor and/or source nodes to the named Kafka topic, using * the supplied partitioner. * The sink will use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key serializer} and * {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value serializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration}. *

          - * The sink will also use the specified {@link StreamPartitioner} to determine how messages are distributed among + * The sink will also use the specified {@link StreamPartitioner} to determine how records are distributed among * the named Kafka topic's partitions. Such control is often useful with topologies that use * {@link #addStateStore(StateStoreSupplier, String...) state stores} - * in its processors. In most other cases, however, a partitioner need not be specified and Kafka will automatically distribute - * messages among partitions using Kafka's default partitioning logic. + * in its processors. In most other cases, however, a partitioner needs not be specified and Kafka will automatically distribute + * records among partitions using Kafka's default partitioning logic. * * @param name the unique name of the sink - * @param topic the name of the Kafka topic to which this sink should write its messages - * @param partitioner the function that should be used to determine the partition for each message processed by the sink - * @param parentNames the name of one or more source or processor nodes whose output message this sink should consume + * @param topic the name of the Kafka topic to which this sink should write its records + * @param partitioner the function that should be used to determine the partition for each record processed by the sink + * @param parentNames the name of one or more source or processor nodes whose output records this sink should consume * and write to its topic * @return this builder instance so methods can be chained together; never null * @see #addSink(String, String, String...) @@ -288,18 +288,18 @@ public final TopologyBuilder addSink(String name, String topic, StreamPartitione } /** - * Add a new sink that forwards messages from upstream parent processor and/or source nodes to the named Kafka topic. + * Add a new sink that forwards records from upstream parent processor and/or source nodes to the named Kafka topic. * The sink will use the specified key and value serializers. * * @param name the unique name of the sink - * @param topic the name of the Kafka topic to which this sink should write its messages - * @param keySerializer the {@link Serializer key serializer} used when consuming messages; may be null if the sink + * @param topic the name of the Kafka topic to which this sink should write its records + * @param keySerializer the {@link Serializer key serializer} used when consuming records; may be null if the sink * should use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key serializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration} - * @param valSerializer the {@link Serializer value serializer} used when consuming messages; may be null if the sink + * @param valSerializer the {@link Serializer value serializer} used when consuming records; may be null if the sink * should use the {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value serializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration} - * @param parentNames the name of one or more source or processor nodes whose output message this sink should consume + * @param parentNames the name of one or more source or processor nodes whose output records this sink should consume * and write to its topic * @return this builder instance so methods can be chained together; never null * @see #addSink(String, String, String...) @@ -311,19 +311,19 @@ public final TopologyBuilder addSink(String name, String topic, Serializer keySe } /** - * Add a new sink that forwards messages from upstream parent processor and/or source nodes to the named Kafka topic. + * Add a new sink that forwards records from upstream parent processor and/or source nodes to the named Kafka topic. * The sink will use the specified key and value serializers, and the supplied partitioner. * * @param name the unique name of the sink - * @param topic the name of the Kafka topic to which this sink should write its messages - * @param keySerializer the {@link Serializer key serializer} used when consuming messages; may be null if the sink + * @param topic the name of the Kafka topic to which this sink should write its records + * @param keySerializer the {@link Serializer key serializer} used when consuming records; may be null if the sink * should use the {@link org.apache.kafka.streams.StreamsConfig#KEY_SERDE_CLASS_CONFIG default key serializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration} - * @param valSerializer the {@link Serializer value serializer} used when consuming messages; may be null if the sink + * @param valSerializer the {@link Serializer value serializer} used when consuming records; may be null if the sink * should use the {@link org.apache.kafka.streams.StreamsConfig#VALUE_SERDE_CLASS_CONFIG default value serializer} specified in the * {@link org.apache.kafka.streams.StreamsConfig stream configuration} - * @param partitioner the function that should be used to determine the partition for each message processed by the sink - * @param parentNames the name of one or more source or processor nodes whose output message this sink should consume + * @param partitioner the function that should be used to determine the partition for each record processed by the sink + * @param parentNames the name of one or more source or processor nodes whose output records this sink should consume * and write to its topic * @return this builder instance so methods can be chained together; never null * @see #addSink(String, String, String...) @@ -354,11 +354,11 @@ public final TopologyBuilder addSink(String name, String topic, Serialize } /** - * Add a new processor node that receives and processes messages output by one or more parent source or processor node. - * Any new messages output by this processor will be forwarded to its child processor or sink nodes. + * Add a new processor node that receives and processes records output by one or more parent source or processor node. + * Any new record output by this processor will be forwarded to its child processor or sink nodes. * @param name the unique name of the processor node * @param supplier the supplier used to obtain this node's {@link Processor} instance - * @param parentNames the name of one or more source or processor nodes whose output messages this processor should receive + * @param parentNames the name of one or more source or processor nodes whose output records this processor should receive * and process * @return this builder instance so methods can be chained together; never null * @throws TopologyBuilderException if parent processor is not added yet, or if this processor's name is equal to the parent's name diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/WallclockTimestampExtractor.java b/streams/src/main/java/org/apache/kafka/streams/processor/WallclockTimestampExtractor.java index 81821cefd4a20..305573b1746bd 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/WallclockTimestampExtractor.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/WallclockTimestampExtractor.java @@ -25,9 +25,16 @@ * Using this extractor effectively provides processing-time semantics. * * If you need event-time semantics, use {@link ConsumerRecordTimestampExtractor} with - * built-in CreateTime timestamp (see KIP-32: Add timestamps to Kafka message for details). + * built-in CreateTime or LogAppendTime timestamp (see KIP-32: Add timestamps to Kafka message for details). */ public class WallclockTimestampExtractor implements TimestampExtractor { + + /** + * Return the current wall clock time as timestamp. + * + * @param record a data record + * @return the current wall clock time, expressed in milliseconds since midnight, January 1, 1970 UTC + */ @Override public long extract(ConsumerRecord record) { return System.currentTimeMillis(); diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java index eff90e88cfb01..d4cb78c30fcf2 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/internals/StreamThread.java @@ -286,7 +286,7 @@ private void shutdown() { removeStandbyTasks(); // We need to first close the underlying clients before closing the state - // manager, for example we need to make sure producer's message sends + // manager, for example we need to make sure producer's record sends // have all been acked before the state manager records // changelog sent offsets try { diff --git a/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java b/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java index 933bf72b394ce..b19510c0b0276 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/StateSerdes.java @@ -24,13 +24,23 @@ /** * Factory for creating serializers / deserializers for state stores in Kafka Streams. * - * @param key type of serdes - * @param value type of serdes + * @param key type of serde + * @param value type of serde */ public final class StateSerdes { - public static StateSerdes withBuiltinTypes(String topic, Class keyClass, Class valueClass) { - return new StateSerdes<>(topic, Serdes.serdeFrom(keyClass), Serdes.serdeFrom(valueClass)); + /** + * Create a new instance of {@link StateSerdes} for the given state name and key-/value-type classes. + * + * @param stateName the name of the state + * @param keyClass the class of the key type + * @param valueClass the class of the value type + * @param the key type + * @param the value type + * @return a new instance of {@link StateSerdes} + */ + public static StateSerdes withBuiltinTypes(String stateName, Class keyClass, Class valueClass) { + return new StateSerdes<>(stateName, Serdes.serdeFrom(keyClass), Serdes.serdeFrom(valueClass)); } private final String stateName; @@ -63,46 +73,105 @@ public StateSerdes(String stateName, this.valueSerde = valueSerde; } + /** + * Return the key serde. + * + * @return the key serde + */ public Serde keySerde() { return keySerde; } + /** + * Return the value serde. + * + * @return the value serde + */ public Serde valueSerde() { return valueSerde; } + /** + * Return the key deserializer. + * + * @return the key deserializer + */ public Deserializer keyDeserializer() { return keySerde.deserializer(); } + /** + * Return the key serializer. + * + * @return the key serializer + */ public Serializer keySerializer() { return keySerde.serializer(); } + /** + * Return the value deserializer. + * + * @return the value deserializer + */ public Deserializer valueDeserializer() { return valueSerde.deserializer(); } + /** + * Return the value serializer. + * + * @return the value serializer + */ public Serializer valueSerializer() { return valueSerde.serializer(); } - public String topic() { + /** + * Return the name of the state. + * + * @return the name of the state + */ + public String stateName() { return stateName; } + /** + * Deserialize the key from raw bytes. + * + * @param rawKey the key as raw bytes + * @return the key as typed object + */ public K keyFrom(byte[] rawKey) { return keySerde.deserializer().deserialize(stateName, rawKey); } + /** + * Deserialize the value from raw bytes. + * + * @param rawValue the value as raw bytes + * @return the value as typed object + */ public V valueFrom(byte[] rawValue) { return valueSerde.deserializer().deserialize(stateName, rawValue); } + /** + * Serialize the given key. + * + * @param key the key to be serialized + * @return the serialized key + */ public byte[] rawKey(K key) { return keySerde.serializer().serialize(stateName, key); } + /** + * Serialize the given value. + * + * @param value the value to be serialized + * @return the serialized value + */ public byte[] rawValue(V value) { return valueSerde.serializer().serialize(stateName, value); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java index c7a882fc3ac0e..e400cefa53494 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java @@ -23,7 +23,7 @@ import org.apache.kafka.streams.processor.StateStore; /** - * A windowed store interface extending {@link StateStore} + * A windowed store interface extending {@link StateStore}. * * @param Type of keys * @param Type of values diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java index 733c1ea33e6e5..fbe7754d5b8a6 100644 --- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java +++ b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestClient.java @@ -213,7 +213,7 @@ public Double apply(Long value1, Long value2) { new KeyValueMapper, Long, KeyValue>() { @Override public KeyValue apply(Windowed key, Long value) { - return new KeyValue<>(key.value() + "@" + key.window().start(), value); + return new KeyValue<>(key.key() + "@" + key.window().start(), value); } } ).to(stringSerde, longSerde, "wcnt"); diff --git a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestUtil.java b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestUtil.java index c5ded5eb53b2d..b0d7a0b8013c1 100644 --- a/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestUtil.java +++ b/streams/src/test/java/org/apache/kafka/streams/smoketest/SmokeTestUtil.java @@ -77,7 +77,7 @@ public void close() { public static final class Unwindow implements KeyValueMapper, V, KeyValue> { public KeyValue apply(Windowed winKey, V value) { - return new KeyValue(winKey.value(), value); + return new KeyValue(winKey.key(), value); } } From cea01af125a33b81f973a96501fe41ca9d698197 Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Fri, 29 Apr 2016 09:41:12 -0700 Subject: [PATCH 194/206] KAFKA-2693: Ducktape tests for SASL/PLAIN and multiple mechanisms Run a sanity test with SASL/PLAIN and a couple of replication tests with SASL/PLAIN and multiple mechanisms. Author: Rajini Sivaram Reviewers: Ismael Juma , Ewen Cheslack-Postava Closes #1282 from rajinisivaram/KAFKA-2693 --- .../sanity_checks/test_console_consumer.py | 5 +- tests/kafkatest/services/kafka/kafka.py | 14 +-- .../services/kafka/templates/kafka.properties | 5 +- .../services/security/security_config.py | 27 ++++-- .../templates/{gssapi_jaas.conf => jaas.conf} | 93 ++++++++++--------- .../kafkatest/tests/core/replication_test.py | 7 +- 6 files changed, 92 insertions(+), 59 deletions(-) rename tests/kafkatest/services/security/templates/{gssapi_jaas.conf => jaas.conf} (61%) diff --git a/tests/kafkatest/sanity_checks/test_console_consumer.py b/tests/kafkatest/sanity_checks/test_console_consumer.py index 139c74acfa4e2..d6a152a10bcec 100644 --- a/tests/kafkatest/sanity_checks/test_console_consumer.py +++ b/tests/kafkatest/sanity_checks/test_console_consumer.py @@ -45,11 +45,14 @@ def setUp(self): self.zk.start() @parametrize(security_protocol='PLAINTEXT', new_consumer=False) + @parametrize(security_protocol='SASL_SSL', sasl_mechanism='PLAIN') @matrix(security_protocol=['PLAINTEXT', 'SSL', 'SASL_PLAINTEXT', 'SASL_SSL']) - def test_lifecycle(self, security_protocol, new_consumer=True): + def test_lifecycle(self, security_protocol, new_consumer=True, sasl_mechanism='GSSAPI'): """Check that console consumer starts/stops properly, and that we are capturing log output.""" self.kafka.security_protocol = security_protocol + self.kafka.client_sasl_mechanism = sasl_mechanism + self.kafka.interbroker_sasl_mechanism = sasl_mechanism self.kafka.start() self.consumer.security_protocol = security_protocol diff --git a/tests/kafkatest/services/kafka/kafka.py b/tests/kafkatest/services/kafka/kafka.py index 33ece3502f8d5..a74bb004c811a 100644 --- a/tests/kafkatest/services/kafka/kafka.py +++ b/tests/kafkatest/services/kafka/kafka.py @@ -63,7 +63,8 @@ class KafkaService(JmxMixin, Service): } def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAINTEXT, interbroker_security_protocol=SecurityConfig.PLAINTEXT, - sasl_mechanism=SecurityConfig.SASL_MECHANISM_GSSAPI, authorizer_class_name=None, topics=None, version=TRUNK, quota_config=None, jmx_object_names=None, + client_sasl_mechanism=SecurityConfig.SASL_MECHANISM_GSSAPI, interbroker_sasl_mechanism=SecurityConfig.SASL_MECHANISM_GSSAPI, + authorizer_class_name=None, topics=None, version=TRUNK, quota_config=None, jmx_object_names=None, jmx_attributes=[], zk_connect_timeout=5000): """ :type context @@ -78,7 +79,8 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI self.security_protocol = security_protocol self.interbroker_security_protocol = interbroker_security_protocol - self.sasl_mechanism = sasl_mechanism + self.client_sasl_mechanism = client_sasl_mechanism + self.interbroker_sasl_mechanism = interbroker_sasl_mechanism self.topics = topics self.minikdc = None self.authorizer_class_name = authorizer_class_name @@ -108,7 +110,9 @@ def __init__(self, context, num_nodes, zk, security_protocol=SecurityConfig.PLAI @property def security_config(self): - return SecurityConfig(self.security_protocol, self.interbroker_security_protocol, zk_sasl = self.zk.zk_sasl , sasl_mechanism=self.sasl_mechanism) + return SecurityConfig(self.security_protocol, self.interbroker_security_protocol, + zk_sasl = self.zk.zk_sasl, + client_sasl_mechanism=self.client_sasl_mechanism, interbroker_sasl_mechanism=self.interbroker_sasl_mechanism) def open_port(self, protocol): self.port_mappings[protocol] = self.port_mappings[protocol]._replace(open=True) @@ -163,9 +167,7 @@ def prop_file(self, node): # TODO - clean up duplicate configuration logic prop_file = cfg.render() prop_file += self.render('kafka.properties', node=node, broker_id=self.idx(node), - security_config=self.security_config, - interbroker_security_protocol=self.interbroker_security_protocol, - sasl_mechanism=self.sasl_mechanism) + security_config=self.security_config) return prop_file def start_cmd(self, node): diff --git a/tests/kafkatest/services/kafka/templates/kafka.properties b/tests/kafkatest/services/kafka/templates/kafka.properties index a718ee2eefd95..1e4f17c0d3ca1 100644 --- a/tests/kafkatest/services/kafka/templates/kafka.properties +++ b/tests/kafkatest/services/kafka/templates/kafka.properties @@ -50,7 +50,7 @@ quota.producer.bytes.per.second.overrides={{ quota_config.quota_producer_bytes_p quota.consumer.bytes.per.second.overrides={{ quota_config.quota_consumer_bytes_per_second_overrides }} {% endif %} -security.inter.broker.protocol={{ interbroker_security_protocol }} +security.inter.broker.protocol={{ security_config.interbroker_security_protocol }} ssl.keystore.location=/mnt/security/test.keystore.jks ssl.keystore.password=test-ks-passwd @@ -59,7 +59,8 @@ ssl.keystore.type=JKS ssl.truststore.location=/mnt/security/test.truststore.jks ssl.truststore.password=test-ts-passwd ssl.truststore.type=JKS -sasl.mechanism={{ sasl_mechanism }} +sasl.mechanism.inter.broker.protocol={{ security_config.interbroker_sasl_mechanism }} +sasl.enabled.mechanisms={{ ",".join(security_config.enabled_sasl_mechanisms) }} sasl.kerberos.service.name=kafka {% if authorizer_class_name is not none %} ssl.client.auth=required diff --git a/tests/kafkatest/services/security/security_config.py b/tests/kafkatest/services/security/security_config.py index 1bbabd2359b02..d7cc3c0a01746 100644 --- a/tests/kafkatest/services/security/security_config.py +++ b/tests/kafkatest/services/security/security_config.py @@ -75,7 +75,9 @@ class SecurityConfig(TemplateRenderer): ssl_stores = Keytool.generate_keystore_truststore('.') - def __init__(self, security_protocol=None, interbroker_security_protocol=None, sasl_mechanism=SASL_MECHANISM_GSSAPI, zk_sasl=False, template_props=""): + def __init__(self, security_protocol=None, interbroker_security_protocol=None, + client_sasl_mechanism=SASL_MECHANISM_GSSAPI, interbroker_sasl_mechanism=SASL_MECHANISM_GSSAPI, + zk_sasl=False, template_props=""): """ Initialize the security properties for the node and copy keystore and truststore to the remote node if the transport protocol @@ -104,13 +106,14 @@ def __init__(self, security_protocol=None, interbroker_security_protocol=None, s 'ssl.key.password' : SecurityConfig.ssl_stores['ssl.key.password'], 'ssl.truststore.location' : SecurityConfig.TRUSTSTORE_PATH, 'ssl.truststore.password' : SecurityConfig.ssl_stores['ssl.truststore.password'], - 'sasl.mechanism' : sasl_mechanism, + 'sasl.mechanism' : client_sasl_mechanism, + 'sasl.mechanism.inter.broker.protocol' : interbroker_sasl_mechanism, 'sasl.kerberos.service.name' : 'kafka' } def client_config(self, template_props=""): - return SecurityConfig(self.security_protocol, sasl_mechanism=self.sasl_mechanism, template_props=template_props) + return SecurityConfig(self.security_protocol, client_sasl_mechanism=self.client_sasl_mechanism, template_props=template_props) def setup_node(self, node): if self.has_ssl: @@ -120,13 +123,15 @@ def setup_node(self, node): if self.has_sasl: node.account.ssh("mkdir -p %s" % SecurityConfig.CONFIG_DIR, allow_fail=False) - jaas_conf_file = self.sasl_mechanism.lower() + "_jaas.conf" + jaas_conf_file = "jaas.conf" java_version = node.account.ssh_capture("java -version") if any('IBM' in line for line in java_version): is_ibm_jdk = True else: is_ibm_jdk = False - jaas_conf = self.render(jaas_conf_file, node=node, is_ibm_jdk=is_ibm_jdk) + jaas_conf = self.render(jaas_conf_file, node=node, is_ibm_jdk=is_ibm_jdk, + client_sasl_mechanism=self.client_sasl_mechanism, + enabled_sasl_mechanisms=self.enabled_sasl_mechanisms) node.account.create_file(SecurityConfig.JAAS_CONF_PATH, jaas_conf) if self.has_sasl_kerberos: node.account.scp_to(MiniKdc.LOCAL_KEYTAB_FILE, SecurityConfig.KEYTAB_PATH) @@ -159,12 +164,20 @@ def security_protocol(self): return self.properties['security.protocol'] @property - def sasl_mechanism(self): + def client_sasl_mechanism(self): return self.properties['sasl.mechanism'] + @property + def interbroker_sasl_mechanism(self): + return self.properties['sasl.mechanism.inter.broker.protocol'] + + @property + def enabled_sasl_mechanisms(self): + return set([self.client_sasl_mechanism, self.interbroker_sasl_mechanism]) + @property def has_sasl_kerberos(self): - return self.has_sasl and self.sasl_mechanism == SecurityConfig.SASL_MECHANISM_GSSAPI + return self.has_sasl and (SecurityConfig.SASL_MECHANISM_GSSAPI in self.enabled_sasl_mechanisms) @property def kafka_opts(self): diff --git a/tests/kafkatest/services/security/templates/gssapi_jaas.conf b/tests/kafkatest/services/security/templates/jaas.conf similarity index 61% rename from tests/kafkatest/services/security/templates/gssapi_jaas.conf rename to tests/kafkatest/services/security/templates/jaas.conf index 6a629d9ec8542..fbfa8af49c2db 100644 --- a/tests/kafkatest/services/security/templates/gssapi_jaas.conf +++ b/tests/kafkatest/services/security/templates/jaas.conf @@ -11,76 +11,85 @@ * specific language governing permissions and limitations under the License. */ -{% if is_ibm_jdk %} KafkaClient { +{% if client_sasl_mechanism == "GSSAPI" %} +{% if is_ibm_jdk %} com.ibm.security.auth.module.Krb5LoginModule required debug=false credsType=both useKeytab="file:/mnt/security/keytab" principal="client@EXAMPLE.COM"; -}; - -KafkaServer { - com.ibm.security.auth.module.Krb5LoginModule required debug=false - credsType=both - useKeytab="file:/mnt/security/keytab" - principal="kafka/{{ node.account.hostname }}@EXAMPLE.COM"; -}; -{% if zk_sasl %} -Client { - com.ibm.security.auth.module.Krb5LoginModule required debug=false - credsType=both - useKeytab="file:/mnt/security/keytab" - principal="zkclient@EXAMPLE.COM"; -}; - -Server { - com.ibm.security.auth.module.Krb5LoginModule required debug=false - credsType=both - useKeyTab="file:/mnt/security/keytab" - principal="zookeeper/{{ node.account.hostname }}@EXAMPLE.COM"; -}; -{% endif %} {% else %} - -KafkaClient { com.sun.security.auth.module.Krb5LoginModule required debug=false doNotPrompt=true useKeyTab=true storeKey=true keyTab="/mnt/security/keytab" principal="client@EXAMPLE.COM"; +{% endif %} +{% elif client_sasl_mechanism == "PLAIN" %} + org.apache.kafka.common.security.plain.PlainLoginModule required + username="client" + password="client-secret"; +{% endif %} + }; KafkaServer { +{% if "GSSAPI" in enabled_sasl_mechanisms %} +{% if is_ibm_jdk %} + com.ibm.security.auth.module.Krb5LoginModule required debug=false + credsType=both + useKeytab="file:/mnt/security/keytab" + principal="kafka/{{ node.account.hostname }}@EXAMPLE.COM"; +{% else %} com.sun.security.auth.module.Krb5LoginModule required debug=false doNotPrompt=true useKeyTab=true storeKey=true keyTab="/mnt/security/keytab" principal="kafka/{{ node.account.hostname }}@EXAMPLE.COM"; +{% endif %} +{% endif %} +{% if "PLAIN" in enabled_sasl_mechanisms %} + org.apache.kafka.common.security.plain.PlainLoginModule required + username="kafka" + password="kafka-secret" + user_client="client-secret" + user_kafka="kafka-secret"; +{% endif %} }; {% if zk_sasl %} Client { - com.sun.security.auth.module.Krb5LoginModule required - useKeyTab=true - keyTab="/mnt/security/keytab" - storeKey=true - useTicketCache=false - principal="zkclient@EXAMPLE.COM"; +{% if is_ibm_jdk %} + com.ibm.security.auth.module.Krb5LoginModule required debug=false + credsType=both + useKeytab="file:/mnt/security/keytab" + principal="zkclient@EXAMPLE.COM"; +{% else %} + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + keyTab="/mnt/security/keytab" + storeKey=true + useTicketCache=false + principal="zkclient@EXAMPLE.COM"; +{% endif %} }; Server { - com.sun.security.auth.module.Krb5LoginModule required - useKeyTab=true - keyTab="/mnt/security/keytab" - storeKey=true - useTicketCache=false - principal="zookeeper/{{ node.account.hostname }}@EXAMPLE.COM"; -}; +{% if is_ibm_jdk %} + com.ibm.security.auth.module.Krb5LoginModule required debug=false + credsType=both + useKeyTab="file:/mnt/security/keytab" + principal="zookeeper/{{ node.account.hostname }}@EXAMPLE.COM"; +{% else %} + com.sun.security.auth.module.Krb5LoginModule required + useKeyTab=true + keyTab="/mnt/security/keytab" + storeKey=true + useTicketCache=false + principal="zookeeper/{{ node.account.hostname }}@EXAMPLE.COM"; {% endif %} +}; {% endif %} - - - diff --git a/tests/kafkatest/tests/core/replication_test.py b/tests/kafkatest/tests/core/replication_test.py index 7b360abb16eff..8e9474aec270a 100644 --- a/tests/kafkatest/tests/core/replication_test.py +++ b/tests/kafkatest/tests/core/replication_test.py @@ -128,7 +128,10 @@ def min_cluster_size(self): @matrix(failure_mode=["clean_shutdown", "hard_shutdown", "clean_bounce", "hard_bounce"], broker_type=["controller"], security_protocol=["PLAINTEXT", "SASL_SSL"]) - def test_replication_with_broker_failure(self, failure_mode, security_protocol, broker_type): + @matrix(failure_mode=["hard_bounce"], + broker_type=["leader"], + security_protocol=["SASL_SSL"], client_sasl_mechanism=["PLAIN"], interbroker_sasl_mechanism=["PLAIN", "GSSAPI"]) + def test_replication_with_broker_failure(self, failure_mode, security_protocol, broker_type, client_sasl_mechanism="GSSAPI", interbroker_sasl_mechanism="GSSAPI"): """Replication tests. These tests verify that replication provides simple durability guarantees by checking that data acked by brokers is still available for consumption in the face of various failure scenarios. @@ -144,6 +147,8 @@ def test_replication_with_broker_failure(self, failure_mode, security_protocol, self.kafka.security_protocol = security_protocol self.kafka.interbroker_security_protocol = security_protocol + self.kafka.client_sasl_mechanism = client_sasl_mechanism + self.kafka.interbroker_sasl_mechanism = interbroker_sasl_mechanism new_consumer = False if self.kafka.security_protocol == "PLAINTEXT" else True self.producer = VerifiableProducer(self.test_context, self.num_producers, self.kafka, self.topic, throughput=self.producer_throughput) self.consumer = ConsoleConsumer(self.test_context, self.num_consumers, self.kafka, self.topic, new_consumer=new_consumer, consumer_timeout_ms=60000, message_validator=is_int) From fe6c481b38d6b1b61341e4e1a6237f64accfbfbc Mon Sep 17 00:00:00 2001 From: Jason Gustafson Date: Fri, 29 Apr 2016 10:26:01 -0700 Subject: [PATCH 195/206] KAFKA-3418: add javadoc section describing consumer failure detection Author: Jason Gustafson Reviewers: Manikumar Reddy , Ismael Juma , Ewen Cheslack-Postava Closes #1129 from hachikuji/KAFKA-3418 --- .../kafka/clients/consumer/KafkaConsumer.java | 71 +++++++++++++++---- 1 file changed, 57 insertions(+), 14 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java index ad44d16250093..7290a38445646 100644 --- a/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java +++ b/clients/src/main/java/org/apache/kafka/clients/consumer/KafkaConsumer.java @@ -76,7 +76,7 @@ *

          Offsets and Consumer Position

          * Kafka maintains a numerical offset for each record in a partition. This offset acts as a kind of unique identifier of * a record within that partition, and also denotes the position of the consumer in the partition. That is, a consumer - * which has position 5 has consumed records with offsets 0 through 4 and will next receive record with offset 5. There + * which has position 5 has consumed records with offsets 0 through 4 and will next receive the record with offset 5. There * are actually two notions of position relevant to the user of the consumer. *

          * The {@link #position(TopicPartition) position} of the consumer gives the offset of the next record that will be given @@ -95,22 +95,23 @@ * *

          Consumer Groups and Topic Subscriptions

          * - * Kafka uses the concept of consumer groups to allow a pool of processes to divide up the work of consuming and + * Kafka uses the concept of consumer groups to allow a pool of processes to divide the work of consuming and * processing records. These processes can either be running on the same machine or, as is more likely, they can be - * distributed over many machines to provide additional scalability and fault tolerance for processing. + * distributed over many machines to provide scalability and fault tolerance for processing. *

          - * Each Kafka consumer is able to configure a consumer group that it belongs to, and can dynamically set the - * list of topics it wants to subscribe to through {@link #subscribe(Collection, ConsumerRebalanceListener)}, - * or subscribe to all topics matching certain pattern through {@link #subscribe(Pattern, ConsumerRebalanceListener)}. - * Kafka will deliver each message in the - * subscribed topics to one process in each consumer group. This is achieved by balancing the partitions in the topic - * over the consumer processes in each group. So if there is a topic with four partitions, and a consumer group with two - * processes, each process would consume from two partitions. This group membership is maintained dynamically: if a - * process fails the partitions assigned to it will be reassigned to other processes in the same group, and if a new - * process joins the group, partitions will be moved from existing consumers to this new process. + * Each Kafka consumer is able to configure a consumer group that it belongs to, and can dynamically set the list + * of topics it wants to subscribe to through one of the {@link #subscribe(Collection, ConsumerRebalanceListener) subscribe} + * APIs. Kafka will deliver each message in the subscribed topics to one process in each consumer group. + * This is achieved by balancing the partitions between all members in the consumer group so that each partition is + * assigned to exactly one consumer in the group. So if there is a topic with four partitions, and a consumer group with two + * processes, each process would consume from two partitions. *

          - * So if two processes subscribe to a topic both specifying different groups they will each get all the records in that - * topic; if they both specify the same group they will each get about half the records. + * Membership in a consumer group is maintained dynamically: if a process fails, the partitions assigned to it will + * be reassigned to other consumers in the same group. Similarly, if a new consumer joins the group, partitions will be moved + * from existing consumers to the new one. This is known as rebalancing the group and is discussed in more + * detail below. Note that the same process is also used when new partitions are added + * to one of the subscribed topics: the group automatically detects the new partitions and rebalances the group so + * that every new partition is assigned to one of the members. *

          * Conceptually you can think of a consumer group as being a single logical subscriber that happens to be made up of * multiple processes. As a multi-subscriber system, Kafka naturally supports having any number of consumer groups for a @@ -131,6 +132,48 @@ * (similar to the older "simple" consumer) using {@link #assign(Collection)}. In this case, dynamic partition * assignment and consumer group coordination will be disabled. * + *

          Detecting Consumer Failures

          + * + * After subscribing to a set of topics, the consumer will automatically join the group when {@link #poll(long)} is + * invoked. The poll API is designed to ensure consumer liveness. As long as you continue to call poll, the consumer + * will stay in the group and continue to receive messages from the partitions it was assigned. Underneath the covers, + * the poll API sends periodic heartbeats to the server; when you stop calling poll (perhaps because an exception was thrown), + * then no heartbeats will be sent. If a period of the configured session timeout elapses before the server + * has received a heartbeat, then the consumer will be kicked out of the group and its partitions will be reassigned. + * This is designed to prevent situations where the consumer has failed, yet continues to hold onto the partitions + * it was assigned (thus preventing active consumers in the group from taking them). To stay in the group, you + * have to prove you are still alive by calling poll. + *

          + * The implication of this design is that message processing time in the poll loop must be bounded so that + * heartbeats can be sent before expiration of the session timeout. What typically happens when processing time + * exceeds the session timeout is that the consumer won't be able to commit offsets for any of the processed records. + * For example, this is indicated by a {@link CommitFailedException} thrown from {@link #commitSync()}. This + * guarantees that only active members of the group are allowed to commit offsets. If the consumer + * has been kicked out of the group, then its partitions will have been assigned to another member, which will be + * committing its own offsets as it handles new records. This gives offset commits an isolation guarantee. + *

          + * The consumer provides two configuration settings to control this behavior: + *

            + *
          1. session.timeout.ms: By increasing the session timeout, you can give the consumer more + * time to handle a batch of records returned from {@link #poll(long)}. The only drawback is that it + * will take longer for the server to detect hard consumer failures, which can cause a delay before + * a rebalance can be completed. However, clean shutdown with {@link #close()} is not impacted since + * the consumer will send an explicit message to the server to leave the group and cause an immediate + * rebalance.
          2. + *
          3. max.poll.records: Processing time in the poll loop is typically proportional to the number + * of records processed, so it's natural to want to set a limit on the number of records handled at once. + * This setting provides that. By default, there is essentially no limit.
          4. + *
          + *

          + * For use cases where message processing time varies unpredictably, neither of these options may be viable. + * The recommended way to handle these cases is to move message processing to another thread, which allows + * the consumer to continue sending heartbeats while the processor is still working. Some care must be taken + * to ensure that committed offsets do not get ahead of the actual position. Typically, you must disable automatic + * commits and manually commit processed offsets for records only after the thread has finished handling them + * (depending on the delivery semantics you need). Note also that you will generally need to {@link #pause(Collection)} + * the partition so that no new records are received from poll until after thread has finished handling those + * previously returned. + * *

          Usage Examples

          * The consumer APIs offer flexibility to cover a variety of consumption use cases. Here are some examples to * demonstrate how to use them. From eb50d2f6ca9dee041689db6fe397a83ece05b9e2 Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Fri, 29 Apr 2016 10:28:33 -0700 Subject: [PATCH 196/206] KAFKA-3615: Exclude test jars in kafka-run-class.sh granders hachikuji Can you take a look when you have time? Appreciate your time to review. Author: Liquan Pei Reviewers: Grant Henke , Geoff Anderson , Ewen Cheslack-Postava Closes #1263 from Ishiihara/classpath-no-test-jar --- bin/kafka-run-class.sh | 57 ++++++++++++++++---- tests/kafkatest/services/security/minikdc.py | 5 +- tests/kafkatest/services/streams.py | 10 ++-- 3 files changed, 54 insertions(+), 18 deletions(-) diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index 88d43be3188d9..e7f8d2ef9fec3 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -20,6 +20,24 @@ then exit 1 fi +if [ -z "$INCLUDE_TEST_JARS" ]; then + INCLUDE_TEST_JARS=false +fi + +# Exclude jars not necessary for running commands. +regex="(-(test|src|scaladoc|javadoc)\.jar|jar.asc)$" +should_include_file() { + if [ "$INCLUDE_TEST_JARS" = true ]; then + return 0 + fi + file=$1 + if [ -z "$(echo "$file" | egrep "$regex")" ] ; then + return 0 + else + return 1 + fi +} + base_dir=$(dirname $0)/.. if [ -z "$SCALA_VERSION" ]; then @@ -41,24 +59,32 @@ do fi done -for file in $base_dir/examples/build/libs//kafka-examples*.jar; +for file in $base_dir/examples/build/libs/kafka-examples*.jar; do - CLASSPATH=$CLASSPATH:$file + if should_include_file "$file"; then + CLASSPATH=$CLASSPATH:$file + fi done for file in $base_dir/clients/build/libs/kafka-clients*.jar; do - CLASSPATH=$CLASSPATH:$file + if should_include_file "$file"; then + CLASSPATH=$CLASSPATH:$file + fi done for file in $base_dir/streams/build/libs/kafka-streams*.jar; do - CLASSPATH=$CLASSPATH:$file + if should_include_file "$file"; then + CLASSPATH=$CLASSPATH:$file + fi done for file in $base_dir/streams/examples/build/libs/kafka-streams-examples*.jar; do - CLASSPATH=$CLASSPATH:$file + if should_include_file "$file"; then + CLASSPATH=$CLASSPATH:$file + fi done for file in $base_dir/streams/build/dependant-libs-${SCALA_VERSION}/rocksdb*.jar; @@ -68,7 +94,9 @@ done for file in $base_dir/tools/build/libs/kafka-tools*.jar; do - CLASSPATH=$CLASSPATH:$file + if should_include_file "$file"; then + CLASSPATH=$CLASSPATH:$file + fi done for dir in $base_dir/tools/build/dependant-libs-${SCALA_VERSION}*; @@ -80,7 +108,9 @@ for cc_pkg in "api" "runtime" "file" "json" "tools" do for file in $base_dir/connect/${cc_pkg}/build/libs/connect-${cc_pkg}*.jar; do - CLASSPATH=$CLASSPATH:$file + if should_include_file "$file"; then + CLASSPATH=$CLASSPATH:$file + fi done if [ -d "$base_dir/connect/${cc_pkg}/build/dependant-libs" ] ; then CLASSPATH=$CLASSPATH:$base_dir/connect/${cc_pkg}/build/dependant-libs/* @@ -88,11 +118,18 @@ do done # classpath addition for release -CLASSPATH=$CLASSPATH:$base_dir/libs/* +for file in $base_dir/libs; +do + if should_include_file "$file"; then + CLASSPATH=$CLASSPATH:$file + fi +done for file in $base_dir/core/build/libs/kafka_${SCALA_BINARY_VERSION}*.jar; do - CLASSPATH=$CLASSPATH:$file + if should_include_file "$file"; then + CLASSPATH=$CLASSPATH:$file + fi done shopt -u nullglob @@ -145,8 +182,6 @@ if [ "x$KAFKA_DEBUG" != "x" ]; then JAVA_DEBUG_OPTS="$DEFAULT_JAVA_DEBUG_OPTS" fi - - echo "Enabling Java debug options: $JAVA_DEBUG_OPTS" KAFKA_OPTS="$JAVA_DEBUG_OPTS $KAFKA_OPTS" fi diff --git a/tests/kafkatest/services/security/minikdc.py b/tests/kafkatest/services/security/minikdc.py index b376e268f7f8b..d83aede11a89b 100644 --- a/tests/kafkatest/services/security/minikdc.py +++ b/tests/kafkatest/services/security/minikdc.py @@ -23,6 +23,7 @@ from io import open import uuid + class MiniKdc(Service): logs = { @@ -54,9 +55,7 @@ def replace_in_file(self, file_path, pattern, subst): remove(file_path) move(abs_path, file_path) - def start_node(self, node): - node.account.ssh("mkdir -p %s" % MiniKdc.WORK_DIR, allow_fail=False) props_file = self.render('minikdc.properties', node=node) node.account.create_file(MiniKdc.PROPS_FILE, props_file) @@ -69,7 +68,7 @@ def start_node(self, node): jar_paths = self.core_jar_paths(node, "dependant-testlibs") + self.core_jar_paths(node, "libs") classpath = ":".join(jar_paths) - cmd = "CLASSPATH=%s /opt/%s/bin/kafka-run-class.sh kafka.security.minikdc.MiniKdc %s %s %s %s 1>> %s 2>> %s &" % (classpath, kafka_dir(node), MiniKdc.WORK_DIR, MiniKdc.PROPS_FILE, MiniKdc.KEYTAB_FILE, principals, MiniKdc.LOG_FILE, MiniKdc.LOG_FILE) + cmd = "INCLUDE_TEST_JARS=true CLASSPATH=%s /opt/%s/bin/kafka-run-class.sh kafka.security.minikdc.MiniKdc %s %s %s %s 1>> %s 2>> %s &" % (classpath, kafka_dir(node), MiniKdc.WORK_DIR, MiniKdc.PROPS_FILE, MiniKdc.KEYTAB_FILE, principals, MiniKdc.LOG_FILE, MiniKdc.LOG_FILE) self.logger.debug("Attempting to start MiniKdc on %s with command: %s" % (str(node.account), cmd)) with node.account.monitor_log(MiniKdc.LOG_FILE) as monitor: node.account.ssh(cmd) diff --git a/tests/kafkatest/services/streams.py b/tests/kafkatest/services/streams.py index dcbcc696b8b3f..53d967e1bc8bd 100644 --- a/tests/kafkatest/services/streams.py +++ b/tests/kafkatest/services/streams.py @@ -15,10 +15,11 @@ from ducktape.services.service import Service from ducktape.utils.util import wait_until -from ducktape.errors import DucktapeError from kafkatest.services.kafka.directory import kafka_dir -import signal, random, requests, os.path, json +import signal +import os.path + class StreamsSmokeTestBaseService(Service): """Base class for Streams Smoke Test services providing some common settings and functionality""" @@ -46,7 +47,7 @@ class StreamsSmokeTestBaseService(Service): def __init__(self, context, kafka, command): super(StreamsSmokeTestBaseService, self).__init__(context, 1) self.kafka = kafka - self.args = { 'command': command } + self.args = {'command': command} @property def node(self): @@ -107,7 +108,7 @@ def start_cmd(self, node): args['kafka_dir'] = kafka_dir(node) cmd = "( export KAFKA_LOG4J_OPTS=\"-Dlog4j.configuration=file:%(log4j)s\"; " \ - "/opt/%(kafka_dir)s/bin/kafka-run-class.sh org.apache.kafka.streams.smoketest.StreamsSmokeTest " \ + "INCLUDE_TEST_JARS=true /opt/%(kafka_dir)s/bin/kafka-run-class.sh org.apache.kafka.streams.smoketest.StreamsSmokeTest " \ " %(command)s %(kafka)s %(zk)s %(state_dir)s " \ " & echo $! >&3 ) 1>> %(stdout)s 2>> %(stderr)s 3> %(pidfile)s" % args @@ -131,6 +132,7 @@ class StreamsSmokeTestDriverService(StreamsSmokeTestBaseService): def __init__(self, context, kafka): super(StreamsSmokeTestDriverService, self).__init__(context, kafka, "run") + class StreamsSmokeTestJobRunnerService(StreamsSmokeTestBaseService): def __init__(self, context, kafka): super(StreamsSmokeTestJobRunnerService, self).__init__(context, kafka, "process") From e29eac4bbb678aa3d5a29a75f413a7b10cc2f0b1 Mon Sep 17 00:00:00 2001 From: Anna Povzner Date: Fri, 29 Apr 2016 10:51:29 -0700 Subject: [PATCH 197/206] KAFKA-3597; Query ConsoleConsumer and VerifiableProducer if they shutdown cleanly Even if a test calls stop() on console_consumer or verifiable_producer, it is still possible that producer/consumer will not shutdown cleanly, and will be killed forcefully after a timeout. It will be useful for some tests to know whether a clean shutdown happened or not. This PR adds methods to console_consumer and verifiable_producer to query whether clean shutdown happened or not. hachikuji and/or granders Please review. Author: Anna Povzner Reviewers: Jason Gustafson, Geoff Anderson, Gwen Shapira Closes #1278 from apovzner/kafka-3597 --- .../main/scala/kafka/tools/ConsoleConsumer.scala | 8 ++++++++ tests/kafkatest/services/console_consumer.py | 15 +++++++++++---- tests/kafkatest/services/verifiable_producer.py | 6 ++++++ .../apache/kafka/tools/VerifiableProducer.java | 8 ++++++++ 4 files changed, 33 insertions(+), 4 deletions(-) diff --git a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala index e9a43f2bf422b..89536400e1cd9 100755 --- a/core/src/main/scala/kafka/tools/ConsoleConsumer.scala +++ b/core/src/main/scala/kafka/tools/ConsoleConsumer.scala @@ -103,6 +103,10 @@ object ConsoleConsumer extends Logging { consumer.stop() shutdownLatch.await() + + if (conf.enableSystestEventsLogging) { + System.out.println("shutdown_complete") + } } }) } @@ -253,6 +257,9 @@ object ConsoleConsumer extends Logging { .withRequiredArg .describedAs("deserializer for values") .ofType(classOf[String]) + val enableSystestEventsLoggingOpt = parser.accepts("enable-systest-events", + "Log lifecycle events of the consumer in addition to logging consumed " + + "messages. (This is specific for system tests.)") if (args.length == 0) CommandLineUtils.printUsageAndDie(parser, "The console consumer is a tool that reads data from Kafka and outputs it to standard output.") @@ -260,6 +267,7 @@ object ConsoleConsumer extends Logging { var groupIdPassed = true val options: OptionSet = tryParse(parser, args) val useNewConsumer = options.has(useNewConsumerOpt) + val enableSystestEventsLogging = options.has(enableSystestEventsLoggingOpt) // If using old consumer, exactly one of whitelist/blacklist/topic is required. // If using new consumer, topic must be specified. diff --git a/tests/kafkatest/services/console_consumer.py b/tests/kafkatest/services/console_consumer.py index e5f2196e18418..37638e2a9812b 100644 --- a/tests/kafkatest/services/console_consumer.py +++ b/tests/kafkatest/services/console_consumer.py @@ -123,6 +123,7 @@ def __init__(self, context, num_nodes, kafka, topic, group_id="test-consumer-gro self.from_beginning = from_beginning self.message_validator = message_validator self.messages_consumed = {idx: [] for idx in range(1, num_nodes + 1)} + self.clean_shutdown_nodes = set() self.client_id = client_id self.print_key = print_key self.log_level = "TRACE" @@ -185,6 +186,7 @@ def start_cmd(self, node): if node.version > LATEST_0_9: cmd+=" --formatter kafka.tools.LoggingMessageFormatter" + cmd += " --enable-systest-events" cmd += " 2>> %(stderr)s | tee -a %(stdout)s &" % args return cmd @@ -226,10 +228,15 @@ def _worker(self, idx, node): for line in itertools.chain([first_line], consumer_output): msg = line.strip() - if self.message_validator is not None: - msg = self.message_validator(msg) - if msg is not None: - self.messages_consumed[idx].append(msg) + if msg == "shutdown_complete": + if node in self.clean_shutdown_nodes: + raise Exception("Unexpected shutdown event from consumer, already shutdown. Consumer index: %d" % idx) + self.clean_shutdown_nodes.add(node) + else: + if self.message_validator is not None: + msg = self.message_validator(msg) + if msg is not None: + self.messages_consumed[idx].append(msg) self.read_jmx_output(idx, node) diff --git a/tests/kafkatest/services/verifiable_producer.py b/tests/kafkatest/services/verifiable_producer.py index 500410f134f10..4fec776719481 100644 --- a/tests/kafkatest/services/verifiable_producer.py +++ b/tests/kafkatest/services/verifiable_producer.py @@ -71,6 +71,7 @@ def __init__(self, context, num_nodes, kafka, topic, max_messages=-1, throughput self.acked_values = [] self.not_acked_values = [] self.produced_count = {} + self.clean_shutdown_nodes = set() self.acks = acks @@ -139,6 +140,11 @@ def _worker(self, idx, node): last_produced_time = t prev_msg = data + elif data["name"] == "shutdown_complete": + if node in self.clean_shutdown_nodes: + raise Exception("Unexpected shutdown event from producer, already shutdown. Producer index: %d" % idx) + self.clean_shutdown_nodes.add(node) + def start_cmd(self, node, idx): cmd = "" diff --git a/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java b/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java index 9b10a9f0dd613..b511fb94c8c6a 100644 --- a/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java +++ b/tools/src/main/java/org/apache/kafka/tools/VerifiableProducer.java @@ -247,6 +247,14 @@ public String getValue(long val) { /** Close the producer to flush any remaining messages. */ public void close() { producer.close(); + System.out.println(shutdownString()); + } + + String shutdownString() { + Map data = new HashMap<>(); + data.put("class", this.getClass().toString()); + data.put("name", "shutdown_complete"); + return toJsonString(data); } /** From e50327331d7d4e2854297a377f6701d737599672 Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Fri, 29 Apr 2016 10:52:42 -0700 Subject: [PATCH 198/206] KAFKA-3578: Allow cross origin HTTP requests on all HTTP methods Author: Liquan Pei Reviewers: Ewen Cheslack-Postava Closes #1288 from Ishiihara/kip-56 --- .../kafka/connect/runtime/WorkerConfig.java | 10 ++++++- .../connect/runtime/rest/RestServer.java | 14 +++++++--- .../connect/runtime/rest/RestServerTest.java | 28 +++++++++++++------ 3 files changed, 39 insertions(+), 13 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java index 471e4a57975a5..7ede1307e12a6 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/WorkerConfig.java @@ -109,6 +109,11 @@ public class WorkerConfig extends AbstractConfig { " from the domain of the REST API."; protected static final String ACCESS_CONTROL_ALLOW_ORIGIN_DEFAULT = ""; + public static final String ACCESS_CONTROL_ALLOW_METHODS_CONFIG = "access.control.allow.methods"; + protected static final String ACCESS_CONTROL_ALLOW_METHODS_DOC = + "Sets the methods supported for cross origin requests by setting the Access-Control-Allow-Methods header. " + + "The default value of the Access-Control-Allow-Methods header allows cross origin requests for GET, POST and HEAD."; + protected static final String ACCESS_CONTROL_ALLOW_METHODS_DEFAULT = ""; /** * Get a basic ConfigDef for a WorkerConfig. This includes all the common settings. Subclasses can use this to @@ -141,7 +146,10 @@ protected static ConfigDef baseConfigDef() { .define(REST_ADVERTISED_PORT_CONFIG, Type.INT, null, Importance.LOW, REST_ADVERTISED_PORT_DOC) .define(ACCESS_CONTROL_ALLOW_ORIGIN_CONFIG, Type.STRING, ACCESS_CONTROL_ALLOW_ORIGIN_DEFAULT, Importance.LOW, - ACCESS_CONTROL_ALLOW_ORIGIN_DOC); + ACCESS_CONTROL_ALLOW_ORIGIN_DOC) + .define(ACCESS_CONTROL_ALLOW_METHODS_CONFIG, Type.STRING, + ACCESS_CONTROL_ALLOW_METHODS_DEFAULT, Importance.LOW, + ACCESS_CONTROL_ALLOW_METHODS_DOC); } public WorkerConfig(ConfigDef definition, Map props) { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java index 3475e1c5683c6..a878fb0eab810 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/rest/RestServer.java @@ -20,6 +20,7 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider; + import org.apache.kafka.connect.errors.ConnectException; import org.apache.kafka.connect.runtime.Herder; import org.apache.kafka.connect.runtime.WorkerConfig; @@ -47,9 +48,6 @@ import org.slf4j.Logger; import org.slf4j.LoggerFactory; -import javax.servlet.DispatcherType; -import javax.ws.rs.core.Response; -import javax.ws.rs.core.UriBuilder; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -60,6 +58,10 @@ import java.util.List; import java.util.Map; +import javax.servlet.DispatcherType; +import javax.ws.rs.core.Response; +import javax.ws.rs.core.UriBuilder; + /** * Embedded server for the REST API that provides the control plane for Kafka Connect workers. */ @@ -115,7 +117,11 @@ public void start(Herder herder) { if (allowedOrigins != null && !allowedOrigins.trim().isEmpty()) { FilterHolder filterHolder = new FilterHolder(new CrossOriginFilter()); filterHolder.setName("cross-origin"); - filterHolder.setInitParameter("allowedOrigins", allowedOrigins); + filterHolder.setInitParameter(CrossOriginFilter.ALLOWED_ORIGINS_PARAM, allowedOrigins); + String allowedMethods = config.getString(WorkerConfig.ACCESS_CONTROL_ALLOW_METHODS_CONFIG); + if (allowedMethods != null && !allowedOrigins.trim().isEmpty()) { + filterHolder.setInitParameter(CrossOriginFilter.ALLOWED_METHODS_PARAM, allowedMethods); + } context.addFilter(filterHolder, "/*", EnumSet.of(DispatcherType.REQUEST)); } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java index 8e9d52b4ca431..64d5b5efc072c 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/runtime/rest/RestServerTest.java @@ -31,11 +31,6 @@ import org.powermock.api.easymock.annotation.MockStrict; import org.powermock.modules.junit4.PowerMockRunner; -import javax.ws.rs.client.Client; -import javax.ws.rs.client.ClientBuilder; -import javax.ws.rs.client.Invocation; -import javax.ws.rs.client.WebTarget; -import javax.ws.rs.core.Response; import java.net.URI; import java.net.URISyntaxException; import java.util.Arrays; @@ -43,6 +38,12 @@ import java.util.HashMap; import java.util.Map; +import javax.ws.rs.client.Client; +import javax.ws.rs.client.ClientBuilder; +import javax.ws.rs.client.Invocation; +import javax.ws.rs.client.WebTarget; +import javax.ws.rs.core.Response; + import static org.junit.Assert.assertEquals; @RunWith(PowerMockRunner.class) @@ -71,15 +72,15 @@ private Map baseWorkerProps() { @Test public void testCORSEnabled() { - checkCORSRequest("*", "http://bar.com", "http://bar.com"); + checkCORSRequest("*", "http://bar.com", "http://bar.com", "PUT"); } @Test public void testCORSDisabled() { - checkCORSRequest("", "http://bar.com", null); + checkCORSRequest("", "http://bar.com", null, null); } - public void checkCORSRequest(String corsDomain, String origin, String expectedHeader) { + public void checkCORSRequest(String corsDomain, String origin, String expectedHeader, String method) { // To be able to set the Origin, we need to toggle this flag System.setProperty("sun.net.http.allowRestrictedHeaders", "true"); @@ -92,10 +93,12 @@ public Object answer() throws Throwable { return null; } }); + PowerMock.replayAll(); Map workerProps = baseWorkerProps(); workerProps.put(WorkerConfig.ACCESS_CONTROL_ALLOW_ORIGIN_CONFIG, corsDomain); + workerProps.put(WorkerConfig.ACCESS_CONTROL_ALLOW_METHODS_CONFIG, method); WorkerConfig workerConfig = new StandaloneConfig(workerProps); server = new RestServer(workerConfig); server.start(herder); @@ -107,6 +110,15 @@ public Object answer() throws Throwable { assertEquals(200, response.getStatus()); assertEquals(expectedHeader, response.getHeaderString("Access-Control-Allow-Origin")); + + response = request("/connector-plugins/FileStreamSource/validate") + .header("Referer", origin + "/page") + .header("Origin", origin) + .header("Access-Control-Request-Method", method) + .options(); + assertEquals(404, response.getStatus()); + assertEquals(expectedHeader, response.getHeaderString("Access-Control-Allow-Origin")); + assertEquals(method, response.getHeaderString("Access-Control-Allow-Methods")); PowerMock.verifyAll(); } From 69d9a669d7bbfec1e33dd6177c5687ef7f9977df Mon Sep 17 00:00:00 2001 From: Rajini Sivaram Date: Fri, 29 Apr 2016 11:15:20 -0700 Subject: [PATCH 199/206] KAFKA-3618; Handle ApiVersionsRequest before SASL authentication Server-side implementation and tests for handling ApiVersionsRequest before SaslHandshakeRequest. Author: Rajini Sivaram Reviewers: Gwen Shapira, Ismael Juma Closes #1286 from rajinisivaram/KAFKA-3618 --- .../common/requests/ApiVersionsResponse.java | 14 ++ .../SaslServerAuthenticator.java | 10 ++ .../common/network/NetworkTestUtils.java | 14 +- .../authenticator/SaslAuthenticatorTest.java | 159 +++++++++++++++++- .../main/scala/kafka/server/KafkaApis.scala | 12 +- .../kafka/server/ApiVersionsRequestTest.scala | 23 ++- .../unit/kafka/server/ApiVersionsTest.scala | 6 +- .../unit/kafka/server/BaseRequestTest.scala | 24 ++- .../server/SaslApiVersionsRequestTest.scala | 78 +++++++++ 9 files changed, 297 insertions(+), 43 deletions(-) create mode 100644 core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala diff --git a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java index 36881a3e090cb..fe995b2c9af3d 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/ApiVersionsResponse.java @@ -15,6 +15,7 @@ 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.Protocol; import org.apache.kafka.common.protocol.types.Schema; import org.apache.kafka.common.protocol.types.Struct; @@ -29,6 +30,7 @@ public class ApiVersionsResponse extends AbstractRequestResponse { private static final Schema CURRENT_SCHEMA = ProtoUtils.currentResponseSchema(ApiKeys.API_VERSIONS.id); + private static final ApiVersionsResponse API_VERSIONS_RESPONSE = createApiVersionsResponse(); public static final String ERROR_CODE_KEY_NAME = "error_code"; public static final String API_VERSIONS_KEY_NAME = "api_versions"; @@ -106,6 +108,18 @@ public static ApiVersionsResponse fromError(Errors error) { return new ApiVersionsResponse(error.code(), Collections.emptyList()); } + public static ApiVersionsResponse apiVersionsResponse() { + return API_VERSIONS_RESPONSE; + } + + private static ApiVersionsResponse createApiVersionsResponse() { + List versionList = new ArrayList<>(); + for (ApiKeys apiKey : ApiKeys.values()) { + versionList.add(new ApiVersion(apiKey.id, Protocol.MIN_VERSIONS[apiKey.id], Protocol.CURR_VERSION[apiKey.id])); + } + return new ApiVersionsResponse(Errors.NONE.code(), versionList); + } + private Map buildApiKeyToApiVersion(List apiVersions) { Map tempApiIdToApiVersion = new HashMap<>(); for (ApiVersion apiVersion: apiVersions) { diff --git a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java index 89c6e6c0c0a7d..a9c19a58f887b 100644 --- a/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java +++ b/clients/src/main/java/org/apache/kafka/common/security/authenticator/SaslServerAuthenticator.java @@ -61,6 +61,8 @@ import org.apache.kafka.common.protocol.types.SchemaException; import org.apache.kafka.common.requests.AbstractRequest; import org.apache.kafka.common.requests.AbstractRequestResponse; +import org.apache.kafka.common.requests.ApiVersionsRequest; +import org.apache.kafka.common.requests.ApiVersionsResponse; import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.requests.ResponseHeader; import org.apache.kafka.common.requests.ResponseSend; @@ -290,7 +292,11 @@ private boolean handleKafkaRequest(byte[] requestBytes) throws IOException, Auth isKafkaRequest = true; ApiKeys apiKey = ApiKeys.forId(requestHeader.apiKey()); + LOG.debug("Handle Kafka request {}", apiKey); switch (apiKey) { + case API_VERSIONS: + handleApiVersionsRequest(requestHeader, (ApiVersionsRequest) request); + break; case SASL_HANDSHAKE: clientMechanism = handleHandshakeRequest(requestHeader, (SaslHandshakeRequest) request); break; @@ -336,6 +342,10 @@ private String handleHandshakeRequest(RequestHeader requestHeader, SaslHandshake } } + private void handleApiVersionsRequest(RequestHeader requestHeader, ApiVersionsRequest versionRequest) throws IOException, UnsupportedSaslMechanismException { + sendKafkaResponse(requestHeader, ApiVersionsResponse.apiVersionsResponse()); + } + private void sendKafkaResponse(RequestHeader requestHeader, AbstractRequestResponse response) throws IOException { ResponseHeader responseHeader = new ResponseHeader(requestHeader.correlationId()); netOutBuffer = new NetworkSend(node, ResponseSend.serialize(responseHeader, response.toStruct())); diff --git a/clients/src/test/java/org/apache/kafka/common/network/NetworkTestUtils.java b/clients/src/test/java/org/apache/kafka/common/network/NetworkTestUtils.java index 53ba954b934b4..969055d0ef840 100644 --- a/clients/src/test/java/org/apache/kafka/common/network/NetworkTestUtils.java +++ b/clients/src/test/java/org/apache/kafka/common/network/NetworkTestUtils.java @@ -42,13 +42,10 @@ public static Selector createSelector(ChannelBuilder channelBuilder) { public static void checkClientConnection(Selector selector, String node, int minMessageSize, int messageCount) throws Exception { + waitForChannelReady(selector, node); String prefix = TestUtils.randomString(minMessageSize); int requests = 0; int responses = 0; - // wait for handshake to finish - while (!selector.isChannelReady(node)) { - selector.poll(1000L); - } selector.send(new NetworkSend(node, ByteBuffer.wrap((prefix + "-0").getBytes()))); requests++; while (responses < messageCount) { @@ -66,6 +63,15 @@ public static void checkClientConnection(Selector selector, String node, int min } } + public static void waitForChannelReady(Selector selector, String node) throws IOException { + // wait for handshake to finish + int secondsLeft = 30; + while (!selector.isChannelReady(node) && secondsLeft-- > 0) { + selector.poll(1000L); + } + assertTrue(selector.isChannelReady(node)); + } + public static void waitForChannelClose(Selector selector, String node) throws IOException { boolean closed = false; for (int i = 0; i < 30; i++) { diff --git a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java index 0a4928b176ecc..368b5a78ce035 100644 --- a/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java +++ b/clients/src/test/java/org/apache/kafka/common/security/authenticator/SaslAuthenticatorTest.java @@ -17,11 +17,14 @@ import java.util.List; import java.util.Map; import java.util.Random; +import java.io.IOException; import java.net.InetSocketAddress; import java.nio.ByteBuffer; +import static org.junit.Assert.assertEquals; import static org.junit.Assert.fail; +import org.apache.kafka.clients.NetworkClient; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.config.SaslConfigs; import org.apache.kafka.common.network.CertStores; @@ -34,11 +37,18 @@ import org.apache.kafka.common.network.NioEchoServer; import org.apache.kafka.common.network.Selector; import org.apache.kafka.common.protocol.ApiKeys; +import org.apache.kafka.common.protocol.Errors; +import org.apache.kafka.common.protocol.Protocol; import org.apache.kafka.common.protocol.SecurityProtocol; +import org.apache.kafka.common.protocol.types.Struct; +import org.apache.kafka.common.requests.AbstractRequestResponse; +import org.apache.kafka.common.requests.ApiVersionsRequest; +import org.apache.kafka.common.requests.ApiVersionsResponse; import org.apache.kafka.common.requests.MetadataRequest; import org.apache.kafka.common.requests.RequestHeader; import org.apache.kafka.common.requests.RequestSend; import org.apache.kafka.common.requests.SaslHandshakeRequest; +import org.apache.kafka.common.requests.SaslHandshakeResponse; import org.apache.kafka.common.security.JaasUtils; import org.junit.After; import org.junit.Before; @@ -209,6 +219,30 @@ public void testMultipleServerMechanisms() throws Exception { NetworkTestUtils.checkClientConnection(selector, node2, 100, 10); } + /** + * Tests that Kafka ApiVersionsRequests are handled by the SASL server authenticator + * prior to SASL handshake flow and that subsequent authentication succeeds + * when transport layer is PLAINTEXT. This test simulates SASL authentication using a + * (non-SASL) PLAINTEXT client and sends ApiVersionsRequest straight after + * connection to the server is established, before any SASL-related packets are sent. + */ + @Test + public void testUnauthenticatedApiVersionsRequestOverPlaintext() throws Exception { + testUnauthenticatedApiVersionsRequest(SecurityProtocol.SASL_PLAINTEXT); + } + + /** + * Tests that Kafka ApiVersionsRequests are handled by the SASL server authenticator + * prior to SASL handshake flow and that subsequent authentication succeeds + * when transport layer is SSL. This test simulates SASL authentication using a + * (non-SASL) SSL client and sends ApiVersionsRequest straight after + * SSL handshake, before any SASL-related packets are sent. + */ + @Test + public void testUnauthenticatedApiVersionsRequestOverSsl() throws Exception { + testUnauthenticatedApiVersionsRequest(SecurityProtocol.SASL_SSL); + } + /** * Tests that any invalid data during Kafka SASL handshake request flow * or the actual SASL authentication flow result in authentication failure @@ -223,7 +257,7 @@ public void testInvalidSaslPacket() throws Exception { // Send invalid SASL packet after valid handshake request String node1 = "invalid1"; createClientConnection(SecurityProtocol.PLAINTEXT, node1); - sendHandshakeRequest(node1); + sendHandshakeRequestReceiveResponse(node1); Random random = new Random(); byte[] bytes = new byte[1024]; random.nextBytes(bytes); @@ -246,6 +280,33 @@ public void testInvalidSaslPacket() throws Exception { createAndCheckClientConnection(securityProtocol, "good2"); } + /** + * Tests that ApiVersionsRequest after Kafka SASL handshake request flow, + * but prior to actual SASL authentication, results in authentication failure. + * This is similar to {@link #testUnauthenticatedApiVersionsRequest(SecurityProtocol)} + * where a non-SASL client is used to send requests that are processed by + * {@link SaslServerAuthenticator} of the server prior to client authentication. + */ + @Test + public void testInvalidApiVersionsRequestSequence() throws Exception { + SecurityProtocol securityProtocol = SecurityProtocol.SASL_PLAINTEXT; + configureMechanisms("PLAIN", Arrays.asList("PLAIN")); + server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs); + + // Send handshake request followed by ApiVersionsRequest + String node1 = "invalid1"; + createClientConnection(SecurityProtocol.PLAINTEXT, node1); + sendHandshakeRequestReceiveResponse(node1); + + RequestHeader versionsHeader = new RequestHeader(ApiKeys.API_VERSIONS.id, "someclient", 2); + selector.send(new NetworkSend(node1, RequestSend.serialize(versionsHeader, new ApiVersionsRequest().toStruct()))); + NetworkTestUtils.waitForChannelClose(selector, node1); + selector.close(); + + // Test good connection still works + createAndCheckClientConnection(securityProtocol, "good1"); + } + /** * Tests that packets that are too big during Kafka SASL handshake request flow * or the actual SASL authentication flow result in authentication failure @@ -260,7 +321,7 @@ public void testPacketSizeTooBig() throws Exception { // Send SASL packet with large size after valid handshake request String node1 = "invalid1"; createClientConnection(SecurityProtocol.PLAINTEXT, node1); - sendHandshakeRequest(node1); + sendHandshakeRequestReceiveResponse(node1); ByteBuffer buffer = ByteBuffer.allocate(1024); buffer.putInt(Integer.MAX_VALUE); buffer.put(new byte[buffer.capacity() - 4]); @@ -312,7 +373,7 @@ public void testDisallowedKafkaRequestsBeforeAuthentication() throws Exception { // Send metadata request after Kafka SASL handshake request String node2 = "invalid2"; createClientConnection(SecurityProtocol.PLAINTEXT, node2); - sendHandshakeRequest(node2); + sendHandshakeRequestReceiveResponse(node2); RequestHeader metadataRequestHeader2 = new RequestHeader(ApiKeys.METADATA.id, "someclient", 2); MetadataRequest metadataRequest2 = new MetadataRequest(Collections.singletonList("sometopic")); selector.send(new NetworkSend(node2, RequestSend.serialize(metadataRequestHeader2, metadataRequest2.toStruct()))); @@ -371,6 +432,68 @@ public void testInvalidMechanism() throws Exception { NetworkTestUtils.waitForChannelClose(selector, node); } + /** + * Tests that Kafka ApiVersionsRequests are handled by the SASL server authenticator + * prior to SASL handshake flow and that subsequent authentication succeeds + * when transport layer is PLAINTEXT/SSL. This test uses a non-SASL client that simulates + * SASL authentication after ApiVersionsRequest. + *

          + * Test sequence (using securityProtocol=PLAINTEXT as an example): + *

            + *
          1. Starts a SASL_PLAINTEXT test server that simply echoes back client requests after authentication.
          2. + *
          3. A (non-SASL) PLAINTEXT test client connects to the SASL server port. Client is now unauthenticated.<./li> + *
          4. The unauthenticated non-SASL client sends an ApiVersionsRequest and validates the response. + * A valid response indicates that {@link SaslServerAuthenticator} of the test server responded to + * the ApiVersionsRequest even though the client is not yet authenticated.
          5. + *
          6. The unauthenticated non-SASL client sends a SaslHandshakeRequest and validates the response. A valid response + * indicates that {@link SaslServerAuthenticator} of the test server responded to the SaslHandshakeRequest + * after processing ApiVersionsRequest.
          7. + *
          8. The unauthenticated non-SASL client sends the SASL/PLAIN packet containing username/password to authenticate + * itself. The client is now authenticated by the server. At this point this test client is at the + * same state as a regular SASL_PLAINTEXT client that is ready.
          9. + *
          10. The authenticated client sends random data to the server and checks that the data is echoed + * back by the test server (ie, not Kafka request-response) to ensure that the client now + * behaves exactly as a regular SASL_PLAINTEXT client that has completed authentication.
          11. + *
          + */ + private void testUnauthenticatedApiVersionsRequest(SecurityProtocol securityProtocol) throws Exception { + configureMechanisms("PLAIN", Arrays.asList("PLAIN")); + server = NetworkTestUtils.createEchoServer(securityProtocol, saslServerConfigs); + + // Create non-SASL connection to manually authenticate after ApiVersionsRequest + String node = "1"; + SecurityProtocol clientProtocol; + switch (securityProtocol) { + case SASL_PLAINTEXT: + clientProtocol = SecurityProtocol.PLAINTEXT; + break; + case SASL_SSL: + clientProtocol = SecurityProtocol.SSL; + break; + default: + throw new IllegalArgumentException("Server protocol " + securityProtocol + " is not SASL"); + } + createClientConnection(clientProtocol, node); + NetworkTestUtils.waitForChannelReady(selector, node); + + // Send ApiVersionsRequest and check response + ApiVersionsResponse versionsResponse = sendVersionRequestReceiveResponse(node); + assertEquals(Protocol.MIN_VERSIONS[ApiKeys.SASL_HANDSHAKE.id], versionsResponse.apiVersion(ApiKeys.SASL_HANDSHAKE.id).minVersion); + assertEquals(Protocol.CURR_VERSION[ApiKeys.SASL_HANDSHAKE.id], versionsResponse.apiVersion(ApiKeys.SASL_HANDSHAKE.id).maxVersion); + + // Send SaslHandshakeRequest and check response + SaslHandshakeResponse handshakeResponse = sendHandshakeRequestReceiveResponse(node); + assertEquals(Collections.singletonList("PLAIN"), handshakeResponse.enabledMechanisms()); + + // Authenticate using PLAIN username/password + String authString = "\u0000" + TestJaasConfig.USERNAME + "\u0000" + TestJaasConfig.PASSWORD; + selector.send(new NetworkSend(node, ByteBuffer.wrap(authString.getBytes("UTF-8")))); + waitForResponse(); + + // Check send/receive on the manually authenticated connection + NetworkTestUtils.checkClientConnection(selector, node, 100, 10); + } + private TestJaasConfig configureMechanisms(String clientMechanism, List serverMechanisms) { saslClientConfigs.put(SaslConfigs.SASL_MECHANISM, clientMechanism); saslServerConfigs.put(SaslConfigs.SASL_ENABLED_MECHANISMS, serverMechanisms); @@ -396,13 +519,35 @@ private void createAndCheckClientConnection(SecurityProtocol securityProtocol, S selector = null; } - private void sendHandshakeRequest(String node) throws Exception { - RequestHeader header = new RequestHeader(ApiKeys.SASL_HANDSHAKE.id, "someclient", 1); + private Struct sendKafkaRequestReceiveResponse(String node, ApiKeys apiKey, AbstractRequestResponse request) throws IOException { + RequestHeader header = new RequestHeader(apiKey.id, "someclient", 1); + selector.send(new NetworkSend(node, RequestSend.serialize(header, request.toStruct()))); + ByteBuffer responseBuffer = waitForResponse(); + return NetworkClient.parseResponse(responseBuffer, header); + } + + private SaslHandshakeResponse sendHandshakeRequestReceiveResponse(String node) throws Exception { SaslHandshakeRequest handshakeRequest = new SaslHandshakeRequest("PLAIN"); - selector.send(new NetworkSend(node, RequestSend.serialize(header, handshakeRequest.toStruct()))); + Struct responseStruct = sendKafkaRequestReceiveResponse(node, ApiKeys.SASL_HANDSHAKE, handshakeRequest); + SaslHandshakeResponse response = new SaslHandshakeResponse(responseStruct); + assertEquals(Errors.NONE.code(), response.errorCode()); + return response; + } + + private ApiVersionsResponse sendVersionRequestReceiveResponse(String node) throws Exception { + ApiVersionsRequest handshakeRequest = new ApiVersionsRequest(); + Struct responseStruct = sendKafkaRequestReceiveResponse(node, ApiKeys.API_VERSIONS, handshakeRequest); + ApiVersionsResponse response = new ApiVersionsResponse(responseStruct); + assertEquals(Errors.NONE.code(), response.errorCode()); + return response; + } + + private ByteBuffer waitForResponse() throws IOException { int waitSeconds = 10; do { selector.poll(1000); - } while (selector.completedSends().isEmpty() && waitSeconds-- > 0); + } while (selector.completedReceives().isEmpty() && waitSeconds-- > 0); + assertEquals(1, selector.completedReceives().size()); + return selector.completedReceives().get(0).payload(); } } diff --git a/core/src/main/scala/kafka/server/KafkaApis.scala b/core/src/main/scala/kafka/server/KafkaApis.scala index 67d46fc092706..cf7814edff617 100644 --- a/core/src/main/scala/kafka/server/KafkaApis.scala +++ b/core/src/main/scala/kafka/server/KafkaApis.scala @@ -47,16 +47,6 @@ import scala.collection._ import scala.collection.JavaConverters._ import org.apache.kafka.common.requests.SaslHandshakeResponse -object KafkaApis { - val apiVersionsResponse = new ApiVersionsResponse(Errors.NONE.code, buildApiKeysToApiVersions.values.toList.asJava) - - private def buildApiKeysToApiVersions: Map[Short, ApiVersionsResponse.ApiVersion] = { - ApiKeys.values.map(apiKey => - apiKey.id -> new ApiVersionsResponse.ApiVersion(apiKey.id, Protocol.MIN_VERSIONS(apiKey.id), Protocol.CURR_VERSION(apiKey.id))).toMap - } -} - - /** * Logic to handle the various Kafka requests */ @@ -1041,7 +1031,7 @@ class KafkaApis(val requestChannel: RequestChannel, val isApiVersionsRequestVersionSupported = request.header.apiVersion <= Protocol.CURR_VERSION(ApiKeys.API_VERSIONS.id) && request.header.apiVersion >= Protocol.MIN_VERSIONS(ApiKeys.API_VERSIONS.id) val responseBody = if (isApiVersionsRequestVersionSupported) - KafkaApis.apiVersionsResponse + ApiVersionsResponse.apiVersionsResponse else ApiVersionsResponse.fromError(Errors.UNSUPPORTED_VERSION) requestChannel.sendResponse(new RequestChannel.Response(request, new ResponseSend(request.connectionId, responseHeader, responseBody))) diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala index ed599300c3f04..8bf4d73643e95 100644 --- a/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/ApiVersionsRequestTest.scala @@ -25,16 +25,10 @@ import org.junit.Test import scala.collection.JavaConversions._ -class ApiVersionsRequestTest extends BaseRequestTest { - - override def numBrokers: Int = 1 - - @Test - def testApiVersionsRequest() { - val apiVersionsResponse = sendApiVersionsRequest(new ApiVersionsRequest, 0) - +object ApiVersionsRequestTest { + def validateApiVersionsResponse(apiVersionsResponse: ApiVersionsResponse) { assertEquals("API keys in ApiVersionsResponse must match API keys supported by broker.", ApiKeys.values.length, apiVersionsResponse.apiVersions.size) - for (expectedApiVersion: ApiVersion <- KafkaApis.apiVersionsResponse.apiVersions) { + for (expectedApiVersion: ApiVersion <- ApiVersionsResponse.apiVersionsResponse.apiVersions) { val actualApiVersion = apiVersionsResponse.apiVersion(expectedApiVersion.apiKey) assertNotNull(s"API key ${actualApiVersion.apiKey} is supported by broker, but not received in ApiVersionsResponse.", actualApiVersion) assertEquals("API key must be supported by the broker.", expectedApiVersion.apiKey, actualApiVersion.apiKey) @@ -42,6 +36,17 @@ class ApiVersionsRequestTest extends BaseRequestTest { assertEquals(s"Received unexpected max version for API key ${actualApiVersion.apiKey}.", expectedApiVersion.maxVersion, actualApiVersion.maxVersion) } } +} + +class ApiVersionsRequestTest extends BaseRequestTest { + + override def numBrokers: Int = 1 + + @Test + def testApiVersionsRequest() { + val apiVersionsResponse = sendApiVersionsRequest(new ApiVersionsRequest, 0) + ApiVersionsRequestTest.validateApiVersionsResponse(apiVersionsResponse) + } private def sendApiVersionsRequest(request: ApiVersionsRequest, version: Short): ApiVersionsResponse = { val response = send(request, ApiKeys.API_VERSIONS, version) diff --git a/core/src/test/scala/unit/kafka/server/ApiVersionsTest.scala b/core/src/test/scala/unit/kafka/server/ApiVersionsTest.scala index 4429f26e72bf8..177b5091f4e0b 100644 --- a/core/src/test/scala/unit/kafka/server/ApiVersionsTest.scala +++ b/core/src/test/scala/unit/kafka/server/ApiVersionsTest.scala @@ -17,7 +17,7 @@ package unit.kafka.server -import kafka.server.KafkaApis +import org.apache.kafka.common.requests.ApiVersionsResponse import org.apache.kafka.common.protocol.{Protocol, ApiKeys} import org.junit.Assert._ import org.junit.Test @@ -26,11 +26,11 @@ class ApiVersionsTest { @Test def testApiVersions { - val apiVersions = KafkaApis.apiVersionsResponse.apiVersions + val apiVersions = ApiVersionsResponse.apiVersionsResponse.apiVersions assertEquals("API versions for all API keys must be maintained.", apiVersions.size, ApiKeys.values().length) for (key <- ApiKeys.values) { - val version = KafkaApis.apiVersionsResponse.apiVersion(key.id) + val version = ApiVersionsResponse.apiVersionsResponse.apiVersion(key.id) assertNotNull(s"Could not find ApiVersion for API ${key.name}", version) assertEquals(s"Incorrect min version for Api ${key.name}.", version.minVersion, Protocol.MIN_VERSIONS(key.id)) assertEquals(s"Incorrect max version for Api ${key.name}.", version.maxVersion, Protocol.CURR_VERSION(key.id)) diff --git a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala index d92ccea81e19d..906c4b27b2e4a 100644 --- a/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala +++ b/core/src/test/scala/unit/kafka/server/BaseRequestTest.scala @@ -39,7 +39,9 @@ abstract class BaseRequestTest extends KafkaServerTestHarness { protected def propertyOverrides(properties: Properties) {} def generateConfigs() = { - val props = TestUtils.createBrokerConfigs(numBrokers, zkConnect, enableControlledShutdown = false) + val props = TestUtils.createBrokerConfigs(numBrokers, zkConnect, enableControlledShutdown = false, + interBrokerSecurityProtocol = Some(securityProtocol), + trustStoreFile = trustStoreFile, saslProperties = saslProperties) props.foreach(propertyOverrides) props.map(KafkaConfig.fromProps) } @@ -57,7 +59,7 @@ abstract class BaseRequestTest extends KafkaServerTestHarness { }.map(_.socketServer).getOrElse(throw new IllegalStateException("No live broker is available")) } - private def connect(s: SocketServer = socketServer, protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): Socket = { + def connect(s: SocketServer = socketServer, protocol: SecurityProtocol = SecurityProtocol.PLAINTEXT): Socket = { new Socket("localhost", s.boundPort(protocol)) } @@ -76,20 +78,24 @@ abstract class BaseRequestTest extends KafkaServerTestHarness { response } - private def requestAndReceive(request: Array[Byte]): Array[Byte] = { - val plainSocket = connect() + def requestAndReceive(socket: Socket, request: Array[Byte]): Array[Byte] = { + sendRequest(socket, request) + receiveResponse(socket) + } + + def send(request: AbstractRequest, apiKey: ApiKeys, version: Short): ByteBuffer = { + val socket = connect() try { - sendRequest(plainSocket, request) - receiveResponse(plainSocket) + send(socket, request, apiKey, version) } finally { - plainSocket.close() + socket.close() } } /** * Serializes and send the request to the given api. A ByteBuffer containing the response is returned. */ - def send(request: AbstractRequest, apiKey: ApiKeys, version: Short): ByteBuffer = { + def send(socket: Socket, request: AbstractRequest, apiKey: ApiKeys, version: Short): ByteBuffer = { correlationId += 1 val serializedBytes = { val header = new RequestHeader(apiKey.id, version, "", correlationId) @@ -99,7 +105,7 @@ abstract class BaseRequestTest extends KafkaServerTestHarness { byteBuffer.array() } - val response = requestAndReceive(serializedBytes) + val response = requestAndReceive(socket, serializedBytes) val responseBuffer = ByteBuffer.wrap(response) ResponseHeader.parse(responseBuffer) // Parse the header to ensure its valid and move the buffer forward diff --git a/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala new file mode 100644 index 0000000000000..632665a02f599 --- /dev/null +++ b/core/src/test/scala/unit/kafka/server/SaslApiVersionsRequestTest.scala @@ -0,0 +1,78 @@ +/** + * 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.server + +import java.io.IOException +import java.net.Socket +import java.util.Collections +import org.apache.kafka.common.protocol.{ApiKeys, SecurityProtocol} +import org.apache.kafka.common.requests.{ApiVersionsRequest, ApiVersionsResponse} +import org.apache.kafka.common.requests.SaslHandshakeRequest +import org.apache.kafka.common.requests.SaslHandshakeResponse +import org.apache.kafka.common.protocol.Errors +import org.junit.Test +import org.junit.Assert._ +import kafka.api.SaslTestHarness + +class SaslApiVersionsRequestTest extends BaseRequestTest with SaslTestHarness { + override protected def securityProtocol = SecurityProtocol.SASL_PLAINTEXT + override protected val kafkaClientSaslMechanism = "PLAIN" + override protected val kafkaServerSaslMechanisms = List("PLAIN") + override protected val saslProperties = Some(kafkaSaslProperties(kafkaClientSaslMechanism, Some(kafkaServerSaslMechanisms))) + override protected val zkSaslEnabled = false + override def numBrokers = 1 + + @Test + def testApiVersionsRequestBeforeSaslHandshakeRequest() { + val plaintextSocket = connect(protocol = securityProtocol) + try { + val apiVersionsResponse = sendApiVersionsRequest(plaintextSocket, new ApiVersionsRequest, 0) + ApiVersionsRequestTest.validateApiVersionsResponse(apiVersionsResponse) + sendSaslHandshakeRequestValidateResponse(plaintextSocket) + } finally { + plaintextSocket.close() + } + } + + @Test + def testApiVersionsRequestAfterSaslHandshakeRequest() { + val plaintextSocket = connect(protocol = securityProtocol) + try { + sendSaslHandshakeRequestValidateResponse(plaintextSocket) + try { + sendApiVersionsRequest(plaintextSocket, new ApiVersionsRequest, 0) + fail("Versions Request during Sasl handshake did not fail") + } catch { + case ioe: IOException => // expected exception + } + } finally { + plaintextSocket.close() + } + } + + private def sendApiVersionsRequest(socket: Socket, request: ApiVersionsRequest, version: Short): ApiVersionsResponse = { + val response = send(socket, request, ApiKeys.API_VERSIONS, version) + ApiVersionsResponse.parse(response) + } + + private def sendSaslHandshakeRequestValidateResponse(socket: Socket) { + val response = send(socket, new SaslHandshakeRequest("PLAIN"), ApiKeys.SASL_HANDSHAKE, 0) + val handshakeResponse = SaslHandshakeResponse.parse(response) + assertEquals(Errors.NONE.code, handshakeResponse.errorCode()) + assertEquals(Collections.singletonList("PLAIN"), handshakeResponse.enabledMechanisms()) + } +} From 3414d56121d8d2f66f8dd613453af71d5b3f0c5f Mon Sep 17 00:00:00 2001 From: "Matthias J. Sax" Date: Fri, 29 Apr 2016 12:50:02 -0700 Subject: [PATCH 200/206] KAFKA-3440: Update streams javadocs - add class doc for KTable, KStream, JoinWindows - add missing return tags Author: Matthias J. Sax Reviewers: Guozhang Wang , Michael G. Noll , Ewen Cheslack-Postava Closes #1287 from mjsax/kafka-3440-JavaDoc --- .../kafka/streams/kstream/JoinWindows.java | 18 ++++ .../kafka/streams/kstream/KGroupedTable.java | 14 ++- .../apache/kafka/streams/kstream/KStream.java | 95 ++++++++++++++++++- .../kafka/streams/kstream/KStreamBuilder.java | 14 ++- .../apache/kafka/streams/kstream/KTable.java | 47 ++++++++- .../kafka/streams/processor/StateStore.java | 4 +- .../kafka/streams/state/WindowStore.java | 2 + 7 files changed, 189 insertions(+), 5 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java index a6d5603f6dd3a..f45c0640dff56 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/JoinWindows.java @@ -23,6 +23,24 @@ /** * The window specifications used for joins. + *

          + * A {@link JoinWindows} instance defines a join over two stream on the same key and a maximum time difference. + * In SQL-style you would express this join as + *

          + *     SELECT * FROM stream1, stream2
          + *     WHERE
          + *       stream1.key = stream2.key
          + *       AND
          + *       stream2.ts - before <= stream1.ts <= stream2.ts + after
          + * 
          + * There are three different window configuration supported: + *
            + *
          • before = after = time-difference
          • + *
          • before = 0 and after = time-difference
          • + *
          • before = time-difference and after = 0
          • + *
          + * A join is symmetric in the sense, that a join specification on the first stream returns the same result record as + * a join specification on the second stream with flipped before and after values. */ public class JoinWindows extends Windows { diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java index 86c34b16bda69..2ebad87635402 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KGroupedTable.java @@ -21,7 +21,11 @@ import org.apache.kafka.common.serialization.Serde; /** - * {@link KGroupedTable} is an abstraction of a grouped changelog stream from a primary-keyed table. + * {@link KGroupedTable} is an abstraction of a grouped changelog stream from a primary-keyed table, + * usually on a different grouping key than the original primary key. + *

          + * It is an intermediate representation after a re-grouping of a {@link KTable} before an aggregation is applied + * to the new partitions resulting in a new {@link KTable}. * * @param Type of primary keys * @param Type of value changes @@ -35,6 +39,8 @@ public interface KGroupedTable { * @param adder the instance of {@link Reducer} for addition * @param subtractor the instance of {@link Reducer} for subtraction * @param name the name of the resulted {@link KTable} + * @return a {@link KTable} with the same key and value types as this {@link KGroupedTable}, + * containing aggregated values for each key */ KTable reduce(Reducer adder, Reducer subtractor, @@ -50,6 +56,8 @@ KTable reduce(Reducer adder, * if not specified the default serdes defined in the configs will be used * @param name the name of the resulted table * @param the value type of the aggregated {@link KTable} + * @return a {@link KTable} with same key and aggregated value type {@code T}, + * containing aggregated values for each key */ KTable aggregate(Initializer initializer, Aggregator adder, @@ -66,6 +74,8 @@ KTable aggregate(Initializer initializer, * @param substractor the instance of {@link Aggregator} for subtraction * @param name the name of the resulted {@link KTable} * @param the value type of the aggregated {@link KTable} + * @return a {@link KTable} with same key and aggregated value type {@code T}, + * containing aggregated values for each key */ KTable aggregate(Initializer initializer, Aggregator adder, @@ -76,6 +86,8 @@ KTable aggregate(Initializer initializer, * Count number of records of this stream by the selected key into a new instance of {@link KTable}. * * @param name the name of the resulted {@link KTable} + * @return a {@link KTable} with same key and {@link Long} value type as this {@link KGroupedTable}, + * containing the number of values for each key */ KTable count(String name); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java index 6df2deb6e71af..a1ecfa44ac250 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStream.java @@ -25,9 +25,17 @@ /** * {@link KStream} is an abstraction of a record stream of key-value pairs. + *

          + * A {@link KStream} is either defined from one or multiple Kafka topics that are consumed message by message or + * the result of a {@link KStream} transformation. A {@link KTable} can also be converted into a {@link KStream}. + *

          + * A {@link KStream} can be transformed record by record, joined with another {@link KStream} or {@link KTable}, or + * can be aggregated into a {@link KTable}. * * @param Type of keys * @param Type of values + * + * @see KTable */ @InterfaceStability.Unstable public interface KStream { @@ -36,6 +44,8 @@ public interface KStream { * Create a new instance of {@link KStream} that consists of all elements of this stream which satisfy a predicate. * * @param predicate the instance of {@link Predicate} + * + * @return a {@link KStream} that contains only those records that satisfy the given predicate */ KStream filter(Predicate predicate); @@ -43,6 +53,8 @@ public interface KStream { * Create a new instance of {@link KStream} that consists all elements of this stream which do not satisfy a predicate. * * @param predicate the instance of {@link Predicate} + * + * @return a {@link KStream} that contains only those records that do not satisfy the given predicate */ KStream filterNot(Predicate predicate); @@ -52,6 +64,8 @@ public interface KStream { * * @param mapper the instance of {@link KeyValueMapper} * @param the new key type on the stream + * + * @return a {@link KStream} that contains records with different key type and same value type */ KStream selectKey(KeyValueMapper mapper); @@ -61,6 +75,8 @@ public interface KStream { * @param mapper the instance of {@link KeyValueMapper} * @param the key type of the new stream * @param the value type of the new stream + * + * @return a {@link KStream} that contains records with new key and value type */ KStream map(KeyValueMapper> mapper); @@ -69,6 +85,8 @@ public interface KStream { * * @param mapper the instance of {@link ValueMapper} * @param the value type of the new stream + * + * @return a {@link KStream} that contains records with unmodified keys and new values of different type */ KStream mapValues(ValueMapper mapper); @@ -124,6 +142,8 @@ public interface KStream { * @param mapper the instance of {@link KeyValueMapper} * @param the key type of the new stream * @param the value type of the new stream + * + * @return a {@link KStream} that contains more or less records with new key and value type */ KStream flatMap(KeyValueMapper>> mapper); @@ -132,6 +152,8 @@ public interface KStream { * * @param processor the instance of {@link ValueMapper} * @param the value type of the new stream + * + * @return a {@link KStream} that contains more or less records with unmodified keys and new values of different type */ KStream flatMapValues(ValueMapper> processor); @@ -143,6 +165,8 @@ public interface KStream { * assigned to this stream only. An element will be dropped if none of the predicates evaluate to true. * * @param predicates the ordered list of {@link Predicate} instances + * + * @return multiple distinct substreams of this {@link KStream} */ KStream[] branch(Predicate... predicates); @@ -152,6 +176,8 @@ public interface KStream { * This is equivalent to calling {@link #to(String)} and {@link org.apache.kafka.streams.kstream.KStreamBuilder#stream(String...)}. * * @param topic the topic name + * + * @return a {@link KStream} that contains the exact same records as this {@link KStream} */ KStream through(String topic); @@ -159,7 +185,7 @@ public interface KStream { * Perform an action on each element of {@link KStream}. * Note that this is a terminal operation that returns void. * - * @param action An action to perform on each element + * @param action an action to perform on each element */ void foreach(ForeachAction action); @@ -171,6 +197,8 @@ public interface KStream { * @param partitioner the function used to determine how records are distributed among partitions of the topic, * if not specified producer's {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} will be used * @param topic the topic name + * + * @return a {@link KStream} that contains the exact same records as this {@link KStream} */ KStream through(StreamPartitioner partitioner, String topic); @@ -187,6 +215,8 @@ public interface KStream { * @param valSerde value serde used to send key-value pairs, * if not specified the default value serde defined in the configuration will be used * @param topic the topic name + * + * @return a {@link KStream} that contains the exact same records as this {@link KStream} */ KStream through(Serde keySerde, Serde valSerde, String topic); @@ -205,6 +235,8 @@ public interface KStream { * {@link org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner} will be used * — otherwise {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} will be used * @param topic the topic name + * + * @return a {@link KStream} that contains the exact same records as this {@link KStream} */ KStream through(Serde keySerde, Serde valSerde, StreamPartitioner partitioner, String topic); @@ -260,6 +292,8 @@ public interface KStream { * * @param transformerSupplier the instance of {@link TransformerSupplier} that generates {@link org.apache.kafka.streams.kstream.Transformer} * @param stateStoreNames the names of the state store used by the processor + * + * @return a new {@link KStream} with transformed key and value types */ KStream transform(TransformerSupplier> transformerSupplier, String... stateStoreNames); @@ -268,6 +302,8 @@ public interface KStream { * * @param valueTransformerSupplier the instance of {@link ValueTransformerSupplier} that generates {@link org.apache.kafka.streams.kstream.ValueTransformer} * @param stateStoreNames the names of the state store used by the processor + * + * @return a {@link KStream} that contains records with unmodified keys and transformed values with type {@code R} */ KStream transformValues(ValueTransformerSupplier valueTransformerSupplier, String... stateStoreNames); @@ -293,6 +329,9 @@ public interface KStream { * if not specified the default serdes defined in the configs will be used * @param the value type of the other stream * @param the value type of the new stream + * + * @return a {@link KStream} that contains join-records for each key and values computed by the given {@link ValueJoiner}, + * one for each matched record-pair with the same key and within the joining window intervals */ KStream join( KStream otherStream, @@ -311,6 +350,9 @@ KStream join( * @param windows the specification of the {@link JoinWindows} * @param the value type of the other stream * @param the value type of the new stream + * + * @return a {@link KStream} that contains join-records for each key and values computed by the given {@link ValueJoiner}, + * one for each matched record-pair with the same key and within the joining window intervals */ KStream join( KStream otherStream, @@ -331,6 +373,9 @@ KStream join( * if not specified the default serdes defined in the configs will be used * @param the value type of the other stream * @param the value type of the new stream + * + * @return a {@link KStream} that contains join-records for each key and values computed by the given {@link ValueJoiner}, + * one for each matched record-pair with the same key and within the joining window intervals */ KStream outerJoin( KStream otherStream, @@ -349,6 +394,9 @@ KStream outerJoin( * @param windows the specification of the {@link JoinWindows} * @param the value type of the other stream * @param the value type of the new stream + * + * @return a {@link KStream} that contains join-records for each key and values computed by the given {@link ValueJoiner}, + * one for each matched record-pair with the same key and within the joining window intervals */ KStream outerJoin( KStream otherStream, @@ -367,6 +415,9 @@ KStream outerJoin( * if not specified the default serdes defined in the configs will be used * @param the value type of the other stream * @param the value type of the new stream + * + * @return a {@link KStream} that contains join-records for each key and values computed by the given {@link ValueJoiner}, + * one for each matched record-pair with the same key and within the joining window intervals */ KStream leftJoin( KStream otherStream, @@ -384,6 +435,9 @@ KStream leftJoin( * @param windows the specification of the {@link JoinWindows} * @param the value type of the other stream * @param the value type of the new stream + * + * @return a {@link KStream} that contains join-records for each key and values computed by the given {@link ValueJoiner}, + * one for each matched record-pair with the same key and within the joining window intervals */ KStream leftJoin( KStream otherStream, @@ -397,6 +451,9 @@ KStream leftJoin( * @param joiner the instance of {@link ValueJoiner} * @param the value type of the table * @param the value type of the new stream + * + * @return a {@link KStream} that contains join-records for each key and values computed by the given {@link ValueJoiner}, + * one for each matched record-pair with the same key and within the joining window intervals */ KStream leftJoin(KTable table, ValueJoiner joiner); @@ -409,6 +466,10 @@ KStream leftJoin( * if not specified the default serdes defined in the configs will be used * @param valueSerde value serdes for materializing the aggregated table, * if not specified the default serdes defined in the configs will be used + * + * @return a windowed {@link KTable} which can be treated as a list of {@code KTable}s + * where each table contains records with unmodified keys and values + * that represent the latest (rolling) aggregate for each key within that window */ KTable, V> reduceByKey(Reducer reducer, Windows windows, @@ -421,6 +482,10 @@ KTable, V> reduceByKey(Reducer reducer, * * @param reducer the instance of {@link Reducer} * @param windows the specification of the aggregation {@link Windows} + * + * @return a windowed {@link KTable} which can be treated as a list of {@code KTable}s + * where each table contains records with unmodified keys and values + * that represent the latest (rolling) aggregate for each key within that window */ KTable, V> reduceByKey(Reducer reducer, Windows windows); @@ -433,6 +498,8 @@ KTable, V> reduceByKey(Reducer reducer, * @param valueSerde value serdes for materializing the aggregated table, * if not specified the default serdes defined in the configs will be used * @param name the name of the resulted {@link KTable} + * + * @return a {@link KTable} that contains records with unmodified keys and values that represent the latest (rolling) aggregate for each key */ KTable reduceByKey(Reducer reducer, Serde keySerde, @@ -444,6 +511,8 @@ KTable reduceByKey(Reducer reducer, * * @param reducer the instance of {@link Reducer} * @param name the name of the resulted {@link KTable} + * + * @return a {@link KTable} that contains records with unmodified keys and values that represent the latest (rolling) aggregate for each key */ KTable reduceByKey(Reducer reducer, String name); @@ -458,6 +527,10 @@ KTable reduceByKey(Reducer reducer, * @param aggValueSerde aggregate value serdes for materializing the aggregated table, * if not specified the default serdes defined in the configs will be used * @param the value type of the resulted {@link KTable} + * + * @return a windowed {@link KTable} which can be treated as a list of {@code KTable}s + * where each table contains records with unmodified keys and values with type {@code T} + * that represent the latest (rolling) aggregate for each key within that window */ KTable, T> aggregateByKey(Initializer initializer, Aggregator aggregator, @@ -473,6 +546,10 @@ KTable, T> aggregateByKey(Initializer initi * @param aggregator the instance of {@link Aggregator} * @param windows the specification of the aggregation {@link Windows} * @param the value type of the resulted {@link KTable} + * + * @return a windowed {@link KTable} which can be treated as a list of {@code KTable}s + * where each table contains records with unmodified keys and values with type {@code T} + * that represent the latest (rolling) aggregate for each key within that window */ KTable, T> aggregateByKey(Initializer initializer, Aggregator aggregator, @@ -489,6 +566,8 @@ KTable, T> aggregateByKey(Initializer initi * if not specified the default serdes defined in the configs will be used * @param name the name of the resulted {@link KTable} * @param the value type of the resulted {@link KTable} + * + * @return a {@link KTable} that contains records with unmodified keys and values (of different type) that represent the latest (rolling) aggregate for each key */ KTable aggregateByKey(Initializer initializer, Aggregator aggregator, @@ -504,6 +583,8 @@ KTable aggregateByKey(Initializer initializer, * @param aggregator the class of {@link Aggregator} * @param name the name of the resulted {@link KTable} * @param the value type of the resulted {@link KTable} + * + * @return a {@link KTable} that contains records with unmodified keys and values (of different type) that represent the latest (rolling) aggregate for each key */ KTable aggregateByKey(Initializer initializer, Aggregator aggregator, @@ -515,6 +596,10 @@ KTable aggregateByKey(Initializer initializer, * @param windows the specification of the aggregation {@link Windows} * @param keySerde key serdes for materializing the counting table, * if not specified the default serdes defined in the configs will be used + * + * @return a windowed {@link KTable} which can be treated as a list of {@code KTable}s + * where each table contains records with unmodified keys and values + * that represent the latest (rolling) count (i.e., number of records) for each key within that window */ KTable, Long> countByKey(Windows windows, Serde keySerde); @@ -523,6 +608,10 @@ KTable aggregateByKey(Initializer initializer, * with default serializers and deserializers. * * @param windows the specification of the aggregation {@link Windows} + * + * @return a windowed {@link KTable} which can be treated as a list of {@code KTable}s + * where each table contains records with unmodified keys and values + * that represent the latest (rolling) count (i.e., number of records) for each key within that window */ KTable, Long> countByKey(Windows windows); @@ -532,6 +621,8 @@ KTable aggregateByKey(Initializer initializer, * @param keySerde key serdes for materializing the counting table, * if not specified the default serdes defined in the configs will be used * @param name the name of the resulted {@link KTable} + * + * @return a {@link KTable} that contains records with unmodified keys and values that represent the latest (rolling) count (i.e., number of records) for each key */ KTable countByKey(Serde keySerde, String name); @@ -540,6 +631,8 @@ KTable aggregateByKey(Initializer initializer, * with default serializers and deserializers. * * @param name the name of the resulted {@link KTable} + * + * @return a {@link KTable} that contains records with unmodified keys and values that represent the latest (rolling) count (i.e., number of records) for each key */ KTable countByKey(String name); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java index 159876c4ab91e..9d90ba053ccee 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KStreamBuilder.java @@ -45,21 +45,27 @@ public KStreamBuilder() { /** * Create a {@link KStream} instance from the specified topics. * The default deserializers specified in the config are used. + *

          + * If multiple topics are specified there are nor ordering guaranteed for records from different topics. * * @param topics the topic names; must contain at least one topic name + * @return a {@link KStream} for the specified topics */ public KStream stream(String... topics) { return stream(null, null, topics); } /** - * Create a {@link KStream} instance for the specified topics. + * Create a {@link KStream} instance from the specified topics. + *

          + * If multiple topics are specified there are nor ordering guaranteed for records from different topics. * * @param keySerde key serde used to read this source {@link KStream}, * if not specified the default serde defined in the configs will be used * @param valSerde value serde used to read this source {@link KStream}, * if not specified the default serde defined in the configs will be used * @param topics the topic names; must contain at least one topic name + * @return a {@link KStream} for the specified topics */ public KStream stream(Serde keySerde, Serde valSerde, String... topics) { String name = newName(KStreamImpl.SOURCE_NAME); @@ -74,6 +80,7 @@ public KStream stream(Serde keySerde, Serde valSerde, String. * The default deserializers specified in the config are used. * * @param topic the topic name; cannot be null + * @return a {@link KTable} for the specified topics */ public KTable table(String topic) { return table(null, null, topic); @@ -87,6 +94,7 @@ public KTable table(String topic) { * @param valSerde value serde used to send key-value pairs, * if not specified the default value serde defined in the configuration will be used * @param topic the topic name; cannot be null + * @return a {@link KTable} for the specified topics */ public KTable table(Serde keySerde, Serde valSerde, String topic) { String source = newName(KStreamImpl.SOURCE_NAME); @@ -102,8 +110,11 @@ public KTable table(Serde keySerde, Serde valSerde, String to /** * Create a new instance of {@link KStream} by merging the given streams. + *

          + * There are nor ordering guaranteed for records from different streams. * * @param streams the instances of {@link KStream} to be merged + * @return a {@link KStream} containing all records of the given streams */ public KStream merge(KStream... streams) { return KStreamImpl.merge(this, streams); @@ -114,6 +125,7 @@ public KStream merge(KStream... streams) { * This function is only for internal usage. * * @param prefix processor name prefix + * @return a new unique name */ public String newName(String prefix) { return prefix + String.format("%010d", index.getAndIncrement()); diff --git a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java index 4ff9b481a9e70..cc5a52180ab17 100644 --- a/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java +++ b/streams/src/main/java/org/apache/kafka/streams/kstream/KTable.java @@ -24,9 +24,18 @@ /** * {@link KTable} is an abstraction of a changelog stream from a primary-keyed table. + * Each record in this stream is an update on the primary-keyed table with the record key as the primary key. + *

          + * A {@link KTable} is either defined from one or multiple Kafka topics that are consumed message by message or + * the result of a {@link KTable} transformation. An aggregation of a {@link KStream} also yields a {@link KTable}. + *

          + * A {@link KTable} can be transformed record by record, joined with another {@link KTable} or {@link KStream}, or + * can be re-partitioned and aggregated into a new {@link KTable}. * * @param Type of primary keys * @param Type of value changes + * + * @see KStream */ @InterfaceStability.Unstable public interface KTable { @@ -35,6 +44,8 @@ public interface KTable { * Create a new instance of {@link KTable} that consists of all elements of this stream which satisfy a predicate. * * @param predicate the instance of {@link Predicate} + * + * @return a {@link KTable} that contains only those records that satisfy the given predicate */ KTable filter(Predicate predicate); @@ -42,6 +53,8 @@ public interface KTable { * Create a new instance of {@link KTable} that consists all elements of this stream which do not satisfy a predicate. * * @param predicate the instance of {@link Predicate} + * + * @return a {@link KTable} that contains only those records that do not satisfy the given predicate */ KTable filterNot(Predicate predicate); @@ -50,6 +63,8 @@ public interface KTable { * * @param mapper the instance of {@link ValueMapper} * @param the value type of the new stream + * + * @return a {@link KTable} that contains records with unmodified keys and new values of different type */ KTable mapValues(ValueMapper mapper); @@ -103,6 +118,8 @@ public interface KTable { * This is equivalent to calling {@link #to(String)} and {@link org.apache.kafka.streams.kstream.KStreamBuilder#table(String)}. * * @param topic the topic name + * + * @return a new {@link KTable} that contains the exact same records as this {@link KTable} */ KTable through(String topic); @@ -114,6 +131,8 @@ public interface KTable { * @param partitioner the function used to determine how records are distributed among partitions of the topic, * if not specified producer's {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} will be used * @param topic the topic name + * + * @return a new {@link KTable} that contains the exact same records as this {@link KTable} */ KTable through(StreamPartitioner partitioner, String topic); @@ -130,6 +149,8 @@ public interface KTable { * @param valSerde value serde used to send key-value pairs, * if not specified the default value serde defined in the configuration will be used * @param topic the topic name + * + * @return a new {@link KTable} that contains the exact same records as this {@link KTable} */ KTable through(Serde keySerde, Serde valSerde, String topic); @@ -148,6 +169,8 @@ public interface KTable { * {@link org.apache.kafka.streams.kstream.internals.WindowedStreamPartitioner} will be used * — otherwise {@link org.apache.kafka.clients.producer.internals.DefaultPartitioner} will be used * @param topic the topic name + * + * @return a new {@link KTable} that contains the exact same records as this {@link KTable} */ KTable through(Serde keySerde, Serde valSerde, StreamPartitioner partitioner, String topic); @@ -200,6 +223,10 @@ public interface KTable { /** * Convert this stream to a new instance of {@link KStream}. + * + * @return a {@link KStream} that contains the same records as this {@link KTable}; + * the records are no longer treated as updates on a primary-keyed table, + * but rather as normal key-value pairs in a record stream */ KStream toStream(); @@ -209,6 +236,11 @@ public interface KTable { * * @param mapper @param mapper the instance of {@link KeyValueMapper} * @param the new key type + * + * @return a {@link KStream} that contains records with new keys of different type for each update of this {@link KTable} + * @return a {@link KStream} that contains the transformed records from this {@link KTable}; + * the records are no longer treated as updates on a primary-keyed table, + * but rather as normal key-value pairs in a record stream */ KStream toStream(KeyValueMapper mapper); @@ -219,6 +251,9 @@ public interface KTable { * @param joiner the instance of {@link ValueJoiner} * @param the value type of the other stream * @param the value type of the new stream + * + * @return a {@link KTable} that contains join-records for each key and values computed by the given {@link ValueJoiner}, + * one for each matched record-pair with the same key */ KTable join(KTable other, ValueJoiner joiner); @@ -229,6 +264,9 @@ public interface KTable { * @param joiner the instance of {@link ValueJoiner} * @param the value type of the other stream * @param the value type of the new stream + * + * @return a {@link KTable} that contains join-records for each key and values computed by the given {@link ValueJoiner}, + * one for each matched record-pair with the same key */ KTable outerJoin(KTable other, ValueJoiner joiner); @@ -239,6 +277,9 @@ public interface KTable { * @param joiner the instance of {@link ValueJoiner} * @param the value type of the other stream * @param the value type of the new stream + * + * @return a {@link KTable} that contains join-records for each key and values computed by the given {@link ValueJoiner}, + * one for each matched record-pair with the same key */ KTable leftJoin(KTable other, ValueJoiner joiner); @@ -252,6 +293,8 @@ public interface KTable { * if not specified the default serdes defined in the configs will be used * @param the key type of the {@link KGroupedTable} * @param the value type of the {@link KGroupedTable} + * + * @return a {@link KGroupedTable} that contains the re-partitioned records of this {@link KTable} */ KGroupedTable groupBy(KeyValueMapper> selector, Serde keySerde, Serde valueSerde); @@ -261,6 +304,8 @@ public interface KTable { * @param selector select the grouping key and value to be aggregated * @param the key type of the {@link KGroupedTable} * @param the value type of the {@link KGroupedTable} + * + * @return a {@link KGroupedTable} that contains the re-partitioned records of this {@link KTable} */ KGroupedTable groupBy(KeyValueMapper> selector); @@ -268,7 +313,7 @@ public interface KTable { * Perform an action on each element of {@link KTable}. * Note that this is a terminal operation that returns void. * - * @param action An action to perform on each element + * @param action an action to perform on each element */ void foreach(ForeachAction action); } diff --git a/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java b/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java index b07e51081f5e5..f79e6f68f5ba7 100644 --- a/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/processor/StateStore.java @@ -51,7 +51,9 @@ public interface StateStore { void close(); /** - * If the storage is persistent + * Return if the storage is persistent or not. + * + * @return {@code true} if the storage is persistent—{@code false} otherwise */ boolean persistent(); } diff --git a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java index e400cefa53494..079a2b2f9b9df 100644 --- a/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java +++ b/streams/src/main/java/org/apache/kafka/streams/state/WindowStore.java @@ -45,6 +45,8 @@ public interface WindowStore extends StateStore { /** * Get all the key-value pairs with the given key and the time range from all * the existing windows. + * + * @return an iterator over key-value pairs {@code } */ WindowStoreIterator fetch(K key, long timeFrom, long timeTo); } From 096b8b84e0dba8cdf0e0f7afae8d118a68e06e6c Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Fri, 29 Apr 2016 14:48:25 -0700 Subject: [PATCH 201/206] KAFKA-3641; Fix RecordMetadata constructor backward compatibility Author: Grant Henke Reviewers: Gwen Shapira, Ismael Juma Closes #1292 from granthenke/recordmeta-compat --- .../org/apache/kafka/clients/producer/RecordMetadata.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java b/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java index 988da16addf97..00a4374ef5543 100644 --- a/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java +++ b/clients/src/main/java/org/apache/kafka/clients/producer/RecordMetadata.java @@ -17,6 +17,7 @@ package org.apache.kafka.clients.producer; import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.record.Record; /** * The metadata for a record that has been acknowledged by the server @@ -51,6 +52,11 @@ private RecordMetadata(TopicPartition topicPartition, long offset, long timestam this.topicPartition = topicPartition; } + @Deprecated + public RecordMetadata(TopicPartition topicPartition, long baseOffset, long relativeOffset) { + this(topicPartition, baseOffset, relativeOffset, Record.NO_TIMESTAMP, -1, -1, -1); + } + public RecordMetadata(TopicPartition topicPartition, long baseOffset, long relativeOffset, long timestamp, long checksum, int serializedKeySize, int serializedValueSize) { // ignore the relativeOffset if the base offset is -1, From d0dedc6314bfd83d9b2b9a9557e3168e715981da Mon Sep 17 00:00:00 2001 From: Liquan Pei Date: Fri, 29 Apr 2016 14:49:22 -0700 Subject: [PATCH 202/206] KAFKA-3459: Returning zero task configurations from a connector does not properly clean up existing tasks hachikuji ewencp Can you take a look when you have time? Author: Liquan Pei Reviewers: Jason Gustafson , Ewen Cheslack-Postava Closes #1248 from Ishiihara/kafka-3459 --- .../distributed/ClusterConfigState.java | 13 +- .../distributed/DistributedHerder.java | 15 +- .../runtime/standalone/StandaloneHerder.java | 16 +-- .../connect/storage/ConfigBackingStore.java | 5 +- .../storage/KafkaConfigBackingStore.java | 82 +++++------ .../storage/MemoryConfigBackingStore.java | 18 ++- .../storage/KafkaConfigBackingStoreTest.java | 131 +++++++++++++++--- 7 files changed, 174 insertions(+), 106 deletions(-) diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ClusterConfigState.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ClusterConfigState.java index c5c217e6f4731..ea5ba82d4ba08 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ClusterConfigState.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/ClusterConfigState.java @@ -22,10 +22,11 @@ import java.util.ArrayList; import java.util.Collections; -import java.util.HashMap; +import java.util.LinkedList; import java.util.List; import java.util.Map; import java.util.Set; +import java.util.TreeMap; /** * An immutable snapshot of the configuration state of connectors and tasks in a Kafka Connect cluster. @@ -116,15 +117,15 @@ public Map taskConfig(ConnectorTaskId task) { /** * Get all task configs for a connector. * @param connector name of the connector - * @return a map from the task id to its configuration + * @return a list of task configurations */ - public Map> allTaskConfigs(String connector) { - Map> taskConfigs = new HashMap<>(); + public List> allTaskConfigs(String connector) { + Map> taskConfigs = new TreeMap<>(); for (Map.Entry> taskConfigEntry : this.taskConfigs.entrySet()) { if (taskConfigEntry.getKey().connector().equals(connector)) - taskConfigs.put(taskConfigEntry.getKey(), taskConfigEntry.getValue()); + taskConfigs.put(taskConfigEntry.getKey().task(), taskConfigEntry.getValue()); } - return taskConfigs; + return new LinkedList<>(taskConfigs.values()); } /** diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java index cbef1867744b4..037eba742816f 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/distributed/DistributedHerder.java @@ -44,7 +44,6 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; @@ -543,7 +542,7 @@ public Void call() throws Exception { else if (!configState.contains(connName)) callback.onCompletion(new NotFoundException("Connector " + connName + " not found"), null); else { - configBackingStore.putTaskConfigs(connName, taskConfigListAsMap(connName, configs)); + configBackingStore.putTaskConfigs(connName, configs); callback.onCompletion(null, null); } return null; @@ -853,7 +852,7 @@ private void reconfigureConnector(final String connName, final Callback cb } if (changed) { if (isLeader()) { - configBackingStore.putTaskConfigs(connName, taskConfigListAsMap(connName, taskProps)); + configBackingStore.putTaskConfigs(connName, taskProps); cb.onCompletion(null, null); } else { // We cannot forward the request on the same thread because this reconfiguration can happen as a result of connector @@ -1064,14 +1063,4 @@ public void onRevoked(String leader, Collection connectors, Collection> taskConfigListAsMap(String connName, List> configs) { - int index = 0; - Map> result = new HashMap<>(); - for (Map taskConfigMap : configs) { - ConnectorTaskId taskId = new ConnectorTaskId(connName, index); - result.put(taskId, taskConfigMap); - index++; - } - return result; - } } diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java index ad02e99587c12..2316baefb3654 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/runtime/standalone/StandaloneHerder.java @@ -40,7 +40,6 @@ import java.util.ArrayList; import java.util.Collection; -import java.util.HashMap; import java.util.List; import java.util.Map; @@ -250,20 +249,13 @@ private String startConnector(Map connectorProps) { return connName; } - private Map> recomputeTaskConfigs(String connName) { + private List> recomputeTaskConfigs(String connName) { Map config = configState.connectorConfig(connName); ConnectorConfig connConfig = new ConnectorConfig(config); - List> taskConfigs = worker.connectorTaskConfigs(connName, + return worker.connectorTaskConfigs(connName, connConfig.getInt(ConnectorConfig.TASKS_MAX_CONFIG), connConfig.getList(ConnectorConfig.TOPICS_CONFIG)); - - int i = 0; - Map> taskConfigMap = new HashMap<>(); - for (Map taskConfig : taskConfigs) - taskConfigMap.put(new ConnectorTaskId(connName, i++), taskConfig); - - return taskConfigMap; } private void createConnectorTasks(String connName, TargetState initialState) { @@ -296,8 +288,8 @@ private void updateConnectorTasks(String connName) { return; } - Map> newTaskConfigs = recomputeTaskConfigs(connName); - Map> oldTaskConfigs = configState.allTaskConfigs(connName); + List> newTaskConfigs = recomputeTaskConfigs(connName); + List> oldTaskConfigs = configState.allTaskConfigs(connName); if (!newTaskConfigs.equals(oldTaskConfigs)) { removeConnectorTasks(connName); diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConfigBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConfigBackingStore.java index 5244842a2c984..77fc43b680c19 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConfigBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/ConfigBackingStore.java @@ -22,6 +22,7 @@ import org.apache.kafka.connect.util.ConnectorTaskId; import java.util.Collection; +import java.util.List; import java.util.Map; import java.util.concurrent.TimeUnit; import java.util.concurrent.TimeoutException; @@ -64,9 +65,9 @@ public interface ConfigBackingStore { /** * Update the task configurations for a connector. * @param connector name of the connector - * @param configs the new task configs + * @param configs the new task configs for the connector */ - void putTaskConfigs(String connector, Map> configs); + void putTaskConfigs(String connector, List> configs); /** * Remove the task configs associated with a connector. diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java index 8d2028852ecf8..9412e42629dc8 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/KafkaConfigBackingStore.java @@ -205,6 +205,7 @@ public static String COMMIT_TASKS_KEY(String connectorName) { // Connector and task configs: name or id -> config map private Map> connectorConfigs = new HashMap<>(); private Map> taskConfigs = new HashMap<>(); + // Set of connectors where we saw a task commit with an incomplete set of task config updates, indicating the data // is in an inconsistent state and we cannot safely use them until they have been refreshed. private Set inconsistent = new HashSet<>(); @@ -339,12 +340,13 @@ private void updateConnectorConfig(String connector, byte[] serializedConfig) { * Write these task configurations and associated commit messages, unless an inconsistency is found that indicates * that we would be leaving one of the referenced connectors with an inconsistent state. * - * @param configs map containing task configurations + * @param connector the connector to write task configuration + * @param configs list of task configurations for the connector * @throws ConnectException if the task configurations do not resolve inconsistencies found in the existing root * and task configurations. */ @Override - public void putTaskConfigs(String connector, Map> configs) { + public void putTaskConfigs(String connector, List> configs) { // Make sure we're at the end of the log. We should be the only writer, but we want to make sure we don't have // any outstanding lagging data to consume. try { @@ -354,46 +356,33 @@ public void putTaskConfigs(String connector, Map newTaskCounts = new HashMap<>(); - synchronized (lock) { - // Validate tasks in this assignment. Any task configuration updates should include updates for *all* tasks - // in the connector -- we should have all task IDs 0 - N-1 within a connector if any task is included here - Map> updatedConfigIdsByConnector = taskIdsByConnector(configs); - for (Map.Entry> taskConfigSetEntry : updatedConfigIdsByConnector.entrySet()) { - if (!completeTaskIdSet(taskConfigSetEntry.getValue(), taskConfigSetEntry.getValue().size())) { - log.error("Submitted task configuration contain invalid range of task IDs, ignoring this submission"); - throw new ConnectException("Error writing task configurations: found some connectors with invalid connectors"); - } - newTaskCounts.put(taskConfigSetEntry.getKey(), taskConfigSetEntry.getValue().size()); - } - } + int taskCount = configs.size(); // Start sending all the individual updates - for (Map.Entry> taskConfigEntry : configs.entrySet()) { + int index = 0; + for (Map taskConfig: configs) { Struct connectConfig = new Struct(TASK_CONFIGURATION_V0); - connectConfig.put("properties", taskConfigEntry.getValue()); + connectConfig.put("properties", taskConfig); byte[] serializedConfig = converter.fromConnectData(topic, TASK_CONFIGURATION_V0, connectConfig); - log.debug("Writing configuration for task " + taskConfigEntry.getKey() + " configuration: " + taskConfigEntry.getValue()); - configLog.send(TASK_KEY(taskConfigEntry.getKey()), serializedConfig); + log.debug("Writing configuration for task " + index + " configuration: " + taskConfig); + ConnectorTaskId connectorTaskId = new ConnectorTaskId(connector, index); + configLog.send(TASK_KEY(connectorTaskId), serializedConfig); + index++; } // Finally, send the commit to update the number of tasks and apply the new configs, then wait until we read to // the end of the log try { // Read to end to ensure all the task configs have been written - configLog.readToEnd().get(READ_TO_END_TIMEOUT_MS, TimeUnit.MILLISECONDS); - - // Write all the commit messages - for (Map.Entry taskCountEntry : newTaskCounts.entrySet()) { - Struct connectConfig = new Struct(CONNECTOR_TASKS_COMMIT_V0); - connectConfig.put("tasks", taskCountEntry.getValue()); - byte[] serializedConfig = converter.fromConnectData(topic, CONNECTOR_TASKS_COMMIT_V0, connectConfig); - log.debug("Writing commit for connector " + taskCountEntry.getKey() + " with " + taskCountEntry.getValue() + " tasks."); - configLog.send(COMMIT_TASKS_KEY(taskCountEntry.getKey()), serializedConfig); + if (taskCount > 0) { + configLog.readToEnd().get(READ_TO_END_TIMEOUT_MS, TimeUnit.MILLISECONDS); } + // Write the commit message + Struct connectConfig = new Struct(CONNECTOR_TASKS_COMMIT_V0); + connectConfig.put("tasks", taskCount); + byte[] serializedConfig = converter.fromConnectData(topic, CONNECTOR_TASKS_COMMIT_V0, connectConfig); + log.debug("Writing commit for connector " + connector + " with " + taskCount + " tasks."); + configLog.send(COMMIT_TASKS_KEY(connector), serializedConfig); // Read to end to ensure all the commit messages have been written configLog.readToEnd().get(READ_TO_END_TIMEOUT_MS, TimeUnit.MILLISECONDS); @@ -426,6 +415,7 @@ private KafkaBasedLog createKafkaBasedLog(String topic, Map(topic, producerProps, consumerProps, consumedCallback, new SystemTime()); } + @SuppressWarnings("unchecked") private class ConsumeCallback implements Callback> { @Override public void onCompletion(Throwable error, ConsumerRecord record) { @@ -562,20 +552,13 @@ public void onCompletion(Throwable error, ConsumerRecord record) log.error("Ignoring connector tasks configuration commit for connector " + connectorName + " because it is in the wrong format: " + value.value()); return; } - Map> deferred = deferredTaskUpdates.get(connectorName); int newTaskCount = intValue(((Map) value.value()).get("tasks")); // Validate the configs we're supposed to update to ensure we're getting a complete configuration // update of all tasks that are expected based on the number of tasks in the commit message. - Map> updatedConfigIdsByConnector = taskIdsByConnector(deferred); - Set taskIdSet = updatedConfigIdsByConnector.get(connectorName); - if (taskIdSet == null) { - //TODO: Figure out why this happens (KAFKA-3321) - log.error("Received a commit message for connector " + connectorName + " but there is no matching configuration for tasks in this connector. This should never happen."); - return; - } + Set taskIdSet = taskIds(connectorName, deferred); if (!completeTaskIdSet(taskIdSet, newTaskCount)) { // Given the logic for writing commit messages, we should only hit this condition due to compacted // historical data, in which case we would not have applied any updates yet and there will be no @@ -622,19 +605,18 @@ private ConnectorTaskId parseTaskId(String key) { } /** - * Given task configurations, get a set of integer task IDs organized by connector name. + * Given task configurations, get a set of integer task IDs for the connector. */ - private Map> taskIdsByConnector(Map> configs) { - Map> connectorTaskIds = new HashMap<>(); - if (configs == null) - return connectorTaskIds; - for (Map.Entry> taskConfigEntry : configs.entrySet()) { - ConnectorTaskId taskId = taskConfigEntry.getKey(); - if (!connectorTaskIds.containsKey(taskId.connector())) - connectorTaskIds.put(taskId.connector(), new TreeSet()); - connectorTaskIds.get(taskId.connector()).add(taskId.task()); + private Set taskIds(String connector, Map> configs) { + Set tasks = new TreeSet<>(); + if (configs == null) { + return tasks; + } + for (ConnectorTaskId taskId : configs.keySet()) { + assert taskId.connector().equals(connector); + tasks.add(taskId.task()); } - return connectorTaskIds; + return tasks; } private boolean completeTaskIdSet(Set idSet, int expectedSize) { diff --git a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java index ec5f2e6c47522..212022dd0fc7a 100644 --- a/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java +++ b/connect/runtime/src/main/java/org/apache/kafka/connect/storage/MemoryConfigBackingStore.java @@ -24,7 +24,9 @@ import java.util.Collections; import java.util.HashMap; import java.util.HashSet; +import java.util.List; import java.util.Map; +import java.util.TreeMap; import java.util.concurrent.TimeUnit; public class MemoryConfigBackingStore implements ConfigBackingStore { @@ -108,15 +110,16 @@ public synchronized void removeTaskConfigs(String connector) { } @Override - public synchronized void putTaskConfigs(String connector, Map> configs) { + public synchronized void putTaskConfigs(String connector, List> configs) { ConnectorState state = connectors.get(connector); if (state == null) throw new IllegalArgumentException("Cannot put tasks for non-existing connector"); - state.taskConfigs = configs; + Map> taskConfigsMap = taskConfigListAsMap(connector, configs); + state.taskConfigs = taskConfigsMap; if (updateListener != null) - updateListener.onTaskConfigUpdate(configs.keySet()); + updateListener.onTaskConfigUpdate(taskConfigsMap.keySet()); } @Override @@ -151,4 +154,13 @@ public ConnectorState(Map connConfig) { this.taskConfigs = new HashMap<>(); } } + + private static Map> taskConfigListAsMap(String connector, List> configs) { + int index = 0; + Map> result = new TreeMap<>(); + for (Map taskConfigMap: configs) { + result.put(new ConnectorTaskId(connector, index++), taskConfigMap); + } + return result; + } } diff --git a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java index eaad34bd72269..617177e1a4f16 100644 --- a/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java +++ b/connect/runtime/src/test/java/org/apache/kafka/connect/storage/KafkaConfigBackingStoreTest.java @@ -21,7 +21,6 @@ import org.apache.kafka.clients.consumer.ConsumerConfig; import org.apache.kafka.clients.consumer.ConsumerRecord; import org.apache.kafka.clients.producer.ProducerConfig; -import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.record.TimestampType; import org.apache.kafka.connect.data.Field; import org.apache.kafka.connect.data.Schema; @@ -57,7 +56,6 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertNull; -import static org.junit.Assert.fail; @RunWith(PowerMockRunner.class) @PrepareForTest(KafkaConfigBackingStore.class) @@ -113,6 +111,9 @@ public class KafkaConfigBackingStoreTest { private static final Struct TASKS_COMMIT_STRUCT_TWO_TASK_CONNECTOR = new Struct(KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0).put("tasks", 2); + private static final Struct TASKS_COMMIT_STRUCT_ZERO_TASK_CONNECTOR + = new Struct(KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0).put("tasks", 0); + // The exact format doesn't matter here since both conversions are mocked private static final List CONFIGS_SERIALIZED = Arrays.asList( "config-bytes-1".getBytes(), "config-bytes-2".getBytes(), "config-bytes-3".getBytes(), @@ -275,9 +276,7 @@ public void testPutTaskConfigs() throws Exception { // Writing task task configs should block until all the writes have been performed and the root record update // has completed - Map> taskConfigs = new HashMap<>(); - taskConfigs.put(TASK_IDS.get(0), SAMPLE_CONFIGS.get(0)); - taskConfigs.put(TASK_IDS.get(1), SAMPLE_CONFIGS.get(1)); + List> taskConfigs = Arrays.asList(SAMPLE_CONFIGS.get(0), SAMPLE_CONFIGS.get(1)); configStorage.putTaskConfigs("connector1", taskConfigs); // Validate root config by listing all connectors and tasks @@ -295,6 +294,57 @@ public void testPutTaskConfigs() throws Exception { PowerMock.verifyAll(); } + @Test + public void testPutTaskConfigsZeroTasks() throws Exception { + expectConfigure(); + expectStart(Collections.EMPTY_LIST, Collections.EMPTY_MAP); + + // Task configs should read to end, write to the log, read to end, write root. + expectReadToEnd(new LinkedHashMap()); + expectConvertWriteRead( + COMMIT_TASKS_CONFIG_KEYS.get(0), KafkaConfigBackingStore.CONNECTOR_TASKS_COMMIT_V0, CONFIGS_SERIALIZED.get(0), + "tasks", 0); // We have 0 tasks + // As soon as root is rewritten, we should see a callback notifying us that we reconfigured some tasks + configUpdateListener.onTaskConfigUpdate(Collections.emptyList()); + EasyMock.expectLastCall(); + + // Records to be read by consumer as it reads to the end of the log + LinkedHashMap serializedConfigs = new LinkedHashMap<>(); + serializedConfigs.put(COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)); + expectReadToEnd(serializedConfigs); + + expectStop(); + + PowerMock.replayAll(); + + configStorage.configure(DEFAULT_DISTRIBUTED_CONFIG); + configStorage.start(); + + // Bootstrap as if we had already added the connector, but no tasks had been added yet + whiteboxAddConnector(CONNECTOR_IDS.get(0), SAMPLE_CONFIGS.get(0), Collections.EMPTY_LIST); + + // Null before writing + ClusterConfigState configState = configStorage.snapshot(); + assertEquals(-1, configState.offset()); + + // Writing task task configs should block until all the writes have been performed and the root record update + // has completed + List> taskConfigs = Collections.emptyList(); + configStorage.putTaskConfigs("connector1", taskConfigs); + + // Validate root config by listing all connectors and tasks + configState = configStorage.snapshot(); + assertEquals(1, configState.offset()); + String connectorName = CONNECTOR_IDS.get(0); + assertEquals(Arrays.asList(connectorName), new ArrayList<>(configState.connectors())); + assertEquals(Collections.emptyList(), configState.tasks(connectorName)); + assertEquals(Collections.EMPTY_SET, configState.inconsistentConnectors()); + + configStorage.stop(); + + PowerMock.verifyAll(); + } + @Test public void testRestore() throws Exception { // Restoring data should notify only of the latest values after loading is complete. This also validates @@ -349,11 +399,64 @@ public void testRestore() throws Exception { PowerMock.verifyAll(); } + @Test + public void testRestoreZeroTasks() throws Exception { + // Restoring data should notify only of the latest values after loading is complete. This also validates + // that inconsistent state is ignored. + expectConfigure(); + // Overwrite each type at least once to ensure we see the latest data after loading + List> existingRecords = Arrays.asList( + new ConsumerRecord<>(TOPIC, 0, 0, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(0)), + new ConsumerRecord<>(TOPIC, 0, 1, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(1)), + new ConsumerRecord<>(TOPIC, 0, 2, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(1), CONFIGS_SERIALIZED.get(2)), + new ConsumerRecord<>(TOPIC, 0, 3, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(3)), + new ConsumerRecord<>(TOPIC, 0, 4, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(4)), + // Connector after root update should make it through, task update shouldn't + new ConsumerRecord<>(TOPIC, 0, 5, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, CONNECTOR_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(5)), + new ConsumerRecord<>(TOPIC, 0, 6, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, TASK_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(6)), + new ConsumerRecord<>(TOPIC, 0, 7, 0L, TimestampType.CREATE_TIME, 0L, 0, 0, COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(7))); + LinkedHashMap deserialized = new LinkedHashMap(); + deserialized.put(CONFIGS_SERIALIZED.get(0), CONNECTOR_CONFIG_STRUCTS.get(0)); + deserialized.put(CONFIGS_SERIALIZED.get(1), TASK_CONFIG_STRUCTS.get(0)); + deserialized.put(CONFIGS_SERIALIZED.get(2), TASK_CONFIG_STRUCTS.get(0)); + deserialized.put(CONFIGS_SERIALIZED.get(3), CONNECTOR_CONFIG_STRUCTS.get(1)); + deserialized.put(CONFIGS_SERIALIZED.get(4), TASKS_COMMIT_STRUCT_TWO_TASK_CONNECTOR); + deserialized.put(CONFIGS_SERIALIZED.get(5), CONNECTOR_CONFIG_STRUCTS.get(2)); + deserialized.put(CONFIGS_SERIALIZED.get(6), TASK_CONFIG_STRUCTS.get(1)); + deserialized.put(CONFIGS_SERIALIZED.get(7), TASKS_COMMIT_STRUCT_ZERO_TASK_CONNECTOR); + logOffset = 8; + expectStart(existingRecords, deserialized); + + // Shouldn't see any callbacks since this is during startup + + expectStop(); + + PowerMock.replayAll(); + + configStorage.configure(DEFAULT_DISTRIBUTED_CONFIG); + configStorage.start(); + + // Should see a single connector and its config should be the last one seen anywhere in the log + ClusterConfigState configState = configStorage.snapshot(); + assertEquals(8, configState.offset()); // Should always be next to be read, even if uncommitted + assertEquals(Arrays.asList(CONNECTOR_IDS.get(0)), new ArrayList<>(configState.connectors())); + // CONNECTOR_CONFIG_STRUCTS[2] -> SAMPLE_CONFIGS[2] + assertEquals(SAMPLE_CONFIGS.get(2), configState.connectorConfig(CONNECTOR_IDS.get(0))); + // Should see 0 tasks for that connector. + assertEquals(Collections.emptyList(), configState.tasks(CONNECTOR_IDS.get(0))); + // Both TASK_CONFIG_STRUCTS[0] -> SAMPLE_CONFIGS[0] + assertEquals(Collections.EMPTY_SET, configState.inconsistentConnectors()); + + configStorage.stop(); + + PowerMock.verifyAll(); + } + @Test public void testPutTaskConfigsDoesNotResolveAllInconsistencies() throws Exception { // Test a case where a failure and compaction has left us in an inconsistent state when reading the log. - // We start out by loading an initial configuration where we started to write a task update and failed before - // writing an the commit, and then compaction cleaned up the earlier record. + // We start out by loading an initial configuration where we started to write a task update, and then + // compaction cleaned up the earlier record. expectConfigure(); List> existingRecords = Arrays.asList( @@ -371,9 +474,6 @@ public void testPutTaskConfigsDoesNotResolveAllInconsistencies() throws Exceptio logOffset = 6; expectStart(existingRecords, deserialized); - // One failed attempt to write new task configs - expectReadToEnd(new LinkedHashMap()); - // Successful attempt to write new task config expectReadToEnd(new LinkedHashMap()); expectConvertWriteRead( @@ -392,7 +492,6 @@ public void testPutTaskConfigsDoesNotResolveAllInconsistencies() throws Exceptio serializedConfigs.put(COMMIT_TASKS_CONFIG_KEYS.get(0), CONFIGS_SERIALIZED.get(2)); expectReadToEnd(serializedConfigs); - expectStop(); PowerMock.replayAll(); @@ -410,17 +509,9 @@ public void testPutTaskConfigsDoesNotResolveAllInconsistencies() throws Exceptio assertNull(configState.taskConfig(TASK_IDS.get(1))); assertEquals(Collections.singleton(CONNECTOR_IDS.get(0)), configState.inconsistentConnectors()); - // First try sending an invalid set of configs (can't possibly represent a valid config set for the tasks) - try { - configStorage.putTaskConfigs("connector1", Collections.singletonMap(TASK_IDS.get(1), SAMPLE_CONFIGS.get(2))); - fail("Should have failed due to incomplete task set."); - } catch (KafkaException e) { - // expected - } - // Next, issue a write that has everything that is needed and it should be accepted. Note that in this case // we are going to shrink the number of tasks to 1 - configStorage.putTaskConfigs("connector1", Collections.singletonMap(TASK_IDS.get(0), SAMPLE_CONFIGS.get(0))); + configStorage.putTaskConfigs("connector1", Collections.singletonList(SAMPLE_CONFIGS.get(0))); // Validate updated config configState = configStorage.snapshot(); // This is only two more ahead of the last one because multiple calls fail, and so their configs are not written From 60380e31d4bf6688d8d26ec44cf514a3c32731cb Mon Sep 17 00:00:00 2001 From: Eno Thereska Date: Fri, 29 Apr 2016 15:14:36 -0700 Subject: [PATCH 203/206] HOTFIX: Fix equality semantics of KeyValue Fixes wrong KeyValue equals logic when keys not equal but values equal. Original hotfix PR at https://github.com/apache/kafka/pull/1293 (/cc enothereska) Please review: ewencp ijuma guozhangwang Author: Eno Thereska Author: Michael G. Noll Reviewers: Michael G. Noll , Ewen Cheslack-Postava Closes #1294 from miguno/KeyValue-equality-hotfix --- .../org/apache/kafka/streams/KeyValue.java | 16 ++--- .../apache/kafka/streams/KeyValueTest.java | 65 +++++++++++++------ 2 files changed, 52 insertions(+), 29 deletions(-) diff --git a/streams/src/main/java/org/apache/kafka/streams/KeyValue.java b/streams/src/main/java/org/apache/kafka/streams/KeyValue.java index 58f2083b8457b..64b38cdfbf36c 100644 --- a/streams/src/main/java/org/apache/kafka/streams/KeyValue.java +++ b/streams/src/main/java/org/apache/kafka/streams/KeyValue.java @@ -63,22 +63,22 @@ public String toString() { } @Override - public boolean equals(Object other) { - if (this == other) + public boolean equals(Object obj) { + if (this == obj) return true; - if (other instanceof KeyValue) { - KeyValue otherKV = (KeyValue) other; - - return key == null ? otherKV.key == null : key.equals(otherKV.key) - && value == null ? otherKV.value == null : value.equals(otherKV.value); - } else { + if (!(obj instanceof KeyValue)) { return false; } + + KeyValue other = (KeyValue) obj; + return (this.key == null ? other.key == null : this.key.equals(other.key)) + && (this.value == null ? other.value == null : this.value.equals(other.value)); } @Override public int hashCode() { return Objects.hash(key, value); } + } diff --git a/streams/src/test/java/org/apache/kafka/streams/KeyValueTest.java b/streams/src/test/java/org/apache/kafka/streams/KeyValueTest.java index 47c8ecd1c2280..805fa18369e62 100644 --- a/streams/src/test/java/org/apache/kafka/streams/KeyValueTest.java +++ b/streams/src/test/java/org/apache/kafka/streams/KeyValueTest.java @@ -24,27 +24,50 @@ public class KeyValueTest { - private KeyValue kv1a = new KeyValue<>("key1", 1L); - private KeyValue kv1b = new KeyValue<>("key1", 1L); - private KeyValue kv2 = new KeyValue<>("key2", 2L); - private KeyValue kv3 = new KeyValue<>("key3", 3L); - @Test - public void testEquals() { - assertTrue(kv1a.equals(kv1a)); - assertTrue(kv1a.equals(kv1b)); - assertTrue(kv1b.equals(kv1a)); - assertFalse(kv1a.equals(kv2)); - assertFalse(kv1a.equals(kv3)); - assertFalse(kv2.equals(kv3)); - assertFalse(kv1a.equals(null)); - } + public void shouldHaveSaneEqualsAndHashCode() { + KeyValue kv = KeyValue.pair("key1", 1L); + KeyValue copyOfKV = KeyValue.pair(kv.key, kv.value); - @Test - public void testHashcode() { - assertTrue(kv1a.hashCode() == kv1b.hashCode()); - assertFalse(kv1a.hashCode() == kv2.hashCode()); - assertFalse(kv1a.hashCode() == kv3.hashCode()); - assertFalse(kv2.hashCode() == kv3.hashCode()); + // Reflexive + assertTrue(kv.equals(kv)); + assertTrue(kv.hashCode() == kv.hashCode()); + + // Symmetric + assertTrue(kv.equals(copyOfKV)); + assertTrue(kv.hashCode() == copyOfKV.hashCode()); + assertTrue(copyOfKV.hashCode() == kv.hashCode()); + + // Transitive + KeyValue copyOfCopyOfKV = KeyValue.pair(copyOfKV.key, copyOfKV.value); + assertTrue(copyOfKV.equals(copyOfCopyOfKV)); + assertTrue(copyOfKV.hashCode() == copyOfCopyOfKV.hashCode()); + assertTrue(kv.equals(copyOfCopyOfKV)); + assertTrue(kv.hashCode() == copyOfCopyOfKV.hashCode()); + + // Inequality scenarios + assertFalse("must be false for null", kv.equals(null)); + assertFalse("must be false if key is non-null and other key is null", kv.equals(KeyValue.pair(null, kv.value))); + assertFalse("must be false if value is non-null and other value is null", kv.equals(KeyValue.pair(kv.key, null))); + KeyValue differentKeyType = KeyValue.pair(1L, kv.value); + assertFalse("must be false for different key types", kv.equals(differentKeyType)); + KeyValue differentValueType = KeyValue.pair(kv.key, "anyString"); + assertFalse("must be false for different value types", kv.equals(differentValueType)); + KeyValue differentKeyValueTypes = KeyValue.pair(1L, "anyString"); + assertFalse("must be false for different key and value types", kv.equals(differentKeyValueTypes)); + assertFalse("must be false for different types of objects", kv.equals(new Object())); + + KeyValue differentKey = KeyValue.pair(kv.key + "suffix", kv.value); + assertFalse("must be false if key is different", kv.equals(differentKey)); + assertFalse("must be false if key is different", differentKey.equals(kv)); + + KeyValue differentValue = KeyValue.pair(kv.key, kv.value + 1L); + assertFalse("must be false if value is different", kv.equals(differentValue)); + assertFalse("must be false if value is different", differentValue.equals(kv)); + + KeyValue differentKeyAndValue = KeyValue.pair(kv.key + "suffix", kv.value + 1L); + assertFalse("must be false if key and value are different", kv.equals(differentKeyAndValue)); + assertFalse("must be false if key and value are different", differentKeyAndValue.equals(kv)); } -} + +} \ No newline at end of file From 94d1878a9b7aaf57c7520fb5bdcd2f05aeec5853 Mon Sep 17 00:00:00 2001 From: Zack Dever Date: Fri, 29 Apr 2016 15:49:48 -0700 Subject: [PATCH 204/206] MINOR: change initial value of Min stat to Double.MAX_VALUE (not MIN) this is consistent with the Max stat implementation. Author: Zack Dever Reviewers: Aditya Auradkar, Gwen Shapira Closes #1143 from zackdever/min-stat-fix --- .../kafka/common/metrics/stats/Min.java | 8 ++--- .../kafka/common/metrics/MetricsTest.java | 32 +++++++++++++++++++ 2 files changed, 36 insertions(+), 4 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java index 9f74417193a04..113d745c3b185 100644 --- a/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java +++ b/clients/src/main/java/org/apache/kafka/common/metrics/stats/Min.java @@ -22,7 +22,7 @@ public class Min extends SampledStat { public Min() { - super(Double.MIN_VALUE); + super(Double.MAX_VALUE); } @Override @@ -32,10 +32,10 @@ protected void update(Sample sample, MetricConfig config, double value, long now @Override public double combine(List samples, MetricConfig config, long now) { - double max = Double.MAX_VALUE; + double min = Double.MAX_VALUE; for (int i = 0; i < samples.size(); i++) - max = Math.min(max, samples.get(i).value); - return max; + min = Math.min(min, samples.get(i).value); + return min; } } diff --git a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java index f5b49ba55ca90..52f0cd7e04bec 100644 --- a/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java +++ b/clients/src/test/java/org/apache/kafka/common/metrics/MetricsTest.java @@ -316,6 +316,38 @@ public void testOldDataHasNoEffect() { assertEquals(Double.NEGATIVE_INFINITY, max.measure(config, time.milliseconds()), EPS); } + @Test + public void testSampledStatInitialValue() { + // initialValue from each SampledStat is set as the initialValue on its Sample. + // The only way to test the initialValue is to infer it by having a SampledStat + // with expired Stats, because their values are reset to the initial values. + // Most implementations of combine on SampledStat end up returning the default + // value, so we can use this. This doesn't work for Percentiles though. + // This test looks a lot like testOldDataHasNoEffect because it's the same + // flow that leads to this state. + Max max = new Max(); + Min min = new Min(); + Avg avg = new Avg(); + Count count = new Count(); + Rate.SampledTotal sampledTotal = new Rate.SampledTotal(); + + long windowMs = 100; + int samples = 2; + MetricConfig config = new MetricConfig().timeWindow(windowMs, TimeUnit.MILLISECONDS).samples(samples); + max.record(config, 50, time.milliseconds()); + min.record(config, 50, time.milliseconds()); + avg.record(config, 50, time.milliseconds()); + count.record(config, 50, time.milliseconds()); + sampledTotal.record(config, 50, time.milliseconds()); + time.sleep(samples * windowMs); + + assertEquals(Double.NEGATIVE_INFINITY, max.measure(config, time.milliseconds()), EPS); + assertEquals(Double.MAX_VALUE, min.measure(config, time.milliseconds()), EPS); + assertEquals(0.0, avg.measure(config, time.milliseconds()), EPS); + assertEquals(0, count.measure(config, time.milliseconds()), EPS); + assertEquals(0.0, sampledTotal.measure(config, time.milliseconds()), EPS); + } + @Test(expected = IllegalArgumentException.class) public void testDuplicateMetricName() { metrics.sensor("test").add(metrics.metricName("test", "grp1"), new Avg()); From 9eaf529ffb168e44a461be24b666eee00fdaf497 Mon Sep 17 00:00:00 2001 From: Grant Henke Date: Fri, 29 Apr 2016 17:27:02 -0700 Subject: [PATCH 205/206] =?UTF-8?q?KAFKA-3644;=20Use=20Boolean=20protocol?= =?UTF-8?q?=20type=20for=20StopReplicaRequest=20delete=5Fp=E2=80=A6?= MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit …artitions Author: Grant Henke Reviewers: Gwen Shapira Closes #1296 from granthenke/stop-boolean --- .../org/apache/kafka/common/protocol/Protocol.java | 6 ++---- .../kafka/common/requests/StopReplicaRequest.java | 4 ++-- .../kafka/common/requests/RequestResponseTest.java | 11 ++++++----- 3 files changed, 10 insertions(+), 11 deletions(-) diff --git a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java index 99cdbf9f57988..3644f9ce079e3 100644 --- a/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java +++ b/clients/src/main/java/org/apache/kafka/common/protocol/Protocol.java @@ -31,7 +31,6 @@ 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.INT8; import static org.apache.kafka.common.protocol.types.Type.STRING; import static org.apache.kafka.common.protocol.types.Type.NULLABLE_STRING; @@ -671,8 +670,7 @@ public class Protocol { public static final Schema STOP_REPLICA_REQUEST_V0 = new Schema(new Field("controller_id", INT32, "The controller id."), new Field("controller_epoch", INT32, "The controller epoch."), - new Field("delete_partitions", - INT8, + new Field("delete_partitions", BOOLEAN, "Boolean which indicates if replica's partitions must be deleted."), new Field("partitions", new ArrayOf(STOP_REPLICA_REQUEST_PARTITION_V0))); @@ -1008,4 +1006,4 @@ public static void main(String[] args) { System.out.println(toHtml()); } -} \ No newline at end of file +} diff --git a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java index 80223a17bc196..bc63521bd8274 100644 --- a/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java +++ b/clients/src/main/java/org/apache/kafka/common/requests/StopReplicaRequest.java @@ -49,7 +49,7 @@ public StopReplicaRequest(int controllerId, int controllerEpoch, boolean deleteP struct.set(CONTROLLER_ID_KEY_NAME, controllerId); struct.set(CONTROLLER_EPOCH_KEY_NAME, controllerEpoch); - struct.set(DELETE_PARTITIONS_KEY_NAME, deletePartitions ? (byte) 1 : (byte) 0); + struct.set(DELETE_PARTITIONS_KEY_NAME, deletePartitions); List partitionDatas = new ArrayList<>(partitions.size()); for (TopicPartition partition : partitions) { @@ -80,7 +80,7 @@ public StopReplicaRequest(Struct struct) { controllerId = struct.getInt(CONTROLLER_ID_KEY_NAME); controllerEpoch = struct.getInt(CONTROLLER_EPOCH_KEY_NAME); - deletePartitions = ((byte) struct.get(DELETE_PARTITIONS_KEY_NAME)) != 0; + deletePartitions = struct.getBoolean(DELETE_PARTITIONS_KEY_NAME); } @Override diff --git a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java index 345de3f957ae0..043582d44b922 100644 --- a/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java +++ b/clients/src/test/java/org/apache/kafka/common/requests/RequestResponseTest.java @@ -83,8 +83,9 @@ public void testSerialization() throws Exception { createProduceRequest(), createProduceRequest().getErrorResponse(2, new UnknownServerException()), createProduceResponse(), - createStopReplicaRequest(), - createStopReplicaRequest().getErrorResponse(0, new UnknownServerException()), + createStopReplicaRequest(true), + createStopReplicaRequest(false), + createStopReplicaRequest(true).getErrorResponse(0, new UnknownServerException()), createStopReplicaResponse(), createUpdateMetadataRequest(2, "rack1"), createUpdateMetadataRequest(2, null), @@ -348,9 +349,9 @@ private AbstractRequestResponse createProduceResponse() { return new ProduceResponse(responseData, 0); } - private AbstractRequest createStopReplicaRequest() { + private AbstractRequest createStopReplicaRequest(boolean deletePartitions) { Set partitions = new HashSet<>(Arrays.asList(new TopicPartition("test", 0))); - return new StopReplicaRequest(0, 1, true, partitions); + return new StopReplicaRequest(0, 1, deletePartitions, partitions); } private AbstractRequestResponse createStopReplicaResponse() { @@ -450,4 +451,4 @@ private AbstractRequestResponse createApiVersionResponse() { List apiVersions = Arrays.asList(new ApiVersionsResponse.ApiVersion((short) 0, (short) 0, (short) 2)); return new ApiVersionsResponse(Errors.NONE.code(), apiVersions); } -} \ No newline at end of file +} From 1e4dd66b19857f4f0ce3f83fd0a808885b0a88c1 Mon Sep 17 00:00:00 2001 From: Dana Powers Date: Sat, 30 Apr 2016 18:53:44 -0700 Subject: [PATCH 206/206] HOTFIX: Fix main classpath libs glob for release (fixup KAFKA-3615 regression) bin/kafka-run-class.sh does not correctly setup the CLASSPATH in release rc2. Author: Dana Powers Reviewers: Ewen Cheslack-Postava Closes #1302 from dpkp/KAFKA-3615-fix --- bin/kafka-run-class.sh | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/bin/kafka-run-class.sh b/bin/kafka-run-class.sh index e7f8d2ef9fec3..688cced286145 100755 --- a/bin/kafka-run-class.sh +++ b/bin/kafka-run-class.sh @@ -118,7 +118,7 @@ do done # classpath addition for release -for file in $base_dir/libs; +for file in $base_dir/libs/*; do if should_include_file "$file"; then CLASSPATH=$CLASSPATH:$file