You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@apex.apache.org by siyuanh <gi...@git.apache.org> on 2015/12/15 19:37:51 UTC

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

GitHub user siyuanh opened a pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134

    New Kafka input operator using 0.9.0 Kafka consumer API

    Some highlights of the operator
     1. Out-of-box One-to-one and one-to-many partition scheme support plus customizable partition schema
     2. Fault-tolerant when the input operator goes down, it redeploys on other node
     3. At-least-once semantics for operator failure (no matter which operator fails)
     4. At-least-once semantics for cold restart (no data loss even if you restart the application)
     5. Multi-cluster support, one operator can consume data from more than one kafka clusters
     6. Multi-topic support, one operator can subscribe multiple topics
     7. Throughput control support, you can throttle number of tuple for each streaming window
    
    
    Things needs to be done,
    More unit test for different partition scheme, application restart
    More integration test
    Metrics for operator
    Bring back idempotency 


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

    $ git pull https://github.com/siyuanh/incubator-apex-malhar MLHR-1904

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

    https://github.com/apache/incubator-apex-malhar/pull/134.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 #134
    
----
commit b68a8ce77b4515f51c58150cb167e6c495c6f62c
Author: Siyuan Hua <hs...@apache.org>
Date:   2015-12-15T18:35:30Z

    Initial commit of new Kafka input operator using 0.9.0 Kafka consumer 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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48647169
  
    --- Diff: kafka/src/main/java/org/apache/apex/malhar/kafka/KafkaConsumerWrapper.java ---
    @@ -0,0 +1,378 @@
    +/**
    + * 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.Set;
    +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.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.TopicPartition;
    +import org.apache.kafka.common.errors.WakeupException;
    +import org.apache.kafka.common.serialization.ByteArrayDeserializer;
    +
    +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 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
    --- End diff --
    
    Remove this?


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48920130
  
    --- Diff: kafka/src/main/java/org/apache/apex/malhar/kafka/KafkaConsumerWrapper.java ---
    @@ -0,0 +1,326 @@
    +/**
    + * 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.Set;
    +import java.util.concurrent.ArrayBlockingQueue;
    +import java.util.concurrent.ConcurrentHashMap;
    +import java.util.concurrent.ExecutorService;
    +import java.util.concurrent.Executors;
    +
    +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.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.TopicPartition;
    +import org.apache.kafka.common.errors.WakeupException;
    +import org.apache.kafka.common.serialization.ByteArrayDeserializer;
    +
    +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
    + *
    + */
    +public class KafkaConsumerWrapper implements Closeable
    +{
    +
    +  private static final Logger logger = LoggerFactory.getLogger(KafkaConsumerWrapper.class);
    +
    +  private boolean isAlive = false;
    +
    +  private Map<String, KafkaConsumer<byte[], byte[]>> consumers = new HashMap<>();
    +
    +  // The in memory buffer hold consumed messages
    +  private ArrayBlockingQueue<Pair<String, ConsumerRecord<byte[], byte[]>>> holdingBuffer;
    +
    +  private AbstractKafkaInputOperator ownerOperator = null;
    +
    +  private ExecutorService kafkaConsumerExecutor;
    +
    +  private Map<String, Map<TopicPartition, OffsetAndMetadata>> offsetsToCommit = new HashMap<>();
    +
    +  /**
    +   *
    +   * 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 implements Runnable
    +  {
    +
    +    private KafkaConsumer<byte[], byte[]> 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.ownerOperator);
    +            offsetToCommit.clear();
    +          }
    +          try {
    +            ConsumerRecords<byte[], byte[]> records = consumer.poll(wrapper.ownerOperator.getConsumerTimeout());
    +            for (ConsumerRecord<byte[], byte[]> record : 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<>(ownerOperator.getHoldingBufferSize());
    +    this.ownerOperator = ownerOperator;
    +    logger.info("Create consumer wrapper with holding buffer size: {} ", ownerOperator.getHoldingBufferSize());
    +    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<>();
    +    Set<AbstractKafkaPartitioner.PartitionMeta> assignments = ownerOperator.getAssignment();
    +    for (AbstractKafkaPartitioner.PartitionMeta partitionMeta : assignments) {
    +      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");
    +      prop.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
    +      prop.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
    +      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);
    --- End diff --
    
    rawtype warnings, elsewhere also


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by siyuanh <gi...@git.apache.org>.
Github user siyuanh commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48507106
  
    --- Diff: kafka/src/test/java/org/apache/apex/malhar/kafka/KafkaInputOperatorTest.java ---
    @@ -0,0 +1,221 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.Collection;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.concurrent.CountDownLatch;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.kafka.clients.consumer.ConsumerConfig;
    +import org.apache.kafka.common.serialization.Deserializer;
    +import org.apache.kafka.common.serialization.StringDeserializer;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.DAG.Locality;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.LocalMode;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +/**
    + * A bunch of test to verify the input operator will be automatically partitioned per kafka partition This test is launching its
    + * own Kafka cluster.
    + */
    +@RunWith(Parameterized.class)
    +public class KafkaInputOperatorTest extends KafkaOperatorTestBase
    +{
    +
    +  private int totalBrokers = 0;
    +
    +
    +
    +  @Parameterized.Parameters(name = "multi-cluster: {0}, multi-partition: {1}")
    +  public static Collection<Boolean[]> testScenario()
    +  {
    +    return Arrays.asList(new Boolean[][]{{true, false}, // multi cluster with single partition
    +      {true, true}, // multi cluster with multi partitions
    +      {false, true}, // single cluster with multi partitions
    +      {false, false}, // single cluster with single partitions
    +    });
    +  }
    +
    +  public KafkaInputOperatorTest(boolean hasMultiCluster, boolean hasMultiPartition)
    +  {
    +    // This class want to initialize several kafka brokers for multiple partitions
    +    this.hasMultiCluster = hasMultiCluster;
    +    this.hasMultiPartition = hasMultiPartition;
    +    int cluster = 1 + (hasMultiCluster ? 1 : 0);
    +    totalBrokers = (1 + (hasMultiPartition ? 1 : 0)) * cluster;
    +
    +  }
    +
    +  static final org.slf4j.Logger logger = LoggerFactory.getLogger(KafkaInputOperatorTest.class);
    +  static List<String> tupleCollection = new LinkedList<>();
    +  static CountDownLatch latch;
    +  static boolean hasFailure = false;
    +  static int failureTrigger = 3000;
    +  static int k = 0;
    +
    +  /**
    +   * Test Operator to collect tuples from KafkaSingleInputStringOperator.
    +   *
    +   * @param
    +   */
    +  public static class CollectorModule extends BaseOperator
    +  {
    +    public final transient CollectorInputPort inputPort = new CollectorInputPort();
    +  }
    +
    +  public static class CollectorInputPort extends DefaultInputPort<byte[]>
    +  {
    +
    +    @Override
    +    public void process(byte[] bt)
    +    {
    +      String tuple = new String(bt);
    +      if (hasFailure && k++ == failureTrigger) {
    +        //you can only kill yourself once
    +        hasFailure = false;
    +        throw new RuntimeException();
    +      }
    +      if (tuple.equals(KafkaOperatorTestBase.END_TUPLE)) {
    +        if (latch != null) {
    +          latch.countDown();
    +        }
    +        return;
    +      }
    +      tupleCollection.add(tuple);
    +    }
    +
    +    @Override
    +    public void setConnected(boolean flag)
    +    {
    +      if (flag) {
    +        tupleCollection.clear();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Test AbstractKafkaSinglePortInputOperator (i.e. an input adapter for Kafka, aka consumer). This module receives
    +   * data from an outside test generator through Kafka message bus and feed that data into Malhar streaming platform.
    +   *
    +   * [Generate message and send that to Kafka message bus] ==> [Receive that message through Kafka input adapter(i.e.
    +   * consumer) and send using emitTuples() interface on output port]
    +   *
    +   *
    +   * @throws Exception
    +   */
    +  @Test
    +  public void testPartitionableInputOperator() throws Exception
    +  {
    +    hasFailure = false;
    +    testInputOperator(false);
    +  }
    +
    +
    +  @Test
    +  public void testPartitionableInputOperatorWithFailure() throws Exception
    +  {
    +    hasFailure = true;
    +    testInputOperator(true);
    +  }
    +
    +  public void testInputOperator(boolean hasFailure) throws Exception
    +  {
    +
    +    // each broker should get a END_TUPLE message
    +    latch = new CountDownLatch(totalBrokers);
    +
    +    int totalCount = 10000;
    +
    +    // Start producer
    +    KafkaTestProducer p = new KafkaTestProducer(TEST_TOPIC, hasMultiPartition, hasMultiCluster);
    +    p.setSendCount(totalCount);
    +    new Thread(p).start();
    +
    +    // Create DAG for testing.
    +    LocalMode lma = LocalMode.newInstance();
    +    DAG dag = lma.getDAG();
    +
    +    // Create KafkaSinglePortStringInputOperator
    +    KafkaSinglePortInputOperator node = dag.addOperator("Kafka input", KafkaSinglePortInputOperator.class);
    +    node.setInitialPartitionCount(1);
    +    // set topic
    +    node.setTopics(TEST_TOPIC);
    +    node.setInitialOffset(AbstractKafkaInputOperator.InitialOffset.EARLIEST.name());
    +    node.setClusters(getClusterConfig());
    +
    +    // Create Test tuple collector
    +    CollectorModule collector = dag.addOperator("TestMessageCollector", new CollectorModule());
    +
    +    // Connect ports
    +    dag.addStream("Kafka message", node.outputPort, collector.inputPort).setLocality(Locality.CONTAINER_LOCAL);
    +
    +    // Create local cluster
    +    final LocalMode.Controller lc = lma.getController();
    +    lc.setHeartbeatMonitoringEnabled(false);
    +
    +    if (hasFailure) {
    +      setupHasFailureTest(node, dag);
    +    }
    +    lc.runAsync();
    +
    +    // Wait 30s for consumer finish consuming all the messages
    +    boolean notTimeout = latch.await(40000, TimeUnit.MILLISECONDS);
    +    Assert.assertTrue("TIMEOUT: 40s Collected " + tupleCollection, notTimeout);
    +
    +    // Check results
    +    Assert.assertEquals("Tuple count", totalCount, tupleCollection.size());
    +    logger.debug(String.format("Number of emitted tuples: %d", tupleCollection.size()));
    +
    +    p.close();
    +    lc.shutdown();
    +    // kafka has a bug shutdown connector you have to make sure kafka client resource has been cleaned before clean the broker
    +    Thread.sleep(5000);
    --- End diff --
    
    Not needed any more, copied from the old code


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48918465
  
    --- Diff: kafka/src/main/java/org/apache/apex/malhar/kafka/AbstractKafkaInputOperator.java ---
    @@ -0,0 +1,500 @@
    +/**
    + * 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.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.tuple.Pair;
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +import org.apache.kafka.clients.consumer.OffsetAndMetadata;
    +import org.apache.kafka.clients.consumer.OffsetCommitCallback;
    +import org.apache.kafka.common.TopicPartition;
    +
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * The abstract kafka input operator using kafka 0.9.0 new consumer API
    + * A scalable, fault-tolerant, at-least-once kafka input operator
    + * Key features includes:
    + *
    + * <ol>
    + * <li>Out-of-box One-to-one and one-to-many partition strategy support plus customizable partition strategy
    + *    refer to AbstractKafkaPartitioner </li>
    + * <li>Fault-tolerant when the input operator goes down, it redeploys on other node</li>
    + * <li>At-least-once semantics for operator failure (no matter which operator fails)</li>
    + * <li>At-least-once semantics for cold restart (no data loss even if you restart the application)</li>
    + * <li>Multi-cluster support, one operator can consume data from more than one kafka clusters</li>
    + * <li>Multi-topic support, one operator can subscribe multiple topics</li>
    + * <li>Throughput control support, you can throttle number of tuple for each streaming window</li>
    + * </ol>
    + */
    +public abstract class AbstractKafkaInputOperator implements InputOperator, Operator.ActivationListener<Context.OperatorContext>, Operator.CheckpointListener, Partitioner<AbstractKafkaInputOperator>, StatsListener, OffsetCommitCallback
    +{
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaInputOperator.class);
    +
    +  public enum InitialOffset
    +  {
    +    EARLIEST, // consume from beginning of the partition every time when application restart
    +    LATEST, // consume from latest of the partition every time when application restart
    +    APPLICATION_OR_EARLIEST, // consume from committed position from last run or earliest if there is no committed offset(s)
    +    APPLICATION_OR_LATEST // consume from committed position from last run or latest if there is no committed offset(s)
    +  }
    +  /**
    +   *  Same setting as bootstrap.servers property to KafkaConsumer
    +   *  refer to http://kafka.apache.org/documentation.html#newconsumerconfigs
    +   *  To support multi cluster, you can have multiple bootstrap.servers separated by ";"
    +   */
    +  @NotNull
    +  private String[] clusters;
    +
    +  /**
    +   * The topics the operator consumes
    +   */
    +  @NotNull
    +  private String[] topics;
    +
    +  /**
    +   * Wrapper consumer object
    +   * It wraps KafkaConsumer, maintains consumer thread and store messages in a queue
    +   */
    +  private transient KafkaConsumerWrapper consumerWrapper = new KafkaConsumerWrapper();
    +
    +  /**
    +   * Assignment for each operator instance
    +   */
    +  private Set<AbstractKafkaPartitioner.PartitionMeta> assignment;
    +
    +  /**
    +   *  offset track for checkpoint
    +   */
    +  private Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetTrack = new HashMap<>();
    +
    +  /**
    +   * store offsets with window id, only keep offsets with windows that have not been committed
    +   */
    +  private transient List<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> offsetHistory = new LinkedList<>();
    +
    +  /**
    +   * initial partition count
    +   * only used with PartitionStrategy.ONE_TO_MANY
    +   */
    +  private int initialPartitionCount = 1;
    +
    +  /**
    +   * Minimal interval between 2 (re)partition actions
    +   */
    +  private long repartitionInterval = 30000L;
    +
    +  /**
    +   * Minimal interval between checking collected stats and decide whether it needs to repartition or not.
    +   * And minimal interval between 2 offset updates
    +   */
    +  private long repartitionCheckInterval = 5000L;
    +
    +
    +
    +  /**
    +   * maximum tuples allowed to be emitted in each window
    +   */
    +  @Min(1)
    +  private int maxTuplesPerWindow = Integer.MAX_VALUE;
    +
    +  /**
    +   *  Initial offset, it should be one of the following
    +   *  <ul>
    +   *    <li>earliest</li>
    +   *    <li>latest</li>
    +   *    <li>application_or_earliest</li>
    +   *    <li>application_or_latest</li>
    +   *  </ul>
    +   */
    +  private InitialOffset initialOffset = InitialOffset.APPLICATION_OR_LATEST;
    +
    +  /**
    +   * metrics refresh interval
    +   */
    +  private long metricsRefreshInterval = 5000L;
    +
    +  /**
    +   * @see <a href="http://kafka.apache.org/090/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#poll(long)">
    +   *   org.apache.kafka.clients.consumer.KafkaConsumer.poll</a>
    +   */
    +  private long consumerTimeout = 5000L;
    +
    +  /**
    +   * Number of messages kept in memory waiting for emission to downstream operator
    +   */
    +  private int holdingBufferSize = 1024;
    +
    +  /**
    +   * Extra kafka consumer properties
    +   * http://kafka.apache.org/090/documentation.html#newconsumerconfigs
    +   *
    +   * Please be aware that the properties below are set by the operator, don't override it
    +   *
    +   * <ul>
    +   * <li>bootstrap.servers</li>
    +   * <li>group.id</li>
    +   * <li>auto.offset.reset</li>
    +   * <li>enable.auto.commit</li>
    +   * <li>partition.assignment.strategy</li>
    +   * <li>key.deserializer</li>
    +   * <li>value.deserializer</li>
    +   * </ul>
    +   *
    +   */
    +  private Properties consumerProps;
    +
    +  /**
    +   * count the emitted message in each window<br>
    +   * non settable
    +   */
    +  private transient int emitCount = 0;
    +
    +  /**
    +   * Application name is used as group.id for kafka consumer
    +   */
    +  private transient String applicationName;
    +
    +  private transient AbstractKafkaPartitioner partitioner;
    +
    +  protected transient long currentWindowId;
    +
    +  /**
    +   * By default the strategy is one to one
    +   * @see PartitionStrategy
    +   */
    +  private PartitionStrategy strategy = PartitionStrategy.ONE_TO_ONE;
    +
    +  private transient long lastCheckTime = 0L;
    +
    +  private transient long lastRepartitionTime = 0L;
    +
    +  @AutoMetric
    +  private transient KafkaMetrics metrics;
    +
    +  @Override
    +  public void activate(Context.OperatorContext context)
    +  {
    +    consumerWrapper.start();
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +    consumerWrapper.stop();
    +  }
    +
    +  @Override
    +  public void checkpointed(long l)
    +  {
    +
    +  }
    +
    +  @Override
    +  public void committed(long windowId)
    +  {
    +    //ask kafka consumer wrapper to store the committed offsets
    +    for (Iterator<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> iter = offsetHistory.iterator(); iter.hasNext(); ) {
    +      Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>> item = iter.next();
    +      if (item.getLeft() <= windowId) {
    +        if (item.getLeft() == windowId) {
    +          consumerWrapper.commitOffsets(item.getRight());
    +        }
    +        iter.remove();
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void emitTuples()
    +  {
    +    int count = consumerWrapper.messageSize();
    +    if (maxTuplesPerWindow > 0) {
    +      count = Math.min(count, maxTuplesPerWindow - emitCount);
    +    }
    +    for (int i = 0; i < count; i++) {
    +      Pair<String, ConsumerRecord<byte[], byte[]>> tuple = consumerWrapper.pollMessage();
    +      ConsumerRecord<byte[], byte[]> msg = tuple.getRight();
    +      emitTuple(tuple.getLeft(), msg);
    +      AbstractKafkaPartitioner.PartitionMeta pm = new AbstractKafkaPartitioner.PartitionMeta(tuple.getLeft(),
    +          msg.topic(), msg.partition());
    +      offsetTrack.put(pm, msg.offset());
    +    }
    +    emitCount += count;
    +  }
    +
    +  protected abstract void emitTuple(String cluster, ConsumerRecord<byte[], byte[]> message);
    +
    +  @Override
    +  public void beginWindow(long wid)
    +  {
    +    emitCount = 0;
    +    currentWindowId = wid;
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    // copy current offset track to history memory
    +    Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetsWithWindow = new HashMap<>(offsetTrack);
    +    offsetHistory.add(Pair.of(currentWindowId, offsetsWithWindow));
    +
    +    //update metrics
    +    metrics.updateMetrics(clusters, consumerWrapper.getAllConsumerMetrics());
    +  }
    +
    +
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    applicationName = context.getValue(Context.DAGContext.APPLICATION_NAME);
    +    consumerWrapper.create(this);
    +    metrics = new KafkaMetrics(metricsRefreshInterval);
    +  }
    +
    +
    +  @Override
    +  public void teardown()
    +  {
    +
    +  }
    +
    +  private void initPartitioner()
    +  {
    +    if (partitioner == null) {
    +      logger.info("Initialize Partitioner");
    +      switch (strategy) {
    +        case ONE_TO_ONE:
    +          partitioner = new OneToOnePartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY:
    +          partitioner = new OneToManyPartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY_HEURISTIC:
    +          throw new UnsupportedOperationException("Not implemented yet");
    +        default:
    +          throw new RuntimeException("Invalid strategy");
    +      }
    +      logger.info("Actual Partitioner is {}", partitioner.getClass());
    +    }
    +
    +  }
    +
    +  @Override
    +  public Response processStats(BatchedOperatorStats batchedOperatorStats)
    +  {
    +    long t = System.currentTimeMillis();
    +    if (repartitionInterval < 0 || repartitionCheckInterval < 0 ||
    +        t - lastCheckTime < repartitionCheckInterval || t - lastRepartitionTime < repartitionInterval) {
    +      // return false if it's within repartitionCheckInterval since last time it check the stats
    +      Response response = new Response();
    +      response.repartitionRequired = false;
    +      return response;
    +    }
    +
    +    try {
    +      logger.debug("Process stats");
    +      initPartitioner();
    +      return partitioner.processStats(batchedOperatorStats);
    +    } finally {
    +      lastCheckTime = System.currentTimeMillis();
    +    }
    +  }
    +
    +  @Override
    +  public Collection<Partition<AbstractKafkaInputOperator>> definePartitions(
    +      Collection<Partition<AbstractKafkaInputOperator>> collection, PartitioningContext partitioningContext)
    +  {
    +    logger.debug("Define partitions");
    +    initPartitioner();
    +    return partitioner.definePartitions(collection, partitioningContext);
    +  }
    +
    +  @Override
    +  public void partitioned(Map<Integer, Partition<AbstractKafkaInputOperator>> map)
    +  {
    +    // update the last repartition time
    +    lastRepartitionTime = System.currentTimeMillis();
    +    initPartitioner();
    +    partitioner.partitioned(map);
    +  }
    +
    +  /**
    +   *
    +   * A callback from consumer after it commits the offset
    +   * @param map
    +   * @param e
    +   */
    +  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);
    +    }
    +  }
    +
    +
    +
    +  //---------------------------------------------setters and getters----------------------------------------
    +  public void setInitialPartitionCount(int partitionCount)
    +  {
    +    this.initialPartitionCount = partitionCount;
    +  }
    +
    +  public int getInitialPartitionCount()
    +  {
    +    return initialPartitionCount;
    +  }
    +
    +  public void setClusters(String clusters)
    +  {
    +    this.clusters = clusters.split(";");
    +  }
    +
    +  public String getClusters()
    +  {
    +    return Joiner.on(';').join(clusters);
    +  }
    +
    +  public void setTopics(String... topics)
    +  {
    +    this.topics = topics;
    +  }
    +
    +  public String[] getTopics()
    +  {
    +    return topics;
    +  }
    +
    +  public void setAssignment(Set<AbstractKafkaPartitioner.PartitionMeta> assignment)
    +  {
    +    this.assignment = assignment;
    +  }
    +
    +  public Set<AbstractKafkaPartitioner.PartitionMeta> getAssignment()
    +  {
    +    return assignment;
    +  }
    +
    +  public Map<AbstractKafkaPartitioner.PartitionMeta, Long> getOffsetTrack()
    +  {
    +    return offsetTrack;
    +  }
    +
    +  public void setStrategy(String policy)
    +  {
    +    this.strategy = PartitionStrategy.valueOf(policy.toUpperCase());
    +  }
    +
    +  public PartitionStrategy getStrategy()
    +  {
    +    return strategy;
    +  }
    +
    +  public void setInitialOffset(String initialOffset)
    +  {
    +    this.initialOffset = InitialOffset.valueOf(initialOffset.toUpperCase());
    +  }
    +
    +  public InitialOffset getInitialOffset()
    +  {
    +    return initialOffset;
    +  }
    +
    +  public String getApplicationName()
    +  {
    +    return applicationName;
    +  }
    +
    +  public void setConsumerProps(Properties consumerProps)
    +  {
    +    this.consumerProps = consumerProps;
    +  }
    +
    +  public Properties getConsumerProps()
    +  {
    +    return consumerProps;
    +  }
    +
    +  public void setMaxTuplesPerWindow(int maxTuplesPerWindow)
    +  {
    +    this.maxTuplesPerWindow = maxTuplesPerWindow;
    +  }
    +
    +  public int getMaxTuplesPerWindow()
    +  {
    +    return maxTuplesPerWindow;
    +  }
    +
    +  public long getConsumerTimeout()
    +  {
    +    return consumerTimeout;
    +  }
    +
    +  public void setConsumerTimeout(long consumerTimeout)
    +  {
    +    this.consumerTimeout = consumerTimeout;
    +  }
    +
    +  public int getHoldingBufferSize()
    --- End diff --
    
    Maybe name this "capacity" instead?


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48646883
  
    --- Diff: kafka/src/main/java/org/apache/apex/malhar/kafka/AbstractKafkaInputOperator.java ---
    @@ -0,0 +1,512 @@
    +/**
    + * 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.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.OffsetAndMetadata;
    +import org.apache.kafka.clients.consumer.OffsetCommitCallback;
    +import org.apache.kafka.common.TopicPartition;
    +import org.apache.kafka.common.serialization.Deserializer;
    +
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * The abstract kafka input operator using kafka 0.9.0 new consumer API
    + * A scalable, fault-tolerant, at-least-once kafka input operator
    + * Key features includes:
    + * 1. Out-of-box One-to-one and one-to-many partition strategy support plus customizable partition strategy
    + *    refer to AbstractKafkaPartitioner
    + * 2. Fault-tolerant when the input operator goes down, it redeploys on other node
    + * 3. At-least-once semantics for operator failure (no matter which operator fails)
    + * 4. At-least-once semantics for cold restart (no data loss even if you restart the application)
    + * 5. Multi-cluster support, one operator can consume data from more than one kafka clusters
    + * 6. Multi-topic support, one operator can subscribe multiple topics
    + * 7. Throughput control support, you can throttle number of tuple for each streaming window
    + */
    +public abstract class AbstractKafkaInputOperator<K, V> implements InputOperator, Operator.ActivationListener<Context.OperatorContext>, Operator.CheckpointListener, Partitioner<AbstractKafkaInputOperator>, StatsListener, OffsetCommitCallback
    +{
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaInputOperator.class);
    +
    +  public long getMetricsRefreshInterval()
    +  {
    +    return metricsRefreshInterval;
    +  }
    +
    +  public void setMetricsRefreshInterval(long metricsRefreshInterval)
    +  {
    +    this.metricsRefreshInterval = metricsRefreshInterval;
    +  }
    +
    +  public enum InitialOffset
    +  {
    +    EARLIEST, // consume from beginning of the partition every time when application restart
    +    LATEST, // consume from latest of the partition every time when application restart
    +    APPLICATION_OR_EARLIEST, // consume from committed position from last run or earliest if there is no committed offset(s)
    +    APPLICATION_OR_LATEST // consume from committed position from last run or latest if there is no committed offset(s)
    +  }
    +  /**
    +   *  Same setting as bootstrap.servers property to KafkaConsumer
    +   *  refer to http://kafka.apache.org/documentation.html#newconsumerconfigs
    +   *  To support multi cluster, you can have multiple bootstrap.servers separated by ";"
    +   */
    +  @NotNull
    +  private String[] clusters;
    +
    +  /**
    +   * The topics the operator consumes
    +   */
    +  @NotNull
    +  private String[] topics;
    +
    +  /**
    +   * Wrapper consumer object
    +   * It wraps KafkaConsumer, maintains consumer thread and store messages in a queue
    +   */
    +  private KafkaConsumerWrapper consumerWrapper = new KafkaConsumerWrapper();
    +
    +  /**
    +   * Assignment for each operator instance
    +   */
    +  private Set<AbstractKafkaPartitioner.PartitionMeta> assignment;
    +
    +  /**
    +   *  offset track for checkpoint
    +   */
    +  private Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetTrack = new HashMap<>();
    +
    +  /**
    +   * store offsets with window id, only keep offsets with windows that have not been committed
    +   */
    +  private transient List<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> offsetHistory = new LinkedList<>();
    +
    +  /**
    +   * initial partition count
    +   * only used with PartitionStrategy.ONE_TO_MANY
    +   */
    +  private int initialPartitionCount = 1;
    +
    +  // Minimal interval between 2 (re)partition actions
    +  private long repartitionInterval = 30000L;
    +
    +  // Minimal interval between checking collected stats and decide whether it needs to repartition or not.
    +  // And minimal interval between 2 offset updates
    +  private long repartitionCheckInterval = 5000L;
    +
    +  // class name of the key deserializer
    +  protected String keyDeserializer;
    +
    +  // class name of the value deserializer
    +  protected String valueDeserializer;
    +
    +  /**
    +   * maximum tuples allowed to be emitted in each window
    +   */
    +  @Min(1)
    +  private int maxTuplesPerWindow = Integer.MAX_VALUE;
    +
    +  private InitialOffset initialOffset = InitialOffset.APPLICATION_OR_LATEST;
    +
    +  private long metricsRefreshInterval = 5000L;
    +
    +  /**
    +   * extra kafka consumer properties
    +   * http://kafka.apache.org/090/documentation.html#newconsumerconfigs
    +   *
    +   * Please be aware that the properties below are set by the operator, don't override it
    +   *
    +   * bootstrap.servers
    +   * group.id
    +   * auto.offset.reset
    +   * enable.auto.commit
    +   * partition.assignment.strategy
    +   *
    +   *
    +   */
    +  private Properties consumerProps;
    +
    +  /**
    +   * count the emitted message in each window
    +   * non settable
    +   */
    +  private transient int emitCount = 0;
    +
    +  /**
    +   * Application name is used as group.id for kafka consumer
    +   */
    +  private transient String applicationName;
    +
    +  private transient AbstractKafkaPartitioner partitioner;
    +
    +  protected transient long currentWindowId;
    +
    +  // By default the partition policy is 1:1
    +  private PartitionStrategy strategy = PartitionStrategy.ONE_TO_ONE;
    +
    +  private transient long lastCheckTime = 0L;
    +
    +  private transient long lastRepartitionTime = 0L;
    +
    +  @AutoMetric
    +  private transient KafkaMetrics metrics;
    +
    +  private transient Deserializer<K> keyDeser;
    +
    +  private transient Deserializer<V> valueDeser;
    +
    +  @Override
    +  public void activate(Context.OperatorContext context)
    +  {
    +    consumerWrapper.start();
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +    consumerWrapper.stop();
    +  }
    +
    +  @Override
    +  public void checkpointed(long l)
    +  {
    +
    +  }
    +
    +  @Override
    +  public void committed(long windowId)
    +  {
    +    //ask kafka consumer wrapper to store the committed offsets
    +    for (Iterator<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> iter = offsetHistory.iterator(); iter.hasNext(); ) {
    +      Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>> item = iter.next();
    +      if (item.getLeft() <= windowId) {
    +        if (item.getLeft() == windowId) {
    +          consumerWrapper.commitOffsets(item.getRight());
    +        }
    +        iter.remove();
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void emitTuples()
    +  {
    +    int count = consumerWrapper.messageSize();
    +    if (maxTuplesPerWindow > 0) {
    +      count = Math.min(count, maxTuplesPerWindow - emitCount);
    +    }
    +    for (int i = 0; i < count; i++) {
    +      Pair<String, ConsumerRecord<byte[], byte[]>> tuple = consumerWrapper.pollMessage();
    +      ConsumerRecord<byte[], byte[]> msg = tuple.getRight();
    +      emitTuple(tuple.getLeft(), msg.topic(), msg.partition(), msg.offset(),
    +          keyDeser.deserialize(msg.topic(), msg.key()), valueDeser.deserialize(msg.topic(), msg.value()));
    +      AbstractKafkaPartitioner.PartitionMeta pm = new AbstractKafkaPartitioner.PartitionMeta(tuple.getLeft(),
    +          msg.topic(), msg.partition());
    +      offsetTrack.put(pm, msg.offset());
    +    }
    +    emitCount += count;
    +  }
    +
    +  protected abstract void emitTuple(String cluster, String topic, long partition, long offset, K key, V value);
    +
    +  @Override
    +  public void beginWindow(long wid)
    +  {
    +    emitCount = 0;
    +    currentWindowId = wid;
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    // copy current offset track to history memory
    +    Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetsWithWindow = new HashMap<>(offsetTrack);
    +    offsetHistory.add(Pair.of(currentWindowId, offsetsWithWindow));
    +
    +    //update metrics
    +    metrics.updateMetrics(clusters, consumerWrapper.getAllConsumerMetrics());
    +  }
    +
    +
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    applicationName = context.getValue(Context.DAGContext.APPLICATION_NAME);
    +    consumerWrapper.create(this);
    +    metrics = new KafkaMetrics(metricsRefreshInterval);
    +    initDeserializer();
    +  }
    +
    +  private void initDeserializer()
    +  {
    +    Map<String, Object> extraProeprties = new HashMap<>();
    +    if (consumerProps != null) {
    +      for (Map.Entry<Object, Object> e : consumerProps.entrySet()) {
    +        extraProeprties.put(e.getKey().toString(), e.getValue());
    +      }
    +    }
    +    try {
    +      String cname = keyDeserializer;
    +      if (cname == null) {
    +        cname = consumerProps.getProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG);
    +      }
    +      keyDeser = (Deserializer<K>)Class.forName(cname).newInstance();
    +      keyDeser.configure(extraProeprties, true);
    +      cname = valueDeserializer;
    +      if (cname == null) {
    +        cname = consumerProps.getProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG);
    +      }
    +      valueDeser = (Deserializer<V>)Class.forName(cname).newInstance();
    +      valueDeser.configure(extraProeprties, false);
    +    } catch (Exception e) {
    +      throw new RuntimeException(e);
    +    }
    +
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +
    +  }
    +
    +  private void initPartitioner()
    +  {
    +    if (partitioner == null) {
    +      logger.info("Initialize Partitioner");
    +      switch (strategy) {
    +        case ONE_TO_ONE:
    +          partitioner = new OneToOnePartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY:
    +          partitioner = new OneToManyPartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY_HEURISTIC:
    +          throw new UnsupportedOperationException("Not implemented yet");
    +        default:
    +          throw new RuntimeException("Invalid strategy");
    +      }
    +      logger.info("Actual Partitioner is {}", partitioner.getClass());
    +    }
    +
    +  }
    +
    +  @Override
    +  public Response processStats(BatchedOperatorStats batchedOperatorStats)
    +  {
    +    long t = System.currentTimeMillis();
    +    if (repartitionInterval < 0 || repartitionCheckInterval < 0 ||
    +        t - lastCheckTime < repartitionCheckInterval || t - lastRepartitionTime < repartitionInterval) {
    +      // return false if it's within repartitionCheckInterval since last time it check the stats
    +      Response response = new Response();
    +      response.repartitionRequired = false;
    +      return response;
    +    }
    +
    +    try {
    +      logger.debug("Process stats");
    +      initPartitioner();
    +      return partitioner.processStats(batchedOperatorStats);
    +    } finally {
    +      lastCheckTime = System.currentTimeMillis();
    +    }
    +  }
    +
    +  @Override
    +  public Collection<Partition<AbstractKafkaInputOperator>> definePartitions(
    +      Collection<Partition<AbstractKafkaInputOperator>> collection, PartitioningContext partitioningContext)
    +  {
    +    logger.debug("Define partitions");
    +    initPartitioner();
    +    return partitioner.definePartitions(collection, partitioningContext);
    +  }
    +
    +  @Override
    +  public void partitioned(Map<Integer, Partition<AbstractKafkaInputOperator>> map)
    +  {
    +    // update the last repartition time
    +    lastRepartitionTime = System.currentTimeMillis();
    +    initPartitioner();
    +    partitioner.partitioned(map);
    +  }
    +
    +  /**
    +   *
    +   * A callback from consumer after it commits the offset
    +   * @param map
    +   * @param e
    +   */
    +  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);
    +    }
    +  }
    +
    +
    +
    +  //---------------------------------------------setters and getters----------------------------------------
    +  public void setInitialPartitionCount(int partitionCount)
    +  {
    +    this.initialPartitionCount = partitionCount;
    +  }
    +
    +  public int getInitialPartitionCount()
    +  {
    +    return initialPartitionCount;
    +  }
    +
    +  public void setClusters(String clusters)
    +  {
    +    this.clusters = clusters.split(";");
    +  }
    +
    +  public String getClusters()
    +  {
    +    return Joiner.on(';').join(clusters);
    +  }
    +
    +  public void setTopics(String... topics)
    +  {
    +    this.topics = topics;
    +  }
    +
    +  public String[] getTopics()
    +  {
    +    return topics;
    +  }
    +
    +  public void setConsumerWrapper(KafkaConsumerWrapper consumerWrapper)
    --- End diff --
    
    Why expose this as property?


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by siyuanh <gi...@git.apache.org>.
Github user siyuanh commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48378389
  
    --- Diff: kafka/src/main/java/org/apache/apex/malhar/kafka/AbstractKafkaInputOperator.java ---
    @@ -0,0 +1,455 @@
    +/**
    + * 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.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.tuple.Pair;
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +import org.apache.kafka.common.serialization.Deserializer;
    +
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * The abstract kafka input operator using kafka 0.9.0 new consumer API
    + * A scalable, fault-tolerant, at-least-once kafka input operator
    + * Key features includes:
    + * 1. Out-of-box One-to-one and one-to-many partition strategy support plus customizable partition strategy
    + *    refer to AbstractKafkaPartitioner
    + * 2. Fault-tolerant when the input operator goes down, it redeploys on other node
    + * 3. At-least-once semantics for operator failure (no matter which operator fails)
    + * 4. At-least-once semantics for cold restart (no data loss even if you restart the application)
    + * 5. Multi-cluster support, one operator can consume data from more than one kafka clusters
    + * 6. Multi-topic support, one operator can subscribe multiple topics
    + * 7. Throughput control support, you can throttle number of tuple for each streaming window
    + */
    +public abstract class AbstractKafkaInputOperator<K, V> implements InputOperator, Operator.ActivationListener<Context.OperatorContext>, Operator.CheckpointListener, Partitioner<AbstractKafkaInputOperator>, StatsListener
    +{
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaInputOperator.class);
    +
    +  public long getMetricsRefreshInterval()
    +  {
    +    return metricsRefreshInterval;
    +  }
    +
    +  public void setMetricsRefreshInterval(long metricsRefreshInterval)
    +  {
    +    this.metricsRefreshInterval = metricsRefreshInterval;
    +  }
    +
    +  public enum InitialOffset
    +  {
    +    EARLIEST, // consume from beginning of the partition every time when application restart
    +    LATEST, // consume from latest of the partition every time when application restart
    +    APPLICATION_OR_EARLIEST, // consume from committed position from last run or earliest if there is no committed offset(s)
    +    APPLICATION_OR_LATEST // consume from committed position from last run or latest if there is no committed offset(s)
    +  }
    +  /**
    +   *  Same setting as bootstrap.servers property to KafkaConsumer
    +   *  refer to http://kafka.apache.org/documentation.html#newconsumerconfigs
    +   *  To support multi cluster, you can have multiple bootstrap.servers separated by ";"
    +   */
    +  @NotNull
    +  private String[] clusters;
    +
    +  /**
    +   * The topics the operator consumes
    +   */
    +  @NotNull
    +  private String[] topics;
    +
    +  /**
    +   * Wrapper consumer object
    +   * It wraps KafkaConsumer, maintains consumer thread and store messages in a queue
    +   */
    +  private KafkaConsumerWrapper<K, V> consumerWrapper = new KafkaConsumerWrapper<>();
    +
    +  /**
    +   * Assignment for each operator instance
    +   */
    +  private Set<AbstractKafkaPartitioner.PartitionMeta> assignment;
    +
    +  /**
    +   *  offset track for checkpoint
    +   */
    +  private Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetTrack = new HashMap<>();
    +
    +  /**
    +   * store offsets with window id, only keep offsets with windows that have not been committed
    +   */
    +  private transient List<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> offsetHistory = new LinkedList<>();
    +
    +  /**
    +   * initial partition count
    +   * only used with PartitionStrategy.ONE_TO_MANY
    +   */
    +  private int initialPartitionCount = 1;
    +
    +  // Minimal interval between 2 (re)partition actions
    +  private long repartitionInterval = 30000L;
    +
    +  // Minimal interval between checking collected stats and decide whether it needs to repartition or not.
    +  // And minimal interval between 2 offset updates
    +  private long repartitionCheckInterval = 5000L;
    +
    +  private Class<? extends Deserializer<K>> keyDeserializer;
    +
    +  private Class<? extends Deserializer<V>> valueDeserializer;
    +
    +  /**
    +   * maximum tuples allowed to be emitted in each window
    +   */
    +  @Min(1)
    +  private int maxTuplesPerWindow = Integer.MAX_VALUE;
    +
    +  private InitialOffset initialOffset = InitialOffset.APPLICATION_OR_LATEST;
    +
    +  private long metricsRefreshInterval = 5000L;
    +
    +  /**
    +   * extra kafka consumer properties
    +   * http://kafka.apache.org/090/documentation.html#newconsumerconfigs
    +   *
    +   * Please be aware that the properties below are set by the operator, don't override it
    +   *
    +   * bootstrap.servers
    +   * group.id
    +   * auto.offset.reset
    +   * enable.auto.commit
    +   * partition.assignment.strategy
    +   *
    +   *
    +   */
    +  private Properties consumerProps;
    +
    +  /**
    +   * count the emitted message in each window
    +   * non settable
    +   */
    +  private transient int emitCount = 0;
    +
    +  /**
    +   * Application name is used as group.id for kafka consumer
    +   */
    +  private transient String applicationName;
    +
    +  private transient AbstractKafkaPartitioner partitioner;
    +
    +  protected transient long currentWindowId;
    +
    +  // By default the partition policy is 1:1
    +  private PartitionStrategy strategy = PartitionStrategy.ONE_TO_ONE;
    +
    +  private transient long lastCheckTime = 0L;
    +
    +  private transient long lastRepartitionTime = 0L;
    +
    +  @AutoMetric
    +  private transient KafkaMetrics metrics;
    +
    +  @Override
    +  public void activate(Context.OperatorContext context)
    +  {
    +    consumerWrapper.start();
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +    consumerWrapper.stop();
    +  }
    +
    +  @Override
    +  public void checkpointed(long l)
    +  {
    +
    +  }
    +
    +  @Override
    +  public void committed(long windowId)
    +  {
    +    //ask kafka consumer wrapper to store the committed offsets
    +    for (Iterator<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> iter = offsetHistory.iterator(); iter.hasNext(); ) {
    +      Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>> item = iter.next();
    +      if (item.getLeft() <= windowId) {
    +        if (item.getLeft() == windowId) {
    +          consumerWrapper.commitOffsets(item.getRight());
    +        }
    +        iter.remove();
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void emitTuples()
    +  {
    +    int count = consumerWrapper.messageSize();
    +    if (maxTuplesPerWindow > 0) {
    +      count = Math.min(count, maxTuplesPerWindow - emitCount);
    +    }
    +    for (int i = 0; i < count; i++) {
    +      Pair<String, ConsumerRecord<K, V>> tuple = consumerWrapper.pollMessage();
    +      ConsumerRecord<K, V> msg = tuple.getRight();
    +      emitTuple(tuple.getLeft(), msg);
    +      AbstractKafkaPartitioner.PartitionMeta pm = new AbstractKafkaPartitioner.PartitionMeta(tuple.getLeft(),
    +          msg.topic(), msg.partition());
    +      offsetTrack.put(pm, msg.offset());
    +    }
    +    emitCount += count;
    +  }
    +
    +  protected abstract void emitTuple(String cluster, ConsumerRecord<K, V> msg);
    +
    +  @Override
    +  public void beginWindow(long wid)
    +  {
    +    emitCount = 0;
    +    currentWindowId = wid;
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    // copy current offset track to history memory
    +    Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetsWithWindow = new HashMap<>(offsetTrack);
    +    offsetHistory.add(Pair.of(currentWindowId, offsetsWithWindow));
    +
    +    //update metrics
    +    metrics.updateMetrics(clusters, consumerWrapper.getAllConsumerMetrics());
    +  }
    +
    +
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    applicationName = context.getValue(Context.DAGContext.APPLICATION_NAME);
    +    consumerWrapper.create(this);
    +    metrics = new KafkaMetrics(metricsRefreshInterval);
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +
    +  }
    +
    +  private void initPartitioner()
    +  {
    +    if (partitioner == null) {
    +      logger.info("Initialize Partitioner");
    +      switch (strategy) {
    +        case ONE_TO_ONE:
    +          partitioner = new OneToOnePartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY:
    +          partitioner = new OneToManyPartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY_HEURISTIC:
    +          throw new UnsupportedOperationException("Not implemented yet");
    +        default:
    +          throw new RuntimeException("Invalid strategy");
    +      }
    +      logger.info("Actual Partitioner is {}", partitioner.getClass());
    +    }
    +
    +  }
    +
    +  @Override
    +  public Response processStats(BatchedOperatorStats batchedOperatorStats)
    +  {
    +    long t = System.currentTimeMillis();
    +    if (repartitionInterval < 0 || repartitionCheckInterval < 0 ||
    +        t - lastCheckTime < repartitionCheckInterval || t - lastRepartitionTime < repartitionInterval) {
    +      // return false if it's within repartitionCheckInterval since last time it check the stats
    +      Response response = new Response();
    +      response.repartitionRequired = false;
    +      return response;
    +    }
    +
    +    try {
    +      logger.debug("Process stats");
    +      initPartitioner();
    +      return partitioner.processStats(batchedOperatorStats);
    --- End diff --
    
    3 possibilities here
    1. within interval, the operator shouldn't not try repartition
    2. exceed interval, the partitioner will always try to repartition based on latest kafka partition info. If there is no kafka partition change, it will return same collection of partition instances and AM will not try to repartition the operator.
    3. exceed interval and there is kafka partition change(ex. new kafka partition added at runtime) it will try to repartition the operator.


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by gauravgopi123 <gi...@git.apache.org>.
Github user gauravgopi123 commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48373387
  
    --- Diff: kafka/src/main/java/org/apache/apex/malhar/kafka/AbstractKafkaPartitioner.java ---
    @@ -0,0 +1,276 @@
    +/**
    + * 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.ByteArrayOutputStream;
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.kafka.clients.consumer.KafkaConsumer;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.TopicPartition;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * Abstract partitioner used to manage the partitions of kafka input operator.
    + * It use a number of kafka consumers(one for each cluster) to get the latest partition metadata for topics that
    + * the consumer subscribes and expose those to subclass which implements the assign method
    + *
    + * The partitioner is always stateless.
    + */
    +public abstract class AbstractKafkaPartitioner implements Partitioner<AbstractKafkaInputOperator>, StatsListener
    +{
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaPartitioner.class);
    +
    +  private static final String META_CONSUMER_GROUP_NAME = AbstractKafkaInputOperator.class.getName() + "META_GROUP";
    +
    +  protected String[] clusters;
    +
    +  protected String[] topics;
    +
    +  protected AbstractKafkaInputOperator prototypeOperator;
    +
    +  private KafkaConsumer[] metadataRefreshClients;
    +
    +
    +  private List<Set<AbstractKafkaPartitioner.PartitionMeta>> currentPartitions = new LinkedList<>(); // prevent null
    +
    +  public AbstractKafkaPartitioner(String[] clusters, String[] topics, AbstractKafkaInputOperator prototypeOperator)
    +  {
    +    this.clusters = clusters;
    +    this.topics = topics;
    +    this.prototypeOperator = prototypeOperator;
    +  }
    +
    +  abstract List<Set<PartitionMeta>> assign(Map<String, Map<String,List<PartitionInfo>>> metadata);
    +
    +
    +
    +  @Override
    +  public Collection<Partition<AbstractKafkaInputOperator>> definePartitions(Collection<Partition<AbstractKafkaInputOperator>> collection, PartitioningContext partitioningContext)
    +  {
    +
    +    initMetadataClients();
    +
    +    Map<String, Map<String, List<PartitionInfo>>> metadata = new HashMap<>();
    +
    +
    +    for (int i = 0; i < clusters.length; i++) {
    +      metadata.put(clusters[i], new HashMap<String, List<PartitionInfo>>());
    +      for (String topic : topics) {
    +        List<PartitionInfo> ptis = metadataRefreshClients[i].partitionsFor(topic);
    +        if (logger.isDebugEnabled()) {
    +          logger.debug("Partition metadata for topic {} : {}", topic, Joiner.on(';').join(ptis));
    +        }
    +        metadata.get(clusters[i]).put(topic, ptis);
    +      }
    +      metadataRefreshClients[i].close();
    +    }
    +
    +    metadataRefreshClients = null;
    +
    +    List<Set<AbstractKafkaPartitioner.PartitionMeta>> parts = assign(metadata);
    +
    +
    +    if (currentPartitions == parts || currentPartitions.equals(parts)) {
    +      logger.debug("No partition change found");
    +      return collection;
    +    } else {
    +      logger.info("Partition change detected: ");
    +      currentPartitions.clear();
    +      currentPartitions.addAll(parts);
    +      int i = 0;
    +      List<Partition<AbstractKafkaInputOperator>> result = new LinkedList<>();
    +      for (Iterator<Partition<AbstractKafkaInputOperator>> iter = collection.iterator(); iter.hasNext();) {
    +        Partition<AbstractKafkaInputOperator> nextPartition = iter.next();
    +        if (parts.remove(nextPartition.getPartitionedInstance().getAssignment())) {
    +          if (logger.isInfoEnabled()) {
    +            logger.info("[Existing] Partition {} with assignment {} ", i,
    +                Joiner.on(';').join(nextPartition.getPartitionedInstance().getAssignment()));
    +          }
    +          result.add(nextPartition);
    +          i++;
    +        }
    +      }
    +
    +      for (Set<AbstractKafkaPartitioner.PartitionMeta> partitionAssignment : parts) {
    +        if (logger.isInfoEnabled()) {
    +          logger.info("[New] Partition {} with assignment {} ", i,
    +              Joiner.on(';').join(partitionAssignment));
    +        }
    +        result.add(createPartition(partitionAssignment));
    +        i++;
    +      }
    +
    +
    +      return result;
    +    }
    +  }
    +
    +  @Override
    +  public void partitioned(Map<Integer, Partition<AbstractKafkaInputOperator>> map)
    +  {
    +
    +  }
    +
    +  @Override
    +  public Response processStats(BatchedOperatorStats batchedOperatorStats)
    +  {
    +    Response response = new Response();
    +    response.repartitionRequired = true;
    --- End diff --
    
    Do you always want to re-partition when processStats is called?? 


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48918303
  
    --- Diff: kafka/src/main/java/org/apache/apex/malhar/kafka/AbstractKafkaInputOperator.java ---
    @@ -0,0 +1,500 @@
    +/**
    + * 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.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.tuple.Pair;
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +import org.apache.kafka.clients.consumer.OffsetAndMetadata;
    +import org.apache.kafka.clients.consumer.OffsetCommitCallback;
    +import org.apache.kafka.common.TopicPartition;
    +
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * The abstract kafka input operator using kafka 0.9.0 new consumer API
    + * A scalable, fault-tolerant, at-least-once kafka input operator
    + * Key features includes:
    + *
    + * <ol>
    + * <li>Out-of-box One-to-one and one-to-many partition strategy support plus customizable partition strategy
    + *    refer to AbstractKafkaPartitioner </li>
    + * <li>Fault-tolerant when the input operator goes down, it redeploys on other node</li>
    + * <li>At-least-once semantics for operator failure (no matter which operator fails)</li>
    + * <li>At-least-once semantics for cold restart (no data loss even if you restart the application)</li>
    + * <li>Multi-cluster support, one operator can consume data from more than one kafka clusters</li>
    + * <li>Multi-topic support, one operator can subscribe multiple topics</li>
    + * <li>Throughput control support, you can throttle number of tuple for each streaming window</li>
    + * </ol>
    + */
    +public abstract class AbstractKafkaInputOperator implements InputOperator, Operator.ActivationListener<Context.OperatorContext>, Operator.CheckpointListener, Partitioner<AbstractKafkaInputOperator>, StatsListener, OffsetCommitCallback
    +{
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaInputOperator.class);
    +
    +  public enum InitialOffset
    +  {
    +    EARLIEST, // consume from beginning of the partition every time when application restart
    +    LATEST, // consume from latest of the partition every time when application restart
    +    APPLICATION_OR_EARLIEST, // consume from committed position from last run or earliest if there is no committed offset(s)
    +    APPLICATION_OR_LATEST // consume from committed position from last run or latest if there is no committed offset(s)
    +  }
    +  /**
    +   *  Same setting as bootstrap.servers property to KafkaConsumer
    +   *  refer to http://kafka.apache.org/documentation.html#newconsumerconfigs
    +   *  To support multi cluster, you can have multiple bootstrap.servers separated by ";"
    +   */
    +  @NotNull
    +  private String[] clusters;
    +
    +  /**
    +   * The topics the operator consumes
    +   */
    +  @NotNull
    +  private String[] topics;
    +
    +  /**
    +   * Wrapper consumer object
    +   * It wraps KafkaConsumer, maintains consumer thread and store messages in a queue
    +   */
    +  private transient KafkaConsumerWrapper consumerWrapper = new KafkaConsumerWrapper();
    +
    +  /**
    +   * Assignment for each operator instance
    +   */
    +  private Set<AbstractKafkaPartitioner.PartitionMeta> assignment;
    +
    +  /**
    +   *  offset track for checkpoint
    +   */
    +  private Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetTrack = new HashMap<>();
    +
    +  /**
    +   * store offsets with window id, only keep offsets with windows that have not been committed
    +   */
    +  private transient List<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> offsetHistory = new LinkedList<>();
    +
    +  /**
    +   * initial partition count
    +   * only used with PartitionStrategy.ONE_TO_MANY
    +   */
    +  private int initialPartitionCount = 1;
    +
    +  /**
    +   * Minimal interval between 2 (re)partition actions
    +   */
    +  private long repartitionInterval = 30000L;
    +
    +  /**
    +   * Minimal interval between checking collected stats and decide whether it needs to repartition or not.
    +   * And minimal interval between 2 offset updates
    +   */
    +  private long repartitionCheckInterval = 5000L;
    +
    +
    +
    +  /**
    +   * maximum tuples allowed to be emitted in each window
    +   */
    +  @Min(1)
    +  private int maxTuplesPerWindow = Integer.MAX_VALUE;
    +
    +  /**
    +   *  Initial offset, it should be one of the following
    +   *  <ul>
    +   *    <li>earliest</li>
    +   *    <li>latest</li>
    +   *    <li>application_or_earliest</li>
    +   *    <li>application_or_latest</li>
    +   *  </ul>
    +   */
    +  private InitialOffset initialOffset = InitialOffset.APPLICATION_OR_LATEST;
    +
    +  /**
    +   * metrics refresh interval
    +   */
    +  private long metricsRefreshInterval = 5000L;
    +
    +  /**
    +   * @see <a href="http://kafka.apache.org/090/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#poll(long)">
    +   *   org.apache.kafka.clients.consumer.KafkaConsumer.poll</a>
    +   */
    +  private long consumerTimeout = 5000L;
    +
    +  /**
    +   * Number of messages kept in memory waiting for emission to downstream operator
    +   */
    +  private int holdingBufferSize = 1024;
    +
    +  /**
    +   * Extra kafka consumer properties
    +   * http://kafka.apache.org/090/documentation.html#newconsumerconfigs
    +   *
    +   * Please be aware that the properties below are set by the operator, don't override it
    +   *
    +   * <ul>
    +   * <li>bootstrap.servers</li>
    +   * <li>group.id</li>
    +   * <li>auto.offset.reset</li>
    +   * <li>enable.auto.commit</li>
    +   * <li>partition.assignment.strategy</li>
    +   * <li>key.deserializer</li>
    +   * <li>value.deserializer</li>
    +   * </ul>
    +   *
    +   */
    +  private Properties consumerProps;
    +
    +  /**
    +   * count the emitted message in each window<br>
    +   * non settable
    +   */
    +  private transient int emitCount = 0;
    +
    +  /**
    +   * Application name is used as group.id for kafka consumer
    +   */
    +  private transient String applicationName;
    +
    +  private transient AbstractKafkaPartitioner partitioner;
    +
    +  protected transient long currentWindowId;
    +
    +  /**
    +   * By default the strategy is one to one
    +   * @see PartitionStrategy
    +   */
    +  private PartitionStrategy strategy = PartitionStrategy.ONE_TO_ONE;
    +
    +  private transient long lastCheckTime = 0L;
    +
    +  private transient long lastRepartitionTime = 0L;
    +
    +  @AutoMetric
    +  private transient KafkaMetrics metrics;
    +
    +  @Override
    +  public void activate(Context.OperatorContext context)
    +  {
    +    consumerWrapper.start();
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +    consumerWrapper.stop();
    +  }
    +
    +  @Override
    +  public void checkpointed(long l)
    +  {
    +
    +  }
    +
    +  @Override
    +  public void committed(long windowId)
    +  {
    +    //ask kafka consumer wrapper to store the committed offsets
    +    for (Iterator<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> iter = offsetHistory.iterator(); iter.hasNext(); ) {
    +      Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>> item = iter.next();
    +      if (item.getLeft() <= windowId) {
    +        if (item.getLeft() == windowId) {
    +          consumerWrapper.commitOffsets(item.getRight());
    +        }
    +        iter.remove();
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void emitTuples()
    +  {
    +    int count = consumerWrapper.messageSize();
    +    if (maxTuplesPerWindow > 0) {
    +      count = Math.min(count, maxTuplesPerWindow - emitCount);
    +    }
    +    for (int i = 0; i < count; i++) {
    +      Pair<String, ConsumerRecord<byte[], byte[]>> tuple = consumerWrapper.pollMessage();
    +      ConsumerRecord<byte[], byte[]> msg = tuple.getRight();
    +      emitTuple(tuple.getLeft(), msg);
    +      AbstractKafkaPartitioner.PartitionMeta pm = new AbstractKafkaPartitioner.PartitionMeta(tuple.getLeft(),
    +          msg.topic(), msg.partition());
    +      offsetTrack.put(pm, msg.offset());
    +    }
    +    emitCount += count;
    +  }
    +
    +  protected abstract void emitTuple(String cluster, ConsumerRecord<byte[], byte[]> message);
    +
    +  @Override
    +  public void beginWindow(long wid)
    +  {
    +    emitCount = 0;
    +    currentWindowId = wid;
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    // copy current offset track to history memory
    +    Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetsWithWindow = new HashMap<>(offsetTrack);
    +    offsetHistory.add(Pair.of(currentWindowId, offsetsWithWindow));
    +
    +    //update metrics
    +    metrics.updateMetrics(clusters, consumerWrapper.getAllConsumerMetrics());
    +  }
    +
    +
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    applicationName = context.getValue(Context.DAGContext.APPLICATION_NAME);
    +    consumerWrapper.create(this);
    +    metrics = new KafkaMetrics(metricsRefreshInterval);
    +  }
    +
    +
    +  @Override
    +  public void teardown()
    +  {
    +
    +  }
    +
    +  private void initPartitioner()
    +  {
    +    if (partitioner == null) {
    +      logger.info("Initialize Partitioner");
    +      switch (strategy) {
    +        case ONE_TO_ONE:
    +          partitioner = new OneToOnePartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY:
    +          partitioner = new OneToManyPartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY_HEURISTIC:
    +          throw new UnsupportedOperationException("Not implemented yet");
    +        default:
    +          throw new RuntimeException("Invalid strategy");
    +      }
    +      logger.info("Actual Partitioner is {}", partitioner.getClass());
    +    }
    +
    +  }
    +
    +  @Override
    +  public Response processStats(BatchedOperatorStats batchedOperatorStats)
    +  {
    +    long t = System.currentTimeMillis();
    +    if (repartitionInterval < 0 || repartitionCheckInterval < 0 ||
    +        t - lastCheckTime < repartitionCheckInterval || t - lastRepartitionTime < repartitionInterval) {
    +      // return false if it's within repartitionCheckInterval since last time it check the stats
    +      Response response = new Response();
    +      response.repartitionRequired = false;
    +      return response;
    +    }
    +
    +    try {
    +      logger.debug("Process stats");
    +      initPartitioner();
    +      return partitioner.processStats(batchedOperatorStats);
    +    } finally {
    +      lastCheckTime = System.currentTimeMillis();
    +    }
    +  }
    +
    +  @Override
    +  public Collection<Partition<AbstractKafkaInputOperator>> definePartitions(
    +      Collection<Partition<AbstractKafkaInputOperator>> collection, PartitioningContext partitioningContext)
    +  {
    +    logger.debug("Define partitions");
    +    initPartitioner();
    +    return partitioner.definePartitions(collection, partitioningContext);
    +  }
    +
    +  @Override
    +  public void partitioned(Map<Integer, Partition<AbstractKafkaInputOperator>> map)
    +  {
    +    // update the last repartition time
    +    lastRepartitionTime = System.currentTimeMillis();
    +    initPartitioner();
    +    partitioner.partitioned(map);
    +  }
    +
    +  /**
    +   *
    +   * A callback from consumer after it commits the offset
    +   * @param map
    +   * @param e
    +   */
    +  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);
    +    }
    +  }
    +
    +
    +
    +  //---------------------------------------------setters and getters----------------------------------------
    +  public void setInitialPartitionCount(int partitionCount)
    +  {
    +    this.initialPartitionCount = partitionCount;
    +  }
    +
    +  public int getInitialPartitionCount()
    +  {
    +    return initialPartitionCount;
    +  }
    +
    +  public void setClusters(String clusters)
    +  {
    +    this.clusters = clusters.split(";");
    +  }
    +
    +  public String getClusters()
    +  {
    +    return Joiner.on(';').join(clusters);
    +  }
    +
    +  public void setTopics(String... topics)
    +  {
    +    this.topics = topics;
    +  }
    +
    +  public String[] getTopics()
    +  {
    +    return topics;
    +  }
    +
    +  public void setAssignment(Set<AbstractKafkaPartitioner.PartitionMeta> assignment)
    +  {
    +    this.assignment = assignment;
    +  }
    +
    +  public Set<AbstractKafkaPartitioner.PartitionMeta> getAssignment()
    +  {
    +    return assignment;
    +  }
    +
    +  public Map<AbstractKafkaPartitioner.PartitionMeta, Long> getOffsetTrack()
    +  {
    +    return offsetTrack;
    +  }
    +
    +  public void setStrategy(String policy)
    +  {
    +    this.strategy = PartitionStrategy.valueOf(policy.toUpperCase());
    +  }
    +
    +  public PartitionStrategy getStrategy()
    +  {
    +    return strategy;
    +  }
    +
    +  public void setInitialOffset(String initialOffset)
    +  {
    +    this.initialOffset = InitialOffset.valueOf(initialOffset.toUpperCase());
    +  }
    +
    +  public InitialOffset getInitialOffset()
    +  {
    +    return initialOffset;
    +  }
    +
    +  public String getApplicationName()
    --- End diff --
    
    Should be protected or package scope.


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r47979129
  
    --- Diff: kafka/src/main/java/com/datatorrent/contrib/kafka090/AbstractKafkaPartitioner.java ---
    @@ -0,0 +1,277 @@
    +/**
    + * 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 com.datatorrent.contrib.kafka090;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.kafka.clients.consumer.KafkaConsumer;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.TopicPartition;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * Abstract partitioner used to manage the partitions of kafka input operator.
    + * It use a number of kafka consumers(one for each cluster) to get the latest partition metadata for topics that
    + * the consumer subscribes and expose those to subclass which implements the assign method
    + *
    + * The partitioner is always stateless.
    + */
    +public abstract class AbstractKafkaPartitioner implements Partitioner<AbstractKafkaInputOperator>, StatsListener
    +{
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaPartitioner.class);
    +
    +  private static final String META_CONSUMER_GROUP_NAME = AbstractKafkaInputOperator.class.getName() + "META_GROUP";
    +
    +  protected String[] clusters;
    +
    +  protected String[] topics;
    +
    +  protected AbstractKafkaInputOperator prototypeOperator;
    +
    +  private KafkaConsumer[] metadataRefreshClients;
    +
    +
    +  private List<Set<AbstractKafkaPartitioner.PartitionMeta>> currentPartitions = new LinkedList<>(); // prevent null
    +
    +  public AbstractKafkaPartitioner(String[] clusters, String[] topics, AbstractKafkaInputOperator prototypeOperator)
    +  {
    +    this.clusters = clusters;
    +    this.topics = topics;
    +    this.prototypeOperator = prototypeOperator;
    +  }
    +
    +  abstract List<Set<PartitionMeta>> assign(Map<String, Map<String,List<PartitionInfo>>> metadata);
    +
    +
    +
    +  @Override
    +  public Collection<Partition<AbstractKafkaInputOperator>> definePartitions(Collection<Partition<AbstractKafkaInputOperator>> collection, PartitioningContext partitioningContext)
    +  {
    +
    +    initMetadataClients();
    +
    +    Map<String, Map<String, List<PartitionInfo>>> metadata = new HashMap<>();
    +
    +
    +    for (int i = 0; i < clusters.length; i++) {
    +      metadata.put(clusters[i], new HashMap<String, List<PartitionInfo>>());
    +      for (String topic : topics) {
    +        List<PartitionInfo> ptis = metadataRefreshClients[i].partitionsFor(topic);
    +        if (logger.isDebugEnabled()) {
    +          logger.debug("Partition metadata for topic {} : {}", topic, Joiner.on(';').join(ptis));
    +        }
    +        metadata.get(clusters[i]).put(topic, ptis);
    +      }
    +      metadataRefreshClients[i].close();
    +    }
    +
    +    metadataRefreshClients = null;
    +
    +    List<Set<AbstractKafkaPartitioner.PartitionMeta>> parts = assign(metadata);
    +
    +
    +    if (currentPartitions == parts || currentPartitions.equals(parts)) {
    +      logger.debug("No partition change found");
    +      return collection;
    +    } else {
    +      logger.info("Partition change detected: ");
    +      currentPartitions.clear();
    +      currentPartitions.addAll(parts);
    +      int i = 0;
    +      List<Partition<AbstractKafkaInputOperator>> result = new LinkedList<>();
    +      for (Iterator<Partition<AbstractKafkaInputOperator>> iter = collection.iterator(); iter.hasNext();) {
    +        Partition<AbstractKafkaInputOperator> nextPartition = iter.next();
    +        if (parts.remove(nextPartition.getPartitionedInstance().getAssignment())) {
    +          if (logger.isInfoEnabled()) {
    +            logger.info("[Existing] Partition {} with assignment {} ", i,
    +                Joiner.on(';').join(nextPartition.getPartitionedInstance().getAssignment()));
    +          }
    +          result.add(nextPartition);
    +          i++;
    +        }
    +      }
    +
    +      for (Set<AbstractKafkaPartitioner.PartitionMeta> partitionAssignment : parts) {
    +        if (logger.isInfoEnabled()) {
    +          logger.info("[New] Partition {} with assignment {} ", i,
    +              Joiner.on(';').join(partitionAssignment));
    +        }
    +        result.add(createPartition(partitionAssignment));
    +        i++;
    +      }
    +
    +
    +      return result;
    +    }
    +  }
    +
    +  @Override
    +  public void partitioned(Map<Integer, Partition<AbstractKafkaInputOperator>> map)
    +  {
    +
    +  }
    +
    +  @Override
    +  public Response processStats(BatchedOperatorStats batchedOperatorStats)
    +  {
    +    Response response = new Response();
    +    response.repartitionRequired = true;
    +    return response;
    +  }
    +
    +  protected Partitioner.Partition<AbstractKafkaInputOperator> createPartition(Set<AbstractKafkaPartitioner.PartitionMeta> partitionAssignment)
    +  {
    +    Kryo kryo = new Kryo();
    +    ByteArrayOutputStream bos = new ByteArrayOutputStream();
    +    Output output = new Output(bos);
    +    kryo.writeObject(output, prototypeOperator);
    +    output.close();
    +    Input lInput = new Input(bos.toByteArray());
    +    @SuppressWarnings("unchecked")
    +    Partitioner.Partition<AbstractKafkaInputOperator> p = new DefaultPartition<>(kryo.readObject(lInput, prototypeOperator.getClass()));
    --- End diff --
    
    Getting a compile error inside Eclipse, goes away with:
    ```
    Partitioner.Partition<AbstractKafkaInputOperator> p = new DefaultPartition<>((AbstractKafkaInputOperator)kryo.readObject(lInput, prototypeOperator.getClass()));
    ```
    SuppressWarnings can be removed with this change also.


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48452313
  
    --- 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) {
    --- End diff --
    
    Wrap the serializer to be able to count the bytes received for bandwidth control or are there other options to accomplish that?


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48917236
  
    --- Diff: kafka/src/main/java/org/apache/apex/malhar/kafka/AbstractKafkaInputOperator.java ---
    @@ -0,0 +1,500 @@
    +/**
    + * 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.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.tuple.Pair;
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +import org.apache.kafka.clients.consumer.OffsetAndMetadata;
    +import org.apache.kafka.clients.consumer.OffsetCommitCallback;
    +import org.apache.kafka.common.TopicPartition;
    +
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * The abstract kafka input operator using kafka 0.9.0 new consumer API
    + * A scalable, fault-tolerant, at-least-once kafka input operator
    + * Key features includes:
    + *
    + * <ol>
    + * <li>Out-of-box One-to-one and one-to-many partition strategy support plus customizable partition strategy
    + *    refer to AbstractKafkaPartitioner </li>
    + * <li>Fault-tolerant when the input operator goes down, it redeploys on other node</li>
    + * <li>At-least-once semantics for operator failure (no matter which operator fails)</li>
    + * <li>At-least-once semantics for cold restart (no data loss even if you restart the application)</li>
    + * <li>Multi-cluster support, one operator can consume data from more than one kafka clusters</li>
    + * <li>Multi-topic support, one operator can subscribe multiple topics</li>
    + * <li>Throughput control support, you can throttle number of tuple for each streaming window</li>
    + * </ol>
    + */
    +public abstract class AbstractKafkaInputOperator implements InputOperator, Operator.ActivationListener<Context.OperatorContext>, Operator.CheckpointListener, Partitioner<AbstractKafkaInputOperator>, StatsListener, OffsetCommitCallback
    +{
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaInputOperator.class);
    +
    +  public enum InitialOffset
    +  {
    +    EARLIEST, // consume from beginning of the partition every time when application restart
    +    LATEST, // consume from latest of the partition every time when application restart
    +    APPLICATION_OR_EARLIEST, // consume from committed position from last run or earliest if there is no committed offset(s)
    +    APPLICATION_OR_LATEST // consume from committed position from last run or latest if there is no committed offset(s)
    +  }
    +  /**
    +   *  Same setting as bootstrap.servers property to KafkaConsumer
    +   *  refer to http://kafka.apache.org/documentation.html#newconsumerconfigs
    +   *  To support multi cluster, you can have multiple bootstrap.servers separated by ";"
    +   */
    +  @NotNull
    +  private String[] clusters;
    +
    +  /**
    +   * The topics the operator consumes
    +   */
    +  @NotNull
    +  private String[] topics;
    +
    +  /**
    +   * Wrapper consumer object
    +   * It wraps KafkaConsumer, maintains consumer thread and store messages in a queue
    +   */
    +  private transient KafkaConsumerWrapper consumerWrapper = new KafkaConsumerWrapper();
    +
    +  /**
    +   * Assignment for each operator instance
    +   */
    +  private Set<AbstractKafkaPartitioner.PartitionMeta> assignment;
    +
    +  /**
    +   *  offset track for checkpoint
    +   */
    +  private Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetTrack = new HashMap<>();
    +
    +  /**
    +   * store offsets with window id, only keep offsets with windows that have not been committed
    +   */
    +  private transient List<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> offsetHistory = new LinkedList<>();
    +
    +  /**
    +   * initial partition count
    +   * only used with PartitionStrategy.ONE_TO_MANY
    +   */
    +  private int initialPartitionCount = 1;
    +
    +  /**
    +   * Minimal interval between 2 (re)partition actions
    +   */
    +  private long repartitionInterval = 30000L;
    +
    +  /**
    +   * Minimal interval between checking collected stats and decide whether it needs to repartition or not.
    +   * And minimal interval between 2 offset updates
    +   */
    +  private long repartitionCheckInterval = 5000L;
    +
    +
    +
    +  /**
    +   * maximum tuples allowed to be emitted in each window
    +   */
    +  @Min(1)
    +  private int maxTuplesPerWindow = Integer.MAX_VALUE;
    +
    +  /**
    +   *  Initial offset, it should be one of the following
    +   *  <ul>
    +   *    <li>earliest</li>
    +   *    <li>latest</li>
    +   *    <li>application_or_earliest</li>
    +   *    <li>application_or_latest</li>
    +   *  </ul>
    +   */
    +  private InitialOffset initialOffset = InitialOffset.APPLICATION_OR_LATEST;
    +
    +  /**
    +   * metrics refresh interval
    +   */
    +  private long metricsRefreshInterval = 5000L;
    +
    +  /**
    +   * @see <a href="http://kafka.apache.org/090/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#poll(long)">
    +   *   org.apache.kafka.clients.consumer.KafkaConsumer.poll</a>
    +   */
    +  private long consumerTimeout = 5000L;
    +
    +  /**
    +   * Number of messages kept in memory waiting for emission to downstream operator
    +   */
    +  private int holdingBufferSize = 1024;
    +
    +  /**
    +   * Extra kafka consumer properties
    +   * http://kafka.apache.org/090/documentation.html#newconsumerconfigs
    +   *
    +   * Please be aware that the properties below are set by the operator, don't override it
    +   *
    +   * <ul>
    +   * <li>bootstrap.servers</li>
    +   * <li>group.id</li>
    +   * <li>auto.offset.reset</li>
    +   * <li>enable.auto.commit</li>
    +   * <li>partition.assignment.strategy</li>
    +   * <li>key.deserializer</li>
    +   * <li>value.deserializer</li>
    +   * </ul>
    +   *
    +   */
    +  private Properties consumerProps;
    +
    +  /**
    +   * count the emitted message in each window<br>
    +   * non settable
    +   */
    +  private transient int emitCount = 0;
    +
    +  /**
    +   * Application name is used as group.id for kafka consumer
    +   */
    +  private transient String applicationName;
    +
    +  private transient AbstractKafkaPartitioner partitioner;
    +
    +  protected transient long currentWindowId;
    +
    +  /**
    +   * By default the strategy is one to one
    +   * @see PartitionStrategy
    +   */
    +  private PartitionStrategy strategy = PartitionStrategy.ONE_TO_ONE;
    +
    +  private transient long lastCheckTime = 0L;
    +
    +  private transient long lastRepartitionTime = 0L;
    +
    +  @AutoMetric
    +  private transient KafkaMetrics metrics;
    +
    +  @Override
    +  public void activate(Context.OperatorContext context)
    +  {
    +    consumerWrapper.start();
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +    consumerWrapper.stop();
    +  }
    +
    +  @Override
    +  public void checkpointed(long l)
    +  {
    +
    +  }
    +
    +  @Override
    +  public void committed(long windowId)
    +  {
    +    //ask kafka consumer wrapper to store the committed offsets
    +    for (Iterator<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> iter = offsetHistory.iterator(); iter.hasNext(); ) {
    +      Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>> item = iter.next();
    +      if (item.getLeft() <= windowId) {
    +        if (item.getLeft() == windowId) {
    +          consumerWrapper.commitOffsets(item.getRight());
    +        }
    +        iter.remove();
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void emitTuples()
    +  {
    +    int count = consumerWrapper.messageSize();
    +    if (maxTuplesPerWindow > 0) {
    +      count = Math.min(count, maxTuplesPerWindow - emitCount);
    +    }
    +    for (int i = 0; i < count; i++) {
    +      Pair<String, ConsumerRecord<byte[], byte[]>> tuple = consumerWrapper.pollMessage();
    +      ConsumerRecord<byte[], byte[]> msg = tuple.getRight();
    +      emitTuple(tuple.getLeft(), msg);
    +      AbstractKafkaPartitioner.PartitionMeta pm = new AbstractKafkaPartitioner.PartitionMeta(tuple.getLeft(),
    +          msg.topic(), msg.partition());
    +      offsetTrack.put(pm, msg.offset());
    +    }
    +    emitCount += count;
    +  }
    +
    +  protected abstract void emitTuple(String cluster, ConsumerRecord<byte[], byte[]> message);
    +
    +  @Override
    +  public void beginWindow(long wid)
    +  {
    +    emitCount = 0;
    +    currentWindowId = wid;
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    // copy current offset track to history memory
    +    Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetsWithWindow = new HashMap<>(offsetTrack);
    +    offsetHistory.add(Pair.of(currentWindowId, offsetsWithWindow));
    +
    +    //update metrics
    +    metrics.updateMetrics(clusters, consumerWrapper.getAllConsumerMetrics());
    +  }
    +
    +
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    applicationName = context.getValue(Context.DAGContext.APPLICATION_NAME);
    +    consumerWrapper.create(this);
    +    metrics = new KafkaMetrics(metricsRefreshInterval);
    +  }
    +
    +
    +  @Override
    +  public void teardown()
    +  {
    +
    +  }
    +
    +  private void initPartitioner()
    +  {
    +    if (partitioner == null) {
    +      logger.info("Initialize Partitioner");
    +      switch (strategy) {
    +        case ONE_TO_ONE:
    +          partitioner = new OneToOnePartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY:
    +          partitioner = new OneToManyPartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY_HEURISTIC:
    +          throw new UnsupportedOperationException("Not implemented yet");
    +        default:
    +          throw new RuntimeException("Invalid strategy");
    +      }
    +      logger.info("Actual Partitioner is {}", partitioner.getClass());
    +    }
    +
    +  }
    +
    +  @Override
    +  public Response processStats(BatchedOperatorStats batchedOperatorStats)
    +  {
    +    long t = System.currentTimeMillis();
    +    if (repartitionInterval < 0 || repartitionCheckInterval < 0 ||
    +        t - lastCheckTime < repartitionCheckInterval || t - lastRepartitionTime < repartitionInterval) {
    +      // return false if it's within repartitionCheckInterval since last time it check the stats
    +      Response response = new Response();
    +      response.repartitionRequired = false;
    +      return response;
    +    }
    +
    +    try {
    +      logger.debug("Process stats");
    +      initPartitioner();
    +      return partitioner.processStats(batchedOperatorStats);
    +    } finally {
    +      lastCheckTime = System.currentTimeMillis();
    +    }
    +  }
    +
    +  @Override
    +  public Collection<Partition<AbstractKafkaInputOperator>> definePartitions(
    +      Collection<Partition<AbstractKafkaInputOperator>> collection, PartitioningContext partitioningContext)
    +  {
    +    logger.debug("Define partitions");
    +    initPartitioner();
    +    return partitioner.definePartitions(collection, partitioningContext);
    +  }
    +
    +  @Override
    +  public void partitioned(Map<Integer, Partition<AbstractKafkaInputOperator>> map)
    +  {
    +    // update the last repartition time
    +    lastRepartitionTime = System.currentTimeMillis();
    +    initPartitioner();
    +    partitioner.partitioned(map);
    +  }
    +
    +  /**
    +   *
    +   * A callback from consumer after it commits the offset
    +   * @param map
    +   * @param e
    +   */
    +  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);
    +    }
    +  }
    +
    +
    +
    +  //---------------------------------------------setters and getters----------------------------------------
    +  public void setInitialPartitionCount(int partitionCount)
    +  {
    +    this.initialPartitionCount = partitionCount;
    +  }
    +
    +  public int getInitialPartitionCount()
    +  {
    +    return initialPartitionCount;
    +  }
    +
    +  public void setClusters(String clusters)
    +  {
    +    this.clusters = clusters.split(";");
    +  }
    +
    +  public String getClusters()
    +  {
    +    return Joiner.on(';').join(clusters);
    +  }
    +
    +  public void setTopics(String... topics)
    +  {
    +    this.topics = topics;
    +  }
    +
    +  public String[] getTopics()
    +  {
    +    return topics;
    +  }
    +
    +  public void setAssignment(Set<AbstractKafkaPartitioner.PartitionMeta> assignment)
    +  {
    +    this.assignment = assignment;
    +  }
    +
    +  public Set<AbstractKafkaPartitioner.PartitionMeta> getAssignment()
    +  {
    +    return assignment;
    +  }
    +
    +  public Map<AbstractKafkaPartitioner.PartitionMeta, Long> getOffsetTrack()
    +  {
    +    return offsetTrack;
    +  }
    +
    +  public void setStrategy(String policy)
    +  {
    +    this.strategy = PartitionStrategy.valueOf(policy.toUpperCase());
    +  }
    +
    +  public PartitionStrategy getStrategy()
    +  {
    +    return strategy;
    +  }
    +
    +  public void setInitialOffset(String initialOffset)
    +  {
    +    this.initialOffset = InitialOffset.valueOf(initialOffset.toUpperCase());
    +  }
    +
    +  public InitialOffset getInitialOffset()
    --- End diff --
    
    Should be JavaBean compliant getter/setter.


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48452268
  
    --- 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()) {
    --- End diff --
    
    What's the purpose of this implementation?


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48075562
  
    --- Diff: kafka/src/main/java/com/datatorrent/contrib/kafka090/AbstractKafkaInputOperator.java ---
    @@ -0,0 +1,410 @@
    +/**
    + * 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 com.datatorrent.contrib.kafka090;
    +
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.tuple.Pair;
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * The abstract kafka input operator using kafka 0.9.0 new consumer API
    + * A scalable, fault-tolerant, at-least-once kafka input operator
    + * Keu features includes:
    + * 1. Out-of-box One-to-one and one-to-many partition schema support plus customizable partition schem
    + *    refer to AbstractKafkaPartitioner
    + * 2. Fault-tolerant when the input operator goes down, it redeploys on other node
    + * 3. At-least-once semantics for operator failure (no matter which operator fails)
    + * 4. At-least-once semantics for cold restart (no data loss even if you restart the application)
    + * 5. Multi-cluster support, one operator can consume data from more than one kafka clusters
    + * 6. Multi-topic support, one operator can subscribe multiple topics
    + * 7. Throughput control support, you can throttle number of tuple for each streaming window
    + */
    +public abstract class AbstractKafkaInputOperator<K, V> implements InputOperator, Operator.ActivationListener<Context.OperatorContext>, Operator.CheckpointListener, Partitioner<AbstractKafkaInputOperator>, StatsListener
    +{
    +
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaInputOperator.class);
    +
    +  public enum InitialOffset {
    +    EARLIEST, // consume from beginning of the partition every time when application restart
    +    LATEST, // consume from latest of the partition every time when application restart
    +    APPLICATION_OR_EARLIEST, // consume from committed position from last run or earliest if there is no committed offset(s)
    +    APPLICATION_OR_LATEST // consume from committed position from last run or latest if there is no committed offset(s)
    +  }
    +  /**
    +   *  node separate by ',' and clusters separate by ';'
    +   *  ex: c1n1,c1n2;c2n1,c2n2
    +   */
    +  @NotNull
    +  private String[] clusters;
    +
    +  /**
    +   * The topics the operator consumes
    +   */
    +  @NotNull
    +  private String[] topics;
    +
    +  /**
    +   * Wrapper consumer object
    +   * It wraps KafkaConsumer, maintains consumer thread and store messages in a queue
    +   */
    +  private KafkaConsumerWrapper<K, V> consumerWrapper = new KafkaConsumerWrapper<>();
    +
    +  /**
    +   * Assignment for each operator instance
    +   */
    +  private Set<AbstractKafkaPartitioner.PartitionMeta> assignment;
    +
    +  /**
    +   *  offset track for checkpoint
    +   */
    +  private Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetTrack = new HashMap<>();
    +
    +  /**
    +   * store offsets with window id, only keep offsets with windows that have not been committed
    +   */
    +  private transient List<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> offsetHistory = new LinkedList<>();
    +
    +  /**
    +   * initial partition count
    +   * only used with PartitionStrategy.ONE_TO_MANY
    +   */
    +  private int initialPartitionCount = 1;
    --- End diff --
    
    This does not work when it is the property of another property (with configuration, you cannot assume the order in which properties are set). If the partitioner won't be a property then it may work. 


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by siyuanh <gi...@git.apache.org>.
Github user siyuanh commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48060846
  
    --- Diff: kafka/src/main/java/com/datatorrent/contrib/kafka090/AbstractKafkaPartitioner.java ---
    @@ -0,0 +1,277 @@
    +/**
    + * 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 com.datatorrent.contrib.kafka090;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.kafka.clients.consumer.KafkaConsumer;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.TopicPartition;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * Abstract partitioner used to manage the partitions of kafka input operator.
    + * It use a number of kafka consumers(one for each cluster) to get the latest partition metadata for topics that
    + * the consumer subscribes and expose those to subclass which implements the assign method
    + *
    + * The partitioner is always stateless.
    + */
    +public abstract class AbstractKafkaPartitioner implements Partitioner<AbstractKafkaInputOperator>, StatsListener
    +{
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaPartitioner.class);
    +
    +  private static final String META_CONSUMER_GROUP_NAME = AbstractKafkaInputOperator.class.getName() + "META_GROUP";
    +
    +  protected String[] clusters;
    +
    +  protected String[] topics;
    +
    +  protected AbstractKafkaInputOperator prototypeOperator;
    +
    +  private KafkaConsumer[] metadataRefreshClients;
    +
    +
    +  private List<Set<AbstractKafkaPartitioner.PartitionMeta>> currentPartitions = new LinkedList<>(); // prevent null
    +
    +  public AbstractKafkaPartitioner(String[] clusters, String[] topics, AbstractKafkaInputOperator prototypeOperator)
    +  {
    +    this.clusters = clusters;
    +    this.topics = topics;
    +    this.prototypeOperator = prototypeOperator;
    +  }
    +
    +  abstract List<Set<PartitionMeta>> assign(Map<String, Map<String,List<PartitionInfo>>> metadata);
    +
    +
    +
    +  @Override
    +  public Collection<Partition<AbstractKafkaInputOperator>> definePartitions(Collection<Partition<AbstractKafkaInputOperator>> collection, PartitioningContext partitioningContext)
    +  {
    +
    +    initMetadataClients();
    +
    +    Map<String, Map<String, List<PartitionInfo>>> metadata = new HashMap<>();
    +
    +
    +    for (int i = 0; i < clusters.length; i++) {
    +      metadata.put(clusters[i], new HashMap<String, List<PartitionInfo>>());
    +      for (String topic : topics) {
    +        List<PartitionInfo> ptis = metadataRefreshClients[i].partitionsFor(topic);
    +        if (logger.isDebugEnabled()) {
    +          logger.debug("Partition metadata for topic {} : {}", topic, Joiner.on(';').join(ptis));
    +        }
    +        metadata.get(clusters[i]).put(topic, ptis);
    +      }
    +      metadataRefreshClients[i].close();
    +    }
    +
    +    metadataRefreshClients = null;
    +
    +    List<Set<AbstractKafkaPartitioner.PartitionMeta>> parts = assign(metadata);
    +
    +
    +    if (currentPartitions == parts || currentPartitions.equals(parts)) {
    +      logger.debug("No partition change found");
    +      return collection;
    +    } else {
    +      logger.info("Partition change detected: ");
    +      currentPartitions.clear();
    +      currentPartitions.addAll(parts);
    +      int i = 0;
    +      List<Partition<AbstractKafkaInputOperator>> result = new LinkedList<>();
    +      for (Iterator<Partition<AbstractKafkaInputOperator>> iter = collection.iterator(); iter.hasNext();) {
    +        Partition<AbstractKafkaInputOperator> nextPartition = iter.next();
    +        if (parts.remove(nextPartition.getPartitionedInstance().getAssignment())) {
    +          if (logger.isInfoEnabled()) {
    +            logger.info("[Existing] Partition {} with assignment {} ", i,
    +                Joiner.on(';').join(nextPartition.getPartitionedInstance().getAssignment()));
    +          }
    +          result.add(nextPartition);
    +          i++;
    +        }
    +      }
    +
    +      for (Set<AbstractKafkaPartitioner.PartitionMeta> partitionAssignment : parts) {
    +        if (logger.isInfoEnabled()) {
    +          logger.info("[New] Partition {} with assignment {} ", i,
    +              Joiner.on(';').join(partitionAssignment));
    +        }
    +        result.add(createPartition(partitionAssignment));
    +        i++;
    +      }
    +
    +
    +      return result;
    +    }
    +  }
    +
    +  @Override
    +  public void partitioned(Map<Integer, Partition<AbstractKafkaInputOperator>> map)
    +  {
    +
    +  }
    +
    +  @Override
    +  public Response processStats(BatchedOperatorStats batchedOperatorStats)
    +  {
    +    Response response = new Response();
    +    response.repartitionRequired = true;
    +    return response;
    +  }
    +
    +  protected Partitioner.Partition<AbstractKafkaInputOperator> createPartition(Set<AbstractKafkaPartitioner.PartitionMeta> partitionAssignment)
    +  {
    +    Kryo kryo = new Kryo();
    +    ByteArrayOutputStream bos = new ByteArrayOutputStream();
    +    Output output = new Output(bos);
    +    kryo.writeObject(output, prototypeOperator);
    +    output.close();
    +    Input lInput = new Input(bos.toByteArray());
    +    @SuppressWarnings("unchecked")
    +    Partitioner.Partition<AbstractKafkaInputOperator> p = new DefaultPartition<>(kryo.readObject(lInput, prototypeOperator.getClass()));
    +    p.getPartitionedInstance().setAssignment(partitionAssignment);
    +    return p;
    +  }
    +  /**
    +   *
    +   */
    +  private void initMetadataClients()
    +  {
    +    if (metadataRefreshClients != null && metadataRefreshClients.length == clusters.length) {
    +      // The metadata client is active
    +      return;
    +    }
    +
    +    if (clusters == null || clusters.length == 0) {
    +      throw new IllegalStateException("clusters can not be null");
    +    }
    +
    +    metadataRefreshClients = new KafkaConsumer[clusters.length];
    +    int index = 0;
    +    for (String c : clusters) {
    +      Properties prop = new Properties();
    +      prop.put("group.id", META_CONSUMER_GROUP_NAME);
    +      prop.put("bootstrap.servers", c);
    +      prop.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer");
    +      prop.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer");
    +      prop.put("enable.auto.commit", "false");
    +      metadataRefreshClients[index++] = new KafkaConsumer(prop);
    +    }
    +  }
    +
    +  /**
    +   * The key object used in the assignment map for each operator
    +   */
    +  public static class PartitionMeta
    +  {
    +
    +    public PartitionMeta()
    --- End diff --
    
    It is used as return type for assign method in AbstractKafkaPartitioner class


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r47960858
  
    --- Diff: kafka/pom.xml ---
    @@ -0,0 +1,278 @@
    +<!--
    +
    +    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.
    +
    +-->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
    +         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +
    +  <parent>
    +    <groupId>org.apache.apex</groupId>
    +    <artifactId>malhar</artifactId>
    +    <version>3.3.0-incubating-SNAPSHOT</version>
    +  </parent>
    +
    +  <artifactId>malhar-kafka</artifactId>
    +  <name>Apache Apex Malhar (incubating) Contrib Library</name>
    +  <packaging>jar</packaging>
    +
    +  <properties>
    +    <!-- skip tests by default as they depend on manual setup -->
    +    <skip.kafka.module.tests>false</skip.kafka.module.tests>
    +    <skipTests>true</skipTests>
    +  </properties>
    +  <repositories>
    +    <repository>
    +      <id>datatorrent-3rd-party</id>
    +      <name>Embedded repository for dependencies not available online</name>
    +      <url>https://www.datatorrent.com/maven/content/repositories/thirdparty</url>
    +      <snapshots>
    +        <updatePolicy>daily</updatePolicy>
    +      </snapshots>
    +      <releases>
    +        <updatePolicy>daily</updatePolicy>
    +      </releases>
    +    </repository>
    +    <repository>
    +      <id>apache</id>
    +      <name>apache maven path for non-public repo</name>
    +      <url>https://repository.apache.org/content/groups/staging/</url>
    +    </repository>
    +  </repositories>
    +
    +  <build>
    +    <plugins>
    +      <!-- Publish tests jar -->
    +      <plugin>
    +        <groupId>org.apache.maven.plugins</groupId>
    +        <artifactId>maven-jar-plugin</artifactId>
    +        <version>2.4</version>
    +        <executions>
    +          <execution>
    +            <goals>
    +              <goal>test-jar</goal>
    +            </goals>
    +            <phase>package</phase>
    +          </execution>
    +        </executions>
    +      </plugin>
    +     <!-- create resource directory for xml javadoc-->
    +     <plugin>
    --- End diff --
    
    We should find a way to not repeat this in every project.


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by siyuanh <gi...@git.apache.org>.
Github user siyuanh commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48877262
  
    --- Diff: kafka/src/main/java/org/apache/apex/malhar/kafka/AbstractKafkaInputOperator.java ---
    @@ -0,0 +1,512 @@
    +/**
    + * 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.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.OffsetAndMetadata;
    +import org.apache.kafka.clients.consumer.OffsetCommitCallback;
    +import org.apache.kafka.common.TopicPartition;
    +import org.apache.kafka.common.serialization.Deserializer;
    +
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * The abstract kafka input operator using kafka 0.9.0 new consumer API
    + * A scalable, fault-tolerant, at-least-once kafka input operator
    + * Key features includes:
    + * 1. Out-of-box One-to-one and one-to-many partition strategy support plus customizable partition strategy
    + *    refer to AbstractKafkaPartitioner
    + * 2. Fault-tolerant when the input operator goes down, it redeploys on other node
    + * 3. At-least-once semantics for operator failure (no matter which operator fails)
    + * 4. At-least-once semantics for cold restart (no data loss even if you restart the application)
    + * 5. Multi-cluster support, one operator can consume data from more than one kafka clusters
    + * 6. Multi-topic support, one operator can subscribe multiple topics
    + * 7. Throughput control support, you can throttle number of tuple for each streaming window
    + */
    +public abstract class AbstractKafkaInputOperator<K, V> implements InputOperator, Operator.ActivationListener<Context.OperatorContext>, Operator.CheckpointListener, Partitioner<AbstractKafkaInputOperator>, StatsListener, OffsetCommitCallback
    +{
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaInputOperator.class);
    +
    +  public long getMetricsRefreshInterval()
    +  {
    +    return metricsRefreshInterval;
    +  }
    +
    +  public void setMetricsRefreshInterval(long metricsRefreshInterval)
    +  {
    +    this.metricsRefreshInterval = metricsRefreshInterval;
    +  }
    +
    +  public enum InitialOffset
    +  {
    +    EARLIEST, // consume from beginning of the partition every time when application restart
    +    LATEST, // consume from latest of the partition every time when application restart
    +    APPLICATION_OR_EARLIEST, // consume from committed position from last run or earliest if there is no committed offset(s)
    +    APPLICATION_OR_LATEST // consume from committed position from last run or latest if there is no committed offset(s)
    +  }
    +  /**
    +   *  Same setting as bootstrap.servers property to KafkaConsumer
    +   *  refer to http://kafka.apache.org/documentation.html#newconsumerconfigs
    +   *  To support multi cluster, you can have multiple bootstrap.servers separated by ";"
    +   */
    +  @NotNull
    +  private String[] clusters;
    +
    +  /**
    +   * The topics the operator consumes
    +   */
    +  @NotNull
    +  private String[] topics;
    +
    +  /**
    +   * Wrapper consumer object
    +   * It wraps KafkaConsumer, maintains consumer thread and store messages in a queue
    +   */
    +  private KafkaConsumerWrapper consumerWrapper = new KafkaConsumerWrapper();
    +
    +  /**
    +   * Assignment for each operator instance
    +   */
    +  private Set<AbstractKafkaPartitioner.PartitionMeta> assignment;
    +
    +  /**
    +   *  offset track for checkpoint
    +   */
    +  private Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetTrack = new HashMap<>();
    +
    +  /**
    +   * store offsets with window id, only keep offsets with windows that have not been committed
    +   */
    +  private transient List<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> offsetHistory = new LinkedList<>();
    +
    +  /**
    +   * initial partition count
    +   * only used with PartitionStrategy.ONE_TO_MANY
    +   */
    +  private int initialPartitionCount = 1;
    +
    +  // Minimal interval between 2 (re)partition actions
    +  private long repartitionInterval = 30000L;
    +
    +  // Minimal interval between checking collected stats and decide whether it needs to repartition or not.
    +  // And minimal interval between 2 offset updates
    +  private long repartitionCheckInterval = 5000L;
    +
    +  // class name of the key deserializer
    +  protected String keyDeserializer;
    +
    +  // class name of the value deserializer
    +  protected String valueDeserializer;
    +
    +  /**
    +   * maximum tuples allowed to be emitted in each window
    +   */
    +  @Min(1)
    +  private int maxTuplesPerWindow = Integer.MAX_VALUE;
    +
    +  private InitialOffset initialOffset = InitialOffset.APPLICATION_OR_LATEST;
    +
    +  private long metricsRefreshInterval = 5000L;
    +
    +  /**
    +   * extra kafka consumer properties
    +   * http://kafka.apache.org/090/documentation.html#newconsumerconfigs
    +   *
    +   * Please be aware that the properties below are set by the operator, don't override it
    +   *
    +   * bootstrap.servers
    +   * group.id
    +   * auto.offset.reset
    +   * enable.auto.commit
    +   * partition.assignment.strategy
    +   *
    +   *
    +   */
    +  private Properties consumerProps;
    +
    +  /**
    +   * count the emitted message in each window
    +   * non settable
    +   */
    +  private transient int emitCount = 0;
    +
    +  /**
    +   * Application name is used as group.id for kafka consumer
    +   */
    +  private transient String applicationName;
    +
    +  private transient AbstractKafkaPartitioner partitioner;
    +
    +  protected transient long currentWindowId;
    +
    +  // By default the partition policy is 1:1
    +  private PartitionStrategy strategy = PartitionStrategy.ONE_TO_ONE;
    +
    +  private transient long lastCheckTime = 0L;
    +
    +  private transient long lastRepartitionTime = 0L;
    +
    +  @AutoMetric
    +  private transient KafkaMetrics metrics;
    +
    +  private transient Deserializer<K> keyDeser;
    +
    +  private transient Deserializer<V> valueDeser;
    +
    +  @Override
    +  public void activate(Context.OperatorContext context)
    +  {
    +    consumerWrapper.start();
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +    consumerWrapper.stop();
    +  }
    +
    +  @Override
    +  public void checkpointed(long l)
    +  {
    +
    +  }
    +
    +  @Override
    +  public void committed(long windowId)
    +  {
    +    //ask kafka consumer wrapper to store the committed offsets
    +    for (Iterator<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> iter = offsetHistory.iterator(); iter.hasNext(); ) {
    +      Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>> item = iter.next();
    +      if (item.getLeft() <= windowId) {
    +        if (item.getLeft() == windowId) {
    +          consumerWrapper.commitOffsets(item.getRight());
    +        }
    +        iter.remove();
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void emitTuples()
    +  {
    +    int count = consumerWrapper.messageSize();
    +    if (maxTuplesPerWindow > 0) {
    +      count = Math.min(count, maxTuplesPerWindow - emitCount);
    +    }
    +    for (int i = 0; i < count; i++) {
    +      Pair<String, ConsumerRecord<byte[], byte[]>> tuple = consumerWrapper.pollMessage();
    +      ConsumerRecord<byte[], byte[]> msg = tuple.getRight();
    +      emitTuple(tuple.getLeft(), msg.topic(), msg.partition(), msg.offset(),
    +          keyDeser.deserialize(msg.topic(), msg.key()), valueDeser.deserialize(msg.topic(), msg.value()));
    +      AbstractKafkaPartitioner.PartitionMeta pm = new AbstractKafkaPartitioner.PartitionMeta(tuple.getLeft(),
    +          msg.topic(), msg.partition());
    +      offsetTrack.put(pm, msg.offset());
    +    }
    +    emitCount += count;
    +  }
    +
    +  protected abstract void emitTuple(String cluster, String topic, long partition, long offset, K key, V value);
    +
    +  @Override
    +  public void beginWindow(long wid)
    +  {
    +    emitCount = 0;
    +    currentWindowId = wid;
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    // copy current offset track to history memory
    +    Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetsWithWindow = new HashMap<>(offsetTrack);
    +    offsetHistory.add(Pair.of(currentWindowId, offsetsWithWindow));
    +
    +    //update metrics
    +    metrics.updateMetrics(clusters, consumerWrapper.getAllConsumerMetrics());
    +  }
    +
    +
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    applicationName = context.getValue(Context.DAGContext.APPLICATION_NAME);
    +    consumerWrapper.create(this);
    +    metrics = new KafkaMetrics(metricsRefreshInterval);
    +    initDeserializer();
    +  }
    +
    +  private void initDeserializer()
    +  {
    +    Map<String, Object> extraProeprties = new HashMap<>();
    +    if (consumerProps != null) {
    +      for (Map.Entry<Object, Object> e : consumerProps.entrySet()) {
    +        extraProeprties.put(e.getKey().toString(), e.getValue());
    +      }
    +    }
    +    try {
    +      String cname = keyDeserializer;
    +      if (cname == null) {
    +        cname = consumerProps.getProperty(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG);
    +      }
    +      keyDeser = (Deserializer<K>)Class.forName(cname).newInstance();
    +      keyDeser.configure(extraProeprties, true);
    +      cname = valueDeserializer;
    +      if (cname == null) {
    +        cname = consumerProps.getProperty(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG);
    +      }
    +      valueDeser = (Deserializer<V>)Class.forName(cname).newInstance();
    +      valueDeser.configure(extraProeprties, false);
    +    } catch (Exception e) {
    +      throw new RuntimeException(e);
    +    }
    +
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +
    +  }
    +
    +  private void initPartitioner()
    +  {
    +    if (partitioner == null) {
    +      logger.info("Initialize Partitioner");
    +      switch (strategy) {
    +        case ONE_TO_ONE:
    +          partitioner = new OneToOnePartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY:
    +          partitioner = new OneToManyPartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY_HEURISTIC:
    +          throw new UnsupportedOperationException("Not implemented yet");
    +        default:
    +          throw new RuntimeException("Invalid strategy");
    +      }
    +      logger.info("Actual Partitioner is {}", partitioner.getClass());
    +    }
    +
    +  }
    +
    +  @Override
    +  public Response processStats(BatchedOperatorStats batchedOperatorStats)
    +  {
    +    long t = System.currentTimeMillis();
    +    if (repartitionInterval < 0 || repartitionCheckInterval < 0 ||
    +        t - lastCheckTime < repartitionCheckInterval || t - lastRepartitionTime < repartitionInterval) {
    +      // return false if it's within repartitionCheckInterval since last time it check the stats
    +      Response response = new Response();
    +      response.repartitionRequired = false;
    +      return response;
    +    }
    +
    +    try {
    +      logger.debug("Process stats");
    +      initPartitioner();
    +      return partitioner.processStats(batchedOperatorStats);
    +    } finally {
    +      lastCheckTime = System.currentTimeMillis();
    +    }
    +  }
    +
    +  @Override
    +  public Collection<Partition<AbstractKafkaInputOperator>> definePartitions(
    +      Collection<Partition<AbstractKafkaInputOperator>> collection, PartitioningContext partitioningContext)
    +  {
    +    logger.debug("Define partitions");
    +    initPartitioner();
    +    return partitioner.definePartitions(collection, partitioningContext);
    +  }
    +
    +  @Override
    +  public void partitioned(Map<Integer, Partition<AbstractKafkaInputOperator>> map)
    +  {
    +    // update the last repartition time
    +    lastRepartitionTime = System.currentTimeMillis();
    +    initPartitioner();
    +    partitioner.partitioned(map);
    +  }
    +
    +  /**
    +   *
    +   * A callback from consumer after it commits the offset
    +   * @param map
    +   * @param e
    +   */
    +  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);
    +    }
    +  }
    +
    +
    +
    +  //---------------------------------------------setters and getters----------------------------------------
    +  public void setInitialPartitionCount(int partitionCount)
    +  {
    +    this.initialPartitionCount = partitionCount;
    +  }
    +
    +  public int getInitialPartitionCount()
    +  {
    +    return initialPartitionCount;
    +  }
    +
    +  public void setClusters(String clusters)
    +  {
    +    this.clusters = clusters.split(";");
    +  }
    +
    +  public String getClusters()
    +  {
    +    return Joiner.on(';').join(clusters);
    +  }
    +
    +  public void setTopics(String... topics)
    +  {
    +    this.topics = topics;
    +  }
    +
    +  public String[] getTopics()
    +  {
    +    return topics;
    +  }
    +
    +  public void setConsumerWrapper(KafkaConsumerWrapper consumerWrapper)
    --- End diff --
    
    There are 2 extra properties you can set on the wrapper, but I think it's better to move them out to Operator so users can easily set the value and we can make this wrapper object transient


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/incubator-apex-malhar/pull/134


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by siyuanh <gi...@git.apache.org>.
Github user siyuanh commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48061524
  
    --- Diff: kafka/src/main/java/com/datatorrent/contrib/kafka090/AbstractKafkaInputOperator.java ---
    @@ -0,0 +1,410 @@
    +/**
    + * 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 com.datatorrent.contrib.kafka090;
    +
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.tuple.Pair;
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * The abstract kafka input operator using kafka 0.9.0 new consumer API
    + * A scalable, fault-tolerant, at-least-once kafka input operator
    + * Keu features includes:
    + * 1. Out-of-box One-to-one and one-to-many partition schema support plus customizable partition schem
    + *    refer to AbstractKafkaPartitioner
    + * 2. Fault-tolerant when the input operator goes down, it redeploys on other node
    + * 3. At-least-once semantics for operator failure (no matter which operator fails)
    + * 4. At-least-once semantics for cold restart (no data loss even if you restart the application)
    + * 5. Multi-cluster support, one operator can consume data from more than one kafka clusters
    + * 6. Multi-topic support, one operator can subscribe multiple topics
    + * 7. Throughput control support, you can throttle number of tuple for each streaming window
    + */
    +public abstract class AbstractKafkaInputOperator<K, V> implements InputOperator, Operator.ActivationListener<Context.OperatorContext>, Operator.CheckpointListener, Partitioner<AbstractKafkaInputOperator>, StatsListener
    +{
    +
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaInputOperator.class);
    +
    +  public enum InitialOffset {
    +    EARLIEST, // consume from beginning of the partition every time when application restart
    +    LATEST, // consume from latest of the partition every time when application restart
    +    APPLICATION_OR_EARLIEST, // consume from committed position from last run or earliest if there is no committed offset(s)
    +    APPLICATION_OR_LATEST // consume from committed position from last run or latest if there is no committed offset(s)
    +  }
    +  /**
    +   *  node separate by ',' and clusters separate by ';'
    +   *  ex: c1n1,c1n2;c2n1,c2n2
    +   */
    +  @NotNull
    +  private String[] clusters;
    +
    +  /**
    +   * The topics the operator consumes
    +   */
    +  @NotNull
    +  private String[] topics;
    +
    +  /**
    +   * Wrapper consumer object
    +   * It wraps KafkaConsumer, maintains consumer thread and store messages in a queue
    +   */
    +  private KafkaConsumerWrapper<K, V> consumerWrapper = new KafkaConsumerWrapper<>();
    +
    +  /**
    +   * Assignment for each operator instance
    +   */
    +  private Set<AbstractKafkaPartitioner.PartitionMeta> assignment;
    +
    +  /**
    +   *  offset track for checkpoint
    +   */
    +  private Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetTrack = new HashMap<>();
    +
    +  /**
    +   * store offsets with window id, only keep offsets with windows that have not been committed
    +   */
    +  private transient List<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> offsetHistory = new LinkedList<>();
    +
    +  /**
    +   * initial partition count
    +   * only used with PartitionStrategy.ONE_TO_MANY
    +   */
    +  private int initialPartitionCount = 1;
    --- End diff --
    
    I want this to be easily set on Operator level, actually I want to pull ConsumerWrapper properties to Operator as well. I think that is a easier way for developers who use this operator 


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48452284
  
    --- Diff: kafka/src/test/java/org/apache/apex/malhar/kafka/KafkaInputOperatorTest.java ---
    @@ -0,0 +1,221 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.Collection;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.concurrent.CountDownLatch;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.kafka.clients.consumer.ConsumerConfig;
    +import org.apache.kafka.common.serialization.Deserializer;
    +import org.apache.kafka.common.serialization.StringDeserializer;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.DAG.Locality;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.LocalMode;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +/**
    + * A bunch of test to verify the input operator will be automatically partitioned per kafka partition This test is launching its
    + * own Kafka cluster.
    + */
    +@RunWith(Parameterized.class)
    +public class KafkaInputOperatorTest extends KafkaOperatorTestBase
    +{
    +
    +  private int totalBrokers = 0;
    +
    +
    +
    +  @Parameterized.Parameters(name = "multi-cluster: {0}, multi-partition: {1}")
    +  public static Collection<Boolean[]> testScenario()
    +  {
    +    return Arrays.asList(new Boolean[][]{{true, false}, // multi cluster with single partition
    +      {true, true}, // multi cluster with multi partitions
    +      {false, true}, // single cluster with multi partitions
    +      {false, false}, // single cluster with single partitions
    +    });
    +  }
    +
    +  public KafkaInputOperatorTest(boolean hasMultiCluster, boolean hasMultiPartition)
    +  {
    +    // This class want to initialize several kafka brokers for multiple partitions
    +    this.hasMultiCluster = hasMultiCluster;
    +    this.hasMultiPartition = hasMultiPartition;
    +    int cluster = 1 + (hasMultiCluster ? 1 : 0);
    +    totalBrokers = (1 + (hasMultiPartition ? 1 : 0)) * cluster;
    +
    +  }
    +
    +  static final org.slf4j.Logger logger = LoggerFactory.getLogger(KafkaInputOperatorTest.class);
    --- End diff --
    
    Why not private?


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by gauravgopi123 <gi...@git.apache.org>.
Github user gauravgopi123 commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48373578
  
    --- Diff: kafka/src/main/java/org/apache/apex/malhar/kafka/AbstractKafkaInputOperator.java ---
    @@ -0,0 +1,455 @@
    +/**
    + * 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.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.tuple.Pair;
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +import org.apache.kafka.common.serialization.Deserializer;
    +
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * The abstract kafka input operator using kafka 0.9.0 new consumer API
    + * A scalable, fault-tolerant, at-least-once kafka input operator
    + * Key features includes:
    + * 1. Out-of-box One-to-one and one-to-many partition strategy support plus customizable partition strategy
    + *    refer to AbstractKafkaPartitioner
    + * 2. Fault-tolerant when the input operator goes down, it redeploys on other node
    + * 3. At-least-once semantics for operator failure (no matter which operator fails)
    + * 4. At-least-once semantics for cold restart (no data loss even if you restart the application)
    + * 5. Multi-cluster support, one operator can consume data from more than one kafka clusters
    + * 6. Multi-topic support, one operator can subscribe multiple topics
    + * 7. Throughput control support, you can throttle number of tuple for each streaming window
    + */
    +public abstract class AbstractKafkaInputOperator<K, V> implements InputOperator, Operator.ActivationListener<Context.OperatorContext>, Operator.CheckpointListener, Partitioner<AbstractKafkaInputOperator>, StatsListener
    +{
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaInputOperator.class);
    +
    +  public long getMetricsRefreshInterval()
    +  {
    +    return metricsRefreshInterval;
    +  }
    +
    +  public void setMetricsRefreshInterval(long metricsRefreshInterval)
    +  {
    +    this.metricsRefreshInterval = metricsRefreshInterval;
    +  }
    +
    +  public enum InitialOffset
    +  {
    +    EARLIEST, // consume from beginning of the partition every time when application restart
    +    LATEST, // consume from latest of the partition every time when application restart
    +    APPLICATION_OR_EARLIEST, // consume from committed position from last run or earliest if there is no committed offset(s)
    +    APPLICATION_OR_LATEST // consume from committed position from last run or latest if there is no committed offset(s)
    +  }
    +  /**
    +   *  Same setting as bootstrap.servers property to KafkaConsumer
    +   *  refer to http://kafka.apache.org/documentation.html#newconsumerconfigs
    +   *  To support multi cluster, you can have multiple bootstrap.servers separated by ";"
    +   */
    +  @NotNull
    +  private String[] clusters;
    +
    +  /**
    +   * The topics the operator consumes
    +   */
    +  @NotNull
    +  private String[] topics;
    +
    +  /**
    +   * Wrapper consumer object
    +   * It wraps KafkaConsumer, maintains consumer thread and store messages in a queue
    +   */
    +  private KafkaConsumerWrapper<K, V> consumerWrapper = new KafkaConsumerWrapper<>();
    +
    +  /**
    +   * Assignment for each operator instance
    +   */
    +  private Set<AbstractKafkaPartitioner.PartitionMeta> assignment;
    +
    +  /**
    +   *  offset track for checkpoint
    +   */
    +  private Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetTrack = new HashMap<>();
    +
    +  /**
    +   * store offsets with window id, only keep offsets with windows that have not been committed
    +   */
    +  private transient List<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> offsetHistory = new LinkedList<>();
    +
    +  /**
    +   * initial partition count
    +   * only used with PartitionStrategy.ONE_TO_MANY
    +   */
    +  private int initialPartitionCount = 1;
    +
    +  // Minimal interval between 2 (re)partition actions
    +  private long repartitionInterval = 30000L;
    +
    +  // Minimal interval between checking collected stats and decide whether it needs to repartition or not.
    +  // And minimal interval between 2 offset updates
    +  private long repartitionCheckInterval = 5000L;
    +
    +  private Class<? extends Deserializer<K>> keyDeserializer;
    +
    +  private Class<? extends Deserializer<V>> valueDeserializer;
    +
    +  /**
    +   * maximum tuples allowed to be emitted in each window
    +   */
    +  @Min(1)
    +  private int maxTuplesPerWindow = Integer.MAX_VALUE;
    +
    +  private InitialOffset initialOffset = InitialOffset.APPLICATION_OR_LATEST;
    +
    +  private long metricsRefreshInterval = 5000L;
    +
    +  /**
    +   * extra kafka consumer properties
    +   * http://kafka.apache.org/090/documentation.html#newconsumerconfigs
    +   *
    +   * Please be aware that the properties below are set by the operator, don't override it
    +   *
    +   * bootstrap.servers
    +   * group.id
    +   * auto.offset.reset
    +   * enable.auto.commit
    +   * partition.assignment.strategy
    +   *
    +   *
    +   */
    +  private Properties consumerProps;
    +
    +  /**
    +   * count the emitted message in each window
    +   * non settable
    +   */
    +  private transient int emitCount = 0;
    +
    +  /**
    +   * Application name is used as group.id for kafka consumer
    +   */
    +  private transient String applicationName;
    +
    +  private transient AbstractKafkaPartitioner partitioner;
    +
    +  protected transient long currentWindowId;
    +
    +  // By default the partition policy is 1:1
    +  private PartitionStrategy strategy = PartitionStrategy.ONE_TO_ONE;
    +
    +  private transient long lastCheckTime = 0L;
    +
    +  private transient long lastRepartitionTime = 0L;
    +
    +  @AutoMetric
    +  private transient KafkaMetrics metrics;
    +
    +  @Override
    +  public void activate(Context.OperatorContext context)
    +  {
    +    consumerWrapper.start();
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +    consumerWrapper.stop();
    +  }
    +
    +  @Override
    +  public void checkpointed(long l)
    +  {
    +
    +  }
    +
    +  @Override
    +  public void committed(long windowId)
    +  {
    +    //ask kafka consumer wrapper to store the committed offsets
    +    for (Iterator<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> iter = offsetHistory.iterator(); iter.hasNext(); ) {
    +      Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>> item = iter.next();
    +      if (item.getLeft() <= windowId) {
    +        if (item.getLeft() == windowId) {
    +          consumerWrapper.commitOffsets(item.getRight());
    +        }
    +        iter.remove();
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void emitTuples()
    +  {
    +    int count = consumerWrapper.messageSize();
    +    if (maxTuplesPerWindow > 0) {
    +      count = Math.min(count, maxTuplesPerWindow - emitCount);
    +    }
    +    for (int i = 0; i < count; i++) {
    +      Pair<String, ConsumerRecord<K, V>> tuple = consumerWrapper.pollMessage();
    +      ConsumerRecord<K, V> msg = tuple.getRight();
    +      emitTuple(tuple.getLeft(), msg);
    +      AbstractKafkaPartitioner.PartitionMeta pm = new AbstractKafkaPartitioner.PartitionMeta(tuple.getLeft(),
    +          msg.topic(), msg.partition());
    +      offsetTrack.put(pm, msg.offset());
    +    }
    +    emitCount += count;
    +  }
    +
    +  protected abstract void emitTuple(String cluster, ConsumerRecord<K, V> msg);
    +
    +  @Override
    +  public void beginWindow(long wid)
    +  {
    +    emitCount = 0;
    +    currentWindowId = wid;
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    // copy current offset track to history memory
    +    Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetsWithWindow = new HashMap<>(offsetTrack);
    +    offsetHistory.add(Pair.of(currentWindowId, offsetsWithWindow));
    +
    +    //update metrics
    +    metrics.updateMetrics(clusters, consumerWrapper.getAllConsumerMetrics());
    +  }
    +
    +
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    applicationName = context.getValue(Context.DAGContext.APPLICATION_NAME);
    +    consumerWrapper.create(this);
    +    metrics = new KafkaMetrics(metricsRefreshInterval);
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +
    +  }
    +
    +  private void initPartitioner()
    +  {
    +    if (partitioner == null) {
    +      logger.info("Initialize Partitioner");
    +      switch (strategy) {
    +        case ONE_TO_ONE:
    +          partitioner = new OneToOnePartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY:
    +          partitioner = new OneToManyPartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY_HEURISTIC:
    +          throw new UnsupportedOperationException("Not implemented yet");
    +        default:
    +          throw new RuntimeException("Invalid strategy");
    +      }
    +      logger.info("Actual Partitioner is {}", partitioner.getClass());
    +    }
    +
    +  }
    +
    +  @Override
    +  public Response processStats(BatchedOperatorStats batchedOperatorStats)
    +  {
    +    long t = System.currentTimeMillis();
    +    if (repartitionInterval < 0 || repartitionCheckInterval < 0 ||
    +        t - lastCheckTime < repartitionCheckInterval || t - lastRepartitionTime < repartitionInterval) {
    +      // return false if it's within repartitionCheckInterval since last time it check the stats
    +      Response response = new Response();
    +      response.repartitionRequired = false;
    +      return response;
    +    }
    +
    +    try {
    +      logger.debug("Process stats");
    +      initPartitioner();
    +      return partitioner.processStats(batchedOperatorStats);
    --- End diff --
    
    Correct me if I am wrong, but from this it looks like after repartitionInterval the repartitioning will always happen as partitioner.processStats evaluates to AbstractKafkaPartitioner.processStats and there repartitionRequired = true


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by gauravgopi123 <gi...@git.apache.org>.
Github user gauravgopi123 commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48380261
  
    --- Diff: kafka/src/main/java/org/apache/apex/malhar/kafka/AbstractKafkaPartitioner.java ---
    @@ -0,0 +1,276 @@
    +/**
    + * 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.ByteArrayOutputStream;
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.kafka.clients.consumer.KafkaConsumer;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.TopicPartition;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * Abstract partitioner used to manage the partitions of kafka input operator.
    + * It use a number of kafka consumers(one for each cluster) to get the latest partition metadata for topics that
    + * the consumer subscribes and expose those to subclass which implements the assign method
    + *
    + * The partitioner is always stateless.
    + */
    +public abstract class AbstractKafkaPartitioner implements Partitioner<AbstractKafkaInputOperator>, StatsListener
    +{
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaPartitioner.class);
    +
    +  private static final String META_CONSUMER_GROUP_NAME = AbstractKafkaInputOperator.class.getName() + "META_GROUP";
    +
    +  protected String[] clusters;
    +
    +  protected String[] topics;
    +
    +  protected AbstractKafkaInputOperator prototypeOperator;
    +
    +  private KafkaConsumer[] metadataRefreshClients;
    +
    +
    +  private List<Set<AbstractKafkaPartitioner.PartitionMeta>> currentPartitions = new LinkedList<>(); // prevent null
    +
    +  public AbstractKafkaPartitioner(String[] clusters, String[] topics, AbstractKafkaInputOperator prototypeOperator)
    +  {
    +    this.clusters = clusters;
    +    this.topics = topics;
    +    this.prototypeOperator = prototypeOperator;
    +  }
    +
    +  abstract List<Set<PartitionMeta>> assign(Map<String, Map<String,List<PartitionInfo>>> metadata);
    +
    +
    +
    +  @Override
    +  public Collection<Partition<AbstractKafkaInputOperator>> definePartitions(Collection<Partition<AbstractKafkaInputOperator>> collection, PartitioningContext partitioningContext)
    +  {
    +
    +    initMetadataClients();
    +
    +    Map<String, Map<String, List<PartitionInfo>>> metadata = new HashMap<>();
    +
    +
    +    for (int i = 0; i < clusters.length; i++) {
    +      metadata.put(clusters[i], new HashMap<String, List<PartitionInfo>>());
    +      for (String topic : topics) {
    +        List<PartitionInfo> ptis = metadataRefreshClients[i].partitionsFor(topic);
    +        if (logger.isDebugEnabled()) {
    +          logger.debug("Partition metadata for topic {} : {}", topic, Joiner.on(';').join(ptis));
    +        }
    +        metadata.get(clusters[i]).put(topic, ptis);
    +      }
    +      metadataRefreshClients[i].close();
    +    }
    +
    +    metadataRefreshClients = null;
    +
    +    List<Set<AbstractKafkaPartitioner.PartitionMeta>> parts = assign(metadata);
    +
    +
    +    if (currentPartitions == parts || currentPartitions.equals(parts)) {
    +      logger.debug("No partition change found");
    +      return collection;
    +    } else {
    +      logger.info("Partition change detected: ");
    +      currentPartitions.clear();
    +      currentPartitions.addAll(parts);
    +      int i = 0;
    +      List<Partition<AbstractKafkaInputOperator>> result = new LinkedList<>();
    +      for (Iterator<Partition<AbstractKafkaInputOperator>> iter = collection.iterator(); iter.hasNext();) {
    +        Partition<AbstractKafkaInputOperator> nextPartition = iter.next();
    +        if (parts.remove(nextPartition.getPartitionedInstance().getAssignment())) {
    +          if (logger.isInfoEnabled()) {
    +            logger.info("[Existing] Partition {} with assignment {} ", i,
    +                Joiner.on(';').join(nextPartition.getPartitionedInstance().getAssignment()));
    +          }
    +          result.add(nextPartition);
    +          i++;
    +        }
    +      }
    +
    +      for (Set<AbstractKafkaPartitioner.PartitionMeta> partitionAssignment : parts) {
    +        if (logger.isInfoEnabled()) {
    +          logger.info("[New] Partition {} with assignment {} ", i,
    +              Joiner.on(';').join(partitionAssignment));
    +        }
    +        result.add(createPartition(partitionAssignment));
    +        i++;
    +      }
    +
    --- End diff --
    
    extra line break


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by siyuanh <gi...@git.apache.org>.
Github user siyuanh commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48507270
  
    --- 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) {
    --- End diff --
    
    We can have the consumer metrics from the consumer API. But if we want to control the bandwidth bytes/window ex., there is no explicit way to do that. One thing I can think of is we always use ByteArrayDeserializer and use the actual Deserializer to deserialize the message in operator code, but I don't know if it's a right idea. 


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r47983322
  
    --- Diff: kafka/src/main/java/com/datatorrent/contrib/kafka090/AbstractKafkaInputOperator.java ---
    @@ -0,0 +1,410 @@
    +/**
    + * 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 com.datatorrent.contrib.kafka090;
    +
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.tuple.Pair;
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * The abstract kafka input operator using kafka 0.9.0 new consumer API
    + * A scalable, fault-tolerant, at-least-once kafka input operator
    + * Keu features includes:
    + * 1. Out-of-box One-to-one and one-to-many partition schema support plus customizable partition schem
    + *    refer to AbstractKafkaPartitioner
    + * 2. Fault-tolerant when the input operator goes down, it redeploys on other node
    + * 3. At-least-once semantics for operator failure (no matter which operator fails)
    + * 4. At-least-once semantics for cold restart (no data loss even if you restart the application)
    + * 5. Multi-cluster support, one operator can consume data from more than one kafka clusters
    + * 6. Multi-topic support, one operator can subscribe multiple topics
    + * 7. Throughput control support, you can throttle number of tuple for each streaming window
    + */
    +public abstract class AbstractKafkaInputOperator<K, V> implements InputOperator, Operator.ActivationListener<Context.OperatorContext>, Operator.CheckpointListener, Partitioner<AbstractKafkaInputOperator>, StatsListener
    +{
    +
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaInputOperator.class);
    +
    +  public enum InitialOffset {
    +    EARLIEST, // consume from beginning of the partition every time when application restart
    +    LATEST, // consume from latest of the partition every time when application restart
    +    APPLICATION_OR_EARLIEST, // consume from committed position from last run or earliest if there is no committed offset(s)
    +    APPLICATION_OR_LATEST // consume from committed position from last run or latest if there is no committed offset(s)
    +  }
    +  /**
    +   *  node separate by ',' and clusters separate by ';'
    +   *  ex: c1n1,c1n2;c2n1,c2n2
    +   */
    +  @NotNull
    +  private String[] clusters;
    +
    +  /**
    +   * The topics the operator consumes
    +   */
    +  @NotNull
    +  private String[] topics;
    +
    +  /**
    +   * Wrapper consumer object
    +   * It wraps KafkaConsumer, maintains consumer thread and store messages in a queue
    +   */
    +  private KafkaConsumerWrapper<K, V> consumerWrapper = new KafkaConsumerWrapper<>();
    +
    +  /**
    +   * Assignment for each operator instance
    +   */
    +  private Set<AbstractKafkaPartitioner.PartitionMeta> assignment;
    +
    +  /**
    +   *  offset track for checkpoint
    +   */
    +  private Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetTrack = new HashMap<>();
    +
    +  /**
    +   * store offsets with window id, only keep offsets with windows that have not been committed
    +   */
    +  private transient List<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> offsetHistory = new LinkedList<>();
    +
    +  /**
    +   * initial partition count
    +   * only used with PartitionStrategy.ONE_TO_MANY
    +   */
    +  private int initialPartitionCount = 1;
    +
    +  // Minimal interval between 2 (re)partition actions
    +  private long repartitionInterval = 30000L;
    +
    +  // Minimal interval between checking collected stats and decide whether it needs to repartition or not.
    +  // And minimal interval between 2 offset updates
    +  private long repartitionCheckInterval = 5000L;
    +
    +  /**
    +   * maximum tuples allowed to be emitted in each window
    +   */
    +  @Min(1)
    +  private int maxTuplesPerWindow = Integer.MAX_VALUE;
    +
    +  private InitialOffset initialOffset = InitialOffset.APPLICATION_OR_LATEST;
    +
    +  /**
    +   * extra kafka consumer properties
    +   * http://kafka.apache.org/090/documentation.html#newconsumerconfigs
    +   *
    +   * Please be aware that the properties below are set by the operator, don't override it
    +   *
    +   * bootstrap.servers
    +   * group.id
    +   * auto.offset.reset
    +   * enable.auto.commit
    +   * partition.assignment.strategy
    +   *
    +   */
    +  private Properties consumerProps;
    +
    +  /**
    +   * count the emitted message in each window
    +   * non settable
    +   */
    +  private transient int emitCount = 0;
    +
    +  /**
    +   * Application name is used as group.id for kafka consumer
    +   */
    +  private transient String applicationName;
    +
    +  private transient AbstractKafkaPartitioner partitioner;
    +
    +  protected transient long currentWindowId;
    +
    +  protected transient int operatorId;
    +
    +  // By default the partition policy is 1:1
    +  private PartitionStrategy strategy = PartitionStrategy.ONE_TO_ONE;
    +
    +  private transient long lastCheckTime = 0L;
    +
    +  private transient long lastRepartitionTime = 0L;
    +
    +
    +  @Override
    +  public void activate(Context.OperatorContext context)
    +  {
    +    consumerWrapper.start();
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +    consumerWrapper.stop();
    +  }
    +
    +  @Override
    +  public void checkpointed(long l)
    +  {
    +
    +  }
    +
    +  @Override
    +  public void committed(long windowId)
    +  {
    +    //ask kafka consumer wrapper to store the committed offsets
    +    for (Iterator<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> iter = offsetHistory.iterator(); iter.hasNext(); ) {
    +      Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>> item = iter.next();
    +      if (item.getLeft() <= windowId) {
    +        if (item.getLeft() == windowId) {
    +          consumerWrapper.commitOffsets(item.getRight());
    +        }
    +        iter.remove();
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void emitTuples()
    +  {
    +    int count = consumerWrapper.messageSize();
    +    if (maxTuplesPerWindow > 0) {
    +      count = Math.min(count, maxTuplesPerWindow - emitCount);
    +    }
    +    for (int i = 0; i < count; i++) {
    +      Pair<String, ConsumerRecord<K, V>> tuple = consumerWrapper.pollMessage();
    +      ConsumerRecord<K, V> msg = tuple.getRight();
    +      emitTuple(tuple.getLeft(), msg);
    +      AbstractKafkaPartitioner.PartitionMeta pm = new AbstractKafkaPartitioner.PartitionMeta(tuple.getLeft(),
    +        msg.topic(), msg.partition());
    +      offsetTrack.put(pm, msg.offset());
    +    }
    +    emitCount += count;
    +  }
    +
    +  protected abstract void emitTuple(String cluster, ConsumerRecord<K, V> msg);
    +
    +  @Override
    +  public void beginWindow(long wid)
    +  {
    +    emitCount = 0;
    +    currentWindowId = wid;
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    // copy current offset track to history memory
    +    Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetsWithWindow = new HashMap<>(offsetTrack);
    +    offsetHistory.add(Pair.of(currentWindowId, offsetsWithWindow));
    +  }
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    applicationName = context.getValue(Context.DAGContext.APPLICATION_NAME);
    +    consumerWrapper.create(this);
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +
    +  }
    +
    +  private void initPartitioner()
    +  {
    +    if(partitioner == null) {
    +      logger.info("Initialize Partitioner");
    +      switch (strategy) {
    +        case ONE_TO_ONE:
    +          partitioner = new OneToOnePartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY:
    +          partitioner = new OneToManyPartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY_HEURISTIC:
    +          throw new UnsupportedOperationException("Not implemented yet");
    +      }
    +      logger.info("Actual Partitioner is {}", partitioner.getClass());
    +    }
    +
    +  }
    +
    +  @Override
    +  public Response processStats(BatchedOperatorStats batchedOperatorStats)
    +  {
    +    long t = System.currentTimeMillis();
    +
    +    if (repartitionInterval < 0 || repartitionCheckInterval < 0 ||
    +      t - lastCheckTime < repartitionCheckInterval || t - lastRepartitionTime < repartitionInterval) {
    +      // return false if it's within repartitionCheckInterval since last time it check the stats
    +      logger.debug("Within Repartition Check Interval {} or Repartition Interval {}", repartitionCheckInterval,
    +        repartitionInterval);
    +      Response response = new Response();
    +      response.repartitionRequired = false;
    +      return response;
    +    }
    +
    +    try {
    +      logger.debug("Process stats");
    +      initPartitioner();
    +      return partitioner.processStats(batchedOperatorStats);
    +    } finally {
    +      lastCheckTime = System.currentTimeMillis();
    +    }
    +  }
    +
    +  @Override
    +  public Collection<Partition<AbstractKafkaInputOperator>> definePartitions(
    +    Collection<Partition<AbstractKafkaInputOperator>> collection, PartitioningContext partitioningContext)
    +  {
    +    logger.debug("Define partitions");
    +    initPartitioner();
    +    return partitioner.definePartitions(collection, partitioningContext);
    +  }
    +
    +  @Override
    +  public void partitioned(Map<Integer, Partition<AbstractKafkaInputOperator>> map)
    +  {
    +    // update the last repartition time
    +    lastRepartitionTime = System.currentTimeMillis();
    +    initPartitioner();
    +    partitioner.partitioned(map);
    +  }
    +
    +
    +  //---------------------------------------------setters and getters----------------------------------------
    +  public void setInitialPartitionCount(int partitionCount)
    +  {
    +    this.initialPartitionCount = partitionCount;
    +  }
    +
    +  public int getInitialPartitionCount()
    +  {
    +    return initialPartitionCount;
    +  }
    +
    +  public void setClusters(String clusters)
    +  {
    +    this.clusters = clusters.split(";");
    +  }
    +
    +  public String getClusters()
    +  {
    +    return Joiner.on(';').join(clusters);
    +  }
    +
    +  public void setTopics(String... topics)
    +  {
    +    this.topics = topics;
    +  }
    +
    +  public String[] getTopics()
    +  {
    +    return topics;
    +  }
    +
    +  public void setConsumerWrapper(KafkaConsumerWrapper consumerWrapper)
    --- End diff --
    
    raw type warning, occurs elsewhere also


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r47960735
  
    --- Diff: kafka/pom.xml ---
    @@ -0,0 +1,278 @@
    +<!--
    +
    +    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.
    +
    +-->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
    +         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +
    +  <parent>
    +    <groupId>org.apache.apex</groupId>
    +    <artifactId>malhar</artifactId>
    +    <version>3.3.0-incubating-SNAPSHOT</version>
    +  </parent>
    +
    +  <artifactId>malhar-kafka</artifactId>
    +  <name>Apache Apex Malhar (incubating) Contrib Library</name>
    +  <packaging>jar</packaging>
    +
    +  <properties>
    +    <!-- skip tests by default as they depend on manual setup -->
    +    <skip.kafka.module.tests>false</skip.kafka.module.tests>
    +    <skipTests>true</skipTests>
    +  </properties>
    +  <repositories>
    +    <repository>
    +      <id>datatorrent-3rd-party</id>
    --- End diff --
    
    What is this needed for?


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48647193
  
    --- Diff: kafka/src/main/java/org/apache/apex/malhar/kafka/AbstractKafkaInputOperator.java ---
    @@ -0,0 +1,512 @@
    +/**
    + * 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.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.OffsetAndMetadata;
    +import org.apache.kafka.clients.consumer.OffsetCommitCallback;
    +import org.apache.kafka.common.TopicPartition;
    +import org.apache.kafka.common.serialization.Deserializer;
    +
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * The abstract kafka input operator using kafka 0.9.0 new consumer API
    + * A scalable, fault-tolerant, at-least-once kafka input operator
    + * Key features includes:
    + * 1. Out-of-box One-to-one and one-to-many partition strategy support plus customizable partition strategy
    + *    refer to AbstractKafkaPartitioner
    + * 2. Fault-tolerant when the input operator goes down, it redeploys on other node
    + * 3. At-least-once semantics for operator failure (no matter which operator fails)
    + * 4. At-least-once semantics for cold restart (no data loss even if you restart the application)
    + * 5. Multi-cluster support, one operator can consume data from more than one kafka clusters
    + * 6. Multi-topic support, one operator can subscribe multiple topics
    + * 7. Throughput control support, you can throttle number of tuple for each streaming window
    + */
    +public abstract class AbstractKafkaInputOperator<K, V> implements InputOperator, Operator.ActivationListener<Context.OperatorContext>, Operator.CheckpointListener, Partitioner<AbstractKafkaInputOperator>, StatsListener, OffsetCommitCallback
    +{
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaInputOperator.class);
    +
    +  public long getMetricsRefreshInterval()
    +  {
    +    return metricsRefreshInterval;
    +  }
    +
    +  public void setMetricsRefreshInterval(long metricsRefreshInterval)
    +  {
    +    this.metricsRefreshInterval = metricsRefreshInterval;
    +  }
    +
    +  public enum InitialOffset
    +  {
    +    EARLIEST, // consume from beginning of the partition every time when application restart
    +    LATEST, // consume from latest of the partition every time when application restart
    +    APPLICATION_OR_EARLIEST, // consume from committed position from last run or earliest if there is no committed offset(s)
    +    APPLICATION_OR_LATEST // consume from committed position from last run or latest if there is no committed offset(s)
    +  }
    +  /**
    +   *  Same setting as bootstrap.servers property to KafkaConsumer
    +   *  refer to http://kafka.apache.org/documentation.html#newconsumerconfigs
    +   *  To support multi cluster, you can have multiple bootstrap.servers separated by ";"
    +   */
    +  @NotNull
    +  private String[] clusters;
    +
    +  /**
    +   * The topics the operator consumes
    +   */
    +  @NotNull
    +  private String[] topics;
    +
    +  /**
    +   * Wrapper consumer object
    +   * It wraps KafkaConsumer, maintains consumer thread and store messages in a queue
    +   */
    +  private KafkaConsumerWrapper consumerWrapper = new KafkaConsumerWrapper();
    +
    +  /**
    +   * Assignment for each operator instance
    +   */
    +  private Set<AbstractKafkaPartitioner.PartitionMeta> assignment;
    +
    +  /**
    +   *  offset track for checkpoint
    +   */
    +  private Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetTrack = new HashMap<>();
    +
    +  /**
    +   * store offsets with window id, only keep offsets with windows that have not been committed
    +   */
    +  private transient List<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> offsetHistory = new LinkedList<>();
    +
    +  /**
    +   * initial partition count
    +   * only used with PartitionStrategy.ONE_TO_MANY
    +   */
    +  private int initialPartitionCount = 1;
    +
    +  // Minimal interval between 2 (re)partition actions
    +  private long repartitionInterval = 30000L;
    +
    +  // Minimal interval between checking collected stats and decide whether it needs to repartition or not.
    +  // And minimal interval between 2 offset updates
    +  private long repartitionCheckInterval = 5000L;
    +
    +  // class name of the key deserializer
    +  protected String keyDeserializer;
    --- End diff --
    
    Why String, why not objects? However, given that we convert in emitTuples, we probably don't need these at all.


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r47982758
  
    --- Diff: kafka/src/main/java/com/datatorrent/contrib/kafka090/AbstractKafkaInputOperator.java ---
    @@ -0,0 +1,410 @@
    +/**
    + * 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 com.datatorrent.contrib.kafka090;
    +
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.tuple.Pair;
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * The abstract kafka input operator using kafka 0.9.0 new consumer API
    + * A scalable, fault-tolerant, at-least-once kafka input operator
    + * Keu features includes:
    + * 1. Out-of-box One-to-one and one-to-many partition schema support plus customizable partition schem
    + *    refer to AbstractKafkaPartitioner
    + * 2. Fault-tolerant when the input operator goes down, it redeploys on other node
    + * 3. At-least-once semantics for operator failure (no matter which operator fails)
    + * 4. At-least-once semantics for cold restart (no data loss even if you restart the application)
    + * 5. Multi-cluster support, one operator can consume data from more than one kafka clusters
    + * 6. Multi-topic support, one operator can subscribe multiple topics
    + * 7. Throughput control support, you can throttle number of tuple for each streaming window
    + */
    +public abstract class AbstractKafkaInputOperator<K, V> implements InputOperator, Operator.ActivationListener<Context.OperatorContext>, Operator.CheckpointListener, Partitioner<AbstractKafkaInputOperator>, StatsListener
    +{
    +
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaInputOperator.class);
    +
    +  public enum InitialOffset {
    +    EARLIEST, // consume from beginning of the partition every time when application restart
    +    LATEST, // consume from latest of the partition every time when application restart
    +    APPLICATION_OR_EARLIEST, // consume from committed position from last run or earliest if there is no committed offset(s)
    +    APPLICATION_OR_LATEST // consume from committed position from last run or latest if there is no committed offset(s)
    +  }
    +  /**
    +   *  node separate by ',' and clusters separate by ';'
    --- End diff --
    
    Add more description.


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48379780
  
    --- Diff: kafka/pom.xml ---
    @@ -0,0 +1,235 @@
    +<!--
    +
    +    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.
    +
    +-->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
    +         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +
    +  <parent>
    +    <groupId>org.apache.apex</groupId>
    +    <artifactId>malhar</artifactId>
    +    <version>3.3.0-incubating-SNAPSHOT</version>
    +  </parent>
    +
    +  <artifactId>malhar-kafka</artifactId>
    +  <name>Apache Apex Malhar (incubating) Kafka Support</name>
    +  <packaging>jar</packaging>
    +
    +  <properties>
    +    <!-- skip tests by default as they depend on manual setup -->
    +    <skip.kafka.module.tests>false</skip.kafka.module.tests>
    --- End diff --
    
    Why are tests skipped?


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48647206
  
    --- Diff: kafka/src/main/java/org/apache/apex/malhar/kafka/AbstractKafkaInputOperator.java ---
    @@ -0,0 +1,512 @@
    +/**
    + * 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.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +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.OffsetAndMetadata;
    +import org.apache.kafka.clients.consumer.OffsetCommitCallback;
    +import org.apache.kafka.common.TopicPartition;
    +import org.apache.kafka.common.serialization.Deserializer;
    +
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * The abstract kafka input operator using kafka 0.9.0 new consumer API
    + * A scalable, fault-tolerant, at-least-once kafka input operator
    + * Key features includes:
    + * 1. Out-of-box One-to-one and one-to-many partition strategy support plus customizable partition strategy
    + *    refer to AbstractKafkaPartitioner
    + * 2. Fault-tolerant when the input operator goes down, it redeploys on other node
    + * 3. At-least-once semantics for operator failure (no matter which operator fails)
    + * 4. At-least-once semantics for cold restart (no data loss even if you restart the application)
    + * 5. Multi-cluster support, one operator can consume data from more than one kafka clusters
    + * 6. Multi-topic support, one operator can subscribe multiple topics
    + * 7. Throughput control support, you can throttle number of tuple for each streaming window
    + */
    +public abstract class AbstractKafkaInputOperator<K, V> implements InputOperator, Operator.ActivationListener<Context.OperatorContext>, Operator.CheckpointListener, Partitioner<AbstractKafkaInputOperator>, StatsListener, OffsetCommitCallback
    +{
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaInputOperator.class);
    +
    +  public long getMetricsRefreshInterval()
    +  {
    +    return metricsRefreshInterval;
    +  }
    +
    +  public void setMetricsRefreshInterval(long metricsRefreshInterval)
    +  {
    +    this.metricsRefreshInterval = metricsRefreshInterval;
    +  }
    +
    +  public enum InitialOffset
    +  {
    +    EARLIEST, // consume from beginning of the partition every time when application restart
    +    LATEST, // consume from latest of the partition every time when application restart
    +    APPLICATION_OR_EARLIEST, // consume from committed position from last run or earliest if there is no committed offset(s)
    +    APPLICATION_OR_LATEST // consume from committed position from last run or latest if there is no committed offset(s)
    +  }
    +  /**
    +   *  Same setting as bootstrap.servers property to KafkaConsumer
    +   *  refer to http://kafka.apache.org/documentation.html#newconsumerconfigs
    +   *  To support multi cluster, you can have multiple bootstrap.servers separated by ";"
    +   */
    +  @NotNull
    +  private String[] clusters;
    +
    +  /**
    +   * The topics the operator consumes
    +   */
    +  @NotNull
    +  private String[] topics;
    +
    +  /**
    +   * Wrapper consumer object
    +   * It wraps KafkaConsumer, maintains consumer thread and store messages in a queue
    +   */
    +  private KafkaConsumerWrapper consumerWrapper = new KafkaConsumerWrapper();
    +
    +  /**
    +   * Assignment for each operator instance
    +   */
    +  private Set<AbstractKafkaPartitioner.PartitionMeta> assignment;
    +
    +  /**
    +   *  offset track for checkpoint
    +   */
    +  private Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetTrack = new HashMap<>();
    +
    +  /**
    +   * store offsets with window id, only keep offsets with windows that have not been committed
    +   */
    +  private transient List<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> offsetHistory = new LinkedList<>();
    +
    +  /**
    +   * initial partition count
    +   * only used with PartitionStrategy.ONE_TO_MANY
    +   */
    +  private int initialPartitionCount = 1;
    +
    +  // Minimal interval between 2 (re)partition actions
    +  private long repartitionInterval = 30000L;
    +
    +  // Minimal interval between checking collected stats and decide whether it needs to repartition or not.
    +  // And minimal interval between 2 offset updates
    +  private long repartitionCheckInterval = 5000L;
    +
    +  // class name of the key deserializer
    +  protected String keyDeserializer;
    +
    +  // class name of the value deserializer
    +  protected String valueDeserializer;
    +
    +  /**
    +   * maximum tuples allowed to be emitted in each window
    +   */
    +  @Min(1)
    +  private int maxTuplesPerWindow = Integer.MAX_VALUE;
    +
    +  private InitialOffset initialOffset = InitialOffset.APPLICATION_OR_LATEST;
    +
    +  private long metricsRefreshInterval = 5000L;
    +
    +  /**
    +   * extra kafka consumer properties
    +   * http://kafka.apache.org/090/documentation.html#newconsumerconfigs
    +   *
    +   * Please be aware that the properties below are set by the operator, don't override it
    +   *
    +   * bootstrap.servers
    +   * group.id
    +   * auto.offset.reset
    +   * enable.auto.commit
    +   * partition.assignment.strategy
    +   *
    +   *
    +   */
    +  private Properties consumerProps;
    +
    +  /**
    +   * count the emitted message in each window
    +   * non settable
    +   */
    +  private transient int emitCount = 0;
    +
    +  /**
    +   * Application name is used as group.id for kafka consumer
    +   */
    +  private transient String applicationName;
    +
    +  private transient AbstractKafkaPartitioner partitioner;
    +
    +  protected transient long currentWindowId;
    +
    +  // By default the partition policy is 1:1
    +  private PartitionStrategy strategy = PartitionStrategy.ONE_TO_ONE;
    +
    +  private transient long lastCheckTime = 0L;
    +
    +  private transient long lastRepartitionTime = 0L;
    +
    +  @AutoMetric
    +  private transient KafkaMetrics metrics;
    +
    +  private transient Deserializer<K> keyDeser;
    +
    +  private transient Deserializer<V> valueDeser;
    +
    +  @Override
    +  public void activate(Context.OperatorContext context)
    +  {
    +    consumerWrapper.start();
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +    consumerWrapper.stop();
    +  }
    +
    +  @Override
    +  public void checkpointed(long l)
    +  {
    +
    +  }
    +
    +  @Override
    +  public void committed(long windowId)
    +  {
    +    //ask kafka consumer wrapper to store the committed offsets
    +    for (Iterator<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> iter = offsetHistory.iterator(); iter.hasNext(); ) {
    +      Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>> item = iter.next();
    +      if (item.getLeft() <= windowId) {
    +        if (item.getLeft() == windowId) {
    +          consumerWrapper.commitOffsets(item.getRight());
    +        }
    +        iter.remove();
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void emitTuples()
    +  {
    +    int count = consumerWrapper.messageSize();
    +    if (maxTuplesPerWindow > 0) {
    +      count = Math.min(count, maxTuplesPerWindow - emitCount);
    +    }
    +    for (int i = 0; i < count; i++) {
    +      Pair<String, ConsumerRecord<byte[], byte[]>> tuple = consumerWrapper.pollMessage();
    +      ConsumerRecord<byte[], byte[]> msg = tuple.getRight();
    +      emitTuple(tuple.getLeft(), msg.topic(), msg.partition(), msg.offset(),
    +          keyDeser.deserialize(msg.topic(), msg.key()), valueDeser.deserialize(msg.topic(), msg.value()));
    --- End diff --
    
    Should leave conversion to emitTuple and derived class.


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48075108
  
    --- Diff: kafka/src/main/java/com/datatorrent/contrib/kafka090/AbstractKafkaPartitioner.java ---
    @@ -0,0 +1,277 @@
    +/**
    + * 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 com.datatorrent.contrib.kafka090;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.kafka.clients.consumer.KafkaConsumer;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.TopicPartition;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * Abstract partitioner used to manage the partitions of kafka input operator.
    + * It use a number of kafka consumers(one for each cluster) to get the latest partition metadata for topics that
    + * the consumer subscribes and expose those to subclass which implements the assign method
    + *
    + * The partitioner is always stateless.
    + */
    +public abstract class AbstractKafkaPartitioner implements Partitioner<AbstractKafkaInputOperator>, StatsListener
    +{
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaPartitioner.class);
    +
    +  private static final String META_CONSUMER_GROUP_NAME = AbstractKafkaInputOperator.class.getName() + "META_GROUP";
    +
    +  protected String[] clusters;
    +
    +  protected String[] topics;
    +
    +  protected AbstractKafkaInputOperator prototypeOperator;
    +
    +  private KafkaConsumer[] metadataRefreshClients;
    +
    +
    +  private List<Set<AbstractKafkaPartitioner.PartitionMeta>> currentPartitions = new LinkedList<>(); // prevent null
    +
    +  public AbstractKafkaPartitioner(String[] clusters, String[] topics, AbstractKafkaInputOperator prototypeOperator)
    +  {
    +    this.clusters = clusters;
    +    this.topics = topics;
    +    this.prototypeOperator = prototypeOperator;
    +  }
    +
    +  abstract List<Set<PartitionMeta>> assign(Map<String, Map<String,List<PartitionInfo>>> metadata);
    +
    +
    +
    +  @Override
    +  public Collection<Partition<AbstractKafkaInputOperator>> definePartitions(Collection<Partition<AbstractKafkaInputOperator>> collection, PartitioningContext partitioningContext)
    +  {
    +
    +    initMetadataClients();
    +
    +    Map<String, Map<String, List<PartitionInfo>>> metadata = new HashMap<>();
    +
    +
    +    for (int i = 0; i < clusters.length; i++) {
    +      metadata.put(clusters[i], new HashMap<String, List<PartitionInfo>>());
    +      for (String topic : topics) {
    +        List<PartitionInfo> ptis = metadataRefreshClients[i].partitionsFor(topic);
    +        if (logger.isDebugEnabled()) {
    +          logger.debug("Partition metadata for topic {} : {}", topic, Joiner.on(';').join(ptis));
    +        }
    +        metadata.get(clusters[i]).put(topic, ptis);
    +      }
    +      metadataRefreshClients[i].close();
    +    }
    +
    +    metadataRefreshClients = null;
    +
    +    List<Set<AbstractKafkaPartitioner.PartitionMeta>> parts = assign(metadata);
    +
    +
    +    if (currentPartitions == parts || currentPartitions.equals(parts)) {
    +      logger.debug("No partition change found");
    +      return collection;
    +    } else {
    +      logger.info("Partition change detected: ");
    +      currentPartitions.clear();
    +      currentPartitions.addAll(parts);
    +      int i = 0;
    +      List<Partition<AbstractKafkaInputOperator>> result = new LinkedList<>();
    +      for (Iterator<Partition<AbstractKafkaInputOperator>> iter = collection.iterator(); iter.hasNext();) {
    +        Partition<AbstractKafkaInputOperator> nextPartition = iter.next();
    +        if (parts.remove(nextPartition.getPartitionedInstance().getAssignment())) {
    +          if (logger.isInfoEnabled()) {
    +            logger.info("[Existing] Partition {} with assignment {} ", i,
    +                Joiner.on(';').join(nextPartition.getPartitionedInstance().getAssignment()));
    +          }
    +          result.add(nextPartition);
    +          i++;
    +        }
    +      }
    +
    +      for (Set<AbstractKafkaPartitioner.PartitionMeta> partitionAssignment : parts) {
    +        if (logger.isInfoEnabled()) {
    +          logger.info("[New] Partition {} with assignment {} ", i,
    +              Joiner.on(';').join(partitionAssignment));
    +        }
    +        result.add(createPartition(partitionAssignment));
    +        i++;
    +      }
    +
    +
    +      return result;
    +    }
    +  }
    +
    +  @Override
    +  public void partitioned(Map<Integer, Partition<AbstractKafkaInputOperator>> map)
    +  {
    +
    +  }
    +
    +  @Override
    +  public Response processStats(BatchedOperatorStats batchedOperatorStats)
    +  {
    +    Response response = new Response();
    +    response.repartitionRequired = true;
    +    return response;
    +  }
    +
    +  protected Partitioner.Partition<AbstractKafkaInputOperator> createPartition(Set<AbstractKafkaPartitioner.PartitionMeta> partitionAssignment)
    +  {
    +    Kryo kryo = new Kryo();
    +    ByteArrayOutputStream bos = new ByteArrayOutputStream();
    +    Output output = new Output(bos);
    +    kryo.writeObject(output, prototypeOperator);
    +    output.close();
    +    Input lInput = new Input(bos.toByteArray());
    +    @SuppressWarnings("unchecked")
    +    Partitioner.Partition<AbstractKafkaInputOperator> p = new DefaultPartition<>(kryo.readObject(lInput, prototypeOperator.getClass()));
    +    p.getPartitionedInstance().setAssignment(partitionAssignment);
    +    return p;
    +  }
    +  /**
    +   *
    +   */
    +  private void initMetadataClients()
    +  {
    +    if (metadataRefreshClients != null && metadataRefreshClients.length == clusters.length) {
    +      // The metadata client is active
    +      return;
    +    }
    +
    +    if (clusters == null || clusters.length == 0) {
    +      throw new IllegalStateException("clusters can not be null");
    +    }
    +
    +    metadataRefreshClients = new KafkaConsumer[clusters.length];
    +    int index = 0;
    +    for (String c : clusters) {
    +      Properties prop = new Properties();
    +      prop.put("group.id", META_CONSUMER_GROUP_NAME);
    +      prop.put("bootstrap.servers", c);
    +      prop.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer");
    +      prop.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer");
    +      prop.put("enable.auto.commit", "false");
    +      metadataRefreshClients[index++] = new KafkaConsumer(prop);
    +    }
    +  }
    +
    +  /**
    +   * The key object used in the assignment map for each operator
    +   */
    +  public static class PartitionMeta
    +  {
    +
    +    public PartitionMeta()
    --- End diff --
    
    I don't find any reference to the constructor.


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r47984961
  
    --- Diff: kafka/src/main/java/com/datatorrent/contrib/kafka090/AbstractKafkaInputOperator.java ---
    @@ -0,0 +1,410 @@
    +/**
    + * 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 com.datatorrent.contrib.kafka090;
    +
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.tuple.Pair;
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * The abstract kafka input operator using kafka 0.9.0 new consumer API
    + * A scalable, fault-tolerant, at-least-once kafka input operator
    + * Keu features includes:
    + * 1. Out-of-box One-to-one and one-to-many partition schema support plus customizable partition schem
    + *    refer to AbstractKafkaPartitioner
    + * 2. Fault-tolerant when the input operator goes down, it redeploys on other node
    + * 3. At-least-once semantics for operator failure (no matter which operator fails)
    + * 4. At-least-once semantics for cold restart (no data loss even if you restart the application)
    + * 5. Multi-cluster support, one operator can consume data from more than one kafka clusters
    + * 6. Multi-topic support, one operator can subscribe multiple topics
    + * 7. Throughput control support, you can throttle number of tuple for each streaming window
    + */
    +public abstract class AbstractKafkaInputOperator<K, V> implements InputOperator, Operator.ActivationListener<Context.OperatorContext>, Operator.CheckpointListener, Partitioner<AbstractKafkaInputOperator>, StatsListener
    +{
    +
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaInputOperator.class);
    +
    +  public enum InitialOffset {
    +    EARLIEST, // consume from beginning of the partition every time when application restart
    +    LATEST, // consume from latest of the partition every time when application restart
    +    APPLICATION_OR_EARLIEST, // consume from committed position from last run or earliest if there is no committed offset(s)
    +    APPLICATION_OR_LATEST // consume from committed position from last run or latest if there is no committed offset(s)
    +  }
    +  /**
    +   *  node separate by ',' and clusters separate by ';'
    +   *  ex: c1n1,c1n2;c2n1,c2n2
    +   */
    +  @NotNull
    +  private String[] clusters;
    +
    +  /**
    +   * The topics the operator consumes
    +   */
    +  @NotNull
    +  private String[] topics;
    +
    +  /**
    +   * Wrapper consumer object
    +   * It wraps KafkaConsumer, maintains consumer thread and store messages in a queue
    +   */
    +  private KafkaConsumerWrapper<K, V> consumerWrapper = new KafkaConsumerWrapper<>();
    +
    +  /**
    +   * Assignment for each operator instance
    +   */
    +  private Set<AbstractKafkaPartitioner.PartitionMeta> assignment;
    +
    +  /**
    +   *  offset track for checkpoint
    +   */
    +  private Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetTrack = new HashMap<>();
    +
    +  /**
    +   * store offsets with window id, only keep offsets with windows that have not been committed
    +   */
    +  private transient List<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> offsetHistory = new LinkedList<>();
    +
    +  /**
    +   * initial partition count
    +   * only used with PartitionStrategy.ONE_TO_MANY
    +   */
    +  private int initialPartitionCount = 1;
    --- End diff --
    
    Why is this not a partitioner property?


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r47961164
  
    --- Diff: kafka/src/main/java/com/datatorrent/contrib/kafka090/AbstractKafkaInputOperator.java ---
    @@ -0,0 +1,410 @@
    +/**
    + * 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 com.datatorrent.contrib.kafka090;
    +
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.tuple.Pair;
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * The abstract kafka input operator using kafka 0.9.0 new consumer API
    + * A scalable, fault-tolerant, at-least-once kafka input operator
    + * Keu features includes:
    + * 1. Out-of-box One-to-one and one-to-many partition schema support plus customizable partition schem
    --- End diff --
    
    Typo


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r47985081
  
    --- Diff: kafka/src/main/java/com/datatorrent/contrib/kafka090/AbstractKafkaInputOperator.java ---
    @@ -0,0 +1,410 @@
    +/**
    + * 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 com.datatorrent.contrib.kafka090;
    +
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.tuple.Pair;
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * The abstract kafka input operator using kafka 0.9.0 new consumer API
    + * A scalable, fault-tolerant, at-least-once kafka input operator
    + * Keu features includes:
    + * 1. Out-of-box One-to-one and one-to-many partition schema support plus customizable partition schem
    + *    refer to AbstractKafkaPartitioner
    + * 2. Fault-tolerant when the input operator goes down, it redeploys on other node
    + * 3. At-least-once semantics for operator failure (no matter which operator fails)
    + * 4. At-least-once semantics for cold restart (no data loss even if you restart the application)
    + * 5. Multi-cluster support, one operator can consume data from more than one kafka clusters
    + * 6. Multi-topic support, one operator can subscribe multiple topics
    + * 7. Throughput control support, you can throttle number of tuple for each streaming window
    + */
    +public abstract class AbstractKafkaInputOperator<K, V> implements InputOperator, Operator.ActivationListener<Context.OperatorContext>, Operator.CheckpointListener, Partitioner<AbstractKafkaInputOperator>, StatsListener
    +{
    +
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaInputOperator.class);
    +
    +  public enum InitialOffset {
    +    EARLIEST, // consume from beginning of the partition every time when application restart
    +    LATEST, // consume from latest of the partition every time when application restart
    +    APPLICATION_OR_EARLIEST, // consume from committed position from last run or earliest if there is no committed offset(s)
    +    APPLICATION_OR_LATEST // consume from committed position from last run or latest if there is no committed offset(s)
    +  }
    +  /**
    +   *  node separate by ',' and clusters separate by ';'
    +   *  ex: c1n1,c1n2;c2n1,c2n2
    +   */
    +  @NotNull
    +  private String[] clusters;
    +
    +  /**
    +   * The topics the operator consumes
    +   */
    +  @NotNull
    +  private String[] topics;
    +
    +  /**
    +   * Wrapper consumer object
    +   * It wraps KafkaConsumer, maintains consumer thread and store messages in a queue
    +   */
    +  private KafkaConsumerWrapper<K, V> consumerWrapper = new KafkaConsumerWrapper<>();
    +
    +  /**
    +   * Assignment for each operator instance
    +   */
    +  private Set<AbstractKafkaPartitioner.PartitionMeta> assignment;
    +
    +  /**
    +   *  offset track for checkpoint
    +   */
    +  private Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetTrack = new HashMap<>();
    +
    +  /**
    +   * store offsets with window id, only keep offsets with windows that have not been committed
    +   */
    +  private transient List<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> offsetHistory = new LinkedList<>();
    +
    +  /**
    +   * initial partition count
    +   * only used with PartitionStrategy.ONE_TO_MANY
    +   */
    +  private int initialPartitionCount = 1;
    +
    +  // Minimal interval between 2 (re)partition actions
    +  private long repartitionInterval = 30000L;
    --- End diff --
    
    Should also go to the partitioner?


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48452288
  
    --- Diff: kafka/src/test/java/org/apache/apex/malhar/kafka/KafkaInputOperatorTest.java ---
    @@ -0,0 +1,221 @@
    +/**
    + * 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.util.Arrays;
    +import java.util.Collection;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Properties;
    +import java.util.concurrent.CountDownLatch;
    +import java.util.concurrent.TimeUnit;
    +
    +import org.junit.Assert;
    +import org.junit.Test;
    +import org.junit.runner.RunWith;
    +import org.junit.runners.Parameterized;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.kafka.clients.consumer.ConsumerConfig;
    +import org.apache.kafka.common.serialization.Deserializer;
    +import org.apache.kafka.common.serialization.StringDeserializer;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.DAG;
    +import com.datatorrent.api.DAG.Locality;
    +import com.datatorrent.api.DefaultInputPort;
    +import com.datatorrent.api.LocalMode;
    +import com.datatorrent.common.util.BaseOperator;
    +
    +/**
    + * A bunch of test to verify the input operator will be automatically partitioned per kafka partition This test is launching its
    + * own Kafka cluster.
    + */
    +@RunWith(Parameterized.class)
    +public class KafkaInputOperatorTest extends KafkaOperatorTestBase
    +{
    +
    +  private int totalBrokers = 0;
    +
    +
    +
    +  @Parameterized.Parameters(name = "multi-cluster: {0}, multi-partition: {1}")
    +  public static Collection<Boolean[]> testScenario()
    +  {
    +    return Arrays.asList(new Boolean[][]{{true, false}, // multi cluster with single partition
    +      {true, true}, // multi cluster with multi partitions
    +      {false, true}, // single cluster with multi partitions
    +      {false, false}, // single cluster with single partitions
    +    });
    +  }
    +
    +  public KafkaInputOperatorTest(boolean hasMultiCluster, boolean hasMultiPartition)
    +  {
    +    // This class want to initialize several kafka brokers for multiple partitions
    +    this.hasMultiCluster = hasMultiCluster;
    +    this.hasMultiPartition = hasMultiPartition;
    +    int cluster = 1 + (hasMultiCluster ? 1 : 0);
    +    totalBrokers = (1 + (hasMultiPartition ? 1 : 0)) * cluster;
    +
    +  }
    +
    +  static final org.slf4j.Logger logger = LoggerFactory.getLogger(KafkaInputOperatorTest.class);
    +  static List<String> tupleCollection = new LinkedList<>();
    +  static CountDownLatch latch;
    +  static boolean hasFailure = false;
    +  static int failureTrigger = 3000;
    +  static int k = 0;
    +
    +  /**
    +   * Test Operator to collect tuples from KafkaSingleInputStringOperator.
    +   *
    +   * @param
    +   */
    +  public static class CollectorModule extends BaseOperator
    +  {
    +    public final transient CollectorInputPort inputPort = new CollectorInputPort();
    +  }
    +
    +  public static class CollectorInputPort extends DefaultInputPort<byte[]>
    +  {
    +
    +    @Override
    +    public void process(byte[] bt)
    +    {
    +      String tuple = new String(bt);
    +      if (hasFailure && k++ == failureTrigger) {
    +        //you can only kill yourself once
    +        hasFailure = false;
    +        throw new RuntimeException();
    +      }
    +      if (tuple.equals(KafkaOperatorTestBase.END_TUPLE)) {
    +        if (latch != null) {
    +          latch.countDown();
    +        }
    +        return;
    +      }
    +      tupleCollection.add(tuple);
    +    }
    +
    +    @Override
    +    public void setConnected(boolean flag)
    +    {
    +      if (flag) {
    +        tupleCollection.clear();
    +      }
    +    }
    +  }
    +
    +  /**
    +   * Test AbstractKafkaSinglePortInputOperator (i.e. an input adapter for Kafka, aka consumer). This module receives
    +   * data from an outside test generator through Kafka message bus and feed that data into Malhar streaming platform.
    +   *
    +   * [Generate message and send that to Kafka message bus] ==> [Receive that message through Kafka input adapter(i.e.
    +   * consumer) and send using emitTuples() interface on output port]
    +   *
    +   *
    +   * @throws Exception
    +   */
    +  @Test
    +  public void testPartitionableInputOperator() throws Exception
    +  {
    +    hasFailure = false;
    +    testInputOperator(false);
    +  }
    +
    +
    +  @Test
    +  public void testPartitionableInputOperatorWithFailure() throws Exception
    +  {
    +    hasFailure = true;
    +    testInputOperator(true);
    +  }
    +
    +  public void testInputOperator(boolean hasFailure) throws Exception
    +  {
    +
    +    // each broker should get a END_TUPLE message
    +    latch = new CountDownLatch(totalBrokers);
    +
    +    int totalCount = 10000;
    +
    +    // Start producer
    +    KafkaTestProducer p = new KafkaTestProducer(TEST_TOPIC, hasMultiPartition, hasMultiCluster);
    +    p.setSendCount(totalCount);
    +    new Thread(p).start();
    +
    +    // Create DAG for testing.
    +    LocalMode lma = LocalMode.newInstance();
    +    DAG dag = lma.getDAG();
    +
    +    // Create KafkaSinglePortStringInputOperator
    +    KafkaSinglePortInputOperator node = dag.addOperator("Kafka input", KafkaSinglePortInputOperator.class);
    +    node.setInitialPartitionCount(1);
    +    // set topic
    +    node.setTopics(TEST_TOPIC);
    +    node.setInitialOffset(AbstractKafkaInputOperator.InitialOffset.EARLIEST.name());
    +    node.setClusters(getClusterConfig());
    +
    +    // Create Test tuple collector
    +    CollectorModule collector = dag.addOperator("TestMessageCollector", new CollectorModule());
    +
    +    // Connect ports
    +    dag.addStream("Kafka message", node.outputPort, collector.inputPort).setLocality(Locality.CONTAINER_LOCAL);
    +
    +    // Create local cluster
    +    final LocalMode.Controller lc = lma.getController();
    +    lc.setHeartbeatMonitoringEnabled(false);
    +
    +    if (hasFailure) {
    +      setupHasFailureTest(node, dag);
    +    }
    +    lc.runAsync();
    +
    +    // Wait 30s for consumer finish consuming all the messages
    +    boolean notTimeout = latch.await(40000, TimeUnit.MILLISECONDS);
    +    Assert.assertTrue("TIMEOUT: 40s Collected " + tupleCollection, notTimeout);
    +
    +    // Check results
    +    Assert.assertEquals("Tuple count", totalCount, tupleCollection.size());
    +    logger.debug(String.format("Number of emitted tuples: %d", tupleCollection.size()));
    +
    +    p.close();
    +    lc.shutdown();
    +    // kafka has a bug shutdown connector you have to make sure kafka client resource has been cleaned before clean the broker
    +    Thread.sleep(5000);
    --- End diff --
    
    Check for that condition instead?


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r47960957
  
    --- Diff: kafka/pom.xml ---
    @@ -0,0 +1,278 @@
    +<!--
    +
    +    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.
    +
    +-->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
    +         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +
    +  <parent>
    +    <groupId>org.apache.apex</groupId>
    +    <artifactId>malhar</artifactId>
    +    <version>3.3.0-incubating-SNAPSHOT</version>
    +  </parent>
    +
    +  <artifactId>malhar-kafka</artifactId>
    +  <name>Apache Apex Malhar (incubating) Contrib Library</name>
    +  <packaging>jar</packaging>
    +
    +  <properties>
    +    <!-- skip tests by default as they depend on manual setup -->
    +    <skip.kafka.module.tests>false</skip.kafka.module.tests>
    +    <skipTests>true</skipTests>
    +  </properties>
    +  <repositories>
    +    <repository>
    +      <id>datatorrent-3rd-party</id>
    +      <name>Embedded repository for dependencies not available online</name>
    +      <url>https://www.datatorrent.com/maven/content/repositories/thirdparty</url>
    +      <snapshots>
    +        <updatePolicy>daily</updatePolicy>
    +      </snapshots>
    +      <releases>
    +        <updatePolicy>daily</updatePolicy>
    +      </releases>
    +    </repository>
    +    <repository>
    +      <id>apache</id>
    +      <name>apache maven path for non-public repo</name>
    +      <url>https://repository.apache.org/content/groups/staging/</url>
    +    </repository>
    +  </repositories>
    +
    +  <build>
    +    <plugins>
    +      <!-- Publish tests jar -->
    +      <plugin>
    +        <groupId>org.apache.maven.plugins</groupId>
    +        <artifactId>maven-jar-plugin</artifactId>
    +        <version>2.4</version>
    +        <executions>
    +          <execution>
    +            <goals>
    +              <goal>test-jar</goal>
    +            </goals>
    +            <phase>package</phase>
    +          </execution>
    +        </executions>
    +      </plugin>
    +     <!-- create resource directory for xml javadoc-->
    +     <plugin>
    +         <groupId>org.apache.maven.plugins</groupId>
    +         <artifactId>maven-antrun-plugin</artifactId>
    +         <executions>
    +           <execution>
    +               <id>createJavadocDirectory</id>
    +               <phase>generate-resources</phase>
    +               <configuration>
    +                   <tasks>
    +                     <delete dir="${project.build.directory}/generated-resources/xml-javadoc"/>
    +                     <mkdir dir="${project.build.directory}/generated-resources/xml-javadoc"/>
    +                   </tasks>
    +               </configuration>
    +               <goals>
    +                   <goal>run</goal>
    +               </goals>
    +           </execution>
    +         </executions>
    +      </plugin>
    +     <!-- generate javdoc -->
    +     <plugin>
    +       <groupId>org.apache.maven.plugins</groupId>
    +       <artifactId>maven-javadoc-plugin</artifactId>
    +       <executions>
    +         <!-- generate xml javadoc -->
    +         <execution>
    +           <id>xml-doclet</id>
    +           <phase>generate-resources</phase>
    +           <goals>
    +             <goal>javadoc</goal>
    +           </goals>
    +           <configuration>
    +             <doclet>com.github.markusbernhardt.xmldoclet.XmlDoclet</doclet>
    +             <additionalparam>-d ${project.build.directory}/generated-resources/xml-javadoc -filename ${project.artifactId}-${project.version}-javadoc.xml</additionalparam>
    +             <useStandardDocletOptions>false</useStandardDocletOptions>
    +             <docletArtifact>
    +               <groupId>com.github.markusbernhardt</groupId>
    +               <artifactId>xml-doclet</artifactId>
    +               <version>1.0.4</version>
    +             </docletArtifact>
    +           </configuration>
    +         </execution>
    +         <!-- generate default javadoc jar with custom tags -->
    +         <execution>
    +           <id>attach-sources</id>
    +           <goals>
    +             <goal>jar</goal>
    +           </goals>
    +           <configuration>
    +             <skip>true</skip>
    +             <tags>
    +               <tag>
    +                 <name>customTag1</name>
    +                 <placement>a</placement>
    +                 <head>Custom Tag One:</head>
    +               </tag>
    +               <tag>
    +                 <name>customTag2</name>
    +                 <placement>a</placement>
    +                 <head>Custom Tag two:</head>
    +               </tag>
    +               <tag>
    +                 <name>customTag3</name>
    +                 <placement>a</placement>
    +                 <head>Custom Tag three:</head>
    +               </tag>
    +             </tags>
    +           </configuration>
    +         </execution>
    +       </executions>
    +     </plugin>
    +     <!-- Transform xml javadoc to stripped down version containing only class/interface comments and tags-->
    +     <plugin>
    +       <groupId>org.codehaus.mojo</groupId>
    +       <artifactId>xml-maven-plugin</artifactId>
    +       <version>1.0</version>
    +       <executions>
    +         <execution>
    +           <id>transform-xmljavadoc</id>
    +           <phase>generate-resources</phase>
    +           <goals>
    +             <goal>transform</goal>
    +           </goals>
    +         </execution>
    +       </executions>
    +       <configuration>
    +         <transformationSets>
    +           <transformationSet>
    +             <dir>${project.build.directory}/generated-resources/xml-javadoc</dir>
    +             <includes>
    +               <include>${project.artifactId}-${project.version}-javadoc.xml</include>
    +             </includes>
    +             <stylesheet>XmlJavadocCommentsExtractor.xsl</stylesheet>
    +             <outputDir>${project.build.directory}/generated-resources/xml-javadoc</outputDir>
    +           </transformationSet>
    +         </transformationSets>
    +       </configuration>
    +     </plugin>
    +     <!-- copy xml javadoc to class jar -->
    +     <plugin>
    +       <artifactId>maven-resources-plugin</artifactId>
    +       <version>2.6</version>
    +       <executions>
    +         <execution>
    +           <id>copy-resources</id>
    +           <phase>process-resources</phase>
    +           <goals>
    +             <goal>copy-resources</goal>
    +           </goals>
    +           <configuration>
    +             <outputDirectory>${basedir}/target/classes</outputDirectory>
    +             <resources>
    +               <resource>
    +                 <directory>${project.build.directory}/generated-resources/xml-javadoc</directory>
    +                 <includes>
    +                   <include>${project.artifactId}-${project.version}-javadoc.xml</include>
    +                 </includes>
    +                 <filtering>true</filtering>
    +               </resource>
    +             </resources>
    +           </configuration>
    +         </execution>
    +       </executions>
    +     </plugin>
    +     <plugin>
    +       <artifactId>maven-enforcer-plugin</artifactId>
    +       <configuration>
    +         <rules>
    +           <requireReleaseDeps>
    +             <message>Snapshots dependencies are not allowed for release build.</message>
    +             <onlyWhenRelease>true</onlyWhenRelease>
    +             <excludes>
    +               <exclude>org.couchbase.mock:CouchbaseMock</exclude>
    +             </excludes>
    +           </requireReleaseDeps>
    +         </rules>
    +       </configuration>
    +     </plugin>
    +
    +    </plugins>
    +  </build>
    +
    +  <dependencies>
    +    <dependency>
    +      <groupId>org.apache.kafka</groupId>
    +      <artifactId>kafka_2.11</artifactId>
    +      <version>0.9.0.0</version>
    +      <optional>true</optional>
    +      <exclusions>
    +        <exclusion>
    +          <groupId>org.slf4j</groupId>
    +          <artifactId>slf4j-log4j12</artifactId>
    +        </exclusion>
    +        <exclusion>
    +          <groupId>org.slf4j</groupId>
    +          <artifactId>slf4j-simple</artifactId>
    +        </exclusion>
    +        <exclusion>
    +          <groupId>log4j</groupId>
    +          <artifactId>log4j</artifactId>
    +        </exclusion>
    +        <exclusion>
    +          <groupId>org.apache.zookeeper</groupId>
    +          <artifactId>zookeeper</artifactId>
    +        </exclusion>
    +      </exclusions>
    +    </dependency>
    +    <dependency>
    +      <groupId>org.apache.kafka</groupId>
    +      <artifactId>kafka-clients</artifactId>
    +      <version>0.9.0.0</version>
    +    </dependency>
    +    <dependency>
    +      <groupId>org.apache.commons</groupId>
    +      <artifactId>commons-lang3</artifactId>
    +      <version>3.1</version>
    +    </dependency>
    +    <dependency>
    --- End diff --
    
    Isn't this an engine dependency?


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48646958
  
    --- Diff: kafka/src/main/java/org/apache/apex/malhar/kafka/KafkaConsumerWrapper.java ---
    @@ -0,0 +1,378 @@
    +/**
    + * 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.Set;
    +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.common.Metric;
    +import org.apache.kafka.common.MetricName;
    +import org.apache.kafka.common.TopicPartition;
    +import org.apache.kafka.common.errors.WakeupException;
    +import org.apache.kafka.common.serialization.ByteArrayDeserializer;
    +
    +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 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  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 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<byte[], byte[]>> consumers = new HashMap<>();
    +
    +  // The in memory buffer hold consumed messages
    +  private transient ArrayBlockingQueue<Pair<String, ConsumerRecord<byte[], byte[]>>> 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 ownerOperator = null;
    +
    +  private transient ExecutorService kafkaConsumerExecutor;
    +
    +  private transient Map<String, Map<TopicPartition, OffsetAndMetadata>> offsetsToCommit = new HashMap<>();
    +
    +  /**
    +   *
    +   * 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.ownerOperator);
    +            offsetToCommit.clear();
    +          }
    +          try {
    +            ConsumerRecords records = consumer.poll(wrapper.timeout);
    +            for (ConsumerRecord<byte[], byte[]> record : (Iterable<ConsumerRecord<byte[], byte[]>>)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<>();
    +    Set<AbstractKafkaPartitioner.PartitionMeta> assignments = ownerOperator.getAssignment();
    +    for (AbstractKafkaPartitioner.PartitionMeta partitionMeta : assignments) {
    +      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");
    +      prop.put(ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
    +      prop.put(ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG, ByteArrayDeserializer.class.getName());
    +      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<byte[], byte[]> 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<byte[], byte[]>> 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()
    --- End diff --
    
    We should correct the property name.


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r47984778
  
    --- Diff: kafka/pom.xml ---
    @@ -0,0 +1,278 @@
    +<!--
    +
    +    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.
    +
    +-->
    +<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
    +         xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
    +  <modelVersion>4.0.0</modelVersion>
    +
    +  <parent>
    +    <groupId>org.apache.apex</groupId>
    +    <artifactId>malhar</artifactId>
    +    <version>3.3.0-incubating-SNAPSHOT</version>
    +  </parent>
    +
    +  <artifactId>malhar-kafka</artifactId>
    +  <name>Apache Apex Malhar (incubating) Contrib Library</name>
    --- End diff --
    
    Contrib Library -> Kafka Support


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48452301
  
    --- Diff: kafka/src/main/java/org/apache/apex/malhar/kafka/AbstractKafkaInputOperator.java ---
    @@ -0,0 +1,455 @@
    +/**
    + * 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.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.tuple.Pair;
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +import org.apache.kafka.common.serialization.Deserializer;
    +
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * The abstract kafka input operator using kafka 0.9.0 new consumer API
    + * A scalable, fault-tolerant, at-least-once kafka input operator
    + * Key features includes:
    + * 1. Out-of-box One-to-one and one-to-many partition strategy support plus customizable partition strategy
    + *    refer to AbstractKafkaPartitioner
    + * 2. Fault-tolerant when the input operator goes down, it redeploys on other node
    + * 3. At-least-once semantics for operator failure (no matter which operator fails)
    + * 4. At-least-once semantics for cold restart (no data loss even if you restart the application)
    + * 5. Multi-cluster support, one operator can consume data from more than one kafka clusters
    + * 6. Multi-topic support, one operator can subscribe multiple topics
    + * 7. Throughput control support, you can throttle number of tuple for each streaming window
    + */
    +public abstract class AbstractKafkaInputOperator<K, V> implements InputOperator, Operator.ActivationListener<Context.OperatorContext>, Operator.CheckpointListener, Partitioner<AbstractKafkaInputOperator>, StatsListener
    +{
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaInputOperator.class);
    +
    +  public long getMetricsRefreshInterval()
    +  {
    +    return metricsRefreshInterval;
    +  }
    +
    +  public void setMetricsRefreshInterval(long metricsRefreshInterval)
    +  {
    +    this.metricsRefreshInterval = metricsRefreshInterval;
    +  }
    +
    +  public enum InitialOffset
    +  {
    +    EARLIEST, // consume from beginning of the partition every time when application restart
    +    LATEST, // consume from latest of the partition every time when application restart
    +    APPLICATION_OR_EARLIEST, // consume from committed position from last run or earliest if there is no committed offset(s)
    +    APPLICATION_OR_LATEST // consume from committed position from last run or latest if there is no committed offset(s)
    +  }
    +  /**
    +   *  Same setting as bootstrap.servers property to KafkaConsumer
    +   *  refer to http://kafka.apache.org/documentation.html#newconsumerconfigs
    +   *  To support multi cluster, you can have multiple bootstrap.servers separated by ";"
    +   */
    +  @NotNull
    +  private String[] clusters;
    +
    +  /**
    +   * The topics the operator consumes
    +   */
    +  @NotNull
    +  private String[] topics;
    +
    +  /**
    +   * Wrapper consumer object
    +   * It wraps KafkaConsumer, maintains consumer thread and store messages in a queue
    +   */
    +  private KafkaConsumerWrapper<K, V> consumerWrapper = new KafkaConsumerWrapper<>();
    +
    +  /**
    +   * Assignment for each operator instance
    +   */
    +  private Set<AbstractKafkaPartitioner.PartitionMeta> assignment;
    +
    +  /**
    +   *  offset track for checkpoint
    +   */
    +  private Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetTrack = new HashMap<>();
    +
    +  /**
    +   * store offsets with window id, only keep offsets with windows that have not been committed
    +   */
    +  private transient List<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> offsetHistory = new LinkedList<>();
    +
    +  /**
    +   * initial partition count
    +   * only used with PartitionStrategy.ONE_TO_MANY
    +   */
    +  private int initialPartitionCount = 1;
    +
    +  // Minimal interval between 2 (re)partition actions
    +  private long repartitionInterval = 30000L;
    +
    +  // Minimal interval between checking collected stats and decide whether it needs to repartition or not.
    +  // And minimal interval between 2 offset updates
    +  private long repartitionCheckInterval = 5000L;
    +
    +  private Class<? extends Deserializer<K>> keyDeserializer;
    +
    +  private Class<? extends Deserializer<V>> valueDeserializer;
    +
    +  /**
    +   * maximum tuples allowed to be emitted in each window
    +   */
    +  @Min(1)
    +  private int maxTuplesPerWindow = Integer.MAX_VALUE;
    +
    +  private InitialOffset initialOffset = InitialOffset.APPLICATION_OR_LATEST;
    +
    +  private long metricsRefreshInterval = 5000L;
    +
    +  /**
    +   * extra kafka consumer properties
    +   * http://kafka.apache.org/090/documentation.html#newconsumerconfigs
    +   *
    +   * Please be aware that the properties below are set by the operator, don't override it
    +   *
    +   * bootstrap.servers
    +   * group.id
    +   * auto.offset.reset
    +   * enable.auto.commit
    +   * partition.assignment.strategy
    +   *
    +   *
    +   */
    +  private Properties consumerProps;
    +
    +  /**
    +   * count the emitted message in each window
    +   * non settable
    +   */
    +  private transient int emitCount = 0;
    +
    +  /**
    +   * Application name is used as group.id for kafka consumer
    +   */
    +  private transient String applicationName;
    +
    +  private transient AbstractKafkaPartitioner partitioner;
    +
    +  protected transient long currentWindowId;
    +
    +  // By default the partition policy is 1:1
    +  private PartitionStrategy strategy = PartitionStrategy.ONE_TO_ONE;
    +
    +  private transient long lastCheckTime = 0L;
    +
    +  private transient long lastRepartitionTime = 0L;
    +
    +  @AutoMetric
    +  private transient KafkaMetrics metrics;
    +
    +  @Override
    +  public void activate(Context.OperatorContext context)
    +  {
    +    consumerWrapper.start();
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +    consumerWrapper.stop();
    +  }
    +
    +  @Override
    +  public void checkpointed(long l)
    +  {
    +
    +  }
    +
    +  @Override
    +  public void committed(long windowId)
    +  {
    +    //ask kafka consumer wrapper to store the committed offsets
    +    for (Iterator<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> iter = offsetHistory.iterator(); iter.hasNext(); ) {
    +      Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>> item = iter.next();
    +      if (item.getLeft() <= windowId) {
    +        if (item.getLeft() == windowId) {
    +          consumerWrapper.commitOffsets(item.getRight());
    +        }
    +        iter.remove();
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void emitTuples()
    +  {
    +    int count = consumerWrapper.messageSize();
    +    if (maxTuplesPerWindow > 0) {
    +      count = Math.min(count, maxTuplesPerWindow - emitCount);
    +    }
    +    for (int i = 0; i < count; i++) {
    +      Pair<String, ConsumerRecord<K, V>> tuple = consumerWrapper.pollMessage();
    +      ConsumerRecord<K, V> msg = tuple.getRight();
    +      emitTuple(tuple.getLeft(), msg);
    +      AbstractKafkaPartitioner.PartitionMeta pm = new AbstractKafkaPartitioner.PartitionMeta(tuple.getLeft(),
    +          msg.topic(), msg.partition());
    +      offsetTrack.put(pm, msg.offset());
    +    }
    +    emitCount += count;
    +  }
    +
    +  protected abstract void emitTuple(String cluster, ConsumerRecord<K, V> msg);
    +
    +  @Override
    +  public void beginWindow(long wid)
    +  {
    +    emitCount = 0;
    +    currentWindowId = wid;
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    // copy current offset track to history memory
    +    Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetsWithWindow = new HashMap<>(offsetTrack);
    +    offsetHistory.add(Pair.of(currentWindowId, offsetsWithWindow));
    +
    +    //update metrics
    +    metrics.updateMetrics(clusters, consumerWrapper.getAllConsumerMetrics());
    +  }
    +
    +
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    applicationName = context.getValue(Context.DAGContext.APPLICATION_NAME);
    +    consumerWrapper.create(this);
    +    metrics = new KafkaMetrics(metricsRefreshInterval);
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +
    +  }
    +
    +  private void initPartitioner()
    +  {
    +    if (partitioner == null) {
    +      logger.info("Initialize Partitioner");
    +      switch (strategy) {
    +        case ONE_TO_ONE:
    +          partitioner = new OneToOnePartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY:
    +          partitioner = new OneToManyPartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY_HEURISTIC:
    +          throw new UnsupportedOperationException("Not implemented yet");
    +        default:
    +          throw new RuntimeException("Invalid strategy");
    +      }
    +      logger.info("Actual Partitioner is {}", partitioner.getClass());
    +    }
    +
    +  }
    +
    +  @Override
    +  public Response processStats(BatchedOperatorStats batchedOperatorStats)
    +  {
    +    long t = System.currentTimeMillis();
    +    if (repartitionInterval < 0 || repartitionCheckInterval < 0 ||
    +        t - lastCheckTime < repartitionCheckInterval || t - lastRepartitionTime < repartitionInterval) {
    +      // return false if it's within repartitionCheckInterval since last time it check the stats
    +      Response response = new Response();
    +      response.repartitionRequired = false;
    +      return response;
    +    }
    +
    +    try {
    +      logger.debug("Process stats");
    +      initPartitioner();
    +      return partitioner.processStats(batchedOperatorStats);
    +    } finally {
    +      lastCheckTime = System.currentTimeMillis();
    +    }
    +  }
    +
    +  @Override
    +  public Collection<Partition<AbstractKafkaInputOperator>> definePartitions(
    +      Collection<Partition<AbstractKafkaInputOperator>> collection, PartitioningContext partitioningContext)
    +  {
    +    logger.debug("Define partitions");
    +    initPartitioner();
    +    return partitioner.definePartitions(collection, partitioningContext);
    +  }
    +
    +  @Override
    +  public void partitioned(Map<Integer, Partition<AbstractKafkaInputOperator>> map)
    +  {
    +    // update the last repartition time
    +    lastRepartitionTime = System.currentTimeMillis();
    +    initPartitioner();
    +    partitioner.partitioned(map);
    +  }
    +
    +
    +  //---------------------------------------------setters and getters----------------------------------------
    +  public void setInitialPartitionCount(int partitionCount)
    +  {
    +    this.initialPartitionCount = partitionCount;
    +  }
    +
    +  public int getInitialPartitionCount()
    +  {
    +    return initialPartitionCount;
    +  }
    +
    +  public void setClusters(String clusters)
    +  {
    +    this.clusters = clusters.split(";");
    +  }
    +
    +  public String getClusters()
    +  {
    +    return Joiner.on(';').join(clusters);
    +  }
    +
    +  public void setTopics(String... topics)
    +  {
    +    this.topics = topics;
    +  }
    +
    +  public String[] getTopics()
    +  {
    +    return topics;
    +  }
    +
    +  public void setConsumerWrapper(KafkaConsumerWrapper consumerWrapper)
    +  {
    +    this.consumerWrapper = consumerWrapper;
    +  }
    +
    +  public KafkaConsumerWrapper getConsumerWrapper()
    +  {
    +    return consumerWrapper;
    +  }
    +
    +  public void setAssignment(Set<AbstractKafkaPartitioner.PartitionMeta> assignment)
    +  {
    +    this.assignment = assignment;
    +  }
    +
    +  public Set<AbstractKafkaPartitioner.PartitionMeta> getAssignment()
    +  {
    +    return assignment;
    +  }
    +
    +  public Map<AbstractKafkaPartitioner.PartitionMeta, Long> getOffsetTrack()
    +  {
    +    return offsetTrack;
    +  }
    +
    +  public void setStrategy(String policy)
    +  {
    +    this.strategy = PartitionStrategy.valueOf(policy.toUpperCase());
    +  }
    +
    +  public PartitionStrategy getStrategy()
    +  {
    +    return strategy;
    +  }
    +
    +  public void setInitialOffset(String initialOffset)
    +  {
    +    this.initialOffset = InitialOffset.valueOf(initialOffset.toUpperCase());
    +  }
    +
    +  public InitialOffset getInitialOffset()
    +  {
    +    return initialOffset;
    +  }
    +
    +  public String getApplicationName()
    +  {
    +    return applicationName;
    +  }
    +
    +  public void setConsumerProps(Properties consumerProps)
    +  {
    +    this.consumerProps = consumerProps;
    +  }
    +
    +  public Properties getConsumerProps()
    +  {
    +    return consumerProps;
    +  }
    +
    +  public void setMaxTuplesPerWindow(int maxTuplesPerWindow)
    +  {
    +    this.maxTuplesPerWindow = maxTuplesPerWindow;
    +  }
    +
    +  public int getMaxTuplesPerWindow()
    +  {
    +    return maxTuplesPerWindow;
    +  }
    +
    +  public void setKeyDeserializer(Class<? extends Deserializer<K>> keyDeserializer)
    --- End diff --
    
    Shouldn't the serializers be protected instead? The derived class has to provide them, but not the user?


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r47979259
  
    --- Diff: kafka/src/main/java/com/datatorrent/contrib/kafka090/AbstractKafkaPartitioner.java ---
    @@ -0,0 +1,277 @@
    +/**
    + * 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 com.datatorrent.contrib.kafka090;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.kafka.clients.consumer.KafkaConsumer;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.TopicPartition;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * Abstract partitioner used to manage the partitions of kafka input operator.
    + * It use a number of kafka consumers(one for each cluster) to get the latest partition metadata for topics that
    + * the consumer subscribes and expose those to subclass which implements the assign method
    + *
    + * The partitioner is always stateless.
    + */
    +public abstract class AbstractKafkaPartitioner implements Partitioner<AbstractKafkaInputOperator>, StatsListener
    +{
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaPartitioner.class);
    +
    +  private static final String META_CONSUMER_GROUP_NAME = AbstractKafkaInputOperator.class.getName() + "META_GROUP";
    +
    +  protected String[] clusters;
    +
    +  protected String[] topics;
    +
    +  protected AbstractKafkaInputOperator prototypeOperator;
    +
    +  private KafkaConsumer[] metadataRefreshClients;
    +
    +
    +  private List<Set<AbstractKafkaPartitioner.PartitionMeta>> currentPartitions = new LinkedList<>(); // prevent null
    +
    +  public AbstractKafkaPartitioner(String[] clusters, String[] topics, AbstractKafkaInputOperator prototypeOperator)
    +  {
    +    this.clusters = clusters;
    +    this.topics = topics;
    +    this.prototypeOperator = prototypeOperator;
    +  }
    +
    +  abstract List<Set<PartitionMeta>> assign(Map<String, Map<String,List<PartitionInfo>>> metadata);
    +
    +
    +
    +  @Override
    +  public Collection<Partition<AbstractKafkaInputOperator>> definePartitions(Collection<Partition<AbstractKafkaInputOperator>> collection, PartitioningContext partitioningContext)
    +  {
    +
    +    initMetadataClients();
    +
    +    Map<String, Map<String, List<PartitionInfo>>> metadata = new HashMap<>();
    +
    +
    +    for (int i = 0; i < clusters.length; i++) {
    +      metadata.put(clusters[i], new HashMap<String, List<PartitionInfo>>());
    +      for (String topic : topics) {
    +        List<PartitionInfo> ptis = metadataRefreshClients[i].partitionsFor(topic);
    +        if (logger.isDebugEnabled()) {
    +          logger.debug("Partition metadata for topic {} : {}", topic, Joiner.on(';').join(ptis));
    +        }
    +        metadata.get(clusters[i]).put(topic, ptis);
    +      }
    +      metadataRefreshClients[i].close();
    +    }
    +
    +    metadataRefreshClients = null;
    +
    +    List<Set<AbstractKafkaPartitioner.PartitionMeta>> parts = assign(metadata);
    +
    +
    +    if (currentPartitions == parts || currentPartitions.equals(parts)) {
    +      logger.debug("No partition change found");
    +      return collection;
    +    } else {
    +      logger.info("Partition change detected: ");
    +      currentPartitions.clear();
    +      currentPartitions.addAll(parts);
    +      int i = 0;
    +      List<Partition<AbstractKafkaInputOperator>> result = new LinkedList<>();
    +      for (Iterator<Partition<AbstractKafkaInputOperator>> iter = collection.iterator(); iter.hasNext();) {
    +        Partition<AbstractKafkaInputOperator> nextPartition = iter.next();
    +        if (parts.remove(nextPartition.getPartitionedInstance().getAssignment())) {
    +          if (logger.isInfoEnabled()) {
    +            logger.info("[Existing] Partition {} with assignment {} ", i,
    +                Joiner.on(';').join(nextPartition.getPartitionedInstance().getAssignment()));
    +          }
    +          result.add(nextPartition);
    +          i++;
    +        }
    +      }
    +
    +      for (Set<AbstractKafkaPartitioner.PartitionMeta> partitionAssignment : parts) {
    +        if (logger.isInfoEnabled()) {
    +          logger.info("[New] Partition {} with assignment {} ", i,
    +              Joiner.on(';').join(partitionAssignment));
    +        }
    +        result.add(createPartition(partitionAssignment));
    +        i++;
    +      }
    +
    +
    +      return result;
    +    }
    +  }
    +
    +  @Override
    +  public void partitioned(Map<Integer, Partition<AbstractKafkaInputOperator>> map)
    +  {
    +
    +  }
    +
    +  @Override
    +  public Response processStats(BatchedOperatorStats batchedOperatorStats)
    +  {
    +    Response response = new Response();
    +    response.repartitionRequired = true;
    +    return response;
    +  }
    +
    +  protected Partitioner.Partition<AbstractKafkaInputOperator> createPartition(Set<AbstractKafkaPartitioner.PartitionMeta> partitionAssignment)
    +  {
    +    Kryo kryo = new Kryo();
    +    ByteArrayOutputStream bos = new ByteArrayOutputStream();
    +    Output output = new Output(bos);
    +    kryo.writeObject(output, prototypeOperator);
    +    output.close();
    +    Input lInput = new Input(bos.toByteArray());
    +    @SuppressWarnings("unchecked")
    +    Partitioner.Partition<AbstractKafkaInputOperator> p = new DefaultPartition<>(kryo.readObject(lInput, prototypeOperator.getClass()));
    --- End diff --
    
    There are actually raw type warnings also.


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r47982383
  
    --- Diff: kafka/src/main/java/com/datatorrent/contrib/kafka090/AbstractKafkaPartitioner.java ---
    @@ -0,0 +1,277 @@
    +/**
    + * 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 com.datatorrent.contrib.kafka090;
    +
    +import java.io.ByteArrayOutputStream;
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.kafka.clients.consumer.KafkaConsumer;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.TopicPartition;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * Abstract partitioner used to manage the partitions of kafka input operator.
    + * It use a number of kafka consumers(one for each cluster) to get the latest partition metadata for topics that
    + * the consumer subscribes and expose those to subclass which implements the assign method
    + *
    + * The partitioner is always stateless.
    + */
    +public abstract class AbstractKafkaPartitioner implements Partitioner<AbstractKafkaInputOperator>, StatsListener
    +{
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaPartitioner.class);
    +
    +  private static final String META_CONSUMER_GROUP_NAME = AbstractKafkaInputOperator.class.getName() + "META_GROUP";
    +
    +  protected String[] clusters;
    +
    +  protected String[] topics;
    +
    +  protected AbstractKafkaInputOperator prototypeOperator;
    +
    +  private KafkaConsumer[] metadataRefreshClients;
    +
    +
    +  private List<Set<AbstractKafkaPartitioner.PartitionMeta>> currentPartitions = new LinkedList<>(); // prevent null
    +
    +  public AbstractKafkaPartitioner(String[] clusters, String[] topics, AbstractKafkaInputOperator prototypeOperator)
    +  {
    +    this.clusters = clusters;
    +    this.topics = topics;
    +    this.prototypeOperator = prototypeOperator;
    +  }
    +
    +  abstract List<Set<PartitionMeta>> assign(Map<String, Map<String,List<PartitionInfo>>> metadata);
    +
    +
    +
    +  @Override
    +  public Collection<Partition<AbstractKafkaInputOperator>> definePartitions(Collection<Partition<AbstractKafkaInputOperator>> collection, PartitioningContext partitioningContext)
    +  {
    +
    +    initMetadataClients();
    +
    +    Map<String, Map<String, List<PartitionInfo>>> metadata = new HashMap<>();
    +
    +
    +    for (int i = 0; i < clusters.length; i++) {
    +      metadata.put(clusters[i], new HashMap<String, List<PartitionInfo>>());
    +      for (String topic : topics) {
    +        List<PartitionInfo> ptis = metadataRefreshClients[i].partitionsFor(topic);
    +        if (logger.isDebugEnabled()) {
    +          logger.debug("Partition metadata for topic {} : {}", topic, Joiner.on(';').join(ptis));
    +        }
    +        metadata.get(clusters[i]).put(topic, ptis);
    +      }
    +      metadataRefreshClients[i].close();
    +    }
    +
    +    metadataRefreshClients = null;
    +
    +    List<Set<AbstractKafkaPartitioner.PartitionMeta>> parts = assign(metadata);
    +
    +
    +    if (currentPartitions == parts || currentPartitions.equals(parts)) {
    +      logger.debug("No partition change found");
    +      return collection;
    +    } else {
    +      logger.info("Partition change detected: ");
    +      currentPartitions.clear();
    +      currentPartitions.addAll(parts);
    +      int i = 0;
    +      List<Partition<AbstractKafkaInputOperator>> result = new LinkedList<>();
    +      for (Iterator<Partition<AbstractKafkaInputOperator>> iter = collection.iterator(); iter.hasNext();) {
    +        Partition<AbstractKafkaInputOperator> nextPartition = iter.next();
    +        if (parts.remove(nextPartition.getPartitionedInstance().getAssignment())) {
    +          if (logger.isInfoEnabled()) {
    +            logger.info("[Existing] Partition {} with assignment {} ", i,
    +                Joiner.on(';').join(nextPartition.getPartitionedInstance().getAssignment()));
    +          }
    +          result.add(nextPartition);
    +          i++;
    +        }
    +      }
    +
    +      for (Set<AbstractKafkaPartitioner.PartitionMeta> partitionAssignment : parts) {
    +        if (logger.isInfoEnabled()) {
    +          logger.info("[New] Partition {} with assignment {} ", i,
    +              Joiner.on(';').join(partitionAssignment));
    +        }
    +        result.add(createPartition(partitionAssignment));
    +        i++;
    +      }
    +
    +
    +      return result;
    +    }
    +  }
    +
    +  @Override
    +  public void partitioned(Map<Integer, Partition<AbstractKafkaInputOperator>> map)
    +  {
    +
    +  }
    +
    +  @Override
    +  public Response processStats(BatchedOperatorStats batchedOperatorStats)
    +  {
    +    Response response = new Response();
    +    response.repartitionRequired = true;
    +    return response;
    +  }
    +
    +  protected Partitioner.Partition<AbstractKafkaInputOperator> createPartition(Set<AbstractKafkaPartitioner.PartitionMeta> partitionAssignment)
    +  {
    +    Kryo kryo = new Kryo();
    +    ByteArrayOutputStream bos = new ByteArrayOutputStream();
    +    Output output = new Output(bos);
    +    kryo.writeObject(output, prototypeOperator);
    +    output.close();
    +    Input lInput = new Input(bos.toByteArray());
    +    @SuppressWarnings("unchecked")
    +    Partitioner.Partition<AbstractKafkaInputOperator> p = new DefaultPartition<>(kryo.readObject(lInput, prototypeOperator.getClass()));
    +    p.getPartitionedInstance().setAssignment(partitionAssignment);
    +    return p;
    +  }
    +  /**
    +   *
    +   */
    +  private void initMetadataClients()
    +  {
    +    if (metadataRefreshClients != null && metadataRefreshClients.length == clusters.length) {
    +      // The metadata client is active
    +      return;
    +    }
    +
    +    if (clusters == null || clusters.length == 0) {
    +      throw new IllegalStateException("clusters can not be null");
    +    }
    +
    +    metadataRefreshClients = new KafkaConsumer[clusters.length];
    +    int index = 0;
    +    for (String c : clusters) {
    +      Properties prop = new Properties();
    +      prop.put("group.id", META_CONSUMER_GROUP_NAME);
    +      prop.put("bootstrap.servers", c);
    +      prop.put("key.deserializer", "org.apache.kafka.common.serialization.StringDeserializer");
    +      prop.put("value.deserializer", "org.apache.kafka.common.serialization.StringDeserializer");
    +      prop.put("enable.auto.commit", "false");
    +      metadataRefreshClients[index++] = new KafkaConsumer(prop);
    +    }
    +  }
    +
    +  /**
    +   * The key object used in the assignment map for each operator
    +   */
    +  public static class PartitionMeta
    +  {
    +
    +    public PartitionMeta()
    --- End diff --
    
    Why public?


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48918567
  
    --- Diff: kafka/src/main/java/org/apache/apex/malhar/kafka/AbstractKafkaInputOperator.java ---
    @@ -0,0 +1,500 @@
    +/**
    + * 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.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.tuple.Pair;
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +import org.apache.kafka.clients.consumer.OffsetAndMetadata;
    +import org.apache.kafka.clients.consumer.OffsetCommitCallback;
    +import org.apache.kafka.common.TopicPartition;
    +
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.AutoMetric;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * The abstract kafka input operator using kafka 0.9.0 new consumer API
    + * A scalable, fault-tolerant, at-least-once kafka input operator
    + * Key features includes:
    + *
    + * <ol>
    + * <li>Out-of-box One-to-one and one-to-many partition strategy support plus customizable partition strategy
    + *    refer to AbstractKafkaPartitioner </li>
    + * <li>Fault-tolerant when the input operator goes down, it redeploys on other node</li>
    + * <li>At-least-once semantics for operator failure (no matter which operator fails)</li>
    + * <li>At-least-once semantics for cold restart (no data loss even if you restart the application)</li>
    + * <li>Multi-cluster support, one operator can consume data from more than one kafka clusters</li>
    + * <li>Multi-topic support, one operator can subscribe multiple topics</li>
    + * <li>Throughput control support, you can throttle number of tuple for each streaming window</li>
    + * </ol>
    + */
    +public abstract class AbstractKafkaInputOperator implements InputOperator, Operator.ActivationListener<Context.OperatorContext>, Operator.CheckpointListener, Partitioner<AbstractKafkaInputOperator>, StatsListener, OffsetCommitCallback
    +{
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaInputOperator.class);
    +
    +  public enum InitialOffset
    +  {
    +    EARLIEST, // consume from beginning of the partition every time when application restart
    +    LATEST, // consume from latest of the partition every time when application restart
    +    APPLICATION_OR_EARLIEST, // consume from committed position from last run or earliest if there is no committed offset(s)
    +    APPLICATION_OR_LATEST // consume from committed position from last run or latest if there is no committed offset(s)
    +  }
    +  /**
    +   *  Same setting as bootstrap.servers property to KafkaConsumer
    +   *  refer to http://kafka.apache.org/documentation.html#newconsumerconfigs
    +   *  To support multi cluster, you can have multiple bootstrap.servers separated by ";"
    +   */
    +  @NotNull
    +  private String[] clusters;
    +
    +  /**
    +   * The topics the operator consumes
    +   */
    +  @NotNull
    +  private String[] topics;
    +
    +  /**
    +   * Wrapper consumer object
    +   * It wraps KafkaConsumer, maintains consumer thread and store messages in a queue
    +   */
    +  private transient KafkaConsumerWrapper consumerWrapper = new KafkaConsumerWrapper();
    +
    +  /**
    +   * Assignment for each operator instance
    +   */
    +  private Set<AbstractKafkaPartitioner.PartitionMeta> assignment;
    +
    +  /**
    +   *  offset track for checkpoint
    +   */
    +  private Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetTrack = new HashMap<>();
    +
    +  /**
    +   * store offsets with window id, only keep offsets with windows that have not been committed
    +   */
    +  private transient List<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> offsetHistory = new LinkedList<>();
    +
    +  /**
    +   * initial partition count
    +   * only used with PartitionStrategy.ONE_TO_MANY
    +   */
    +  private int initialPartitionCount = 1;
    +
    +  /**
    +   * Minimal interval between 2 (re)partition actions
    +   */
    +  private long repartitionInterval = 30000L;
    +
    +  /**
    +   * Minimal interval between checking collected stats and decide whether it needs to repartition or not.
    +   * And minimal interval between 2 offset updates
    +   */
    +  private long repartitionCheckInterval = 5000L;
    +
    +
    +
    +  /**
    +   * maximum tuples allowed to be emitted in each window
    +   */
    +  @Min(1)
    +  private int maxTuplesPerWindow = Integer.MAX_VALUE;
    +
    +  /**
    +   *  Initial offset, it should be one of the following
    +   *  <ul>
    +   *    <li>earliest</li>
    +   *    <li>latest</li>
    +   *    <li>application_or_earliest</li>
    +   *    <li>application_or_latest</li>
    +   *  </ul>
    +   */
    +  private InitialOffset initialOffset = InitialOffset.APPLICATION_OR_LATEST;
    +
    +  /**
    +   * metrics refresh interval
    +   */
    +  private long metricsRefreshInterval = 5000L;
    +
    +  /**
    +   * @see <a href="http://kafka.apache.org/090/javadoc/org/apache/kafka/clients/consumer/KafkaConsumer.html#poll(long)">
    +   *   org.apache.kafka.clients.consumer.KafkaConsumer.poll</a>
    +   */
    +  private long consumerTimeout = 5000L;
    +
    +  /**
    +   * Number of messages kept in memory waiting for emission to downstream operator
    +   */
    +  private int holdingBufferSize = 1024;
    +
    +  /**
    +   * Extra kafka consumer properties
    +   * http://kafka.apache.org/090/documentation.html#newconsumerconfigs
    +   *
    +   * Please be aware that the properties below are set by the operator, don't override it
    +   *
    +   * <ul>
    +   * <li>bootstrap.servers</li>
    +   * <li>group.id</li>
    +   * <li>auto.offset.reset</li>
    +   * <li>enable.auto.commit</li>
    +   * <li>partition.assignment.strategy</li>
    +   * <li>key.deserializer</li>
    +   * <li>value.deserializer</li>
    +   * </ul>
    +   *
    +   */
    +  private Properties consumerProps;
    +
    +  /**
    +   * count the emitted message in each window<br>
    +   * non settable
    +   */
    +  private transient int emitCount = 0;
    +
    +  /**
    +   * Application name is used as group.id for kafka consumer
    +   */
    +  private transient String applicationName;
    +
    +  private transient AbstractKafkaPartitioner partitioner;
    +
    +  protected transient long currentWindowId;
    +
    +  /**
    +   * By default the strategy is one to one
    +   * @see PartitionStrategy
    +   */
    +  private PartitionStrategy strategy = PartitionStrategy.ONE_TO_ONE;
    +
    +  private transient long lastCheckTime = 0L;
    +
    +  private transient long lastRepartitionTime = 0L;
    +
    +  @AutoMetric
    +  private transient KafkaMetrics metrics;
    +
    +  @Override
    +  public void activate(Context.OperatorContext context)
    +  {
    +    consumerWrapper.start();
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +    consumerWrapper.stop();
    +  }
    +
    +  @Override
    +  public void checkpointed(long l)
    +  {
    +
    +  }
    +
    +  @Override
    +  public void committed(long windowId)
    +  {
    +    //ask kafka consumer wrapper to store the committed offsets
    +    for (Iterator<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> iter = offsetHistory.iterator(); iter.hasNext(); ) {
    +      Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>> item = iter.next();
    +      if (item.getLeft() <= windowId) {
    +        if (item.getLeft() == windowId) {
    +          consumerWrapper.commitOffsets(item.getRight());
    +        }
    +        iter.remove();
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void emitTuples()
    +  {
    +    int count = consumerWrapper.messageSize();
    +    if (maxTuplesPerWindow > 0) {
    +      count = Math.min(count, maxTuplesPerWindow - emitCount);
    +    }
    +    for (int i = 0; i < count; i++) {
    +      Pair<String, ConsumerRecord<byte[], byte[]>> tuple = consumerWrapper.pollMessage();
    +      ConsumerRecord<byte[], byte[]> msg = tuple.getRight();
    +      emitTuple(tuple.getLeft(), msg);
    +      AbstractKafkaPartitioner.PartitionMeta pm = new AbstractKafkaPartitioner.PartitionMeta(tuple.getLeft(),
    +          msg.topic(), msg.partition());
    +      offsetTrack.put(pm, msg.offset());
    +    }
    +    emitCount += count;
    +  }
    +
    +  protected abstract void emitTuple(String cluster, ConsumerRecord<byte[], byte[]> message);
    +
    +  @Override
    +  public void beginWindow(long wid)
    +  {
    +    emitCount = 0;
    +    currentWindowId = wid;
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    // copy current offset track to history memory
    +    Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetsWithWindow = new HashMap<>(offsetTrack);
    +    offsetHistory.add(Pair.of(currentWindowId, offsetsWithWindow));
    +
    +    //update metrics
    +    metrics.updateMetrics(clusters, consumerWrapper.getAllConsumerMetrics());
    +  }
    +
    +
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    applicationName = context.getValue(Context.DAGContext.APPLICATION_NAME);
    +    consumerWrapper.create(this);
    +    metrics = new KafkaMetrics(metricsRefreshInterval);
    +  }
    +
    +
    +  @Override
    +  public void teardown()
    +  {
    +
    +  }
    +
    +  private void initPartitioner()
    +  {
    +    if (partitioner == null) {
    +      logger.info("Initialize Partitioner");
    +      switch (strategy) {
    +        case ONE_TO_ONE:
    +          partitioner = new OneToOnePartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY:
    +          partitioner = new OneToManyPartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY_HEURISTIC:
    +          throw new UnsupportedOperationException("Not implemented yet");
    +        default:
    +          throw new RuntimeException("Invalid strategy");
    +      }
    +      logger.info("Actual Partitioner is {}", partitioner.getClass());
    +    }
    +
    +  }
    +
    +  @Override
    +  public Response processStats(BatchedOperatorStats batchedOperatorStats)
    +  {
    +    long t = System.currentTimeMillis();
    +    if (repartitionInterval < 0 || repartitionCheckInterval < 0 ||
    +        t - lastCheckTime < repartitionCheckInterval || t - lastRepartitionTime < repartitionInterval) {
    +      // return false if it's within repartitionCheckInterval since last time it check the stats
    +      Response response = new Response();
    +      response.repartitionRequired = false;
    +      return response;
    +    }
    +
    +    try {
    +      logger.debug("Process stats");
    +      initPartitioner();
    +      return partitioner.processStats(batchedOperatorStats);
    +    } finally {
    +      lastCheckTime = System.currentTimeMillis();
    +    }
    +  }
    +
    +  @Override
    +  public Collection<Partition<AbstractKafkaInputOperator>> definePartitions(
    +      Collection<Partition<AbstractKafkaInputOperator>> collection, PartitioningContext partitioningContext)
    +  {
    +    logger.debug("Define partitions");
    +    initPartitioner();
    +    return partitioner.definePartitions(collection, partitioningContext);
    +  }
    +
    +  @Override
    +  public void partitioned(Map<Integer, Partition<AbstractKafkaInputOperator>> map)
    +  {
    +    // update the last repartition time
    +    lastRepartitionTime = System.currentTimeMillis();
    +    initPartitioner();
    +    partitioner.partitioned(map);
    +  }
    +
    +  /**
    +   *
    +   * A callback from consumer after it commits the offset
    +   * @param map
    +   * @param e
    +   */
    +  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);
    +    }
    +  }
    +
    +
    +
    +  //---------------------------------------------setters and getters----------------------------------------
    +  public void setInitialPartitionCount(int partitionCount)
    +  {
    +    this.initialPartitionCount = partitionCount;
    +  }
    +
    +  public int getInitialPartitionCount()
    +  {
    +    return initialPartitionCount;
    +  }
    +
    +  public void setClusters(String clusters)
    +  {
    +    this.clusters = clusters.split(";");
    +  }
    +
    +  public String getClusters()
    +  {
    +    return Joiner.on(';').join(clusters);
    +  }
    +
    +  public void setTopics(String... topics)
    +  {
    +    this.topics = topics;
    +  }
    +
    +  public String[] getTopics()
    +  {
    +    return topics;
    +  }
    +
    +  public void setAssignment(Set<AbstractKafkaPartitioner.PartitionMeta> assignment)
    +  {
    +    this.assignment = assignment;
    +  }
    +
    +  public Set<AbstractKafkaPartitioner.PartitionMeta> getAssignment()
    +  {
    +    return assignment;
    +  }
    +
    +  public Map<AbstractKafkaPartitioner.PartitionMeta, Long> getOffsetTrack()
    +  {
    +    return offsetTrack;
    +  }
    +
    +  public void setStrategy(String policy)
    +  {
    +    this.strategy = PartitionStrategy.valueOf(policy.toUpperCase());
    +  }
    +
    +  public PartitionStrategy getStrategy()
    +  {
    +    return strategy;
    +  }
    +
    +  public void setInitialOffset(String initialOffset)
    +  {
    +    this.initialOffset = InitialOffset.valueOf(initialOffset.toUpperCase());
    +  }
    +
    +  public InitialOffset getInitialOffset()
    +  {
    +    return initialOffset;
    +  }
    +
    +  public String getApplicationName()
    +  {
    +    return applicationName;
    +  }
    +
    +  public void setConsumerProps(Properties consumerProps)
    +  {
    +    this.consumerProps = consumerProps;
    +  }
    +
    +  public Properties getConsumerProps()
    +  {
    +    return consumerProps;
    +  }
    +
    +  public void setMaxTuplesPerWindow(int maxTuplesPerWindow)
    +  {
    +    this.maxTuplesPerWindow = maxTuplesPerWindow;
    +  }
    +
    +  public int getMaxTuplesPerWindow()
    +  {
    +    return maxTuplesPerWindow;
    +  }
    +
    +  public long getConsumerTimeout()
    +  {
    +    return consumerTimeout;
    +  }
    +
    +  public void setConsumerTimeout(long consumerTimeout)
    +  {
    +    this.consumerTimeout = consumerTimeout;
    +  }
    +
    +  public int getHoldingBufferSize()
    --- End diff --
    
    getQueueCapacity() ?


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r48379994
  
    --- Diff: kafka/src/main/java/org/apache/apex/malhar/kafka/AbstractKafkaPartitioner.java ---
    @@ -0,0 +1,276 @@
    +/**
    + * 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.ByteArrayOutputStream;
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Objects;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.kafka.clients.consumer.KafkaConsumer;
    +import org.apache.kafka.common.PartitionInfo;
    +import org.apache.kafka.common.TopicPartition;
    +
    +import com.esotericsoftware.kryo.Kryo;
    +import com.esotericsoftware.kryo.io.Input;
    +import com.esotericsoftware.kryo.io.Output;
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.DefaultPartition;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * Abstract partitioner used to manage the partitions of kafka input operator.
    + * It use a number of kafka consumers(one for each cluster) to get the latest partition metadata for topics that
    + * the consumer subscribes and expose those to subclass which implements the assign method
    + *
    + * The partitioner is always stateless.
    + */
    +public abstract class AbstractKafkaPartitioner implements Partitioner<AbstractKafkaInputOperator>, StatsListener
    +{
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaPartitioner.class);
    +
    +  private static final String META_CONSUMER_GROUP_NAME = AbstractKafkaInputOperator.class.getName() + "META_GROUP";
    +
    +  protected String[] clusters;
    +
    +  protected String[] topics;
    +
    +  protected AbstractKafkaInputOperator prototypeOperator;
    +
    +  private KafkaConsumer[] metadataRefreshClients;
    +
    +
    +  private List<Set<AbstractKafkaPartitioner.PartitionMeta>> currentPartitions = new LinkedList<>(); // prevent null
    +
    +  public AbstractKafkaPartitioner(String[] clusters, String[] topics, AbstractKafkaInputOperator prototypeOperator)
    +  {
    +    this.clusters = clusters;
    +    this.topics = topics;
    +    this.prototypeOperator = prototypeOperator;
    +  }
    +
    +  abstract List<Set<PartitionMeta>> assign(Map<String, Map<String,List<PartitionInfo>>> metadata);
    +
    +
    +
    +  @Override
    +  public Collection<Partition<AbstractKafkaInputOperator>> definePartitions(Collection<Partition<AbstractKafkaInputOperator>> collection, PartitioningContext partitioningContext)
    +  {
    +
    +    initMetadataClients();
    +
    +    Map<String, Map<String, List<PartitionInfo>>> metadata = new HashMap<>();
    +
    +
    +    for (int i = 0; i < clusters.length; i++) {
    +      metadata.put(clusters[i], new HashMap<String, List<PartitionInfo>>());
    +      for (String topic : topics) {
    +        List<PartitionInfo> ptis = metadataRefreshClients[i].partitionsFor(topic);
    +        if (logger.isDebugEnabled()) {
    +          logger.debug("Partition metadata for topic {} : {}", topic, Joiner.on(';').join(ptis));
    +        }
    +        metadata.get(clusters[i]).put(topic, ptis);
    +      }
    +      metadataRefreshClients[i].close();
    +    }
    +
    +    metadataRefreshClients = null;
    +
    +    List<Set<AbstractKafkaPartitioner.PartitionMeta>> parts = assign(metadata);
    +
    +
    +    if (currentPartitions == parts || currentPartitions.equals(parts)) {
    +      logger.debug("No partition change found");
    +      return collection;
    +    } else {
    +      logger.info("Partition change detected: ");
    +      currentPartitions.clear();
    +      currentPartitions.addAll(parts);
    +      int i = 0;
    +      List<Partition<AbstractKafkaInputOperator>> result = new LinkedList<>();
    +      for (Iterator<Partition<AbstractKafkaInputOperator>> iter = collection.iterator(); iter.hasNext();) {
    +        Partition<AbstractKafkaInputOperator> nextPartition = iter.next();
    +        if (parts.remove(nextPartition.getPartitionedInstance().getAssignment())) {
    +          if (logger.isInfoEnabled()) {
    +            logger.info("[Existing] Partition {} with assignment {} ", i,
    +                Joiner.on(';').join(nextPartition.getPartitionedInstance().getAssignment()));
    +          }
    +          result.add(nextPartition);
    +          i++;
    +        }
    +      }
    +
    +      for (Set<AbstractKafkaPartitioner.PartitionMeta> partitionAssignment : parts) {
    +        if (logger.isInfoEnabled()) {
    +          logger.info("[New] Partition {} with assignment {} ", i,
    +              Joiner.on(';').join(partitionAssignment));
    +        }
    +        result.add(createPartition(partitionAssignment));
    +        i++;
    +      }
    +
    +
    +      return result;
    +    }
    +  }
    +
    +  @Override
    +  public void partitioned(Map<Integer, Partition<AbstractKafkaInputOperator>> map)
    +  {
    +
    +  }
    +
    +  @Override
    +  public Response processStats(BatchedOperatorStats batchedOperatorStats)
    +  {
    +    Response response = new Response();
    +    response.repartitionRequired = true;
    +    return response;
    +  }
    +
    +  protected Partitioner.Partition<AbstractKafkaInputOperator> createPartition(Set<AbstractKafkaPartitioner.PartitionMeta> partitionAssignment)
    +  {
    +    Kryo kryo = new Kryo();
    +    ByteArrayOutputStream bos = new ByteArrayOutputStream();
    +    Output output = new Output(bos);
    +    kryo.writeObject(output, prototypeOperator);
    +    output.close();
    +    Input lInput = new Input(bos.toByteArray());
    +    Partitioner.Partition<AbstractKafkaInputOperator> p = new DefaultPartition<>(kryo.readObject(lInput, prototypeOperator.getClass()));
    --- End diff --
    
    Still getting the compile error in Eclipse and code is still littered with type warnings.


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r47983102
  
    --- Diff: kafka/src/main/java/com/datatorrent/contrib/kafka090/AbstractKafkaInputOperator.java ---
    @@ -0,0 +1,410 @@
    +/**
    + * 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 com.datatorrent.contrib.kafka090;
    +
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.tuple.Pair;
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * The abstract kafka input operator using kafka 0.9.0 new consumer API
    + * A scalable, fault-tolerant, at-least-once kafka input operator
    + * Keu features includes:
    + * 1. Out-of-box One-to-one and one-to-many partition schema support plus customizable partition schem
    + *    refer to AbstractKafkaPartitioner
    + * 2. Fault-tolerant when the input operator goes down, it redeploys on other node
    + * 3. At-least-once semantics for operator failure (no matter which operator fails)
    + * 4. At-least-once semantics for cold restart (no data loss even if you restart the application)
    + * 5. Multi-cluster support, one operator can consume data from more than one kafka clusters
    + * 6. Multi-topic support, one operator can subscribe multiple topics
    + * 7. Throughput control support, you can throttle number of tuple for each streaming window
    + */
    +public abstract class AbstractKafkaInputOperator<K, V> implements InputOperator, Operator.ActivationListener<Context.OperatorContext>, Operator.CheckpointListener, Partitioner<AbstractKafkaInputOperator>, StatsListener
    +{
    +
    +
    +  private static final Logger logger = LoggerFactory.getLogger(AbstractKafkaInputOperator.class);
    +
    +  public enum InitialOffset {
    +    EARLIEST, // consume from beginning of the partition every time when application restart
    +    LATEST, // consume from latest of the partition every time when application restart
    +    APPLICATION_OR_EARLIEST, // consume from committed position from last run or earliest if there is no committed offset(s)
    +    APPLICATION_OR_LATEST // consume from committed position from last run or latest if there is no committed offset(s)
    +  }
    +  /**
    +   *  node separate by ',' and clusters separate by ';'
    +   *  ex: c1n1,c1n2;c2n1,c2n2
    +   */
    +  @NotNull
    +  private String[] clusters;
    +
    +  /**
    +   * The topics the operator consumes
    +   */
    +  @NotNull
    +  private String[] topics;
    +
    +  /**
    +   * Wrapper consumer object
    +   * It wraps KafkaConsumer, maintains consumer thread and store messages in a queue
    +   */
    +  private KafkaConsumerWrapper<K, V> consumerWrapper = new KafkaConsumerWrapper<>();
    +
    +  /**
    +   * Assignment for each operator instance
    +   */
    +  private Set<AbstractKafkaPartitioner.PartitionMeta> assignment;
    +
    +  /**
    +   *  offset track for checkpoint
    +   */
    +  private Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetTrack = new HashMap<>();
    +
    +  /**
    +   * store offsets with window id, only keep offsets with windows that have not been committed
    +   */
    +  private transient List<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> offsetHistory = new LinkedList<>();
    +
    +  /**
    +   * initial partition count
    +   * only used with PartitionStrategy.ONE_TO_MANY
    +   */
    +  private int initialPartitionCount = 1;
    +
    +  // Minimal interval between 2 (re)partition actions
    +  private long repartitionInterval = 30000L;
    +
    +  // Minimal interval between checking collected stats and decide whether it needs to repartition or not.
    +  // And minimal interval between 2 offset updates
    +  private long repartitionCheckInterval = 5000L;
    +
    +  /**
    +   * maximum tuples allowed to be emitted in each window
    +   */
    +  @Min(1)
    +  private int maxTuplesPerWindow = Integer.MAX_VALUE;
    +
    +  private InitialOffset initialOffset = InitialOffset.APPLICATION_OR_LATEST;
    +
    +  /**
    +   * extra kafka consumer properties
    +   * http://kafka.apache.org/090/documentation.html#newconsumerconfigs
    +   *
    +   * Please be aware that the properties below are set by the operator, don't override it
    +   *
    +   * bootstrap.servers
    +   * group.id
    +   * auto.offset.reset
    +   * enable.auto.commit
    +   * partition.assignment.strategy
    +   *
    +   */
    +  private Properties consumerProps;
    +
    +  /**
    +   * count the emitted message in each window
    +   * non settable
    +   */
    +  private transient int emitCount = 0;
    +
    +  /**
    +   * Application name is used as group.id for kafka consumer
    +   */
    +  private transient String applicationName;
    +
    +  private transient AbstractKafkaPartitioner partitioner;
    +
    +  protected transient long currentWindowId;
    +
    +  protected transient int operatorId;
    +
    +  // By default the partition policy is 1:1
    +  private PartitionStrategy strategy = PartitionStrategy.ONE_TO_ONE;
    +
    +  private transient long lastCheckTime = 0L;
    +
    +  private transient long lastRepartitionTime = 0L;
    +
    +
    +  @Override
    +  public void activate(Context.OperatorContext context)
    +  {
    +    consumerWrapper.start();
    +  }
    +
    +  @Override
    +  public void deactivate()
    +  {
    +    consumerWrapper.stop();
    +  }
    +
    +  @Override
    +  public void checkpointed(long l)
    +  {
    +
    +  }
    +
    +  @Override
    +  public void committed(long windowId)
    +  {
    +    //ask kafka consumer wrapper to store the committed offsets
    +    for (Iterator<Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>>> iter = offsetHistory.iterator(); iter.hasNext(); ) {
    +      Pair<Long, Map<AbstractKafkaPartitioner.PartitionMeta, Long>> item = iter.next();
    +      if (item.getLeft() <= windowId) {
    +        if (item.getLeft() == windowId) {
    +          consumerWrapper.commitOffsets(item.getRight());
    +        }
    +        iter.remove();
    +      }
    +    }
    +  }
    +
    +  @Override
    +  public void emitTuples()
    +  {
    +    int count = consumerWrapper.messageSize();
    +    if (maxTuplesPerWindow > 0) {
    +      count = Math.min(count, maxTuplesPerWindow - emitCount);
    +    }
    +    for (int i = 0; i < count; i++) {
    +      Pair<String, ConsumerRecord<K, V>> tuple = consumerWrapper.pollMessage();
    +      ConsumerRecord<K, V> msg = tuple.getRight();
    +      emitTuple(tuple.getLeft(), msg);
    +      AbstractKafkaPartitioner.PartitionMeta pm = new AbstractKafkaPartitioner.PartitionMeta(tuple.getLeft(),
    +        msg.topic(), msg.partition());
    +      offsetTrack.put(pm, msg.offset());
    +    }
    +    emitCount += count;
    +  }
    +
    +  protected abstract void emitTuple(String cluster, ConsumerRecord<K, V> msg);
    +
    +  @Override
    +  public void beginWindow(long wid)
    +  {
    +    emitCount = 0;
    +    currentWindowId = wid;
    +  }
    +
    +  @Override
    +  public void endWindow()
    +  {
    +    // copy current offset track to history memory
    +    Map<AbstractKafkaPartitioner.PartitionMeta, Long> offsetsWithWindow = new HashMap<>(offsetTrack);
    +    offsetHistory.add(Pair.of(currentWindowId, offsetsWithWindow));
    +  }
    +
    +  @Override
    +  public void setup(Context.OperatorContext context)
    +  {
    +    applicationName = context.getValue(Context.DAGContext.APPLICATION_NAME);
    +    consumerWrapper.create(this);
    +  }
    +
    +  @Override
    +  public void teardown()
    +  {
    +
    +  }
    +
    +  private void initPartitioner()
    +  {
    +    if(partitioner == null) {
    +      logger.info("Initialize Partitioner");
    +      switch (strategy) {
    +        case ONE_TO_ONE:
    +          partitioner = new OneToOnePartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY:
    +          partitioner = new OneToManyPartitioner(clusters, topics, this);
    +          break;
    +        case ONE_TO_MANY_HEURISTIC:
    +          throw new UnsupportedOperationException("Not implemented yet");
    +      }
    +      logger.info("Actual Partitioner is {}", partitioner.getClass());
    +    }
    +
    +  }
    +
    +  @Override
    +  public Response processStats(BatchedOperatorStats batchedOperatorStats)
    +  {
    +    long t = System.currentTimeMillis();
    +
    +    if (repartitionInterval < 0 || repartitionCheckInterval < 0 ||
    +      t - lastCheckTime < repartitionCheckInterval || t - lastRepartitionTime < repartitionInterval) {
    +      // return false if it's within repartitionCheckInterval since last time it check the stats
    +      logger.debug("Within Repartition Check Interval {} or Repartition Interval {}", repartitionCheckInterval,
    --- End diff --
    
    No need to log on every call.


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r47961214
  
    --- Diff: kafka/src/main/java/com/datatorrent/contrib/kafka090/AbstractKafkaInputOperator.java ---
    @@ -0,0 +1,410 @@
    +/**
    + * 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 com.datatorrent.contrib.kafka090;
    +
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.tuple.Pair;
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * The abstract kafka input operator using kafka 0.9.0 new consumer API
    + * A scalable, fault-tolerant, at-least-once kafka input operator
    + * Keu features includes:
    + * 1. Out-of-box One-to-one and one-to-many partition schema support plus customizable partition schem
    + *    refer to AbstractKafkaPartitioner
    + * 2. Fault-tolerant when the input operator goes down, it redeploys on other node
    + * 3. At-least-once semantics for operator failure (no matter which operator fails)
    + * 4. At-least-once semantics for cold restart (no data loss even if you restart the application)
    + * 5. Multi-cluster support, one operator can consume data from more than one kafka clusters
    + * 6. Multi-topic support, one operator can subscribe multiple topics
    + * 7. Throughput control support, you can throttle number of tuple for each streaming window
    + */
    +public abstract class AbstractKafkaInputOperator<K, V> implements InputOperator, Operator.ActivationListener<Context.OperatorContext>, Operator.CheckpointListener, Partitioner<AbstractKafkaInputOperator>, StatsListener
    +{
    +
    --- End diff --
    
    extra blank lines


---
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.
---

[GitHub] incubator-apex-malhar pull request: New Kafka input operator using...

Posted by tweise <gi...@git.apache.org>.
Github user tweise commented on a diff in the pull request:

    https://github.com/apache/incubator-apex-malhar/pull/134#discussion_r47961112
  
    --- Diff: kafka/src/main/java/com/datatorrent/contrib/kafka090/AbstractKafkaInputOperator.java ---
    @@ -0,0 +1,410 @@
    +/**
    + * 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 com.datatorrent.contrib.kafka090;
    +
    +import java.util.Collection;
    +import java.util.HashMap;
    +import java.util.Iterator;
    +import java.util.LinkedList;
    +import java.util.List;
    +import java.util.Map;
    +import java.util.Properties;
    +import java.util.Set;
    +
    +import javax.validation.constraints.Min;
    +import javax.validation.constraints.NotNull;
    +
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +
    +import org.apache.commons.lang3.tuple.Pair;
    +import org.apache.kafka.clients.consumer.ConsumerRecord;
    +
    +import com.google.common.base.Joiner;
    +
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.InputOperator;
    +import com.datatorrent.api.Operator;
    +import com.datatorrent.api.Partitioner;
    +import com.datatorrent.api.StatsListener;
    +
    +/**
    + * The abstract kafka input operator using kafka 0.9.0 new consumer API
    + * A scalable, fault-tolerant, at-least-once kafka input operator
    + * Keu features includes:
    --- End diff --
    
    Typo


---
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.
---