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

    https://github.com/apache/nifi/pull/295#discussion_r57015759
  
    --- Diff: 
nifi-nar-bundles/nifi-kafka-bundle/nifi-kafka-processors/src/main/java/org/apache/nifi/processors/kafka/KafkaPublisher.java
 ---
    @@ -0,0 +1,159 @@
    +/*
    + * 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;
    +
    +import java.io.InputStream;
    +import java.util.ArrayList;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.Scanner;
    +
    +import org.apache.kafka.clients.producer.KafkaProducer;
    +import org.apache.nifi.flowfile.FlowFile;
    +import org.apache.nifi.logging.ProcessorLog;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import kafka.javaapi.producer.Producer;
    +import kafka.producer.KeyedMessage;
    +import kafka.producer.Partitioner;
    +import kafka.producer.ProducerConfig;
    +
    +/**
    + * Wrapper over {@link KafkaProducer} to assist {@link PutKafka} processor 
with
    + * sending content of {@link FlowFile}s to Kafka.
    + */
    +public class KafkaPublisher implements AutoCloseable {
    +
    +    private static final Logger logger = 
LoggerFactory.getLogger(KafkaPublisher.class);
    +
    +    private final Producer<byte[], byte[]> producer;
    +
    +    private ProcessorLog processLog;
    +
    +    /**
    +     * Creates an instance of this class as well as the instance of the
    +     * corresponding Kafka {@link KafkaProducer} using provided Kafka
    +     * configuration properties.
    +     */
    +    KafkaPublisher(Properties kafkaProperties) {
    +        ProducerConfig producerConfig = new 
ProducerConfig(kafkaProperties);
    +        this.producer = new Producer<>(producerConfig);
    +    }
    +
    +    /**
    +     *
    +     */
    +    void setProcessLog(ProcessorLog processLog) {
    +        this.processLog = processLog;
    +    }
    +
    +    /**
    +     * Publishes messages to Kafka topic. It supports three publishing
    +     * mechanisms.
    +     * <ul>
    +     * <li>Sending the entire content stream as a single Kafka 
message.</li>
    +     * <li>Splitting the incoming content stream into chunks and sending
    +     * individual chunks as separate Kafka messages.</li>
    +     * <li>Splitting the incoming content stream into chunks and sending 
only
    +     * the chunks that have failed previously @see
    +     * {@link SplittableMessageContext#getFailedSegments()}.</li>
    +     * </ul>
    +     * This method assumes content stream affinity where it is expected 
that the
    +     * content stream that represents the same Kafka message(s) will 
remain the
    +     * same across possible retries. This is required specifically for 
cases
    +     * where delimiter is used and a single content stream may represent
    +     * multiple Kafka messages. The failed segment list will keep the 
index of
    +     * of each content stream segment that had failed to be sent to Kafka, 
so
    +     * upon retry only the failed segments are sent.
    +     *
    +     * @param messageContext
    +     *            instance of {@link SplittableMessageContext} which hold
    +     *            context information about the message to be sent
    +     * @param contentStream
    +     *            instance of open {@link InputStream} carrying the 
content of
    +     *            the message(s) to be send to Kafka
    +     * @param partitionKey
    +     *            the value of the partition key. Only relevant is user 
wishes
    +     *            to provide a custom partition key instead of relying on
    +     *            variety of provided {@link Partitioner}(s)
    +     * @return The list containing the failed segment indexes for messages 
that
    +     *         failed to be sent to Kafka.
    +     */
    +    List<Integer> publish(SplittableMessageContext messageContext, 
InputStream contentStream, Object partitionKey) {
    +        List<Integer> prevFailedSegmentIndexes = 
messageContext.getFailedSegments();
    --- End diff --
    
    Rather than List<Integer> here, did you consider using a BitSet? Would be a 
lot less heap-intensive if there are a lot of things added, and would certainly 
give better results when calling contains() below.


---
If your project is set up for it, you can reply to this email and have your
reply appear on GitHub as well. If your project does not have this feature
enabled and wishes so, or if the feature is enabled but not working, please
contact infrastructure at infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---

Reply via email to