Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Disable loading lookups by default in CompactionTask #16420

Merged
Merged
Show file tree
Hide file tree
Changes from 7 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Jump to
Jump to file
Failed to load files.
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -55,7 +55,7 @@
import org.apache.druid.rpc.indexing.OverlordClient;
import org.apache.druid.segment.realtime.firehose.ChatHandler;
import org.apache.druid.server.DruidNode;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.apache.druid.server.lookup.cache.LookupLoadingSpec;

import java.util.Map;
import java.util.concurrent.ThreadLocalRandom;
Expand Down Expand Up @@ -271,16 +271,16 @@ public static Map<String, Object> makeTaskContext(
.put(MultiStageQueryContext.CTX_MAX_CONCURRENT_STAGES, queryKernelConfig.getMaxConcurrentStages());

// Put the lookup loading info in the task context to facilitate selective loading of lookups.
if (controllerTaskContext.get(PlannerContext.CTX_LOOKUP_LOADING_MODE) != null) {
if (controllerTaskContext.get(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE) != null) {
taskContextOverridesBuilder.put(
PlannerContext.CTX_LOOKUP_LOADING_MODE,
controllerTaskContext.get(PlannerContext.CTX_LOOKUP_LOADING_MODE)
LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE,
controllerTaskContext.get(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE)
);
}
if (controllerTaskContext.get(PlannerContext.CTX_LOOKUPS_TO_LOAD) != null) {
if (controllerTaskContext.get(LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD) != null) {
taskContextOverridesBuilder.put(
PlannerContext.CTX_LOOKUPS_TO_LOAD,
controllerTaskContext.get(PlannerContext.CTX_LOOKUPS_TO_LOAD)
LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD,
controllerTaskContext.get(LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD)
);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableSet;
import com.google.inject.Injector;
import org.apache.druid.error.InvalidInput;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.TaskToolbox;
import org.apache.druid.indexing.common.actions.TaskActionClient;
Expand All @@ -38,13 +37,9 @@
import org.apache.druid.msq.exec.Worker;
import org.apache.druid.msq.exec.WorkerContext;
import org.apache.druid.msq.exec.WorkerImpl;
import org.apache.druid.server.lookup.cache.LookupLoadingSpec;
import org.apache.druid.server.security.ResourceAction;
import org.apache.druid.sql.calcite.planner.PlannerContext;

import javax.annotation.Nonnull;
import java.util.Collection;
import java.util.HashSet;
import java.util.Map;
import java.util.Objects;
import java.util.Set;
Expand Down Expand Up @@ -190,26 +185,4 @@ public int hashCode()
{
return Objects.hash(super.hashCode(), controllerTaskId, workerNumber, retryCount, worker);
}

@Override
public LookupLoadingSpec getLookupLoadingSpec()
{
final Object lookupModeValue = getContext().get(PlannerContext.CTX_LOOKUP_LOADING_MODE);
if (lookupModeValue == null) {
return LookupLoadingSpec.ALL;
}

final LookupLoadingSpec.Mode lookupLoadingMode = LookupLoadingSpec.Mode.valueOf(lookupModeValue.toString());
if (lookupLoadingMode == LookupLoadingSpec.Mode.NONE) {
return LookupLoadingSpec.NONE;
} else if (lookupLoadingMode == LookupLoadingSpec.Mode.ONLY_REQUIRED) {
Collection<String> lookupsToLoad = (Collection<String>) getContext().get(PlannerContext.CTX_LOOKUPS_TO_LOAD);
if (lookupsToLoad == null || lookupsToLoad.isEmpty()) {
throw InvalidInput.exception("Set of lookups to load cannot be %s for mode[ONLY_REQUIRED].", lookupsToLoad);
}
return LookupLoadingSpec.loadOnly(new HashSet<>(lookupsToLoad));
} else {
return LookupLoadingSpec.ALL;
}
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -285,9 +285,9 @@ public QueryResponse<Object[]> runQuery(final DruidQuery druidQuery)
MSQTaskQueryMakerUtils.validateRealtimeReindex(querySpec);

final Map<String, Object> context = new HashMap<>();
context.put(PlannerContext.CTX_LOOKUP_LOADING_MODE, plannerContext.getLookupLoadingSpec().getMode());
context.put(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, plannerContext.getLookupLoadingSpec().getMode());
if (plannerContext.getLookupLoadingSpec().getMode() == LookupLoadingSpec.Mode.ONLY_REQUIRED) {
context.put(PlannerContext.CTX_LOOKUPS_TO_LOAD, plannerContext.getLookupLoadingSpec().getLookupsToLoad());
context.put(LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD, plannerContext.getLookupLoadingSpec().getLookupsToLoad());
}

final MSQControllerTask controllerTask = new MSQControllerTask(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -38,7 +38,6 @@
import org.apache.druid.server.lookup.cache.LookupLoadingSpec;
import org.apache.druid.sql.calcite.planner.ColumnMapping;
import org.apache.druid.sql.calcite.planner.ColumnMappings;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Test;
Expand Down Expand Up @@ -114,8 +113,8 @@ public void testGetLookupLoadingSpecUsingLookupLoadingInfoInContext()
.dataSource("target")
.context(
ImmutableMap.of(
PlannerContext.CTX_LOOKUPS_TO_LOAD, Arrays.asList("lookupName1", "lookupName2"),
PlannerContext.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.ONLY_REQUIRED)
LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD, Arrays.asList("lookupName1", "lookupName2"),
LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.ONLY_REQUIRED)
)
.build()
)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,7 +23,6 @@
import com.google.common.collect.ImmutableSet;
import org.apache.druid.error.DruidException;
import org.apache.druid.server.lookup.cache.LookupLoadingSpec;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.junit.Assert;
import org.junit.Test;

Expand Down Expand Up @@ -125,7 +124,7 @@ public void testGetDefaultLookupLoadingSpec()
@Test
public void testGetLookupLoadingWithModeNoneInContext()
{
final ImmutableMap<String, Object> context = ImmutableMap.of(PlannerContext.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.NONE);
final ImmutableMap<String, Object> context = ImmutableMap.of(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.NONE);
MSQWorkerTask msqWorkerTask = new MSQWorkerTask(controllerTaskId, dataSource, workerNumber, context, retryCount);
Assert.assertEquals(LookupLoadingSpec.NONE, msqWorkerTask.getLookupLoadingSpec());
}
Expand All @@ -134,8 +133,8 @@ public void testGetLookupLoadingWithModeNoneInContext()
public void testGetLookupLoadingSpecWithLookupListInContext()
{
final ImmutableMap<String, Object> context = ImmutableMap.of(
PlannerContext.CTX_LOOKUPS_TO_LOAD, Arrays.asList("lookupName1", "lookupName2"),
PlannerContext.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.ONLY_REQUIRED);
LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD, Arrays.asList("lookupName1", "lookupName2"),
LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.ONLY_REQUIRED);
MSQWorkerTask msqWorkerTask = new MSQWorkerTask(controllerTaskId, dataSource, workerNumber, context, retryCount);
Assert.assertEquals(LookupLoadingSpec.Mode.ONLY_REQUIRED, msqWorkerTask.getLookupLoadingSpec().getMode());
Assert.assertEquals(ImmutableSet.of("lookupName1", "lookupName2"), msqWorkerTask.getLookupLoadingSpec().getLookupsToLoad());
Expand All @@ -145,10 +144,10 @@ public void testGetLookupLoadingSpecWithLookupListInContext()
public void testGetLookupLoadingSpecWithInvalidInput()
{
final HashMap<String, Object> context = new HashMap<>();
context.put(PlannerContext.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.ONLY_REQUIRED);
context.put(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.ONLY_REQUIRED);

// Setting CTX_LOOKUPS_TO_LOAD as null
context.put(PlannerContext.CTX_LOOKUPS_TO_LOAD, null);
context.put(LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD, null);

MSQWorkerTask taskWithNullLookups = new MSQWorkerTask(controllerTaskId, dataSource, workerNumber, context, retryCount);
DruidException exception = Assert.assertThrows(
Expand All @@ -160,7 +159,7 @@ public void testGetLookupLoadingSpecWithInvalidInput()
exception.getMessage());

// Setting CTX_LOOKUPS_TO_LOAD as empty list
context.put(PlannerContext.CTX_LOOKUPS_TO_LOAD, Collections.emptyList());
context.put(LookupLoadingSpec.CTX_LOOKUPS_TO_LOAD, Collections.emptyList());

MSQWorkerTask taskWithEmptyLookups = new MSQWorkerTask(controllerTaskId, dataSource, workerNumber, context, retryCount);
exception = Assert.assertThrows(
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -854,7 +854,7 @@ public abstract class MSQTester<Builder extends MSQTester<Builder>>
protected CompactionState expectedLastCompactionState = null;
protected Set<Interval> expectedTombstoneIntervals = null;
protected List<Object[]> expectedResultRows = null;
protected LookupLoadingSpec expectedLookupLoadingSpec = null;
protected LookupLoadingSpec expectedLookupLoadingSpec = LookupLoadingSpec.NONE;
protected Matcher<Throwable> expectedValidationErrorMatcher = null;
protected List<Pair<Predicate<MSQTaskReportPayload>, String>> adhocReportAssertionAndReasons = new ArrayList<>();
protected Matcher<Throwable> expectedExecutionErrorMatcher = null;
Expand Down Expand Up @@ -1020,19 +1020,8 @@ public void verifyPlanningErrors()

protected void verifyLookupLoadingInfoInTaskContext(Map<String, Object> context)
{
String lookupLoadingMode = context.get(PlannerContext.CTX_LOOKUP_LOADING_MODE).toString();
List<String> lookupsToLoad = (List<String>) context.get(PlannerContext.CTX_LOOKUPS_TO_LOAD);
if (expectedLookupLoadingSpec != null) {
Assert.assertEquals(expectedLookupLoadingSpec.getMode().toString(), lookupLoadingMode);
if (expectedLookupLoadingSpec.getMode().equals(LookupLoadingSpec.Mode.ONLY_REQUIRED)) {
Assert.assertEquals(new ArrayList<>(expectedLookupLoadingSpec.getLookupsToLoad()), lookupsToLoad);
} else {
Assert.assertNull(lookupsToLoad);
}
} else {
Assert.assertEquals(LookupLoadingSpec.Mode.NONE.toString(), lookupLoadingMode);
Assert.assertNull(lookupsToLoad);
}
LookupLoadingSpec specFromContext = LookupLoadingSpec.createFromContext(context, LookupLoadingSpec.ALL);
Assert.assertEquals(expectedLookupLoadingSpec, specFromContext);
}

protected void verifyWorkerCount(CounterSnapshotsTree counterSnapshotsTree)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -92,6 +92,7 @@
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
import org.apache.druid.server.coordinator.duty.CompactSegments;
import org.apache.druid.server.lookup.cache.LookupLoadingSpec;
import org.apache.druid.server.security.ResourceAction;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentTimeline;
Expand Down Expand Up @@ -249,6 +250,9 @@ public CompactionTask(
this.segmentProvider = new SegmentProvider(dataSource, this.ioConfig.getInputSpec());
this.partitionConfigurationManager = new PartitionConfigurationManager(this.tuningConfig);
this.segmentCacheManagerFactory = segmentCacheManagerFactory;

// By default, do not load any lookups in sub-tasks launched by compaction task.
addToContextIfAbsent(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.NONE.toString());
}

@VisibleForTesting
Expand Down Expand Up @@ -1522,4 +1526,10 @@ public CompactionTuningConfig withPartitionsSpec(PartitionsSpec partitionsSpec)
);
}
}

@Override
public LookupLoadingSpec getLookupLoadingSpec()
{
return LookupLoadingSpec.createFromContext(getContext(), LookupLoadingSpec.NONE);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -334,9 +334,14 @@ static TaskInfo<TaskIdentifier, TaskStatus> toTaskIdentifierInfo(TaskInfo<Task,
);
}

/**
* Specifies the list of lookups to load for this task. Tasks load ALL lookups by default.
* This behaviour can be overridden by passing parameters {@link LookupLoadingSpec#CTX_LOOKUP_LOADING_MODE}
* and {@link LookupLoadingSpec#CTX_LOOKUPS_TO_LOAD} in the task context.
*/
@Nullable
default LookupLoadingSpec getLookupLoadingSpec()
{
return LookupLoadingSpec.ALL;
return LookupLoadingSpec.createFromContext(getContext(), LookupLoadingSpec.ALL);
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -63,6 +63,7 @@
import org.apache.druid.segment.realtime.firehose.ChatHandlerProvider;
import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider;
import org.apache.druid.segment.writeout.TmpFileSegmentWriteOutMediumFactory;
import org.apache.druid.server.lookup.cache.LookupLoadingSpec;
import org.apache.druid.server.security.AuthTestUtils;
import org.apache.druid.server.security.AuthorizerMapper;
import org.joda.time.Duration;
Expand All @@ -71,6 +72,7 @@

import java.io.IOException;
import java.util.HashMap;
import java.util.Map;

public class ClientCompactionTaskQuerySerdeTest
{
Expand All @@ -82,6 +84,7 @@ public class ClientCompactionTaskQuerySerdeTest
@Test
public void testClientCompactionTaskQueryToCompactionTask() throws IOException
{
final Map<String, Object> context = ImmutableMap.of("key", "value");
final ObjectMapper mapper = setupInjectablesInObjectMapper(new DefaultObjectMapper());
final ClientCompactionTaskQuery query = new ClientCompactionTaskQuery(
"id",
Expand Down Expand Up @@ -127,7 +130,7 @@ public void testClientCompactionTaskQueryToCompactionTask() throws IOException
new ClientCompactionTaskDimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))),
new AggregatorFactory[] {new CountAggregatorFactory("cnt")},
new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null)),
ImmutableMap.of("key", "value")
context
);

final byte[] json = mapper.writeValueAsBytes(query);
Expand Down Expand Up @@ -220,7 +223,6 @@ public void testClientCompactionTaskQueryToCompactionTask() throws IOException
query.getIoConfig().isDropExisting(),
task.getIoConfig().isDropExisting()
);
Assert.assertEquals(query.getContext(), task.getContext());
Assert.assertEquals(
query.getDimensionsSpec().getDimensions(),
task.getDimensionsSpec().getDimensions()
Expand All @@ -233,6 +235,13 @@ public void testClientCompactionTaskQueryToCompactionTask() throws IOException
query.getMetricsSpec(),
task.getMetricsSpec()
);

