Return-Path: X-Original-To: apmail-apex-dev-archive@minotaur.apache.org Delivered-To: apmail-apex-dev-archive@minotaur.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 1E87F18198 for ; Mon, 28 Dec 2015 21:37:30 +0000 (UTC) Received: (qmail 94073 invoked by uid 500); 28 Dec 2015 21:37:30 -0000 Delivered-To: apmail-apex-dev-archive@apex.apache.org Received: (qmail 94002 invoked by uid 500); 28 Dec 2015 21:37:30 -0000 Mailing-List: contact dev-help@apex.incubator.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@apex.incubator.apache.org Delivered-To: mailing list dev@apex.incubator.apache.org Received: (qmail 93991 invoked by uid 99); 28 Dec 2015 21:37:29 -0000 Received: from Unknown (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 28 Dec 2015 21:37:29 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd3-us-west.apache.org (ASF Mail Server at spamd3-us-west.apache.org) with ESMTP id 3D0FE180186 for ; Mon, 28 Dec 2015 21:37:29 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 0.427 X-Spam-Level: X-Spam-Status: No, score=0.427 tagged_above=-999 required=6.31 tests=[KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-0.554, URIBL_BLOCKED=0.001] autolearn=disabled Received: from mx1-us-west.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id T0nW6eNBY9T5 for ; Mon, 28 Dec 2015 21:37:22 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-us-west.apache.org (ASF Mail Server at mx1-us-west.apache.org) with SMTP id 36E7720F46 for ; Mon, 28 Dec 2015 21:37:22 +0000 (UTC) Received: (qmail 93985 invoked by uid 99); 28 Dec 2015 21:37:22 -0000 Received: from git1-us-west.apache.org (HELO git1-us-west.apache.org) (140.211.11.23) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 28 Dec 2015 21:37:21 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id D7996E0446; Mon, 28 Dec 2015 21:37:21 +0000 (UTC) From: siyuanh To: dev@apex.incubator.apache.org Reply-To: dev@apex.incubator.apache.org References: In-Reply-To: Subject: [GitHub] incubator-apex-malhar pull request: New Kafka input operator using... Content-Type: text/plain Message-Id: <20151228213721.D7996E0446@git1-us-west.apache.org> Date: Mon, 28 Dec 2015 21:37:21 +0000 (UTC) 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 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> consumers = new HashMap<>(); + + // The in memory buffer hold consumed messages + private transient ArrayBlockingQueue>> 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> offsetsToCommit = new HashMap<>(); + + /** + * You can customize the offsetCommit + */ + private transient OffsetCommitCallback offsetCommitCallback = new OffsetCommitCallback() + { + @Override + public void onComplete(Map 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 offsetsInWindow) + { + if (offsetsInWindow == null) { + return; + } + + // group offsets by cluster and topic partition + for (Map.Entry e : offsetsInWindow.entrySet()) { + String cluster = e.getKey().getCluster(); + Map 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 consumer; + + private String cluster; + + private KafkaConsumerWrapper wrapper; + + private Map 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 record : (Iterable>)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> consumerAssignment = new HashMap<>(); + for (AbstractKafkaPartitioner.PartitionMeta partitionMeta : ownerOperator.getAssignment()) { + String cluster = partitionMeta.getCluster(); + List cAssignment = consumerAssignment.get(cluster); + if (cAssignment == null) { + cAssignment = new LinkedList<>(); + consumerAssignment.put(cluster, cAssignment); + } + cAssignment.add(new TopicPartition(partitionMeta.getTopic(), partitionMeta.getPartitionId())); + } + + Map 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> 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. ---