Merge pull request #2734 from b-slim/LookupIntrospection2

[QTL][Lookup] adding introspection endpoint
This commit is contained in:
Slim 2016-04-21 12:15:57 -05:00
parent c74391e54c
commit 984a518c9f
14 changed files with 448 additions and 27 deletions

View File

@ -301,3 +301,9 @@ It is possible to save the configuration across restarts such that a node will n
|Property|Description|Default| |Property|Description|Default|
|--------|-----------|-------| |--------|-----------|-------|
|`druid.lookup.snapshotWorkingDir`| Working path used to store snapshot of current lookup configuration, leaving this property null will disable snapshot/bootstrap utility|null| |`druid.lookup.snapshotWorkingDir`| Working path used to store snapshot of current lookup configuration, leaving this property null will disable snapshot/bootstrap utility|null|
## Introspect a Lookup
Lookup implementations can provide some introspection capabilities by implementing `LookupIntrospectHandler`. User will send request to `/druid/lookups/v1/introspect/{lookupId}` to enable introspection on a given lookup.
For instance you can list all the keys/values of a map based lookup by issuing a `GET` request to `/druid/lookups/v1/introspect/{lookupId}/keys"` or `/druid/lookups/v1/introspect/{lookupId}/values"`

View File

@ -19,10 +19,8 @@
package io.druid.query.lookup; package io.druid.query.lookup;
import com.fasterxml.jackson.annotation.JsonSubTypes;
import com.fasterxml.jackson.annotation.JsonTypeInfo; import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.google.common.base.Supplier; import com.google.common.base.Supplier;
import io.druid.query.extraction.MapLookupExtractorFactory;
import javax.annotation.Nullable; import javax.annotation.Nullable;
@ -32,9 +30,6 @@ import javax.annotation.Nullable;
* If a LookupExtractorFactory wishes to support idempotent updates, it needs to implement the `replaces` method * If a LookupExtractorFactory wishes to support idempotent updates, it needs to implement the `replaces` method
*/ */
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type") @JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonSubTypes(value = {
@JsonSubTypes.Type(name = "map", value = MapLookupExtractorFactory.class)
})
public interface LookupExtractorFactory extends Supplier<LookupExtractor> public interface LookupExtractorFactory extends Supplier<LookupExtractor>
{ {
/** /**
@ -55,7 +50,6 @@ public interface LookupExtractorFactory extends Supplier<LookupExtractor>
* @return true if successfully closed the {@link LookupExtractor} * @return true if successfully closed the {@link LookupExtractor}
*/ */
public boolean close(); public boolean close();
/** /**
* Determine if this LookupExtractorFactory should replace some other LookupExtractorFactory. * Determine if this LookupExtractorFactory should replace some other LookupExtractorFactory.
* This is used to implement no-down-time * This is used to implement no-down-time
@ -63,4 +57,11 @@ public interface LookupExtractorFactory extends Supplier<LookupExtractor>
* @return `true` if the other should be replaced by this one. `false` if this one should not replace the other factory * @return `true` if the other should be replaced by this one. `false` if this one should not replace the other factory
*/ */
boolean replaces(@Nullable LookupExtractorFactory other); boolean replaces(@Nullable LookupExtractorFactory other);
/**
* @return Returns the actual introspection request handler, can return {@code null} if it is not supported.
* This will be called once per HTTP request to introspect the actual lookup.
*/
@Nullable
public LookupIntrospectHandler getIntrospectHandler();
} }

View File

@ -0,0 +1,30 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.query.lookup;
/**
* This interface is empty because it only exists to signal intent. The actual http endpoints are provided
* through JAX-RS annotations on the {@link LookupIntrospectHandler} objects.
* Note that, if you decide to implement {@link LookupExtractorFactory#getIntrospectHandler()} as request scoped, therefore {@link LookupIntrospectHandler} should have as light of a footprint as possible.
*/
public interface LookupIntrospectHandler
{
}

View File