// Verify values of context keys originally present in the ClientCompactionTaskQuery
for (String key : context.keySet()) {
Akshat-Jain marked this conversation as resolved.
Show resolved Hide resolved
Assert.assertEquals(context.get(key), task.getContext().get(key));
}
// Verify values of context parameters added by the CompactionTask
Assert.assertEquals(LookupLoadingSpec.Mode.NONE.toString(), task.getContext().get(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE));
Akshat-Jain marked this conversation as resolved.
Show resolved Hide resolved
}

@Test
Expand Down Expand Up @@ -301,6 +310,7 @@ public void testCompactionTaskToClientCompactionTaskQuery() throws IOException
.transformSpec(new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null)))
.build();

Map<String, Object> expectedContext = new HashMap<>();
final ClientCompactionTaskQuery expected = new ClientCompactionTaskQuery(
task.getId(),
"datasource",
Expand Down Expand Up @@ -345,12 +355,16 @@ public void testCompactionTaskToClientCompactionTaskQuery() throws IOException
new ClientCompactionTaskDimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))),
new AggregatorFactory[] {new CountAggregatorFactory("cnt")},
new ClientCompactionTaskTransformSpec(new SelectorDimFilter("dim1", "foo", null)),
new HashMap<>()
expectedContext
);

final byte[] json = mapper.writeValueAsBytes(task);
final ClientCompactionTaskQuery actual = (ClientCompactionTaskQuery) mapper.readValue(json, ClientTaskQuery.class);

