[ 
https://issues.apache.org/jira/browse/GOBBLIN-758?focusedWorklogId=237128&page=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-237128
 ]

ASF GitHub Bot logged work on GOBBLIN-758:
------------------------------------------

                Author: ASF GitHub Bot
            Created on: 04/May/19 00:29
            Start Date: 04/May/19 00:29
    Worklog Time Spent: 10m 
      Work Description: zxcware commented on pull request #2622: [GOBBLIN-758] 
Added new reporters to emit MetricReport and GobblinTrackingEvent without 
serializing them. Also added random key generator for reporters.
URL: https://github.com/apache/incubator-gobblin/pull/2622#discussion_r280960930
 
 

 ##########
 File path: 
gobblin-modules/gobblin-kafka-common/src/main/java/org/apache/gobblin/metrics/kafka/KeyValueEventObjectReporter.java
 ##########
 @@ -0,0 +1,159 @@
+/*
+ * 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.gobblin.metrics.kafka;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Queue;
+import java.util.Random;
+import java.util.StringJoiner;
+
+import org.apache.avro.generic.GenericRecord;
+import org.apache.commons.lang3.tuple.Pair;
+
+import com.google.common.base.Optional;
+import com.google.common.base.Splitter;
+import com.google.common.collect.Lists;
+import com.typesafe.config.Config;
+
+import lombok.extern.slf4j.Slf4j;
+
+import org.apache.gobblin.configuration.ConfigurationKeys;
+import org.apache.gobblin.metrics.GobblinTrackingEvent;
+import org.apache.gobblin.metrics.MetricContext;
+import org.apache.gobblin.metrics.reporter.EventReporter;
+import org.apache.gobblin.util.AvroUtils;
+import org.apache.gobblin.util.ConfigUtils;
+
+
+/**
+ * This is a raw event (GobblinTrackingEvent) key value reporter that reports 
events as GenericRecords without serialization
+ * Configuration for this reporter start with the prefix 
"metrics.reporting.events"
+ */
+@Slf4j
+public class KeyValueEventObjectReporter extends EventReporter {
+  private static final String PUSHER_CONFIG = "pusherConfig";
+  private static final String PUSHER_CLASS = "pusherClass";
+  private static final String PUSHER_KEYS = "pusherKeys";
+
+  protected List<String> keys;
+  protected final String randomKey;
+  protected KeyValuePusher pusher;
+  protected Optional<Map<String, String>> namespaceOverride;
+  protected final String topic;
+
+  public KeyValueEventObjectReporter(Builder builder) {
+    super(builder);
+
+    this.topic = builder.topic;
+    this.namespaceOverride = builder.namespaceOverride;
+
+    Config config = builder.config.get();
+    Config pusherConfig = ConfigUtils.getConfigOrEmpty(config, 
PUSHER_CONFIG).withFallback(config);
+    String pusherClassName =
+        ConfigUtils.getString(config, PUSHER_CLASS, 
PusherUtils.DEFAULT_KEY_VALUE_PUSHER_CLASS_NAME);
+    this.pusher =
+        PusherUtils.getKeyValuePusher(pusherClassName, builder.brokers, 
builder.topic, Optional.of(pusherConfig));
+    this.closer.register(this.pusher);
+
+    randomKey = String.valueOf(new Random().nextInt(
+        ConfigUtils.getInt(config, ConfigurationKeys.KEY_SIZE_KEY, 
ConfigurationKeys.DEFAULT_REPORTER_KEY_SIZE)));
+    if (config.hasPath(PUSHER_KEYS)) {
+      List<String> keys = 
Splitter.on(",").omitEmptyStrings().trimResults().splitToList(config.getString(PUSHER_KEYS));
+      this.keys = keys;
+    } else {
+      log.info(
+          "Key not assigned from config. Please set it with property {} Using 
randomly generated number {} as key ",
+          ConfigurationKeys.METRICS_REPORTING_EVENTS_PUSHERKEYS, randomKey);
+    }
+  }
+
+  @Override
+  public void reportEventQueue(Queue<GobblinTrackingEvent> queue) {
+    log.info("Emitting report using KeyValueEventObjectReporter");
+
+    List<Pair<String, GenericRecord>> events = Lists.newArrayList();
+    GobblinTrackingEvent event;
+
+    while (null != (event = queue.poll())) {
+      GenericRecord record = AvroUtils.overrideNameAndNamespace(event, 
this.topic, this.namespaceOverride);
+      events.add(Pair.of(buildKey(record), record));
+    }
+
+    if (!events.isEmpty()) {
+      this.pusher.pushKeyValueMessages(events);
+    }
+  }
+
+  private String buildKey(GenericRecord record) {
+
+    String key = randomKey;
+    if (this.keys != null && this.keys.size() > 0) {
+      StringJoiner joiner = new StringJoiner(ConfigurationKeys.KEY_DELIMITER);
 
 Review comment:
   Same with above for `KEY_DELIMITER`
 
----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
[email protected]


Issue Time Tracking
-------------------

    Worklog Id:     (was: 237128)
    Time Spent: 1h 10m  (was: 1h)

> Use keys to push messages to kafka. Implemented Metric and Event reporters 
> that push messages without serialization.
> --------------------------------------------------------------------------------------------------------------------
>
>                 Key: GOBBLIN-758
>                 URL: https://issues.apache.org/jira/browse/GOBBLIN-758
>             Project: Apache Gobblin
>          Issue Type: Improvement
>            Reporter: Vikram Bohra
>            Priority: Major
>          Time Spent: 1h 10m
>  Remaining Estimate: 0h
>
> This ticket introduces the following changes.
> 1) It reduces the number of connections to Kafka by using a randomly 
> generated key for each instance of the reporter.
> 2) Introduces a KeyValuePusher interface that provides a cleaner way to push 
> key value messages to kafka.
> 3) Default implementation of the above interface for kafka. 
> 4) New Reporters for reporting GobblinTrackingEvent and MetricReport without 
> serializing them.
>  



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

Reply via email to