@ -246,6 +246,13 @@ public class RegisteredLookupExtractionFnTest
return false; return false;
} }
@Nullable
@Override
public LookupIntrospectHandler getIntrospectHandler()
{
return null;
}
@Override @Override
public LookupExtractor get() public LookupExtractor get()
{ {

View File

@ -207,6 +207,24 @@
<artifactId>caliper</artifactId> <artifactId>caliper</artifactId>
<scope>test</scope> <scope>test</scope>
</dependency> </dependency>
<dependency>
<groupId>com.sun.jersey.jersey-test-framework</groupId>
<artifactId>jersey-test-framework-core</artifactId>
<version>1.19</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>com.sun.jersey.jersey-test-framework</groupId>
<artifactId>jersey-test-framework-grizzly2</artifactId>
<version>1.19</version>
<scope>test</scope>
</dependency>
<dependency>
<groupId>pl.pragmatists</groupId>
<artifactId>JUnitParams</artifactId>
<version>1.0.4</version>
<scope>test</scope>
</dependency>
</dependencies> </dependencies>
<build> <build>

View File

@ -0,0 +1,63 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.query.lookup;
import com.google.inject.Inject;
import com.metamx.common.logger.Logger;
import javax.ws.rs.Path;
import javax.ws.rs.PathParam;
import javax.ws.rs.core.Context;
import javax.ws.rs.core.Response;
@Path("/druid/v1/lookups/introspect")
public class LookupIntrospectionResource
{
private static final Logger LOGGER = new Logger(LookupIntrospectionResource.class);
private final LookupReferencesManager lookupReferencesManager;
@Inject
public LookupIntrospectionResource(@Context LookupReferencesManager lookupReferencesManager)
{
this.lookupReferencesManager = lookupReferencesManager;
}
@Path("/{lookupId}")
public Object introspectLookup(@PathParam("lookupId") final String lookupId)
{
final LookupExtractorFactory lookupExtractorFactory = lookupReferencesManager.get(lookupId);
if (lookupExtractorFactory == null) {
LOGGER.error("trying to introspect non existing lookup [%s]", lookupId);
return Response.status(Response.Status.NOT_FOUND).build();
}
LookupIntrospectHandler introspectHandler = lookupExtractorFactory.getIntrospectHandler();
if (introspectHandler != null) {
return introspectHandler;
} else {
LOGGER.warn(
"Trying to introspect lookup [%s] of type [%s] but implementation doesn't provide resource",
lookupId,
lookupExtractorFactory.get().getClass()
);
return Response.status(Response.Status.NOT_FOUND).build();
}
}
}

View File

@ -25,6 +25,8 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.Module; import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import com.google.common.net.HostAndPort; import com.google.common.net.HostAndPort;
import com.google.inject.Binder; import com.google.inject.Binder;
@ -50,7 +52,6 @@ import io.druid.server.lookup.cache.LookupCoordinatorManager;
import org.apache.curator.utils.ZKPaths; import org.apache.curator.utils.ZKPaths;
import javax.ws.rs.Path; import javax.ws.rs.Path;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.List; import java.util.List;
import java.util.Map; import java.util.Map;
@ -68,7 +69,9 @@ public class LookupModule implements DruidModule
@Override @Override
public List<? extends Module> getJacksonModules() public List<? extends Module> getJacksonModules()
{ {
return Collections.emptyList(); return ImmutableList.<Module>of(
new SimpleModule("DruidLookupModule").registerSubtypes(MapLookupExtractorFactory.class)
);
} }
@Override @Override
@ -78,6 +81,7 @@ public class LookupModule implements DruidModule
LifecycleModule.register(binder, LookupReferencesManager.class); LifecycleModule.register(binder, LookupReferencesManager.class);
JsonConfigProvider.bind(binder, PROPERTY_BASE, LookupListeningAnnouncerConfig.class); JsonConfigProvider.bind(binder, PROPERTY_BASE, LookupListeningAnnouncerConfig.class);
Jerseys.addResource(binder, LookupListeningResource.class); Jerseys.addResource(binder, LookupListeningResource.class);
Jerseys.addResource(binder, LookupIntrospectionResource.class);
LifecycleModule.register(binder, LookupResourceListenerAnnouncer.class); LifecycleModule.register(binder, LookupResourceListenerAnnouncer.class);
} }
} }