// Verify that CompactionTask has added new parameters into the context
Assert.assertNotEquals(expected, actual);

expectedContext.put(LookupLoadingSpec.CTX_LOOKUP_LOADING_MODE, LookupLoadingSpec.Mode.NONE.toString());
Assert.assertEquals(expected, actual);
}

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -133,6 +133,7 @@
import org.apache.druid.segment.realtime.firehose.NoopChatHandlerProvider;
import org.apache.druid.segment.selector.settable.SettableColumnValueSelector;
import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory;
import org.apache.druid.server.lookup.cache.LookupLoadingSpec;
import org.apache.druid.server.security.AuthTestUtils;
import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.server.security.ResourceAction;
Expand Down Expand Up @@ -1738,6 +1739,20 @@ public void testChooseFinestGranularityAllNulls()
Assert.assertNull(chooseFinestGranularityHelper(input));
}

@Test
public void testGetDefaultLookupLoadingSpec()
{
final Builder builder = new Builder(
DATA_SOURCE,
segmentCacheManagerFactory,
RETRY_POLICY_FACTORY
);
final CompactionTask task = builder
.interval(Intervals.of("2000-01-01/2000-01-02"))
.build();
Assert.assertEquals(LookupLoadingSpec.NONE, task.getLookupLoadingSpec());
}

private Granularity chooseFinestGranularityHelper(List<Granularity> granularities)
{
SettableSupplier<Granularity> queryGranularity = new SettableSupplier<>();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -26,6 +26,7 @@
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.query.Query;
import org.apache.druid.query.QueryRunner;
import org.apache.druid.server.lookup.cache.LookupLoadingSpec;
import org.junit.Assert;
import org.junit.Test;

Expand Down Expand Up @@ -128,4 +129,10 @@ public void testGetInputSourceResources()
TASK::getInputSourceResources
);
}

@Test
public void testGetLookupLoadingSpec()
{
Assert.assertEquals(LookupLoadingSpec.ALL, TASK.getLookupLoadingSpec());
}
}