[GitHub] apex-malhar pull request #298: *For review only* [APEXMALHAR-2086] Kafka out...

2016-06-10 Thread devtagare
Github user devtagare commented on a diff in the pull request:

https://github.com/apache/apex-malhar/pull/298#discussion_r66696821
  
--- 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");
+  }
+
--- End diff --

can't we have leader only acknowledgement as the default . acks =1 ?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] apex-malhar pull request #298: *For review only* [APEXMALHAR-2086] Kafka out...

2016-06-10 Thread devtagare
Github user devtagare commented on a diff in the pull request:

https://github.com/apache/apex-malhar/pull/298#discussion_r66696282
  
--- 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();
--- End diff --

this.key = (new Integer(operatorId)) + partitionId (kafkaPartition) can be 
a more deterministic key.Since you want to peek into only the partitions to 
which this operator had written.Also APP_ID is a constant and does not add to 
arriving at uniqueness



---
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 infrastruct...@apache.org or 

[GitHub] apex-malhar pull request #298: *For review only* [APEXMALHAR-2086] Kafka out...

2016-06-10 Thread devtagare
Github user devtagare commented on a diff in the pull request:

https://github.com/apache/apex-malhar/pull/298#discussion_r66696137
  
--- 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
--- End diff --

Why unique records in a window ?


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] apex-malhar pull request #298: *For review only* [APEXMALHAR-2086] Kafka out...

2016-06-10 Thread chinmaykolhatkar
Github user chinmaykolhatkar commented on a diff in the pull request:

https://github.com/apache/apex-malhar/pull/298#discussion_r66694101
  
--- 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
--- End diff --

I see this class is extended by KafkaSinglePortExactlyOnceOutputOperator.. 
The subclass does following 2 things:
1. Add a port
2. call sendTuple method of parent class.

I see there is no need to have this one as abstract operator. This can 
become concreate and add the port here itself



---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] apex-malhar pull request #298: *For review only* [APEXMALHAR-2086] Kafka out...

2016-06-10 Thread chinmaykolhatkar
Github user chinmaykolhatkar commented on a diff in the pull request:

https://github.com/apache/apex-malhar/pull/298#discussion_r66693682
  
--- 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
--- End diff --

Can you remove @since? They'll get added during release time.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] apex-malhar pull request #298: *For review only* [APEXMALHAR-2086] Kafka out...

2016-06-08 Thread chinmaykolhatkar
Github user chinmaykolhatkar commented on a diff in the pull request:

https://github.com/apache/apex-malhar/pull/298#discussion_r66365356
  
--- Diff: 
kafka/src/main/java/org/apache/apex/malhar/kafka/AbstractExactlyOnceKafkaOutputOperator.java
 ---