View File

@ -1,33 +1,39 @@
/* /*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one * Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file * regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the * to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance * "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at * with the License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
* *
* Unless required by applicable law or agreed to in writing, * Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an * software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the * KIND, either express or implied. See the License for the
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package io.druid.query.extraction; package io.druid.query.lookup;
import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import io.druid.query.lookup.LookupExtractor; import io.druid.query.extraction.MapLookupExtractor;
import io.druid.query.lookup.LookupExtractorFactory;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import javax.ws.rs.GET;
import javax.ws.rs.Path;
import javax.ws.rs.Produces;
import javax.ws.rs.core.MediaType;
import javax.ws.rs.core.Response;
import java.util.Map; import java.util.Map;
@JsonTypeName("map")
public class MapLookupExtractorFactory implements LookupExtractorFactory public class MapLookupExtractorFactory implements LookupExtractorFactory
{ {
@JsonProperty @JsonProperty
@ -35,6 +41,7 @@ public class MapLookupExtractorFactory implements LookupExtractorFactory
@JsonProperty @JsonProperty
private final boolean isOneToOne; private final boolean isOneToOne;
private final MapLookupExtractor lookupExtractor; private final MapLookupExtractor lookupExtractor;
private final LookupIntrospectHandler lookupIntrospectHandler;
@JsonCreator @JsonCreator
public MapLookupExtractorFactory( public MapLookupExtractorFactory(
@ -45,6 +52,7 @@ public class MapLookupExtractorFactory implements LookupExtractorFactory
this.map = Preconditions.checkNotNull(map, "map cannot be null"); this.map = Preconditions.checkNotNull(map, "map cannot be null");
this.isOneToOne = isOneToOne; this.isOneToOne = isOneToOne;
this.lookupExtractor = new MapLookupExtractor(map, isOneToOne); this.lookupExtractor = new MapLookupExtractor(map, isOneToOne);
this.lookupIntrospectHandler = new MapLookupIntrospectionHandler(this.map);
} }
@Override @Override
@ -63,6 +71,7 @@ public class MapLookupExtractorFactory implements LookupExtractorFactory
* For MapLookups, the replaces consideration is very easy, it simply considers if the other is the same as this one * For MapLookups, the replaces consideration is very easy, it simply considers if the other is the same as this one
* *
* @param other Some other LookupExtractorFactory which might need replaced * @param other Some other LookupExtractorFactory which might need replaced
*
* @return true - should replace, false - should not replace * @return true - should replace, false - should not replace
*/ */
@Override @Override
@ -71,6 +80,13 @@ public class MapLookupExtractorFactory implements LookupExtractorFactory
return !equals(other); return !equals(other);
} }
@Nullable
@Override
public LookupIntrospectHandler getIntrospectHandler()
{
return lookupIntrospectHandler;
}
@Override @Override
public LookupExtractor get() public LookupExtractor get()
{ {
@ -103,4 +119,34 @@ public class MapLookupExtractorFactory implements LookupExtractorFactory
result = 31 * result + (isOneToOne ? 1 : 0); result = 31 * result + (isOneToOne ? 1 : 0);
return result; return result;
} }
public static class MapLookupIntrospectionHandler implements LookupIntrospectHandler
{
final private Map<String, String> map;
public MapLookupIntrospectionHandler(Map<String, String> map)
{
this.map = map;
}
@GET
@Path("/keys")
@Produces(MediaType.APPLICATION_JSON)
public Response getKeys()
{
return Response.ok(map.keySet().toString()).build();
}
@GET
@Path("/values")
@Produces(MediaType.APPLICATION_JSON)
public Response getValues()
{
return Response.ok(map.values().toString()).build();
}
@GET
@Produces(MediaType.APPLICATION_JSON)
public Response getMap()
{return Response.ok(map).build();}
}
} }

View File

