Refactored Kafka Application to simplify dependencies

1. Fixed a Bug in KafkaProducer. Without this fix the App will not send data in GPB format.
2. Added two new services - KafkaProducerService and KafkaConfigService.
3. Fixed a TODO in the register API to return Kafka server information.
4. Removed the use of LeadershipService and ClusterService, since we are not ready for clustering yet.

Change-Id: If20ef5238bb4629af0c6769129494eb44abf1d3c
diff --git a/apps/kafka-integration/core/src/main/java/org/onosproject/kafkaintegration/kafka/EventMonitor.java b/apps/kafka-integration/core/src/main/java/org/onosproject/kafkaintegration/kafka/EventMonitor.java
index 6e7291b..4bc0450 100644
--- a/apps/kafka-integration/core/src/main/java/org/onosproject/kafkaintegration/kafka/EventMonitor.java
+++ b/apps/kafka-integration/core/src/main/java/org/onosproject/kafkaintegration/kafka/EventMonitor.java
@@ -19,36 +19,32 @@
 import org.apache.felix.scr.annotations.Activate;
 import org.apache.felix.scr.annotations.Component;
 import org.apache.felix.scr.annotations.Deactivate;
-import org.apache.felix.scr.annotations.Modified;
-import org.apache.felix.scr.annotations.Property;
 import org.apache.felix.scr.annotations.Reference;
 import org.apache.felix.scr.annotations.ReferenceCardinality;
 import org.apache.kafka.clients.producer.ProducerRecord;
-import org.onosproject.cfg.ComponentConfigService;
 import org.onosproject.kafkaintegration.api.EventConversionService;
 import org.onosproject.kafkaintegration.api.EventSubscriptionService;
+import org.onosproject.kafkaintegration.api.KafkaProducerService;
+import org.onosproject.kafkaintegration.api.KafkaConfigService;
+import org.onosproject.kafkaintegration.api.dto.OnosEvent;
 import org.onosproject.net.device.DeviceEvent;
 import org.onosproject.net.device.DeviceListener;
 import org.onosproject.net.device.DeviceService;
 import org.onosproject.net.link.LinkEvent;
 import org.onosproject.net.link.LinkListener;
 import org.onosproject.net.link.LinkService;
-import org.osgi.service.component.ComponentContext;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
 
-import java.util.Dictionary;
-import java.util.UUID;
 import java.util.concurrent.ExecutionException;
 import java.util.concurrent.ExecutorService;
 
-import static com.google.common.base.Strings.isNullOrEmpty;
 import static java.util.concurrent.Executors.newSingleThreadScheduledExecutor;
-import static org.onlab.util.Tools.get;
 import static org.onlab.util.Tools.groupedThreads;
 import static org.onosproject.kafkaintegration.api.dto.OnosEvent.Type.DEVICE;
 import static org.onosproject.kafkaintegration.api.dto.OnosEvent.Type.LINK;
 
+
 /**
  * Encapsulates the behavior of monitoring various ONOS events.
  * */
@@ -63,152 +59,66 @@
     protected EventConversionService eventConversionService;
 
     @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected KafkaProducerService kafkaProducer;
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
     protected DeviceService deviceService;
 
     @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
     protected LinkService linkService;
 
     @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
-    protected ComponentConfigService componentConfigService;
+    protected KafkaConfigService kafkaConfigService;
 
     private final DeviceListener deviceListener = new InternalDeviceListener();
     private final LinkListener linkListener = new InternalLinkListener();
 
     protected ExecutorService eventExecutor;
 