@@ -0,0 +1,385 @@
+/**
+ * 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.HashSet;
+import java.util.List;
+import java.util.Map;
+import java.util.Properties;
+import java.util.Set;
+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;
+
+//   1. JavaDoc
+//   2. Unit Test
+//   3. Comparator -- not required
+//   4. Generic Type
+//   5. remove e.printStackTrace()
+//   6. Should the Consumer be kept open?
+
+/**
+ * This is a base implementation of a Kafka output operator,
+ * which, in most cases, guarantees to send tuples to Kafka MQ only 
once.
+ * Subclasses should implement the methods for converting tuples into a 
format appropriate for Kafka.
+ * 
+ * Assuming messages kept in kafka are ordered by either key or value or 
keyvalue pair
+ * (For example, use timestamps as key), this Kafka OutputOperator always 
retrieve the last message from MQ as initial offset.
+ *  So that replayed message wouldn't be sent to kafka again.
+ *
+ * This is not "perfect exact once" in 2 cases:
+ * 1 Multiple producers produce messages to same kafka partition
+ * 2 You have same message sent out and before kafka synchronized this 
message among all the brokers, the operator is
+ * started again.
+ *
+ * 
+ * Ports:
+ * Input: One input port
+ * Output: No output port
+ * 
+ * Properties:
+ * configProperties
+ * 
+ * Compile time checks:
+ * Class derived from has to implement 2 methods:
+ * tupleToKeyValue() to convert input tuples to kafka key value objects
+ * compareToLastMsg() to compare incoming tuple with the last received msg 
in kafka so that the operator could skip the received ones
+ * 
+ * Run time checks:
+ * None
+ * 
+ * Benchmarks:
+ * TBD
+ * 
+ *
+ * @displayName Abstract Exactly Once Kafka Output(0.9.0)
+ * @category Messaging
+ * @tags output operator
+ *
+ * @since 3.5
+ */
+public abstract class AbstractExactlyOnceKafkaOutputOperator extends 
AbstractKafkaOutputOperator
+implements Operator.CheckpointNotificationListener
+{
+  private WindowDataManager windowDataManager = new FSWindowDataManager();
+  private String key;
+  private Integer operatorId;
+  private String appId;
+  private transient Long windowId;
+  private transient Set recoveredTuples = new HashSet<>();
+  private int KAFKA_CONNECT_ATTEMPT = 10;
+
+  @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 + '#' + (new Integer(operatorId));
+  }
+
+  @Override
+  public void beginWindow(long windowId)
+  {
+this.windowId = 

[GitHub] apex-malhar pull request #298: *For review only* [APEXMALHAR-2086] Kafka out...

2016-06-07 Thread chinmaykolhatkar
Github user chinmaykolhatkar commented on a diff in the pull request:

https://github.com/apache/apex-malhar/pull/298#discussion_r66131514
  
--- 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 implements Operator
+{
+  @SuppressWarnings("unused")
+  private static final Logger logger = 
LoggerFactory.getLogger(AbstractKafkaOutputOperator.class);
+  private transient Producer 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()
+  {
+return configProperties;
+  }
+
+  public void setConfigProperties(Properties configProperties)
+  {
+this.configProperties.putAll(configProperties);
+  }
+
+  /**
+   * setup producer configuration.
+   * @return ProducerConfig
+   */
+  protected Properties createKafkaProducerConfig()
+  {
+Properties prop = new Properties();
+for (String propString : producerProperties.split(",")) {
+  if (!propString.contains("=")) {
+continue;
+  }
+  String[] keyVal = StringUtils.trim(propString).split("=");
+  prop.put(StringUtils.trim(keyVal[0]), StringUtils.trim(keyVal[1]));
+}
+
+configProperties.putAll(prop);
+
+return configProperties;
+  }
+
+  public Producer getProducer()
--- End diff --

Any reason why this is made public? Its being accessed from subclass.. So 
can be made protected.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] apex-malhar pull request #298: *For review only* [APEXMALHAR-2086] Kafka out...

2016-06-07 Thread chinmaykolhatkar
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 implements Operator
+{
+  @SuppressWarnings("unused")
+  private static final Logger logger = 
LoggerFactory.getLogger(AbstractKafkaOutputOperator.class);
+  private transient Producer 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.


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---


[GitHub] apex-malhar pull request #298: *For review only* [APEXMALHAR-2086] Kafka out...

2016-06-07 Thread chinmaykolhatkar
Github user chinmaykolhatkar commented on a diff in the pull request:

https://github.com/apache/apex-malhar/pull/298#discussion_r66130894
  
--- 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 implements Operator
+{
+  @SuppressWarnings("unused")
+  private static final Logger logger = 
LoggerFactory.getLogger(AbstractKafkaOutputOperator.class);
+  private transient Producer producer;  // K is key partitioner, V 
is value type
+  @NotNull
+  private String topic = "topic1";
--- End diff --

Why is default value of "topic" property set to "topic1"... That 
invalidates the use of "@NotNull"...


---
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 infrastruct...@apache.org or file a JIRA ticket
with INFRA.
---