@ -26,9 +26,9 @@ import com.google.common.collect.ImmutableMap;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.extraction.ExtractionFn; import io.druid.query.extraction.ExtractionFn;
import io.druid.query.extraction.MapLookupExtractor; import io.druid.query.extraction.MapLookupExtractor;
import io.druid.query.extraction.MapLookupExtractorFactory;
import io.druid.query.lookup.LookupExtractor; import io.druid.query.lookup.LookupExtractor;
import io.druid.query.lookup.LookupReferencesManager; import io.druid.query.lookup.LookupReferencesManager;
import io.druid.query.lookup.MapLookupExtractorFactory;
import junitparams.JUnitParamsRunner; import junitparams.JUnitParamsRunner;
import junitparams.Parameters; import junitparams.Parameters;
import org.easymock.EasyMock; import org.easymock.EasyMock;

View File

@ -0,0 +1,119 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.query.lookup;
import com.google.common.collect.ImmutableMap;
import com.sun.jersey.api.client.WebResource;
import com.sun.jersey.api.core.ClassNamesResourceConfig;
import com.sun.jersey.spi.container.servlet.WebComponent;
import com.sun.jersey.spi.inject.SingletonTypeInjectableProvider;
import com.sun.jersey.test.framework.JerseyTest;
import com.sun.jersey.test.framework.WebAppDescriptor;
import com.sun.jersey.test.framework.spi.container.TestContainerFactory;
import com.sun.jersey.test.framework.spi.container.grizzly2.GrizzlyTestContainerFactory;
import org.easymock.EasyMock;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import javax.ws.rs.core.Context;
import javax.ws.rs.ext.Provider;
public class LookupIntrospectionResourceImplTest extends JerseyTest
{
static LookupReferencesManager lookupReferencesManager = EasyMock.createMock(LookupReferencesManager.class);
@Before
public void setUp() throws Exception
{
super.setUp();
EasyMock.reset(lookupReferencesManager);
LookupExtractorFactory lookupExtractorFactory1 = new MapLookupExtractorFactory(ImmutableMap.of(
"key",
"value",
"key2",
"value2"
), false);
EasyMock.expect(lookupReferencesManager.get("lookupId1")).andReturn(lookupExtractorFactory1).anyTimes();
EasyMock.replay(lookupReferencesManager);
}
@Provider
public static class MockTodoServiceProvider extends
SingletonTypeInjectableProvider<Context, LookupReferencesManager>
{
public MockTodoServiceProvider()
{
super(LookupReferencesManager.class, lookupReferencesManager);
}
}
public LookupIntrospectionResourceImplTest()
{
super(new WebAppDescriptor.Builder().initParam(
WebComponent.RESOURCE_CONFIG_CLASS,
ClassNamesResourceConfig.class.getName()
)
.initParam(
ClassNamesResourceConfig.PROPERTY_CLASSNAMES,
LookupIntrospectionResource.class.getName()
+ ';'
+ MockTodoServiceProvider.class.getName()
+ ';'
+ LookupIntrospectHandler.class.getName()
)
.build());
}
@Override
protected TestContainerFactory getTestContainerFactory()
{
return new GrizzlyTestContainerFactory();
}
@Test
public void testGetKey()
{
WebResource r = resource().path("/druid/v1/lookups/introspect/lookupId1/keys");
String s = r.get(String.class);
Assert.assertEquals("[key, key2]", s);
}
@Test
public void testGetValue()
{
WebResource r = resource().path("/druid/v1/lookups/introspect/lookupId1/values");
String s = r.get(String.class);
Assert.assertEquals("[value, value2]", s);
}
@Test
public void testGetMap()
{
WebResource r = resource().path("/druid/v1/lookups/introspect/lookupId1/");
String s = r.get(String.class);
Assert.assertEquals("{\"key\":\"value\",\"key2\":\"value2\"}", s);
}
}

View File

