http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafka.java ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafka.java new file mode 100644 index 0000000..c7d1a60 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublishKafka.java @@ -0,0 +1,262 @@ +/* + * 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.nifi.processors.kafka.pubsub; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Matchers.eq; +import static org.mockito.Mockito.mock; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; +import static org.mockito.Mockito.when; + +import java.io.IOException; +import java.io.InputStream; +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; +import java.util.Set; +import java.util.stream.Collectors; + +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.processor.ProcessContext; +import org.apache.nifi.util.MockFlowFile; +import org.apache.nifi.util.TestRunner; +import org.apache.nifi.util.TestRunners; +import org.junit.Before; +import org.junit.Test; + +public class TestPublishKafka { + private static final String TOPIC_NAME = "unit-test"; + + private PublisherPool mockPool; + private PublisherLease mockLease; + private TestRunner runner; + + @Before + public void setup() { + mockPool = mock(PublisherPool.class); + mockLease = mock(PublisherLease.class); + + when(mockPool.obtainPublisher()).thenReturn(mockLease); + + runner = TestRunners.newTestRunner(new PublishKafka_0_10() { + @Override + protected PublisherPool createPublisherPool(final ProcessContext context) { + return mockPool; + } + }); + + runner.setProperty(PublishKafka_0_10.TOPIC, TOPIC_NAME); + } + + @Test + public void testSingleSuccess() throws IOException { + final MockFlowFile flowFile = runner.enqueue("hello world"); + + when(mockLease.complete()).thenReturn(createAllSuccessPublishResult(flowFile, 1)); + + runner.run(); + runner.assertAllFlowFilesTransferred(PublishKafka_0_10.REL_SUCCESS, 1); + + verify(mockLease, times(1)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME)); + verify(mockLease, times(1)).complete(); + verify(mockLease, times(0)).poison(); + verify(mockLease, times(1)).close(); + } + + @Test + public void testMultipleSuccess() throws IOException { + final Set<FlowFile> flowFiles = new HashSet<>(); + flowFiles.add(runner.enqueue("hello world")); + flowFiles.add(runner.enqueue("hello world")); + flowFiles.add(runner.enqueue("hello world")); + + + when(mockLease.complete()).thenReturn(createAllSuccessPublishResult(flowFiles, 1)); + + runner.run(); + runner.assertAllFlowFilesTransferred(PublishKafka_0_10.REL_SUCCESS, 3); + + verify(mockLease, times(3)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME)); + verify(mockLease, times(1)).complete(); + verify(mockLease, times(0)).poison(); + verify(mockLease, times(1)).close(); + } + + @Test + public void testSingleFailure() throws IOException { + final MockFlowFile flowFile = runner.enqueue("hello world"); + + when(mockLease.complete()).thenReturn(createFailurePublishResult(flowFile)); + + runner.run(); + runner.assertAllFlowFilesTransferred(PublishKafka_0_10.REL_FAILURE, 1); + + verify(mockLease, times(1)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME)); + verify(mockLease, times(1)).complete(); + verify(mockLease, times(1)).close(); + } + + @Test + public void testMultipleFailures() throws IOException { + final Set<FlowFile> flowFiles = new HashSet<>(); + flowFiles.add(runner.enqueue("hello world")); + flowFiles.add(runner.enqueue("hello world")); + flowFiles.add(runner.enqueue("hello world")); + + when(mockLease.complete()).thenReturn(createFailurePublishResult(flowFiles)); + + runner.run(); + runner.assertAllFlowFilesTransferred(PublishKafka_0_10.REL_FAILURE, 3); + + verify(mockLease, times(3)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME)); + verify(mockLease, times(1)).complete(); + verify(mockLease, times(1)).close(); + } + + @Test + public void testMultipleMessagesPerFlowFile() throws IOException { + final List<FlowFile> flowFiles = new ArrayList<>(); + flowFiles.add(runner.enqueue("hello world")); + flowFiles.add(runner.enqueue("hello world")); + + final Map<FlowFile, Integer> msgCounts = new HashMap<>(); + msgCounts.put(flowFiles.get(0), 10); + msgCounts.put(flowFiles.get(1), 20); + + final PublishResult result = createPublishResult(msgCounts, new HashSet<>(flowFiles), Collections.emptyMap()); + + when(mockLease.complete()).thenReturn(result); + + runner.run(); + runner.assertAllFlowFilesTransferred(PublishKafka_0_10.REL_SUCCESS, 2); + + verify(mockLease, times(2)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME)); + verify(mockLease, times(1)).complete(); + verify(mockLease, times(0)).poison(); + verify(mockLease, times(1)).close(); + + runner.assertAllFlowFilesContainAttribute("msg.count"); + assertEquals(1, runner.getFlowFilesForRelationship(PublishKafka_0_10.REL_SUCCESS).stream() + .filter(ff -> ff.getAttribute("msg.count").equals("10")) + .count()); + assertEquals(1, runner.getFlowFilesForRelationship(PublishKafka_0_10.REL_SUCCESS).stream() + .filter(ff -> ff.getAttribute("msg.count").equals("20")) + .count()); + } + + + @Test + public void testSomeSuccessSomeFailure() throws IOException { + final List<FlowFile> flowFiles = new ArrayList<>(); + flowFiles.add(runner.enqueue("hello world")); + flowFiles.add(runner.enqueue("hello world")); + flowFiles.add(runner.enqueue("hello world")); + flowFiles.add(runner.enqueue("hello world")); + + final Map<FlowFile, Integer> msgCounts = new HashMap<>(); + msgCounts.put(flowFiles.get(0), 10); + msgCounts.put(flowFiles.get(1), 20); + + final Map<FlowFile, Exception> failureMap = new HashMap<>(); + failureMap.put(flowFiles.get(2), new RuntimeException("Intentional Unit Test Exception")); + failureMap.put(flowFiles.get(3), new RuntimeException("Intentional Unit Test Exception")); + + final PublishResult result = createPublishResult(msgCounts, new HashSet<>(flowFiles.subList(0, 2)), failureMap); + + when(mockLease.complete()).thenReturn(result); + + runner.run(); + runner.assertTransferCount(PublishKafka_0_10.REL_SUCCESS, 2); + runner.assertTransferCount(PublishKafka_0_10.REL_FAILURE, 2); + + verify(mockLease, times(4)).publish(any(FlowFile.class), any(InputStream.class), eq(null), eq(null), eq(TOPIC_NAME)); + verify(mockLease, times(1)).complete(); + verify(mockLease, times(1)).close(); + + assertEquals(1, runner.getFlowFilesForRelationship(PublishKafka_0_10.REL_SUCCESS).stream() + .filter(ff -> "10".equals(ff.getAttribute("msg.count"))) + .count()); + assertEquals(1, runner.getFlowFilesForRelationship(PublishKafka_0_10.REL_SUCCESS).stream() + .filter(ff -> "20".equals(ff.getAttribute("msg.count"))) + .count()); + + assertTrue(runner.getFlowFilesForRelationship(PublishKafka_0_10.REL_FAILURE).stream() + .noneMatch(ff -> ff.getAttribute("msg.count") != null)); + } + + + private PublishResult createAllSuccessPublishResult(final FlowFile successfulFlowFile, final int msgCount) { + return createAllSuccessPublishResult(Collections.singleton(successfulFlowFile), msgCount); + } + + private PublishResult createAllSuccessPublishResult(final Set<FlowFile> successfulFlowFiles, final int msgCountPerFlowFile) { + final Map<FlowFile, Integer> msgCounts = new HashMap<>(); + for (final FlowFile ff : successfulFlowFiles) { + msgCounts.put(ff, msgCountPerFlowFile); + } + return createPublishResult(msgCounts, successfulFlowFiles, Collections.emptyMap()); + } + + private PublishResult createFailurePublishResult(final FlowFile failure) { + return createFailurePublishResult(Collections.singleton(failure)); + } + + private PublishResult createFailurePublishResult(final Set<FlowFile> failures) { + final Map<FlowFile, Exception> failureMap = failures.stream().collect(Collectors.toMap(ff -> ff, ff -> new RuntimeException("Intentional Unit Test Exception"))); + return createPublishResult(Collections.emptyMap(), Collections.emptySet(), failureMap); + } + + private PublishResult createPublishResult(final Map<FlowFile, Integer> msgCounts, final Set<FlowFile> successFlowFiles, final Map<FlowFile, Exception> failures) { + // sanity check. + for (final FlowFile success : successFlowFiles) { + if (failures.containsKey(success)) { + throw new IllegalArgumentException("Found same FlowFile in both 'success' and 'failures' collections: " + success); + } + } + + return new PublishResult() { + @Override + public Collection<FlowFile> getSuccessfulFlowFiles() { + return successFlowFiles; + } + + @Override + public Collection<FlowFile> getFailedFlowFiles() { + return failures.keySet(); + } + + @Override + public int getSuccessfulMessageCount(FlowFile flowFile) { + Integer count = msgCounts.get(flowFile); + return count == null ? 0 : count.intValue(); + } + + @Override + public Exception getReasonForFailure(FlowFile flowFile) { + return failures.get(flowFile); + } + }; + } +}
http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherLease.java ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherLease.java new file mode 100644 index 0000000..c2d143c --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherLease.java @@ -0,0 +1,194 @@ +/* + * 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.nifi.processors.kafka.pubsub; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.any; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.times; +import static org.mockito.Mockito.verify; + +import java.io.ByteArrayInputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.charset.StandardCharsets; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.kafka.clients.producer.Callback; +import org.apache.kafka.clients.producer.Producer; +import org.apache.kafka.clients.producer.ProducerRecord; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.util.MockFlowFile; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.invocation.InvocationOnMock; +import org.mockito.stubbing.Answer; + + +public class TestPublisherLease { + private ComponentLog logger; + private Producer<byte[], byte[]> producer; + + @Before + @SuppressWarnings("unchecked") + public void setup() { + logger = Mockito.mock(ComponentLog.class); + producer = Mockito.mock(Producer.class); + } + + @Test + public void testPoisonOnException() throws IOException { + final AtomicInteger poisonCount = new AtomicInteger(0); + + final PublisherLease lease = new PublisherLease(producer, 1024 * 1024, 1000L, logger) { + @Override + public void poison() { + poisonCount.incrementAndGet(); + super.poison(); + } + }; + + final FlowFile flowFile = new MockFlowFile(1L); + final String topic = "unit-test"; + final byte[] messageKey = null; + final byte[] demarcatorBytes = null; + + final InputStream failureInputStream = new InputStream() { + @Override + public int read() throws IOException { + throw new IOException("Intentional Unit Test Exception"); + } + }; + + try { + lease.publish(flowFile, failureInputStream, messageKey, demarcatorBytes, topic); + Assert.fail("Expected IOException"); + } catch (final IOException ioe) { + // expected + } + + assertEquals(1, poisonCount.get()); + + final PublishResult result = lease.complete(); + assertTrue(result.getFailedFlowFiles().contains(flowFile)); + assertFalse(result.getSuccessfulFlowFiles().contains(flowFile)); + } + + @Test + @SuppressWarnings("unchecked") + public void testPoisonOnFailure() throws IOException { + final AtomicInteger poisonCount = new AtomicInteger(0); + + final PublisherLease lease = new PublisherLease(producer, 1024 * 1024, 1000L, logger) { + @Override + public void poison() { + poisonCount.incrementAndGet(); + super.poison(); + } + }; + + final FlowFile flowFile = new MockFlowFile(1L); + final String topic = "unit-test"; + final byte[] messageKey = null; + final byte[] demarcatorBytes = null; + + doAnswer(new Answer<Object>() { + @Override + public Object answer(final InvocationOnMock invocation) throws Throwable { + final Callback callback = invocation.getArgumentAt(1, Callback.class); + callback.onCompletion(null, new RuntimeException("Unit Test Intentional Exception")); + return null; + } + }).when(producer).send(any(ProducerRecord.class), any(Callback.class)); + + lease.publish(flowFile, new ByteArrayInputStream(new byte[1]), messageKey, demarcatorBytes, topic); + + assertEquals(1, poisonCount.get()); + + final PublishResult result = lease.complete(); + assertTrue(result.getFailedFlowFiles().contains(flowFile)); + assertFalse(result.getSuccessfulFlowFiles().contains(flowFile)); + } + + @Test + @SuppressWarnings("unchecked") + public void testAllDelimitedMessagesSent() throws IOException { + final AtomicInteger poisonCount = new AtomicInteger(0); + + final PublisherLease lease = new PublisherLease(producer, 1024 * 1024, 10L, logger) { + @Override + protected void poison() { + poisonCount.incrementAndGet(); + super.poison(); + } + }; + + final AtomicInteger correctMessages = new AtomicInteger(0); + final AtomicInteger incorrectMessages = new AtomicInteger(0); + doAnswer(new Answer<Object>() { + @Override + public Object answer(InvocationOnMock invocation) throws Throwable { + final ProducerRecord<byte[], byte[]> record = invocation.getArgumentAt(0, ProducerRecord.class); + final byte[] value = record.value(); + final String valueString = new String(value, StandardCharsets.UTF_8); + if ("1234567890".equals(valueString)) { + correctMessages.incrementAndGet(); + } else { + incorrectMessages.incrementAndGet(); + } + + return null; + } + }).when(producer).send(any(ProducerRecord.class), any(Callback.class)); + + final FlowFile flowFile = new MockFlowFile(1L); + final String topic = "unit-test"; + final byte[] messageKey = null; + final byte[] demarcatorBytes = "\n".getBytes(StandardCharsets.UTF_8); + + final byte[] flowFileContent = "1234567890\n1234567890\n1234567890\n\n\n\n1234567890\n\n\n1234567890\n\n\n\n".getBytes(StandardCharsets.UTF_8); + lease.publish(flowFile, new ByteArrayInputStream(flowFileContent), messageKey, demarcatorBytes, topic); + + final byte[] flowFileContent2 = new byte[0]; + lease.publish(new MockFlowFile(2L), new ByteArrayInputStream(flowFileContent2), messageKey, demarcatorBytes, topic); + + final byte[] flowFileContent3 = "1234567890\n1234567890".getBytes(StandardCharsets.UTF_8); // no trailing new line + lease.publish(new MockFlowFile(3L), new ByteArrayInputStream(flowFileContent3), messageKey, demarcatorBytes, topic); + + final byte[] flowFileContent4 = "\n\n\n".getBytes(StandardCharsets.UTF_8); + lease.publish(new MockFlowFile(4L), new ByteArrayInputStream(flowFileContent4), messageKey, demarcatorBytes, topic); + + assertEquals(0, poisonCount.get()); + + verify(producer, times(0)).flush(); + + final PublishResult result = lease.complete(); + assertTrue(result.getFailedFlowFiles().contains(flowFile)); + assertFalse(result.getSuccessfulFlowFiles().contains(flowFile)); + + assertEquals(7, correctMessages.get()); + assertEquals(0, incorrectMessages.get()); + + verify(producer, times(1)).flush(); + } +} http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java new file mode 100644 index 0000000..7c70194 --- /dev/null +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/test/java/org/apache/nifi/processors/kafka/pubsub/TestPublisherPool.java @@ -0,0 +1,68 @@ +/* + * 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.nifi.processors.kafka.pubsub; + +import static org.junit.Assert.assertEquals; + +import java.util.HashMap; +import java.util.Map; + +import org.apache.kafka.common.serialization.ByteArraySerializer; +import org.apache.nifi.logging.ComponentLog; +import org.junit.Test; +import org.mockito.Mockito; + + +public class TestPublisherPool { + + @Test + public void testLeaseCloseReturnsToPool() { + final Map<String, Object> kafkaProperties = new HashMap<>(); + kafkaProperties.put("bootstrap.servers", "localhost:1111"); + kafkaProperties.put("key.serializer", ByteArraySerializer.class.getName()); + kafkaProperties.put("value.serializer", ByteArraySerializer.class.getName()); + + final PublisherPool pool = new PublisherPool(kafkaProperties, Mockito.mock(ComponentLog.class), 1024 * 1024, 1000L); + assertEquals(0, pool.available()); + + final PublisherLease lease = pool.obtainPublisher(); + assertEquals(0, pool.available()); + + lease.close(); + assertEquals(1, pool.available()); + } + + @Test + public void testPoisonedLeaseNotReturnedToPool() { + final Map<String, Object> kafkaProperties = new HashMap<>(); + kafkaProperties.put("bootstrap.servers", "localhost:1111"); + kafkaProperties.put("key.serializer", ByteArraySerializer.class.getName()); + kafkaProperties.put("value.serializer", ByteArraySerializer.class.getName()); + + final PublisherPool pool = new PublisherPool(kafkaProperties, Mockito.mock(ComponentLog.class), 1024 * 1024, 1000L); + assertEquals(0, pool.available()); + + final PublisherLease lease = pool.obtainPublisher(); + assertEquals(0, pool.available()); + + lease.poison(); + lease.close(); + assertEquals(0, pool.available()); + } + +} http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaPublisher.java ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaPublisher.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaPublisher.java index 5bc0e0e..e524589 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaPublisher.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-8-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaPublisher.java @@ -17,6 +17,7 @@ package org.apache.nifi.processors.kafka; import java.io.Closeable; +import java.io.IOException; import java.io.InputStream; import java.util.ArrayList; import java.util.List; @@ -111,8 +112,9 @@ class KafkaPublisher implements Closeable { * instance of {@link PublishingContext} which hold context * information about the message(s) to be sent. * @return The index of the last successful offset. + * @throws IOException if unable to read from the Input Stream */ - KafkaPublisherResult publish(PublishingContext publishingContext) { + KafkaPublisherResult publish(PublishingContext publishingContext) throws IOException { StreamDemarcator streamTokenizer = new StreamDemarcator(publishingContext.getContentStream(), publishingContext.getDelimiterBytes(), publishingContext.getMaxRequestSize()); http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java index 0a3fe5d..3e01e51 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumeKafka.java @@ -21,17 +21,14 @@ 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; import java.util.Set; import java.util.concurrent.TimeUnit; -import javax.xml.bind.DatatypeConverter; 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.OffsetAndMetadata; import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.TopicPartition; +import org.apache.kafka.common.errors.WakeupException; import org.apache.kafka.common.serialization.ByteArrayDeserializer; import org.apache.nifi.annotation.behavior.DynamicProperty; import org.apache.nifi.annotation.behavior.InputRequirement; @@ -39,13 +36,12 @@ import org.apache.nifi.annotation.behavior.WritesAttribute; import org.apache.nifi.annotation.behavior.WritesAttributes; import org.apache.nifi.annotation.documentation.CapabilityDescription; import org.apache.nifi.annotation.documentation.Tags; -import org.apache.nifi.annotation.lifecycle.OnScheduled; import org.apache.nifi.annotation.lifecycle.OnStopped; +import org.apache.nifi.annotation.lifecycle.OnUnscheduled; import org.apache.nifi.components.AllowableValue; import org.apache.nifi.components.PropertyDescriptor; import org.apache.nifi.components.ValidationContext; import org.apache.nifi.components.ValidationResult; -import org.apache.nifi.flowfile.FlowFile; import org.apache.nifi.logging.ComponentLog; import org.apache.nifi.processor.AbstractProcessor; import org.apache.nifi.processor.ProcessContext; @@ -53,17 +49,18 @@ import org.apache.nifi.processor.ProcessSession; import org.apache.nifi.processor.Relationship; import org.apache.nifi.processor.exception.ProcessException; import org.apache.nifi.processor.util.StandardValidators; -import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.SECURITY_PROTOCOL; +import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.HEX_ENCODING; +import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING; -@CapabilityDescription("Consumes messages from Apache Kafka specifically built against the Kafka 0.9 Consumer API. " +@CapabilityDescription("Consumes messages from Apache Kafka specifically built against the Kafka 0.9.x Consumer API. " + " Please note there are cases where the publisher can get into an indefinite stuck state. We are closely monitoring" + " how this evolves in the Kafka community and will take advantage of those fixes as soon as we can. In the mean time" - + " it is possible to enter states where the only resolution will be to restart the JVM NiFi runs on.") + + " it is possible to enter states where the only resolution will be to restart the JVM NiFi runs on. The complementary NiFi processor for sending messages is PublishKafka.") @Tags({"Kafka", "Get", "Ingest", "Ingress", "Topic", "PubSub", "Consume", "0.9.x"}) @WritesAttributes({ @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_COUNT, description = "The number of messages written if more than one"), @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_KEY, description = "The key of message if present and if single message. " - + "How the key is encoded depends on the value of the 'Key Attribute Encoding' property."), + + "How the key is encoded depends on the value of the 'Key Attribute Encoding' property."), @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_OFFSET, description = "The offset of the message in the partition of the topic."), @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_PARTITION, description = "The partition of the topic the message or message bundle is from"), @WritesAttribute(attribute = KafkaProcessorUtils.KAFKA_TOPIC, description = "The topic the message or message bundle is from") @@ -75,22 +72,16 @@ import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.SECURI + " For the list of available Kafka properties please refer to: http://kafka.apache.org/documentation.html#configuration. ") public class ConsumeKafka extends AbstractProcessor { - private static final long TWO_MB = 2L * 1024L * 1024L; - static final AllowableValue OFFSET_EARLIEST = new AllowableValue("earliest", "earliest", "Automatically reset the offset to the earliest offset"); static final AllowableValue OFFSET_LATEST = new AllowableValue("latest", "latest", "Automatically reset the offset to the latest offset"); static final AllowableValue OFFSET_NONE = new AllowableValue("none", "none", "Throw exception to the consumer if no previous offset is found for the consumer's group"); - static final AllowableValue UTF8_ENCODING = new AllowableValue("utf-8", "UTF-8 Encoded", "The key is interpreted as a UTF-8 Encoded string."); - static final AllowableValue HEX_ENCODING = new AllowableValue("hex", "Hex Encoded", - "The key is interpreted as arbitrary binary data and is encoded using hexadecimal characters with uppercase letters"); - static final PropertyDescriptor TOPICS = new PropertyDescriptor.Builder() .name("topic") .displayName("Topic Name(s)") - .description("The name of the Kafka Topic(s) to pull from. More than one can be supplied if comma seperated.") + .description("The name of the Kafka Topic(s) to pull from. More than one can be supplied if comma separated.") .required(true) .addValidator(StandardValidators.NON_BLANK_VALIDATOR) .expressionLanguageSupported(true) @@ -136,6 +127,7 @@ public class ConsumeKafka extends AbstractProcessor { + "will result in a single FlowFile which " + "time it is triggered. To enter special character such as 'new line' use CTRL+Enter or Shift+Enter depending on the OS") .build(); + static final PropertyDescriptor MAX_POLL_RECORDS = new PropertyDescriptor.Builder() .name("max.poll.records") .displayName("Max Poll Records") @@ -145,6 +137,20 @@ public class ConsumeKafka extends AbstractProcessor { .addValidator(StandardValidators.POSITIVE_INTEGER_VALIDATOR) .build(); + static final PropertyDescriptor MAX_UNCOMMITTED_TIME = new PropertyDescriptor.Builder() + .name("max-uncommit-offset-wait") + .displayName("Max Uncommitted Time") + .description("Specifies the maximum amount of time allowed to pass before offsets must be committed. " + + "This value impacts how often offsets will be committed. Committing offsets less often increases " + + "throughput but also increases the window of potential data duplication in the event of a rebalance " + + "or JVM restart between commits. This value is also related to maximum poll records and the use " + + "of a message demarcator. When using a message demarcator we can have far more uncommitted messages " + + "than when we're not as there is much less for us to keep track of in memory.") + .required(false) + .defaultValue("1 secs") + .addValidator(StandardValidators.TIME_PERIOD_VALIDATOR) + .build(); + static final Relationship REL_SUCCESS = new Relationship.Builder() .name("success") .description("FlowFiles received from Kafka. Depending on demarcation strategy it is a flow file per message or a bundle of messages grouped by topic and partition.") @@ -153,8 +159,8 @@ public class ConsumeKafka extends AbstractProcessor { static final List<PropertyDescriptor> DESCRIPTORS; static final Set<Relationship> RELATIONSHIPS; - private volatile byte[] demarcatorBytes = null; private volatile ConsumerPool consumerPool = null; + private final Set<ConsumerLease> activeLeases = Collections.synchronizedSet(new HashSet<>()); static { List<PropertyDescriptor> descriptors = new ArrayList<>(); @@ -165,6 +171,7 @@ public class ConsumeKafka extends AbstractProcessor { descriptors.add(KEY_ATTRIBUTE_ENCODING); descriptors.add(MESSAGE_DEMARCATOR); descriptors.add(MAX_POLL_RECORDS); + descriptors.add(MAX_UNCOMMITTED_TIME); DESCRIPTORS = Collections.unmodifiableList(descriptors); RELATIONSHIPS = Collections.singleton(REL_SUCCESS); } @@ -179,16 +186,8 @@ public class ConsumeKafka extends AbstractProcessor { return DESCRIPTORS; } - @OnScheduled - public void prepareProcessing(final ProcessContext context) { - this.demarcatorBytes = context.getProperty(MESSAGE_DEMARCATOR).isSet() - ? context.getProperty(MESSAGE_DEMARCATOR).evaluateAttributeExpressions().getValue().getBytes(StandardCharsets.UTF_8) - : null; - } - @OnStopped public void close() { - demarcatorBytes = null; final ConsumerPool pool = consumerPool; consumerPool = null; if (pool != null) { @@ -215,9 +214,22 @@ public class ConsumeKafka extends AbstractProcessor { return pool; } - final Map<String, String> props = new HashMap<>(); + return consumerPool = createConsumerPool(context, getLogger()); + } + + protected ConsumerPool createConsumerPool(final ProcessContext context, final ComponentLog log) { + final int maxLeases = context.getMaxConcurrentTasks(); + final long maxUncommittedTime = context.getProperty(MAX_UNCOMMITTED_TIME).asTimePeriod(TimeUnit.MILLISECONDS); + final byte[] demarcator = context.getProperty(ConsumeKafka.MESSAGE_DEMARCATOR).isSet() + ? context.getProperty(ConsumeKafka.MESSAGE_DEMARCATOR).evaluateAttributeExpressions().getValue().getBytes(StandardCharsets.UTF_8) + : null; + + final Map<String, Object> props = new HashMap<>(); KafkaProcessorUtils.buildCommonKafkaProperties(context, ConsumerConfig.class, props); - final String topicListing = context.getProperty(TOPICS).evaluateAttributeExpressions().getValue(); + props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); + props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); + final String topicListing = context.getProperty(ConsumeKafka.TOPICS).evaluateAttributeExpressions().getValue(); final List<String> topics = new ArrayList<>(); for (final String topic : topicListing.split(",", 100)) { final String trimmedName = topic.trim(); @@ -225,213 +237,78 @@ public class ConsumeKafka extends AbstractProcessor { topics.add(trimmedName); } } - props.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false"); - props.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); - props.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName()); - return consumerPool = createConsumerPool(context.getMaxConcurrentTasks(), topics, props, getLogger()); - } + final String keyEncoding = context.getProperty(KEY_ATTRIBUTE_ENCODING).getValue(); + final String securityProtocol = context.getProperty(KafkaProcessorUtils.SECURITY_PROTOCOL).getValue(); + final String bootstrapServers = context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).getValue(); - protected ConsumerPool createConsumerPool(final int maxLeases, final List<String> topics, final Map<String, String> props, final ComponentLog log) { - return new ConsumerPool(maxLeases, topics, props, log); + return new ConsumerPool(maxLeases, demarcator, props, topics, maxUncommittedTime, keyEncoding, securityProtocol, bootstrapServers, log); } - @Override - public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { - final long startTimeNanos = System.nanoTime(); - final ConsumerPool pool = getConsumerPool(context); - if (pool == null) { - context.yield(); - return; - } - final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecordMap = new HashMap<>(); - - try (final ConsumerLease lease = pool.obtainConsumer()) { + @OnUnscheduled + public void interruptActiveThreads() { + // There are known issues with the Kafka client library that result in the client code hanging + // indefinitely when unable to communicate with the broker. In order to address this, we will wait + // up to 30 seconds for the Threads to finish and then will call Consumer.wakeup() to trigger the + // thread to wakeup when it is blocked, waiting on a response. + final long nanosToWait = TimeUnit.SECONDS.toNanos(5L); + final long start = System.nanoTime(); + while (System.nanoTime() - start < nanosToWait && !activeLeases.isEmpty()) { try { - if (lease == null) { - context.yield(); - return; - } - - final boolean foundData = gatherDataFromKafka(lease, partitionRecordMap, context); - if (!foundData) { - session.rollback(); - return; - } - - writeSessionData(context, session, partitionRecordMap, startTimeNanos); - //At-least once commit handling (if order is reversed it is at-most once) - session.commit(); - commitOffsets(lease, partitionRecordMap); - } catch (final KafkaException ke) { - lease.poison(); - getLogger().error("Problem while accessing kafka consumer " + ke, ke); - context.yield(); - session.rollback(); + Thread.sleep(100L); + } catch (final InterruptedException ie) { + Thread.currentThread().interrupt(); + return; } } - } - private void commitOffsets(final ConsumerLease lease, final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecordMap) { - final Map<TopicPartition, OffsetAndMetadata> partOffsetMap = new HashMap<>(); - partitionRecordMap.entrySet().stream() - .filter(entry -> !entry.getValue().isEmpty()) - .forEach((entry) -> { - long maxOffset = entry.getValue().stream() - .mapToLong(record -> record.offset()) - .max() - .getAsLong(); - partOffsetMap.put(entry.getKey(), new OffsetAndMetadata(maxOffset + 1L)); - }); - lease.commitOffsets(partOffsetMap); - } + if (!activeLeases.isEmpty()) { + int count = 0; + for (final ConsumerLease lease : activeLeases) { + getLogger().info("Consumer {} has not finished after waiting 30 seconds; will attempt to wake-up the lease", new Object[] {lease}); + lease.wakeup(); + count++; + } - private void writeSessionData( - final ProcessContext context, final ProcessSession session, - final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecordMap, - final long startTimeNanos) { - if (demarcatorBytes != null) { - partitionRecordMap.entrySet().stream() - .filter(entry -> !entry.getValue().isEmpty()) - .forEach(entry -> { - writeData(context, session, entry.getValue(), startTimeNanos); - }); - } else { - partitionRecordMap.entrySet().stream() - .filter(entry -> !entry.getValue().isEmpty()) - .flatMap(entry -> entry.getValue().stream()) - .forEach(record -> { - writeData(context, session, Collections.singletonList(record), startTimeNanos); - }); + getLogger().info("Woke up {} consumers", new Object[] {count}); } - } - private String encodeKafkaKey(final byte[] key, final String encoding) { - if (key == null) { - return null; - } + activeLeases.clear(); + } - if (HEX_ENCODING.getValue().equals(encoding)) { - return DatatypeConverter.printHexBinary(key); - } else if (UTF8_ENCODING.getValue().equals(encoding)) { - return new String(key, StandardCharsets.UTF_8); - } else { - return null; // won't happen because it is guaranteed by the Allowable Values + @Override + public void onTrigger(ProcessContext context, ProcessSession session) throws ProcessException { + final ConsumerPool pool = getConsumerPool(context); + if (pool == null) { + context.yield(); + return; } - } - private void writeData(final ProcessContext context, final ProcessSession session, final List<ConsumerRecord<byte[], byte[]>> records, final long startTimeNanos) { - final ConsumerRecord<byte[], byte[]> firstRecord = records.get(0); - final String offset = String.valueOf(firstRecord.offset()); - final String keyValue = encodeKafkaKey(firstRecord.key(), context.getProperty(KEY_ATTRIBUTE_ENCODING).getValue()); - final String topic = firstRecord.topic(); - final String partition = String.valueOf(firstRecord.partition()); - FlowFile flowFile = session.create(); - flowFile = session.write(flowFile, out -> { - boolean useDemarcator = false; - for (final ConsumerRecord<byte[], byte[]> record : records) { - if (useDemarcator) { - out.write(demarcatorBytes); - } - out.write(record.value()); - useDemarcator = true; + try (final ConsumerLease lease = pool.obtainConsumer(session)) { + if (lease == null) { + context.yield(); + return; } - }); - final Map<String, String> kafkaAttrs = new HashMap<>(); - kafkaAttrs.put(KafkaProcessorUtils.KAFKA_OFFSET, offset); - if (keyValue != null && records.size() == 1) { - kafkaAttrs.put(KafkaProcessorUtils.KAFKA_KEY, keyValue); - } - kafkaAttrs.put(KafkaProcessorUtils.KAFKA_PARTITION, partition); - kafkaAttrs.put(KafkaProcessorUtils.KAFKA_TOPIC, topic); - if (records.size() > 1) { - kafkaAttrs.put(KafkaProcessorUtils.KAFKA_COUNT, String.valueOf(records.size())); - } - flowFile = session.putAllAttributes(flowFile, kafkaAttrs); - final long executionDurationMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNanos); - final String transitUri = KafkaProcessorUtils.buildTransitURI( - context.getProperty(SECURITY_PROTOCOL).getValue(), - context.getProperty(KafkaProcessorUtils.BOOTSTRAP_SERVERS).getValue(), - topic); - session.getProvenanceReporter().receive(flowFile, transitUri, executionDurationMillis); - this.getLogger().debug("Created {} containing {} messages from Kafka topic {}, partition {}, starting offset {} in {} millis", - new Object[]{flowFile, records.size(), topic, partition, offset, executionDurationMillis}); - session.transfer(flowFile, REL_SUCCESS); - } - /** - * Populates the given partitionRecordMap with new records until we poll - * that returns no records or until we have enough data. It is important to - * ensure we keep items grouped by their topic and partition so that when we - * bundle them we bundle them intelligently and so that we can set offsets - * properly even across multiple poll calls. - */ - private boolean gatherDataFromKafka(final ConsumerLease lease, final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecordMap, ProcessContext context) { - final long startNanos = System.nanoTime(); - boolean foundData = false; - ConsumerRecords<byte[], byte[]> records; - final int maxRecords = context.getProperty(MAX_POLL_RECORDS).asInteger(); - - do { - records = lease.poll(); - - for (final TopicPartition partition : records.partitions()) { - List<ConsumerRecord<byte[], byte[]>> currList = partitionRecordMap.get(partition); - if (currList == null) { - currList = new ArrayList<>(); - partitionRecordMap.put(partition, currList); + activeLeases.add(lease); + try { + while (this.isScheduled() && lease.continuePolling()) { + lease.poll(); } - currList.addAll(records.records(partition)); - if (currList.size() > 0) { - foundData = true; + if (this.isScheduled() && !lease.commit()) { + context.yield(); } + } catch (final WakeupException we) { + getLogger().warn("Was interrupted while trying to communicate with Kafka with lease {}. " + + "Will roll back session and discard any partially received data.", new Object[] {lease}); + } catch (final KafkaException kex) { + getLogger().error("Exception while interacting with Kafka so will close the lease {} due to {}", + new Object[] {lease, kex}, kex); + } catch (final Throwable t) { + getLogger().error("Exception while processing data from kafka so will close the lease {} due to {}", + new Object[] {lease, t}, t); + } finally { + activeLeases.remove(lease); } - //If we received data and we still want to get more - } while (!records.isEmpty() && !checkIfGatheredEnoughData(partitionRecordMap, maxRecords, startNanos)); - return foundData; - } - - /** - * Determines if we have enough data as-is and should move on. - * - * @return true if we've been gathering for more than 500 ms or if we're - * demarcating and have more than 50 flowfiles worth or if we're per message - * and have more than 2000 flowfiles or if totalMessageSize is greater than - * two megabytes; false otherwise - * - * Implementation note: 500 millis and 5 MB are magic numbers. These may - * need to be tuned. They get at how often offsets will get committed to - * kafka relative to how many records will get buffered into memory in a - * poll call before writing to repos. - */ - private boolean checkIfGatheredEnoughData(final Map<TopicPartition, List<ConsumerRecord<byte[], byte[]>>> partitionRecordMap, final long maxRecords, final long startTimeNanos) { - - final long durationMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - startTimeNanos); - - if (durationMillis > 500) { - return true; - } - - int topicPartitionsFilled = 0; - int totalRecords = 0; - long totalRecordSize = 0; - - for (final List<ConsumerRecord<byte[], byte[]>> recordList : partitionRecordMap.values()) { - if (!recordList.isEmpty()) { - topicPartitionsFilled++; - } - totalRecords += recordList.size(); - for (final ConsumerRecord<byte[], byte[]> rec : recordList) { - totalRecordSize += rec.value().length; - } - } - - if (demarcatorBytes != null && demarcatorBytes.length > 0) { - return topicPartitionsFilled > 50; - } else if (totalRecordSize > TWO_MB) { - return true; - } else { - return totalRecords > maxRecords; } } - } http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java index b954eba..cd9365d 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerLease.java @@ -17,11 +17,28 @@ package org.apache.nifi.processors.kafka.pubsub; import java.io.Closeable; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Collection; +import java.util.HashMap; +import java.util.List; import java.util.Map; +import java.util.concurrent.TimeUnit; +import javax.xml.bind.DatatypeConverter; +import org.apache.kafka.clients.consumer.Consumer; +import org.apache.kafka.clients.consumer.ConsumerRebalanceListener; +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.consumer.OffsetAndMetadata; import org.apache.kafka.common.KafkaException; import org.apache.kafka.common.TopicPartition; +import org.apache.nifi.flowfile.FlowFile; +import org.apache.nifi.logging.ComponentLog; +import org.apache.nifi.processor.ProcessSession; +import static org.apache.nifi.processors.kafka.pubsub.ConsumeKafka.REL_SUCCESS; +import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.HEX_ENCODING; +import static org.apache.nifi.processors.kafka.pubsub.KafkaProcessorUtils.UTF8_ENCODING; /** * This class represents a lease to access a Kafka Consumer object. The lease is @@ -30,15 +47,108 @@ import org.apache.kafka.common.TopicPartition; * the lease will be returned to the pool for future use by others. A given * lease may only belong to a single thread a time. */ -public interface ConsumerLease extends Closeable { +public abstract class ConsumerLease implements Closeable, ConsumerRebalanceListener { + + private final long maxWaitMillis; + private final Consumer<byte[], byte[]> kafkaConsumer; + private final ComponentLog logger; + private final byte[] demarcatorBytes; + private final String keyEncoding; + private final String securityProtocol; + private final String bootstrapServers; + private boolean poisoned = false; + //used for tracking demarcated flowfiles to their TopicPartition so we can append + //to them on subsequent poll calls + private final Map<TopicPartition, BundleTracker> bundleMap = new HashMap<>(); + private final Map<TopicPartition, OffsetAndMetadata> uncommittedOffsetsMap = new HashMap<>(); + private long leaseStartNanos = -1; + private boolean lastPollEmpty = false; + private int totalFlowFiles = 0; + + ConsumerLease( + final long maxWaitMillis, + final Consumer<byte[], byte[]> kafkaConsumer, + final byte[] demarcatorBytes, + final String keyEncoding, + final String securityProtocol, + final String bootstrapServers, + final ComponentLog logger) { + this.maxWaitMillis = maxWaitMillis; + this.kafkaConsumer = kafkaConsumer; + this.demarcatorBytes = demarcatorBytes; + this.keyEncoding = keyEncoding; + this.securityProtocol = securityProtocol; + this.bootstrapServers = bootstrapServers; + this.logger = logger; + } + + /** + * clears out internal state elements excluding session and consumer as + * those are managed by the pool itself + */ + private void resetInternalState() { + bundleMap.clear(); + uncommittedOffsetsMap.clear(); + leaseStartNanos = -1; + lastPollEmpty = false; + totalFlowFiles = 0; + } /** - * Executes a poll on the underlying Kafka Consumer. + * Kafka will call this method whenever it is about to rebalance the + * consumers for the given partitions. We'll simply take this to mean that + * we need to quickly commit what we've got and will return the consumer to + * the pool. This method will be called during the poll() method call of + * this class and will be called by the same thread calling poll according + * to the Kafka API docs. After this method executes the session and kafka + * offsets are committed and this lease is closed. * - * @return ConsumerRecords retrieved in the poll. - * @throws KafkaException if issue occurs talking to underlying resource. + * @param partitions partitions being reassigned + */ + @Override + public void onPartitionsRevoked(final Collection<TopicPartition> partitions) { + logger.debug("Rebalance Alert: Paritions '{}' revoked for lease '{}' with consumer '{}'", new Object[]{partitions, this, kafkaConsumer}); + //force a commit here. Can reuse the session and consumer after this but must commit now to avoid duplicates if kafka reassigns parittion + commit(); + } + + /** + * This will be called by Kafka when the rebalance has completed. We don't + * need to do anything with this information other than optionally log it as + * by this point we've committed what we've got and moved on. + * + * @param partitions topic partition set being reassigned + */ + @Override + public void onPartitionsAssigned(final Collection<TopicPartition> partitions) { + logger.debug("Rebalance Alert: Paritions '{}' assigned for lease '{}' with consumer '{}'", new Object[]{partitions, this, kafkaConsumer}); + } + + /** + * Executes a poll on the underlying Kafka Consumer and creates any new + * flowfiles necessary or appends to existing ones if in demarcation mode. */ - ConsumerRecords<byte[], byte[]> poll() throws KafkaException; + void poll() { + /** + * Implementation note: If we take too long (30 secs?) between kafka + * poll calls and our own record processing to any subsequent poll calls + * or the commit we can run into a situation where the commit will + * succeed to the session but fail on committing offsets. This is + * apparently different than the Kafka scenario of electing to rebalance + * for other reasons but in this case is due a session timeout. It + * appears Kafka KIP-62 aims to offer more control over the meaning of + * various timeouts. If we do run into this case it could result in + * duplicates. + */ + try { + final ConsumerRecords<byte[], byte[]> records = kafkaConsumer.poll(10); + lastPollEmpty = records.count() == 0; + processRecords(records); + } catch (final Throwable t) { + this.poison(); + throw t; + } + } /** * Notifies Kafka to commit the offsets for the specified topic/partition @@ -47,22 +157,251 @@ public interface ConsumerLease extends Closeable { * kafka client to collect more data from Kafka before committing the * offsets. * - * @param offsets offsets - * @throws KafkaException if issue occurs talking to underlying resource. + * if false then we didn't do anything and should probably yield if true + * then we committed new data + * */ - void commitOffsets(Map<TopicPartition, OffsetAndMetadata> offsets) throws KafkaException; + boolean commit() { + if (uncommittedOffsetsMap.isEmpty()) { + resetInternalState(); + return false; + } + try { + /** + * Committing the nifi session then the offsets means we have an at + * least once guarantee here. If we reversed the order we'd have at + * most once. + */ + final Collection<FlowFile> bundledFlowFiles = getBundles(); + if (!bundledFlowFiles.isEmpty()) { + getProcessSession().transfer(bundledFlowFiles, REL_SUCCESS); + } + getProcessSession().commit(); + kafkaConsumer.commitSync(uncommittedOffsetsMap); + resetInternalState(); + return true; + } catch (final KafkaException kex) { + poison(); + logger.warn("Duplicates are likely as we were able to commit the process" + + " session but received an exception from Kafka while committing" + + " offsets."); + throw kex; + } catch (final Throwable t) { + poison(); + throw t; + } + } /** - * Notifies that this lease is poisoned and should not be reused. + * Indicates whether we should continue polling for data. If we are not + * writing data with a demarcator then we're writing individual flow files + * per kafka message therefore we must be very mindful of memory usage for + * the flow file objects (not their content) being held in memory. The + * content of kafka messages will be written to the content repository + * immediately upon each poll call but we must still be mindful of how much + * memory can be used in each poll call. We will indicate that we should + * stop polling our last poll call produced no new results or if we've + * polling and processing data longer than the specified maximum polling + * time or if we have reached out specified max flow file limit or if a + * rebalance has been initiated for one of the partitions we're watching; + * otherwise true. + * + * @return true if should keep polling; false otherwise */ - void poison(); + boolean continuePolling() { + //stop if the last poll produced new no data + if (lastPollEmpty) { + return false; + } + + //stop if we've gone past our desired max uncommitted wait time + if (leaseStartNanos < 0) { + leaseStartNanos = System.nanoTime(); + } + final long durationMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - leaseStartNanos); + if (durationMillis > maxWaitMillis) { + return false; + } + + //stop if we've generated enough flowfiles that we need to be concerned about memory usage for the objects + if (bundleMap.size() > 200) { //a magic number - the number of simultaneous bundles to track + return false; + } else { + return totalFlowFiles < 15000;//admittedly a magic number - good candidate for processor property + } + } /** - * Notifies that this lease is to be returned. The pool may optionally reuse - * this lease with another client. No further references by the caller - * should occur after calling close. + * Indicates that the underlying session and consumer should be immediately + * considered invalid. Once closed the session will be rolled back and the + * pool should destroy the underlying consumer. This is useful if due to + * external reasons, such as the processor no longer being scheduled, this + * lease should be terminated immediately. + */ + private void poison() { + poisoned = true; + } + + /** + * @return true if this lease has been poisoned; false otherwise + */ + boolean isPoisoned() { + return poisoned; + } + + /** + * Trigger the consumer's {@link KafkaConsumer#wakeup() wakeup()} method. + */ + public void wakeup() { + kafkaConsumer.wakeup(); + } + + /** + * Abstract method that is intended to be extended by the pool that created + * this ConsumerLease object. It should ensure that the session given to + * create this session is rolled back and that the underlying kafka consumer + * is either returned to the pool for continued use or destroyed if this + * lease has been poisoned. It can only be called once. Calling it more than + * once can result in undefined and non threadsafe behavior. */ @Override - void close(); + public void close() { + resetInternalState(); + } + + public abstract ProcessSession getProcessSession(); + + private void processRecords(final ConsumerRecords<byte[], byte[]> records) { + + records.partitions().stream().forEach(partition -> { + List<ConsumerRecord<byte[], byte[]>> messages = records.records(partition); + if (!messages.isEmpty()) { + //update maximum offset map for this topic partition + long maxOffset = messages.stream() + .mapToLong(record -> record.offset()) + .max() + .getAsLong(); + uncommittedOffsetsMap.put(partition, new OffsetAndMetadata(maxOffset + 1L)); + + //write records to content repository and session + if (demarcatorBytes == null) { + totalFlowFiles += messages.size(); + messages.stream().forEach(message -> { + writeData(getProcessSession(), message, partition); + }); + } else { + writeData(getProcessSession(), messages, partition); + } + } + }); + } + + private static String encodeKafkaKey(final byte[] key, final String encoding) { + if (key == null) { + return null; + } + + if (HEX_ENCODING.getValue().equals(encoding)) { + return DatatypeConverter.printHexBinary(key); + } else if (UTF8_ENCODING.getValue().equals(encoding)) { + return new String(key, StandardCharsets.UTF_8); + } else { + return null; // won't happen because it is guaranteed by the Allowable Values + } + } + + private Collection<FlowFile> getBundles() { + final List<FlowFile> flowFiles = new ArrayList<>(); + for (final BundleTracker tracker : bundleMap.values()) { + populateAttributes(tracker); + flowFiles.add(tracker.flowFile); + } + return flowFiles; + } + + private void writeData(final ProcessSession session, ConsumerRecord<byte[], byte[]> record, final TopicPartition topicPartition) { + FlowFile flowFile = session.create(); + final BundleTracker tracker = new BundleTracker(record, topicPartition, keyEncoding); + tracker.incrementRecordCount(1); + flowFile = session.write(flowFile, out -> { + out.write(record.value()); + }); + tracker.updateFlowFile(flowFile); + populateAttributes(tracker); + session.transfer(tracker.flowFile, REL_SUCCESS); + } + + private void writeData(final ProcessSession session, final List<ConsumerRecord<byte[], byte[]>> records, final TopicPartition topicPartition) { + final ConsumerRecord<byte[], byte[]> firstRecord = records.get(0); + final boolean demarcateFirstRecord; + BundleTracker tracker = bundleMap.get(topicPartition); + FlowFile flowFile; + if (tracker == null) { + tracker = new BundleTracker(firstRecord, topicPartition, keyEncoding); + flowFile = session.create(); + tracker.updateFlowFile(flowFile); + demarcateFirstRecord = false; //have not yet written records for this topic/partition in this lease + } else { + demarcateFirstRecord = true; //have already been writing records for this topic/partition in this lease + } + flowFile = tracker.flowFile; + tracker.incrementRecordCount(records.size()); + flowFile = session.append(flowFile, out -> { + boolean useDemarcator = demarcateFirstRecord; + for (final ConsumerRecord<byte[], byte[]> record : records) { + if (useDemarcator) { + out.write(demarcatorBytes); + } + out.write(record.value()); + useDemarcator = true; + } + }); + tracker.updateFlowFile(flowFile); + bundleMap.put(topicPartition, tracker); + } + + private void populateAttributes(final BundleTracker tracker) { + final Map<String, String> kafkaAttrs = new HashMap<>(); + kafkaAttrs.put(KafkaProcessorUtils.KAFKA_OFFSET, String.valueOf(tracker.initialOffset)); + if (tracker.key != null && tracker.totalRecords == 1) { + kafkaAttrs.put(KafkaProcessorUtils.KAFKA_KEY, tracker.key); + } + kafkaAttrs.put(KafkaProcessorUtils.KAFKA_PARTITION, String.valueOf(tracker.partition)); + kafkaAttrs.put(KafkaProcessorUtils.KAFKA_TOPIC, tracker.topic); + if (tracker.totalRecords > 1) { + kafkaAttrs.put(KafkaProcessorUtils.KAFKA_COUNT, String.valueOf(tracker.totalRecords)); + } + final FlowFile newFlowFile = getProcessSession().putAllAttributes(tracker.flowFile, kafkaAttrs); + final long executionDurationMillis = TimeUnit.NANOSECONDS.toMillis(System.nanoTime() - leaseStartNanos); + final String transitUri = KafkaProcessorUtils.buildTransitURI(securityProtocol, bootstrapServers, tracker.topic); + getProcessSession().getProvenanceReporter().receive(newFlowFile, transitUri, executionDurationMillis); + tracker.updateFlowFile(newFlowFile); + } + + private static class BundleTracker { + + final long initialOffset; + final int partition; + final String topic; + final String key; + FlowFile flowFile; + long totalRecords = 0; + + private BundleTracker(final ConsumerRecord<byte[], byte[]> initialRecord, final TopicPartition topicPartition, final String keyEncoding) { + this.initialOffset = initialRecord.offset(); + this.partition = topicPartition.partition(); + this.topic = topicPartition.topic(); + this.key = encodeKafkaKey(initialRecord.key(), keyEncoding); + } + + private void incrementRecordCount(final long count) { + totalRecords += count; + } + + private void updateFlowFile(final FlowFile flowFile) { + this.flowFile = flowFile; + } + + } } http://git-wip-us.apache.org/repos/asf/nifi/blob/ed17df50/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java ---------------------------------------------------------------------- diff --git a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java index 3f20b8f..e13a8c3 100644 --- a/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java +++ b/nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/ConsumerPool.java @@ -21,18 +21,15 @@ import org.apache.kafka.clients.consumer.KafkaConsumer; import org.apache.nifi.logging.ComponentLog; import java.io.Closeable; -import java.util.ArrayDeque; import java.util.ArrayList; -import java.util.HashMap; +import java.util.Collections; import java.util.List; import java.util.Map; -import java.util.Queue; -import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.BlockingQueue; import java.util.concurrent.atomic.AtomicLong; -import org.apache.kafka.clients.consumer.ConsumerRecords; -import org.apache.kafka.clients.consumer.OffsetAndMetadata; import org.apache.kafka.common.KafkaException; -import org.apache.kafka.common.TopicPartition; +import org.apache.nifi.processor.ProcessSession; /** * A pool of Kafka Consumers for a given topic. Consumers can be obtained by @@ -41,176 +38,119 @@ import org.apache.kafka.common.TopicPartition; */ public class ConsumerPool implements Closeable { - private final AtomicInteger activeLeaseCount = new AtomicInteger(0); - private final int maxLeases; - private final Queue<ConsumerLease> consumerLeases; + private final BlockingQueue<SimpleConsumerLease> pooledLeases; private final List<String> topics; private final Map<String, Object> kafkaProperties; + private final long maxWaitMillis; private final ComponentLog logger; - + private final byte[] demarcatorBytes; + private final String keyEncoding; + private final String securityProtocol; + private final String bootstrapServers; private final AtomicLong consumerCreatedCountRef = new AtomicLong(); private final AtomicLong consumerClosedCountRef = new AtomicLong(); private final AtomicLong leasesObtainedCountRef = new AtomicLong(); - private final AtomicLong productivePollCountRef = new AtomicLong(); - private final AtomicLong unproductivePollCountRef = new AtomicLong(); /** * Creates a pool of KafkaConsumer objects that will grow up to the maximum - * indicated leases. Consumers are lazily initialized. + * indicated threads from the given context. Consumers are lazily + * initialized. We may elect to not create up to the maximum number of + * configured consumers if the broker reported lag time for all topics is + * below a certain threshold. * - * @param maxLeases maximum number of active leases in the pool - * @param topics the topics to consume from - * @param kafkaProperties the properties for each consumer + * @param maxConcurrentLeases max allowable consumers at once + * @param demarcator bytes to use as demarcator between messages; null or + * empty means no demarcator + * @param kafkaProperties properties to use to initialize kafka consumers + * @param topics the topics to subscribe to + * @param maxWaitMillis maximum time to wait for a given lease to acquire + * data before committing + * @param keyEncoding the encoding to use for the key of a kafka message if + * found + * @param securityProtocol the security protocol used + * @param bootstrapServers the bootstrap servers * @param logger the logger to report any errors/warnings */ - public ConsumerPool(final int maxLeases, final List<String> topics, final Map<String, String> kafkaProperties, final ComponentLog logger) { - this.maxLeases = maxLeases; - if (maxLeases <= 0) { - throw new IllegalArgumentException("Max leases value must be greather than zero."); - } + public ConsumerPool( + final int maxConcurrentLeases, + final byte[] demarcator, + final Map<String, Object> kafkaProperties, + final List<String> topics, + final long maxWaitMillis, + final String keyEncoding, + final String securityProtocol, + final String bootstrapServers, + final ComponentLog logger) { + this.pooledLeases = new ArrayBlockingQueue<>(maxConcurrentLeases); + this.maxWaitMillis = maxWaitMillis; this.logger = logger; - if (topics == null || topics.isEmpty()) { - throw new IllegalArgumentException("Must have a list of one or more topics"); - } - this.topics = topics; - this.kafkaProperties = new HashMap<>(kafkaProperties); - this.consumerLeases = new ArrayDeque<>(); + this.demarcatorBytes = demarcator; + this.keyEncoding = keyEncoding; + this.securityProtocol = securityProtocol; + this.bootstrapServers = bootstrapServers; + this.kafkaProperties = Collections.unmodifiableMap(kafkaProperties); + this.topics = Collections.unmodifiableList(topics); } /** - * Obtains a consumer from the pool if one is available + * Obtains a consumer from the pool if one is available or lazily + * initializes a new one if deemed necessary. * - * @return consumer from the pool - * @throws IllegalArgumentException if pool already contains + * @param session the session for which the consumer lease will be + * associated + * @return consumer to use or null if not available or necessary */ - public ConsumerLease obtainConsumer() { - final ConsumerLease lease; - final int activeLeases; - synchronized (this) { - lease = consumerLeases.poll(); - activeLeases = activeLeaseCount.get(); - } - if (lease == null && activeLeases >= maxLeases) { - logger.warn("No available consumers and cannot create any as max consumer leases limit reached - verify pool settings"); - return null; + public ConsumerLease obtainConsumer(final ProcessSession session) { + SimpleConsumerLease lease = pooledLeases.poll(); + if (lease == null) { + final Consumer<byte[], byte[]> consumer = createKafkaConsumer(); + consumerCreatedCountRef.incrementAndGet(); + /** + * For now return a new consumer lease. But we could later elect to + * have this return null if we determine the broker indicates that + * the lag time on all topics being monitored is sufficiently low. + * For now we should encourage conservative use of threads because + * having too many means we'll have at best useless threads sitting + * around doing frequent network calls and at worst having consumers + * sitting idle which could prompt excessive rebalances. + */ + lease = new SimpleConsumerLease(consumer); + + /** + * This subscription tightly couples the lease to the given + * consumer. They cannot be separated from then on. + */ + consumer.subscribe(topics, lease); } + lease.setProcessSession(session); leasesObtainedCountRef.incrementAndGet(); - return (lease == null) ? createConsumer() : lease; + return lease; } + /** + * Exposed as protected method for easier unit testing + * + * @return consumer + * @throws KafkaException if unable to subscribe to the given topics + */ protected Consumer<byte[], byte[]> createKafkaConsumer() { return new KafkaConsumer<>(kafkaProperties); } - private ConsumerLease createConsumer() { - final Consumer<byte[], byte[]> kafkaConsumer = createKafkaConsumer(); - consumerCreatedCountRef.incrementAndGet(); - try { - kafkaConsumer.subscribe(topics); - } catch (final KafkaException kex) { - try { - kafkaConsumer.close(); - consumerClosedCountRef.incrementAndGet(); - } catch (final Exception ex) { - consumerClosedCountRef.incrementAndGet(); - //ignore - } - throw kex; - } - - final ConsumerLease lease = new ConsumerLease() { - - private volatile boolean poisoned = false; - private volatile boolean closed = false; - - @Override - public ConsumerRecords<byte[], byte[]> poll() { - - if (poisoned) { - throw new KafkaException("The consumer is poisoned and should no longer be used"); - } - - try { - final ConsumerRecords<byte[], byte[]> records = kafkaConsumer.poll(50); - if (records.isEmpty()) { - unproductivePollCountRef.incrementAndGet(); - } else { - productivePollCountRef.incrementAndGet(); - } - return records; - } catch (final KafkaException kex) { - logger.warn("Unable to poll from Kafka consumer so will poison and close this " + kafkaConsumer, kex); - poison(); - close(); - throw kex; - } - } - - @Override - public void commitOffsets(final Map<TopicPartition, OffsetAndMetadata> offsets) { - - if (poisoned) { - throw new KafkaException("The consumer is poisoned and should no longer be used"); - } - try { - kafkaConsumer.commitSync(offsets); - } catch (final KafkaException kex) { - logger.warn("Unable to commit kafka consumer offsets so will poison and close this " + kafkaConsumer, kex); - poison(); - close(); - throw kex; - } - } - - @Override - public void close() { - if (closed) { - return; - } - if (poisoned || activeLeaseCount.get() > maxLeases) { - closeConsumer(kafkaConsumer); - activeLeaseCount.decrementAndGet(); - closed = true; - } else { - final boolean added; - synchronized (ConsumerPool.this) { - added = consumerLeases.offer(this); - } - if (!added) { - closeConsumer(kafkaConsumer); - activeLeaseCount.decrementAndGet(); - } - } - } - - @Override - public void poison() { - poisoned = true; - } - }; - activeLeaseCount.incrementAndGet(); - return lease; - } - /** - * Closes all consumers in the pool. Can be safely recalled. + * Closes all consumers in the pool. Can be safely called repeatedly. */ @Override public void close() { - final List<ConsumerLease> leases = new ArrayList<>(); - synchronized (this) { - ConsumerLease lease = null; - while ((lease = consumerLeases.poll()) != null) { - leases.add(lease); - } - } - for (final ConsumerLease lease : leases) { - lease.poison(); - lease.close(); - } + final List<SimpleConsumerLease> leases = new ArrayList<>(); + pooledLeases.drainTo(leases); + leases.stream().forEach((lease) -> { + lease.close(true); + }); } - private void closeConsumer(final Consumer consumer) { + private void closeConsumer(final Consumer<?, ?> consumer) { + consumerClosedCountRef.incrementAndGet(); try { consumer.unsubscribe(); } catch (Exception e) { @@ -219,15 +159,55 @@ public class ConsumerPool implements Closeable { try { consumer.close(); - consumerClosedCountRef.incrementAndGet(); } catch (Exception e) { - consumerClosedCountRef.incrementAndGet(); logger.warn("Failed while closing " + consumer, e); } } PoolStats getPoolStats() { - return new PoolStats(consumerCreatedCountRef.get(), consumerClosedCountRef.get(), leasesObtainedCountRef.get(), productivePollCountRef.get(), unproductivePollCountRef.get()); + return new PoolStats(consumerCreatedCountRef.get(), consumerClosedCountRef.get(), leasesObtainedCountRef.get()); + } + + private class SimpleConsumerLease extends ConsumerLease { + + private final Consumer<byte[], byte[]> consumer; + private volatile ProcessSession session; + private volatile boolean closedConsumer; + + private SimpleConsumerLease(final Consumer<byte[], byte[]> consumer) { + super(maxWaitMillis, consumer, demarcatorBytes, keyEncoding, securityProtocol, bootstrapServers, logger); + this.consumer = consumer; + } + + void setProcessSession(final ProcessSession session) { + this.session = session; + } + + @Override + public ProcessSession getProcessSession() { + return session; + } + + @Override + public void close() { + super.close(); + close(false); + } + + public void close(final boolean forceClose) { + if (closedConsumer) { + return; + } + super.close(); + if (session != null) { + session.rollback(); + setProcessSession(null); + } + if (forceClose || isPoisoned() || !pooledLeases.offer(this)) { + closedConsumer = true; + closeConsumer(consumer); + } + } } static final class PoolStats { @@ -235,30 +215,22 @@ public class ConsumerPool implements Closeable { final long consumerCreatedCount; final long consumerClosedCount; final long leasesObtainedCount; - final long productivePollCount; - final long unproductivePollCount; PoolStats( final long consumerCreatedCount, final long consumerClosedCount, - final long leasesObtainedCount, - final long productivePollCount, - final long unproductivePollCount + final long leasesObtainedCount ) { this.consumerCreatedCount = consumerCreatedCount; this.consumerClosedCount = consumerClosedCount; this.leasesObtainedCount = leasesObtainedCount; - this.productivePollCount = productivePollCount; - this.unproductivePollCount = unproductivePollCount; } @Override public String toString() { return "Created Consumers [" + consumerCreatedCount + "]\n" + "Closed Consumers [" + consumerClosedCount + "]\n" - + "Leases Obtained [" + leasesObtainedCount + "]\n" - + "Productive Polls [" + productivePollCount + "]\n" - + "Unproductive Polls [" + unproductivePollCount + "]\n"; + + "Leases Obtained [" + leasesObtainedCount + "]\n"; } }