apex-dev mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From tweise <...@git.apache.org>
Subject [GitHub] incubator-apex-malhar pull request: New Kafka input operator using...
Date Sun, 27 Dec 2015 07:09:13 GMT
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48452276
  
    --- Diff: kafka/src/main/java/org/apache/apex/malhar/kafka/KafkaConsumerWrapper.java ---
    @@ -0,0 +1,408 @@
    +/**
    + * Licensed to the Apache Software Foundation (ASF) under one
    + * or more contributor license agreements.  See the NOTICE file
    + * distributed with this work for additional information
    + * regarding copyright ownership.  The ASF licenses this file
    + * to you under the Apache License, Version 2.0 (the
    + * "License"); you may not use this file except in compliance
    + * with the License.  You may obtain a copy of the License at
    + *
    + *   http://www.apache.org/licenses/LICENSE-2.0
    + *
    + * Unless required by applicable law or agreed to in writing,
    + * software distributed under the License is distributed on an
    + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
    + * KIND, either express or implied.  See the License for the
    + * specific language governing permissions and limitations
    + * under the License.
    + */
    +
    +package org.apache.apex.malhar.kafka;
    +
    +import java.io.Closeable;
    +import java.io.IOException;
    +import java.util.HashMap;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.concurrent.ArrayBlockingQueue;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +
    +import javax.validation.constraints.Pattern;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.io.IOUtils;
    +import org.apache.commons.lang3.tuple.Pair;
    +import org.apache.kafka.clients.consumer.ConsumerConfig;
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +import org.apache.kafka.clients.consumer.ConsumerRecords;
    +import org.apache.kafka.clients.consumer.KafkaConsumer;
    +import org.apache.kafka.clients.consumer.NoOffsetForPartitionException;
    +import org.apache.kafka.clients.consumer.OffsetAndMetadata;
    +import org.apache.kafka.clients.consumer.OffsetCommitCallback;
    +import org.apache.kafka.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.TopicPartition;
    +import org.apache.kafka.common.errors.WakeupException;
    +
    +import com.google.common.base.Joiner;
    +import com.google.common.util.concurrent.ThreadFactoryBuilder;
    +
    +/**
    + * This is the wrapper class for new Kafka consumer API
    + *
    + * It starts number of consumers(one for each cluster) in same number of threads.
    + * Maintains the consumer offsets
    + *
    + * It also use the consumers to commit the application processed offsets along with the
application name
    + *
    + * @param <K> The key object, needs to be used with http://kafka.apache.org/090/javadoc/org/apache/kafka/clients/consumer/ConsumerConfig.html#KEY_DESERIALIZER_CLASS_CONFIG
    + * @param <V> The value object, needs to be used with http://kafka.apache.org/090/javadoc/org/apache/kafka/clients/consumer/ConsumerConfig.html#VALUE_DESERIALIZER_CLASS_CONFIG
    + */
    +public class KafkaConsumerWrapper<K, V> implements Closeable
    +{
    +
    +  private static final Logger logger = LoggerFactory.getLogger(KafkaConsumerWrapper.class);
    +
    +  /**
    +   * number of messages in buffer
    +   */
    +  private int cacheSize = 1024;
    +
    +  /**
    +   * consumer timeout
    +   */
    +  private long timeout = 5000;
    +
    +  private boolean isAlive = false;
    +
    +  private transient Map<String, KafkaConsumer<K, V>> consumers = new HashMap<>();
    +
    +  // The in memory buffer hold consumed messages
    +  private transient ArrayBlockingQueue<Pair<String, ConsumerRecord<K, V>>>
holdingBuffer;
    +
    +
    +  /**
    +   * refer to AbstractKafkaInputOperator.initialOffset
    +   */
    +  @Pattern(flags = {Pattern.Flag.CASE_INSENSITIVE},
    +      regexp = "earliest|latest|application_or_earliest|application_or_latest")
    +  private String initialOffset = "latest";
    +
    +  private transient AbstractKafkaInputOperator<K, V> ownerOperator = null;
    +
    +  private transient ExecutorService kafkaConsumerExecutor;
    +
    +  private transient Map<String, Map<TopicPartition, OffsetAndMetadata>> offsetsToCommit
= new HashMap<>();
    +
    +  /**
    +   * You can customize the offsetCommit
    +   */
    +  private transient OffsetCommitCallback offsetCommitCallback = new OffsetCommitCallback()
    +  {
    +    @Override
    +    public void onComplete(Map<TopicPartition, OffsetAndMetadata> map, Exception
e)
    +    {
    +      if (logger.isDebugEnabled()) {
    +        logger.debug("Commit offsets complete {} ", Joiner.on(';').withKeyValueSeparator("=").join(map));
    +      }
    +      if (e != null) {
    +        logger.warn("Exceptions in committing offsets {} : {} ",
    +            Joiner.on(';').withKeyValueSeparator("=").join(map), e);
    +      }
    +    }
    +  };
    +
    +  /**
    +   *
    +   * Only put the offset needs to be committed in the ConsumerThread.offsetToCommit map
    +   * The consumer thread will commit the offset(s)
    +   *
    +   * @param offsetsInWindow
    +   */
    +  public void commitOffsets(Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetsInWindow)
    +  {
    +    if (offsetsInWindow == null) {
    +      return;
    +    }
    +
    +    // group offsets by cluster and topic partition
    +    for (Map.Entry<AbstractKafkaPartitioner.PartitionMeta, Long> e : offsetsInWindow.entrySet())
{
    +      String cluster = e.getKey().getCluster();
    +      Map<TopicPartition, OffsetAndMetadata> topicPartitionOffsetMap = offsetsToCommit.get(cluster);
    +      if (topicPartitionOffsetMap == null) {
    +        logger.warn("committed offset map should be initialized by consumer thread!");
    +        continue;
    +      }
    +      topicPartitionOffsetMap.put(e.getKey().getTopicPartition(), new OffsetAndMetadata(e.getValue()));
    +    }
    +
    +  }
    +
    +
    +  static final class ConsumerThread<K, V> implements Runnable
    +  {
    +
    +    private KafkaConsumer consumer;
    +
    +    private String cluster;
    +
    +    private KafkaConsumerWrapper wrapper;
    +
    +    private Map<TopicPartition, OffsetAndMetadata> offsetToCommit = null;
    +
    +    public ConsumerThread(String cluster, KafkaConsumer consumer, KafkaConsumerWrapper
wrapper)
    +    {
    +      this.cluster = cluster;
    +      this.consumer = consumer;
    +      this.wrapper = wrapper;
    +      this.offsetToCommit = new ConcurrentHashMap<>();
    +      wrapper.offsetsToCommit.put(cluster, offsetToCommit);
    +    }
    +
    +    @Override
    +    public void run()
    +    {
    +      try {
    +
    +
    +        while (wrapper.isAlive) {
    +          if (!this.offsetToCommit.isEmpty()) {
    +            // in each fetch cycle commit the offset if needed
    +            if (logger.isDebugEnabled()) {
    +              logger.debug("Commit offsets {}", Joiner.on(';').withKeyValueSeparator("=").join(this.offsetToCommit));
    +            }
    +            consumer.commitAsync(offsetToCommit, wrapper.getOffsetCommitCallback());
    +            offsetToCommit.clear();
    +          }
    +          try {
    +            ConsumerRecords records = consumer.poll(wrapper.timeout);
    +            for (ConsumerRecord<K, V> record : (Iterable<ConsumerRecord<K,
V>>)records) {
    +              wrapper.putMessage(Pair.of(cluster, record));
    +            }
    +          } catch (NoOffsetForPartitionException e) {
    +            // if initialOffset is set to EARLIST or LATEST
    +            // and the application is run as first time
    +            // then there is no existing committed offset and this error will be caught
    +            // we need to seek to either beginning or end of the partition
    +            // based on the initial offset setting
    +            if (wrapper.ownerOperator.getInitialOffset() ==
    +                AbstractKafkaInputOperator.InitialOffset.APPLICATION_OR_EARLIEST
    +                || wrapper.ownerOperator.getInitialOffset() == AbstractKafkaInputOperator.InitialOffset.EARLIEST)
{
    +              consumer.seekToBeginning(e.partitions().toArray(new TopicPartition[0]));
    +            } else {
    +              consumer.seekToEnd(e.partitions().toArray(new TopicPartition[0]));
    +            }
    +          } catch (InterruptedException e) {
    +            throw new IllegalStateException("Consumer thread is interrupted unexpectedly",
e);
    +          }
    +        }
    +      } catch (WakeupException we) {
    +        logger.info("The consumer is being stopped");
    +      } finally {
    +        consumer.close();
    +      }
    +    }
    +  }
    +
    +
    +  /**
    +   * This method is called in setup method of Abstract Kafka Input Operator
    +   */
    +  public void create(AbstractKafkaInputOperator ownerOperator)
    +  {
    +    holdingBuffer = new ArrayBlockingQueue<>(cacheSize);
    +    this.ownerOperator = ownerOperator;
    +    logger.info("Create consumer wrapper with holding buffer size: {} ", cacheSize);
    +    if (logger.isInfoEnabled()) {
    +      logger.info("Assignments are {} ", Joiner.on('\n').join(ownerOperator.getAssignment()));
    +    }
    +  }
    +
    +
    +  /**
    +   * This method is called in the activate method of the operator
    +   */
    +  public void start()
    +  {
    +    isAlive = true;
    +
    +
    +    // thread to consume the kafka data
    +    // create thread pool for consumer threads
    +    kafkaConsumerExecutor = Executors.newCachedThreadPool(
    +      new ThreadFactoryBuilder().setNameFormat("kafka-consumer-%d").build());
    +
    +    // group list of PartitionMeta by cluster
    +    Map<String, List<TopicPartition>> consumerAssignment = new HashMap<>();
    +    for (AbstractKafkaPartitioner.PartitionMeta partitionMeta : ownerOperator.getAssignment())
{
    +      String cluster = partitionMeta.getCluster();
    +      List<TopicPartition> cAssignment = consumerAssignment.get(cluster);
    +      if (cAssignment == null) {
    +        cAssignment = new LinkedList<>();
    +        consumerAssignment.put(cluster, cAssignment);
    +      }
    +      cAssignment.add(new TopicPartition(partitionMeta.getTopic(), partitionMeta.getPartitionId()));
    +    }
    +
    +    Map<AbstractKafkaPartitioner.PartitionMeta, Long> currentOffset = ownerOperator.getOffsetTrack();
    +
    +
    +    //  create one thread for each cluster
    +    // each thread use one KafkaConsumer to consume from 1+ partition(s) of 1+ topic(s)
    +    for (Map.Entry<String, List<TopicPartition>> e : consumerAssignment.entrySet())
{
    +
    +      Properties prop = new Properties();
    +      if (ownerOperator.getConsumerProps() != null) {
    +        prop.putAll(ownerOperator.getConsumerProps());
    +      }
    +
    +      prop.put(ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG, e.getKey());
    +      prop.put(ConsumerConfig.AUTO_OFFSET_RESET_CONFIG, "none");
    +      // never auto commit the offsets
    +      prop.put(ConsumerConfig.ENABLE_AUTO_COMMIT_CONFIG, "false");
    +      if (ownerOperator.getKeyDeserializer() != null) {
    +        prop.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ownerOperator.getKeyDeserializer().getCanonicalName());
    +      }
    +      if (ownerOperator.getValueDeserializer() != null) {
    +        prop.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ownerOperator.getValueDeserializer().getCanonicalName());
    +      }
    +      AbstractKafkaInputOperator.InitialOffset initialOffset = ownerOperator.getInitialOffset();
    +
    +      if (initialOffset == AbstractKafkaInputOperator.InitialOffset.APPLICATION_OR_EARLIEST
||
    +          initialOffset == AbstractKafkaInputOperator.InitialOffset.APPLICATION_OR_LATEST)
{
    +        // commit the offset with application name if we set initialoffset to application
    +        prop.put(ConsumerConfig.GROUP_ID_CONFIG, ownerOperator.getApplicationName() +
"_Consumer");
    +      }
    +
    +      KafkaConsumer kc = new KafkaConsumer(prop);
    +      kc.assign(e.getValue());
    +      if (logger.isInfoEnabled()) {
    +        logger.info("Create consumer with properties {} ", Joiner.on(";").withKeyValueSeparator("=").join(prop));
    +        logger.info("Assign consumer to {}", Joiner.on('#').join(e.getValue()));
    +      }
    +      if (currentOffset != null && !currentOffset.isEmpty()) {
    +        for (TopicPartition tp : e.getValue()) {
    +          AbstractKafkaPartitioner.PartitionMeta partitionKey =
    +              new AbstractKafkaPartitioner.PartitionMeta(e.getKey(), tp.topic(), tp.partition());
    +          if (currentOffset.containsKey(partitionKey)) {
    +            kc.seek(tp, currentOffset.get(partitionKey));
    +          }
    +        }
    +      }
    +
    +      consumers.put(e.getKey(), kc);
    +      kafkaConsumerExecutor.submit(new ConsumerThread(e.getKey(), kc, this));
    +    }
    +
    +
    +  }
    +
    +  /**
    +   * The method is called in the deactivate method of the operator
    +   */
    +  public void stop()
    +  {
    +    for (KafkaConsumer<K, V> c : consumers.values()) {
    +      c.wakeup();
    +    }
    +    kafkaConsumerExecutor.shutdownNow();
    +    isAlive = false;
    +    holdingBuffer.clear();
    +    IOUtils.closeQuietly(this);
    +  }
    +
    +  /**
    +   * This method is called in teardown method of the operator
    +   */
    +  public void teardown()
    +  {
    +    holdingBuffer.clear();
    +  }
    +
    +  public boolean isAlive()
    +  {
    +    return isAlive;
    +  }
    +
    +  public void setAlive(boolean isAlive)
    +  {
    +    this.isAlive = isAlive;
    +  }
    +
    +  public void setTimeout(long timeout)
    +  {
    +    this.timeout = timeout;
    +  }
    +
    +  public long getTimeout()
    +  {
    +    return timeout;
    +  }
    +
    +  public Pair<String, ConsumerRecord<K, V>> pollMessage()
    +  {
    +    return holdingBuffer.poll();
    +  }
    +
    +  public int messageSize()
    +  {
    +    return holdingBuffer.size();
    +  }
    +
    +  public void setInitialOffset(String initialOffset)
    +  {
    +    this.initialOffset = initialOffset;
    +  }
    +
    +  public String getInitialOffset()
    +  {
    +    return initialOffset;
    +  }
    +
    +  public int getCacheSize()
    +  {
    +    return cacheSize;
    +  }
    +
    +  public void setCacheSize(int cacheSize)
    +  {
    +    this.cacheSize = cacheSize;
    +  }
    +
    +  public void setOffsetCommitCallback(OffsetCommitCallback offsetCommitCallback)
    --- End diff --
    
    Can't find a reference, what's the purpose of this API?


---
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 infrastructure@apache.org or file a JIRA ticket
with INFRA.
---

Mime
View raw message