@ -0,0 +1,128 @@
/*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. Metamarkets 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 io.druid.query.lookup;
import com.google.common.collect.ImmutableMap;
import io.druid.query.extraction.MapLookupExtractor;
import org.easymock.EasyMock;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Ignore;
import org.junit.Test;
import javax.annotation.Nullable;
import javax.ws.rs.POST;
import javax.ws.rs.core.Response;
import java.io.InputStream;
public class LookupIntrospectionResourceTest
{
LookupReferencesManager lookupReferencesManager = EasyMock.createMock(LookupReferencesManager.class);
LookupExtractorFactory lookupExtractorFactory = EasyMock.createMock(LookupExtractorFactory.class);
LookupIntrospectHandler lookupIntrospectHandler = EasyMock.createMock(LookupIntrospectHandler.class);
LookupIntrospectionResource lookupIntrospectionResource = new LookupIntrospectionResource(lookupReferencesManager);
@Before
public void setUp()
{
EasyMock.expect(lookupReferencesManager.get("lookupId")).andReturn(lookupExtractorFactory).anyTimes();
EasyMock.expect(lookupReferencesManager.get(EasyMock.anyString())).andReturn(null).anyTimes();
EasyMock.replay(lookupReferencesManager);
}
@Test
public void testNotImplementedIntrospectLookup()
{
EasyMock.expect(lookupExtractorFactory.getIntrospectHandler()).andReturn(null);
EasyMock.expect(lookupExtractorFactory.get()).andReturn(new MapLookupExtractor(ImmutableMap.<String, String>of(), false)).anyTimes();
EasyMock.replay(lookupExtractorFactory);
Assert.assertEquals(Response.status(Response.Status.NOT_FOUND).build().getStatus(), ((Response) lookupIntrospectionResource.introspectLookup("lookupId")).getStatus());
}
@Test
public void testNotExistingLookup()
{
Assert.assertEquals(Response.status(Response.Status.NOT_FOUND).build().getStatus(), ((Response) lookupIntrospectionResource.introspectLookup("not there")).getStatus());
}
@Test public void testExistingLookup()
{
EasyMock.expect(lookupExtractorFactory.getIntrospectHandler()).andReturn(lookupIntrospectHandler);
EasyMock.expect(lookupExtractorFactory.get()).andReturn(new MapLookupExtractor(ImmutableMap.<String, String>of(), false)).anyTimes();
EasyMock.replay(lookupExtractorFactory);
Assert.assertEquals(lookupIntrospectHandler, lookupIntrospectionResource.introspectLookup("lookupId"));
}
@Test
@Ignore
public void testIntrospection()
{
LookupIntrospectHandler lookupIntrospectHandler = new LookupIntrospectHandler()
{
@POST
public Response postMock(InputStream inputStream){
return Response.ok().build();
}
};
LookupExtractorFactory lookupExtractorFactory1 = new LookupExtractorFactory()
{
final LookupExtractor mapLookup = new MapLookupExtractor(ImmutableMap.<String, String>of("key", "value"), true);
@Override
public boolean start()
{
return true;
}
@Override
public boolean close()
{
return true;
}
@Override
public boolean replaces(@Nullable LookupExtractorFactory other)
{
return true;
}
@Nullable
@Override
public LookupIntrospectHandler getIntrospectHandler()
{
return null;
}
@Override
public LookupExtractor get()
{
return mapLookup;
}
};
LookupIntrospectionResource lookupIntrospectionResource = new LookupIntrospectionResource(lookupReferencesManager);
EasyMock.expect(lookupReferencesManager.get("lookupId1")).andReturn(lookupExtractorFactory1).anyTimes();
EasyMock.replay(lookupReferencesManager);
}
}

View File

@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableMap;
import com.google.common.io.Files; import com.google.common.io.Files;
import com.metamx.common.ISE; import com.metamx.common.ISE;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.extraction.MapLookupExtractorFactory;
import org.easymock.EasyMock; import org.easymock.EasyMock;
import org.junit.After; import org.junit.After;
import org.junit.Assert; import org.junit.Assert;
@ -45,6 +44,7 @@ public class LookupReferencesManagerTest
@Before @Before
public void setUp() throws IOException public void setUp() throws IOException
{ {
mapper.registerSubtypes(MapLookupExtractorFactory.class);
lookupReferencesManager = new LookupReferencesManager(new LookupConfig(Files.createTempDir().getAbsolutePath()), mapper); lookupReferencesManager = new LookupReferencesManager(new LookupConfig(Files.createTempDir().getAbsolutePath()), mapper);
Assert.assertTrue("must be closed before start call", lookupReferencesManager.isClosed()); Assert.assertTrue("must be closed before start call", lookupReferencesManager.isClosed());
lookupReferencesManager.start(); lookupReferencesManager.start();
@ -260,5 +260,4 @@ public class LookupReferencesManagerTest
Assert.assertEquals(lookupExtractorFactory, lookupReferencesManager.get("testMockForBootstrap")); Assert.assertEquals(lookupExtractorFactory, lookupReferencesManager.get("testMockForBootstrap"));
} }
} }

View File

@ -25,7 +25,6 @@ import com.google.common.collect.Lists;
import com.google.common.io.Files; import com.google.common.io.Files;
import com.metamx.common.ISE; import com.metamx.common.ISE;
import com.metamx.common.StringUtils; import com.metamx.common.StringUtils;
import io.druid.query.extraction.MapLookupExtractorFactory;
import io.druid.segment.TestHelper; import io.druid.segment.TestHelper;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Before; import org.junit.Before;
@ -52,6 +51,7 @@ public class LookupSnapshotTakerTest
@Before @Before
public void setUp() throws IOException public void setUp() throws IOException
{ {
mapper.registerSubtypes(MapLookupExtractorFactory.class);
basePersistDirectory = temporaryFolder.newFolder().getAbsolutePath(); basePersistDirectory = temporaryFolder.newFolder().getAbsolutePath();
lookupSnapshotTaker = new LookupSnapshotTaker(mapper, basePersistDirectory); lookupSnapshotTaker = new LookupSnapshotTaker(mapper, basePersistDirectory);
} }

View File

@ -1,28 +1,27 @@
/* /*
* Licensed to Metamarkets Group Inc. (Metamarkets) under one * Licensed to Metamarkets Group Inc. (Metamarkets) under one
* or more contributor license agreements. See the NOTICE file * or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information * distributed with this work for additional information
* regarding copyright ownership. Metamarkets licenses this file * regarding copyright ownership. Metamarkets licenses this file
* to you under the Apache License, Version 2.0 (the * to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance * "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at * with the License. You may obtain a copy of the License at
* *
* http://www.apache.org/licenses/LICENSE-2.0 * http://www.apache.org/licenses/LICENSE-2.0
* *
* Unless required by applicable law or agreed to in writing, * Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an * software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the * KIND, either express or implied. See the License for the
* specific language governing permissions and limitations * specific language governing permissions and limitations
* under the License. * under the License.
*/ */
package io.druid.query.extraction; package io.druid.query.lookup;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import io.druid.jackson.DefaultObjectMapper; import io.druid.jackson.DefaultObjectMapper;
import io.druid.query.lookup.LookupExtractorFactory;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -56,6 +55,7 @@ public class MapLookupExtractorFactoryTest
public void testSerDeserMapLookupExtractorFactory() throws IOException public void testSerDeserMapLookupExtractorFactory() throws IOException
{ {
ObjectMapper mapper = new DefaultObjectMapper(); ObjectMapper mapper = new DefaultObjectMapper();
mapper.registerSubtypes(MapLookupExtractorFactory.class);
LookupExtractorFactory lookupExtractorFactory = new MapLookupExtractorFactory(ImmutableMap.of("key", "value"), true); LookupExtractorFactory lookupExtractorFactory = new MapLookupExtractorFactory(ImmutableMap.of("key", "value"), true);
Assert.assertEquals(lookupExtractorFactory, mapper.reader(LookupExtractorFactory.class).readValue(mapper.writeValueAsString(lookupExtractorFactory))); Assert.assertEquals(lookupExtractorFactory, mapper.reader(LookupExtractorFactory.class).readValue(mapper.writeValueAsString(lookupExtractorFactory)));
} }