-    private static final String BOOTSTRAP_SERVERS = "localhost:9092";
-    private static final int RETRIES = 1;
-    private static final int MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION = 5;
-    private static final int REQUEST_REQUIRED_ACKS = 1;
-    private static final String KEY_SERIALIZER = "org.apache.kafka.common.serialization.StringSerializer";
-    private static final String VALUE_SERIALIZER = "org.apache.kafka.common.serialization.ByteArraySerializer";
-
-    @Property(name = "bootstrap.servers", value = BOOTSTRAP_SERVERS,
-            label = "Default host/post pair to establish initial connection to Kafka cluster.")
-    private String bootstrapServers = BOOTSTRAP_SERVERS;
-
-    @Property(name = "retries", intValue = RETRIES,
-            label = "Number of times the producer can retry to send after first failure")
-    private int retries = RETRIES;
-
-    @Property(name = "max.in.flight.requests.per.connection", intValue = MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION,
-            label = "The maximum number of unacknowledged requests the client will send before blocking")
-    private int maxInFlightRequestsPerConnection = MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION;
-
-    @Property(name = "request.required.acks", intValue = 1,
-            label = "Producer will get an acknowledgement after the leader has replicated the data")
-    private int requestRequiredAcks = REQUEST_REQUIRED_ACKS;
-
-    @Property(name = "key.serializer", value = KEY_SERIALIZER,
-            label = "Serializer class for key that implements the Serializer interface.")
-    private String keySerializer = KEY_SERIALIZER;
-
-    @Property(name = "value.serializer", value = VALUE_SERIALIZER,
-            label = "Serializer class for value that implements the Serializer interface.")
-    private String valueSerializer = VALUE_SERIALIZER;
-
-    private Producer producer;
-
     @Activate
