Add RequestLogEventBuilderFactory (#6477)

This PR allows to control the fields in `RequestLogEvent`, emitted in `EmittingRequestLogger`. In our case, we want to get rid of the `intervals` fields of the query objects that are a part of `DefaultRequestLogEvent`. They are enormous (thousands of segments) and not useful.

Related to #5522, FYI @a2l007.
This commit is contained in:
Roman Leventov 2018-10-31 22:24:37 +01:00 committed by GitHub
parent d5e9e5686e
commit 2cdce2e2a6
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
8 changed files with 251 additions and 109 deletions

View File

@ -30,7 +30,6 @@ import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
@JsonSubTypes.Type(name = "all", value = SendAllTimelineEventConverter.class), @JsonSubTypes.Type(name = "all", value = SendAllTimelineEventConverter.class),
@JsonSubTypes.Type(name = "whiteList", value = WhiteListBasedDruidToTimelineEventConverter.class) @JsonSubTypes.Type(name = "whiteList", value = WhiteListBasedDruidToTimelineEventConverter.class)
}) })
public interface DruidToTimelineMetricConverter public interface DruidToTimelineMetricConverter
{ {
/** /**

View File

@ -29,7 +29,7 @@ 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.Event;
import org.apache.druid.java.util.emitter.service.AlertEvent; import org.apache.druid.java.util.emitter.service.AlertEvent;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent; import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.server.log.EmittingRequestLogger; import org.apache.druid.server.log.RequestLogEvent;
import java.io.IOException; import java.io.IOException;
import java.net.SocketException; import java.net.SocketException;
@ -124,7 +124,7 @@ public class GraphiteEmitter implements Emitter
log.error(e, "got interrupted with message [%s]", e.getMessage()); log.error(e, "got interrupted with message [%s]", e.getMessage());
Thread.currentThread().interrupt(); Thread.currentThread().interrupt();
} }
} else if (event instanceof EmittingRequestLogger.RequestLogEvent) { } else if (event instanceof RequestLogEvent) {
for (Emitter emitter : requestLogEmitters) { for (Emitter emitter : requestLogEmitters) {
emitter.emit(event); emitter.emit(event);
} }

View File

@ -0,0 +1,104 @@
/*
* 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.server.log;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonValue;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.guice.annotations.PublicApi;
import org.apache.druid.query.Query;
import org.apache.druid.server.QueryStats;
import org.apache.druid.server.RequestLogLine;
import org.joda.time.DateTime;
import java.util.Map;
/**
* The default implementation of {@link RequestLogEvent}. This class is annotated {@link PublicApi} because it's getters
* could be used in proprietary {@link org.apache.druid.java.util.emitter.core.Emitter} implementations.
*/
@PublicApi
public final class DefaultRequestLogEvent implements RequestLogEvent
{
private final ImmutableMap<String, String> serviceDimensions;
private final String feed;
private final RequestLogLine request;
DefaultRequestLogEvent(ImmutableMap<String, String> serviceDimensions, String feed, RequestLogLine request)
{
this.serviceDimensions = serviceDimensions;
this.request = request;
this.feed = feed;
}
/**
* Override {@link JsonValue} serialization, instead use annotations to include type information for polymorphic
* {@link Query} objects.
*/
@JsonValue(value = false)
@Override
public Map<String, Object> toMap()
{
return ImmutableMap.of();
}
@Override
@JsonProperty("feed")
public String getFeed()
{
return feed;
}
@JsonProperty("timestamp")
public DateTime getCreatedTime()
{
return request.getTimestamp();
}
@JsonProperty("service")
public String getService()
{
return serviceDimensions.get("service");
}
@JsonProperty("host")
public String getHost()
{
return serviceDimensions.get("host");
}
@JsonProperty("query")
public Query getQuery()
{
return request.getQuery();
}
@JsonProperty("remoteAddr")
public String getRemoteAddr()
{
return request.getRemoteAddr();
}
@JsonProperty("queryStats")
public QueryStats getQueryStats()
{
return request.getQueryStats();
}
}

View File

@ -0,0 +1,60 @@
/*
* 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.server.log;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.java.util.emitter.service.ServiceEventBuilder;
import org.apache.druid.server.RequestLogLine;
/**
* This {@link RequestLogEventBuilderFactory} creates builders that return {@link DefaultRequestLogEvent}s.
*/
public final class DefaultRequestLogEventBuilderFactory implements RequestLogEventBuilderFactory
{
private static final DefaultRequestLogEventBuilderFactory INSTANCE = new DefaultRequestLogEventBuilderFactory();
@JsonCreator
public static DefaultRequestLogEventBuilderFactory instance()
{
return INSTANCE;
}
private DefaultRequestLogEventBuilderFactory() {}
@Override
public ServiceEventBuilder<RequestLogEvent> createRequestLogEventBuilder(String feed, RequestLogLine requestLogLine)
{
return new ServiceEventBuilder<RequestLogEvent>()
{
@Override
public RequestLogEvent build(ImmutableMap<String, String> serviceDimensions)
{
return new DefaultRequestLogEvent(serviceDimensions, feed, requestLogLine);
}
};
}
@Override
public String toString()
{
return getClass().getName();
}
}

View File

@ -19,35 +19,30 @@
package org.apache.druid.server.log; package org.apache.druid.server.log;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonValue;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.guice.annotations.PublicApi;
import org.apache.druid.java.util.emitter.core.Event;
import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.java.util.emitter.service.ServiceEventBuilder;
import org.apache.druid.query.Query;
import org.apache.druid.server.QueryStats;
import org.apache.druid.server.RequestLogLine; import org.apache.druid.server.RequestLogLine;
import org.joda.time.DateTime;
import java.util.Map;
public class EmittingRequestLogger implements RequestLogger public class EmittingRequestLogger implements RequestLogger
{ {
final ServiceEmitter emitter; private final ServiceEmitter emitter;
final String feed; private final String feed;
private final RequestLogEventBuilderFactory requestLogEventBuilderFactory;
public EmittingRequestLogger(ServiceEmitter emitter, String feed) EmittingRequestLogger(
ServiceEmitter emitter,
String feed,
RequestLogEventBuilderFactory requestLogEventBuilderFactory
)
{ {
this.emitter = emitter; this.emitter = emitter;
this.feed = feed; this.feed = feed;
this.requestLogEventBuilderFactory = requestLogEventBuilderFactory;
} }
@Override @Override
public void log(final RequestLogLine requestLogLine) public void log(final RequestLogLine requestLogLine)
{ {
emitter.emit(new RequestLogEventBuilder(feed, requestLogLine)); emitter.emit(requestLogEventBuilderFactory.createRequestLogEventBuilder(feed, requestLogLine));
} }
@Override @Override
@ -56,96 +51,7 @@ public class EmittingRequestLogger implements RequestLogger
return "EmittingRequestLogger{" + return "EmittingRequestLogger{" +
"emitter=" + emitter + "emitter=" + emitter +
", feed='" + feed + '\'' + ", feed='" + feed + '\'' +
", requestLogEventBuilderFactory=" + requestLogEventBuilderFactory +
'}'; '}';
} }
@PublicApi
public static class RequestLogEvent implements Event
{
final ImmutableMap<String, String> serviceDimensions;
final String feed;
final RequestLogLine request;
RequestLogEvent(ImmutableMap<String, String> serviceDimensions, String feed, RequestLogLine request)
{
this.serviceDimensions = serviceDimensions;
this.request = request;
this.feed = feed;
}
@Override
// override JsonValue serialization, instead use annotations
// to include type information for polymorphic Query objects
@JsonValue(value = false)
public Map<String, Object> toMap()
{
return ImmutableMap.of();
}
@Override
@JsonProperty("feed")
public String getFeed()
{
return feed;
}
@JsonProperty("timestamp")
public DateTime getCreatedTime()
{
return request.getTimestamp();
}
@JsonProperty("service")
public String getService()
{
return serviceDimensions.get("service");
}
@JsonProperty("host")
public String getHost()
{
return serviceDimensions.get("host");
}
@JsonProperty("query")
public Query getQuery()
{
return request.getQuery();
}
@JsonProperty("remoteAddr")
public String getRemoteAddr()
{
return request.getRemoteAddr();
}
@JsonProperty("queryStats")
public QueryStats getQueryStats()
{
return request.getQueryStats();
}
}
private static class RequestLogEventBuilder extends ServiceEventBuilder<Event>
{
private final String feed;
private final RequestLogLine requestLogLine;
public RequestLogEventBuilder(
String feed,
RequestLogLine requestLogLine
)
{
this.feed = feed;
this.requestLogLine = requestLogLine;
}
@Override
public Event build(ImmutableMap<String, String> serviceDimensions)
{
return new RequestLogEvent(serviceDimensions, feed, requestLogLine);
}
}
} }

