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 53D3F200B41 for ; Thu, 23 Jun 2016 08:09:22 +0200 (CEST) Received: by cust-asf.ponee.io (Postfix) id 5288F160A68; Thu, 23 Jun 2016 06:09:22 +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 742AC160A36 for ; Thu, 23 Jun 2016 08:09:21 +0200 (CEST) Received: (qmail 35912 invoked by uid 500); 23 Jun 2016 06:09:20 -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 35901 invoked by uid 99); 23 Jun 2016 06:09:20 -0000 Received: from pnap-us-west-generic-nat.apache.org (HELO spamd2-us-west.apache.org) (209.188.14.142) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 23 Jun 2016 06:09:20 +0000 Received: from localhost (localhost [127.0.0.1]) by spamd2-us-west.apache.org (ASF Mail Server at spamd2-us-west.apache.org) with ESMTP id 164BB1A554B for ; Thu, 23 Jun 2016 06:09:20 +0000 (UTC) X-Virus-Scanned: Debian amavisd-new at spamd2-us-west.apache.org X-Spam-Flag: NO X-Spam-Score: -4.646 X-Spam-Level: X-Spam-Status: No, score=-4.646 tagged_above=-999 required=6.31 tests=[KAM_ASCII_DIVIDERS=0.8, 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 mx1-lw-us.apache.org ([10.40.0.8]) by localhost (spamd2-us-west.apache.org [10.40.0.9]) (amavisd-new, port 10024) with ESMTP id Dg6PS64NLPfQ for ; Thu, 23 Jun 2016 06:09:18 +0000 (UTC) Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by mx1-lw-us.apache.org (ASF Mail Server at mx1-lw-us.apache.org) with SMTP id B1B1E5FCF7 for ; Thu, 23 Jun 2016 06:09:17 +0000 (UTC) Received: (qmail 34476 invoked by uid 99); 23 Jun 2016 06:09:16 -0000 Received: from arcas.apache.org (HELO arcas) (140.211.11.28) by apache.org (qpsmtpd/0.29) with ESMTP; Thu, 23 Jun 2016 06:09:16 +0000 Received: from arcas.apache.org (localhost [127.0.0.1]) by arcas (Postfix) with ESMTP id C66232C14F8 for ; Thu, 23 Jun 2016 06:09:16 +0000 (UTC) Date: Thu, 23 Jun 2016 06:09:16 +0000 (UTC) From: "ASF GitHub Bot (JIRA)" To: dev@apex.incubator.apache.org Message-ID: In-Reply-To: References: Subject: [jira] [Commented] (APEXMALHAR-2086) Kafka Output Operator with Kafka 0.9 API MIME-Version: 1.0 Content-Type: text/plain; charset=utf-8 Content-Transfer-Encoding: 7bit X-JIRA-FingerPrint: 30527f35849b9dde25b450d4833f0394 archived-at: Thu, 23 Jun 2016 06:09:22 -0000 [ https://issues.apache.org/jira/browse/APEXMALHAR-2086?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15345838#comment-15345838 ] ASF GitHub Bot commented on APEXMALHAR-2086: -------------------------------------------- Github user siyuanh commented on a diff in the pull request: https://github.com/apache/apex-malhar/pull/298#discussion_r68179972 --- Diff: kafka/src/main/java/org/apache/apex/malhar/kafka/KafkaSinglePortExactlyOnceOutputOperator.java --- @@ -0,0 +1,369 @@ +/** + * 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.DefaultInputPort; +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; + +/** + * Kafka output operator with exactly once processing semantics under certain conditions., + * + * 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 Single Port Exactly Once Kafka Output(0.9.0) + * @category Messaging + * @tags output operator + * + */ +@org.apache.hadoop.classification.InterfaceStability.Evolving +public class KafkaSinglePortExactlyOnceOutputOperator extends AbstractKafkaOutputOperator + implements Operator.CheckpointNotificationListener +{ + private transient String key; + private transient String appName; + 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"; + + public final transient DefaultInputPort inputPort = new DefaultInputPort() + { + @Override + public void process(T tuple) + { + sendTuple(tuple); + } + }; + + @Override + public void setup(Context.OperatorContext context) + { + super.setup(context); + + this.operatorId = context.getId(); + this.windowDataManager.setup(context); + this.appName = context.getValue(Context.DAGContext.APPLICATION_NAME); + this.key = appName + KEY_SEPARATOR + (new Integer(operatorId)); + this.consumer = KafkaConsumerInit(); + } + + @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() && windowId > windowDataManager.getLargestRecoveryWindow()) { + throw new RuntimeException("Violates Exactly once. Not all the tuples received after operator reset."); + } + + try { + this.windowDataManager.save(getPartitionsAndOffsets(), operatorId, windowId); + } catch (IOException e) { + throw new RuntimeException(e); + } catch (InterruptedException e) { + e.printStackTrace(); + } catch (ExecutionException e) { + e.printStackTrace(); + } + } + + 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(appName))) { + return true; + } + + return false; + } + + private boolean alreadyInKafka(T message) + { + if ( windowId <= windowDataManager.getLargestRecoveryWindow() ) { + return true; + } + + if (partialWindowTuples.containsKey(message)) { + + Integer val = partialWindowTuples.get(message); + + if ( val == 0 ) { + return false; + } else if ( val == 1 ) { + partialWindowTuples.remove(message); + } else { + partialWindowTuples.put(message, val - 1); + } + + return true; + } + + return false; + } + + private Map getPartitionsAndOffsets() throws ExecutionException, InterruptedException + { + List partitionInfoList = consumer.partitionsFor(getTopic()); + List topicPartitionList = new java.util.ArrayList<>(); + + for ( PartitionInfo partitionInfo: partitionInfoList) { + topicPartitionList.add(new TopicPartition(getTopic(), partitionInfo.partition()) ); + } + + Map parttionsAndOffset = new HashMap<>(); + consumer.assign(topicPartitionList); + + for ( PartitionInfo partitionInfo: partitionInfoList) { + + try { + + TopicPartition topicPartition = new TopicPartition(getTopic(), partitionInfo.partition()); + consumer.seekToEnd(topicPartition); + parttionsAndOffset.put(partitionInfo.partition(), consumer.position(topicPartition)); + + } catch (Exception ex) { + throw new RuntimeException(ex); + } + } + + return parttionsAndOffset; + } + + private void rebuildPartialWindow() + { + logger.info("Rebuild the partial window after " + windowDataManager.getLargestRecoveryWindow()); + + Map storedOffsets = getStoredOffsets(); + Map currentOffsets = getCurrentOffsets(); --- End diff -- Same problem here > Kafka Output Operator with Kafka 0.9 API > ---------------------------------------- > > Key: APEXMALHAR-2086 > URL: https://issues.apache.org/jira/browse/APEXMALHAR-2086 > Project: Apache Apex Malhar > Issue Type: New Feature > Reporter: Sandesh > Assignee: Sandesh > > Goal : 2 Operartors for Kafka Output > 1. Simple Kafka Output Operator > - Supports Atleast Once > - Expose most used producer properties as class properties > 2. Exactly Once Kafka Output ( Not possible in all the cases, will be documented later ) > > Design for Exactly Once > Window Data Manager - Stores the Kafka partitions offsets. > Kafka Key - Used by the operator = AppID#OperatorId > During recovery. Partially written window is re-created using the following approach: > Tuples between the largest recovery offsets and the current offset are checked. Based on the key, tuples written by the other entities are discarded. > Only tuples which are not in the recovered set are emitted. > Tuples needs to be unique within the window. > -- This message was sent by Atlassian JIRA (v6.3.4#6332)