[ 
https://issues.apache.org/jira/browse/APEXMALHAR-2086?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=15319117#comment-15319117
 ] 

ASF GitHub Bot commented on APEXMALHAR-2086:
--------------------------------------------

Github user chinmaykolhatkar commented on a diff in the pull request:

    https://github.com/apache/apex-malhar/pull/298#discussion_r66131266
  
    --- Diff: 
kafka/src/main/java/org/apache/apex/malhar/kafka/AbstractKafkaOutputOperator.java
 ---
    @@ -0,0 +1,141 @@
    +/**
    + * 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.Properties;
    +import javax.validation.constraints.NotNull;
    +import org.slf4j.Logger;
    +import org.slf4j.LoggerFactory;
    +import org.apache.commons.lang3.StringUtils;
    +import org.apache.kafka.clients.producer.KafkaProducer;
    +import org.apache.kafka.clients.producer.Producer;
    +import com.datatorrent.api.Context;
    +import com.datatorrent.api.Operator;
    +
    +public abstract class AbstractKafkaOutputOperator<K, V> implements Operator
    +{
    +  @SuppressWarnings("unused")
    +  private static final Logger logger = 
LoggerFactory.getLogger(AbstractKafkaOutputOperator.class);
    +  private transient Producer<K, V> producer;  // K is key partitioner, V 
is value type
    +  @NotNull
    +  private String topic = "topic1";
    +  private String producerProperties = "";
    +  private Properties configProperties = new Properties();
    +  public Properties getConfigProperties()
    --- End diff --
    
    We don't have a constraint like this, but would be good if you could 
organize the property access method at one place.


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

Reply via email to