View File

@ -40,6 +40,10 @@ public class EmittingRequestLoggerProvider implements RequestLoggerProvider
@NotNull @NotNull
private String feed = null; private String feed = null;
@JsonProperty
@NotNull
private RequestLogEventBuilderFactory requestLogEventBuilderFactory = null;
@JacksonInject @JacksonInject
@NotNull @NotNull
private ServiceEmitter emitter = null; private ServiceEmitter emitter = null;
@ -52,7 +56,7 @@ public class EmittingRequestLoggerProvider implements RequestLoggerProvider
@Override @Override
public RequestLogger get() public RequestLogger get()
{ {
EmittingRequestLogger logger = new EmittingRequestLogger(emitter, feed); EmittingRequestLogger logger = new EmittingRequestLogger(emitter, feed, requestLogEventBuilderFactory);
log.debug(new Exception("Stack trace"), "Creating %s at", logger); log.debug(new Exception("Stack trace"), "Creating %s at", logger);
return logger; return logger;
} }

View File

@ -0,0 +1,31 @@
/*
* 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.server.log;
import org.apache.druid.guice.annotations.PublicApi;
import org.apache.druid.java.util.emitter.core.Event;
/**
* Marker subtype of events emitted from {@link EmittingRequestLogger}.
*/
@PublicApi
public interface RequestLogEvent extends Event
{
}

View File

@ -0,0 +1,38 @@
/*
* 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.server.log;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import org.apache.druid.guice.annotations.PublicApi;
import org.apache.druid.java.util.emitter.service.ServiceEventBuilder;
import org.apache.druid.server.RequestLogLine;
/**
* This factory allows to customize {@link RequestLogEvent}s, emitted in {@link EmittingRequestLogger}, e. g. to exclude
* some fields (compared to {@link DefaultRequestLogEvent}) to make the events smaller.
*
* The default factory creates builders that return {@link DefaultRequestLogEvent}.
*/
@PublicApi
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = DefaultRequestLogEventBuilderFactory.class)
public interface RequestLogEventBuilderFactory
{
ServiceEventBuilder<RequestLogEvent> createRequestLogEventBuilder(String feed, RequestLogLine requestLogLine);
}