FilteredRequestLogger: Fix start/stop, invalid delegate behavior. (#6637)

* FilteredRequestLogger: Fix start/stop, invalid delegate behavior.

Fixes two bugs:

1) FilteredRequestLogger did not start/stop the delegate.

2) FilteredRequestLogger would ignore an invalid delegate type, and
instead silently substitute the "noop" logger. This was due to a larger
problem with RequestLoggerProvider setup in general; the fix here is
to remove "defaultImpl" from the RequestLoggerProvider interface, and
instead have JsonConfigurator be responsible for creating the
default implementations. It is stricter about things than the old system
was, and is only willing to make a noop logger if it doesn't see any
request logger configs. Otherwise, it'll raise a provision error.

* Remove unneeded annotations.
This commit is contained in:
Gian Merlino 2018-12-14 00:55:44 -08:00 committed by Benedict Jin
parent 4ec068642d
commit 04e7c7fbdc
13 changed files with 412 additions and 13 deletions

View File

@ -28,6 +28,7 @@ import com.google.inject.Provider;
import com.google.inject.util.Types;
import org.apache.druid.guice.annotations.PublicApi;
import javax.annotation.Nullable;
import java.lang.annotation.Annotation;
import java.lang.reflect.ParameterizedType;
import java.util.Properties;
@ -92,6 +93,24 @@ public class JsonConfigProvider<T> implements Provider<Supplier<T>>
);
}
@SuppressWarnings("unchecked")
public static <T> void bindWithDefault(
Binder binder,
String propertyBase,
Class<T> classToProvide,
Class<? extends T> defaultClass
)
{
bind(
binder,
propertyBase,
classToProvide,
defaultClass,
Key.get(classToProvide),
(Key) Key.get(Types.newParameterizedType(Supplier.class, classToProvide))
);
}
@SuppressWarnings("unchecked")
public static <T> void bind(Binder binder, String propertyBase, Class<T> classToProvide, Annotation annotation)
{
@ -121,7 +140,6 @@ public class JsonConfigProvider<T> implements Provider<Supplier<T>>
);
}
@SuppressWarnings("unchecked")
public static <T> void bind(
Binder binder,
String propertyBase,
@ -134,6 +152,19 @@ public class JsonConfigProvider<T> implements Provider<Supplier<T>>
binder.bind(instanceKey).toProvider(new SupplierProvider<>(supplierKey));
}
public static <T> void bind(
Binder binder,
String propertyBase,
Class<T> clazz,
Class<? extends T> defaultClass,
Key<T> instanceKey,
Key<Supplier<T>> supplierKey
)
{
binder.bind(supplierKey).toProvider(of(propertyBase, clazz, defaultClass)).in(LazySingleton.class);
binder.bind(instanceKey).toProvider(new SupplierProvider<>(supplierKey));
}
@SuppressWarnings("unchecked")
public static <T> void bindInstance(
Binder binder,
@ -159,11 +190,21 @@ public class JsonConfigProvider<T> implements Provider<Supplier<T>>
public static <T> JsonConfigProvider<T> of(String propertyBase, Class<T> classToProvide)
{
return new JsonConfigProvider<T>(propertyBase, classToProvide);
return of(propertyBase, classToProvide, null);
}
public static <T> JsonConfigProvider<T> of(
String propertyBase,
Class<T> classToProvide,
Class<? extends T> defaultClass
)
{
return new JsonConfigProvider<>(propertyBase, classToProvide, defaultClass);
}
private final String propertyBase;
private final Class<T> classToProvide;
private final Class<? extends T> defaultClass;
private Properties props;
private JsonConfigurator configurator;
@ -172,11 +213,13 @@ public class JsonConfigProvider<T> implements Provider<Supplier<T>>
public JsonConfigProvider(
String propertyBase,
Class<T> classToProvide
Class<T> classToProvide,
@Nullable Class<? extends T> defaultClass
)
{
this.propertyBase = propertyBase;
this.classToProvide = classToProvide;
this.defaultClass = defaultClass;
}
@Inject
@ -197,7 +240,7 @@ public class JsonConfigProvider<T> implements Provider<Supplier<T>>
}
try {
final T config = configurator.configurate(props, propertyBase, classToProvide);
final T config = configurator.configurate(props, propertyBase, classToProvide, defaultClass);
retVal = Suppliers.ofInstance(config);
}
catch (RuntimeException e) {

View File

@ -35,12 +35,14 @@ import com.google.inject.spi.Message;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
import javax.annotation.Nullable;
import javax.validation.ConstraintViolation;
import javax.validation.ElementKind;
import javax.validation.Path;
import javax.validation.Validator;
import java.io.IOException;
import java.lang.reflect.Field;
import java.lang.reflect.InvocationTargetException;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.Iterator;
@ -70,7 +72,17 @@ public class JsonConfigurator
public <T> T configurate(Properties props, String propertyPrefix, Class<T> clazz) throws ProvisionException
{
verifyClazzIsConfigurable(jsonMapper, clazz);
return configurate(props, propertyPrefix, clazz, null);
}
public <T> T configurate(
Properties props,
String propertyPrefix,
Class<T> clazz,
@Nullable Class<? extends T> defaultClass
) throws ProvisionException
{
verifyClazzIsConfigurable(jsonMapper, clazz, defaultClass);
// Make it end with a period so we only include properties with sub-object thingies.
final String propertyBase = propertyPrefix.endsWith(".") ? propertyPrefix : propertyPrefix + ".";
@ -98,13 +110,30 @@ public class JsonConfigurator
final T config;
try {
config = jsonMapper.convertValue(jsonMap, clazz);
if (defaultClass != null && jsonMap.isEmpty()) {
// No configs were provided. Don't use the jsonMapper; instead create a default instance of the default class
// using the no-arg constructor. We know it exists because verifyClazzIsConfigurable checks for it.
config = defaultClass.getConstructor().newInstance();
} else {
config = jsonMapper.convertValue(jsonMap, clazz);
}
}
catch (IllegalArgumentException e) {
throw new ProvisionException(
StringUtils.format("Problem parsing object at prefix[%s]: %s.", propertyPrefix, e.getMessage()), e
);
}
catch (NoSuchMethodException | InstantiationException | IllegalAccessException | InvocationTargetException e) {
throw new ProvisionException(
StringUtils.format(
"Problem instantiating object at prefix[%s]: %s: %s.",
propertyPrefix,
e.getClass().getSimpleName(),
e.getMessage()
),
e
);
}
final Set<ConstraintViolation<T>> violations = validator.validate(config);
if (!violations.isEmpty()) {
@ -205,8 +234,26 @@ public class JsonConfigurator
}
@VisibleForTesting
public static <T> void verifyClazzIsConfigurable(ObjectMapper mapper, Class<T> clazz)
public static <T> void verifyClazzIsConfigurable(
ObjectMapper mapper,
Class<T> clazz,
@Nullable Class<? extends T> defaultClass
)
{
if (defaultClass != null) {
try {
defaultClass.getConstructor();
}
catch (NoSuchMethodException e) {
throw new ProvisionException(
StringUtils.format(
"JsonConfigurator requires default classes to have zero-arg constructors. %s doesn't",
defaultClass
)
);
}
}
final List<BeanPropertyDefinition> beanDefs = mapper.getSerializationConfig()
.introspect(mapper.constructType(clazz))
.findProperties();

View File

@ -310,7 +310,7 @@ All nodes that can serve queries can also log the query requests they see.
|Property|Description|Default|
|--------|-----------|-------|
|`druid.request.logging.type`|Choices: noop, file, emitter, slf4j, filtered, composing. How to log every query request.|noop|
|`druid.request.logging.type`|Choices: noop, file, emitter, slf4j, filtered, composing. How to log every query request.|[required to configure request logging]|
Note that, you can enable sending all the HTTP requests to log by setting "org.apache.druid.jetty.RequestLog" to DEBUG level. See [Logging](../configuration/logging.html)

View File

@ -44,7 +44,7 @@ public class RemoteTaskRunnerConfigTest
@Test
public void testIsJsonConfiguratable()
{
JsonConfigurator.verifyClazzIsConfigurable(mapper, RemoteTaskRunnerConfig.class);
JsonConfigurator.verifyClazzIsConfigurable(mapper, RemoteTaskRunnerConfig.class, null);
}
@Test

View File

@ -30,6 +30,7 @@ import org.apache.druid.server.log.EmittingRequestLoggerProvider;
import org.apache.druid.server.log.FileRequestLoggerProvider;
import org.apache.druid.server.log.FilteredRequestLoggerProvider;
import org.apache.druid.server.log.LoggingRequestLoggerProvider;
import org.apache.druid.server.log.NoopRequestLoggerProvider;
import org.apache.druid.server.log.RequestLogger;
import org.apache.druid.server.log.RequestLoggerProvider;
@ -44,7 +45,12 @@ public class QueryableModule implements DruidModule
public void configure(Binder binder)
{
binder.bind(RequestLogger.class).toProvider(RequestLoggerProvider.class).in(ManageLifecycle.class);
JsonConfigProvider.bind(binder, "druid.request.logging", RequestLoggerProvider.class);
JsonConfigProvider.bindWithDefault(
binder,
"druid.request.logging",
RequestLoggerProvider.class,
NoopRequestLoggerProvider.class
);
binder.bind(QueryRunnerFactoryConglomerate.class)
.to(DefaultQueryRunnerFactoryConglomerate.class)

View File

@ -103,7 +103,7 @@ public class AuthenticatorMapperModule implements DruidModule
for (String authenticatorName : authenticators) {
final String authenticatorPropertyBase = StringUtils.format(AUTHENTICATOR_PROPERTIES_FORMAT_STRING, authenticatorName);
final JsonConfigProvider<Authenticator> authenticatorProvider = new JsonConfigProvider<>(
final JsonConfigProvider<Authenticator> authenticatorProvider = JsonConfigProvider.of(
authenticatorPropertyBase,
Authenticator.class
);

View File

@ -115,7 +115,7 @@ public class AuthorizerMapperModule implements DruidModule
for (String authorizerName : authorizers) {
final String authorizerPropertyBase = StringUtils.format(AUTHORIZER_PROPERTIES_FORMAT_STRING, authorizerName);
final JsonConfigProvider<Authorizer> authorizerProvider = new JsonConfigProvider<>(
final JsonConfigProvider<Authorizer> authorizerProvider = JsonConfigProvider.of(
authorizerPropertyBase,
Authorizer.class
);

View File

@ -21,6 +21,8 @@ package org.apache.druid.server.log;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import org.apache.druid.java.util.common.lifecycle.LifecycleStart;
import org.apache.druid.java.util.common.lifecycle.LifecycleStop;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.server.RequestLogLine;
@ -61,6 +63,30 @@ public class FilteredRequestLoggerProvider implements RequestLoggerProvider
this.queryTimeThresholdMs = queryTimeThresholdMs;
}
public long getQueryTimeThresholdMs()
{
return queryTimeThresholdMs;
}
public RequestLogger getDelegate()
{
return logger;
}
@LifecycleStart
@Override
public void start() throws Exception
{
logger.start();
}
@LifecycleStop
@Override
public void stop()
{
logger.stop();
}
@Override
public void log(RequestLogLine requestLogLine) throws IOException
{

View File

@ -26,7 +26,7 @@ import com.google.inject.Provider;
* A Marker interface for things that can provide a RequestLogger. This can be combined with jackson polymorphic serde
* to provide new RequestLogger implementations as plugins.
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = NoopRequestLoggerProvider.class)
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
public interface RequestLoggerProvider extends Provider<RequestLogger>
{
}

View File

@ -19,16 +19,45 @@
package org.apache.druid.server.log;
import com.fasterxml.jackson.databind.InjectableValues;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap;
import com.google.inject.ProvisionException;
import org.apache.druid.guice.JsonConfigurator;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.server.QueryStats;
import org.apache.druid.server.RequestLogLine;
import org.easymock.EasyMock;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import javax.validation.Validation;
import java.io.IOException;
import java.util.Properties;
public class FilteredRequestLoggerTest
{
private final DefaultObjectMapper mapper = new DefaultObjectMapper();
@Rule
public final ExpectedException expectedException = ExpectedException.none();
public FilteredRequestLoggerTest()
{
mapper.registerSubtypes(
LoggingRequestLoggerProvider.class,
FilteredRequestLoggerProvider.class,
TestRequestLoggerProvider.class,
NoopRequestLoggerProvider.class
);
final InjectableValues.Std injectableValues = new InjectableValues.Std();
injectableValues.addValue(ObjectMapper.class, mapper);
mapper.setInjectableValues(injectableValues);
}
@Test
public void testFilterBelowThreshold() throws IOException
{
@ -70,4 +99,104 @@ public class FilteredRequestLoggerTest
EasyMock.verify(requestLogLine, delegate);
}
@Test
public void testConfiguration()
{
final Properties properties = new Properties();
properties.setProperty("log.type", "filtered");
properties.setProperty("log.queryTimeThresholdMs", "100");
properties.setProperty("log.delegate.type", "slf4j");
properties.setProperty("log.delegate.setMDC", "true");
properties.setProperty("log.delegate.setContextMDC", "true");
final JsonConfigurator configurator = new JsonConfigurator(
mapper,
Validation.buildDefaultValidatorFactory()
.getValidator()
);
final FilteredRequestLoggerProvider provider = (FilteredRequestLoggerProvider) configurator.configurate(
properties,
"log",
RequestLoggerProvider.class
);
final FilteredRequestLoggerProvider.FilteredRequestLogger logger =
((FilteredRequestLoggerProvider.FilteredRequestLogger) provider.get());
final LoggingRequestLogger delegate = (LoggingRequestLogger) logger.getDelegate();
Assert.assertEquals(100, logger.getQueryTimeThresholdMs());
Assert.assertTrue(delegate.isSetContextMDC());
Assert.assertTrue(delegate.isSetMDC());
}
@Test
public void testStartStop() throws Exception
{
final Properties properties = new Properties();
properties.setProperty("log.type", "filtered");
properties.setProperty("log.queryTimeThresholdMs", "100");
properties.setProperty("log.delegate.type", "test");
final JsonConfigurator configurator = new JsonConfigurator(
mapper,
Validation.buildDefaultValidatorFactory()
.getValidator()
);
final FilteredRequestLoggerProvider provider = (FilteredRequestLoggerProvider) configurator.configurate(
properties,
"log",
RequestLoggerProvider.class
);
final FilteredRequestLoggerProvider.FilteredRequestLogger logger =
((FilteredRequestLoggerProvider.FilteredRequestLogger) provider.get());
final TestRequestLogger delegate = (TestRequestLogger) logger.getDelegate();
Assert.assertFalse(delegate.isStarted());
logger.start();
Assert.assertTrue(delegate.isStarted());
logger.stop();
Assert.assertFalse(delegate.isStarted());
}
@Test
public void testInvalidDelegateType()
{
final Properties properties = new Properties();
properties.setProperty("log.type", "filtered");
properties.setProperty("log.queryTimeThresholdMs", "100");
properties.setProperty("log.delegate.type", "nope");
final JsonConfigurator configurator = new JsonConfigurator(
mapper,
Validation.buildDefaultValidatorFactory()
.getValidator()
);
expectedException.expect(ProvisionException.class);
expectedException.expectMessage("Could not resolve type id 'nope'");
configurator.configurate(properties, "log", RequestLoggerProvider.class);
}
@Test
public void testNoDelegate()
{
final Properties properties = new Properties();
properties.setProperty("log.type", "filtered");
properties.setProperty("log.queryTimeThresholdMs", "100");
final JsonConfigurator configurator = new JsonConfigurator(
mapper,
Validation.buildDefaultValidatorFactory()
.getValidator()
);
expectedException.expect(ProvisionException.class);
expectedException.expectMessage("log.delegate - may not be null");
configurator.configurate(properties, "log", RequestLoggerProvider.class);
}
}

View File

@ -0,0 +1,97 @@
/*
* 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.databind.InjectableValues;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.ProvisionException;
import org.apache.druid.guice.JsonConfigurator;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.hamcrest.CoreMatchers;
import org.junit.Assert;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import javax.validation.Validation;
import java.util.Properties;
public class RequestLoggerProviderTest
{
private final DefaultObjectMapper mapper = new DefaultObjectMapper();
@Rule
public final ExpectedException expectedException = ExpectedException.none();
public RequestLoggerProviderTest()
{
mapper.registerSubtypes(
NoopRequestLoggerProvider.class,
TestRequestLoggerProvider.class
);
final InjectableValues.Std injectableValues = new InjectableValues.Std();
injectableValues.addValue(ObjectMapper.class, mapper);
mapper.setInjectableValues(injectableValues);
}
@Test
public void testNoLoggerAtAll()
{
final Properties properties = new Properties();
properties.setProperty("dummy", "unrelated");
final JsonConfigurator configurator = new JsonConfigurator(
mapper,
Validation.buildDefaultValidatorFactory()
.getValidator()
);
final RequestLoggerProvider provider = configurator.configurate(
properties,
"log",
RequestLoggerProvider.class,
NoopRequestLoggerProvider.class
);
Assert.assertThat(provider, CoreMatchers.instanceOf(NoopRequestLoggerProvider.class));
}
@Test
public void testLoggerPropertiesWithNoType()
{
final Properties properties = new Properties();
properties.setProperty("dummy", "unrelated");
properties.setProperty("log.foo", "bar");
final JsonConfigurator configurator = new JsonConfigurator(
mapper,
Validation.buildDefaultValidatorFactory()
.getValidator()
);
expectedException.expect(ProvisionException.class);
expectedException.expectMessage("missing property 'type'");
configurator.configurate(
properties,
"log",
RequestLoggerProvider.class,
NoopRequestLoggerProvider.class
);
}
}

View File

@ -24,16 +24,30 @@ import org.apache.druid.server.RequestLogLine;
import java.util.ArrayList;
import java.util.List;
import java.util.concurrent.atomic.AtomicBoolean;
public class TestRequestLogger implements RequestLogger
{
private final List<RequestLogLine> logs;
private final AtomicBoolean started = new AtomicBoolean();
public TestRequestLogger()
{
this.logs = new ArrayList<>();
}
@Override
public void start()
{
started.set(true);
}
@Override
public void stop()
{
started.set(false);
}
@Override
public void log(final RequestLogLine requestLogLine)
{
@ -42,6 +56,11 @@ public class TestRequestLogger implements RequestLogger
}
}
public boolean isStarted()
{
return started.get();
}
public List<RequestLogLine> getLogs()
{
synchronized (logs) {

View File

@ -0,0 +1,32 @@
/*
* 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.JsonTypeName;
@JsonTypeName("test")
public class TestRequestLoggerProvider implements RequestLoggerProvider
{
@Override
public RequestLogger get()
{
return new TestRequestLogger();
}
}