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 9BAA11895E for ; Fri, 18 Dec 2015 00:58:06 +0000 (UTC) Received: (qmail 31435 invoked by uid 500); 18 Dec 2015 00:58:06 -0000 Delivered-To: apmail-apex-dev-archive@apex.apache.org Received: (qmail 31368 invoked by uid 500); 18 Dec 2015 00:58:06 -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 31357 invoked by uid 99); 18 Dec 2015 00:58:06 -0000 Received: from Unknown (HELO spamd4-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Fri, 18 Dec 2015 00:58:06 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd4-us-west.apache.org (ASF Mail Server at spamd4-us-west.apache.org) with ESMTP id C3057C07DB for ; Fri, 18 Dec 2015 00:58:05 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd4-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: 1.227 X-Spam-Level: * X-Spam-Status: No, score=1.227 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, 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-eu-west.apache.org ([10.40.0.8]) by localhost (spamd4-us-west.apache.org [10.40.0.11]) (amavisd-new, port 10024) with ESMTP id GeGP8hGax-De for ; Fri, 18 Dec 2015 00:57:57 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-eu-west.apache.org (ASF Mail Server at mx1-eu-west.apache.org) with SMTP id 0B8522026F for ; Fri, 18 Dec 2015 00:57:55 +0000 (UTC) Received: (qmail 31309 invoked by uid 99); 18 Dec 2015 00:57:55 -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; Fri, 18 Dec 2015 00:57:55 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 14708E6967; Fri, 18 Dec 2015 00:57:55 +0000 (UTC) From: tweise 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: <20151218005755.14708E6967@git1-us-west.apache.org> Date: Fri, 18 Dec 2015 00:57:55 +0000 (UTC) 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 implements InputOperator, Operator.ActivationListener, Operator.CheckpointListener, Partitioner, 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 consumerWrapper = new KafkaConsumerWrapper<>(); + + /** + * Assignment for each operator instance + */ + private Set assignment; + + /** + * offset track for checkpoint + */ + private Map offsetTrack = new HashMap<>(); + + /** + * store offsets with window id, only keep offsets with windows that have not been committed + */ + private transient List>> 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>> iter = offsetHistory.iterator(); iter.hasNext(); ) { + Pair> 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> tuple = consumerWrapper.pollMessage(); + ConsumerRecord 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 msg); + + @Override + public void beginWindow(long wid) + { + emitCount = 0; + currentWindowId = wid; + } + + @Override + public void endWindow() + { + // copy current offset track to history memory + Map 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> definePartitions( + Collection> collection, PartitioningContext partitioningContext) + { + logger.debug("Define partitions"); + initPartitioner(); + return partitioner.definePartitions(collection, partitioningContext); + } + + @Override + public void partitioned(Map> 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. ---