mirror of https://github.com/apache/druid.git
secure supervisor endpoints (#3985)
This commit is contained in:
parent
7e4e5510e0
commit
7e0d4c9555
|
@ -24,6 +24,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import io.druid.guice.annotations.Json;
|
||||
import io.druid.indexing.kafka.KafkaIndexTaskClientFactory;
|
||||
import io.druid.indexing.overlord.IndexerMetadataStorageCoordinator;
|
||||
|
@ -33,6 +34,7 @@ import io.druid.indexing.overlord.supervisor.Supervisor;
|
|||
import io.druid.indexing.overlord.supervisor.SupervisorSpec;
|
||||
import io.druid.segment.indexing.DataSchema;
|
||||
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class KafkaSupervisorSpec implements SupervisorSpec
|
||||
|
@ -134,6 +136,12 @@ public class KafkaSupervisorSpec implements SupervisorSpec
|
|||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getDataSources()
|
||||
{
|
||||
return ImmutableList.of(getDataSchema().getDataSource());
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
|
|
|
@ -0,0 +1,127 @@
|
|||
/*
|
||||
* 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.indexing.overlord.http.security;
|
||||
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.inject.Inject;
|
||||
import com.sun.jersey.spi.container.ContainerRequest;
|
||||
import io.druid.indexing.overlord.supervisor.SupervisorManager;
|
||||
import io.druid.indexing.overlord.supervisor.SupervisorSpec;
|
||||
import io.druid.server.http.security.AbstractResourceFilter;
|
||||
import io.druid.server.security.Access;
|
||||
import io.druid.server.security.AuthConfig;
|
||||
import io.druid.server.security.AuthorizationInfo;
|
||||
import io.druid.server.security.Resource;
|
||||
import io.druid.server.security.ResourceType;
|
||||
|
||||
import javax.ws.rs.WebApplicationException;
|
||||
import javax.ws.rs.core.PathSegment;
|
||||
import javax.ws.rs.core.Response;
|
||||
import java.util.List;
|
||||
|
||||
public class SupervisorResourceFilter extends AbstractResourceFilter
|
||||
{
|
||||
private final SupervisorManager supervisorManager;
|
||||
|
||||
@Inject
|
||||
public SupervisorResourceFilter(AuthConfig authConfig, SupervisorManager supervisorManager)
|
||||
{
|
||||
super(authConfig);
|
||||
this.supervisorManager = supervisorManager;
|
||||
}
|
||||
|
||||
@Override
|
||||
public ContainerRequest filter(ContainerRequest request)
|
||||
{
|
||||
if (getAuthConfig().isEnabled()) {
|
||||
// This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424
|
||||
final String supervisorId = Preconditions.checkNotNull(
|
||||
request.getPathSegments()
|
||||
.get(
|
||||
Iterables.indexOf(
|
||||
request.getPathSegments(),
|
||||
new Predicate<PathSegment>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(PathSegment input)
|
||||
{
|
||||
return input.getPath().equals("supervisor");
|
||||
}
|
||||
}
|
||||
) + 1
|
||||
).getPath()
|
||||
);
|
||||
|
||||
Optional<SupervisorSpec> supervisorSpecOptional = supervisorManager.getSupervisorSpec(supervisorId);
|
||||
if (!supervisorSpecOptional.isPresent()) {
|
||||
throw new WebApplicationException(
|
||||
Response.status(Response.Status.BAD_REQUEST)
|
||||
.entity(String.format("Cannot find any supervisor with id: [%s]", supervisorId))
|
||||
.build()
|
||||
);
|
||||
}
|
||||
|
||||
final AuthorizationInfo authorizationInfo = (AuthorizationInfo) getReq().getAttribute(AuthConfig.DRUID_AUTH_TOKEN);
|
||||
Preconditions.checkNotNull(
|
||||
authorizationInfo,
|
||||
"Security is enabled but no authorization info found in the request"
|
||||
);
|
||||
|
||||
final SupervisorSpec spec = supervisorSpecOptional.get();
|
||||
Preconditions.checkArgument(
|
||||
spec.getDataSources() != null && spec.getDataSources().size() > 0,
|
||||
"No dataSources found to perform authorization checks"
|
||||
);
|
||||
|
||||
for (String dataSource : spec.getDataSources()) {
|
||||
Access authResult = authorizationInfo.isAuthorized(
|
||||
new Resource(dataSource, ResourceType.DATASOURCE),
|
||||
getAction(request)
|
||||
);
|
||||
if (!authResult.isAllowed()) {
|
||||
throw new WebApplicationException(Response.status(Response.Status.FORBIDDEN)
|
||||
.entity(
|
||||
String.format("Access-Check-Result: %s", authResult.toString())
|
||||
)
|
||||
.build());
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return request;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean isApplicable(String requestPath)
|
||||
{
|
||||
List<String> applicablePaths = ImmutableList.of("druid/indexer/v1/supervisor/");
|
||||
for (String path : applicablePaths) {
|
||||
if (requestPath.startsWith(path) && !requestPath.equals(path)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
return false;
|
||||
}
|
||||
|
||||
}
|
|
@ -51,7 +51,8 @@ public class TaskResourceFilter extends AbstractResourceFilter
|
|||
private final TaskStorageQueryAdapter taskStorageQueryAdapter;
|
||||
|
||||
@Inject
|
||||
public TaskResourceFilter(TaskStorageQueryAdapter taskStorageQueryAdapter, AuthConfig authConfig) {
|
||||
public TaskResourceFilter(TaskStorageQueryAdapter taskStorageQueryAdapter, AuthConfig authConfig)
|
||||
{
|
||||
super(authConfig);
|
||||
this.taskStorageQueryAdapter = taskStorageQueryAdapter;
|
||||
}
|
||||
|
@ -114,7 +115,7 @@ public class TaskResourceFilter extends AbstractResourceFilter
|
|||
{
|
||||
List<String> applicablePaths = ImmutableList.of("druid/indexer/v1/task/");
|
||||
for (String path : applicablePaths) {
|
||||
if(requestPath.startsWith(path) && !requestPath.equals(path)) {
|
||||
if (requestPath.startsWith(path) && !requestPath.equals(path)) {
|
||||
return true;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,20 +21,36 @@ package io.druid.indexing.overlord.supervisor;
|
|||
|
||||
import com.google.common.base.Function;
|
||||
import com.google.common.base.Optional;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Iterables;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.common.collect.Sets;
|
||||
import com.google.inject.Inject;
|
||||
import com.sun.jersey.spi.container.ResourceFilters;
|
||||
import io.druid.indexing.overlord.TaskMaster;
|
||||
import io.druid.indexing.overlord.http.security.SupervisorResourceFilter;
|
||||
import io.druid.server.security.Access;
|
||||
import io.druid.server.security.Action;
|
||||
import io.druid.server.security.AuthConfig;
|
||||
import io.druid.server.security.AuthorizationInfo;
|
||||
import io.druid.server.security.Resource;
|
||||
import io.druid.server.security.ResourceType;
|
||||
|
||||
import javax.servlet.http.HttpServletRequest;
|
||||
import javax.ws.rs.Consumes;
|
||||
import javax.ws.rs.GET;
|
||||
import javax.ws.rs.POST;
|
||||
import javax.ws.rs.Path;
|
||||
import javax.ws.rs.PathParam;
|
||||
import javax.ws.rs.Produces;
|
||||
import javax.ws.rs.core.Context;
|
||||
import javax.ws.rs.core.MediaType;
|
||||
import javax.ws.rs.core.Response;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
* Endpoints for submitting and starting a {@link SupervisorSpec}, getting running supervisors, stopping supervisors,
|
||||
|
@ -44,17 +60,19 @@ import java.util.Map;
|
|||
public class SupervisorResource
|
||||
{
|
||||
private final TaskMaster taskMaster;
|
||||
private final AuthConfig authConfig;
|
||||
|
||||
@Inject
|
||||
public SupervisorResource(TaskMaster taskMaster)
|
||||
public SupervisorResource(TaskMaster taskMaster, AuthConfig authConfig)
|
||||
{
|
||||
this.taskMaster = taskMaster;
|
||||
this.authConfig = authConfig;
|
||||
}
|
||||
|
||||
@POST
|
||||
@Consumes(MediaType.APPLICATION_JSON)
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
public Response specPost(final SupervisorSpec spec)
|
||||
public Response specPost(final SupervisorSpec spec, @Context final HttpServletRequest req)
|
||||
{
|
||||
return asLeaderWithSupervisorManager(
|
||||
new Function<SupervisorManager, Response>()
|
||||
|
@ -62,6 +80,18 @@ public class SupervisorResource
|
|||
@Override
|
||||
public Response apply(SupervisorManager manager)
|
||||
{
|
||||
if (authConfig.isEnabled()) {
|
||||
// This is an experimental feature, see - https://github.com/druid-io/druid/pull/2424
|
||||
final AuthorizationInfo authorizationInfo = (AuthorizationInfo) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN);
|
||||
Preconditions.checkNotNull(
|
||||
authorizationInfo,
|
||||
"Security is enabled but no authorization info found in the request"
|
||||
);
|
||||
Access authResult = checkSupervisorAccess(authorizationInfo, spec);
|
||||
if (!authResult.isAllowed()) {
|
||||
return Response.status(Response.Status.FORBIDDEN).header("Access-Check-Result", authResult).build();
|
||||
}
|
||||
}
|
||||
manager.createOrUpdateAndStartSupervisor(spec);
|
||||
return Response.ok(ImmutableMap.of("id", spec.getId())).build();
|
||||
}
|
||||
|
@ -69,17 +99,63 @@ public class SupervisorResource
|
|||
);
|
||||
}
|
||||
|
||||
private Access checkSupervisorAccess(final AuthorizationInfo authorizationInfo, final SupervisorSpec spec)
|
||||
{
|
||||
Preconditions.checkArgument(
|
||||
spec.getDataSources() != null && spec.getDataSources().size() > 0,
|
||||
"No dataSources found to perform authorization checks"
|
||||
);
|
||||
Access result = new Access(true);
|
||||
for (String dataSource : spec.getDataSources()) {
|
||||
result = authorizationInfo.isAuthorized(
|
||||
new Resource(dataSource, ResourceType.DATASOURCE),
|
||||
Action.WRITE
|
||||
);
|
||||
if (!result.isAllowed()) {
|
||||
return result;
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
@GET
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
public Response specGetAll()
|
||||
public Response specGetAll(@Context final HttpServletRequest req)
|
||||
{
|
||||
return asLeaderWithSupervisorManager(
|
||||
new Function<SupervisorManager, Response>()
|
||||
{
|
||||
@Override
|
||||
public Response apply(SupervisorManager manager)
|
||||
public Response apply(final SupervisorManager manager)
|
||||
{
|
||||
return Response.ok(manager.getSupervisorIds()).build();
|
||||
final Set<String> supervisorIds;
|
||||
if (authConfig.isEnabled()) {
|
||||
final AuthorizationInfo authorizationInfo = (AuthorizationInfo) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN);
|
||||
Preconditions.checkNotNull(
|
||||
authorizationInfo,
|
||||
"Security is enabled but no authorization info found in the request"
|
||||
);
|
||||
supervisorIds = Sets.newHashSet(
|
||||
Iterables.filter(
|
||||
manager.getSupervisorIds(),
|
||||
new Predicate<String>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(String id)
|
||||
{
|
||||
return manager.getSupervisorSpec(id).isPresent() &&
|
||||
checkSupervisorAccess(
|
||||
authorizationInfo,
|
||||
manager.getSupervisorSpec(id).get()
|
||||
).isAllowed();
|
||||
}
|
||||
}
|
||||
)
|
||||
);
|
||||
} else {
|
||||
supervisorIds = manager.getSupervisorIds();
|
||||
}
|
||||
return Response.ok(supervisorIds).build();
|
||||
}
|
||||
}
|
||||
);
|
||||
|
@ -88,6 +164,7 @@ public class SupervisorResource
|
|||
@GET
|
||||
@Path("/{id}")
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@ResourceFilters(SupervisorResourceFilter.class)
|
||||
public Response specGet(@PathParam("id") final String id)
|
||||
{
|
||||
return asLeaderWithSupervisorManager(
|
||||
|
@ -112,6 +189,7 @@ public class SupervisorResource
|
|||
@GET
|
||||
@Path("/{id}/status")
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@ResourceFilters(SupervisorResourceFilter.class)
|
||||
public Response specGetStatus(@PathParam("id") final String id)
|
||||
{
|
||||
return asLeaderWithSupervisorManager(
|
||||
|
@ -136,6 +214,7 @@ public class SupervisorResource
|
|||
@POST
|
||||
@Path("/{id}/shutdown")
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@ResourceFilters(SupervisorResourceFilter.class)
|
||||
public Response shutdown(@PathParam("id") final String id)
|
||||
{
|
||||
return asLeaderWithSupervisorManager(
|
||||
|
@ -159,15 +238,40 @@ public class SupervisorResource
|
|||
@GET
|
||||
@Path("/history")
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
public Response specGetAllHistory()
|
||||
public Response specGetAllHistory(@Context final HttpServletRequest req)
|
||||
{
|
||||
return asLeaderWithSupervisorManager(
|
||||
new Function<SupervisorManager, Response>()
|
||||
{
|
||||
@Override
|
||||
public Response apply(SupervisorManager manager)
|
||||
public Response apply(final SupervisorManager manager)
|
||||
{
|
||||
return Response.ok(manager.getSupervisorHistory()).build();
|
||||
final Map<String, List<VersionedSupervisorSpec>> supervisorHistory;
|
||||
if (authConfig.isEnabled()) {
|
||||
final AuthorizationInfo authorizationInfo = (AuthorizationInfo) req.getAttribute(AuthConfig.DRUID_AUTH_TOKEN);
|
||||
Preconditions.checkNotNull(
|
||||
authorizationInfo,
|
||||
"Security is enabled but no authorization info found in the request"
|
||||
);
|
||||
supervisorHistory = Maps.filterKeys(
|
||||
manager.getSupervisorHistory(),
|
||||
new Predicate<String>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(String id)
|
||||
{
|
||||
return manager.getSupervisorSpec(id).isPresent() &&
|
||||
checkSupervisorAccess(
|
||||
authorizationInfo,
|
||||
manager.getSupervisorSpec(id).get()
|
||||
).isAllowed();
|
||||
}
|
||||
}
|
||||
);
|
||||
} else {
|
||||
supervisorHistory = manager.getSupervisorHistory();
|
||||
}
|
||||
return Response.ok(supervisorHistory).build();
|
||||
}
|
||||
}
|
||||
);
|
||||
|
@ -176,6 +280,7 @@ public class SupervisorResource
|
|||
@GET
|
||||
@Path("/{id}/history")
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@ResourceFilters(SupervisorResourceFilter.class)
|
||||
public Response specGetHistory(@PathParam("id") final String id)
|
||||
{
|
||||
return asLeaderWithSupervisorManager(
|
||||
|
@ -209,6 +314,7 @@ public class SupervisorResource
|
|||
@POST
|
||||
@Path("/{id}/reset")
|
||||
@Produces(MediaType.APPLICATION_JSON)
|
||||
@ResourceFilters(SupervisorResourceFilter.class)
|
||||
public Response reset(@PathParam("id") final String id)
|
||||
{
|
||||
return asLeaderWithSupervisorManager(
|
||||
|
|
|
@ -28,6 +28,10 @@ import io.druid.indexing.common.task.NoopTask;
|
|||
import io.druid.indexing.common.task.Task;
|
||||
import io.druid.indexing.overlord.TaskStorageQueryAdapter;
|
||||
import io.druid.indexing.overlord.http.OverlordResource;
|
||||
import io.druid.indexing.overlord.supervisor.Supervisor;
|
||||
import io.druid.indexing.overlord.supervisor.SupervisorManager;
|
||||
import io.druid.indexing.overlord.supervisor.SupervisorResource;
|
||||
import io.druid.indexing.overlord.supervisor.SupervisorSpec;
|
||||
import io.druid.indexing.worker.http.WorkerResource;
|
||||
import io.druid.server.http.security.AbstractResourceFilter;
|
||||
import io.druid.server.http.security.ResourceFilterTestHelper;
|
||||
|
@ -42,18 +46,20 @@ import org.junit.runners.Parameterized;
|
|||
import javax.ws.rs.WebApplicationException;
|
||||
import javax.ws.rs.core.Response;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
|
||||
@RunWith(Parameterized.class)
|
||||
public class SecurityResourceFilterTest extends ResourceFilterTestHelper
|
||||
{
|
||||
|
||||
@Parameterized.Parameters
|
||||
@Parameterized.Parameters(name = "{index}: requestPath={0}, requestMethod={1}, resourceFilter={2}")
|
||||
public static Collection<Object[]> data()
|
||||
{
|
||||
return ImmutableList.copyOf(
|
||||
Iterables.concat(
|
||||
getRequestPaths(OverlordResource.class, ImmutableList.<Class<?>>of(TaskStorageQueryAdapter.class)),
|
||||
getRequestPaths(WorkerResource.class)
|
||||
getRequestPaths(WorkerResource.class),
|
||||
getRequestPaths(SupervisorResource.class, ImmutableList.<Class<?>>of(SupervisorManager.class))
|
||||
)
|
||||
);
|
||||
}
|
||||
|
@ -64,8 +70,10 @@ public class SecurityResourceFilterTest extends ResourceFilterTestHelper
|
|||
private final Injector injector;
|
||||
private final Task noopTask = new NoopTask(null, 0, 0, null, null, null);
|
||||
|
||||
private static boolean mockedOnce;
|
||||
private static boolean mockedOnceTsqa;
|
||||
private static boolean mockedOnceSM;
|
||||
private TaskStorageQueryAdapter tsqa;
|
||||
private SupervisorManager supervisorManager;
|
||||
|
||||
public SecurityResourceFilterTest(
|
||||
String requestPath,
|
||||
|
@ -83,20 +91,48 @@ public class SecurityResourceFilterTest extends ResourceFilterTestHelper
|
|||
@Before
|
||||
public void setUp() throws Exception
|
||||
{
|
||||
if (resourceFilter instanceof TaskResourceFilter && !mockedOnce) {
|
||||
if (resourceFilter instanceof TaskResourceFilter && !mockedOnceTsqa) {
|
||||
// Since we are creating the mocked tsqa object only once and getting that object from Guice here therefore
|
||||
// if the mockedOnce check is not done then we will call EasyMock.expect and EasyMock.replay on the mocked object
|
||||
// multiple times and it will throw exceptions
|
||||
tsqa = injector.getInstance(TaskStorageQueryAdapter.class);
|
||||
EasyMock.expect(tsqa.getTask(EasyMock.anyString())).andReturn(Optional.of(noopTask)).anyTimes();
|
||||
EasyMock.replay(tsqa);
|
||||
mockedOnce = true;
|
||||
mockedOnceTsqa = true;
|
||||
}
|
||||
if (resourceFilter instanceof SupervisorResourceFilter && !mockedOnceSM) {
|
||||
supervisorManager = injector.getInstance(SupervisorManager.class);
|
||||
SupervisorSpec supervisorSpec = new SupervisorSpec()
|
||||
{
|
||||
@Override
|
||||
public String getId()
|
||||
{
|
||||
return "id";
|
||||
}
|
||||
|
||||
@Override
|
||||
public Supervisor createSupervisor()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getDataSources()
|
||||
{
|
||||
return ImmutableList.of("test");
|
||||
}
|
||||
};
|
||||
EasyMock.expect(supervisorManager.getSupervisorSpec(EasyMock.anyString()))
|
||||
.andReturn(Optional.of(supervisorSpec))
|
||||
.anyTimes();
|
||||
EasyMock.replay(supervisorManager);
|
||||
mockedOnceSM = true;
|
||||
}
|
||||
setUp(resourceFilter);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testDatasourcesResourcesFilteringAccess()
|
||||
public void testResourcesFilteringAccess()
|
||||
{
|
||||
setUpMockExpectations(requestPath, true, requestMethod);
|
||||
EasyMock.expect(request.getEntity(Task.class)).andReturn(noopTask).anyTimes();
|
||||
|
@ -113,7 +149,6 @@ public class SecurityResourceFilterTest extends ResourceFilterTestHelper
|
|||
{
|
||||
setUpMockExpectations(requestPath, false, requestMethod);
|
||||
EasyMock.expect(request.getEntity(Task.class)).andReturn(noopTask).anyTimes();
|
||||
EasyMock.expect(request.getMethod()).andReturn(requestMethod).anyTimes();
|
||||
EasyMock.replay(req, request, authorizationInfo);
|
||||
Assert.assertTrue(((AbstractResourceFilter) resourceFilter.getRequestFilter()).isApplicable(requestPath));
|
||||
try {
|
||||
|
@ -141,6 +176,9 @@ public class SecurityResourceFilterTest extends ResourceFilterTestHelper
|
|||
if (tsqa != null) {
|
||||
EasyMock.verify(tsqa);
|
||||
}
|
||||
if (supervisorManager != null) {
|
||||
EasyMock.verify(supervisorManager);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
||||
|
|
|
@ -291,5 +291,12 @@ public class SupervisorManagerTest extends EasyMockSupport
|
|||
{
|
||||
return supervisor;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getDataSources()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -27,6 +27,7 @@ import com.google.common.collect.Maps;
|
|||
import io.druid.indexing.overlord.DataSourceMetadata;
|
||||
import io.druid.indexing.overlord.TaskMaster;
|
||||
import org.easymock.Capture;
|
||||
import io.druid.server.security.AuthConfig;
|
||||
import org.easymock.EasyMock;
|
||||
import org.easymock.EasyMockRunner;
|
||||
import org.easymock.EasyMockSupport;
|
||||
|
@ -37,6 +38,7 @@ import org.junit.Before;
|
|||
import org.junit.Test;
|
||||
import org.junit.runner.RunWith;
|
||||
|
||||
import javax.servlet.http.HttpServletRequest;
|
||||
import javax.ws.rs.core.Response;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
@ -51,12 +53,15 @@ public class SupervisorResourceTest extends EasyMockSupport
|
|||
@Mock
|
||||
private SupervisorManager supervisorManager;
|
||||
|
||||
@Mock
|
||||
private HttpServletRequest request;
|
||||
|
||||
private SupervisorResource supervisorResource;
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception
|
||||
{
|
||||
supervisorResource = new SupervisorResource(taskMaster);
|
||||
supervisorResource = new SupervisorResource(taskMaster, new AuthConfig());
|
||||
}
|
||||
|
||||
@Test
|
||||
|
@ -68,7 +73,7 @@ public class SupervisorResourceTest extends EasyMockSupport
|
|||
EasyMock.expect(supervisorManager.createOrUpdateAndStartSupervisor(spec)).andReturn(true);
|
||||
replayAll();
|
||||
|
||||
Response response = supervisorResource.specPost(spec);
|
||||
Response response = supervisorResource.specPost(spec, request);
|
||||
verifyAll();
|
||||
|
||||
Assert.assertEquals(200, response.getStatus());
|
||||
|
@ -78,7 +83,7 @@ public class SupervisorResourceTest extends EasyMockSupport
|
|||
EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.<SupervisorManager>absent());
|
||||
replayAll();
|
||||
|
||||
response = supervisorResource.specPost(spec);
|
||||
response = supervisorResource.specPost(spec, request);
|
||||
verifyAll();
|
||||
|
||||
Assert.assertEquals(503, response.getStatus());
|
||||
|
@ -93,7 +98,7 @@ public class SupervisorResourceTest extends EasyMockSupport
|
|||
EasyMock.expect(supervisorManager.getSupervisorIds()).andReturn(supervisorIds);
|
||||
replayAll();
|
||||
|
||||
Response response = supervisorResource.specGetAll();
|
||||
Response response = supervisorResource.specGetAll(request);
|
||||
verifyAll();
|
||||
|
||||
Assert.assertEquals(200, response.getStatus());
|
||||
|
@ -103,7 +108,7 @@ public class SupervisorResourceTest extends EasyMockSupport
|
|||
EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.<SupervisorManager>absent());
|
||||
replayAll();
|
||||
|
||||
response = supervisorResource.specGetAll();
|
||||
response = supervisorResource.specGetAll(request);
|
||||
verifyAll();
|
||||
|
||||
Assert.assertEquals(503, response.getStatus());
|
||||
|
@ -218,7 +223,7 @@ public class SupervisorResourceTest extends EasyMockSupport
|
|||
EasyMock.expect(supervisorManager.getSupervisorHistory()).andReturn(history);
|
||||
replayAll();
|
||||
|
||||
Response response = supervisorResource.specGetAllHistory();
|
||||
Response response = supervisorResource.specGetAllHistory(request);
|
||||
|
||||
Assert.assertEquals(200, response.getStatus());
|
||||
Assert.assertEquals(history, response.getEntity());
|
||||
|
@ -228,7 +233,7 @@ public class SupervisorResourceTest extends EasyMockSupport
|
|||
EasyMock.expect(taskMaster.getSupervisorManager()).andReturn(Optional.<SupervisorManager>absent());
|
||||
replayAll();
|
||||
|
||||
response = supervisorResource.specGetAllHistory();
|
||||
response = supervisorResource.specGetAllHistory(request);
|
||||
verifyAll();
|
||||
|
||||
Assert.assertEquals(503, response.getStatus());
|
||||
|
@ -324,5 +329,11 @@ public class SupervisorResourceTest extends EasyMockSupport
|
|||
{
|
||||
return supervisor;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getDataSources()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -20,6 +20,7 @@
|
|||
package io.druid.indexing.overlord.supervisor;
|
||||
|
||||
import io.druid.indexing.overlord.DataSourceMetadata;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Used as a tombstone marker in the supervisors metadata table to indicate that the supervisor has been removed.
|
||||
|
@ -53,4 +54,10 @@ public class NoopSupervisorSpec implements SupervisorSpec
|
|||
public void reset(DataSourceMetadata dataSourceMetadata) {}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getDataSources()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -22,6 +22,8 @@ package io.druid.indexing.overlord.supervisor;
|
|||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
|
||||
@JsonSubTypes(value = {
|
||||
@JsonSubTypes.Type(name = "NoopSupervisorSpec", value = NoopSupervisorSpec.class)
|
||||
|
@ -31,4 +33,6 @@ public interface SupervisorSpec
|
|||
String getId();
|
||||
|
||||
Supervisor createSupervisor();
|
||||
|
||||
List<String> getDataSources();
|
||||
}
|
||||
|
|
|
@ -24,6 +24,8 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||
import io.druid.indexing.overlord.supervisor.Supervisor;
|
||||
import io.druid.indexing.overlord.supervisor.SupervisorSpec;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class TestSupervisorSpec implements SupervisorSpec
|
||||
{
|
||||
private String id;
|
||||
|
@ -49,6 +51,12 @@ public class TestSupervisorSpec implements SupervisorSpec
|
|||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getDataSources()
|
||||
{
|
||||
return null;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public Object getData()
|
||||
{
|
||||
|
|
|
@ -99,7 +99,7 @@ public class SecurityResourceFilterTest extends ResourceFilterTestHelper
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testDatasourcesResourcesFilteringAccess()
|
||||
public void testResourcesFilteringAccess()
|
||||
{
|
||||
setUpMockExpectations(requestPath, true, requestMethod);
|
||||
EasyMock.replay(req, request, authorizationInfo);
|
||||
|
@ -109,11 +109,11 @@ public class SecurityResourceFilterTest extends ResourceFilterTestHelper
|
|||
}
|
||||
|
||||
@Test(expected = WebApplicationException.class)
|
||||
public void testDatasourcesResourcesFilteringNoAccess()
|
||||
public void testResourcesFilteringNoAccess()
|
||||
{
|
||||
setUpMockExpectations(requestPath, false, requestMethod);
|
||||
EasyMock.replay(req, request, authorizationInfo);
|
||||
//Assert.assertTrue(((AbstractResourceFilter) resourceFilter.getRequestFilter()).isApplicable(requestPath));
|
||||
Assert.assertTrue(((AbstractResourceFilter) resourceFilter.getRequestFilter()).isApplicable(requestPath));
|
||||
try {
|
||||
resourceFilter.getRequestFilter().filter(request);
|
||||
}
|
||||
|
@ -125,7 +125,7 @@ public class SecurityResourceFilterTest extends ResourceFilterTestHelper
|
|||
}
|
||||
|
||||
@Test
|
||||
public void testDatasourcesResourcesFilteringBadPath()
|
||||
public void testResourcesFilteringBadPath()
|
||||
{
|
||||
EasyMock.replay(req, request, authorizationInfo);
|
||||
final String badRequestPath = requestPath.replaceAll("\\w+", "droid");
|
||||
|
|
Loading…
Reference in New Issue