Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -19,13 +19,11 @@

package org.apache.druid.java.util.emitter.core;

import java.util.Map;

/**
*/
public interface Event
{
Map<String, Object> toMap();
EventMap toMap();

String getFeed();
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,109 @@
/*
* 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.core;

import com.fasterxml.jackson.databind.annotation.JsonSerialize;

import java.util.HashMap;
import java.util.Map;

/**
* EventMap is a hash map implementation. It can be safely serialzed to JSON using Jackson serializer as it
* respects the polymorphic annotations on entires (unlike standard Map). The example of polymorphic class is a query
* interface, where different native query types are resolved by additional field called "queryType".
* This implementation ensures that the annotation on the values are respected during serialization.
*/
@JsonSerialize(using = EventMapSerializer.class)
public class EventMap extends HashMap<String, Object>
Comment thread
rohangarg marked this conversation as resolved.
{
/**
* Returns builder with Fluent API to build EventMap instance using method chaining
*/
public static Builder builder()
{
return new Builder();
}

/**
* Convert this EventMap to a builder. Performs copy of the whole EventMap.
*/
public Builder asBuilder()
{
return new Builder().putAll(this);
}

public static class Builder
{

private final EventMap map;

protected Builder()
{
map = new EventMap();
}

/**
* Adds key -> value pair to the map
*/
public Builder put(String key, Object value)
{
map.put(key, value);
return this;
}

/**
* Adds key -> value pair to the map only if value is not null
*/
public Builder putNonNull(String key, Object value)
{
if (value != null) {
map.put(key, value);
}
return this;
}

/**
* Adds map entry to the map
*/
public Builder put(Map.Entry<String, Object> entry)
{
map.put(entry.getKey(), entry.getValue());
return this;
}

/**
* Adds all key -> value pairs from other map
*/
public Builder putAll(Map<? extends String, ? extends Object> other)
{
map.putAll(other);
return this;
}

/**
* Builds and returns the EventMap
*/
public EventMap build()
{
return map;
}
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,40 @@
/*
* 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.core;

import com.fasterxml.jackson.core.JsonGenerator;
import com.fasterxml.jackson.databind.JsonSerializer;
import com.fasterxml.jackson.databind.SerializerProvider;

import java.io.IOException;
import java.util.Map;

public class EventMapSerializer extends JsonSerializer<EventMap>
{
@Override
public void serialize(EventMap map, JsonGenerator gen, SerializerProvider serializers) throws IOException
{
gen.writeStartObject();
for (Map.Entry<String, Object> entry : map.entrySet()) {
gen.writeObjectField(entry.getKey(), entry.getValue());
}
gen.writeEndObject();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -24,6 +24,7 @@
import org.apache.druid.guice.annotations.PublicApi;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.emitter.core.Event;
import org.apache.druid.java.util.emitter.core.EventMap;
import org.joda.time.DateTime;

import java.util.Collections;
Expand Down Expand Up @@ -148,9 +149,10 @@ public Map<String, Object> getDataMap()

@Override
@JsonValue
public Map<String, Object> toMap()
public EventMap toMap()
{
return ImmutableMap.<String, Object>builder()
return EventMap
.builder()
.put("feed", getFeed())
.put("timestamp", createdTime.toString())
.putAll(serviceDimensions)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,13 +27,15 @@
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.emitter.core.Event;
import org.apache.druid.java.util.emitter.core.EventMap;
import org.joda.time.DateTime;

import java.util.Arrays;
import java.util.Map;
import java.util.TreeMap;

/**
*
*/
@PublicApi
public class ServiceMetricEvent implements Event
Expand Down Expand Up @@ -105,28 +107,29 @@ public Number getValue()

@Override
@JsonValue
public Map<String, Object> toMap()
public EventMap toMap()
{
return ImmutableMap.<String, Object>builder()
.put("feed", getFeed())
.put("timestamp", createdTime.toString())
.putAll(serviceDims)
.put("metric", metric)
.put("value", value)
.putAll(
Maps.filterEntries(
userDims,
new Predicate<Map.Entry<String, Object>>()
{
@Override
public boolean apply(Map.Entry<String, Object> input)
{
return input.getKey() != null;
}
}
)
)
.build();
return EventMap
.builder()
.put("feed", getFeed())
.put("timestamp", createdTime.toString())
.putAll(serviceDims)
.put("metric", metric)
.put("value", value)
.putAll(
Maps.filterEntries(
userDims,
new Predicate<Map.Entry<String, Object>>()
{
@Override
public boolean apply(Map.Entry<String, Object> input)
{
return input.getKey() != null;
}
}
)
)
.build();
}

public static class Builder
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,6 @@

package org.apache.druid.java.util.emitter.core;

import java.util.Map;

class IntEvent implements Event
{
int index;
Expand All @@ -30,7 +28,7 @@ class IntEvent implements Event
}

@Override
public Map<String, Object> toMap()
public EventMap toMap()
{
return null;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -290,4 +290,5 @@ public void testNaN2()
{
ServiceMetricEvent.builder().build("foo", 0 / 0f);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -23,12 +23,13 @@
import com.fasterxml.jackson.annotation.JsonValue;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.java.util.emitter.core.Event;
import org.apache.druid.java.util.emitter.core.EventMap;

import java.util.Collections;
import java.util.HashMap;
import java.util.Map;

/**
*
*/
public class UnitEvent implements Event
{
Expand All @@ -50,12 +51,14 @@ public UnitEvent(String feed, Number value, Map<String, String> dimensions)

@Override
@JsonValue
public Map<String, Object> toMap()
public EventMap toMap()
{
Map<String, Object> result = new HashMap<>(dimensions);
result.put("feed", feed);
result.put("metrics", ImmutableMap.of("value", value));
return ImmutableMap.copyOf(result);
return EventMap
.builder()
.putAll(dimensions)
.put("feed", feed)
.put("metrics", ImmutableMap.of("value", value))
.build();
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,13 +22,13 @@
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.emitter.kafka.MemoryBoundLinkedBlockingQueue.ObjectContainer;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.emitter.core.Emitter;
import org.apache.druid.java.util.emitter.core.Event;
import org.apache.druid.java.util.emitter.core.EventMap;
import org.apache.druid.java.util.emitter.service.AlertEvent;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.server.log.RequestLogEvent;
Expand All @@ -39,7 +39,6 @@
import org.apache.kafka.clients.producer.ProducerRecord;
import org.apache.kafka.common.serialization.StringSerializer;

import java.util.Map;
import java.util.Properties;
import java.util.concurrent.Executors;
import java.util.concurrent.ScheduledExecutorService;
Expand Down Expand Up @@ -126,8 +125,12 @@ public void start()
scheduler.schedule(this::sendRequestToKafka, sendInterval, TimeUnit.SECONDS);
}
scheduler.scheduleWithFixedDelay(() -> {
log.info("Message lost counter: metricLost=[%d], alertLost=[%d], requestLost=[%d], invalidLost=[%d]",
metricLost.get(), alertLost.get(), requestLost.get(), invalidLost.get()
log.info(
"Message lost counter: metricLost=[%d], alertLost=[%d], requestLost=[%d], invalidLost=[%d]",
metricLost.get(),
alertLost.get(),
requestLost.get(),
invalidLost.get()
);
}, DEFAULT_SEND_LOST_INTERVAL_MINUTES, DEFAULT_SEND_LOST_INTERVAL_MINUTES, TimeUnit.MINUTES);
log.info("Starting Kafka Emitter.");
Expand Down Expand Up @@ -166,14 +169,16 @@ private void sendToKafka(final String topic, MemoryBoundLinkedBlockingQueue<Stri
public void emit(final Event event)
{
if (event != null) {
ImmutableMap.Builder<String, Object> resultBuilder = ImmutableMap.<String, Object>builder().putAll(event.toMap());
if (config.getClusterName() != null) {
resultBuilder.put("clusterName", config.getClusterName());
}
Map<String, Object> result = resultBuilder.build();

try {
String resultJson = jsonMapper.writeValueAsString(result);
EventMap map = event.toMap();
if (config.getClusterName() != null) {
map = map.asBuilder()
.put("clusterName", config.getClusterName())
.build();
}

String resultJson = jsonMapper.writeValueAsString(map);

ObjectContainer<String> objectContainer = new ObjectContainer<>(
resultJson,
StringUtils.toUtf8(resultJson).length
Expand Down
Loading