mirror of https://github.com/apache/druid.git
Input source security feature should work for MSQ tasks (#14056)
### Description Previously msq controller and worker tasks did not have implementations for the `getInputSourceResources()` method. This causes the submission of these tasks to fail if the following auth config is enabled: `druid.auth.enableInputSourceSecurity=true` Added implementations of this method for these tasks that return an empty set of input sources. This means that for these task types, if `druid.auth.enableInputSourceSecurity=true` config is used, the input source types will be properly computed and authorized in the SQL layer, but not if the equivalent controller / worker tasks are submitted to the task endpoint.
This commit is contained in:
parent
d61bd7f8f1
commit
89bdbdc3ed
|
@ -21,9 +21,11 @@ package org.apache.druid.msq.indexing;
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||||
import com.fasterxml.jackson.annotation.JsonInclude;
|
import com.fasterxml.jackson.annotation.JsonInclude;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||||
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.inject.Injector;
|
import com.google.inject.Injector;
|
||||||
import com.google.inject.Key;
|
import com.google.inject.Key;
|
||||||
import org.apache.druid.guice.annotations.EscalatedGlobal;
|
import org.apache.druid.guice.annotations.EscalatedGlobal;
|
||||||
|
@ -45,11 +47,14 @@ import org.apache.druid.msq.exec.MSQTasks;
|
||||||
import org.apache.druid.rpc.ServiceClientFactory;
|
import org.apache.druid.rpc.ServiceClientFactory;
|
||||||
import org.apache.druid.rpc.StandardRetryPolicy;
|
import org.apache.druid.rpc.StandardRetryPolicy;
|
||||||
import org.apache.druid.rpc.indexing.OverlordClient;
|
import org.apache.druid.rpc.indexing.OverlordClient;
|
||||||
|
import org.apache.druid.server.security.ResourceAction;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import javax.annotation.Nonnull;
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
@JsonTypeName(MSQControllerTask.TYPE)
|
@JsonTypeName(MSQControllerTask.TYPE)
|
||||||
public class MSQControllerTask extends AbstractTask
|
public class MSQControllerTask extends AbstractTask
|
||||||
|
@ -109,6 +114,15 @@ public class MSQControllerTask extends AbstractTask
|
||||||
return TYPE;
|
return TYPE;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nonnull
|
||||||
|
@JsonIgnore
|
||||||
|
@Override
|
||||||
|
public Set<ResourceAction> getInputSourceResources()
|
||||||
|
{
|
||||||
|
// the input sources are properly computed in the SQL / calcite layer, but not in the native MSQ task here.
|
||||||
|
return ImmutableSet.of();
|
||||||
|
}
|
||||||
|
|
||||||
@JsonProperty("spec")
|
@JsonProperty("spec")
|
||||||
public MSQSpec getQuerySpec()
|
public MSQSpec getQuerySpec()
|
||||||
{
|
{
|
||||||
|
|
|
@ -21,9 +21,11 @@ package org.apache.druid.msq.indexing;
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
|
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeName;
|
import com.fasterxml.jackson.annotation.JsonTypeName;
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
|
import com.google.common.collect.ImmutableSet;
|
||||||
import com.google.inject.Injector;
|
import com.google.inject.Injector;
|
||||||
import org.apache.druid.indexer.TaskStatus;
|
import org.apache.druid.indexer.TaskStatus;
|
||||||
import org.apache.druid.indexing.common.TaskToolbox;
|
import org.apache.druid.indexing.common.TaskToolbox;
|
||||||
|
@ -35,9 +37,12 @@ import org.apache.druid.msq.exec.MSQTasks;
|
||||||
import org.apache.druid.msq.exec.Worker;
|
import org.apache.druid.msq.exec.Worker;
|
||||||
import org.apache.druid.msq.exec.WorkerContext;
|
import org.apache.druid.msq.exec.WorkerContext;
|
||||||
import org.apache.druid.msq.exec.WorkerImpl;
|
import org.apache.druid.msq.exec.WorkerImpl;
|
||||||
|
import org.apache.druid.server.security.ResourceAction;
|
||||||
|
|
||||||
|
import javax.annotation.Nonnull;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
@JsonTypeName(MSQWorkerTask.TYPE)
|
@JsonTypeName(MSQWorkerTask.TYPE)
|
||||||
public class MSQWorkerTask extends AbstractTask
|
public class MSQWorkerTask extends AbstractTask
|
||||||
|
@ -111,6 +116,15 @@ public class MSQWorkerTask extends AbstractTask
|
||||||
return TYPE;
|
return TYPE;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Nonnull
|
||||||
|
@JsonIgnore
|
||||||
|
@Override
|
||||||
|
public Set<ResourceAction> getInputSourceResources()
|
||||||
|
{
|
||||||
|
// the input sources are properly computed in the SQL / calcite layer, but not in the native MSQ task here.
|
||||||
|
return ImmutableSet.of();
|
||||||
|
}
|
||||||
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public boolean isReady(final TaskActionClient taskActionClient)
|
public boolean isReady(final TaskActionClient taskActionClient)
|
||||||
|
|
|
@ -0,0 +1,68 @@
|
||||||
|
/*
|
||||||
|
* 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.msq.indexing;
|
||||||
|
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
|
import org.apache.druid.query.Druids;
|
||||||
|
import org.apache.druid.query.scan.ScanQuery;
|
||||||
|
import org.apache.druid.query.spec.MultipleIntervalSegmentSpec;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.util.Collections;
|
||||||
|
|
||||||
|
public class MSQControllerTaskTest
|
||||||
|
{
|
||||||
|
MSQSpec MSQ_SPEC = MSQSpec
|
||||||
|
.builder()
|
||||||
|
.destination(new DataSourceMSQDestination(
|
||||||
|
"target",
|
||||||
|
Granularities.DAY,
|
||||||
|
null,
|
||||||
|
null
|
||||||
|
))
|
||||||
|
.query(new Druids.ScanQueryBuilder()
|
||||||
|
.resultFormat(ScanQuery.ResultFormat.RESULT_FORMAT_COMPACTED_LIST)
|
||||||
|
.legacy(false)
|
||||||
|
.intervals(new MultipleIntervalSegmentSpec(
|
||||||
|
Collections.singletonList(Intervals.of(
|
||||||
|
"2011-04-01T00:00:00.000Z/2011-04-03T00:00:00.000Z"))))
|
||||||
|
.dataSource("target")
|
||||||
|
.build()
|
||||||
|
)
|
||||||
|
.columnMappings(new ColumnMappings(ImmutableList.of(new ColumnMapping("a0", "cnt"))))
|
||||||
|
.tuningConfig(MSQTuningConfig.defaultConfig())
|
||||||
|
.build();
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetInputSourceResources()
|
||||||
|
{
|
||||||
|
MSQControllerTask msqWorkerTask = new MSQControllerTask(
|
||||||
|
null,
|
||||||
|
MSQ_SPEC,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null);
|
||||||
|
Assert.assertTrue(msqWorkerTask.getInputSourceResources().isEmpty());
|
||||||
|
}
|
||||||
|
}
|
|
@ -103,4 +103,11 @@ public class MSQWorkerTaskTest
|
||||||
Assert.assertEquals(retryCount, msqWorkerTask.getRetryCount());
|
Assert.assertEquals(retryCount, msqWorkerTask.getRetryCount());
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Test
|
||||||
|
public void testGetInputSourceResources()
|
||||||
|
{
|
||||||
|
MSQWorkerTask msqWorkerTask = new MSQWorkerTask(controllerTaskId, dataSource, workerNumber, context, retryCount);
|
||||||
|
Assert.assertTrue(msqWorkerTask.getInputSourceResources().isEmpty());
|
||||||
|
}
|
||||||
|
|
||||||
}
|
}
|
||||||
|
|
Loading…
Reference in New Issue