-    protected void activate(ComponentContext context) {
-        componentConfigService.registerProperties(getClass());
+    protected void activate() {
+
         eventExecutor = newSingleThreadScheduledExecutor(groupedThreads("onos/onosEvents", "events-%d", log));
         deviceService.addListener(deviceListener);
         linkService.addListener(linkListener);
-        producer = new Producer(bootstrapServers, retries, maxInFlightRequestsPerConnection,
-                                requestRequiredAcks, keySerializer, valueSerializer);
-        producer.start();
+
+        kafkaProducer.start(kafkaConfigService.getConfigParams());
 
         log.info("Started");
     }
 
     @Deactivate
     protected void deactivate() {
-        componentConfigService.unregisterProperties(getClass(), false);
         deviceService.removeListener(deviceListener);
         linkService.removeListener(linkListener);
-        producer.stop();
+
         eventExecutor.shutdownNow();
         eventExecutor = null;
 
+        kafkaProducer.stop();
+
         log.info("Stopped");
     }
 
-    @Modified
-    private void modified(ComponentContext context) {
-        if (context == null) {
-            bootstrapServers = BOOTSTRAP_SERVERS;
-            retries = RETRIES;
-            maxInFlightRequestsPerConnection = MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION;
-            requestRequiredAcks = REQUEST_REQUIRED_ACKS;
-            keySerializer = KEY_SERIALIZER;
-            valueSerializer = VALUE_SERIALIZER;
-            return;
-        }
-        Dictionary properties = context.getProperties();
-
-        String newBootstrapServers = BOOTSTRAP_SERVERS;
-        int newRetries = RETRIES;
-        int newMaxInFlightRequestsPerConnection = MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION;
-        int newRequestRequiredAcks = REQUEST_REQUIRED_ACKS;
-        try {
-            String s = get(properties, "bootstrapServers");
-            newBootstrapServers = isNullOrEmpty(s)
-                    ? bootstrapServers : s.trim();
-
-            s = get(properties, "retries");
-            newRetries = isNullOrEmpty(s)
-                    ? retries : Integer.parseInt(s.trim());
-
-            s = get(properties, "maxInFlightRequestsPerConnection");
-            newMaxInFlightRequestsPerConnection = isNullOrEmpty(s)
-                    ? maxInFlightRequestsPerConnection : Integer.parseInt(s.trim());
-
-            s = get(properties, "requestRequiredAcks");
-            newRequestRequiredAcks = isNullOrEmpty(s)
-                    ? requestRequiredAcks : Integer.parseInt(s.trim());
-
-        } catch (NumberFormatException | ClassCastException e) {
-            return;
-        }
-
-        boolean modified = newBootstrapServers != bootstrapServers ||
-                newRetries != retries ||
-                newMaxInFlightRequestsPerConnection != maxInFlightRequestsPerConnection ||
-                newRequestRequiredAcks != requestRequiredAcks;
-
-        if (modified) {
-            bootstrapServers = newBootstrapServers;
-            retries = newRetries;
-            maxInFlightRequestsPerConnection = newMaxInFlightRequestsPerConnection;
-            requestRequiredAcks = newRequestRequiredAcks;
-            if (producer != null) {
-                producer.stop();
-            }
-            producer = new Producer(bootstrapServers, retries, maxInFlightRequestsPerConnection,
-                                        requestRequiredAcks, keySerializer, valueSerializer);
-            producer.start();
-            log.info("Modified");
-        } else {
-            return;
-        }
-    }
-
     private class InternalDeviceListener implements DeviceListener {
 
         @Override
         public void event(DeviceEvent event) {
+
             if (!eventSubscriptionService.getEventSubscribers(DEVICE).isEmpty()) {
+                OnosEvent onosEvent = eventConversionService.convertEvent(event);
                 eventExecutor.execute(() -> {
                     try {
-                        String id = UUID.randomUUID().toString();
-                        producer.send(new ProducerRecord<>(DEVICE.toString(),
-                                                                id, event.subject().toString().getBytes())).get();
-                        log.debug("Device event sent successfully.");
-                    } catch (InterruptedException e) {
+                        kafkaProducer.send(new ProducerRecord<>(DEVICE.toString(),
+                                                               onosEvent.subject().toByteArray())).get();
+
+                        log.debug("Event Type - {}, Subject {} sent successfully.",
+                                  DEVICE, onosEvent.subject());
+
+                    } catch (InterruptedException e1) {
                         Thread.currentThread().interrupt();
-                    } catch (ExecutionException e) {
-                        log.error("Exception thrown {}", e);
+                    } catch (ExecutionException e2) {
+                        log.error("Exception thrown {}", e2);
                     }
                 });
             } else {
@@ -221,17 +131,21 @@
 
         @Override
         public void event(LinkEvent event) {
+
             if (!eventSubscriptionService.getEventSubscribers(LINK).isEmpty()) {
+                OnosEvent onosEvent = eventConversionService.convertEvent(event);
                 eventExecutor.execute(() -> {
                     try {
-                        String id = UUID.randomUUID().toString();
-                        producer.send(new ProducerRecord<>(LINK.toString(),
-                                                                id, event.subject().toString().getBytes())).get();
-                        log.debug("Link event sent successfully.");
-                    } catch (InterruptedException e) {
+                        kafkaProducer.send(new ProducerRecord<>(LINK.toString(),
+                                onosEvent.subject().toByteArray())).get();
+
+                        log.debug("Event Type - {}, Subject {} sent successfully.",
+                              LINK, onosEvent.subject());
+
+                    } catch (InterruptedException e1) {
                         Thread.currentThread().interrupt();
-                    } catch (ExecutionException e) {
-                        log.error("Exception thrown {}", e);
+                    } catch (ExecutionException e2) {
+                        log.error("Exception thrown {}", e2);
                     }
                 });
             } else {
diff --git a/apps/kafka-integration/core/src/main/java/org/onosproject/kafkaintegration/kafka/KafkaConfigManager.java b/apps/kafka-integration/core/src/main/java/org/onosproject/kafkaintegration/kafka/KafkaConfigManager.java
new file mode 100644
index 0000000..35a2207
--- /dev/null
+++ b/apps/kafka-integration/core/src/main/java/org/onosproject/kafkaintegration/kafka/KafkaConfigManager.java
@@ -0,0 +1,208 @@
+/**
+ * Copyright 2016-present Open Networking Laboratory
+ *
+ * Licensed 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.onosproject.kafkaintegration.kafka;
+
+import static com.google.common.base.Strings.isNullOrEmpty;
+import static org.onlab.util.Tools.get;
+
+import java.util.Dictionary;
+
+import org.apache.felix.scr.annotations.Activate;
+import org.apache.felix.scr.annotations.Component;
+import org.apache.felix.scr.annotations.Deactivate;
+import org.apache.felix.scr.annotations.Modified;
+import org.apache.felix.scr.annotations.Property;
+import org.apache.felix.scr.annotations.Reference;
+import org.apache.felix.scr.annotations.ReferenceCardinality;
+import org.apache.felix.scr.annotations.Service;
+import org.onosproject.cfg.ComponentConfigService;
+import org.onosproject.kafkaintegration.api.KafkaConfigService;
+import org.onosproject.kafkaintegration.api.KafkaProducerService;
+import org.onosproject.kafkaintegration.api.dto.KafkaServerConfig;
+import org.osgi.service.component.ComponentContext;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+@Component(immediate = true)
+@Service
+public class KafkaConfigManager implements KafkaConfigService {
+
+    private final Logger log = LoggerFactory.getLogger(getClass());
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected ComponentConfigService componentConfigService;
+
+    @Reference(cardinality = ReferenceCardinality.MANDATORY_UNARY)
+    protected KafkaProducerService producer;
+
+    public static final String BOOTSTRAP_SERVERS = "localhost:9092";
+    private String kafkaServerIp =
+            BOOTSTRAP_SERVERS.substring(0, BOOTSTRAP_SERVERS.indexOf(":"));
+    private String kafkaServerPortNum =
+            BOOTSTRAP_SERVERS.substring(BOOTSTRAP_SERVERS.indexOf(":") + 1,
+                                        BOOTSTRAP_SERVERS.length());
+
+    private static final int RETRIES = 1;
+    private static final int MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION = 5;
+    private static final int REQUEST_REQUIRED_ACKS = 1;
+    private static final String KEY_SERIALIZER =
+            "org.apache.kafka.common.serialization.StringSerializer";
+    private static final String VALUE_SERIALIZER =
+            "org.apache.kafka.common.serialization.ByteArraySerializer";
+
+    @Property(name = "bootstrap.servers", value = BOOTSTRAP_SERVERS,
+            label = "Default IP/Port pair to establish initial connection to Kafka cluster.")
+    protected String bootstrapServers = BOOTSTRAP_SERVERS;
+
+    @Property(name = "retries", intValue = RETRIES,
+            label = "Number of times the producer can retry to send after first failure")
+    protected int retries = RETRIES;
+
+    @Property(name = "max.in.flight.requests.per.connection",
+            intValue = MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION,
+            label = "The maximum number of unacknowledged requests the client will send before blocking")
+    protected int maxInFlightRequestsPerConnection =
+            MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION;
+
+    @Property(name = "request.required.acks", intValue = 1,
+            label = "Producer will get an acknowledgement after the leader has replicated the data")
+    protected int requestRequiredAcks = REQUEST_REQUIRED_ACKS;
+
+    @Property(name = "key.serializer", value = KEY_SERIALIZER,
+            label = "Serializer class for key that implements the Serializer interface.")
+    protected String keySerializer = KEY_SERIALIZER;
+
+    @Property(name = "value.serializer", value = VALUE_SERIALIZER,
+            label = "Serializer class for value that implements the Serializer interface.")
+    protected String valueSerializer = VALUE_SERIALIZER;
+
+    @Activate
+    protected void activate(ComponentContext context) {
+        componentConfigService.registerProperties(getClass());
+        log.info("Started");
+    }
+
+    @Deactivate
+    protected void deactivate() {
+        componentConfigService.unregisterProperties(getClass(), false);
+        log.info("Stopped");
+    }
+
+    @Modified
+    private void modified(ComponentContext context) {
+        if (context == null) {
+            bootstrapServers = BOOTSTRAP_SERVERS;
+            retries = RETRIES;
+            maxInFlightRequestsPerConnection =
+                    MAX_IN_FLIGHT_REQUESTS_PER_CONNECTION;
+            requestRequiredAcks = REQUEST_REQUIRED_ACKS;
+            keySerializer = KEY_SERIALIZER;
+            valueSerializer = VALUE_SERIALIZER;
+            return;
+        }
+
+        Dictionary<?, ?> properties = context.getProperties();
+
+        String newBootstrapServers;
+        int newRetries;
+        int newMaxInFlightRequestsPerConnection;
+        int newRequestRequiredAcks;
+        try {
+            String s = get(properties, "bootstrap.servers");
+            newBootstrapServers =
+                    isNullOrEmpty(s) ? bootstrapServers : s.trim();
+
+            s = get(properties, "retries");
+            newRetries =
+                    isNullOrEmpty(s) ? retries : Integer.parseInt(s.trim());
+
+            s = get(properties, "max.in.flight.requests.per.connection");
+            newMaxInFlightRequestsPerConnection =
+                    isNullOrEmpty(s) ? maxInFlightRequestsPerConnection
+                                     : Integer.parseInt(s.trim());
+
+            s = get(properties, "request.required.acks");
+            newRequestRequiredAcks =
+                    isNullOrEmpty(s) ? requestRequiredAcks
+                                     : Integer.parseInt(s.trim());
+
+        } catch (NumberFormatException | ClassCastException e) {
+            return;
+        }
+
+        if (configModified(newBootstrapServers, newRetries,
+                           newMaxInFlightRequestsPerConnection,
+                           newRequestRequiredAcks)) {
+            bootstrapServers = newBootstrapServers;
+            kafkaServerIp = bootstrapServers
+                    .substring(0, bootstrapServers.indexOf(":"));
+            kafkaServerPortNum = bootstrapServers
+                    .substring(bootstrapServers.indexOf(":") + 1,
+                               bootstrapServers.length());
+
+            retries = newRetries;
+
+            maxInFlightRequestsPerConnection =
+                    newMaxInFlightRequestsPerConnection;
+
+            requestRequiredAcks = newRequestRequiredAcks;
+
+            producer.restart(KafkaServerConfig.builder()
+                    .ipAddress(kafkaServerIp).port(kafkaServerPortNum)
+                    .numOfRetries(retries)
+                    .maxInFlightRequestsPerConnection(maxInFlightRequestsPerConnection)
+                    .acksRequired(requestRequiredAcks)
+                    .keySerializer(keySerializer)
+                    .valueSerializer(valueSerializer).build());
+
+            log.info("Kafka Server Config has been Modified - "
+                    + "bootstrapServers {}, retries {}, "
+                    + "maxInFlightRequestsPerConnection {}, "
+                    + "requestRequiredAcks {}", bootstrapServers, retries,
+                     maxInFlightRequestsPerConnection, requestRequiredAcks);
+        } else {
+            return;
+        }
+    }
+
+    private boolean configModified(String newBootstrapServers, int newRetries,
+                                   int newMaxInFlightRequestsPerConnection,
+                                   int newRequestRequiredAcks) {
+
+        return !newBootstrapServers.equals(bootstrapServers)
+                || newRetries != retries
+                || newMaxInFlightRequestsPerConnection != maxInFlightRequestsPerConnection
+                || newRequestRequiredAcks != requestRequiredAcks;
+
+    }
+
+    @Override
+    public KafkaServerConfig getConfigParams() {
+        String ipAddr =
+                bootstrapServers.substring(0, bootstrapServers.indexOf(":"));
+        String port =
+                bootstrapServers.substring(bootstrapServers.indexOf(":") + 1,
+                                           bootstrapServers.length());
+
+        return KafkaServerConfig.builder().ipAddress(ipAddr).port(port)
+                .numOfRetries(retries)
+                .maxInFlightRequestsPerConnection(maxInFlightRequestsPerConnection)
+                .acksRequired(requestRequiredAcks).keySerializer(keySerializer)
+                .valueSerializer(valueSerializer).build();
+
+    }
+
+}
diff --git a/apps/kafka-integration/core/src/main/java/org/onosproject/kafkaintegration/kafka/Producer.java b/apps/kafka-integration/core/src/main/java/org/onosproject/kafkaintegration/kafka/Producer.java
index bbdff81..4267d87 100644
--- a/apps/kafka-integration/core/src/main/java/org/onosproject/kafkaintegration/kafka/Producer.java
+++ b/apps/kafka-integration/core/src/main/java/org/onosproject/kafkaintegration/kafka/Producer.java
@@ -16,50 +16,84 @@
 
 package org.onosproject.kafkaintegration.kafka;
 
-import org.apache.kafka.clients.producer.KafkaProducer;
-import org.apache.kafka.clients.producer.ProducerRecord;
-import org.apache.kafka.clients.producer.RecordMetadata;
-import org.slf4j.Logger;
-import org.slf4j.LoggerFactory;
-
 import java.util.Properties;
 import java.util.concurrent.Future;
 
+import org.apache.felix.scr.annotations.Activate;
+import org.apache.felix.scr.annotations.Component;
+import org.apache.felix.scr.annotations.Deactivate;
+import org.apache.felix.scr.annotations.Service;
+import org.apache.kafka.clients.producer.KafkaProducer;
+import org.apache.kafka.clients.producer.ProducerRecord;
+import org.apache.kafka.clients.producer.RecordMetadata;
+import org.onosproject.kafkaintegration.api.KafkaProducerService;
+import org.onosproject.kafkaintegration.api.dto.KafkaServerConfig;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
 /**
- * Implementation of Kafka Producer.
+ * Implementation of a Kafka Producer.
  */
-public class Producer {
+@Component
+@Service
+public class Producer implements KafkaProducerService {
     private KafkaProducer<String, byte[]> kafkaProducer = null;
 
     private final Logger log = LoggerFactory.getLogger(getClass());
 
-    Producer(String bootstrapServers, int retries, int maxInFlightRequestsPerConnection,
-             int requestRequiredAcks, String keySerializer, String valueSerializer) {
-
-        Properties prop = new Properties();
-        prop.put("bootstrap.servers", bootstrapServers);
-        prop.put("retries", retries);
-        prop.put("max.in.flight.requests.per.connection", maxInFlightRequestsPerConnection);
-        prop.put("request.required.acks", requestRequiredAcks);
-        prop.put("key.serializer", keySerializer);
-        prop.put("value.serializer", valueSerializer);
-
-        kafkaProducer = new KafkaProducer<>(prop);
-    }
-
-    public void start() {
+    @Activate
+    protected void activate() {
         log.info("Started");
     }
 
+    @Deactivate
+    protected void deactivate() {
+        log.info("Stopped");
+    }
+
+    @Override
+    public void start(KafkaServerConfig config) {
+
+        if (kafkaProducer != null) {
+            log.info("Producer has already started");
+            return;
+        }
+
+        String bootstrapServer =
+                new StringBuilder().append(config.getIpAddress()).append(":")
+                        .append(config.getPort()).toString();
+
+        // Set Server Properties
+        Properties prop = new Properties();
+        prop.put("bootstrap.servers", bootstrapServer);
+        prop.put("retries", config.getNumOfRetries());
+        prop.put("max.in.flight.requests.per.connection",
+                 config.getMaxInFlightRequestsPerConnection());
+        prop.put("request.required.acks", config.getAcksRequired());
+        prop.put("key.serializer", config.getKeySerializer());
+        prop.put("value.serializer", config.getValueSerializer());
+
+        kafkaProducer = new KafkaProducer<>(prop);
+        log.info("Kafka Producer has started.");
+    }
+
+    @Override
     public void stop() {
         if (kafkaProducer != null) {
             kafkaProducer.close();
             kafkaProducer = null;
         }
 
-        log.info("Stopped");
+        log.info("Kafka Producer has Stopped");
     }
 
+    @Override
+    public void restart(KafkaServerConfig config) {
+        stop();
+        start(config);
+    }
+
+    @Override
     public Future<RecordMetadata> send(ProducerRecord<String, byte[]> record) {
         return kafkaProducer.send(record);
     }