You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@druid.apache.org by "abhishekagarwal87 (via GitHub)" <gi...@apache.org> on 2023/05/31 10:35:03 UTC

[GitHub] [druid] abhishekagarwal87 commented on a diff in pull request #14281: Adding SegmentMetadataEvent and publishing them via KafkaEmitter

abhishekagarwal87 commented on code in PR #14281:
URL: https://github.com/apache/druid/pull/14281#discussion_r1211482293


##########
extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/KafkaEmitter.java:
##########
@@ -119,18 +126,26 @@ protected Producer<String, String> setKafkaProducer()
   @Override
   public void start()
   {
-    scheduler.schedule(this::sendMetricToKafka, sendInterval, TimeUnit.SECONDS);
-    scheduler.schedule(this::sendAlertToKafka, sendInterval, TimeUnit.SECONDS);
-    if (config.getRequestTopic() != null) {
+    Set<EventType> eventTypes = config.getEventTypes();
+    if (eventTypes.contains(EventType.METRICS)) {
+      scheduler.schedule(this::sendMetricToKafka, sendInterval, TimeUnit.SECONDS);
+    }
+    if (eventTypes.contains(EventType.ALERTS)) {
+      scheduler.schedule(this::sendAlertToKafka, sendInterval, TimeUnit.SECONDS);
+    }
+    if (eventTypes.contains(EventType.REQUESTS)) {
       scheduler.schedule(this::sendRequestToKafka, sendInterval, TimeUnit.SECONDS);
     }
+    if (eventTypes.contains(EventType.SEGMENT_METADATA)) {
+      scheduler.schedule(this::sendSegmentMetadataToKafka, sendInterval, TimeUnit.SECONDS);
+    }
     scheduler.scheduleWithFixedDelay(() -> {
-      log.info(
-          "Message lost counter: metricLost=[%d], alertLost=[%d], requestLost=[%d], invalidLost=[%d]",
+      log.info("Message lost counter: metricLost=[%d], alertLost=[%d], requestLost=[%d], invalidLost=[%d] segmentMetadataLost=[%d]",

Review Comment:
   nit. lets have invalidLost at the end. 



##########
extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/KafkaEmitter.java:
##########
@@ -183,24 +203,31 @@ public void emit(final Event event)
             resultJson,
             StringUtils.toUtf8(resultJson).length
         );
+
+        Set<EventType> eventTypes = config.getEventTypes();
         if (event instanceof ServiceMetricEvent) {
-          if (!metricQueue.offer(objectContainer)) {
+          if (!eventTypes.contains(EventType.METRICS) || !metricQueue.offer(objectContainer)) {

Review Comment:
   why do we want to increment metricsLost in this case? Since not reporting is intentional. 



##########
processing/src/main/java/org/apache/druid/java/util/emitter/service/SegmentMetadataEvent.java:
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.druid.java.util.emitter.service;
+
+import com.fasterxml.jackson.annotation.JsonValue;
+import org.apache.druid.java.util.emitter.core.Event;
+import org.apache.druid.java.util.emitter.core.EventMap;
+import org.joda.time.DateTime;
+
+public class SegmentMetadataEvent implements Event

Review Comment:
   please add javadocs about the class. 



##########
extensions-contrib/kafka-emitter/src/main/java/org/apache/druid/emitter/kafka/KafkaEmitterConfig.java:
##########
@@ -21,53 +21,108 @@
 
 import com.fasterxml.jackson.annotation.JsonCreator;
 import com.fasterxml.jackson.annotation.JsonProperty;
+import com.fasterxml.jackson.annotation.JsonValue;
 import com.google.common.base.Preconditions;
 import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import org.apache.druid.java.util.common.StringUtils;
 import org.apache.kafka.clients.producer.ProducerConfig;
 
 import javax.annotation.Nullable;
+import java.util.HashSet;
 import java.util.Map;
+import java.util.Set;
 
 public class KafkaEmitterConfig
 {
+  public enum EventType
+  {
+    METRICS,
+    ALERTS,
+    REQUESTS,
+    SEGMENT_METADATA;
+
+    @JsonValue
+    @Override
+    public String toString()
+    {
+      return StringUtils.toLowerCase(this.name());
+    }
 
+    @JsonCreator
+    public static EventType fromString(String name)
+    {
+      return valueOf(StringUtils.toUpperCase(name));
+    }
+  }
+
+  public static final Set<EventType> DEFAULT_EVENT_TYPES = ImmutableSet.of(EventType.ALERTS, EventType.METRICS);
   @JsonProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG)
   private final String bootstrapServers;
-  @JsonProperty("metric.topic")
+  @Nullable @JsonProperty("event.types")
+  private final Set<EventType> eventTypes;
+  @Nullable @JsonProperty("metric.topic")
   private final String metricTopic;
-  @JsonProperty("alert.topic")
+  @Nullable @JsonProperty("alert.topic")
   private final String alertTopic;
   @Nullable @JsonProperty("request.topic")
   private final String requestTopic;
+  @Nullable @JsonProperty("segmentMetadata.topic")
+  private final String segmentMetadataTopic;
   @JsonProperty
   private final String clusterName;
   @JsonProperty("producer.config")
-  private Map<String, String> kafkaProducerConfig;
+  private final Map<String, String> kafkaProducerConfig;
 
   @JsonCreator
   public KafkaEmitterConfig(
       @JsonProperty(ProducerConfig.BOOTSTRAP_SERVERS_CONFIG) String bootstrapServers,
-      @JsonProperty("metric.topic") String metricTopic,
-      @JsonProperty("alert.topic") String alertTopic,
+      @Nullable @JsonProperty("event.types") Set<EventType> eventTypes,
+      @Nullable @JsonProperty("metric.topic") String metricTopic,
+      @Nullable @JsonProperty("alert.topic") String alertTopic,
       @Nullable @JsonProperty("request.topic") String requestTopic,
+      @Nullable @JsonProperty("segmentMetadata.topic") String segmentMetadataTopic,
       @JsonProperty("clusterName") String clusterName,
       @JsonProperty("producer.config") @Nullable Map<String, String> kafkaProducerConfig
   )
   {
     this.bootstrapServers = Preconditions.checkNotNull(bootstrapServers, "bootstrap.servers can not be null");
-    this.metricTopic = Preconditions.checkNotNull(metricTopic, "metric.topic can not be null");
-    this.alertTopic = Preconditions.checkNotNull(alertTopic, "alert.topic can not be null");
-    this.requestTopic = requestTopic;
+    this.eventTypes = maybeUpdateEventTypes(eventTypes, requestTopic);
+    this.metricTopic = this.eventTypes.contains(EventType.METRICS) ? Preconditions.checkNotNull(metricTopic, "metric.topic can not be null") : null;

Review Comment:
   unrelated to your PR but can the error message include the full config name. E.g. `druid.emitter.kafka.metric.topic can not be null` 
   



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

To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@druid.apache.org
For additional commands, e-mail: commits-help@druid.apache.org