[jira] [Commented] (NIFI-2865) Address issues of PublishKafka blocking when having trouble communicating with Kafka broker and improve performance

2016-10-06 Thread ASF subversion and git services (JIRA)

[ 
https://issues.apache.org/jira/browse/NIFI-2865?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15553053#comment-15553053
 ] 

ASF subversion and git services commented on NIFI-2865:
---

Commit b9cb6b1b475eb4688b7cd32f6d343c5dffb20567 in nifi's branch 
refs/heads/master from [~markap14]
[ https://git-wip-us.apache.org/repos/asf?p=nifi.git;h=b9cb6b1 ]

NIFI-2865: Refactored PublishKafka and PublishKafka_0_10 to allow batching of 
FlowFiles within a single publish and to let messages timeout if not 
acknowledged

Signed-off-by: Bryan Bende 


> Address issues of PublishKafka blocking when having trouble communicating 
> with Kafka broker and improve performance
> ---
>
> Key: NIFI-2865
> URL: https://issues.apache.org/jira/browse/NIFI-2865
> Project: Apache NiFi
>  Issue Type: Bug
>  Components: Extensions
>Reporter: Mark Payne
>Assignee: Mark Payne
> Fix For: 1.1.0
>
>
> When NiFi is unable to communicate properly with the Kafka broker, we see the 
> NiFi threads occasionally block. This should be resolvable by calling the 
> wakeup() method of the client. Additionally, if Kafka takes too long to 
> respond, we should be able to route the FlowFile to failure and move on.
> PublishKafka has a nice feature that allows a demarcated stream to be sent as 
> separate messages, so that a large number of messages can be sent as a single 
> FlowFile. However, in the case of individual messages per FlowFile, the 
> performance could be improved by batching together multiple FlowFiles per 
> session



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (NIFI-2865) Address issues of PublishKafka blocking when having trouble communicating with Kafka broker and improve performance

2016-10-06 Thread ASF subversion and git services (JIRA)

[ 
https://issues.apache.org/jira/browse/NIFI-2865?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15553054#comment-15553054
 ] 

ASF subversion and git services commented on NIFI-2865:
---

Commit 9304df4de060335526d29a77aa093db4004c8b2e in nifi's branch 
refs/heads/master from [~markap14]
[ https://git-wip-us.apache.org/repos/asf?p=nifi.git;h=9304df4 ]

NIFI-2865: Fixed bug in StreamDemarcator that is exposed when the final bit of 
data in a stream is smaller than the previous and the previous demarcation 
ended on a buffer length boundary

This closes #1110.

Signed-off-by: Bryan Bende 


> Address issues of PublishKafka blocking when having trouble communicating 
> with Kafka broker and improve performance
> ---
>
> Key: NIFI-2865
> URL: https://issues.apache.org/jira/browse/NIFI-2865
> Project: Apache NiFi
>  Issue Type: Bug
>  Components: Extensions
>Reporter: Mark Payne
>Assignee: Mark Payne
> Fix For: 1.1.0
>
>
> When NiFi is unable to communicate properly with the Kafka broker, we see the 
> NiFi threads occasionally block. This should be resolvable by calling the 
> wakeup() method of the client. Additionally, if Kafka takes too long to 
> respond, we should be able to route the FlowFile to failure and move on.
> PublishKafka has a nice feature that allows a demarcated stream to be sent as 
> separate messages, so that a large number of messages can be sent as a single 
> FlowFile. However, in the case of individual messages per FlowFile, the 
> performance could be improved by batching together multiple FlowFiles per 
> session



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (NIFI-2865) Address issues of PublishKafka blocking when having trouble communicating with Kafka broker and improve performance

2016-10-06 Thread Bryan Bende (JIRA)

[ 
https://issues.apache.org/jira/browse/NIFI-2865?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15552908#comment-15552908
 ] 

Bryan Bende commented on NIFI-2865:
---

Reverted this commit on master due to a failing test with stream demarcator 
that exposed a bug that needs to be resolved.

> Address issues of PublishKafka blocking when having trouble communicating 
> with Kafka broker and improve performance
> ---
>
> Key: NIFI-2865
> URL: https://issues.apache.org/jira/browse/NIFI-2865
> Project: Apache NiFi
>  Issue Type: Bug
>  Components: Extensions
>Reporter: Mark Payne
>Assignee: Mark Payne
> Fix For: 1.1.0
>
>
> When NiFi is unable to communicate properly with the Kafka broker, we see the 
> NiFi threads occasionally block. This should be resolvable by calling the 
> wakeup() method of the client. Additionally, if Kafka takes too long to 
> respond, we should be able to route the FlowFile to failure and move on.
> PublishKafka has a nice feature that allows a demarcated stream to be sent as 
> separate messages, so that a large number of messages can be sent as a single 
> FlowFile. However, in the case of individual messages per FlowFile, the 
> performance could be improved by batching together multiple FlowFiles per 
> session



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (NIFI-2865) Address issues of PublishKafka blocking when having trouble communicating with Kafka broker and improve performance

2016-10-06 Thread ASF subversion and git services (JIRA)

[ 
https://issues.apache.org/jira/browse/NIFI-2865?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15552904#comment-15552904
 ] 

ASF subversion and git services commented on NIFI-2865:
---

Commit a4ed622152187155463af2b748c9bf492621bbc7 in nifi's branch 
refs/heads/master from [~bbende]
[ https://git-wip-us.apache.org/repos/asf?p=nifi.git;h=a4ed622 ]

Revert "NIFI-2865: Refactored PublishKafka and PublishKafka_0_10 to allow 
batching of FlowFiles within a single publish and to let messages timeout if 
not acknowledged"

This reverts commit 92cca96d49042f9898f93b3a2d2210b924708e52.


> Address issues of PublishKafka blocking when having trouble communicating 
> with Kafka broker and improve performance
> ---
>
> Key: NIFI-2865
> URL: https://issues.apache.org/jira/browse/NIFI-2865
> Project: Apache NiFi
>  Issue Type: Bug
>  Components: Extensions
>Reporter: Mark Payne
>Assignee: Mark Payne
> Fix For: 1.1.0
>
>
> When NiFi is unable to communicate properly with the Kafka broker, we see the 
> NiFi threads occasionally block. This should be resolvable by calling the 
> wakeup() method of the client. Additionally, if Kafka takes too long to 
> respond, we should be able to route the FlowFile to failure and move on.
> PublishKafka has a nice feature that allows a demarcated stream to be sent as 
> separate messages, so that a large number of messages can be sent as a single 
> FlowFile. However, in the case of individual messages per FlowFile, the 
> performance could be improved by batching together multiple FlowFiles per 
> session



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (NIFI-2865) Address issues of PublishKafka blocking when having trouble communicating with Kafka broker and improve performance

2016-10-06 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/NIFI-2865?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15552020#comment-15552020
 ] 

ASF GitHub Bot commented on NIFI-2865:
--

Github user asfgit closed the pull request at:

https://github.com/apache/nifi/pull/1097


> Address issues of PublishKafka blocking when having trouble communicating 
> with Kafka broker and improve performance
> ---
>
> Key: NIFI-2865
> URL: https://issues.apache.org/jira/browse/NIFI-2865
> Project: Apache NiFi
>  Issue Type: Bug
>  Components: Extensions
>Reporter: Mark Payne
>Assignee: Mark Payne
> Fix For: 1.1.0
>
>
> When NiFi is unable to communicate properly with the Kafka broker, we see the 
> NiFi threads occasionally block. This should be resolvable by calling the 
> wakeup() method of the client. Additionally, if Kafka takes too long to 
> respond, we should be able to route the FlowFile to failure and move on.
> PublishKafka has a nice feature that allows a demarcated stream to be sent as 
> separate messages, so that a large number of messages can be sent as a single 
> FlowFile. However, in the case of individual messages per FlowFile, the 
> performance could be improved by batching together multiple FlowFiles per 
> session



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (NIFI-2865) Address issues of PublishKafka blocking when having trouble communicating with Kafka broker and improve performance

2016-10-06 Thread ASF subversion and git services (JIRA)

[ 
https://issues.apache.org/jira/browse/NIFI-2865?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15552019#comment-15552019
 ] 

ASF subversion and git services commented on NIFI-2865:
---

Commit 92cca96d49042f9898f93b3a2d2210b924708e52 in nifi's branch 
refs/heads/master from [~markap14]
[ https://git-wip-us.apache.org/repos/asf?p=nifi.git;h=92cca96 ]

NIFI-2865: Refactored PublishKafka and PublishKafka_0_10 to allow batching of 
FlowFiles within a single publish and to let messages timeout if not 
acknowledged

This closes #1097.

Signed-off-by: Bryan Bende 


> Address issues of PublishKafka blocking when having trouble communicating 
> with Kafka broker and improve performance
> ---
>
> Key: NIFI-2865
> URL: https://issues.apache.org/jira/browse/NIFI-2865
> Project: Apache NiFi
>  Issue Type: Bug
>  Components: Extensions
>Reporter: Mark Payne
>Assignee: Mark Payne
> Fix For: 1.1.0
>
>
> When NiFi is unable to communicate properly with the Kafka broker, we see the 
> NiFi threads occasionally block. This should be resolvable by calling the 
> wakeup() method of the client. Additionally, if Kafka takes too long to 
> respond, we should be able to route the FlowFile to failure and move on.
> PublishKafka has a nice feature that allows a demarcated stream to be sent as 
> separate messages, so that a large number of messages can be sent as a single 
> FlowFile. However, in the case of individual messages per FlowFile, the 
> performance could be improved by batching together multiple FlowFiles per 
> session



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (NIFI-2865) Address issues of PublishKafka blocking when having trouble communicating with Kafka broker and improve performance

2016-10-06 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/NIFI-2865?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15552008#comment-15552008
 ] 

ASF GitHub Bot commented on NIFI-2865:
--

Github user markap14 commented on a diff in the pull request:

https://github.com/apache/nifi/pull/1097#discussion_r82194639
  
--- Diff: 
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java
 ---
@@ -0,0 +1,132 @@
+/*
+ * 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 java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.stream.io.exception.TokenTooLargeException;
+import org.apache.nifi.stream.io.util.StreamDemarcator;
+
+public class PublisherLease implements Closeable {
+private final ComponentLog logger;
+private final Producer producer;
+private final int maxMessageSize;
+private final long maxAckWaitMillis;
+private volatile boolean poisoned = false;
+
+private InFlightMessageTracker tracker;
+
+public PublisherLease(final Producer producer, final 
int maxMessageSize, final long maxAckWaitMillis, final ComponentLog logger) {
+this.producer = producer;
+this.maxMessageSize = maxMessageSize;
+this.logger = logger;
+this.maxAckWaitMillis = maxAckWaitMillis;
+}
+
+protected void poison() {
+this.poisoned = true;
+}
+
+public boolean isPoisoned() {
+return poisoned;
+}
+
+void publish(final FlowFile flowFile, final InputStream 
flowFileContent, final byte[] messageKey, final byte[] demarcatorBytes, final 
String topic) throws IOException {
+if (tracker == null) {
--- End diff --

We probably could... but I held off on doing that, because it may get 
poisoned in a background thread, and the caller may not know when calling 
publish() that it was poisoned. If that happens, the new FlowFile is likely to 
fail anyway. If it doesn't then it's okay - there's no reason it can't get 
published to Kafka at that point. We simply poison the lease to ensure that a 
new one gets created, in case there's an issue with the connection.


> Address issues of PublishKafka blocking when having trouble communicating 
> with Kafka broker and improve performance
> ---
>
> Key: NIFI-2865
> URL: https://issues.apache.org/jira/browse/NIFI-2865
> Project: Apache NiFi
>  Issue Type: Bug
>  Components: Extensions
>Reporter: Mark Payne
>Assignee: Mark Payne
> Fix For: 1.1.0
>
>
> When NiFi is unable to communicate properly with the Kafka broker, we see the 
> NiFi threads occasionally block. This should be resolvable by calling the 
> wakeup() method of the client. Additionally, if Kafka takes too long to 
> respond, we should be able to route the FlowFile to failure and move on.
> PublishKafka has a nice feature that allows a demarcated stream to be sent as 
> separate messages, so that a large number of messages can be sent as a single 
> FlowFile. However, in the case of individual messages per FlowFile, the 
> performance could be improved by batching together multiple FlowFiles per 
> session



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (NIFI-2865) Address issues of PublishKafka blocking when having trouble communicating with Kafka broker and improve performance

2016-10-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/NIFI-2865?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15550068#comment-15550068
 ] 

ASF GitHub Bot commented on NIFI-2865:
--

Github user bbende commented on a diff in the pull request:

https://github.com/apache/nifi/pull/1097#discussion_r82078303
  
--- Diff: 
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublisherLease.java
 ---
@@ -0,0 +1,132 @@
+/*
+ * 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 java.io.Closeable;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.TimeoutException;
+
+import org.apache.kafka.clients.producer.Callback;
+import org.apache.kafka.clients.producer.Producer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.apache.nifi.flowfile.FlowFile;
+import org.apache.nifi.logging.ComponentLog;
+import org.apache.nifi.stream.io.exception.TokenTooLargeException;
+import org.apache.nifi.stream.io.util.StreamDemarcator;
+
+public class PublisherLease implements Closeable {
+private final ComponentLog logger;
+private final Producer producer;
+private final int maxMessageSize;
+private final long maxAckWaitMillis;
+private volatile boolean poisoned = false;
+
+private InFlightMessageTracker tracker;
+
+public PublisherLease(final Producer producer, final 
int maxMessageSize, final long maxAckWaitMillis, final ComponentLog logger) {
+this.producer = producer;
+this.maxMessageSize = maxMessageSize;
+this.logger = logger;
+this.maxAckWaitMillis = maxAckWaitMillis;
+}
+
+protected void poison() {
+this.poisoned = true;
+}
+
+public boolean isPoisoned() {
+return poisoned;
+}
+
+void publish(final FlowFile flowFile, final InputStream 
flowFileContent, final byte[] messageKey, final byte[] demarcatorBytes, final 
String topic) throws IOException {
+if (tracker == null) {
--- End diff --

Not sure if this is necessary, but would we want to check if poisoned here 
and throw an exception if publish is being called after being poisoned?


> Address issues of PublishKafka blocking when having trouble communicating 
> with Kafka broker and improve performance
> ---
>
> Key: NIFI-2865
> URL: https://issues.apache.org/jira/browse/NIFI-2865
> Project: Apache NiFi
>  Issue Type: Bug
>  Components: Extensions
>Reporter: Mark Payne
>Assignee: Mark Payne
> Fix For: 1.1.0
>
>
> When NiFi is unable to communicate properly with the Kafka broker, we see the 
> NiFi threads occasionally block. This should be resolvable by calling the 
> wakeup() method of the client. Additionally, if Kafka takes too long to 
> respond, we should be able to route the FlowFile to failure and move on.
> PublishKafka has a nice feature that allows a demarcated stream to be sent as 
> separate messages, so that a large number of messages can be sent as a single 
> FlowFile. However, in the case of individual messages per FlowFile, the 
> performance could be improved by batching together multiple FlowFiles per 
> session



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)


[jira] [Commented] (NIFI-2865) Address issues of PublishKafka blocking when having trouble communicating with Kafka broker and improve performance

2016-10-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/NIFI-2865?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15549944#comment-15549944
 ] 

ASF GitHub Bot commented on NIFI-2865:
--

Github user bbende commented on a diff in the pull request:

https://github.com/apache/nifi/pull/1097#discussion_r82070115
  
--- Diff: 
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-10-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka_0_10.java
 ---
@@ -249,242 +241,141 @@
 
 @Override
 protected List getSupportedPropertyDescriptors() {
-return DESCRIPTORS;
+return PROPERTIES;
 }
 
 @Override
 protected PropertyDescriptor 
getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
 return new PropertyDescriptor.Builder()
-.description("Specifies the value for '" + 
propertyDescriptorName + "' Kafka Configuration.")
-.name(propertyDescriptorName).addValidator(new 
KafkaProcessorUtils.KafkaConfigValidator(ProducerConfig.class)).dynamic(true)
-.build();
+.description("Specifies the value for '" + 
propertyDescriptorName + "' Kafka Configuration.")
+.name(propertyDescriptorName)
+.addValidator(new 
KafkaProcessorUtils.KafkaConfigValidator(ProducerConfig.class))
+.dynamic(true)
+.build();
 }
 
 @Override
 protected Collection customValidate(final 
ValidationContext validationContext) {
 return 
KafkaProcessorUtils.validateCommonProperties(validationContext);
 }
 
-volatile KafkaPublisher kafkaPublisher;
-
-/**
- * This thread-safe operation will delegate to
- * {@link #rendezvousWithKafka(ProcessContext, ProcessSession)} after 
first
- * checking and creating (if necessary) Kafka resource which could be 
either
- * {@link KafkaPublisher} or {@link KafkaConsumer}. It will also close 
and
- * destroy the underlying Kafka resource upon catching an {@link 
Exception}
- * raised by {@link #rendezvousWithKafka(ProcessContext, 
ProcessSession)}.
- * After Kafka resource is destroyed it will be re-created upon the 
next
- * invocation of this operation essentially providing a self healing
- * mechanism to deal with potentially corrupted resource.
- * 
- * Keep in mind that upon catching an exception the state of this 
processor
- * will be set to no longer accept any more tasks, until Kafka 
resource is
- * reset. This means that in a multi-threaded situation currently 
executing
- * tasks will be given a chance to complete while no new tasks will be
- * accepted.
- *
- * @param context context
- * @param sessionFactory factory
- */
-@Override
-public final void onTrigger(final ProcessContext context, final 
ProcessSessionFactory sessionFactory) throws ProcessException {
-if (this.acceptTask) { // acts as a circuit breaker to allow 
existing tasks to wind down so 'kafkaPublisher' can be reset before new tasks 
are accepted.
-this.taskCounter.incrementAndGet();
-final ProcessSession session = sessionFactory.createSession();
-try {
-/*
- * We can't be doing double null check here since as a 
pattern
- * it only works for lazy init but not reset, which is 
what we
- * are doing here. In fact the first null check is 
dangerous
- * since 'kafkaPublisher' can become null right after its 
null
- * check passed causing subsequent NPE.
- */
-synchronized (this) {
-if (this.kafkaPublisher == null) {
-this.kafkaPublisher = 
this.buildKafkaResource(context, session);
-}
-}
-
-/*
- * The 'processed' boolean flag does not imply any failure 
or success. It simply states that:
- * - ConsumeKafka - some messages were received form Kafka 
and 1_ FlowFile were generated
- * - PublishKafka0_10 - some messages were sent to Kafka 
based on existence of the input FlowFile
- */
-boolean processed = this.rendezvousWithKafka(context, 
session);
-session.commit();
-if (!processed) {
-context.yield();
-}
-} catch (Throwable e) {
-this.acceptTask = false;
-session.rollback(true);
-this.getLogger().error("{} 

[jira] [Commented] (NIFI-2865) Address issues of PublishKafka blocking when having trouble communicating with Kafka broker and improve performance

2016-10-05 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/NIFI-2865?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15549945#comment-15549945
 ] 

ASF GitHub Bot commented on NIFI-2865:
--

Github user bbende commented on a diff in the pull request:

https://github.com/apache/nifi/pull/1097#discussion_r82070354
  
--- Diff: 
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-0-9-processors/src/main/java/org/apache/nifi/processors/kafka/pubsub/PublishKafka.java
 ---
@@ -250,242 +241,141 @@
 
 @Override
 protected List getSupportedPropertyDescriptors() {
-return DESCRIPTORS;
+return PROPERTIES;
 }
 
 @Override
 protected PropertyDescriptor 
getSupportedDynamicPropertyDescriptor(final String propertyDescriptorName) {
 return new PropertyDescriptor.Builder()
-.description("Specifies the value for '" + 
propertyDescriptorName + "' Kafka Configuration.")
-.name(propertyDescriptorName).addValidator(new 
KafkaProcessorUtils.KafkaConfigValidator(ProducerConfig.class)).dynamic(true)
-.build();
+.description("Specifies the value for '" + 
propertyDescriptorName + "' Kafka Configuration.")
+.name(propertyDescriptorName)
+.addValidator(new 
KafkaProcessorUtils.KafkaConfigValidator(ProducerConfig.class))
+.dynamic(true)
+.build();
 }
 
 @Override
 protected Collection customValidate(final 
ValidationContext validationContext) {
 return 
KafkaProcessorUtils.validateCommonProperties(validationContext);
 }
 
-volatile KafkaPublisher kafkaPublisher;
-
-/**
- * This thread-safe operation will delegate to
- * {@link #rendezvousWithKafka(ProcessContext, ProcessSession)} after 
first
- * checking and creating (if necessary) Kafka resource which could be 
either
- * {@link KafkaPublisher} or {@link KafkaConsumer}. It will also close 
and
- * destroy the underlying Kafka resource upon catching an {@link 
Exception}
- * raised by {@link #rendezvousWithKafka(ProcessContext, 
ProcessSession)}.
- * After Kafka resource is destroyed it will be re-created upon the 
next
- * invocation of this operation essentially providing a self healing
- * mechanism to deal with potentially corrupted resource.
- * 
- * Keep in mind that upon catching an exception the state of this 
processor
- * will be set to no longer accept any more tasks, until Kafka 
resource is
- * reset. This means that in a multi-threaded situation currently 
executing
- * tasks will be given a chance to complete while no new tasks will be
- * accepted.
- *
- * @param context context
- * @param sessionFactory factory
- */
-@Override
-public final void onTrigger(final ProcessContext context, final 
ProcessSessionFactory sessionFactory) throws ProcessException {
-if (this.acceptTask) { // acts as a circuit breaker to allow 
existing tasks to wind down so 'kafkaPublisher' can be reset before new tasks 
are accepted.
-this.taskCounter.incrementAndGet();
-final ProcessSession session = sessionFactory.createSession();
-try {
-/*
- * We can't be doing double null check here since as a 
pattern
- * it only works for lazy init but not reset, which is 
what we
- * are doing here. In fact the first null check is 
dangerous
- * since 'kafkaPublisher' can become null right after its 
null
- * check passed causing subsequent NPE.
- */
-synchronized (this) {
-if (this.kafkaPublisher == null) {
-this.kafkaPublisher = 
this.buildKafkaResource(context, session);
-}
-}
-
-/*
- * The 'processed' boolean flag does not imply any failure 
or success. It simply states that:
- * - ConsumeKafka - some messages were received form Kafka 
and 1_ FlowFile were generated
- * - PublishKafka0_10 - some messages were sent to Kafka 
based on existence of the input FlowFile
- */
-boolean processed = this.rendezvousWithKafka(context, 
session);
-session.commit();
-if (!processed) {
-context.yield();
-}
-} catch (Throwable e) {
-this.acceptTask = false;
-session.rollback(true);
-this.getLogger().error("{} failed to 

[jira] [Commented] (NIFI-2865) Address issues of PublishKafka blocking when having trouble communicating with Kafka broker and improve performance

2016-10-04 Thread ASF GitHub Bot (JIRA)

[ 
https://issues.apache.org/jira/browse/NIFI-2865?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=15546630#comment-15546630
 ] 

ASF GitHub Bot commented on NIFI-2865:
--

GitHub user markap14 opened a pull request:

https://github.com/apache/nifi/pull/1097

NIFI-2865: Refactored PublishKafka and PublishKafka_0_10 to allow bat…

…ching of FlowFiles within a single publish and to let messages timeout if 
not acknowledged

You can merge this pull request into a Git repository by running:

$ git pull https://github.com/markap14/nifi NIFI-2865

Alternatively you can review and apply these changes as the patch at:

https://github.com/apache/nifi/pull/1097.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

This closes #1097


commit 5b10a50ff34150c2642b544e4ee7a855c080b285
Author: Mark Payne 
Date:   2016-09-08T23:37:35Z

NIFI-2865: Refactored PublishKafka and PublishKafka_0_10 to allow batching 
of FlowFiles within a single publish and to let messages timeout if not 
acknowledged




> Address issues of PublishKafka blocking when having trouble communicating 
> with Kafka broker and improve performance
> ---
>
> Key: NIFI-2865
> URL: https://issues.apache.org/jira/browse/NIFI-2865
> Project: Apache NiFi
>  Issue Type: Bug
>  Components: Extensions
>Reporter: Mark Payne
>Assignee: Mark Payne
> Fix For: 1.1.0
>
>
> When NiFi is unable to communicate properly with the Kafka broker, we see the 
> NiFi threads occasionally block. This should be resolvable by calling the 
> wakeup() method of the client. Additionally, if Kafka takes too long to 
> respond, we should be able to route the FlowFile to failure and move on.
> PublishKafka has a nice feature that allows a demarcated stream to be sent as 
> separate messages, so that a large number of messages can be sent as a single 
> FlowFile. However, in the case of individual messages per FlowFile, the 
> performance could be improved by batching together multiple FlowFiles per 
> session



--
This message was sent by Atlassian JIRA
(v6.3.4#6332)