mirror of https://github.com/apache/druid.git
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:
parent
4ec068642d
commit
04e7c7fbdc
|
@ -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) {
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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)
|
||||
|
||||
|
|
|
@ -44,7 +44,7 @@ public class RemoteTaskRunnerConfigTest
|
|||
@Test
|
||||
public void testIsJsonConfiguratable()
|
||||
{
|
||||
JsonConfigurator.verifyClazzIsConfigurable(mapper, RemoteTaskRunnerConfig.class);
|
||||
JsonConfigurator.verifyClazzIsConfigurable(mapper, RemoteTaskRunnerConfig.class, null);
|
||||
}
|
||||
|
||||
@Test
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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
|
||||
);
|
||||
|
|
|
@ -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
|
||||
);
|
||||
|
|
|
@ -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
|
||||
{
|
||||
|
|
|
@ -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>
|
||||
{
|
||||
}
|
||||
|
|
|
@ -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);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
}
|
|
@ -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) {
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
Loading…
Reference in New Issue