Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 35125200B26 for ; Mon, 13 Jun 2016 07:25:37 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 3397A160A5C; Mon, 13 Jun 2016 05:25:37 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 53458160A2C for ; Mon, 13 Jun 2016 07:25:36 +0200 (CEST) Received: (qmail 74695 invoked by uid 500); 13 Jun 2016 05:25:35 -0000 Mailing-List: contact dev-help@apex.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@apex.apache.org Delivered-To: mailing list dev@apex.apache.org Received: (qmail 74684 invoked by uid 99); 13 Jun 2016 05:25:35 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd3-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Mon, 13 Jun 2016 05:25:35 +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 B976F1800B7 for ; Mon, 13 Jun 2016 05:25:34 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd3-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -5.446 X-Spam-Level: X-Spam-Status: No, score=-5.446 tagged_above=-999 required=6.31 tests=[KAM_LAZY_DOMAIN_SECURITY=1, RCVD_IN_DNSWL_HI=-5, RCVD_IN_MSPIKE_H3=-0.01, RCVD_IN_MSPIKE_WL=-0.01, RP_MATCHES_RCVD=-1.426] autolearn=disabled Received: from mx2-lw-eu.apache.org ([10.40.0.8]) by localhost (spamd3-us-west.apache.org [10.40.0.10]) (amavisd-new, port 10024) with ESMTP id tjg1NFFVTfh5 for ; Mon, 13 Jun 2016 05:25:33 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx2-lw-eu.apache.org (ASF Mail Server at mx2-lw-eu.apache.org) with SMTP id F2C905F474 for ; Mon, 13 Jun 2016 05:25:31 +0000 (UTC) Received: (qmail 74681 invoked by uid 99); 13 Jun 2016 05:25:30 -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, 13 Jun 2016 05:25:30 +0000 Received: by git1-us-west.apache.org (ASF Mail Server at git1-us-west.apache.org, from userid 33) id 8AA53E01E2; Mon, 13 Jun 2016 05:25:30 +0000 (UTC) From: devtagare To: dev@apex.incubator.apache.org Reply-To: dev@apex.incubator.apache.org References: In-Reply-To: Subject: [GitHub] apex-malhar pull request #298: *For review only* [APEXMALHAR-2086] Kafka out... Content-Type: text/plain Message-Id: <20160613052530.8AA53E01E2@git1-us-west.apache.org> Date: Mon, 13 Jun 2016 05:25:30 +0000 (UTC) archived-at: Mon, 13 Jun 2016 05:25:37 -0000 Github user devtagare commented on a diff in the pull request: https://github.com/apache/apex-malhar/pull/298#discussion_r66741302 --- Diff: kafka/src/main/java/org/apache/apex/malhar/kafka/AbstractExactlyOnceKafkaOutputOperator.java --- @@ -0,0 +1,366 @@ +/** + * 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.IOException; +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Properties; +import java.util.concurrent.ExecutionException; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import org.apache.apex.malhar.lib.wal.FSWindowDataManager; +import org.apache.apex.malhar.lib.wal.WindowDataManager; + +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.producer.ProducerRecord; +import org.apache.kafka.common.PartitionInfo; +import org.apache.kafka.common.TopicPartition; + +import com.datatorrent.api.Context; +import com.datatorrent.api.Operator; + +import static org.apache.kafka.clients.consumer.ConsumerConfig.BOOTSTRAP_SERVERS_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.KEY_DESERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.consumer.ConsumerConfig.VALUE_DESERIALIZER_CLASS_CONFIG; +import static org.apache.kafka.clients.producer.ProducerConfig.ACKS_CONFIG; + +/** + * Kafka output operator with exactly once processing semantics under certain conditions., + * + * Requirement for Exactly Once: + * Every message within the Window is unique + * + * This operator uses *Key* to distinguish the messages written by particular instance of the Output operator. + * Operator users can only use *value* for storing the data. + * + * @displayName Abstract Exactly Once Kafka Output(0.9.0) + * @category Messaging + * @tags output operator + * + * @since 3.5 + */ +@org.apache.hadoop.classification.InterfaceStability.Evolving +public abstract class AbstractExactlyOnceKafkaOutputOperator extends AbstractKafkaOutputOperator + implements Operator.CheckpointNotificationListener +{ + private transient String key; + private transient String appId; + private transient Integer operatorId; + private transient Long windowId; + private transient Map partialWindowTuples = new HashMap<>(); + private transient KafkaConsumer consumer; + + private WindowDataManager windowDataManager = new FSWindowDataManager(); + private final int KAFKA_CONNECT_ATTEMPT = 10; + private final String KEY_SEPARATOR = "#"; + private final String KEY_SERIALIZER = "org.apache.kafka.common.serialization.StringDeserializer"; + private final String VALUE_SERIALIZER = "org.apache.kafka.common.serialization.StringDeserializer"; + + @Override + public void setup(Context.OperatorContext context) + { + super.setup(context); + + this.operatorId = context.getId(); + this.windowDataManager.setup(context); + this.appId = context.getValue(Context.DAGContext.APPLICATION_ID); + this.key = appId + KEY_SEPARATOR + (new Integer(operatorId)); + this.consumer = KafkaConsumerInit(); + + setProperty(ACKS_CONFIG, "all"); + } + + @Override + public void beginWindow(long windowId) + { + this.windowId = windowId; + + if (windowId == windowDataManager.getLargestRecoveryWindow()) { + rebuildPartialWindow(); + } + } + + @Override + public void checkpointed(long windowId) + { + } + + @Override + public void committed(long windowId) + { + try { + windowDataManager.deleteUpTo(operatorId, windowId); + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public void beforeCheckpoint(long windowId) + { + } + + @Override + public void teardown() + { + consumer.close(); + super.teardown(); + } + + @Override + public void endWindow() + { + if (!partialWindowTuples.isEmpty()) { + // Throw exception + } + + try { + this.windowDataManager.save(getPartitionsAndOffsets(), operatorId, windowId); + } catch (IOException e) { + throw new RuntimeException(e); + } catch (ExecutionException e) { + throw new RuntimeException(e); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + + public WindowDataManager getWindowDataManager() + { + return windowDataManager; + } + + public void setWindowDataManager(WindowDataManager windowDataManager) + { + this.windowDataManager = windowDataManager; + } + + private boolean doesKeyBelongsToThisInstance(Integer operatorId, String key) + { + String[] split = key.split(KEY_SEPARATOR); + + if (split.length != 2) { + return false; + } + + if ((Integer.parseInt(split[1]) == operatorId) && (split[0].equals(appId))) { --- End diff -- why not have a partitionId instead of appId.A re-launch will give a new applicationId so the second check will not 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. ---