From 6220985559b4fa941e25e61caa5846c7ce8e368a Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 29 Jan 2020 13:37:05 -0800 Subject: [PATCH 01/37] prototype --- .../CachingClusteredClientBenchmark.java | 3 +- .../movingaverage/MovingAverageQueryTest.java | 3 +- .../query/QueryCapacityExceededException.java | 28 ++ .../apache/druid/query/QueryScheduler.java | 113 ++++++++ .../druid/query/QuerySchedulerConfig.java | 44 +++ .../druid/query/QuerySchedulerProvider.java | 32 +++ .../druid/query/QuerySchedulingStrategy.java | 41 +++ .../HiLoQuerySchedulingStrategy.java | 66 +++++ .../scheduling/NoQuerySchedulingStrategy.java | 45 +++ .../druid/query/QuerySchedulerTest.java | 267 ++++++++++++++++++ .../HiLoQuerySchedulingStrategyTest.java | 5 + .../NoQuerySchedulingStrategyTest.java | 5 + .../druid/client/CachingClusteredClient.java | 18 +- .../guice/DruidProcessingConfigModule.java | 10 + .../druid/guice/DruidProcessingModule.java | 9 + .../apache/druid/server/QueryResource.java | 11 + ...chingClusteredClientFunctionalityTest.java | 3 +- .../client/CachingClusteredClientTest.java | 3 +- .../apache/druid/sql/http/SqlResource.java | 4 + 19 files changed, 703 insertions(+), 7 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/query/QueryCapacityExceededException.java create mode 100644 processing/src/main/java/org/apache/druid/query/QueryScheduler.java create mode 100644 processing/src/main/java/org/apache/druid/query/QuerySchedulerConfig.java create mode 100644 processing/src/main/java/org/apache/druid/query/QuerySchedulerProvider.java create mode 100644 processing/src/main/java/org/apache/druid/query/QuerySchedulingStrategy.java create mode 100644 processing/src/main/java/org/apache/druid/query/scheduling/HiLoQuerySchedulingStrategy.java create mode 100644 processing/src/main/java/org/apache/druid/query/scheduling/NoQuerySchedulingStrategy.java create mode 100644 processing/src/test/java/org/apache/druid/query/QuerySchedulerTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/scheduling/HiLoQuerySchedulingStrategyTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/scheduling/NoQuerySchedulingStrategyTest.java diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index b89ae495b7de..38086b04e384 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -338,7 +338,8 @@ public > QueryToolChest getToolChest new CacheConfig(), new DruidHttpClientConfig(), processingConfig, - forkJoinPool + forkJoinPool, + null ); } diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index 9090bfe168d8..a00bfe55c831 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -361,7 +361,8 @@ public String getFormatString() return null; } }, - ForkJoinPool.commonPool() + ForkJoinPool.commonPool(), + null ); ClientQuerySegmentWalker walker = new ClientQuerySegmentWalker( diff --git a/processing/src/main/java/org/apache/druid/query/QueryCapacityExceededException.java b/processing/src/main/java/org/apache/druid/query/QueryCapacityExceededException.java new file mode 100644 index 000000000000..1fc0714cfa8b --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/QueryCapacityExceededException.java @@ -0,0 +1,28 @@ +/* + * 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.query; + +public class QueryCapacityExceededException extends RuntimeException +{ + public QueryCapacityExceededException() + { + super("too many cooks"); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/QueryScheduler.java b/processing/src/main/java/org/apache/druid/query/QueryScheduler.java new file mode 100644 index 000000000000..5e2ec0b6edea --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/QueryScheduler.java @@ -0,0 +1,113 @@ +/* + * 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.query; + +import com.google.common.collect.Sets; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import org.apache.druid.java.util.common.guava.Sequence; + +import java.util.HashMap; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; + + +public class QueryScheduler +{ + // maybe instead use a fancy library? + private final Semaphore totes; + private final Map lanes; + private final QuerySchedulingStrategy strategy; + private final Set> runningQueries; + + public QueryScheduler( + int totalNumThreads, + QuerySchedulingStrategy strategy + ) + { + this.strategy = strategy; + this.totes = new Semaphore(totalNumThreads); + this.lanes = new HashMap<>(); + + for (Object2IntMap.Entry entry : strategy.getLaneLimits().object2IntEntrySet()) { + lanes.put(entry.getKey(), new Semaphore(entry.getIntValue())); + } + this.runningQueries = Sets.newConcurrentHashSet(); + } + + public Query schedule(QueryPlus query, Set descriptors) + { + try { + if (totes.tryAcquire(0, TimeUnit.MILLISECONDS)) { + Query prioritizedAndLaned = strategy.prioritizeQuery(query, descriptors); + String lane = prioritizedAndLaned.getContextValue("queryLane"); + if (lanes.containsKey(lane)) { + if (lanes.get(lane).tryAcquire(0, TimeUnit.MILLISECONDS)) { + runningQueries.add(prioritizedAndLaned); + return prioritizedAndLaned; + } + } else { + runningQueries.add(prioritizedAndLaned); + return prioritizedAndLaned; + } + } + } catch (InterruptedException ex) { + throw new QueryCapacityExceededException(); + } + throw new QueryCapacityExceededException(); + } + + public Sequence run(Query query, Sequence resultSequence) + { + return resultSequence.withBaggage(() -> complete(query)); + } + + public int getTotalAvailableCapacity() + { + return totes.availablePermits(); + } + + public int getLaneAvailableCapacity(String lane) + { + if (lanes.containsKey(lane)) { + return lanes.get(lane).availablePermits(); + } + return -1; + } + + private void complete(Query query) + { +// try { +// Thread.sleep(10000); +// } +// catch (InterruptedException ie) { +// // eat it +// } + + if (runningQueries.remove(query)) { + String lane = query.getContextValue("queryLane"); + if (lanes.containsKey(lane)) { + lanes.get(lane).release(); + } + totes.release(); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/query/QuerySchedulerConfig.java b/processing/src/main/java/org/apache/druid/query/QuerySchedulerConfig.java new file mode 100644 index 000000000000..c27c6a336956 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/QuerySchedulerConfig.java @@ -0,0 +1,44 @@ +/* + * 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.query; + +import com.fasterxml.jackson.annotation.JsonProperty; + +public class QuerySchedulerConfig +{ + @JsonProperty + private Integer numThreads; + + @JsonProperty + private QuerySchedulingStrategy strategy; + + + @JsonProperty + public int getNumThreads() + { + return numThreads; + } + + @JsonProperty + public QuerySchedulingStrategy getStrategy() + { + return strategy; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/QuerySchedulerProvider.java b/processing/src/main/java/org/apache/druid/query/QuerySchedulerProvider.java new file mode 100644 index 000000000000..11867cbfecf5 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/QuerySchedulerProvider.java @@ -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.query; + +import com.google.inject.Provider; + + +public class QuerySchedulerProvider extends QuerySchedulerConfig implements Provider +{ + @Override + public QueryScheduler get() + { + return new QueryScheduler(getNumThreads(), getStrategy()); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/QuerySchedulingStrategy.java b/processing/src/main/java/org/apache/druid/query/QuerySchedulingStrategy.java new file mode 100644 index 000000000000..4e8a9124f635 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/QuerySchedulingStrategy.java @@ -0,0 +1,41 @@ +/* + * 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.query; + +import com.fasterxml.jackson.annotation.JsonSubTypes; +import com.fasterxml.jackson.annotation.JsonTypeInfo; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import org.apache.druid.query.scheduling.HiLoQuerySchedulingStrategy; +import org.apache.druid.query.scheduling.NoQuerySchedulingStrategy; + +import java.util.Set; + + +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = NoQuerySchedulingStrategy.class) +@JsonSubTypes(value = { + @JsonSubTypes.Type(name = "none", value = NoQuerySchedulingStrategy.class), + @JsonSubTypes.Type(name = "hilo", value = HiLoQuerySchedulingStrategy.class) +}) +public interface QuerySchedulingStrategy +{ + Object2IntMap getLaneLimits(); + + Query prioritizeQuery(QueryPlus query, Set segments); +} diff --git a/processing/src/main/java/org/apache/druid/query/scheduling/HiLoQuerySchedulingStrategy.java b/processing/src/main/java/org/apache/druid/query/scheduling/HiLoQuerySchedulingStrategy.java new file mode 100644 index 000000000000..01dec3dd96f6 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/scheduling/HiLoQuerySchedulingStrategy.java @@ -0,0 +1,66 @@ +/* + * 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.query.scheduling; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.collect.ImmutableMap; +import it.unimi.dsi.fastutil.objects.Object2IntArrayMap; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.QuerySchedulingStrategy; +import org.apache.druid.query.SegmentDescriptor; + +import java.util.Set; + +public class HiLoQuerySchedulingStrategy implements QuerySchedulingStrategy +{ + @JsonProperty + private int maxLowThreads; + + @JsonCreator + public HiLoQuerySchedulingStrategy( + @JsonProperty("maxLowThreads") Integer maxLowThreads + ) + { + this.maxLowThreads = maxLowThreads; + } + + @Override + public Object2IntMap getLaneLimits() + { + Object2IntMap onlyLow = new Object2IntArrayMap<>(1); + onlyLow.put("low", maxLowThreads); + return onlyLow; + } + + @Override + public Query prioritizeQuery(QueryPlus query, Set segments) + { + final Query theQuery = query.getQuery(); + final Integer priority = theQuery.getContextValue("priority"); + final String lane = theQuery.getContextValue("queryLane"); + if (lane == null && priority != null && priority < 0) { + return theQuery.withOverriddenContext(ImmutableMap.builder().putAll(theQuery.getContext()).put("queryLane", "low").build()); + } + return theQuery; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/scheduling/NoQuerySchedulingStrategy.java b/processing/src/main/java/org/apache/druid/query/scheduling/NoQuerySchedulingStrategy.java new file mode 100644 index 000000000000..abfb4a233de5 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/scheduling/NoQuerySchedulingStrategy.java @@ -0,0 +1,45 @@ +/* + * 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.query.scheduling; + +import it.unimi.dsi.fastutil.objects.Object2IntArrayMap; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.QuerySchedulingStrategy; +import org.apache.druid.query.SegmentDescriptor; + +import java.util.Set; + +public class NoQuerySchedulingStrategy implements QuerySchedulingStrategy +{ + private static final Object2IntMap NONE = new Object2IntArrayMap<>(); + @Override + public Object2IntMap getLaneLimits() + { + return NONE; + } + + @Override + public Query prioritizeQuery(QueryPlus query, Set segments) + { + return query.getQuery(); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/QuerySchedulerTest.java b/processing/src/test/java/org/apache/druid/query/QuerySchedulerTest.java new file mode 100644 index 000000000000..3fae93454765 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/QuerySchedulerTest.java @@ -0,0 +1,267 @@ +/* + * 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.query; + +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import org.apache.druid.java.util.common.guava.BaseSequence; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.scheduling.HiLoQuerySchedulingStrategy; +import org.apache.druid.query.topn.TopNQuery; +import org.apache.druid.query.topn.TopNQueryBuilder; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +import java.io.IOException; +import java.util.Iterator; + +public class QuerySchedulerTest +{ + TopNQuery interactive = new TopNQueryBuilder() + .dataSource("foo") + .intervals("2020-01-01/2020-01-02") + .dimension("bar") + .metric("chocula") + .aggregators(new CountAggregatorFactory("chocula")) + .threshold(10) + .context(ImmutableMap.of("priority", 10, "queryId", "1234")) + .build(); + + TopNQuery report = new TopNQueryBuilder() + .dataSource("foo") + .intervals("2020-01-01/2020-01-02") + .dimension("bar") + .metric("chocula") + .aggregators(new CountAggregatorFactory("chocula")) + .threshold(10) + .context(ImmutableMap.of("priority", -1, "queryId", "1234")) + .build(); + + @Rule + public ExpectedException expected = ExpectedException.none(); + + @Test + public void testHiLoHi() throws IOException + { + QueryScheduler scheduler = new QueryScheduler(5, new HiLoQuerySchedulingStrategy(2)); + + Query n2 = scheduler.schedule(QueryPlus.wrap(interactive), ImmutableSet.of()); + + Assert.assertNotNull(n2); + Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); + Assert.assertEquals(2, scheduler.getLaneAvailableCapacity("low")); + + Sequence exploder = makeSequence(10); + Sequence scheduled = scheduler.run(n2, exploder); + + int rowCount = consumeAndCloseSequence(scheduled); + + Assert.assertEquals(10, rowCount); + Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); + } + + @Test + public void testHiLoLo() throws IOException + { + QueryScheduler scheduler = new QueryScheduler(5, new HiLoQuerySchedulingStrategy(2)); + + Query q = scheduler.schedule(QueryPlus.wrap(report), ImmutableSet.of()); + Assert.assertNotNull(q); + Assert.assertEquals("low", q.getContextValue("queryLane")); + Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); + Assert.assertEquals(1, scheduler.getLaneAvailableCapacity("low")); + + Sequence exploder = makeSequence(10); + Sequence scheduled = scheduler.run(q, exploder); + + int rowCount = consumeAndCloseSequence(scheduled); + + Assert.assertEquals(10, rowCount); + Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); + Assert.assertEquals(2, scheduler.getLaneAvailableCapacity("low")); + } + + @Test + public void testHiLoFailsWhenOutOfTotalCapacity() throws IOException + { + expected.expectMessage("too many cooks"); + expected.expect(QueryCapacityExceededException.class); + + + QueryScheduler scheduler = new QueryScheduler(5, new HiLoQuerySchedulingStrategy(2)); + + Query n2 = scheduler.schedule(QueryPlus.wrap(interactive), ImmutableSet.of()); + Assert.assertNotNull(n2); + Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); + + Query n3 = scheduler.schedule(QueryPlus.wrap(report), ImmutableSet.of()); + Assert.assertNotNull(n3); + Assert.assertEquals(3, scheduler.getTotalAvailableCapacity()); + Assert.assertEquals(1, scheduler.getLaneAvailableCapacity("low")); + + Query n4 = scheduler.schedule(QueryPlus.wrap(interactive), ImmutableSet.of()); + Assert.assertNotNull(n4); + Assert.assertEquals(2, scheduler.getTotalAvailableCapacity()); + + Query n5 = scheduler.schedule(QueryPlus.wrap(report), ImmutableSet.of()); + Assert.assertNotNull(n5); + Assert.assertEquals(1, scheduler.getTotalAvailableCapacity()); + Assert.assertEquals(0, scheduler.getLaneAvailableCapacity("low")); + + Query n6 = scheduler.schedule(QueryPlus.wrap(interactive), ImmutableSet.of()); + Assert.assertNotNull(n6); + Assert.assertEquals(0, scheduler.getTotalAvailableCapacity()); + + scheduler.schedule(QueryPlus.wrap(interactive), ImmutableSet.of()); + } + + @Test + public void testHiLoFailsWhenOutOfLaneCapacity() + { + expected.expectMessage("too many cooks"); + expected.expect(QueryCapacityExceededException.class); + + QueryScheduler scheduler = new QueryScheduler(5, new HiLoQuerySchedulingStrategy(2)); + + Query n2 = scheduler.schedule(QueryPlus.wrap(report), ImmutableSet.of()); + Assert.assertNotNull(n2); + Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); + Assert.assertEquals(1, scheduler.getLaneAvailableCapacity("low")); + + Query n3 = scheduler.schedule(QueryPlus.wrap(report), ImmutableSet.of()); + Assert.assertNotNull(n3); + Assert.assertEquals(3, scheduler.getTotalAvailableCapacity()); + Assert.assertEquals(0, scheduler.getLaneAvailableCapacity("low")); + + scheduler.schedule(QueryPlus.wrap(report), ImmutableSet.of()); + } + + @Test + public void testHiLoReleaseSemaphoreWhenSequenceExplodes() throws Exception + { + expected.expectMessage("exploded"); + expected.expect(RuntimeException.class); + + QueryScheduler scheduler = new QueryScheduler(5, new HiLoQuerySchedulingStrategy(2)); + + Query n2 = scheduler.schedule(QueryPlus.wrap(interactive), ImmutableSet.of()); + + Assert.assertNotNull(n2); + Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); + + Sequence exploder = makeExplodingSequence(10); + Sequence scheduled = scheduler.run(n2, exploder); + + consumeAndCloseSequence(scheduled); + Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); + } + + private int consumeAndCloseSequence(Sequence sequence) throws IOException + { + Yielder yielder = Yielders.each(sequence); + int rowCount = 0; + while (!yielder.isDone()) { + rowCount++; + yielder = yielder.next(yielder.get()); + } + yielder.close(); + return rowCount; + } + + private Sequence makeSequence(int count) + { + return new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public Iterator make() + { + return new Iterator() + { + int rowCounter = 0; + @Override + public boolean hasNext() + { + return rowCounter < count; + } + + @Override + public Integer next() + { + rowCounter++; + return rowCounter; + } + }; + } + + @Override + public void cleanup(Iterator iterFromMake) + { + // nothing to cleanup + } + } + ); + } + + private Sequence makeExplodingSequence(int explodeAfter) + { + final int explodeAt = explodeAfter + 1; + return new BaseSequence<>( + new BaseSequence.IteratorMaker>() + { + @Override + public Iterator make() + { + return new Iterator() + { + int rowCounter = 0; + @Override + public boolean hasNext() + { + return rowCounter < explodeAt; + } + + @Override + public Integer next() + { + if (rowCounter == explodeAfter) { + throw new RuntimeException("exploded"); + } + + rowCounter++; + return rowCounter; + } + }; + } + + @Override + public void cleanup(Iterator iterFromMake) + { + // nothing to cleanup + } + } + ); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/scheduling/HiLoQuerySchedulingStrategyTest.java b/processing/src/test/java/org/apache/druid/query/scheduling/HiLoQuerySchedulingStrategyTest.java new file mode 100644 index 000000000000..b1720855b13a --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/scheduling/HiLoQuerySchedulingStrategyTest.java @@ -0,0 +1,5 @@ +package org.apache.druid.query.scheduling; + +public class HiLoQuerySchedulingStrategyTest +{ +} diff --git a/processing/src/test/java/org/apache/druid/query/scheduling/NoQuerySchedulingStrategyTest.java b/processing/src/test/java/org/apache/druid/query/scheduling/NoQuerySchedulingStrategyTest.java new file mode 100644 index 000000000000..9688e7110af1 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/scheduling/NoQuerySchedulingStrategyTest.java @@ -0,0 +1,5 @@ +package org.apache.druid.query.scheduling; + +public class NoQuerySchedulingStrategyTest +{ +} diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index 5254591bb0e1..93358bbd9c4e 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -58,6 +58,7 @@ import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QueryScheduler; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.QueryToolChestWarehouse; @@ -113,6 +114,7 @@ public class CachingClusteredClient implements QuerySegmentWalker private final DruidHttpClientConfig httpClientConfig; private final DruidProcessingConfig processingConfig; private final ForkJoinPool pool; + private final QueryScheduler scheduler; @Inject public CachingClusteredClient( @@ -124,7 +126,8 @@ public CachingClusteredClient( CacheConfig cacheConfig, @Client DruidHttpClientConfig httpClientConfig, DruidProcessingConfig processingConfig, - @Merging ForkJoinPool pool + @Merging ForkJoinPool pool, + QueryScheduler scheduler ) { this.warehouse = warehouse; @@ -136,6 +139,7 @@ public CachingClusteredClient( this.httpClientConfig = httpClientConfig; this.processingConfig = processingConfig; this.pool = pool; + this.scheduler = scheduler; if (cacheConfig.isQueryCacheable(Query.GROUP_BY) && (cacheConfig.isUseCache() || cacheConfig.isPopulateCache())) { log.warn( @@ -224,7 +228,7 @@ private class SpecificQueryRunnable { private final QueryPlus queryPlus; private final ResponseContext responseContext; - private final Query query; + private Query query; private final QueryToolChest> toolChest; @Nullable private final CacheStrategy> strategy; @@ -302,13 +306,21 @@ Sequence run(final UnaryOperator> time } final List> alreadyCachedResults = pruneSegmentsWithCachedResults(queryCacheKey, segments); + + query = scheduler.schedule( + queryPlus.withQuery(query), + segments.stream().map(x -> x.getSegmentDescriptor()).collect(Collectors.toSet()) + ); + final SortedMap> segmentsByServer = groupSegmentsByServer(segments); - return new LazySequence<>(() -> { + LazySequence sequence = new LazySequence<>(() -> { List> sequencesByInterval = new ArrayList<>(alreadyCachedResults.size() + segmentsByServer.size()); addSequencesFromCache(sequencesByInterval, alreadyCachedResults); addSequencesFromServer(sequencesByInterval, segmentsByServer); return merge(sequencesByInterval); }); + + return scheduler.run(query, sequence); } private Sequence merge(List> sequencesByInterval) diff --git a/server/src/main/java/org/apache/druid/guice/DruidProcessingConfigModule.java b/server/src/main/java/org/apache/druid/guice/DruidProcessingConfigModule.java index 7e83817de26a..868f3a6d692a 100644 --- a/server/src/main/java/org/apache/druid/guice/DruidProcessingConfigModule.java +++ b/server/src/main/java/org/apache/druid/guice/DruidProcessingConfigModule.java @@ -21,8 +21,12 @@ import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; +import com.google.inject.Key; import com.google.inject.Module; +import org.apache.druid.guice.annotations.Global; import org.apache.druid.query.DruidProcessingConfig; +import org.apache.druid.query.QueryScheduler; +import org.apache.druid.query.QuerySchedulerProvider; public class DruidProcessingConfigModule implements Module { @@ -31,5 +35,11 @@ public class DruidProcessingConfigModule implements Module public void configure(Binder binder) { ConfigProvider.bind(binder, DruidProcessingConfig.class, ImmutableMap.of("base_path", "druid.processing")); + + // clintropolis todo: find me a better home + binder.bind(QueryScheduler.class) + .toProvider(Key.get(QuerySchedulerProvider.class, Global.class)) + .in(LazySingleton.class); + JsonConfigProvider.bind(binder, "druid.query.scheduler", QuerySchedulerProvider.class, Global.class); } } diff --git a/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java b/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java index 59700160090a..dfe53b433a40 100644 --- a/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java +++ b/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java @@ -149,6 +149,15 @@ public LifecycleForkJoinPoolProvider getMergeProcessingPoolProvider(DruidProcess ); } +// @Provides +// public QueryScheduler getQueryScheduler(QuerySchedulerProvider config) +// { +// if (config == null) { +// return new QueryScheduler(100, new NoQuerySchedulingStrategy()); +// } +// return new QueryScheduler(config.getNumThreads(), config.getStrategy()); +// } + @Provides @Merging public ForkJoinPool getMergeProcessingPool(LifecycleForkJoinPoolProvider poolProvider) diff --git a/server/src/main/java/org/apache/druid/server/QueryResource.java b/server/src/main/java/org/apache/druid/server/QueryResource.java index e2fb263def4c..c7f29432a0dc 100644 --- a/server/src/main/java/org/apache/druid/server/QueryResource.java +++ b/server/src/main/java/org/apache/druid/server/QueryResource.java @@ -41,6 +41,7 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.GenericQueryMetricsFactory; import org.apache.druid.query.Query; +import org.apache.druid.query.QueryCapacityExceededException; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryInterruptedException; import org.apache.druid.query.QueryToolChest; @@ -310,6 +311,11 @@ public void write(OutputStream outputStream) throws WebApplicationException queryLifecycle.emitLogsAndMetrics(e, req.getRemoteAddr(), -1); return ioReaderWriter.gotError(e); } + catch (QueryCapacityExceededException cap) { + failedQueryCount.incrementAndGet(); + queryLifecycle.emitLogsAndMetrics(cap, req.getRemoteAddr(), -1); + return ioReaderWriter.gotLimited(cap); + } catch (ForbiddenException e) { // don't do anything for an authorization failure, ForbiddenExceptionMapper will catch this later and // send an error response if this is thrown. @@ -434,6 +440,11 @@ Response gotError(Exception e) throws IOException ) .build(); } + + Response gotLimited(QueryCapacityExceededException e) + { + return Response.status(429).entity(e.getMessage()).build(); + } } @Override diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java index 3efe7bc5d759..0ad4fc3b0c19 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java @@ -330,7 +330,8 @@ public int getMergePoolParallelism() return 4; } }, - ForkJoinPool.commonPool() + ForkJoinPool.commonPool(), + null ); } diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index 9c0588837ffd..7c51df016863 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -2473,7 +2473,8 @@ public int getMergePoolParallelism() return 4; } }, - ForkJoinPool.commonPool() + ForkJoinPool.commonPool(), + null ); } diff --git a/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java b/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java index cc7e6cdf2371..00c8cbd1eeb4 100644 --- a/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java +++ b/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java @@ -32,6 +32,7 @@ import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.logger.Logger; +import org.apache.druid.query.QueryCapacityExceededException; import org.apache.druid.query.QueryInterruptedException; import org.apache.druid.server.security.ForbiddenException; import org.apache.druid.sql.SqlLifecycle; @@ -171,6 +172,9 @@ public Response doPost( throw new RuntimeException(e); } } + catch (QueryCapacityExceededException cap) { + return Response.status(429).entity(cap.getMessage()).build(); + } catch (ForbiddenException e) { throw e; // let ForbiddenExceptionMapper handle this } From 3f410014da090c7df66df293a1894da3486cabaa Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 31 Jan 2020 13:29:32 -0800 Subject: [PATCH 02/37] merge QueryScheduler and QueryManager --- .../CachingClusteredClientBenchmark.java | 4 +- .../movingaverage/MovingAverageQueryTest.java | 4 +- .../apache/druid/query/QueryScheduler.java | 87 +++++-- .../org/apache/druid/query/QueryWatcher.java | 2 +- .../scheduling/NoQuerySchedulingStrategy.java | 3 + .../druid/query/QuerySchedulerTest.java | 220 +++++++++++------- .../HiLoQuerySchedulingStrategyTest.java | 5 - .../NoQuerySchedulingStrategyTest.java | 5 - .../guice/DruidProcessingConfigModule.java | 10 - .../druid/guice/DruidProcessingModule.java | 9 - .../druid/guice/QueryRunnerFactoryModule.java | 29 ++- .../druid/server/BrokerQueryResource.java | 5 +- .../org/apache/druid/server/QueryManager.java | 86 ------- .../apache/druid/server/QueryResource.java | 13 +- ...chingClusteredClientFunctionalityTest.java | 5 +- .../client/CachingClusteredClientTest.java | 4 +- .../druid/server/QueryResourceTest.java | 18 +- 17 files changed, 258 insertions(+), 251 deletions(-) delete mode 100644 processing/src/test/java/org/apache/druid/query/scheduling/HiLoQuerySchedulingStrategyTest.java delete mode 100644 processing/src/test/java/org/apache/druid/query/scheduling/NoQuerySchedulingStrategyTest.java delete mode 100644 server/src/main/java/org/apache/druid/server/QueryManager.java diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index 38086b04e384..36347e2215e9 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -71,6 +71,7 @@ import org.apache.druid.query.QueryRunnerFactory; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.QueryRunnerTestHelper; +import org.apache.druid.query.QueryScheduler; import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.Result; @@ -89,6 +90,7 @@ import org.apache.druid.query.groupby.strategy.GroupByStrategyV1; import org.apache.druid.query.groupby.strategy.GroupByStrategyV2; import org.apache.druid.query.planning.DataSourceAnalysis; +import org.apache.druid.query.scheduling.NoQuerySchedulingStrategy; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.query.timeseries.TimeseriesQuery; @@ -339,7 +341,7 @@ public > QueryToolChest getToolChest new DruidHttpClientConfig(), processingConfig, forkJoinPool, - null + new QueryScheduler(8, NoQuerySchedulingStrategy.INSTANCE) ); } diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index a00bfe55c831..fd6f19a6639e 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -53,6 +53,7 @@ import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QueryScheduler; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.Result; @@ -62,6 +63,7 @@ import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.movingaverage.test.TestConfig; import org.apache.druid.query.planning.DataSourceAnalysis; +import org.apache.druid.query.scheduling.NoQuerySchedulingStrategy; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.druid.server.ClientQuerySegmentWalker; @@ -362,7 +364,7 @@ public String getFormatString() } }, ForkJoinPool.commonPool(), - null + new QueryScheduler(8, NoQuerySchedulingStrategy.INSTANCE) ); ClientQuerySegmentWalker walker = new ClientQuerySegmentWalker( diff --git a/processing/src/main/java/org/apache/druid/query/QueryScheduler.java b/processing/src/main/java/org/apache/druid/query/QueryScheduler.java index 5e2ec0b6edea..3920de3fbc4d 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryScheduler.java +++ b/processing/src/main/java/org/apache/druid/query/QueryScheduler.java @@ -19,8 +19,13 @@ package org.apache.druid.query; +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimaps; +import com.google.common.collect.SetMultimap; import com.google.common.collect.Sets; +import com.google.common.util.concurrent.ListenableFuture; import it.unimi.dsi.fastutil.objects.Object2IntMap; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.Sequence; import java.util.HashMap; @@ -30,12 +35,15 @@ import java.util.concurrent.TimeUnit; -public class QueryScheduler +public class QueryScheduler implements QueryWatcher { // maybe instead use a fancy library? - private final Semaphore totes; + private final Semaphore activeQueries; private final Map lanes; private final QuerySchedulingStrategy strategy; + + private final SetMultimap> queries; + private final SetMultimap queryDatasources; private final Set> runningQueries; public QueryScheduler( @@ -44,19 +52,25 @@ public QueryScheduler( ) { this.strategy = strategy; - this.totes = new Semaphore(totalNumThreads); + this.activeQueries = new Semaphore(totalNumThreads); this.lanes = new HashMap<>(); for (Object2IntMap.Entry entry : strategy.getLaneLimits().object2IntEntrySet()) { lanes.put(entry.getKey(), new Semaphore(entry.getIntValue())); } + this.queries = Multimaps.synchronizedSetMultimap( + HashMultimap.create() + ); + this.queryDatasources = Multimaps.synchronizedSetMultimap( + HashMultimap.create() + ); this.runningQueries = Sets.newConcurrentHashSet(); } public Query schedule(QueryPlus query, Set descriptors) { try { - if (totes.tryAcquire(0, TimeUnit.MILLISECONDS)) { + if (activeQueries.tryAcquire(0, TimeUnit.MILLISECONDS)) { Query prioritizedAndLaned = strategy.prioritizeQuery(query, descriptors); String lane = prioritizedAndLaned.getContextValue("queryLane"); if (lanes.containsKey(lane)) { @@ -69,7 +83,8 @@ public Query schedule(QueryPlus query, Set descript return prioritizedAndLaned; } } - } catch (InterruptedException ex) { + } + catch (InterruptedException ex) { throw new QueryCapacityExceededException(); } throw new QueryCapacityExceededException(); @@ -77,12 +92,23 @@ public Query schedule(QueryPlus query, Set descript public Sequence run(Query query, Sequence resultSequence) { - return resultSequence.withBaggage(() -> complete(query)); + // if really doing release on the bagage, should merge most of schedule into here to tie acquire with release + // alternatively, if we want to release as late as possible, rework this to release in the registered queries future + // listener, though we need a guaranteed unique queryId and slight refactor to make this work.. + return resultSequence.withBaggage(() -> { + if (runningQueries.remove(query)) { + String lane = query.getContextValue("queryLane"); + if (lanes.containsKey(lane)) { + lanes.get(lane).release(); + } + activeQueries.release(); + } + }); } public int getTotalAvailableCapacity() { - return totes.availablePermits(); + return activeQueries.availablePermits(); } public int getLaneAvailableCapacity(String lane) @@ -93,21 +119,40 @@ public int getLaneAvailableCapacity(String lane) return -1; } - private void complete(Query query) + + public boolean cancelQuery(String id) { -// try { -// Thread.sleep(10000); -// } -// catch (InterruptedException ie) { -// // eat it -// } - - if (runningQueries.remove(query)) { - String lane = query.getContextValue("queryLane"); - if (lanes.containsKey(lane)) { - lanes.get(lane).release(); - } - totes.release(); + queryDatasources.removeAll(id); + Set> futures = queries.removeAll(id); + boolean success = true; + for (ListenableFuture future : futures) { + success = success && future.cancel(true); } + return success; + } + + @Override + public void registerQuery(Query query, final ListenableFuture future) + { + final String id = query.getId(); + final Set datasources = query.getDataSource().getTableNames(); + queries.put(id, future); + queryDatasources.putAll(id, datasources); + future.addListener( + () -> { + // if you re-use queryId and cancel queries... you are going to have a bad time + queries.remove(id, future); + + for (String datasource : datasources) { + queryDatasources.remove(id, datasource); + } + }, + Execs.directExecutor() + ); + } + + public Set getQueryDatasources(final String queryId) + { + return queryDatasources.get(queryId); } } diff --git a/processing/src/main/java/org/apache/druid/query/QueryWatcher.java b/processing/src/main/java/org/apache/druid/query/QueryWatcher.java index 0457bea25c45..204fa9b4e324 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryWatcher.java +++ b/processing/src/main/java/org/apache/druid/query/QueryWatcher.java @@ -43,5 +43,5 @@ public interface QueryWatcher * @param query a query, which may be a subset of a larger query, as long as the underlying queryId is unchanged * @param future the future holding the execution status of the query */ - void registerQuery(Query query, ListenableFuture future); + void registerQuery(Query query, ListenableFuture future); } diff --git a/processing/src/main/java/org/apache/druid/query/scheduling/NoQuerySchedulingStrategy.java b/processing/src/main/java/org/apache/druid/query/scheduling/NoQuerySchedulingStrategy.java index abfb4a233de5..298f6b107e4e 100644 --- a/processing/src/main/java/org/apache/druid/query/scheduling/NoQuerySchedulingStrategy.java +++ b/processing/src/main/java/org/apache/druid/query/scheduling/NoQuerySchedulingStrategy.java @@ -31,6 +31,9 @@ public class NoQuerySchedulingStrategy implements QuerySchedulingStrategy { private static final Object2IntMap NONE = new Object2IntArrayMap<>(); + + public static NoQuerySchedulingStrategy INSTANCE = new NoQuerySchedulingStrategy(); + @Override public Object2IntMap getLaneLimits() { diff --git a/processing/src/test/java/org/apache/druid/query/QuerySchedulerTest.java b/processing/src/test/java/org/apache/druid/query/QuerySchedulerTest.java index 3fae93454765..9d1de0ace97a 100644 --- a/processing/src/test/java/org/apache/druid/query/QuerySchedulerTest.java +++ b/processing/src/test/java/org/apache/druid/query/QuerySchedulerTest.java @@ -21,6 +21,9 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.MoreExecutors; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Yielder; @@ -36,146 +39,189 @@ import java.io.IOException; import java.util.Iterator; +import java.util.UUID; +import java.util.concurrent.ExecutionException; public class QuerySchedulerTest { - TopNQuery interactive = new TopNQueryBuilder() - .dataSource("foo") - .intervals("2020-01-01/2020-01-02") - .dimension("bar") - .metric("chocula") - .aggregators(new CountAggregatorFactory("chocula")) - .threshold(10) - .context(ImmutableMap.of("priority", 10, "queryId", "1234")) - .build(); - - TopNQuery report = new TopNQueryBuilder() - .dataSource("foo") - .intervals("2020-01-01/2020-01-02") - .dimension("bar") - .metric("chocula") - .aggregators(new CountAggregatorFactory("chocula")) - .threshold(10) - .context(ImmutableMap.of("priority", -1, "queryId", "1234")) - .build(); - @Rule public ExpectedException expected = ExpectedException.none(); @Test - public void testHiLoHi() throws IOException + public void testHiLoHi() throws ExecutionException, InterruptedException { QueryScheduler scheduler = new QueryScheduler(5, new HiLoQuerySchedulingStrategy(2)); - Query n2 = scheduler.schedule(QueryPlus.wrap(interactive), ImmutableSet.of()); + TopNQuery interactive = makeInteractiveQuery(); + ListenableFuture future = MoreExecutors.listeningDecorator( + Execs.singleThreaded("test_query_scheduler_%s") + ).submit(() -> { + try { + Query scheduled = scheduler.schedule(QueryPlus.wrap(interactive), ImmutableSet.of()); + + Assert.assertNotNull(scheduled); + Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); + Assert.assertEquals(2, scheduler.getLaneAvailableCapacity("low")); + + Sequence underlyingSequence = makeSequence(10); + Sequence results = scheduler.run(scheduled, underlyingSequence); + int rowCount = consumeAndCloseSequence(results); + + Assert.assertEquals(10, rowCount); + } + catch (IOException ex) { + throw new RuntimeException(ex); + } + }); + scheduler.registerQuery(interactive, future); + future.get(); + Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); + } - Assert.assertNotNull(n2); - Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); + @Test + public void testHiLoLo() throws ExecutionException, InterruptedException + { + QueryScheduler scheduler = new QueryScheduler(5, new HiLoQuerySchedulingStrategy(2)); + TopNQuery report = makeReportQuery(); + ListenableFuture future = MoreExecutors.listeningDecorator( + Execs.singleThreaded("test_query_scheduler_%s") + ).submit(() -> { + try { + Query scheduledReport = scheduler.schedule(QueryPlus.wrap(report), ImmutableSet.of()); + Assert.assertNotNull(scheduledReport); + Assert.assertEquals("low", scheduledReport.getContextValue("queryLane")); + Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); + Assert.assertEquals(1, scheduler.getLaneAvailableCapacity("low")); + + Sequence underlyingSequence = makeSequence(10); + Sequence results = scheduler.run(scheduledReport, underlyingSequence); + + int rowCount = consumeAndCloseSequence(results); + Assert.assertEquals(10, rowCount); + } + catch (IOException ex) { + throw new RuntimeException(ex); + } + }); + scheduler.registerQuery(report, future); + future.get(); + Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); Assert.assertEquals(2, scheduler.getLaneAvailableCapacity("low")); + } - Sequence exploder = makeSequence(10); - Sequence scheduled = scheduler.run(n2, exploder); - int rowCount = consumeAndCloseSequence(scheduled); + @Test + public void testHiLoReleaseSemaphoreWhenSequenceExplodes() throws Exception + { + expected.expectMessage("exploded"); + expected.expect(ExecutionException.class); - Assert.assertEquals(10, rowCount); + QueryScheduler scheduler = new QueryScheduler(5, new HiLoQuerySchedulingStrategy(2)); + + TopNQuery interactive = makeInteractiveQuery(); + ListenableFuture future = MoreExecutors.listeningDecorator( + Execs.singleThreaded("test_query_scheduler_%s") + ).submit(() -> { + try { + Query scheduled = scheduler.schedule(QueryPlus.wrap(interactive), ImmutableSet.of()); + + Assert.assertNotNull(scheduled); + Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); + + Sequence underlyingSequence = makeExplodingSequence(10); + Sequence results = scheduler.run(scheduled, underlyingSequence); + + consumeAndCloseSequence(results); + } + catch (IOException ex) { + throw new RuntimeException(ex); + } + }); + scheduler.registerQuery(interactive, future); + future.get(); Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); } @Test - public void testHiLoLo() throws IOException + public void testHiLoFailsWhenOutOfLaneCapacity() { + expected.expectMessage("too many cooks"); + expected.expect(QueryCapacityExceededException.class); + QueryScheduler scheduler = new QueryScheduler(5, new HiLoQuerySchedulingStrategy(2)); - Query q = scheduler.schedule(QueryPlus.wrap(report), ImmutableSet.of()); - Assert.assertNotNull(q); - Assert.assertEquals("low", q.getContextValue("queryLane")); + Query report1 = scheduler.schedule(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); + Assert.assertNotNull(report1); Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); Assert.assertEquals(1, scheduler.getLaneAvailableCapacity("low")); - Sequence exploder = makeSequence(10); - Sequence scheduled = scheduler.run(q, exploder); - - int rowCount = consumeAndCloseSequence(scheduled); + Query report2 = scheduler.schedule(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); + Assert.assertNotNull(report2); + Assert.assertEquals(3, scheduler.getTotalAvailableCapacity()); + Assert.assertEquals(0, scheduler.getLaneAvailableCapacity("low")); - Assert.assertEquals(10, rowCount); - Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); - Assert.assertEquals(2, scheduler.getLaneAvailableCapacity("low")); + // too many reports + scheduler.schedule(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); } @Test - public void testHiLoFailsWhenOutOfTotalCapacity() throws IOException + public void testHiLoFailsWhenOutOfTotalCapacity() { expected.expectMessage("too many cooks"); expected.expect(QueryCapacityExceededException.class); QueryScheduler scheduler = new QueryScheduler(5, new HiLoQuerySchedulingStrategy(2)); - - Query n2 = scheduler.schedule(QueryPlus.wrap(interactive), ImmutableSet.of()); - Assert.assertNotNull(n2); + Query interactive1 = scheduler.schedule(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()); + Assert.assertNotNull(interactive1); Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); - Query n3 = scheduler.schedule(QueryPlus.wrap(report), ImmutableSet.of()); - Assert.assertNotNull(n3); + Query report1 = scheduler.schedule(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); + Assert.assertNotNull(report1); Assert.assertEquals(3, scheduler.getTotalAvailableCapacity()); Assert.assertEquals(1, scheduler.getLaneAvailableCapacity("low")); - Query n4 = scheduler.schedule(QueryPlus.wrap(interactive), ImmutableSet.of()); - Assert.assertNotNull(n4); + Query interactive2 = scheduler.schedule(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()); + Assert.assertNotNull(interactive2); Assert.assertEquals(2, scheduler.getTotalAvailableCapacity()); - Query n5 = scheduler.schedule(QueryPlus.wrap(report), ImmutableSet.of()); - Assert.assertNotNull(n5); + Query report2 = scheduler.schedule(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); + Assert.assertNotNull(report2); Assert.assertEquals(1, scheduler.getTotalAvailableCapacity()); Assert.assertEquals(0, scheduler.getLaneAvailableCapacity("low")); - Query n6 = scheduler.schedule(QueryPlus.wrap(interactive), ImmutableSet.of()); - Assert.assertNotNull(n6); + Query interactive3 = scheduler.schedule(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()); + Assert.assertNotNull(interactive3); Assert.assertEquals(0, scheduler.getTotalAvailableCapacity()); - scheduler.schedule(QueryPlus.wrap(interactive), ImmutableSet.of()); + // one too many + scheduler.schedule(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()); } - @Test - public void testHiLoFailsWhenOutOfLaneCapacity() - { - expected.expectMessage("too many cooks"); - expected.expect(QueryCapacityExceededException.class); - QueryScheduler scheduler = new QueryScheduler(5, new HiLoQuerySchedulingStrategy(2)); - - Query n2 = scheduler.schedule(QueryPlus.wrap(report), ImmutableSet.of()); - Assert.assertNotNull(n2); - Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); - Assert.assertEquals(1, scheduler.getLaneAvailableCapacity("low")); - - Query n3 = scheduler.schedule(QueryPlus.wrap(report), ImmutableSet.of()); - Assert.assertNotNull(n3); - Assert.assertEquals(3, scheduler.getTotalAvailableCapacity()); - Assert.assertEquals(0, scheduler.getLaneAvailableCapacity("low")); - - scheduler.schedule(QueryPlus.wrap(report), ImmutableSet.of()); + private TopNQuery makeInteractiveQuery() + { + return makeBaseBuilder() + .context(ImmutableMap.of("priority", 10, "queryId", "high-" + UUID.randomUUID())) + .build(); } - @Test - public void testHiLoReleaseSemaphoreWhenSequenceExplodes() throws Exception + private TopNQuery makeReportQuery() { - expected.expectMessage("exploded"); - expected.expect(RuntimeException.class); - - QueryScheduler scheduler = new QueryScheduler(5, new HiLoQuerySchedulingStrategy(2)); - - Query n2 = scheduler.schedule(QueryPlus.wrap(interactive), ImmutableSet.of()); - - Assert.assertNotNull(n2); - Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); - - Sequence exploder = makeExplodingSequence(10); - Sequence scheduled = scheduler.run(n2, exploder); + return makeBaseBuilder() + .context(ImmutableMap.of("priority", -1, "queryId", "low-" + UUID.randomUUID())) + .build(); + } - consumeAndCloseSequence(scheduled); - Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); + private TopNQueryBuilder makeBaseBuilder() + { + return new TopNQueryBuilder() + .dataSource("foo") + .intervals("2020-01-01/2020-01-02") + .dimension("bar") + .metric("chocula") + .aggregators(new CountAggregatorFactory("chocula")) + .threshold(10); } private int consumeAndCloseSequence(Sequence sequence) throws IOException @@ -201,6 +247,7 @@ public Iterator make() return new Iterator() { int rowCounter = 0; + @Override public boolean hasNext() { @@ -237,6 +284,7 @@ public Iterator make() return new Iterator() { int rowCounter = 0; + @Override public boolean hasNext() { diff --git a/processing/src/test/java/org/apache/druid/query/scheduling/HiLoQuerySchedulingStrategyTest.java b/processing/src/test/java/org/apache/druid/query/scheduling/HiLoQuerySchedulingStrategyTest.java deleted file mode 100644 index b1720855b13a..000000000000 --- a/processing/src/test/java/org/apache/druid/query/scheduling/HiLoQuerySchedulingStrategyTest.java +++ /dev/null @@ -1,5 +0,0 @@ -package org.apache.druid.query.scheduling; - -public class HiLoQuerySchedulingStrategyTest -{ -} diff --git a/processing/src/test/java/org/apache/druid/query/scheduling/NoQuerySchedulingStrategyTest.java b/processing/src/test/java/org/apache/druid/query/scheduling/NoQuerySchedulingStrategyTest.java deleted file mode 100644 index 9688e7110af1..000000000000 --- a/processing/src/test/java/org/apache/druid/query/scheduling/NoQuerySchedulingStrategyTest.java +++ /dev/null @@ -1,5 +0,0 @@ -package org.apache.druid.query.scheduling; - -public class NoQuerySchedulingStrategyTest -{ -} diff --git a/server/src/main/java/org/apache/druid/guice/DruidProcessingConfigModule.java b/server/src/main/java/org/apache/druid/guice/DruidProcessingConfigModule.java index 868f3a6d692a..7e83817de26a 100644 --- a/server/src/main/java/org/apache/druid/guice/DruidProcessingConfigModule.java +++ b/server/src/main/java/org/apache/druid/guice/DruidProcessingConfigModule.java @@ -21,12 +21,8 @@ import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; -import com.google.inject.Key; import com.google.inject.Module; -import org.apache.druid.guice.annotations.Global; import org.apache.druid.query.DruidProcessingConfig; -import org.apache.druid.query.QueryScheduler; -import org.apache.druid.query.QuerySchedulerProvider; public class DruidProcessingConfigModule implements Module { @@ -35,11 +31,5 @@ public class DruidProcessingConfigModule implements Module public void configure(Binder binder) { ConfigProvider.bind(binder, DruidProcessingConfig.class, ImmutableMap.of("base_path", "druid.processing")); - - // clintropolis todo: find me a better home - binder.bind(QueryScheduler.class) - .toProvider(Key.get(QuerySchedulerProvider.class, Global.class)) - .in(LazySingleton.class); - JsonConfigProvider.bind(binder, "druid.query.scheduler", QuerySchedulerProvider.class, Global.class); } } diff --git a/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java b/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java index dfe53b433a40..59700160090a 100644 --- a/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java +++ b/server/src/main/java/org/apache/druid/guice/DruidProcessingModule.java @@ -149,15 +149,6 @@ public LifecycleForkJoinPoolProvider getMergeProcessingPoolProvider(DruidProcess ); } -// @Provides -// public QueryScheduler getQueryScheduler(QuerySchedulerProvider config) -// { -// if (config == null) { -// return new QueryScheduler(100, new NoQuerySchedulingStrategy()); -// } -// return new QueryScheduler(config.getNumThreads(), config.getStrategy()); -// } - @Provides @Merging public ForkJoinPool getMergeProcessingPool(LifecycleForkJoinPoolProvider poolProvider) diff --git a/server/src/main/java/org/apache/druid/guice/QueryRunnerFactoryModule.java b/server/src/main/java/org/apache/druid/guice/QueryRunnerFactoryModule.java index 6071973cc37a..a68c1dd03b22 100644 --- a/server/src/main/java/org/apache/druid/guice/QueryRunnerFactoryModule.java +++ b/server/src/main/java/org/apache/druid/guice/QueryRunnerFactoryModule.java @@ -21,9 +21,14 @@ import com.google.common.collect.ImmutableMap; import com.google.inject.Binder; +import com.google.inject.Key; +import com.google.inject.Provides; import com.google.inject.multibindings.MapBinder; +import org.apache.druid.guice.annotations.Global; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunnerFactory; +import org.apache.druid.query.QueryScheduler; +import org.apache.druid.query.QuerySchedulerProvider; import org.apache.druid.query.QueryWatcher; import org.apache.druid.query.datasourcemetadata.DataSourceMetadataQuery; import org.apache.druid.query.datasourcemetadata.DataSourceMetadataQueryRunnerFactory; @@ -42,7 +47,6 @@ import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; import org.apache.druid.query.topn.TopNQuery; import org.apache.druid.query.topn.TopNQueryRunnerFactory; -import org.apache.druid.server.QueryManager; import java.util.Map; @@ -50,8 +54,8 @@ */ public class QueryRunnerFactoryModule extends QueryToolChestModule { - private static final Map, Class> MAPPINGS = - ImmutableMap., Class>builder() + private static final Map>, Class>> MAPPINGS = + ImmutableMap.>, Class>>builder() .put(TimeseriesQuery.class, TimeseriesQueryRunnerFactory.class) .put(SearchQuery.class, SearchQueryRunnerFactory.class) .put(TimeBoundaryQuery.class, TimeBoundaryQueryRunnerFactory.class) @@ -67,21 +71,30 @@ public void configure(Binder binder) { super.configure(binder); - binder.bind(QueryWatcher.class) - .to(QueryManager.class) - .in(LazySingleton.class); - binder.bind(QueryManager.class) + binder.bind(QueryScheduler.class) + .toProvider(Key.get(QuerySchedulerProvider.class, Global.class)) .in(LazySingleton.class); + // binder.bind(QueryWatcher.class) + // .toProvider(Key.get(QuerySchedulerProvider.class, Global.class)) + // .in(LazySingleton.class); + JsonConfigProvider.bind(binder, "druid.query.scheduler", QuerySchedulerProvider.class, Global.class); final MapBinder, QueryRunnerFactory> queryFactoryBinder = DruidBinders.queryRunnerFactoryBinder( binder ); - for (Map.Entry, Class> entry : MAPPINGS.entrySet()) { + for (Map.Entry>, Class>> entry : MAPPINGS.entrySet()) { queryFactoryBinder.addBinding(entry.getKey()).to(entry.getValue()); binder.bind(entry.getValue()).in(LazySingleton.class); } binder.bind(GroupByQueryEngine.class).in(LazySingleton.class); } + + @LazySingleton + @Provides + public QueryWatcher getWatcher(QueryScheduler scheduler) + { + return scheduler; + } } diff --git a/server/src/main/java/org/apache/druid/server/BrokerQueryResource.java b/server/src/main/java/org/apache/druid/server/BrokerQueryResource.java index b75ce5df4cb5..c44c43b303ca 100644 --- a/server/src/main/java/org/apache/druid/server/BrokerQueryResource.java +++ b/server/src/main/java/org/apache/druid/server/BrokerQueryResource.java @@ -29,6 +29,7 @@ import org.apache.druid.guice.annotations.Smile; import org.apache.druid.query.GenericQueryMetricsFactory; import org.apache.druid.query.Query; +import org.apache.druid.query.QueryScheduler; import org.apache.druid.server.http.security.StateResourceFilter; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthorizerMapper; @@ -58,7 +59,7 @@ public BrokerQueryResource( QueryLifecycleFactory queryLifecycleFactory, @Json ObjectMapper jsonMapper, @Smile ObjectMapper smileMapper, - QueryManager queryManager, + QueryScheduler queryScheduler, AuthConfig authConfig, AuthorizerMapper authorizerMapper, GenericQueryMetricsFactory queryMetricsFactory, @@ -69,7 +70,7 @@ public BrokerQueryResource( queryLifecycleFactory, jsonMapper, smileMapper, - queryManager, + queryScheduler, authConfig, authorizerMapper, queryMetricsFactory diff --git a/server/src/main/java/org/apache/druid/server/QueryManager.java b/server/src/main/java/org/apache/druid/server/QueryManager.java deleted file mode 100644 index 0fd1807d86e3..000000000000 --- a/server/src/main/java/org/apache/druid/server/QueryManager.java +++ /dev/null @@ -1,86 +0,0 @@ -/* - * 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; - -import com.google.common.collect.HashMultimap; -import com.google.common.collect.Multimaps; -import com.google.common.collect.SetMultimap; -import com.google.common.util.concurrent.ListenableFuture; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.query.Query; -import org.apache.druid.query.QueryWatcher; - -import java.util.Set; - -public class QueryManager implements QueryWatcher -{ - - private final SetMultimap queries; - private final SetMultimap queryDatasources; - - public QueryManager() - { - this.queries = Multimaps.synchronizedSetMultimap( - HashMultimap.create() - ); - this.queryDatasources = Multimaps.synchronizedSetMultimap( - HashMultimap.create() - ); - } - - public boolean cancelQuery(String id) - { - queryDatasources.removeAll(id); - Set futures = queries.removeAll(id); - boolean success = true; - for (ListenableFuture future : futures) { - success = success && future.cancel(true); - } - return success; - } - - @Override - public void registerQuery(Query query, final ListenableFuture future) - { - final String id = query.getId(); - final Set datasources = query.getDataSource().getTableNames(); - queries.put(id, future); - queryDatasources.putAll(id, datasources); - future.addListener( - new Runnable() - { - @Override - public void run() - { - queries.remove(id, future); - for (String datasource : datasources) { - queryDatasources.remove(id, datasource); - } - } - }, - Execs.directExecutor() - ); - } - - public Set getQueryDatasources(final String queryId) - { - return queryDatasources.get(queryId); - } -} diff --git a/server/src/main/java/org/apache/druid/server/QueryResource.java b/server/src/main/java/org/apache/druid/server/QueryResource.java index c7f29432a0dc..6203f7bf57e7 100644 --- a/server/src/main/java/org/apache/druid/server/QueryResource.java +++ b/server/src/main/java/org/apache/druid/server/QueryResource.java @@ -44,6 +44,7 @@ import org.apache.druid.query.QueryCapacityExceededException; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryInterruptedException; +import org.apache.druid.query.QueryScheduler; import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.server.metrics.QueryCountStatsProvider; @@ -100,7 +101,7 @@ public class QueryResource implements QueryCountStatsProvider protected final ObjectMapper smileMapper; protected final ObjectMapper serializeDateTimeAsLongJsonMapper; protected final ObjectMapper serializeDateTimeAsLongSmileMapper; - protected final QueryManager queryManager; + protected final QueryScheduler queryScheduler; protected final AuthConfig authConfig; protected final AuthorizerMapper authorizerMapper; @@ -114,7 +115,7 @@ public QueryResource( QueryLifecycleFactory queryLifecycleFactory, @Json ObjectMapper jsonMapper, @Smile ObjectMapper smileMapper, - QueryManager queryManager, + QueryScheduler queryScheduler, AuthConfig authConfig, AuthorizerMapper authorizerMapper, GenericQueryMetricsFactory queryMetricsFactory @@ -125,7 +126,7 @@ public QueryResource( this.smileMapper = smileMapper; this.serializeDateTimeAsLongJsonMapper = serializeDataTimeAsLong(jsonMapper); this.serializeDateTimeAsLongSmileMapper = serializeDataTimeAsLong(smileMapper); - this.queryManager = queryManager; + this.queryScheduler = queryScheduler; this.authConfig = authConfig; this.authorizerMapper = authorizerMapper; this.queryMetricsFactory = queryMetricsFactory; @@ -139,9 +140,9 @@ public Response cancelQuery(@PathParam("id") String queryId, @Context final Http if (log.isDebugEnabled()) { log.debug("Received cancel request for query [%s]", queryId); } - Set datasources = queryManager.getQueryDatasources(queryId); + Set datasources = queryScheduler.getQueryDatasources(queryId); if (datasources == null) { - log.warn("QueryId [%s] not registered with QueryManager, cannot cancel", queryId); + log.warn("QueryId [%s] not registered with QueryScheduler, cannot cancel", queryId); datasources = new TreeSet<>(); } @@ -155,7 +156,7 @@ public Response cancelQuery(@PathParam("id") String queryId, @Context final Http throw new ForbiddenException(authResult.toString()); } - queryManager.cancelQuery(queryId); + queryScheduler.cancelQuery(queryId); return Response.status(Response.Status.ACCEPTED).build(); } diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java index 0ad4fc3b0c19..b1fc4426b01f 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java @@ -43,10 +43,12 @@ import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QueryScheduler; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.planning.DataSourceAnalysis; +import org.apache.druid.query.scheduling.NoQuerySchedulingStrategy; import org.apache.druid.server.coordination.ServerType; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.TimelineLookup; @@ -331,7 +333,8 @@ public int getMergePoolParallelism() } }, ForkJoinPool.commonPool(), - null + // need at least 9 total since runner doesn't actually run queries and release semaphores + new QueryScheduler(10, NoQuerySchedulingStrategy.INSTANCE) ); } diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index 7c51df016863..509c3f6e0aef 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -74,6 +74,7 @@ import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QueryScheduler; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.Result; import org.apache.druid.query.SegmentDescriptor; @@ -98,6 +99,7 @@ import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.planning.DataSourceAnalysis; +import org.apache.druid.query.scheduling.NoQuerySchedulingStrategy; import org.apache.druid.query.search.SearchHit; import org.apache.druid.query.search.SearchQuery; import org.apache.druid.query.search.SearchQueryConfig; @@ -2474,7 +2476,7 @@ public int getMergePoolParallelism() } }, ForkJoinPool.commonPool(), - null + new QueryScheduler(8, NoQuerySchedulingStrategy.INSTANCE) ); } diff --git a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java index 202ade04caed..5251f026f2ab 100644 --- a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java @@ -36,11 +36,13 @@ import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; +import org.apache.druid.query.QueryScheduler; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.Result; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.context.ResponseContext; +import org.apache.druid.query.scheduling.NoQuerySchedulingStrategy; import org.apache.druid.query.timeboundary.TimeBoundaryResultValue; import org.apache.druid.server.log.TestRequestLogger; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -109,7 +111,7 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable Date: Tue, 25 Feb 2020 02:47:58 -0800 Subject: [PATCH 03/37] everything in its right place --- .../CachingClusteredClientBenchmark.java | 6 +- .../movingaverage/MovingAverageQueryTest.java | 6 +- .../org/apache/druid/query/QueryContexts.java | 6 + .../apache/druid/query/QueryScheduler.java | 158 ------------- .../druid/client/CachingClusteredClient.java | 73 +++--- .../apache/druid/client/SegmentServer.java | 42 ++++ .../druid/guice/QueryRunnerFactoryModule.java | 7 +- .../druid/server/BrokerQueryResource.java | 1 - .../QueryCapacityExceededException.java | 15 +- .../apache/druid/server/QueryResource.java | 4 +- .../apache/druid/server/QueryScheduler.java | 203 ++++++++++++++++ .../druid/server}/QuerySchedulerConfig.java | 2 +- .../druid/server}/QuerySchedulerProvider.java | 2 +- .../server}/QuerySchedulingStrategy.java | 11 +- .../HiLoQuerySchedulingStrategy.java | 24 +- .../scheduling/NoQuerySchedulingStrategy.java | 8 +- ...chingClusteredClientFunctionalityTest.java | 6 +- .../client/CachingClusteredClientTest.java | 6 +- .../druid/server/QueryResourceTest.java | 217 ++++++++++++++++-- .../druid/server}/QuerySchedulerTest.java | 99 +++++--- .../apache/druid/sql/http/SqlResource.java | 2 +- 21 files changed, 605 insertions(+), 293 deletions(-) delete mode 100644 processing/src/main/java/org/apache/druid/query/QueryScheduler.java create mode 100644 server/src/main/java/org/apache/druid/client/SegmentServer.java rename {processing/src/main/java/org/apache/druid/query => server/src/main/java/org/apache/druid/server}/QueryCapacityExceededException.java (68%) create mode 100644 server/src/main/java/org/apache/druid/server/QueryScheduler.java rename {processing/src/main/java/org/apache/druid/query => server/src/main/java/org/apache/druid/server}/QuerySchedulerConfig.java (97%) rename {processing/src/main/java/org/apache/druid/query => server/src/main/java/org/apache/druid/server}/QuerySchedulerProvider.java (96%) rename {processing/src/main/java/org/apache/druid/query => server/src/main/java/org/apache/druid/server}/QuerySchedulingStrategy.java (78%) rename {processing/src/main/java/org/apache/druid/query => server/src/main/java/org/apache/druid/server}/scheduling/HiLoQuerySchedulingStrategy.java (67%) rename {processing/src/main/java/org/apache/druid/query => server/src/main/java/org/apache/druid/server}/scheduling/NoQuerySchedulingStrategy.java (85%) rename {processing/src/test/java/org/apache/druid/query => server/src/test/java/org/apache/druid/server}/QuerySchedulerTest.java (67%) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index 36347e2215e9..df90336ba19f 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -71,7 +71,6 @@ import org.apache.druid.query.QueryRunnerFactory; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.QueryRunnerTestHelper; -import org.apache.druid.query.QueryScheduler; import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.Result; @@ -90,7 +89,6 @@ import org.apache.druid.query.groupby.strategy.GroupByStrategyV1; import org.apache.druid.query.groupby.strategy.GroupByStrategyV2; import org.apache.druid.query.planning.DataSourceAnalysis; -import org.apache.druid.query.scheduling.NoQuerySchedulingStrategy; import org.apache.druid.query.spec.MultipleIntervalSegmentSpec; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.query.timeseries.TimeseriesQuery; @@ -106,7 +104,9 @@ import org.apache.druid.query.topn.TopNResultValue; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.QueryableIndexSegment; +import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.scheduling.NoQuerySchedulingStrategy; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment.PruneSpecsHolder; import org.apache.druid.timeline.SegmentId; @@ -341,7 +341,7 @@ public > QueryToolChest getToolChest new DruidHttpClientConfig(), processingConfig, forkJoinPool, - new QueryScheduler(8, NoQuerySchedulingStrategy.INSTANCE) + new QueryScheduler(Integer.MAX_VALUE, NoQuerySchedulingStrategy.INSTANCE) ); } diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index fd6f19a6639e..937917ff4553 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -53,7 +53,6 @@ import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; -import org.apache.druid.query.QueryScheduler; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.Result; @@ -63,11 +62,12 @@ import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.movingaverage.test.TestConfig; import org.apache.druid.query.planning.DataSourceAnalysis; -import org.apache.druid.query.scheduling.NoQuerySchedulingStrategy; import org.apache.druid.query.timeseries.TimeseriesQuery; import org.apache.druid.query.timeseries.TimeseriesResultValue; import org.apache.druid.server.ClientQuerySegmentWalker; +import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.initialization.ServerConfig; +import org.apache.druid.server.scheduling.NoQuerySchedulingStrategy; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.TimelineLookup; import org.hamcrest.core.IsInstanceOf; @@ -364,7 +364,7 @@ public String getFormatString() } }, ForkJoinPool.commonPool(), - new QueryScheduler(8, NoQuerySchedulingStrategy.INSTANCE) + new QueryScheduler(Integer.MAX_VALUE, NoQuerySchedulingStrategy.INSTANCE) ); ClientQuerySegmentWalker walker = new ClientQuerySegmentWalker( diff --git a/processing/src/main/java/org/apache/druid/query/QueryContexts.java b/processing/src/main/java/org/apache/druid/query/QueryContexts.java index 70cd1e61ea57..f5fcc554306b 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryContexts.java +++ b/processing/src/main/java/org/apache/druid/query/QueryContexts.java @@ -35,6 +35,7 @@ public class QueryContexts { public static final String PRIORITY_KEY = "priority"; + public static final String LANE_KEY = "lane"; public static final String TIMEOUT_KEY = "timeout"; public static final String MAX_SCATTER_GATHER_BYTES_KEY = "maxScatterGatherBytes"; public static final String MAX_QUEUED_BYTES_KEY = "maxQueuedBytes"; @@ -200,6 +201,11 @@ public static int getPriority(Query query, int defaultValue) return parseInt(query, PRIORITY_KEY, defaultValue); } + public static String getLane(Query query) + { + return (String) query.getContextValue(LANE_KEY); + } + public static boolean getEnableParallelMerges(Query query) { return parseBoolean(query, BROKER_PARALLEL_MERGE_KEY, DEFAULT_ENABLE_PARALLEL_MERGE); diff --git a/processing/src/main/java/org/apache/druid/query/QueryScheduler.java b/processing/src/main/java/org/apache/druid/query/QueryScheduler.java deleted file mode 100644 index 3920de3fbc4d..000000000000 --- a/processing/src/main/java/org/apache/druid/query/QueryScheduler.java +++ /dev/null @@ -1,158 +0,0 @@ -/* - * 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.query; - -import com.google.common.collect.HashMultimap; -import com.google.common.collect.Multimaps; -import com.google.common.collect.SetMultimap; -import com.google.common.collect.Sets; -import com.google.common.util.concurrent.ListenableFuture; -import it.unimi.dsi.fastutil.objects.Object2IntMap; -import org.apache.druid.java.util.common.concurrent.Execs; -import org.apache.druid.java.util.common.guava.Sequence; - -import java.util.HashMap; -import java.util.Map; -import java.util.Set; -import java.util.concurrent.Semaphore; -import java.util.concurrent.TimeUnit; - - -public class QueryScheduler implements QueryWatcher -{ - // maybe instead use a fancy library? - private final Semaphore activeQueries; - private final Map lanes; - private final QuerySchedulingStrategy strategy; - - private final SetMultimap> queries; - private final SetMultimap queryDatasources; - private final Set> runningQueries; - - public QueryScheduler( - int totalNumThreads, - QuerySchedulingStrategy strategy - ) - { - this.strategy = strategy; - this.activeQueries = new Semaphore(totalNumThreads); - this.lanes = new HashMap<>(); - - for (Object2IntMap.Entry entry : strategy.getLaneLimits().object2IntEntrySet()) { - lanes.put(entry.getKey(), new Semaphore(entry.getIntValue())); - } - this.queries = Multimaps.synchronizedSetMultimap( - HashMultimap.create() - ); - this.queryDatasources = Multimaps.synchronizedSetMultimap( - HashMultimap.create() - ); - this.runningQueries = Sets.newConcurrentHashSet(); - } - - public Query schedule(QueryPlus query, Set descriptors) - { - try { - if (activeQueries.tryAcquire(0, TimeUnit.MILLISECONDS)) { - Query prioritizedAndLaned = strategy.prioritizeQuery(query, descriptors); - String lane = prioritizedAndLaned.getContextValue("queryLane"); - if (lanes.containsKey(lane)) { - if (lanes.get(lane).tryAcquire(0, TimeUnit.MILLISECONDS)) { - runningQueries.add(prioritizedAndLaned); - return prioritizedAndLaned; - } - } else { - runningQueries.add(prioritizedAndLaned); - return prioritizedAndLaned; - } - } - } - catch (InterruptedException ex) { - throw new QueryCapacityExceededException(); - } - throw new QueryCapacityExceededException(); - } - - public Sequence run(Query query, Sequence resultSequence) - { - // if really doing release on the bagage, should merge most of schedule into here to tie acquire with release - // alternatively, if we want to release as late as possible, rework this to release in the registered queries future - // listener, though we need a guaranteed unique queryId and slight refactor to make this work.. - return resultSequence.withBaggage(() -> { - if (runningQueries.remove(query)) { - String lane = query.getContextValue("queryLane"); - if (lanes.containsKey(lane)) { - lanes.get(lane).release(); - } - activeQueries.release(); - } - }); - } - - public int getTotalAvailableCapacity() - { - return activeQueries.availablePermits(); - } - - public int getLaneAvailableCapacity(String lane) - { - if (lanes.containsKey(lane)) { - return lanes.get(lane).availablePermits(); - } - return -1; - } - - - public boolean cancelQuery(String id) - { - queryDatasources.removeAll(id); - Set> futures = queries.removeAll(id); - boolean success = true; - for (ListenableFuture future : futures) { - success = success && future.cancel(true); - } - return success; - } - - @Override - public void registerQuery(Query query, final ListenableFuture future) - { - final String id = query.getId(); - final Set datasources = query.getDataSource().getTableNames(); - queries.put(id, future); - queryDatasources.putAll(id, datasources); - future.addListener( - () -> { - // if you re-use queryId and cancel queries... you are going to have a bad time - queries.remove(id, future); - - for (String datasource : datasources) { - queryDatasources.remove(id, datasource); - } - }, - Execs.directExecutor() - ); - } - - public Set getQueryDatasources(final String queryId) - { - return queryDatasources.get(queryId); - } -} diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index 93358bbd9c4e..31d139fe6fb7 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -58,7 +58,6 @@ import org.apache.druid.query.QueryMetrics; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; -import org.apache.druid.query.QueryScheduler; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.QueryToolChestWarehouse; @@ -70,6 +69,7 @@ import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.query.spec.QuerySegmentSpec; import org.apache.druid.server.QueryResource; +import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.coordination.DruidServerMetadata; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -292,27 +292,28 @@ Sequence run(final UnaryOperator> time computeUncoveredIntervals(timeline); } - final Set segments = computeSegmentsToQuery(timeline); + final Set segmentServers = computeSegmentsToQuery(timeline); @Nullable final byte[] queryCacheKey = computeQueryCacheKey(); if (query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH) != null) { @Nullable final String prevEtag = (String) query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH); @Nullable - final String currentEtag = computeCurrentEtag(segments, queryCacheKey); + final String currentEtag = computeCurrentEtag(segmentServers, queryCacheKey); if (currentEtag != null && currentEtag.equals(prevEtag)) { return Sequences.empty(); } } - final List> alreadyCachedResults = pruneSegmentsWithCachedResults(queryCacheKey, segments); + final List> alreadyCachedResults = + pruneSegmentsWithCachedResults(queryCacheKey, segmentServers); - query = scheduler.schedule( + query = scheduler.prioritizeAndLaneQuery( queryPlus.withQuery(query), - segments.stream().map(x -> x.getSegmentDescriptor()).collect(Collectors.toSet()) + segmentServers ); - final SortedMap> segmentsByServer = groupSegmentsByServer(segments); + final SortedMap> segmentsByServer = groupSegmentsByServer(segmentServers); LazySequence sequence = new LazySequence<>(() -> { List> sequencesByInterval = new ArrayList<>(alreadyCachedResults.size() + segmentsByServer.size()); addSequencesFromCache(sequencesByInterval, alreadyCachedResults); @@ -359,14 +360,14 @@ private Sequence merge(List> sequencesByInterval) } } - private Set computeSegmentsToQuery(TimelineLookup timeline) + private Set computeSegmentsToQuery(TimelineLookup timeline) { final List> serversLookup = toolChest.filterSegments( query, intervals.stream().flatMap(i -> timeline.lookup(i).stream()).collect(Collectors.toList()) ); - final Set segments = new LinkedHashSet<>(); + final Set segments = new LinkedHashSet<>(); final Map>> dimensionRangeCache = new HashMap<>(); // Filter unneeded chunks based on partition dimension for (TimelineObjectHolder holder : serversLookup) { @@ -383,7 +384,7 @@ private Set computeSegmentsToQuery(TimelineLookup segments, @Nullable byte[] queryCacheKey) + private String computeCurrentEtag(final Set segments, @Nullable byte[] queryCacheKey) { Hasher hasher = Hashing.sha1().newHasher(); boolean hasOnlyHistoricalSegments = true; - for (ServerToSegment p : segments) { + for (SegmentServer p : segments) { if (!p.getServer().pick().getServer().segmentReplicatable()) { hasOnlyHistoricalSegments = false; break; @@ -472,14 +473,14 @@ private String computeCurrentEtag(final Set segments, @Nullable private List> pruneSegmentsWithCachedResults( final byte[] queryCacheKey, - final Set segments + final Set segments ) { if (queryCacheKey == null) { return Collections.emptyList(); } final List> alreadyCachedResults = new ArrayList<>(); - Map perSegmentCacheKeys = computePerSegmentCacheKeys(segments, queryCacheKey); + Map perSegmentCacheKeys = computePerSegmentCacheKeys(segments, queryCacheKey); // Pull cached segments from cache and remove from set of segments to query final Map cachedValues = computeCachedValues(perSegmentCacheKeys); @@ -500,25 +501,25 @@ private List> pruneSegmentsWithCachedResults( return alreadyCachedResults; } - private Map computePerSegmentCacheKeys( - Set segments, + private Map computePerSegmentCacheKeys( + Set segments, byte[] queryCacheKey ) { // cacheKeys map must preserve segment ordering, in order for shards to always be combined in the same order - Map cacheKeys = Maps.newLinkedHashMap(); - for (ServerToSegment serverToSegment : segments) { + Map cacheKeys = Maps.newLinkedHashMap(); + for (SegmentServer segmentServer : segments) { final Cache.NamedKey segmentCacheKey = CacheUtil.computeSegmentCacheKey( - serverToSegment.getServer().getSegment().getId().toString(), - serverToSegment.getSegmentDescriptor(), + segmentServer.getServer().getSegment().getId().toString(), + segmentServer.getSegmentDescriptor(), queryCacheKey ); - cacheKeys.put(serverToSegment, segmentCacheKey); + cacheKeys.put(segmentServer, segmentCacheKey); } return cacheKeys; } - private Map computeCachedValues(Map cacheKeys) + private Map computeCachedValues(Map cacheKeys) { if (useCache) { return cache.getBulk(Iterables.limit(cacheKeys.values(), cacheConfig.getCacheBulkMergeLimit())); @@ -542,21 +543,21 @@ private Cache.NamedKey getCachePopulatorKey(String segmentId, Interval segmentIn return cachePopulatorKeyMap.get(StringUtils.format("%s_%s", segmentId, segmentInterval)); } - private SortedMap> groupSegmentsByServer(Set segments) + private SortedMap> groupSegmentsByServer(Set segments) { final SortedMap> serverSegments = new TreeMap<>(); - for (ServerToSegment serverToSegment : segments) { - final QueryableDruidServer queryableDruidServer = serverToSegment.getServer().pick(); + for (SegmentServer segmentServer : segments) { + final QueryableDruidServer queryableDruidServer = segmentServer.getServer().pick(); if (queryableDruidServer == null) { log.makeAlert( "No servers found for SegmentDescriptor[%s] for DataSource[%s]?! How can this be?!", - serverToSegment.getSegmentDescriptor(), + segmentServer.getSegmentDescriptor(), query.getDataSource() ).emit(); } else { final DruidServer server = queryableDruidServer.getServer(); - serverSegments.computeIfAbsent(server, s -> new ArrayList<>()).add(serverToSegment.getSegmentDescriptor()); + serverSegments.computeIfAbsent(server, s -> new ArrayList<>()).add(segmentServer.getSegmentDescriptor()); } } return serverSegments; @@ -709,22 +710,4 @@ private Sequence getAndCacheServerResults( .flatMerge(seq -> seq, query.getResultOrdering()); } } - - private static class ServerToSegment extends Pair - { - private ServerToSegment(ServerSelector server, SegmentDescriptor segment) - { - super(server, segment); - } - - ServerSelector getServer() - { - return lhs; - } - - SegmentDescriptor getSegmentDescriptor() - { - return rhs; - } - } } diff --git a/server/src/main/java/org/apache/druid/client/SegmentServer.java b/server/src/main/java/org/apache/druid/client/SegmentServer.java new file mode 100644 index 000000000000..8499fbcf81a8 --- /dev/null +++ b/server/src/main/java/org/apache/druid/client/SegmentServer.java @@ -0,0 +1,42 @@ +/* + * 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.client; + +import org.apache.druid.client.selector.ServerSelector; +import org.apache.druid.java.util.common.Pair; +import org.apache.druid.query.SegmentDescriptor; + +public class SegmentServer extends Pair +{ + SegmentServer(ServerSelector server, SegmentDescriptor segment) + { + super(server, segment); + } + + public ServerSelector getServer() + { + return lhs; + } + + public SegmentDescriptor getSegmentDescriptor() + { + return rhs; + } +} diff --git a/server/src/main/java/org/apache/druid/guice/QueryRunnerFactoryModule.java b/server/src/main/java/org/apache/druid/guice/QueryRunnerFactoryModule.java index a68c1dd03b22..9570231ab487 100644 --- a/server/src/main/java/org/apache/druid/guice/QueryRunnerFactoryModule.java +++ b/server/src/main/java/org/apache/druid/guice/QueryRunnerFactoryModule.java @@ -27,8 +27,6 @@ import org.apache.druid.guice.annotations.Global; import org.apache.druid.query.Query; import org.apache.druid.query.QueryRunnerFactory; -import org.apache.druid.query.QueryScheduler; -import org.apache.druid.query.QuerySchedulerProvider; import org.apache.druid.query.QueryWatcher; import org.apache.druid.query.datasourcemetadata.DataSourceMetadataQuery; import org.apache.druid.query.datasourcemetadata.DataSourceMetadataQueryRunnerFactory; @@ -47,6 +45,8 @@ import org.apache.druid.query.timeseries.TimeseriesQueryRunnerFactory; import org.apache.druid.query.topn.TopNQuery; import org.apache.druid.query.topn.TopNQueryRunnerFactory; +import org.apache.druid.server.QueryScheduler; +import org.apache.druid.server.QuerySchedulerProvider; import java.util.Map; @@ -74,9 +74,6 @@ public void configure(Binder binder) binder.bind(QueryScheduler.class) .toProvider(Key.get(QuerySchedulerProvider.class, Global.class)) .in(LazySingleton.class); - // binder.bind(QueryWatcher.class) - // .toProvider(Key.get(QuerySchedulerProvider.class, Global.class)) - // .in(LazySingleton.class); JsonConfigProvider.bind(binder, "druid.query.scheduler", QuerySchedulerProvider.class, Global.class); final MapBinder, QueryRunnerFactory> queryFactoryBinder = DruidBinders.queryRunnerFactoryBinder( diff --git a/server/src/main/java/org/apache/druid/server/BrokerQueryResource.java b/server/src/main/java/org/apache/druid/server/BrokerQueryResource.java index c44c43b303ca..39e6291a134e 100644 --- a/server/src/main/java/org/apache/druid/server/BrokerQueryResource.java +++ b/server/src/main/java/org/apache/druid/server/BrokerQueryResource.java @@ -29,7 +29,6 @@ import org.apache.druid.guice.annotations.Smile; import org.apache.druid.query.GenericQueryMetricsFactory; import org.apache.druid.query.Query; -import org.apache.druid.query.QueryScheduler; import org.apache.druid.server.http.security.StateResourceFilter; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthorizerMapper; diff --git a/processing/src/main/java/org/apache/druid/query/QueryCapacityExceededException.java b/server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java similarity index 68% rename from processing/src/main/java/org/apache/druid/query/QueryCapacityExceededException.java rename to server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java index 1fc0714cfa8b..e3893b1ef583 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryCapacityExceededException.java +++ b/server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java @@ -17,12 +17,23 @@ * under the License. */ -package org.apache.druid.query; +package org.apache.druid.server; + +import org.apache.druid.java.util.common.StringUtils; public class QueryCapacityExceededException extends RuntimeException { + public static String ERROR_MESSAGE = "Query capacity exceeded"; + public static String ERROR_MESSAGE_TEMPLATE = "Query capacity exceeded for lane %s"; + public static int STATUS_CODE = 429; + public QueryCapacityExceededException() { - super("too many cooks"); + super(ERROR_MESSAGE); + } + + public QueryCapacityExceededException(String lane) + { + super(StringUtils.format(ERROR_MESSAGE_TEMPLATE, lane)); } } diff --git a/server/src/main/java/org/apache/druid/server/QueryResource.java b/server/src/main/java/org/apache/druid/server/QueryResource.java index 6203f7bf57e7..2f17284e288a 100644 --- a/server/src/main/java/org/apache/druid/server/QueryResource.java +++ b/server/src/main/java/org/apache/druid/server/QueryResource.java @@ -41,10 +41,8 @@ import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.query.GenericQueryMetricsFactory; import org.apache.druid.query.Query; -import org.apache.druid.query.QueryCapacityExceededException; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryInterruptedException; -import org.apache.druid.query.QueryScheduler; import org.apache.druid.query.QueryToolChest; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.server.metrics.QueryCountStatsProvider; @@ -444,7 +442,7 @@ Response gotError(Exception e) throws IOException Response gotLimited(QueryCapacityExceededException e) { - return Response.status(429).entity(e.getMessage()).build(); + return Response.status(QueryCapacityExceededException.STATUS_CODE).entity(e.getMessage()).build(); } } diff --git a/server/src/main/java/org/apache/druid/server/QueryScheduler.java b/server/src/main/java/org/apache/druid/server/QueryScheduler.java new file mode 100644 index 000000000000..57a10a8f71d0 --- /dev/null +++ b/server/src/main/java/org/apache/druid/server/QueryScheduler.java @@ -0,0 +1,203 @@ +/* + * 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; + +import com.google.common.collect.HashMultimap; +import com.google.common.collect.Multimaps; +import com.google.common.collect.SetMultimap; +import com.google.common.collect.Sets; +import com.google.common.util.concurrent.ListenableFuture; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +import org.apache.druid.client.SegmentServer; +import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.QueryWatcher; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; +import java.util.Set; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; + + +/** + * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' and potentially + * priority using the + * {@link QuerySchedulingStrategy} that is defined in {@link QuerySchedulerConfig} + * The purpose of the QueryScheduler is to give overall visibility into queries running + * or pending at the QueryRunner level. This is currently used to cancel all the + * parts of a pending query, but may be expanded in the future to offer more direct + * visibility into query execution and resource usage. + * + * QueryRunners executing any computation asynchronously must register their queries with the QueryScheduler. + * + */ +public class QueryScheduler implements QueryWatcher +{ + private final QuerySchedulingStrategy strategy; + // maybe instead use a fancy library? + private final Optional totalQueryLimit; + private final Map laneLimits; + + private final SetMultimap> queries; + private final SetMultimap queryDatasources; + private final Set> runningQueries; + + public QueryScheduler(int totalNumThreads, QuerySchedulingStrategy strategy) + { + this.strategy = strategy; + if (totalNumThreads > 0) { + this.totalQueryLimit = Optional.of(new Semaphore(totalNumThreads)); + } else { + this.totalQueryLimit = Optional.empty(); + } + this.laneLimits = new HashMap<>(); + + for (Object2IntMap.Entry entry : strategy.getLaneLimits().object2IntEntrySet()) { + laneLimits.put(entry.getKey(), new Semaphore(entry.getIntValue())); + } + this.queries = Multimaps.synchronizedSetMultimap( + HashMultimap.create() + ); + this.queryDatasources = Multimaps.synchronizedSetMultimap( + HashMultimap.create() + ); + this.runningQueries = Sets.newConcurrentHashSet(); + } + + public Query prioritizeAndLaneQuery(QueryPlus query, Set segments) + { + return strategy.prioritizeAndLaneQuery(query, segments); + } + + public void scheduleQuery(Query query) + { + final String lane; + try { + if (!totalQueryLimit.isPresent() || totalQueryLimit.get().tryAcquire(0, TimeUnit.MILLISECONDS)) { + lane = QueryContexts.getLane(query); + if (!laneLimits.containsKey(lane)) { + runningQueries.add(query); + return; + } + } else { + throw new QueryCapacityExceededException(); + } + } + catch (InterruptedException ex) { + throw new QueryCapacityExceededException(); + } + try { + if (laneLimits.get(lane).tryAcquire(0, TimeUnit.MILLISECONDS)) { + runningQueries.add(query); + return; + } else { + throw new QueryCapacityExceededException(lane); + } + } + catch (InterruptedException e) { + throw new QueryCapacityExceededException(lane); + } + } + + public void completeQuery(Query query) + { + if (runningQueries.remove(query)) { + String lane = QueryContexts.getLane(query); + if (laneLimits.containsKey(lane)) { + laneLimits.get(lane).release(); + } + totalQueryLimit.ifPresent(Semaphore::release); + } + } + + public Sequence run(Query query, Sequence resultSequence) + { + scheduleQuery(query); + return resultSequence.withBaggage(() -> completeQuery(query)); + } + + public int getTotalAvailableCapacity() + { + return totalQueryLimit.map(Semaphore::availablePermits).orElse(-1); + } + + public int getLaneAvailableCapacity(String lane) + { + if (laneLimits.containsKey(lane)) { + return laneLimits.get(lane).availablePermits(); + } + return -1; + } + + + public boolean cancelQuery(String id) + { + queryDatasources.removeAll(id); + Set> futures = queries.removeAll(id); + boolean success = true; + for (ListenableFuture future : futures) { + success = success && future.cancel(true); + } + return success; + } + + /** + * QueryRunners must use this method to register any pending queries. + * + * The given future may have cancel(true) called at any time, if cancellation of this query has been requested. + * + * @param query a query, which may be a subset of a larger query, as long as the underlying queryId is unchanged + * @param future the future holding the execution status of the query + */ + public void registerQueryFuture(Query query, final ListenableFuture future) + { + final String id = query.getId(); + final Set datasources = query.getDataSource().getTableNames(); + queries.put(id, future); + queryDatasources.putAll(id, datasources); + future.addListener( + () -> { + // if you re-use queryId and cancel queries... you are going to have a bad time + queries.remove(id, future); + + for (String datasource : datasources) { + queryDatasources.remove(id, datasource); + } + }, + Execs.directExecutor() + ); + } + + public Set getQueryDatasources(final String queryId) + { + return queryDatasources.get(queryId); + } + + @Override + public void registerQuery(Query query, ListenableFuture future) + { + registerQueryFuture(query, future); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/QuerySchedulerConfig.java b/server/src/main/java/org/apache/druid/server/QuerySchedulerConfig.java similarity index 97% rename from processing/src/main/java/org/apache/druid/query/QuerySchedulerConfig.java rename to server/src/main/java/org/apache/druid/server/QuerySchedulerConfig.java index c27c6a336956..5052b4b7ed92 100644 --- a/processing/src/main/java/org/apache/druid/query/QuerySchedulerConfig.java +++ b/server/src/main/java/org/apache/druid/server/QuerySchedulerConfig.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.query; +package org.apache.druid.server; import com.fasterxml.jackson.annotation.JsonProperty; diff --git a/processing/src/main/java/org/apache/druid/query/QuerySchedulerProvider.java b/server/src/main/java/org/apache/druid/server/QuerySchedulerProvider.java similarity index 96% rename from processing/src/main/java/org/apache/druid/query/QuerySchedulerProvider.java rename to server/src/main/java/org/apache/druid/server/QuerySchedulerProvider.java index 11867cbfecf5..3f46b9d1ef1d 100644 --- a/processing/src/main/java/org/apache/druid/query/QuerySchedulerProvider.java +++ b/server/src/main/java/org/apache/druid/server/QuerySchedulerProvider.java @@ -17,7 +17,7 @@ * under the License. */ -package org.apache.druid.query; +package org.apache.druid.server; import com.google.inject.Provider; diff --git a/processing/src/main/java/org/apache/druid/query/QuerySchedulingStrategy.java b/server/src/main/java/org/apache/druid/server/QuerySchedulingStrategy.java similarity index 78% rename from processing/src/main/java/org/apache/druid/query/QuerySchedulingStrategy.java rename to server/src/main/java/org/apache/druid/server/QuerySchedulingStrategy.java index 4e8a9124f635..435dfa17cc08 100644 --- a/processing/src/main/java/org/apache/druid/query/QuerySchedulingStrategy.java +++ b/server/src/main/java/org/apache/druid/server/QuerySchedulingStrategy.java @@ -17,13 +17,16 @@ * under the License. */ -package org.apache.druid.query; +package org.apache.druid.server; import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import it.unimi.dsi.fastutil.objects.Object2IntMap; -import org.apache.druid.query.scheduling.HiLoQuerySchedulingStrategy; -import org.apache.druid.query.scheduling.NoQuerySchedulingStrategy; +import org.apache.druid.client.SegmentServer; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.server.scheduling.HiLoQuerySchedulingStrategy; +import org.apache.druid.server.scheduling.NoQuerySchedulingStrategy; import java.util.Set; @@ -37,5 +40,5 @@ public interface QuerySchedulingStrategy { Object2IntMap getLaneLimits(); - Query prioritizeQuery(QueryPlus query, Set segments); + Query prioritizeAndLaneQuery(QueryPlus query, Set segments); } diff --git a/processing/src/main/java/org/apache/druid/query/scheduling/HiLoQuerySchedulingStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQuerySchedulingStrategy.java similarity index 67% rename from processing/src/main/java/org/apache/druid/query/scheduling/HiLoQuerySchedulingStrategy.java rename to server/src/main/java/org/apache/druid/server/scheduling/HiLoQuerySchedulingStrategy.java index 01dec3dd96f6..58615fe1f41f 100644 --- a/processing/src/main/java/org/apache/druid/query/scheduling/HiLoQuerySchedulingStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQuerySchedulingStrategy.java @@ -17,22 +17,25 @@ * under the License. */ -package org.apache.druid.query.scheduling; +package org.apache.druid.server.scheduling; import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.collect.ImmutableMap; import it.unimi.dsi.fastutil.objects.Object2IntArrayMap; import it.unimi.dsi.fastutil.objects.Object2IntMap; +import org.apache.druid.client.SegmentServer; import org.apache.druid.query.Query; +import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryPlus; -import org.apache.druid.query.QuerySchedulingStrategy; -import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.server.QuerySchedulingStrategy; import java.util.Set; public class HiLoQuerySchedulingStrategy implements QuerySchedulingStrategy { + public static String LOW = "low"; + @JsonProperty private int maxLowThreads; @@ -48,18 +51,23 @@ public HiLoQuerySchedulingStrategy( public Object2IntMap getLaneLimits() { Object2IntMap onlyLow = new Object2IntArrayMap<>(1); - onlyLow.put("low", maxLowThreads); + onlyLow.put(LOW, maxLowThreads); return onlyLow; } @Override - public Query prioritizeQuery(QueryPlus query, Set segments) + public Query prioritizeAndLaneQuery(QueryPlus query, Set segments) { final Query theQuery = query.getQuery(); - final Integer priority = theQuery.getContextValue("priority"); - final String lane = theQuery.getContextValue("queryLane"); + // QueryContexts.getPriority gives a default, since we are setting priority + final Integer priority = theQuery.getContextValue(QueryContexts.PRIORITY_KEY); + final String lane = theQuery.getContextValue(QueryContexts.LANE_KEY); if (lane == null && priority != null && priority < 0) { - return theQuery.withOverriddenContext(ImmutableMap.builder().putAll(theQuery.getContext()).put("queryLane", "low").build()); + return theQuery.withOverriddenContext( + ImmutableMap.builder().putAll(theQuery.getContext()) + .put(QueryContexts.LANE_KEY, LOW) + .build() + ); } return theQuery; } diff --git a/processing/src/main/java/org/apache/druid/query/scheduling/NoQuerySchedulingStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/NoQuerySchedulingStrategy.java similarity index 85% rename from processing/src/main/java/org/apache/druid/query/scheduling/NoQuerySchedulingStrategy.java rename to server/src/main/java/org/apache/druid/server/scheduling/NoQuerySchedulingStrategy.java index 298f6b107e4e..18ab8d24e57b 100644 --- a/processing/src/main/java/org/apache/druid/query/scheduling/NoQuerySchedulingStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/NoQuerySchedulingStrategy.java @@ -17,14 +17,14 @@ * under the License. */ -package org.apache.druid.query.scheduling; +package org.apache.druid.server.scheduling; import it.unimi.dsi.fastutil.objects.Object2IntArrayMap; import it.unimi.dsi.fastutil.objects.Object2IntMap; +import org.apache.druid.client.SegmentServer; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; -import org.apache.druid.query.QuerySchedulingStrategy; -import org.apache.druid.query.SegmentDescriptor; +import org.apache.druid.server.QuerySchedulingStrategy; import java.util.Set; @@ -41,7 +41,7 @@ public Object2IntMap getLaneLimits() } @Override - public Query prioritizeQuery(QueryPlus query, Set segments) + public Query prioritizeAndLaneQuery(QueryPlus query, Set segments) { return query.getQuery(); } diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java index b1fc4426b01f..337cc4903fcd 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java @@ -43,13 +43,13 @@ import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; -import org.apache.druid.query.QueryScheduler; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.planning.DataSourceAnalysis; -import org.apache.druid.query.scheduling.NoQuerySchedulingStrategy; +import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.scheduling.NoQuerySchedulingStrategy; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.TimelineLookup; import org.apache.druid.timeline.VersionedIntervalTimeline; @@ -334,7 +334,7 @@ public int getMergePoolParallelism() }, ForkJoinPool.commonPool(), // need at least 9 total since runner doesn't actually run queries and release semaphores - new QueryScheduler(10, NoQuerySchedulingStrategy.INSTANCE) + new QueryScheduler(Integer.MAX_VALUE, NoQuerySchedulingStrategy.INSTANCE) ); } diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index 509c3f6e0aef..e63ed66e7e44 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -74,7 +74,6 @@ import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; -import org.apache.druid.query.QueryScheduler; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.Result; import org.apache.druid.query.SegmentDescriptor; @@ -99,7 +98,6 @@ import org.apache.druid.query.groupby.strategy.GroupByStrategySelector; import org.apache.druid.query.ordering.StringComparators; import org.apache.druid.query.planning.DataSourceAnalysis; -import org.apache.druid.query.scheduling.NoQuerySchedulingStrategy; import org.apache.druid.query.search.SearchHit; import org.apache.druid.query.search.SearchQuery; import org.apache.druid.query.search.SearchQueryConfig; @@ -118,7 +116,9 @@ import org.apache.druid.query.topn.TopNQueryQueryToolChest; import org.apache.druid.query.topn.TopNResultValue; import org.apache.druid.segment.TestHelper; +import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.scheduling.NoQuerySchedulingStrategy; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.VersionedIntervalTimeline; @@ -2476,7 +2476,7 @@ public int getMergePoolParallelism() } }, ForkJoinPool.commonPool(), - new QueryScheduler(8, NoQuerySchedulingStrategy.INSTANCE) + new QueryScheduler(Integer.MAX_VALUE, NoQuerySchedulingStrategy.INSTANCE) ); } diff --git a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java index 5251f026f2ab..23566bc89923 100644 --- a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java @@ -22,11 +22,15 @@ import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.jaxrs.smile.SmileMediaTypes; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.jackson.DefaultObjectMapper; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; +import org.apache.druid.java.util.common.guava.LazySequence; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.emitter.EmittingLogger; @@ -34,18 +38,19 @@ import org.apache.druid.query.DefaultGenericQueryMetricsFactory; import org.apache.druid.query.MapQueryToolChestWarehouse; import org.apache.druid.query.Query; +import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; -import org.apache.druid.query.QueryScheduler; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.Result; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.context.ResponseContext; -import org.apache.druid.query.scheduling.NoQuerySchedulingStrategy; import org.apache.druid.query.timeboundary.TimeBoundaryResultValue; import org.apache.druid.server.log.TestRequestLogger; import org.apache.druid.server.metrics.NoopServiceEmitter; +import org.apache.druid.server.scheduling.HiLoQuerySchedulingStrategy; +import org.apache.druid.server.scheduling.NoQuerySchedulingStrategy; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthConfig; @@ -71,9 +76,12 @@ import java.io.ByteArrayInputStream; import java.io.ByteArrayOutputStream; import java.io.IOException; +import java.util.Collection; import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executors; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Consumer; /** * @@ -127,7 +135,7 @@ public void setup() EasyMock.expect(testServletRequest.getHeader("Accept")).andReturn(MediaType.APPLICATION_JSON).anyTimes(); EasyMock.expect(testServletRequest.getHeader(QueryResource.HEADER_IF_NONE_MATCH)).andReturn(null).anyTimes(); EasyMock.expect(testServletRequest.getRemoteAddr()).andReturn("localhost").anyTimes(); - queryScheduler = new QueryScheduler(20, NoQuerySchedulingStrategy.INSTANCE); + queryScheduler = new QueryScheduler(8, NoQuerySchedulingStrategy.INSTANCE); testRequestLogger = new TestRequestLogger(); queryResource = new QueryResource( new QueryLifecycleFactory( @@ -163,22 +171,28 @@ public void setup() + " ]\n" + "}"; + private static final String SIMPLE_TIMESERIES_QUERY_LOW_PRIORITY = + "{\n" + + " \"queryType\": \"timeseries\",\n" + + " \"dataSource\": \"mmx_metrics\",\n" + + " \"granularity\": \"hour\",\n" + + " \"intervals\": [\n" + + " \"2014-12-17/2015-12-30\"\n" + + " ],\n" + + " \"aggregations\": [\n" + + " {\n" + + " \"type\": \"count\",\n" + + " \"name\": \"rows\"\n" + + " }\n" + + " ],\n" + + " \"context\": { \"priority\": -1 }" + + "}"; + @Test public void testGoodQuery() throws IOException { - EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) - .andReturn(null) - .anyTimes(); - EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).anyTimes(); + expectPermissiveHappyPath(); - EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) - .andReturn(AUTHENTICATION_RESULT) - .anyTimes(); - - testServletRequest.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); - EasyMock.expectLastCall().anyTimes(); - - EasyMock.replay(testServletRequest); Response response = queryResource.doPost( new ByteArrayInputStream(SIMPLE_TIMESERIES_QUERY.getBytes("UTF-8")), null /*pretty*/, @@ -513,7 +527,7 @@ public void run() } ); - queryScheduler.registerQuery(query, future); + queryScheduler.registerQueryFuture(query, future); startAwaitLatch.await(); Executors.newSingleThreadExecutor().submit( @@ -635,7 +649,7 @@ public void run() } ); - queryScheduler.registerQuery(query, future); + queryScheduler.registerQueryFuture(query, future); startAwaitLatch.await(); Executors.newSingleThreadExecutor().submit( @@ -657,9 +671,178 @@ public void run() waitFinishLatch.await(); } + @Test(timeout = 10_000L) + public void testTooManyQuery() throws InterruptedException + { + expectPermissiveHappyPath(); + + final CountDownLatch waitFirstStart = new CountDownLatch(1); + final CountDownLatch waitSecondStart = new CountDownLatch(2); + final CountDownLatch waitFinishLatch = new CountDownLatch(3); + final QueryScheduler laningScheduler = new QueryScheduler(2, NoQuerySchedulingStrategy.INSTANCE); + + createScheduledQueryResource(laningScheduler, ImmutableList.of(waitFirstStart, waitSecondStart)); + assertResponseAndCountdownOrBlockForever( + SIMPLE_TIMESERIES_QUERY, + waitFinishLatch, + response -> Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()) + ); + waitFirstStart.await(); + assertResponseAndCountdownOrBlockForever( + SIMPLE_TIMESERIES_QUERY, + waitFinishLatch, + response -> Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()) + ); + waitSecondStart.await(); + assertResponseAndCountdownOrBlockForever( + SIMPLE_TIMESERIES_QUERY, + waitFinishLatch, + response -> { + Assert.assertEquals(QueryCapacityExceededException.STATUS_CODE, response.getStatus()); + Assert.assertEquals(QueryCapacityExceededException.ERROR_MESSAGE, response.getEntity()); + } + ); + waitFinishLatch.await(); + } + + @Test(timeout = 10_000L) + public void testTooManyQueryInLane() throws InterruptedException + { + expectPermissiveHappyPath(); + final CountDownLatch waitFirstStart = new CountDownLatch(1); + final CountDownLatch waitSecondStart = new CountDownLatch(2); + final CountDownLatch waitFinishLatch = new CountDownLatch(3); + final QueryScheduler scheduler = new QueryScheduler(40, new HiLoQuerySchedulingStrategy(1)); + + createScheduledQueryResource(scheduler, ImmutableList.of(waitFirstStart, waitSecondStart)); + + assertResponseAndCountdownOrBlockForever( + SIMPLE_TIMESERIES_QUERY_LOW_PRIORITY, + waitFinishLatch, + response -> Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()) + ); + waitFirstStart.await(); + assertResponseAndCountdownOrBlockForever( + SIMPLE_TIMESERIES_QUERY_LOW_PRIORITY, + waitFinishLatch, + response -> { + Assert.assertEquals(QueryCapacityExceededException.STATUS_CODE, response.getStatus()); + Assert.assertEquals( + StringUtils.format( + QueryCapacityExceededException.ERROR_MESSAGE_TEMPLATE, + HiLoQuerySchedulingStrategy.LOW + ), + response.getEntity()); + } + ); + waitSecondStart.await(); + assertResponseAndCountdownOrBlockForever( + SIMPLE_TIMESERIES_QUERY, + waitFinishLatch, + response -> Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()) + ); + + waitFinishLatch.await(); + } + @After public void tearDown() { EasyMock.verify(testServletRequest); } + + + + private void createScheduledQueryResource(QueryScheduler scheduler, Collection latches) + { + AtomicInteger counter = new AtomicInteger(0); + + QuerySegmentWalker texasRanger = new QuerySegmentWalker() + { + @Override + public QueryRunner getQueryRunnerForIntervals(Query query, Iterable intervals) + { + return (queryPlus, responseContext) -> { + int count = counter.getAndIncrement(); + if (count < 2) { + queryPlus = queryPlus.withQuery( + queryPlus.getQuery().withOverriddenContext(ImmutableMap.of(QueryContexts.PRIORITY_KEY, -1)) + ); + } + latches.forEach(CountDownLatch::countDown); + + return scheduler.run( + scheduler.prioritizeAndLaneQuery(queryPlus, ImmutableSet.of()), + new LazySequence(() -> { + try { + Thread.sleep(500); + } + catch (InterruptedException ignored) { + } + return Sequences.empty(); + }) + ); + }; + } + + @Override + public QueryRunner getQueryRunnerForSegments(Query query, Iterable specs) + { + return getQueryRunnerForIntervals(null, null); + } + }; + + queryResource = new QueryResource( + new QueryLifecycleFactory( + WAREHOUSE, + texasRanger, + new DefaultGenericQueryMetricsFactory(), + new NoopServiceEmitter(), + testRequestLogger, + new AuthConfig(), + AuthTestUtils.TEST_AUTHORIZER_MAPPER + ), + JSON_MAPPER, + JSON_MAPPER, + scheduler, + new AuthConfig(), + null, + new DefaultGenericQueryMetricsFactory() + ); + } + + private void assertResponseAndCountdownOrBlockForever(String query, CountDownLatch waitFinishLatch, Consumer asserts) + { + Executors.newSingleThreadExecutor().submit(() -> { + try { + Response response = queryResource.doPost( + new ByteArrayInputStream(query.getBytes("UTF-8")), + null, + testServletRequest + ); + asserts.accept(response); + } + catch (IOException e) { + throw new RuntimeException(e); + } + waitFinishLatch.countDown(); + }); + } + + private void expectPermissiveHappyPath() + { + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) + .andReturn(null) + .anyTimes(); + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).anyTimes(); + + EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) + .andReturn(AUTHENTICATION_RESULT) + .anyTimes(); + + testServletRequest.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); + EasyMock.expectLastCall().anyTimes(); + + EasyMock.replay(testServletRequest); + } } diff --git a/processing/src/test/java/org/apache/druid/query/QuerySchedulerTest.java b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java similarity index 67% rename from processing/src/test/java/org/apache/druid/query/QuerySchedulerTest.java rename to server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java index 9d1de0ace97a..816dd45ddf0c 100644 --- a/processing/src/test/java/org/apache/druid/query/QuerySchedulerTest.java +++ b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java @@ -17,21 +17,27 @@ * under the License. */ -package org.apache.druid.query; +package org.apache.druid.server; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.MoreExecutors; +import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.SequenceWrapper; +import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.query.Query; +import org.apache.druid.query.QueryContexts; +import org.apache.druid.query.QueryPlus; import org.apache.druid.query.aggregation.CountAggregatorFactory; -import org.apache.druid.query.scheduling.HiLoQuerySchedulingStrategy; import org.apache.druid.query.topn.TopNQuery; import org.apache.druid.query.topn.TopNQueryBuilder; +import org.apache.druid.server.scheduling.HiLoQuerySchedulingStrategy; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -57,13 +63,20 @@ public void testHiLoHi() throws ExecutionException, InterruptedException Execs.singleThreaded("test_query_scheduler_%s") ).submit(() -> { try { - Query scheduled = scheduler.schedule(QueryPlus.wrap(interactive), ImmutableSet.of()); + Query scheduled = scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(interactive), ImmutableSet.of()); Assert.assertNotNull(scheduled); - Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); - Assert.assertEquals(2, scheduler.getLaneAvailableCapacity("low")); Sequence underlyingSequence = makeSequence(10); + underlyingSequence = Sequences.wrap(underlyingSequence, new SequenceWrapper() + { + @Override + public void before() + { + Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); + Assert.assertEquals(2, scheduler.getLaneAvailableCapacity(HiLoQuerySchedulingStrategy.LOW)); + } + }); Sequence results = scheduler.run(scheduled, underlyingSequence); int rowCount = consumeAndCloseSequence(results); @@ -73,7 +86,7 @@ public void testHiLoHi() throws ExecutionException, InterruptedException throw new RuntimeException(ex); } }); - scheduler.registerQuery(interactive, future); + scheduler.registerQueryFuture(interactive, future); future.get(); Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); } @@ -87,13 +100,20 @@ public void testHiLoLo() throws ExecutionException, InterruptedException Execs.singleThreaded("test_query_scheduler_%s") ).submit(() -> { try { - Query scheduledReport = scheduler.schedule(QueryPlus.wrap(report), ImmutableSet.of()); + Query scheduledReport = scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(report), ImmutableSet.of()); Assert.assertNotNull(scheduledReport); - Assert.assertEquals("low", scheduledReport.getContextValue("queryLane")); - Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); - Assert.assertEquals(1, scheduler.getLaneAvailableCapacity("low")); + Assert.assertEquals(HiLoQuerySchedulingStrategy.LOW, QueryContexts.getLane(scheduledReport)); Sequence underlyingSequence = makeSequence(10); + underlyingSequence = Sequences.wrap(underlyingSequence, new SequenceWrapper() + { + @Override + public void before() + { + Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); + Assert.assertEquals(1, scheduler.getLaneAvailableCapacity(HiLoQuerySchedulingStrategy.LOW)); + } + }); Sequence results = scheduler.run(scheduledReport, underlyingSequence); int rowCount = consumeAndCloseSequence(results); @@ -103,10 +123,10 @@ public void testHiLoLo() throws ExecutionException, InterruptedException throw new RuntimeException(ex); } }); - scheduler.registerQuery(report, future); + scheduler.registerQueryFuture(report, future); future.get(); Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); - Assert.assertEquals(2, scheduler.getLaneAvailableCapacity("low")); + Assert.assertEquals(2, scheduler.getLaneAvailableCapacity(HiLoQuerySchedulingStrategy.LOW)); } @@ -123,12 +143,19 @@ public void testHiLoReleaseSemaphoreWhenSequenceExplodes() throws Exception Execs.singleThreaded("test_query_scheduler_%s") ).submit(() -> { try { - Query scheduled = scheduler.schedule(QueryPlus.wrap(interactive), ImmutableSet.of()); + Query scheduled = scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(interactive), ImmutableSet.of()); Assert.assertNotNull(scheduled); - Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); Sequence underlyingSequence = makeExplodingSequence(10); + underlyingSequence = Sequences.wrap(underlyingSequence, new SequenceWrapper() + { + @Override + public void before() + { + Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); + } + }); Sequence results = scheduler.run(scheduled, underlyingSequence); consumeAndCloseSequence(results); @@ -137,7 +164,7 @@ public void testHiLoReleaseSemaphoreWhenSequenceExplodes() throws Exception throw new RuntimeException(ex); } }); - scheduler.registerQuery(interactive, future); + scheduler.registerQueryFuture(interactive, future); future.get(); Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); } @@ -145,57 +172,67 @@ public void testHiLoReleaseSemaphoreWhenSequenceExplodes() throws Exception @Test public void testHiLoFailsWhenOutOfLaneCapacity() { - expected.expectMessage("too many cooks"); + expected.expectMessage( + StringUtils.format(QueryCapacityExceededException.ERROR_MESSAGE_TEMPLATE, HiLoQuerySchedulingStrategy.LOW) + ); expected.expect(QueryCapacityExceededException.class); QueryScheduler scheduler = new QueryScheduler(5, new HiLoQuerySchedulingStrategy(2)); - Query report1 = scheduler.schedule(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); + Query report1 = scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); + scheduler.scheduleQuery(report1); Assert.assertNotNull(report1); Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); - Assert.assertEquals(1, scheduler.getLaneAvailableCapacity("low")); + Assert.assertEquals(1, scheduler.getLaneAvailableCapacity(HiLoQuerySchedulingStrategy.LOW)); - Query report2 = scheduler.schedule(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); + Query report2 = scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); + scheduler.scheduleQuery(report2); Assert.assertNotNull(report2); Assert.assertEquals(3, scheduler.getTotalAvailableCapacity()); - Assert.assertEquals(0, scheduler.getLaneAvailableCapacity("low")); + Assert.assertEquals(0, scheduler.getLaneAvailableCapacity(HiLoQuerySchedulingStrategy.LOW)); // too many reports - scheduler.schedule(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); + scheduler.scheduleQuery(scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of())); } @Test public void testHiLoFailsWhenOutOfTotalCapacity() { - expected.expectMessage("too many cooks"); + expected.expectMessage(QueryCapacityExceededException.ERROR_MESSAGE); expected.expect(QueryCapacityExceededException.class); - QueryScheduler scheduler = new QueryScheduler(5, new HiLoQuerySchedulingStrategy(2)); - Query interactive1 = scheduler.schedule(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()); + Query interactive1 = scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()); + scheduler.scheduleQuery(interactive1); Assert.assertNotNull(interactive1); Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); - Query report1 = scheduler.schedule(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); + Query report1 = scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); + scheduler.scheduleQuery(report1); Assert.assertNotNull(report1); Assert.assertEquals(3, scheduler.getTotalAvailableCapacity()); - Assert.assertEquals(1, scheduler.getLaneAvailableCapacity("low")); + Assert.assertEquals(1, scheduler.getLaneAvailableCapacity(HiLoQuerySchedulingStrategy.LOW)); - Query interactive2 = scheduler.schedule(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()); + Query interactive2 = scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()); + scheduler.scheduleQuery(interactive2); Assert.assertNotNull(interactive2); Assert.assertEquals(2, scheduler.getTotalAvailableCapacity()); - Query report2 = scheduler.schedule(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); + Query report2 = scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); + scheduler.scheduleQuery(report2); Assert.assertNotNull(report2); Assert.assertEquals(1, scheduler.getTotalAvailableCapacity()); - Assert.assertEquals(0, scheduler.getLaneAvailableCapacity("low")); + Assert.assertEquals(0, scheduler.getLaneAvailableCapacity(HiLoQuerySchedulingStrategy.LOW)); - Query interactive3 = scheduler.schedule(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()); + Query interactive3 = scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()); + scheduler.scheduleQuery(interactive3); Assert.assertNotNull(interactive3); Assert.assertEquals(0, scheduler.getTotalAvailableCapacity()); // one too many - scheduler.schedule(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()); + scheduler.scheduleQuery( + scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()) + ); } diff --git a/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java b/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java index 00c8cbd1eeb4..d96a5ee85e33 100644 --- a/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java +++ b/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java @@ -32,8 +32,8 @@ import org.apache.druid.java.util.common.guava.Yielder; import org.apache.druid.java.util.common.guava.Yielders; import org.apache.druid.java.util.common.logger.Logger; -import org.apache.druid.query.QueryCapacityExceededException; import org.apache.druid.query.QueryInterruptedException; +import org.apache.druid.server.QueryCapacityExceededException; import org.apache.druid.server.security.ForbiddenException; import org.apache.druid.sql.SqlLifecycle; import org.apache.druid.sql.SqlLifecycleFactory; From 554b8b514a6c92d04f806960eaf33ebb366ab21a Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 25 Feb 2020 10:41:00 -0800 Subject: [PATCH 04/37] adjustments --- .../CachingClusteredClientBenchmark.java | 4 +- .../movingaverage/MovingAverageQueryTest.java | 4 +- .../common/task/RealtimeIndexTaskTest.java | 2 +- .../query/ChainedExecutionQueryRunner.java | 2 +- .../druid/query/GroupByMergedQueryRunner.java | 2 +- .../org/apache/druid/query/QueryWatcher.java | 2 +- .../GroupByMergingQueryRunnerV2.java | 2 +- .../SegmentMetadataQueryRunnerFactory.java | 2 +- .../ChainedExecutionQueryRunnerTest.java | 4 +- ...ByLimitPushDownInsufficientBufferTest.java | 2 +- .../groupby/GroupByMultiSegmentTest.java | 2 +- .../druid/client/CachingClusteredClient.java | 18 +- .../druid/client/DirectDruidClient.java | 2 +- .../apache/druid/client/SegmentServer.java | 2 +- ...Strategy.java => QueryLaningStrategy.java} | 14 +- .../apache/druid/server/QueryScheduler.java | 174 +++++++++--------- .../druid/server/QuerySchedulerConfig.java | 11 +- .../druid/server/QuerySchedulerProvider.java | 2 +- ...tegy.java => HiLoQueryLaningStrategy.java} | 8 +- ...rategy.java => NoQueryLaningStrategy.java} | 8 +- ...chingClusteredClientFunctionalityTest.java | 4 +- .../client/CachingClusteredClientTest.java | 4 +- .../druid/server/QueryResourceTest.java | 14 +- .../druid/server/QuerySchedulerTest.java | 54 +++--- 24 files changed, 170 insertions(+), 173 deletions(-) rename server/src/main/java/org/apache/druid/server/{QuerySchedulingStrategy.java => QueryLaningStrategy.java} (73%) rename server/src/main/java/org/apache/druid/server/scheduling/{HiLoQuerySchedulingStrategy.java => HiLoQueryLaningStrategy.java} (90%) rename server/src/main/java/org/apache/druid/server/scheduling/{NoQuerySchedulingStrategy.java => NoQueryLaningStrategy.java} (81%) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index df90336ba19f..84147b8d46cd 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -106,7 +106,7 @@ import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.scheduling.NoQuerySchedulingStrategy; +import org.apache.druid.server.scheduling.NoQueryLaningStrategy; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment.PruneSpecsHolder; import org.apache.druid.timeline.SegmentId; @@ -341,7 +341,7 @@ public > QueryToolChest getToolChest new DruidHttpClientConfig(), processingConfig, forkJoinPool, - new QueryScheduler(Integer.MAX_VALUE, NoQuerySchedulingStrategy.INSTANCE) + new QueryScheduler(Integer.MAX_VALUE, NoQueryLaningStrategy.INSTANCE) ); } diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index 937917ff4553..c3b150b30fe4 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -67,7 +67,7 @@ import org.apache.druid.server.ClientQuerySegmentWalker; import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.initialization.ServerConfig; -import org.apache.druid.server.scheduling.NoQuerySchedulingStrategy; +import org.apache.druid.server.scheduling.NoQueryLaningStrategy; import org.apache.druid.testing.InitializedNullHandlingTest; import org.apache.druid.timeline.TimelineLookup; import org.hamcrest.core.IsInstanceOf; @@ -364,7 +364,7 @@ public String getFormatString() } }, ForkJoinPool.commonPool(), - new QueryScheduler(Integer.MAX_VALUE, NoQuerySchedulingStrategy.INSTANCE) + new QueryScheduler(Integer.MAX_VALUE, NoQueryLaningStrategy.INSTANCE) ); ClientQuerySegmentWalker walker = new ClientQuerySegmentWalker( diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java index 6437488b1f16..8cc21d02e19a 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/RealtimeIndexTaskTest.java @@ -909,7 +909,7 @@ private TaskToolbox makeToolbox( new QueryWatcher() { @Override - public void registerQuery(Query query, ListenableFuture future) + public void registerQueryFuture(Query query, ListenableFuture future) { // do nothing } diff --git a/processing/src/main/java/org/apache/druid/query/ChainedExecutionQueryRunner.java b/processing/src/main/java/org/apache/druid/query/ChainedExecutionQueryRunner.java index 01cd0d9f7a6d..cf149138ff28 100644 --- a/processing/src/main/java/org/apache/druid/query/ChainedExecutionQueryRunner.java +++ b/processing/src/main/java/org/apache/druid/query/ChainedExecutionQueryRunner.java @@ -144,7 +144,7 @@ public Iterable call() ) ); - queryWatcher.registerQuery(query, futures); + queryWatcher.registerQueryFuture(query, futures); try { return new MergeIterable<>( diff --git a/processing/src/main/java/org/apache/druid/query/GroupByMergedQueryRunner.java b/processing/src/main/java/org/apache/druid/query/GroupByMergedQueryRunner.java index 90e9f6f3ff58..1653fb170637 100644 --- a/processing/src/main/java/org/apache/druid/query/GroupByMergedQueryRunner.java +++ b/processing/src/main/java/org/apache/druid/query/GroupByMergedQueryRunner.java @@ -178,7 +178,7 @@ private void waitForFutureCompletion( ) { try { - queryWatcher.registerQuery(query, future); + queryWatcher.registerQueryFuture(query, future); if (QueryContexts.hasTimeout(query)) { future.get(QueryContexts.getTimeout(query), TimeUnit.MILLISECONDS); } else { diff --git a/processing/src/main/java/org/apache/druid/query/QueryWatcher.java b/processing/src/main/java/org/apache/druid/query/QueryWatcher.java index 204fa9b4e324..b10834b54241 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryWatcher.java +++ b/processing/src/main/java/org/apache/druid/query/QueryWatcher.java @@ -43,5 +43,5 @@ public interface QueryWatcher * @param query a query, which may be a subset of a larger query, as long as the underlying queryId is unchanged * @param future the future holding the execution status of the query */ - void registerQuery(Query query, ListenableFuture future); + void registerQueryFuture(Query query, ListenableFuture future); } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java index 398410ba8ca3..2de214c4c4eb 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/epinephelinae/GroupByMergingQueryRunnerV2.java @@ -346,7 +346,7 @@ private void waitForFutureCompletion( { try { if (queryWatcher != null) { - queryWatcher.registerQuery(query, future); + queryWatcher.registerQueryFuture(query, future); } if (hasTimeout && timeout <= 0) { diff --git a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java index ba5c5824637a..5e2ba0e54a1c 100644 --- a/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java +++ b/processing/src/main/java/org/apache/druid/query/metadata/SegmentMetadataQueryRunnerFactory.java @@ -219,7 +219,7 @@ public Sequence call() } ); try { - queryWatcher.registerQuery(query, future); + queryWatcher.registerQueryFuture(query, future); if (QueryContexts.hasTimeout(query)) { return future.get(QueryContexts.getTimeout(query), TimeUnit.MILLISECONDS); } else { diff --git a/processing/src/test/java/org/apache/druid/query/ChainedExecutionQueryRunnerTest.java b/processing/src/test/java/org/apache/druid/query/ChainedExecutionQueryRunnerTest.java index 275fb637f722..b2385044b560 100644 --- a/processing/src/test/java/org/apache/druid/query/ChainedExecutionQueryRunnerTest.java +++ b/processing/src/test/java/org/apache/druid/query/ChainedExecutionQueryRunnerTest.java @@ -84,7 +84,7 @@ public int getNumThreads() Capture capturedFuture = EasyMock.newCapture(); QueryWatcher watcher = EasyMock.createStrictMock(QueryWatcher.class); - watcher.registerQuery( + watcher.registerQueryFuture( EasyMock.anyObject(), EasyMock.and(EasyMock.anyObject(), EasyMock.capture(capturedFuture)) ); @@ -207,7 +207,7 @@ public int getNumThreads() Capture capturedFuture = Capture.newInstance(); QueryWatcher watcher = EasyMock.createStrictMock(QueryWatcher.class); - watcher.registerQuery( + watcher.registerQueryFuture( EasyMock.anyObject(), EasyMock.and(EasyMock.anyObject(), EasyMock.capture(capturedFuture)) ); diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java index 25877f10dfdd..53d2b97696d4 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByLimitPushDownInsufficientBufferTest.java @@ -676,7 +676,7 @@ public static > QueryRunner makeQueryRunner( public static final QueryWatcher NOOP_QUERYWATCHER = new QueryWatcher() { @Override - public void registerQuery(Query query, ListenableFuture future) + public void registerQueryFuture(Query query, ListenableFuture future) { } diff --git a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java index 35fca34e1c3c..180f1af01e81 100644 --- a/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java +++ b/processing/src/test/java/org/apache/druid/query/groupby/GroupByMultiSegmentTest.java @@ -411,7 +411,7 @@ public static > QueryRunner makeQueryRunner( public static final QueryWatcher NOOP_QUERYWATCHER = new QueryWatcher() { @Override - public void registerQuery(Query query, ListenableFuture future) + public void registerQueryFuture(Query query, ListenableFuture future) { } diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index 31d139fe6fb7..af2c09d26639 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -226,8 +226,8 @@ public Sequence run(final QueryPlus queryPlus, final ResponseContext respo */ private class SpecificQueryRunnable { - private final QueryPlus queryPlus; private final ResponseContext responseContext; + private QueryPlus queryPlus; private Query query; private final QueryToolChest> toolChest; @Nullable @@ -236,7 +236,6 @@ private class SpecificQueryRunnable private final boolean populateCache; private final boolean isBySegment; private final int uncoveredIntervalsLimit; - private final Query downstreamQuery; private final Map cachePopulatorKeyMap = new HashMap<>(); private final DataSourceAnalysis dataSourceAnalysis; private final List intervals; @@ -255,7 +254,6 @@ private class SpecificQueryRunnable // Note that enabling this leads to putting uncovered intervals information in the response headers // and might blow up in some cases https://github.com/apache/druid/issues/2108 this.uncoveredIntervalsLimit = QueryContexts.getUncoveredIntervalsLimit(query); - this.downstreamQuery = query.withOverriddenContext(makeDownstreamQueryContext()); this.dataSourceAnalysis = DataSourceAnalysis.forDataSource(query.getDataSource()); // For nested queries, we need to look at the intervals of the inner most query. this.intervals = dataSourceAnalysis.getBaseQuerySegmentSpec() @@ -269,6 +267,7 @@ private ImmutableMap makeDownstreamQueryContext() final int priority = QueryContexts.getPriority(query); contextBuilder.put(QueryContexts.PRIORITY_KEY, priority); + contextBuilder.put(QueryContexts.LANE_KEY, QueryContexts.getLane(query)); if (populateCache) { // prevent down-stream nodes from caching results as well if we are populating the cache @@ -308,20 +307,18 @@ Sequence run(final UnaryOperator> time final List> alreadyCachedResults = pruneSegmentsWithCachedResults(queryCacheKey, segmentServers); - query = scheduler.prioritizeAndLaneQuery( - queryPlus.withQuery(query), - segmentServers - ); + query = scheduler.laneQuery(queryPlus, segmentServers); + queryPlus = queryPlus.withQuery(query); final SortedMap> segmentsByServer = groupSegmentsByServer(segmentServers); - LazySequence sequence = new LazySequence<>(() -> { + LazySequence mergedResultSequence = new LazySequence<>(() -> { List> sequencesByInterval = new ArrayList<>(alreadyCachedResults.size() + segmentsByServer.size()); addSequencesFromCache(sequencesByInterval, alreadyCachedResults); addSequencesFromServer(sequencesByInterval, segmentsByServer); return merge(sequencesByInterval); }); - return scheduler.run(query, sequence); + return scheduler.run(query, mergedResultSequence); } private Sequence merge(List> sequencesByInterval) @@ -681,11 +678,12 @@ private Sequence getAndCacheServerResults( ) { @SuppressWarnings("unchecked") + final Query downstreamQuery = query.withOverriddenContext(makeDownstreamQueryContext()); final Sequence>> resultsBySegments = serverRunner.run( queryPlus .withQuery( Queries.withSpecificSegments( - (Query>>) downstreamQuery, + downstreamQuery, segmentsOfServer ) ) diff --git a/server/src/main/java/org/apache/druid/client/DirectDruidClient.java b/server/src/main/java/org/apache/druid/client/DirectDruidClient.java index 06976c1a5b41..020446d69f84 100644 --- a/server/src/main/java/org/apache/druid/client/DirectDruidClient.java +++ b/server/src/main/java/org/apache/druid/client/DirectDruidClient.java @@ -451,7 +451,7 @@ private void checkTotalBytesLimit(long bytes) Duration.millis(timeLeft) ); - queryWatcher.registerQuery(query, future); + queryWatcher.registerQueryFuture(query, future); openConnections.getAndIncrement(); Futures.addCallback( diff --git a/server/src/main/java/org/apache/druid/client/SegmentServer.java b/server/src/main/java/org/apache/druid/client/SegmentServer.java index 8499fbcf81a8..4063a9715c05 100644 --- a/server/src/main/java/org/apache/druid/client/SegmentServer.java +++ b/server/src/main/java/org/apache/druid/client/SegmentServer.java @@ -25,7 +25,7 @@ public class SegmentServer extends Pair { - SegmentServer(ServerSelector server, SegmentDescriptor segment) + public SegmentServer(ServerSelector server, SegmentDescriptor segment) { super(server, segment); } diff --git a/server/src/main/java/org/apache/druid/server/QuerySchedulingStrategy.java b/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java similarity index 73% rename from server/src/main/java/org/apache/druid/server/QuerySchedulingStrategy.java rename to server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java index 435dfa17cc08..43682e0f0713 100644 --- a/server/src/main/java/org/apache/druid/server/QuerySchedulingStrategy.java +++ b/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java @@ -25,20 +25,20 @@ import org.apache.druid.client.SegmentServer; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; -import org.apache.druid.server.scheduling.HiLoQuerySchedulingStrategy; -import org.apache.druid.server.scheduling.NoQuerySchedulingStrategy; +import org.apache.druid.server.scheduling.HiLoQueryLaningStrategy; +import org.apache.druid.server.scheduling.NoQueryLaningStrategy; import java.util.Set; -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = NoQuerySchedulingStrategy.class) +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = NoQueryLaningStrategy.class) @JsonSubTypes(value = { - @JsonSubTypes.Type(name = "none", value = NoQuerySchedulingStrategy.class), - @JsonSubTypes.Type(name = "hilo", value = HiLoQuerySchedulingStrategy.class) + @JsonSubTypes.Type(name = "none", value = NoQueryLaningStrategy.class), + @JsonSubTypes.Type(name = "hilo", value = HiLoQueryLaningStrategy.class) }) -public interface QuerySchedulingStrategy +public interface QueryLaningStrategy { Object2IntMap getLaneLimits(); - Query prioritizeAndLaneQuery(QueryPlus query, Set segments); + Query laneQuery(QueryPlus query, Set segments); } diff --git a/server/src/main/java/org/apache/druid/server/QueryScheduler.java b/server/src/main/java/org/apache/druid/server/QueryScheduler.java index 57a10a8f71d0..9744ae372315 100644 --- a/server/src/main/java/org/apache/druid/server/QueryScheduler.java +++ b/server/src/main/java/org/apache/druid/server/QueryScheduler.java @@ -19,6 +19,7 @@ package org.apache.druid.server; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.HashMultimap; import com.google.common.collect.Multimaps; import com.google.common.collect.SetMultimap; @@ -42,21 +43,15 @@ /** - * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' and potentially - * priority using the - * {@link QuerySchedulingStrategy} that is defined in {@link QuerySchedulerConfig} - * The purpose of the QueryScheduler is to give overall visibility into queries running - * or pending at the QueryRunner level. This is currently used to cancel all the - * parts of a pending query, but may be expanded in the future to offer more direct - * visibility into query execution and resource usage. + * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the + * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}. * - * QueryRunners executing any computation asynchronously must register their queries with the QueryScheduler. + * As a {@link QueryWatcher}, it also provides cancellation facilities. * */ public class QueryScheduler implements QueryWatcher { - private final QuerySchedulingStrategy strategy; - // maybe instead use a fancy library? + private final QueryLaningStrategy laningStrategy; private final Optional totalQueryLimit; private final Map laneLimits; @@ -64,9 +59,9 @@ public class QueryScheduler implements QueryWatcher private final SetMultimap queryDatasources; private final Set> runningQueries; - public QueryScheduler(int totalNumThreads, QuerySchedulingStrategy strategy) + public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy) { - this.strategy = strategy; + this.laningStrategy = laningStrategy; if (totalNumThreads > 0) { this.totalQueryLimit = Optional.of(new Semaphore(totalNumThreads)); } else { @@ -74,7 +69,7 @@ public QueryScheduler(int totalNumThreads, QuerySchedulingStrategy strategy) } this.laneLimits = new HashMap<>(); - for (Object2IntMap.Entry entry : strategy.getLaneLimits().object2IntEntrySet()) { + for (Object2IntMap.Entry entry : laningStrategy.getLaneLimits().object2IntEntrySet()) { laneLimits.put(entry.getKey(), new Semaphore(entry.getIntValue())); } this.queries = Multimaps.synchronizedSetMultimap( @@ -86,17 +81,88 @@ public QueryScheduler(int totalNumThreads, QuerySchedulingStrategy strategy) this.runningQueries = Sets.newConcurrentHashSet(); } - public Query prioritizeAndLaneQuery(QueryPlus query, Set segments) + @Override + public void registerQueryFuture(Query query, ListenableFuture future) { - return strategy.prioritizeAndLaneQuery(query, segments); + final String id = query.getId(); + final Set datasources = query.getDataSource().getTableNames(); + queries.put(id, future); + queryDatasources.putAll(id, datasources); + future.addListener( + () -> { + queries.remove(id, future); + for (String datasource : datasources) { + queryDatasources.remove(id, datasource); + } + }, + Execs.directExecutor() + ); } - public void scheduleQuery(Query query) + /** + * Assign a query a lane (if not set) + */ + public Query laneQuery(QueryPlus query, Set segments) + { + if (QueryContexts.getLane(query.getQuery()) != null) { + return query.getQuery(); + } + return laningStrategy.laneQuery(query, segments); + } + + /** + * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities + */ + public Sequence run(Query query, Sequence resultSequence) + { + scheduleQuery(query); + return resultSequence.withBaggage(() -> completeQuery(query)); + } + + /** + * Forcibly cancel all futures that have been registered to a specific query id + */ + public boolean cancelQuery(String id) + { + // if you re-use queryId and cancel queries... you are going to have a bad time + queryDatasources.removeAll(id); + Set> futures = queries.removeAll(id); + boolean success = true; + for (ListenableFuture future : futures) { + success = success && future.cancel(true); + } + return success; + } + + public Set getQueryDatasources(final String queryId) + { + return queryDatasources.get(queryId); + } + + public int getTotalAvailableCapacity() + { + return totalQueryLimit.map(Semaphore::availablePermits).orElse(-1); + } + + public int getLaneAvailableCapacity(String lane) + { + if (laneLimits.containsKey(lane)) { + return laneLimits.get(lane).availablePermits(); + } + return -1; + } + + /** + * Acquire semaphore from total capacity and lane capacity (if query is assigned a lane that exists) + */ + @VisibleForTesting + void scheduleQuery(Query query) { final String lane; try { if (!totalQueryLimit.isPresent() || totalQueryLimit.get().tryAcquire(0, TimeUnit.MILLISECONDS)) { lane = QueryContexts.getLane(query); + // if no lane, we are done if (!laneLimits.containsKey(lane)) { runningQueries.add(query); return; @@ -108,10 +174,10 @@ public void scheduleQuery(Query query) catch (InterruptedException ex) { throw new QueryCapacityExceededException(); } + // if we got here, the query belongs to a lane, acquire the semaphore for it try { if (laneLimits.get(lane).tryAcquire(0, TimeUnit.MILLISECONDS)) { runningQueries.add(query); - return; } else { throw new QueryCapacityExceededException(lane); } @@ -121,7 +187,10 @@ public void scheduleQuery(Query query) } } - public void completeQuery(Query query) + /** + * Release semaphores help by query + */ + private void completeQuery(Query query) { if (runningQueries.remove(query)) { String lane = QueryContexts.getLane(query); @@ -131,73 +200,4 @@ public void completeQuery(Query query) totalQueryLimit.ifPresent(Semaphore::release); } } - - public Sequence run(Query query, Sequence resultSequence) - { - scheduleQuery(query); - return resultSequence.withBaggage(() -> completeQuery(query)); - } - - public int getTotalAvailableCapacity() - { - return totalQueryLimit.map(Semaphore::availablePermits).orElse(-1); - } - - public int getLaneAvailableCapacity(String lane) - { - if (laneLimits.containsKey(lane)) { - return laneLimits.get(lane).availablePermits(); - } - return -1; - } - - - public boolean cancelQuery(String id) - { - queryDatasources.removeAll(id); - Set> futures = queries.removeAll(id); - boolean success = true; - for (ListenableFuture future : futures) { - success = success && future.cancel(true); - } - return success; - } - - /** - * QueryRunners must use this method to register any pending queries. - * - * The given future may have cancel(true) called at any time, if cancellation of this query has been requested. - * - * @param query a query, which may be a subset of a larger query, as long as the underlying queryId is unchanged - * @param future the future holding the execution status of the query - */ - public void registerQueryFuture(Query query, final ListenableFuture future) - { - final String id = query.getId(); - final Set datasources = query.getDataSource().getTableNames(); - queries.put(id, future); - queryDatasources.putAll(id, datasources); - future.addListener( - () -> { - // if you re-use queryId and cancel queries... you are going to have a bad time - queries.remove(id, future); - - for (String datasource : datasources) { - queryDatasources.remove(id, datasource); - } - }, - Execs.directExecutor() - ); - } - - public Set getQueryDatasources(final String queryId) - { - return queryDatasources.get(queryId); - } - - @Override - public void registerQuery(Query query, ListenableFuture future) - { - registerQueryFuture(query, future); - } } diff --git a/server/src/main/java/org/apache/druid/server/QuerySchedulerConfig.java b/server/src/main/java/org/apache/druid/server/QuerySchedulerConfig.java index 5052b4b7ed92..c6fcb5888ae6 100644 --- a/server/src/main/java/org/apache/druid/server/QuerySchedulerConfig.java +++ b/server/src/main/java/org/apache/druid/server/QuerySchedulerConfig.java @@ -26,9 +26,8 @@ public class QuerySchedulerConfig @JsonProperty private Integer numThreads; - @JsonProperty - private QuerySchedulingStrategy strategy; - + @JsonProperty("laning") + private QueryLaningStrategy laningStrategy; @JsonProperty public int getNumThreads() @@ -36,9 +35,9 @@ public int getNumThreads() return numThreads; } - @JsonProperty - public QuerySchedulingStrategy getStrategy() + @JsonProperty("laning") + public QueryLaningStrategy getLaningStrategy() { - return strategy; + return laningStrategy; } } diff --git a/server/src/main/java/org/apache/druid/server/QuerySchedulerProvider.java b/server/src/main/java/org/apache/druid/server/QuerySchedulerProvider.java index 3f46b9d1ef1d..bc474a49d532 100644 --- a/server/src/main/java/org/apache/druid/server/QuerySchedulerProvider.java +++ b/server/src/main/java/org/apache/druid/server/QuerySchedulerProvider.java @@ -27,6 +27,6 @@ public class QuerySchedulerProvider extends QuerySchedulerConfig implements Prov @Override public QueryScheduler get() { - return new QueryScheduler(getNumThreads(), getStrategy()); + return new QueryScheduler(getNumThreads(), getLaningStrategy()); } } diff --git a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQuerySchedulingStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java similarity index 90% rename from server/src/main/java/org/apache/druid/server/scheduling/HiLoQuerySchedulingStrategy.java rename to server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java index 58615fe1f41f..ed9617f4bade 100644 --- a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQuerySchedulingStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java @@ -28,11 +28,11 @@ import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryPlus; -import org.apache.druid.server.QuerySchedulingStrategy; +import org.apache.druid.server.QueryLaningStrategy; import java.util.Set; -public class HiLoQuerySchedulingStrategy implements QuerySchedulingStrategy +public class HiLoQueryLaningStrategy implements QueryLaningStrategy { public static String LOW = "low"; @@ -40,7 +40,7 @@ public class HiLoQuerySchedulingStrategy implements QuerySchedulingStrategy private int maxLowThreads; @JsonCreator - public HiLoQuerySchedulingStrategy( + public HiLoQueryLaningStrategy( @JsonProperty("maxLowThreads") Integer maxLowThreads ) { @@ -56,7 +56,7 @@ public Object2IntMap getLaneLimits() } @Override - public Query prioritizeAndLaneQuery(QueryPlus query, Set segments) + public Query laneQuery(QueryPlus query, Set segments) { final Query theQuery = query.getQuery(); // QueryContexts.getPriority gives a default, since we are setting priority diff --git a/server/src/main/java/org/apache/druid/server/scheduling/NoQuerySchedulingStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java similarity index 81% rename from server/src/main/java/org/apache/druid/server/scheduling/NoQuerySchedulingStrategy.java rename to server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java index 18ab8d24e57b..2df58c002173 100644 --- a/server/src/main/java/org/apache/druid/server/scheduling/NoQuerySchedulingStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java @@ -24,15 +24,15 @@ import org.apache.druid.client.SegmentServer; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; -import org.apache.druid.server.QuerySchedulingStrategy; +import org.apache.druid.server.QueryLaningStrategy; import java.util.Set; -public class NoQuerySchedulingStrategy implements QuerySchedulingStrategy +public class NoQueryLaningStrategy implements QueryLaningStrategy { private static final Object2IntMap NONE = new Object2IntArrayMap<>(); - public static NoQuerySchedulingStrategy INSTANCE = new NoQuerySchedulingStrategy(); + public static NoQueryLaningStrategy INSTANCE = new NoQueryLaningStrategy(); @Override public Object2IntMap getLaneLimits() @@ -41,7 +41,7 @@ public Object2IntMap getLaneLimits() } @Override - public Query prioritizeAndLaneQuery(QueryPlus query, Set segments) + public Query laneQuery(QueryPlus query, Set segments) { return query.getQuery(); } diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java index 337cc4903fcd..b3540fd56814 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java @@ -49,7 +49,7 @@ import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.scheduling.NoQuerySchedulingStrategy; +import org.apache.druid.server.scheduling.NoQueryLaningStrategy; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.TimelineLookup; import org.apache.druid.timeline.VersionedIntervalTimeline; @@ -334,7 +334,7 @@ public int getMergePoolParallelism() }, ForkJoinPool.commonPool(), // need at least 9 total since runner doesn't actually run queries and release semaphores - new QueryScheduler(Integer.MAX_VALUE, NoQuerySchedulingStrategy.INSTANCE) + new QueryScheduler(Integer.MAX_VALUE, NoQueryLaningStrategy.INSTANCE) ); } diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index e63ed66e7e44..83d7a415ec80 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -118,7 +118,7 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.coordination.ServerType; -import org.apache.druid.server.scheduling.NoQuerySchedulingStrategy; +import org.apache.druid.server.scheduling.NoQueryLaningStrategy; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; import org.apache.druid.timeline.VersionedIntervalTimeline; @@ -2476,7 +2476,7 @@ public int getMergePoolParallelism() } }, ForkJoinPool.commonPool(), - new QueryScheduler(Integer.MAX_VALUE, NoQuerySchedulingStrategy.INSTANCE) + new QueryScheduler(Integer.MAX_VALUE, NoQueryLaningStrategy.INSTANCE) ); } diff --git a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java index 23566bc89923..60c840bb072f 100644 --- a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java @@ -49,8 +49,8 @@ import org.apache.druid.query.timeboundary.TimeBoundaryResultValue; import org.apache.druid.server.log.TestRequestLogger; import org.apache.druid.server.metrics.NoopServiceEmitter; -import org.apache.druid.server.scheduling.HiLoQuerySchedulingStrategy; -import org.apache.druid.server.scheduling.NoQuerySchedulingStrategy; +import org.apache.druid.server.scheduling.HiLoQueryLaningStrategy; +import org.apache.druid.server.scheduling.NoQueryLaningStrategy; import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Action; import org.apache.druid.server.security.AuthConfig; @@ -135,7 +135,7 @@ public void setup() EasyMock.expect(testServletRequest.getHeader("Accept")).andReturn(MediaType.APPLICATION_JSON).anyTimes(); EasyMock.expect(testServletRequest.getHeader(QueryResource.HEADER_IF_NONE_MATCH)).andReturn(null).anyTimes(); EasyMock.expect(testServletRequest.getRemoteAddr()).andReturn("localhost").anyTimes(); - queryScheduler = new QueryScheduler(8, NoQuerySchedulingStrategy.INSTANCE); + queryScheduler = new QueryScheduler(8, NoQueryLaningStrategy.INSTANCE); testRequestLogger = new TestRequestLogger(); queryResource = new QueryResource( new QueryLifecycleFactory( @@ -679,7 +679,7 @@ public void testTooManyQuery() throws InterruptedException final CountDownLatch waitFirstStart = new CountDownLatch(1); final CountDownLatch waitSecondStart = new CountDownLatch(2); final CountDownLatch waitFinishLatch = new CountDownLatch(3); - final QueryScheduler laningScheduler = new QueryScheduler(2, NoQuerySchedulingStrategy.INSTANCE); + final QueryScheduler laningScheduler = new QueryScheduler(2, NoQueryLaningStrategy.INSTANCE); createScheduledQueryResource(laningScheduler, ImmutableList.of(waitFirstStart, waitSecondStart)); assertResponseAndCountdownOrBlockForever( @@ -712,7 +712,7 @@ public void testTooManyQueryInLane() throws InterruptedException final CountDownLatch waitFirstStart = new CountDownLatch(1); final CountDownLatch waitSecondStart = new CountDownLatch(2); final CountDownLatch waitFinishLatch = new CountDownLatch(3); - final QueryScheduler scheduler = new QueryScheduler(40, new HiLoQuerySchedulingStrategy(1)); + final QueryScheduler scheduler = new QueryScheduler(40, new HiLoQueryLaningStrategy(1)); createScheduledQueryResource(scheduler, ImmutableList.of(waitFirstStart, waitSecondStart)); @@ -730,7 +730,7 @@ public void testTooManyQueryInLane() throws InterruptedException Assert.assertEquals( StringUtils.format( QueryCapacityExceededException.ERROR_MESSAGE_TEMPLATE, - HiLoQuerySchedulingStrategy.LOW + HiLoQueryLaningStrategy.LOW ), response.getEntity()); } @@ -772,7 +772,7 @@ public QueryRunner getQueryRunnerForIntervals(Query query, Iterable(() -> { try { Thread.sleep(500); diff --git a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java index 816dd45ddf0c..e043b75a3517 100644 --- a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java +++ b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java @@ -37,7 +37,7 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.topn.TopNQuery; import org.apache.druid.query.topn.TopNQueryBuilder; -import org.apache.druid.server.scheduling.HiLoQuerySchedulingStrategy; +import org.apache.druid.server.scheduling.HiLoQueryLaningStrategy; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -56,14 +56,14 @@ public class QuerySchedulerTest @Test public void testHiLoHi() throws ExecutionException, InterruptedException { - QueryScheduler scheduler = new QueryScheduler(5, new HiLoQuerySchedulingStrategy(2)); + QueryScheduler scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)); TopNQuery interactive = makeInteractiveQuery(); ListenableFuture future = MoreExecutors.listeningDecorator( Execs.singleThreaded("test_query_scheduler_%s") ).submit(() -> { try { - Query scheduled = scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(interactive), ImmutableSet.of()); + Query scheduled = scheduler.laneQuery(QueryPlus.wrap(interactive), ImmutableSet.of()); Assert.assertNotNull(scheduled); @@ -74,7 +74,7 @@ public void testHiLoHi() throws ExecutionException, InterruptedException public void before() { Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); - Assert.assertEquals(2, scheduler.getLaneAvailableCapacity(HiLoQuerySchedulingStrategy.LOW)); + Assert.assertEquals(2, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); } }); Sequence results = scheduler.run(scheduled, underlyingSequence); @@ -94,15 +94,15 @@ public void before() @Test public void testHiLoLo() throws ExecutionException, InterruptedException { - QueryScheduler scheduler = new QueryScheduler(5, new HiLoQuerySchedulingStrategy(2)); + QueryScheduler scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)); TopNQuery report = makeReportQuery(); ListenableFuture future = MoreExecutors.listeningDecorator( Execs.singleThreaded("test_query_scheduler_%s") ).submit(() -> { try { - Query scheduledReport = scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(report), ImmutableSet.of()); + Query scheduledReport = scheduler.laneQuery(QueryPlus.wrap(report), ImmutableSet.of()); Assert.assertNotNull(scheduledReport); - Assert.assertEquals(HiLoQuerySchedulingStrategy.LOW, QueryContexts.getLane(scheduledReport)); + Assert.assertEquals(HiLoQueryLaningStrategy.LOW, QueryContexts.getLane(scheduledReport)); Sequence underlyingSequence = makeSequence(10); underlyingSequence = Sequences.wrap(underlyingSequence, new SequenceWrapper() @@ -111,7 +111,7 @@ public void testHiLoLo() throws ExecutionException, InterruptedException public void before() { Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); - Assert.assertEquals(1, scheduler.getLaneAvailableCapacity(HiLoQuerySchedulingStrategy.LOW)); + Assert.assertEquals(1, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); } }); Sequence results = scheduler.run(scheduledReport, underlyingSequence); @@ -126,7 +126,7 @@ public void before() scheduler.registerQueryFuture(report, future); future.get(); Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); - Assert.assertEquals(2, scheduler.getLaneAvailableCapacity(HiLoQuerySchedulingStrategy.LOW)); + Assert.assertEquals(2, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); } @@ -136,14 +136,14 @@ public void testHiLoReleaseSemaphoreWhenSequenceExplodes() throws Exception expected.expectMessage("exploded"); expected.expect(ExecutionException.class); - QueryScheduler scheduler = new QueryScheduler(5, new HiLoQuerySchedulingStrategy(2)); + QueryScheduler scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)); TopNQuery interactive = makeInteractiveQuery(); ListenableFuture future = MoreExecutors.listeningDecorator( Execs.singleThreaded("test_query_scheduler_%s") ).submit(() -> { try { - Query scheduled = scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(interactive), ImmutableSet.of()); + Query scheduled = scheduler.laneQuery(QueryPlus.wrap(interactive), ImmutableSet.of()); Assert.assertNotNull(scheduled); @@ -173,26 +173,26 @@ public void before() public void testHiLoFailsWhenOutOfLaneCapacity() { expected.expectMessage( - StringUtils.format(QueryCapacityExceededException.ERROR_MESSAGE_TEMPLATE, HiLoQuerySchedulingStrategy.LOW) + StringUtils.format(QueryCapacityExceededException.ERROR_MESSAGE_TEMPLATE, HiLoQueryLaningStrategy.LOW) ); expected.expect(QueryCapacityExceededException.class); - QueryScheduler scheduler = new QueryScheduler(5, new HiLoQuerySchedulingStrategy(2)); + QueryScheduler scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)); - Query report1 = scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); + Query report1 = scheduler.laneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); scheduler.scheduleQuery(report1); Assert.assertNotNull(report1); Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); - Assert.assertEquals(1, scheduler.getLaneAvailableCapacity(HiLoQuerySchedulingStrategy.LOW)); + Assert.assertEquals(1, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); - Query report2 = scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); + Query report2 = scheduler.laneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); scheduler.scheduleQuery(report2); Assert.assertNotNull(report2); Assert.assertEquals(3, scheduler.getTotalAvailableCapacity()); - Assert.assertEquals(0, scheduler.getLaneAvailableCapacity(HiLoQuerySchedulingStrategy.LOW)); + Assert.assertEquals(0, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); // too many reports - scheduler.scheduleQuery(scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of())); + scheduler.scheduleQuery(scheduler.laneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of())); } @Test @@ -201,37 +201,37 @@ public void testHiLoFailsWhenOutOfTotalCapacity() expected.expectMessage(QueryCapacityExceededException.ERROR_MESSAGE); expected.expect(QueryCapacityExceededException.class); - QueryScheduler scheduler = new QueryScheduler(5, new HiLoQuerySchedulingStrategy(2)); - Query interactive1 = scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()); + QueryScheduler scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)); + Query interactive1 = scheduler.laneQuery(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()); scheduler.scheduleQuery(interactive1); Assert.assertNotNull(interactive1); Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); - Query report1 = scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); + Query report1 = scheduler.laneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); scheduler.scheduleQuery(report1); Assert.assertNotNull(report1); Assert.assertEquals(3, scheduler.getTotalAvailableCapacity()); - Assert.assertEquals(1, scheduler.getLaneAvailableCapacity(HiLoQuerySchedulingStrategy.LOW)); + Assert.assertEquals(1, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); - Query interactive2 = scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()); + Query interactive2 = scheduler.laneQuery(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()); scheduler.scheduleQuery(interactive2); Assert.assertNotNull(interactive2); Assert.assertEquals(2, scheduler.getTotalAvailableCapacity()); - Query report2 = scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); + Query report2 = scheduler.laneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); scheduler.scheduleQuery(report2); Assert.assertNotNull(report2); Assert.assertEquals(1, scheduler.getTotalAvailableCapacity()); - Assert.assertEquals(0, scheduler.getLaneAvailableCapacity(HiLoQuerySchedulingStrategy.LOW)); + Assert.assertEquals(0, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); - Query interactive3 = scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()); + Query interactive3 = scheduler.laneQuery(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()); scheduler.scheduleQuery(interactive3); Assert.assertNotNull(interactive3); Assert.assertEquals(0, scheduler.getTotalAvailableCapacity()); // one too many scheduler.scheduleQuery( - scheduler.prioritizeAndLaneQuery(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()) + scheduler.laneQuery(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()) ); } From 0597c3f87117a8f35785a3cea7ba8f9b4e7e8511 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 26 Feb 2020 04:15:34 -0800 Subject: [PATCH 05/37] docs --- docs/configuration/index.md | 24 ++++++++++++++- .../druid/client/CachingClusteredClient.java | 30 +++++++++---------- ...Server.java => SegmentServerSelector.java} | 10 +++++-- .../druid/server/QueryLaningStrategy.java | 4 +-- .../apache/druid/server/QueryScheduler.java | 4 +-- .../druid/server/QuerySchedulerConfig.java | 5 ++-- .../scheduling/HiLoQueryLaningStrategy.java | 7 +++-- .../scheduling/NoQueryLaningStrategy.java | 4 +-- 8 files changed, 59 insertions(+), 29 deletions(-) rename server/src/main/java/org/apache/druid/client/{SegmentServer.java => SegmentServerSelector.java} (73%) diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 69bd0abc83c7..ee9d17ea6d72 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1476,9 +1476,31 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi |`druid.broker.select.tier`|`highestPriority`, `lowestPriority`, `custom`|If segments are cross-replicated across tiers in a cluster, you can tell the broker to prefer to select segments in a tier with a certain priority.|`highestPriority`| |`druid.broker.select.tier.custom.priorities`|`An array of integer priorities.`|Select servers in tiers with a custom priority list.|None| +##### Query laning + +Druid provides facilities to aid in query capacity reservation for heterogenous query workloads in the form of 'laning' strategies, which provide a variety of mechanisms examine and classify a query at the broker, assigning it to a 'lane'. Lanes are defined with capacity limits which the broker will enforce, causing requests in excess of the capacity to be discarded with an HTTP 429 status code, reserving resources for other lanes or for interactive queries (with no lane). + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.query.scheduler.numThreads`|Maximum number of HTTP threads to dedicate to query processing. To save HTTP thread capacity, this should be lower than `druid.server.http.numThreads`.|Unbounded| +|`druid.query.scheduler.laning.type`|Query laning strategy to use to assign queries to a lane in order to control capacities for certain classes of queries.|`none`| + +###### No laning strategy + +In this mode, queries are never assigned a lane and only limited by `druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, if set. This is the default Druid query scheduler operating mode. + +###### 'High/Low' laning strategy +This laning strategy splits queries with a `priority` below zero into a `low` query lane, automatically. The limit on `low` queries can be set to some desired fraction of the total capacity (or HTTP thread pool size), reserving capacity for interactive queries. Queries in the `low` lane are _not_ guaranteed their capacity, which may be consumed by interactive queries, but may use up to this limit if total capacity is available. + +This strategy can be enabled by setting `druid.query.scheduler.laning.type` to `hilo`. + +|Property|Description|Default| +|--------|-----------|-------| +|`druid.query.scheduler.laning.maxLowThreads`|Maximum number of HTTP threads that can be used by queries with a priority lower than 0.|No default, must be set if using this mode| + ##### Server Configuration -Druid uses Jetty to serve HTTP requests. +Druid uses Jetty to serve HTTP requests. Each query being processed consumes a single thread from `druid.server.http.numThreads`, so consider defining `druid.query.scheduler.numThreads` to a lower value in order to reserve HTTP threads for responding to health checks, lookup loading, and other non-query, and in most cases comparatively very short lived, HTTP requests. |Property|Description|Default| |--------|-----------|-------| diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index af2c09d26639..54f8e9853970 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -291,7 +291,7 @@ Sequence run(final UnaryOperator> time computeUncoveredIntervals(timeline); } - final Set segmentServers = computeSegmentsToQuery(timeline); + final Set segmentServers = computeSegmentsToQuery(timeline); @Nullable final byte[] queryCacheKey = computeQueryCacheKey(); if (query.getContext().get(QueryResource.HEADER_IF_NONE_MATCH) != null) { @@ -357,14 +357,14 @@ private Sequence merge(List> sequencesByInterval) } } - private Set computeSegmentsToQuery(TimelineLookup timeline) + private Set computeSegmentsToQuery(TimelineLookup timeline) { final List> serversLookup = toolChest.filterSegments( query, intervals.stream().flatMap(i -> timeline.lookup(i).stream()).collect(Collectors.toList()) ); - final Set segments = new LinkedHashSet<>(); + final Set segments = new LinkedHashSet<>(); final Map>> dimensionRangeCache = new HashMap<>(); // Filter unneeded chunks based on partition dimension for (TimelineObjectHolder holder : serversLookup) { @@ -381,7 +381,7 @@ private Set computeSegmentsToQuery(TimelineLookup segments, @Nullable byte[] queryCacheKey) + private String computeCurrentEtag(final Set segments, @Nullable byte[] queryCacheKey) { Hasher hasher = Hashing.sha1().newHasher(); boolean hasOnlyHistoricalSegments = true; - for (SegmentServer p : segments) { + for (SegmentServerSelector p : segments) { if (!p.getServer().pick().getServer().segmentReplicatable()) { hasOnlyHistoricalSegments = false; break; @@ -470,14 +470,14 @@ private String computeCurrentEtag(final Set segments, @Nullable b private List> pruneSegmentsWithCachedResults( final byte[] queryCacheKey, - final Set segments + final Set segments ) { if (queryCacheKey == null) { return Collections.emptyList(); } final List> alreadyCachedResults = new ArrayList<>(); - Map perSegmentCacheKeys = computePerSegmentCacheKeys(segments, queryCacheKey); + Map perSegmentCacheKeys = computePerSegmentCacheKeys(segments, queryCacheKey); // Pull cached segments from cache and remove from set of segments to query final Map cachedValues = computeCachedValues(perSegmentCacheKeys); @@ -498,14 +498,14 @@ private List> pruneSegmentsWithCachedResults( return alreadyCachedResults; } - private Map computePerSegmentCacheKeys( - Set segments, + private Map computePerSegmentCacheKeys( + Set segments, byte[] queryCacheKey ) { // cacheKeys map must preserve segment ordering, in order for shards to always be combined in the same order - Map cacheKeys = Maps.newLinkedHashMap(); - for (SegmentServer segmentServer : segments) { + Map cacheKeys = Maps.newLinkedHashMap(); + for (SegmentServerSelector segmentServer : segments) { final Cache.NamedKey segmentCacheKey = CacheUtil.computeSegmentCacheKey( segmentServer.getServer().getSegment().getId().toString(), segmentServer.getSegmentDescriptor(), @@ -516,7 +516,7 @@ private Map computePerSegmentCacheKeys( return cacheKeys; } - private Map computeCachedValues(Map cacheKeys) + private Map computeCachedValues(Map cacheKeys) { if (useCache) { return cache.getBulk(Iterables.limit(cacheKeys.values(), cacheConfig.getCacheBulkMergeLimit())); @@ -540,10 +540,10 @@ private Cache.NamedKey getCachePopulatorKey(String segmentId, Interval segmentIn return cachePopulatorKeyMap.get(StringUtils.format("%s_%s", segmentId, segmentInterval)); } - private SortedMap> groupSegmentsByServer(Set segments) + private SortedMap> groupSegmentsByServer(Set segments) { final SortedMap> serverSegments = new TreeMap<>(); - for (SegmentServer segmentServer : segments) { + for (SegmentServerSelector segmentServer : segments) { final QueryableDruidServer queryableDruidServer = segmentServer.getServer().pick(); if (queryableDruidServer == null) { diff --git a/server/src/main/java/org/apache/druid/client/SegmentServer.java b/server/src/main/java/org/apache/druid/client/SegmentServerSelector.java similarity index 73% rename from server/src/main/java/org/apache/druid/client/SegmentServer.java rename to server/src/main/java/org/apache/druid/client/SegmentServerSelector.java index 4063a9715c05..c5dd11def680 100644 --- a/server/src/main/java/org/apache/druid/client/SegmentServer.java +++ b/server/src/main/java/org/apache/druid/client/SegmentServerSelector.java @@ -23,9 +23,15 @@ import org.apache.druid.java.util.common.Pair; import org.apache.druid.query.SegmentDescriptor; -public class SegmentServer extends Pair +/** + * Given a {@link SegmentDescriptor}, get a {@link ServerSelector} to use to pick a {@link DruidServer} to query the + * segment. + * + * Used by {@link CachingClusteredClient} on the broker to fan out queries to historical and realtime servers + */ +public class SegmentServerSelector extends Pair { - public SegmentServer(ServerSelector server, SegmentDescriptor segment) + public SegmentServerSelector(ServerSelector server, SegmentDescriptor segment) { super(server, segment); } diff --git a/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java index 43682e0f0713..ad8e2672169a 100644 --- a/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java @@ -22,7 +22,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; import it.unimi.dsi.fastutil.objects.Object2IntMap; -import org.apache.druid.client.SegmentServer; +import org.apache.druid.client.SegmentServerSelector; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.server.scheduling.HiLoQueryLaningStrategy; @@ -40,5 +40,5 @@ public interface QueryLaningStrategy { Object2IntMap getLaneLimits(); - Query laneQuery(QueryPlus query, Set segments); + Query laneQuery(QueryPlus query, Set segments); } diff --git a/server/src/main/java/org/apache/druid/server/QueryScheduler.java b/server/src/main/java/org/apache/druid/server/QueryScheduler.java index 9744ae372315..66c5307872c2 100644 --- a/server/src/main/java/org/apache/druid/server/QueryScheduler.java +++ b/server/src/main/java/org/apache/druid/server/QueryScheduler.java @@ -26,7 +26,7 @@ import com.google.common.collect.Sets; import com.google.common.util.concurrent.ListenableFuture; import it.unimi.dsi.fastutil.objects.Object2IntMap; -import org.apache.druid.client.SegmentServer; +import org.apache.druid.client.SegmentServerSelector; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.query.Query; @@ -102,7 +102,7 @@ public void registerQueryFuture(Query query, ListenableFuture future) /** * Assign a query a lane (if not set) */ - public Query laneQuery(QueryPlus query, Set segments) + public Query laneQuery(QueryPlus query, Set segments) { if (QueryContexts.getLane(query.getQuery()) != null) { return query.getQuery(); diff --git a/server/src/main/java/org/apache/druid/server/QuerySchedulerConfig.java b/server/src/main/java/org/apache/druid/server/QuerySchedulerConfig.java index c6fcb5888ae6..a787da88ea6b 100644 --- a/server/src/main/java/org/apache/druid/server/QuerySchedulerConfig.java +++ b/server/src/main/java/org/apache/druid/server/QuerySchedulerConfig.java @@ -20,14 +20,15 @@ package org.apache.druid.server; import com.fasterxml.jackson.annotation.JsonProperty; +import org.apache.druid.server.scheduling.NoQueryLaningStrategy; public class QuerySchedulerConfig { @JsonProperty - private Integer numThreads; + private Integer numThreads = 0; @JsonProperty("laning") - private QueryLaningStrategy laningStrategy; + private QueryLaningStrategy laningStrategy = NoQueryLaningStrategy.INSTANCE; @JsonProperty public int getNumThreads() diff --git a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java index ed9617f4bade..735f1c05e1f0 100644 --- a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java @@ -21,10 +21,11 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; +import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableMap; import it.unimi.dsi.fastutil.objects.Object2IntArrayMap; import it.unimi.dsi.fastutil.objects.Object2IntMap; -import org.apache.druid.client.SegmentServer; +import org.apache.druid.client.SegmentServerSelector; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryPlus; @@ -44,7 +45,7 @@ public HiLoQueryLaningStrategy( @JsonProperty("maxLowThreads") Integer maxLowThreads ) { - this.maxLowThreads = maxLowThreads; + this.maxLowThreads = Preconditions.checkNotNull(maxLowThreads, "maxLowThreads must be set"); } @Override @@ -56,7 +57,7 @@ public Object2IntMap getLaneLimits() } @Override - public Query laneQuery(QueryPlus query, Set segments) + public Query laneQuery(QueryPlus query, Set segments) { final Query theQuery = query.getQuery(); // QueryContexts.getPriority gives a default, since we are setting priority diff --git a/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java index 2df58c002173..ecda3a0d1373 100644 --- a/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java @@ -21,7 +21,7 @@ import it.unimi.dsi.fastutil.objects.Object2IntArrayMap; import it.unimi.dsi.fastutil.objects.Object2IntMap; -import org.apache.druid.client.SegmentServer; +import org.apache.druid.client.SegmentServerSelector; import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.server.QueryLaningStrategy; @@ -41,7 +41,7 @@ public Object2IntMap getLaneLimits() } @Override - public Query laneQuery(QueryPlus query, Set segments) + public Query laneQuery(QueryPlus query, Set segments) { return query.getQuery(); } From 405c94e52988ebb1381b3e9337e1d6ff11331ada Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 27 Feb 2020 03:38:39 -0800 Subject: [PATCH 06/37] fixes --- .../org/apache/druid/client/CachingClusteredClient.java | 5 ++++- .../apache/druid/server/QueryCapacityExceededException.java | 6 +++--- .../druid/server/scheduling/HiLoQueryLaningStrategy.java | 2 +- .../druid/server/scheduling/NoQueryLaningStrategy.java | 2 +- 4 files changed, 9 insertions(+), 6 deletions(-) diff --git a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java index 54f8e9853970..392483144662 100644 --- a/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java +++ b/server/src/main/java/org/apache/druid/client/CachingClusteredClient.java @@ -267,7 +267,10 @@ private ImmutableMap makeDownstreamQueryContext() final int priority = QueryContexts.getPriority(query); contextBuilder.put(QueryContexts.PRIORITY_KEY, priority); - contextBuilder.put(QueryContexts.LANE_KEY, QueryContexts.getLane(query)); + final String lane = QueryContexts.getLane(query); + if (lane != null) { + contextBuilder.put(QueryContexts.LANE_KEY, lane); + } if (populateCache) { // prevent down-stream nodes from caching results as well if we are populating the cache diff --git a/server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java b/server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java index e3893b1ef583..687e311b0a15 100644 --- a/server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java +++ b/server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java @@ -23,9 +23,9 @@ public class QueryCapacityExceededException extends RuntimeException { - public static String ERROR_MESSAGE = "Query capacity exceeded"; - public static String ERROR_MESSAGE_TEMPLATE = "Query capacity exceeded for lane %s"; - public static int STATUS_CODE = 429; + public static final String ERROR_MESSAGE = "Query capacity exceeded"; + public static final String ERROR_MESSAGE_TEMPLATE = "Query capacity exceeded for lane %s"; + public static final int STATUS_CODE = 429; public QueryCapacityExceededException() { diff --git a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java index 735f1c05e1f0..6d033a65ea3b 100644 --- a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java @@ -35,7 +35,7 @@ public class HiLoQueryLaningStrategy implements QueryLaningStrategy { - public static String LOW = "low"; + public static final String LOW = "low"; @JsonProperty private int maxLowThreads; diff --git a/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java index ecda3a0d1373..ae73406b48db 100644 --- a/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java @@ -32,7 +32,7 @@ public class NoQueryLaningStrategy implements QueryLaningStrategy { private static final Object2IntMap NONE = new Object2IntArrayMap<>(); - public static NoQueryLaningStrategy INSTANCE = new NoQueryLaningStrategy(); + public static final NoQueryLaningStrategy INSTANCE = new NoQueryLaningStrategy(); @Override public Object2IntMap getLaneLimits() From e22ece12861765ec3eeccc6531c283e717785969 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 27 Feb 2020 16:11:50 -0800 Subject: [PATCH 07/37] doc fixes --- docs/configuration/index.md | 2 +- website/.spelling | 1 + 2 files changed, 2 insertions(+), 1 deletion(-) diff --git a/docs/configuration/index.md b/docs/configuration/index.md index ee9d17ea6d72..6d899786b830 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1478,7 +1478,7 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi ##### Query laning -Druid provides facilities to aid in query capacity reservation for heterogenous query workloads in the form of 'laning' strategies, which provide a variety of mechanisms examine and classify a query at the broker, assigning it to a 'lane'. Lanes are defined with capacity limits which the broker will enforce, causing requests in excess of the capacity to be discarded with an HTTP 429 status code, reserving resources for other lanes or for interactive queries (with no lane). +The Broker provides facilities to aid in query capacity reservation for heterogeneous query workloads in the form of 'laning' strategies, which provide a variety of mechanisms to examine and classify a query, assigning it to a 'lane'. Lanes are defined with capacity limits which the broker will enforce, causing requests in excess of the capacity to be discarded with an HTTP 429 status code, reserving resources for other lanes or for interactive queries (with no lane). |Property|Description|Default| |--------|-----------|-------| diff --git a/website/.spelling b/website/.spelling index 54293ab2e256..83ceba6f72d6 100644 --- a/website/.spelling +++ b/website/.spelling @@ -259,6 +259,7 @@ javadoc kerberos keystore keytab +laning lifecycle localhost log4j From e98cad76992ced1659ab75834bab7a8967eaa33e Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 27 Feb 2020 21:48:15 -0800 Subject: [PATCH 08/37] use resilience4j instead of semaphore --- distribution/bin/check-licenses.py | 1 + licenses.yaml | 22 +++ pom.xml | 6 + server/pom.xml | 4 + .../apache/druid/server/QueryScheduler.java | 123 ++++++-------- .../druid/server/QueryResourceTest.java | 153 +++++++++--------- .../druid/server/QuerySchedulerTest.java | 75 ++++----- 7 files changed, 199 insertions(+), 185 deletions(-) diff --git a/distribution/bin/check-licenses.py b/distribution/bin/check-licenses.py index aad34cb5ca3d..2e584690d7f3 100755 --- a/distribution/bin/check-licenses.py +++ b/distribution/bin/check-licenses.py @@ -214,6 +214,7 @@ def build_compatible_license_names(): compatible_licenses['Apache License, Version 2.0'] = 'Apache License version 2.0' compatible_licenses['The Apache Software License, Version 2.0'] = 'Apache License version 2.0' compatible_licenses['Apache 2.0'] = 'Apache License version 2.0' + compatible_licenses['Apache-2.0'] = 'Apache License version 2.0' compatible_licenses['Apache 2'] = 'Apache License version 2.0' compatible_licenses['Apache License 2.0'] = 'Apache License version 2.0' compatible_licenses['Apache Software License - Version 2.0'] = 'Apache License version 2.0' diff --git a/licenses.yaml b/licenses.yaml index 4c7c2c7ae3e0..9574227e81b5 100644 --- a/licenses.yaml +++ b/licenses.yaml @@ -1869,6 +1869,17 @@ libraries: --- +name: Resilience4j +license_category: binary +module: java-core +license_name: Apache License version 2.0 +version: 1.3.1 +libraries: + - io.github.resilience4j: resilience4j-core + - io.github.resilience4j: resilience4j-bulkhead + +--- + name: RoaringBitmap license_category: binary module: java-core @@ -1880,6 +1891,17 @@ libraries: --- +name: vavr +license_category: binary +module: java-core +license_name: Apache License version 2.0 +version: 0.10.2 +libraries: + - io.vavr: vavr + - io.vavr: vavr-match + +--- + name: Config Magic license_category: binary module: java-core diff --git a/pom.xml b/pom.xml index f6f14f4d7f76..10d4505bfd45 100644 --- a/pom.xml +++ b/pom.xml @@ -94,6 +94,7 @@ 1.9.13 2.8.2 3.10.6.Final + 1.3.1 4.1.45.Final v10.14.2 @@ -1181,6 +1182,11 @@ + + io.github.resilience4j + resilience4j-bulkhead + ${resilience4j.version} + org.testng diff --git a/server/pom.xml b/server/pom.xml index cf903f7e4d65..7bfa26651c4b 100644 --- a/server/pom.xml +++ b/server/pom.xml @@ -311,6 +311,10 @@ org.slf4j slf4j-api + + io.github.resilience4j + resilience4j-bulkhead + diff --git a/server/src/main/java/org/apache/druid/server/QueryScheduler.java b/server/src/main/java/org/apache/druid/server/QueryScheduler.java index 66c5307872c2..2192a805c483 100644 --- a/server/src/main/java/org/apache/druid/server/QueryScheduler.java +++ b/server/src/main/java/org/apache/druid/server/QueryScheduler.java @@ -19,12 +19,13 @@ package org.apache.druid.server; -import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.HashMultimap; import com.google.common.collect.Multimaps; import com.google.common.collect.SetMultimap; -import com.google.common.collect.Sets; import com.google.common.util.concurrent.ListenableFuture; +import io.github.resilience4j.bulkhead.BulkheadConfig; +import io.github.resilience4j.bulkhead.BulkheadFullException; +import io.github.resilience4j.bulkhead.BulkheadRegistry; import it.unimi.dsi.fastutil.objects.Object2IntMap; import org.apache.druid.client.SegmentServerSelector; import org.apache.druid.java.util.common.concurrent.Execs; @@ -34,12 +35,11 @@ import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryWatcher; +import java.time.Duration; import java.util.HashMap; import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.concurrent.Semaphore; -import java.util.concurrent.TimeUnit; /** @@ -47,38 +47,22 @@ * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}. * * As a {@link QueryWatcher}, it also provides cancellation facilities. - * */ public class QueryScheduler implements QueryWatcher { + private static final String TOTAL = "default"; private final QueryLaningStrategy laningStrategy; - private final Optional totalQueryLimit; - private final Map laneLimits; + private final BulkheadRegistry laneRegistery; - private final SetMultimap> queries; + private final SetMultimap> queryFutures; private final SetMultimap queryDatasources; - private final Set> runningQueries; public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy) { this.laningStrategy = laningStrategy; - if (totalNumThreads > 0) { - this.totalQueryLimit = Optional.of(new Semaphore(totalNumThreads)); - } else { - this.totalQueryLimit = Optional.empty(); - } - this.laneLimits = new HashMap<>(); - - for (Object2IntMap.Entry entry : laningStrategy.getLaneLimits().object2IntEntrySet()) { - laneLimits.put(entry.getKey(), new Semaphore(entry.getIntValue())); - } - this.queries = Multimaps.synchronizedSetMultimap( - HashMultimap.create() - ); - this.queryDatasources = Multimaps.synchronizedSetMultimap( - HashMultimap.create() - ); - this.runningQueries = Sets.newConcurrentHashSet(); + this.laneRegistery = BulkheadRegistry.of(getLaneConfigs(totalNumThreads)); + this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create()); + this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create()); } @Override @@ -86,11 +70,11 @@ public void registerQueryFuture(Query query, ListenableFuture future) { final String id = query.getId(); final Set datasources = query.getDataSource().getTableNames(); - queries.put(id, future); + queryFutures.put(id, future); queryDatasources.putAll(id, datasources); future.addListener( () -> { - queries.remove(id, future); + queryFutures.remove(id, future); for (String datasource : datasources) { queryDatasources.remove(id, datasource); } @@ -115,8 +99,17 @@ public Query laneQuery(QueryPlus query, Set seg */ public Sequence run(Query query, Sequence resultSequence) { - scheduleQuery(query); - return resultSequence.withBaggage(() -> completeQuery(query)); + final String lane = QueryContexts.getLane(query); + final Optional totalConfig = laneRegistery.getConfiguration(TOTAL); + final Optional laneConfig = lane == null ? Optional.empty() : laneRegistery.getConfiguration(lane); + + totalConfig.ifPresent(this::acquireTotal); + laneConfig.ifPresent(config -> acquireLane(lane, config)); + + return resultSequence.withBaggage(() -> { + totalConfig.ifPresent(config -> laneRegistery.bulkhead(TOTAL, config).releasePermission()); + laneConfig.ifPresent(config -> laneRegistery.bulkhead(lane, config).releasePermission()); + }); } /** @@ -126,7 +119,7 @@ public boolean cancelQuery(String id) { // if you re-use queryId and cancel queries... you are going to have a bad time queryDatasources.removeAll(id); - Set> futures = queries.removeAll(id); + Set> futures = queryFutures.removeAll(id); boolean success = true; for (ListenableFuture future : futures) { success = success && future.cancel(true); @@ -141,63 +134,53 @@ public Set getQueryDatasources(final String queryId) public int getTotalAvailableCapacity() { - return totalQueryLimit.map(Semaphore::availablePermits).orElse(-1); + return laneRegistery.getConfiguration(TOTAL) + .map(config -> laneRegistery.bulkhead(TOTAL, config).getMetrics().getAvailableConcurrentCalls()) + .orElse(-1); } public int getLaneAvailableCapacity(String lane) { - if (laneLimits.containsKey(lane)) { - return laneLimits.get(lane).availablePermits(); - } - return -1; + return laneRegistery.getConfiguration(lane) + .map(config -> laneRegistery.bulkhead(lane, config).getMetrics().getAvailableConcurrentCalls()) + .orElse(-1); } - /** - * Acquire semaphore from total capacity and lane capacity (if query is assigned a lane that exists) - */ - @VisibleForTesting - void scheduleQuery(Query query) + private void acquireTotal(BulkheadConfig config) { - final String lane; try { - if (!totalQueryLimit.isPresent() || totalQueryLimit.get().tryAcquire(0, TimeUnit.MILLISECONDS)) { - lane = QueryContexts.getLane(query); - // if no lane, we are done - if (!laneLimits.containsKey(lane)) { - runningQueries.add(query); - return; - } - } else { - throw new QueryCapacityExceededException(); - } + laneRegistery.bulkhead(TOTAL, config).acquirePermission(); } - catch (InterruptedException ex) { + catch (BulkheadFullException full) { throw new QueryCapacityExceededException(); } - // if we got here, the query belongs to a lane, acquire the semaphore for it + } + + private void acquireLane(String lane, BulkheadConfig config) + { try { - if (laneLimits.get(lane).tryAcquire(0, TimeUnit.MILLISECONDS)) { - runningQueries.add(query); - } else { - throw new QueryCapacityExceededException(lane); - } + laneRegistery.bulkhead(lane, config).acquirePermission(); } - catch (InterruptedException e) { + catch (BulkheadFullException full) { throw new QueryCapacityExceededException(lane); } } - /** - * Release semaphores help by query - */ - private void completeQuery(Query query) + private Map getLaneConfigs(int totalNumThreads) { - if (runningQueries.remove(query)) { - String lane = QueryContexts.getLane(query); - if (laneLimits.containsKey(lane)) { - laneLimits.get(lane).release(); - } - totalQueryLimit.ifPresent(Semaphore::release); + Map configs = new HashMap<>(); + if (totalNumThreads > 0) { + configs.put( + TOTAL, + BulkheadConfig.custom().maxConcurrentCalls(totalNumThreads).maxWaitDuration(Duration.ZERO).build() + ); + } + for (Object2IntMap.Entry entry : laningStrategy.getLaneLimits().object2IntEntrySet()) { + configs.put( + entry.getKey(), + BulkheadConfig.custom().maxConcurrentCalls(entry.getIntValue()).maxWaitDuration(Duration.ZERO).build() + ); } + return configs; } } diff --git a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java index 60c840bb072f..5fbf8190a1c2 100644 --- a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java @@ -31,7 +31,6 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.LazySequence; -import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.service.ServiceEmitter; @@ -39,13 +38,11 @@ import org.apache.druid.query.MapQueryToolChestWarehouse; import org.apache.druid.query.Query; import org.apache.druid.query.QueryContexts; -import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.QueryToolChestWarehouse; import org.apache.druid.query.Result; import org.apache.druid.query.SegmentDescriptor; -import org.apache.druid.query.context.ResponseContext; import org.apache.druid.query.timeboundary.TimeBoundaryResultValue; import org.apache.druid.server.log.TestRequestLogger; import org.apache.druid.server.metrics.NoopServiceEmitter; @@ -77,6 +74,7 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.util.Collection; +import java.util.Collections; import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executors; @@ -88,24 +86,19 @@ */ public class QueryResourceTest { + private static final QueryToolChestWarehouse WAREHOUSE = new MapQueryToolChestWarehouse(ImmutableMap.of()); private static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper(); private static final AuthenticationResult AUTHENTICATION_RESULT = new AuthenticationResult("druid", "druid", null, null); private final HttpServletRequest testServletRequest = EasyMock.createMock(HttpServletRequest.class); - public static final QuerySegmentWalker TEST_SEGMENT_WALKER = new QuerySegmentWalker() + + private static final QuerySegmentWalker TEST_SEGMENT_WALKER = new QuerySegmentWalker() { @Override public QueryRunner getQueryRunnerForIntervals(Query query, Iterable intervals) { - return new QueryRunner() - { - @Override - public Sequence run(QueryPlus query, ResponseContext responseContext) - { - return Sequences.empty(); - } - }; + return (queryPlus, responseContext) -> Sequences.empty(); } @Override @@ -115,6 +108,38 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()) ); - waitFirstStart.await(); assertResponseAndCountdownOrBlockForever( SIMPLE_TIMESERIES_QUERY, - waitFinishLatch, + waitAllFinished, response -> Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()) ); - waitSecondStart.await(); + waitTwoScheduled.await(); assertResponseAndCountdownOrBlockForever( SIMPLE_TIMESERIES_QUERY, - waitFinishLatch, + waitAllFinished, response -> { Assert.assertEquals(QueryCapacityExceededException.STATUS_CODE, response.getStatus()); Assert.assertEquals(QueryCapacityExceededException.ERROR_MESSAGE, response.getEntity()); } ); - waitFinishLatch.await(); + waitAllFinished.await(); } @Test(timeout = 10_000L) public void testTooManyQueryInLane() throws InterruptedException { - expectPermissiveHappyPath(); - final CountDownLatch waitFirstStart = new CountDownLatch(1); - final CountDownLatch waitSecondStart = new CountDownLatch(2); - final CountDownLatch waitFinishLatch = new CountDownLatch(3); + expectPermissiveHappyPathAuth(); + final CountDownLatch waitTwoStarted = new CountDownLatch(2); + final CountDownLatch waitOneScheduled = new CountDownLatch(1); + final CountDownLatch waitAllFinished = new CountDownLatch(3); final QueryScheduler scheduler = new QueryScheduler(40, new HiLoQueryLaningStrategy(1)); - createScheduledQueryResource(scheduler, ImmutableList.of(waitFirstStart, waitSecondStart)); + createScheduledQueryResource(scheduler, ImmutableList.of(waitTwoStarted), ImmutableList.of(waitOneScheduled)); assertResponseAndCountdownOrBlockForever( SIMPLE_TIMESERIES_QUERY_LOW_PRIORITY, - waitFinishLatch, + waitAllFinished, response -> Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()) ); - waitFirstStart.await(); + waitOneScheduled.await(); assertResponseAndCountdownOrBlockForever( SIMPLE_TIMESERIES_QUERY_LOW_PRIORITY, - waitFinishLatch, + waitAllFinished, response -> { Assert.assertEquals(QueryCapacityExceededException.STATUS_CODE, response.getStatus()); Assert.assertEquals( @@ -735,25 +733,21 @@ public void testTooManyQueryInLane() throws InterruptedException response.getEntity()); } ); - waitSecondStart.await(); + waitTwoStarted.await(); assertResponseAndCountdownOrBlockForever( SIMPLE_TIMESERIES_QUERY, - waitFinishLatch, + waitAllFinished, response -> Assert.assertEquals(Response.Status.OK.getStatusCode(), response.getStatus()) ); - waitFinishLatch.await(); - } - - @After - public void tearDown() - { - EasyMock.verify(testServletRequest); + waitAllFinished.await(); } - - - private void createScheduledQueryResource(QueryScheduler scheduler, Collection latches) + private void createScheduledQueryResource( + QueryScheduler scheduler, + Collection beforeScheduler, + Collection inScheduler + ) { AtomicInteger counter = new AtomicInteger(0); @@ -769,16 +763,19 @@ public QueryRunner getQueryRunnerForIntervals(Query query, Iterable(() -> { + inScheduler.forEach(CountDownLatch::countDown); try { + // pretend to be a query that is waiting on results Thread.sleep(500); } catch (InterruptedException ignored) { } + // all that waiting for nothing :( return Sequences.empty(); }) ); @@ -811,7 +808,7 @@ public QueryRunner getQueryRunnerForSegments(Query query, Iterable asserts) + private void assertResponseAndCountdownOrBlockForever(String query, CountDownLatch done, Consumer asserts) { Executors.newSingleThreadExecutor().submit(() -> { try { @@ -825,11 +822,11 @@ private void assertResponseAndCountdownOrBlockForever(String query, CountDownLat catch (IOException e) { throw new RuntimeException(e); } - waitFinishLatch.countDown(); + done.countDown(); }); } - private void expectPermissiveHappyPath() + private void expectPermissiveHappyPathAuth() { EasyMock.expect(testServletRequest.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) .andReturn(null) diff --git a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java index e043b75a3517..5e4d756cfa43 100644 --- a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java +++ b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java @@ -26,6 +26,7 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.BaseSequence; +import org.apache.druid.java.util.common.guava.LazySequence; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.SequenceWrapper; import org.apache.druid.java.util.common.guava.Sequences; @@ -180,19 +181,19 @@ public void testHiLoFailsWhenOutOfLaneCapacity() QueryScheduler scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)); Query report1 = scheduler.laneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); - scheduler.scheduleQuery(report1); + scheduler.run(report1, Sequences.empty()); Assert.assertNotNull(report1); Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); Assert.assertEquals(1, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); Query report2 = scheduler.laneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); - scheduler.scheduleQuery(report2); + scheduler.run(report2, Sequences.empty()); Assert.assertNotNull(report2); Assert.assertEquals(3, scheduler.getTotalAvailableCapacity()); Assert.assertEquals(0, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); // too many reports - scheduler.scheduleQuery(scheduler.laneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of())); + scheduler.run(scheduler.laneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()), Sequences.empty()); } @Test @@ -203,36 +204,34 @@ public void testHiLoFailsWhenOutOfTotalCapacity() QueryScheduler scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)); Query interactive1 = scheduler.laneQuery(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()); - scheduler.scheduleQuery(interactive1); + scheduler.run(interactive1, Sequences.empty()); Assert.assertNotNull(interactive1); Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); Query report1 = scheduler.laneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); - scheduler.scheduleQuery(report1); + scheduler.run(report1, Sequences.empty()); Assert.assertNotNull(report1); Assert.assertEquals(3, scheduler.getTotalAvailableCapacity()); Assert.assertEquals(1, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); Query interactive2 = scheduler.laneQuery(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()); - scheduler.scheduleQuery(interactive2); + scheduler.run(interactive2, Sequences.empty()); Assert.assertNotNull(interactive2); Assert.assertEquals(2, scheduler.getTotalAvailableCapacity()); Query report2 = scheduler.laneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); - scheduler.scheduleQuery(report2); + scheduler.run(report2, Sequences.empty()); Assert.assertNotNull(report2); Assert.assertEquals(1, scheduler.getTotalAvailableCapacity()); Assert.assertEquals(0, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); Query interactive3 = scheduler.laneQuery(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()); - scheduler.scheduleQuery(interactive3); + scheduler.run(interactive3, Sequences.empty()); Assert.assertNotNull(interactive3); Assert.assertEquals(0, scheduler.getTotalAvailableCapacity()); // one too many - scheduler.scheduleQuery( - scheduler.laneQuery(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()) - ); + scheduler.run(scheduler.laneQuery(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()), Sequences.empty()); } @@ -275,38 +274,40 @@ private int consumeAndCloseSequence(Sequence sequence) throws IOException private Sequence makeSequence(int count) { - return new BaseSequence<>( - new BaseSequence.IteratorMaker>() - { - @Override - public Iterator make() + return new LazySequence<>(() -> { + return new BaseSequence<>( + new BaseSequence.IteratorMaker>() { - return new Iterator() + @Override + public Iterator make() { - int rowCounter = 0; - - @Override - public boolean hasNext() + return new Iterator() { - return rowCounter < count; - } + int rowCounter = 0; - @Override - public Integer next() - { - rowCounter++; - return rowCounter; - } - }; - } + @Override + public boolean hasNext() + { + return rowCounter < count; + } - @Override - public void cleanup(Iterator iterFromMake) - { - // nothing to cleanup + @Override + public Integer next() + { + rowCounter++; + return rowCounter; + } + }; + } + + @Override + public void cleanup(Iterator iterFromMake) + { + // nothing to cleanup + } } - } - ); + ); + }); } private Sequence makeExplodingSequence(int explodeAfter) From 2069437298c6b111804d0fd18034514c47ae8f1b Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Thu, 27 Feb 2020 22:59:43 -0800 Subject: [PATCH 09/37] more tests --- .../scheduling/HiLoQueryLaningStrategy.java | 2 +- .../HiLoQueryLaningStrategyTest.java | 107 ++++++++++++++++++ 2 files changed, 108 insertions(+), 1 deletion(-) create mode 100644 server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java diff --git a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java index 6d033a65ea3b..b31409724059 100644 --- a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java @@ -38,7 +38,7 @@ public class HiLoQueryLaningStrategy implements QueryLaningStrategy public static final String LOW = "low"; @JsonProperty - private int maxLowThreads; + private final int maxLowThreads; @JsonCreator public HiLoQueryLaningStrategy( diff --git a/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java b/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java new file mode 100644 index 000000000000..94c61b680dd9 --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java @@ -0,0 +1,107 @@ +/* + * 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.scheduling; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +import it.unimi.dsi.fastutil.objects.Object2IntMap; +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.QueryContexts; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.apache.druid.server.QueryLaningStrategy; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.ExpectedException; + +public class HiLoQueryLaningStrategyTest +{ + @Rule + public ExpectedException expectedException = ExpectedException.none(); + + private Druids.TimeseriesQueryBuilder queryBuilder; + private HiLoQueryLaningStrategy strategy; + + @Before + public void setup() + { + this.queryBuilder = Druids.newTimeseriesQueryBuilder() + .dataSource("test") + .intervals(ImmutableList.of(Intervals.ETERNITY)) + .granularity(Granularities.DAY) + .aggregators(new CountAggregatorFactory("count")); + + this.strategy = new HiLoQueryLaningStrategy(10); + } + + @Test + public void testMaxLowThreadsRequired() + { + expectedException.expect(NullPointerException.class); + expectedException.expectMessage("maxLowThreads must be set"); + QueryLaningStrategy strategy = new HiLoQueryLaningStrategy(null); + } + + @Test + public void testLaneLimits() + { + Object2IntMap laneConfig = strategy.getLaneLimits(); + Assert.assertEquals(1, laneConfig.size()); + Assert.assertTrue(laneConfig.containsKey(HiLoQueryLaningStrategy.LOW)); + Assert.assertEquals(10, laneConfig.getInt(HiLoQueryLaningStrategy.LOW)); + } + + @Test + public void testLaningNoPriority() + { + TimeseriesQuery interactive = queryBuilder.build(); + Assert.assertNull(QueryContexts.getLane(strategy.laneQuery(QueryPlus.wrap(interactive), ImmutableSet.of()))); + } + + @Test + public void testLaningZeroPriority() + { + TimeseriesQuery interactive = queryBuilder.context(ImmutableMap.of(QueryContexts.PRIORITY_KEY, 0)).build(); + Assert.assertNull(QueryContexts.getLane(strategy.laneQuery(QueryPlus.wrap(interactive), ImmutableSet.of()))); + } + + @Test + public void testLaningInteractivePriority() + { + TimeseriesQuery interactive = queryBuilder.context(ImmutableMap.of(QueryContexts.PRIORITY_KEY, 100)).build(); + Assert.assertNull(QueryContexts.getLane(strategy.laneQuery(QueryPlus.wrap(interactive), ImmutableSet.of()))); + } + + @Test + public void testLaningLowPriority() + { + TimeseriesQuery interactive = queryBuilder.context(ImmutableMap.of(QueryContexts.PRIORITY_KEY, -1)).build(); + Assert.assertEquals( + HiLoQueryLaningStrategy.LOW, + QueryContexts.getLane(strategy.laneQuery(QueryPlus.wrap(interactive), ImmutableSet.of())) + ); + } +} From eaf1449a360506a85257f16abe25201875af8387 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 28 Feb 2020 00:42:09 -0800 Subject: [PATCH 10/37] simplify --- .../main/java/org/apache/druid/query/Query.java | 7 +++++++ .../druid/server/QueryLaningStrategy.java | 4 ++-- .../org/apache/druid/server/QueryScheduler.java | 11 +++++++---- .../scheduling/HiLoQueryLaningStrategy.java | 13 ++++++------- .../scheduling/NoQueryLaningStrategy.java | 5 +++-- .../apache/druid/server/QueryResourceTest.java | 9 --------- .../scheduling/HiLoQueryLaningStrategyTest.java | 17 +++++++++-------- 7 files changed, 34 insertions(+), 32 deletions(-) diff --git a/processing/src/main/java/org/apache/druid/query/Query.java b/processing/src/main/java/org/apache/druid/query/Query.java index 37868b4a25d4..e538c09ec14b 100644 --- a/processing/src/main/java/org/apache/druid/query/Query.java +++ b/processing/src/main/java/org/apache/druid/query/Query.java @@ -21,6 +21,7 @@ import com.fasterxml.jackson.annotation.JsonSubTypes; import com.fasterxml.jackson.annotation.JsonTypeInfo; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.Ordering; import org.apache.druid.guice.annotations.ExtensionPoint; import org.apache.druid.java.util.common.granularity.Granularity; @@ -146,4 +147,10 @@ default Query optimizeForSegment(PerSegmentQueryOptimizationContext optimizat { return this; } + + default Query withLane(String lane) + { + return withOverriddenContext(ImmutableMap.of(QueryContexts.LANE_KEY, lane)); + } + } diff --git a/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java index ad8e2672169a..d0245063f232 100644 --- a/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java @@ -23,11 +23,11 @@ import com.fasterxml.jackson.annotation.JsonTypeInfo; import it.unimi.dsi.fastutil.objects.Object2IntMap; import org.apache.druid.client.SegmentServerSelector; -import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.server.scheduling.HiLoQueryLaningStrategy; import org.apache.druid.server.scheduling.NoQueryLaningStrategy; +import java.util.Optional; import java.util.Set; @@ -40,5 +40,5 @@ public interface QueryLaningStrategy { Object2IntMap getLaneLimits(); - Query laneQuery(QueryPlus query, Set segments); + Optional computeLane(QueryPlus query, Set segments); } diff --git a/server/src/main/java/org/apache/druid/server/QueryScheduler.java b/server/src/main/java/org/apache/druid/server/QueryScheduler.java index 2192a805c483..f0755ab6abf1 100644 --- a/server/src/main/java/org/apache/druid/server/QueryScheduler.java +++ b/server/src/main/java/org/apache/druid/server/QueryScheduler.java @@ -86,12 +86,15 @@ public void registerQueryFuture(Query query, ListenableFuture future) /** * Assign a query a lane (if not set) */ - public Query laneQuery(QueryPlus query, Set segments) + public Query laneQuery(QueryPlus queryPlus, Set segments) { - if (QueryContexts.getLane(query.getQuery()) != null) { - return query.getQuery(); + Query query = queryPlus.getQuery(); + // man wins over machine.. for now. + if (QueryContexts.getLane(query) != null) { + return query; } - return laningStrategy.laneQuery(query, segments); + Optional lane = laningStrategy.computeLane(queryPlus, segments); + return lane.map(query::withLane).orElse(query); } /** diff --git a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java index b31409724059..411a7fa40dfc 100644 --- a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java @@ -31,6 +31,7 @@ import org.apache.druid.query.QueryPlus; import org.apache.druid.server.QueryLaningStrategy; +import java.util.Optional; import java.util.Set; public class HiLoQueryLaningStrategy implements QueryLaningStrategy @@ -57,19 +58,17 @@ public Object2IntMap getLaneLimits() } @Override - public Query laneQuery(QueryPlus query, Set segments) + public Optional computeLane( + QueryPlus query, Set segments + ) { final Query theQuery = query.getQuery(); // QueryContexts.getPriority gives a default, since we are setting priority final Integer priority = theQuery.getContextValue(QueryContexts.PRIORITY_KEY); final String lane = theQuery.getContextValue(QueryContexts.LANE_KEY); if (lane == null && priority != null && priority < 0) { - return theQuery.withOverriddenContext( - ImmutableMap.builder().putAll(theQuery.getContext()) - .put(QueryContexts.LANE_KEY, LOW) - .build() - ); + return Optional.of(LOW); } - return theQuery; + return Optional.empty(); } } diff --git a/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java index ae73406b48db..49aadf195054 100644 --- a/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java @@ -26,6 +26,7 @@ import org.apache.druid.query.QueryPlus; import org.apache.druid.server.QueryLaningStrategy; +import java.util.Optional; import java.util.Set; public class NoQueryLaningStrategy implements QueryLaningStrategy @@ -41,8 +42,8 @@ public Object2IntMap getLaneLimits() } @Override - public Query laneQuery(QueryPlus query, Set segments) + public Optional computeLane(QueryPlus query, Set segments) { - return query.getQuery(); + return Optional.empty(); } } diff --git a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java index 5fbf8190a1c2..7ca4bda246e5 100644 --- a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java @@ -37,7 +37,6 @@ import org.apache.druid.query.DefaultGenericQueryMetricsFactory; import org.apache.druid.query.MapQueryToolChestWarehouse; import org.apache.druid.query.Query; -import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryRunner; import org.apache.druid.query.QuerySegmentWalker; import org.apache.druid.query.QueryToolChestWarehouse; @@ -78,7 +77,6 @@ import java.util.List; import java.util.concurrent.CountDownLatch; import java.util.concurrent.Executors; -import java.util.concurrent.atomic.AtomicInteger; import java.util.function.Consumer; /** @@ -749,7 +747,6 @@ private void createScheduledQueryResource( Collection inScheduler ) { - AtomicInteger counter = new AtomicInteger(0); QuerySegmentWalker texasRanger = new QuerySegmentWalker() { @@ -757,12 +754,6 @@ private void createScheduledQueryResource( public QueryRunner getQueryRunnerForIntervals(Query query, Iterable intervals) { return (queryPlus, responseContext) -> { - int count = counter.getAndIncrement(); - if (count < 2) { - queryPlus = queryPlus.withQuery( - queryPlus.getQuery().withOverriddenContext(ImmutableMap.of(QueryContexts.PRIORITY_KEY, -1)) - ); - } beforeScheduler.forEach(CountDownLatch::countDown); return scheduler.run( diff --git a/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java b/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java index 94c61b680dd9..10eceac56c96 100644 --- a/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java @@ -77,31 +77,32 @@ public void testLaneLimits() @Test public void testLaningNoPriority() { - TimeseriesQuery interactive = queryBuilder.build(); - Assert.assertNull(QueryContexts.getLane(strategy.laneQuery(QueryPlus.wrap(interactive), ImmutableSet.of()))); + TimeseriesQuery query = queryBuilder.build(); + Assert.assertFalse(strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).isPresent()); } @Test public void testLaningZeroPriority() { - TimeseriesQuery interactive = queryBuilder.context(ImmutableMap.of(QueryContexts.PRIORITY_KEY, 0)).build(); - Assert.assertNull(QueryContexts.getLane(strategy.laneQuery(QueryPlus.wrap(interactive), ImmutableSet.of()))); + TimeseriesQuery query = queryBuilder.context(ImmutableMap.of(QueryContexts.PRIORITY_KEY, 0)).build(); + Assert.assertFalse(strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).isPresent()); } @Test public void testLaningInteractivePriority() { - TimeseriesQuery interactive = queryBuilder.context(ImmutableMap.of(QueryContexts.PRIORITY_KEY, 100)).build(); - Assert.assertNull(QueryContexts.getLane(strategy.laneQuery(QueryPlus.wrap(interactive), ImmutableSet.of()))); + TimeseriesQuery query = queryBuilder.context(ImmutableMap.of(QueryContexts.PRIORITY_KEY, 100)).build(); + Assert.assertFalse(strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).isPresent()); } @Test public void testLaningLowPriority() { - TimeseriesQuery interactive = queryBuilder.context(ImmutableMap.of(QueryContexts.PRIORITY_KEY, -1)).build(); + TimeseriesQuery query = queryBuilder.context(ImmutableMap.of(QueryContexts.PRIORITY_KEY, -1)).build(); + Assert.assertTrue(strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).isPresent()); Assert.assertEquals( HiLoQueryLaningStrategy.LOW, - QueryContexts.getLane(strategy.laneQuery(QueryPlus.wrap(interactive), ImmutableSet.of())) + strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).get() ); } } From 688ca43af1f246dd6578e2ace96778cdb3571cff Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 28 Feb 2020 03:09:06 -0800 Subject: [PATCH 11/37] checkstyle --- .../druid/server/scheduling/HiLoQueryLaningStrategy.java | 5 +---- .../druid/server/scheduling/NoQueryLaningStrategy.java | 1 - 2 files changed, 1 insertion(+), 5 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java index 411a7fa40dfc..fdefb68193ef 100644 --- a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java @@ -22,7 +22,6 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableMap; import it.unimi.dsi.fastutil.objects.Object2IntArrayMap; import it.unimi.dsi.fastutil.objects.Object2IntMap; import org.apache.druid.client.SegmentServerSelector; @@ -58,9 +57,7 @@ public Object2IntMap getLaneLimits() } @Override - public Optional computeLane( - QueryPlus query, Set segments - ) + public Optional computeLane(QueryPlus query, Set segments) { final Query theQuery = query.getQuery(); // QueryContexts.getPriority gives a default, since we are setting priority diff --git a/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java index 49aadf195054..730d049e2875 100644 --- a/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java @@ -22,7 +22,6 @@ import it.unimi.dsi.fastutil.objects.Object2IntArrayMap; import it.unimi.dsi.fastutil.objects.Object2IntMap; import org.apache.druid.client.SegmentServerSelector; -import org.apache.druid.query.Query; import org.apache.druid.query.QueryPlus; import org.apache.druid.server.QueryLaningStrategy; From f0b3f9f02be406e42e8fa523716c630d2b21aa36 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 28 Feb 2020 11:11:38 -0800 Subject: [PATCH 12/37] spelling --- .../apache/druid/server/QueryScheduler.java | 28 +++++++++---------- 1 file changed, 14 insertions(+), 14 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/QueryScheduler.java b/server/src/main/java/org/apache/druid/server/QueryScheduler.java index f0755ab6abf1..2bf13f1d79a5 100644 --- a/server/src/main/java/org/apache/druid/server/QueryScheduler.java +++ b/server/src/main/java/org/apache/druid/server/QueryScheduler.java @@ -52,7 +52,7 @@ public class QueryScheduler implements QueryWatcher { private static final String TOTAL = "default"; private final QueryLaningStrategy laningStrategy; - private final BulkheadRegistry laneRegistery; + private final BulkheadRegistry laneRegistry; private final SetMultimap> queryFutures; private final SetMultimap queryDatasources; @@ -60,7 +60,7 @@ public class QueryScheduler implements QueryWatcher public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy) { this.laningStrategy = laningStrategy; - this.laneRegistery = BulkheadRegistry.of(getLaneConfigs(totalNumThreads)); + this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(totalNumThreads)); this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create()); this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create()); } @@ -103,15 +103,15 @@ public Query laneQuery(QueryPlus queryPlus, Set public Sequence run(Query query, Sequence resultSequence) { final String lane = QueryContexts.getLane(query); - final Optional totalConfig = laneRegistery.getConfiguration(TOTAL); - final Optional laneConfig = lane == null ? Optional.empty() : laneRegistery.getConfiguration(lane); + final Optional totalConfig = laneRegistry.getConfiguration(TOTAL); + final Optional laneConfig = lane == null ? Optional.empty() : laneRegistry.getConfiguration(lane); totalConfig.ifPresent(this::acquireTotal); laneConfig.ifPresent(config -> acquireLane(lane, config)); return resultSequence.withBaggage(() -> { - totalConfig.ifPresent(config -> laneRegistery.bulkhead(TOTAL, config).releasePermission()); - laneConfig.ifPresent(config -> laneRegistery.bulkhead(lane, config).releasePermission()); + totalConfig.ifPresent(config -> laneRegistry.bulkhead(TOTAL, config).releasePermission()); + laneConfig.ifPresent(config -> laneRegistry.bulkhead(lane, config).releasePermission()); }); } @@ -137,22 +137,22 @@ public Set getQueryDatasources(final String queryId) public int getTotalAvailableCapacity() { - return laneRegistery.getConfiguration(TOTAL) - .map(config -> laneRegistery.bulkhead(TOTAL, config).getMetrics().getAvailableConcurrentCalls()) - .orElse(-1); + return laneRegistry.getConfiguration(TOTAL) + .map(config -> laneRegistry.bulkhead(TOTAL, config).getMetrics().getAvailableConcurrentCalls()) + .orElse(-1); } public int getLaneAvailableCapacity(String lane) { - return laneRegistery.getConfiguration(lane) - .map(config -> laneRegistery.bulkhead(lane, config).getMetrics().getAvailableConcurrentCalls()) - .orElse(-1); + return laneRegistry.getConfiguration(lane) + .map(config -> laneRegistry.bulkhead(lane, config).getMetrics().getAvailableConcurrentCalls()) + .orElse(-1); } private void acquireTotal(BulkheadConfig config) { try { - laneRegistery.bulkhead(TOTAL, config).acquirePermission(); + laneRegistry.bulkhead(TOTAL, config).acquirePermission(); } catch (BulkheadFullException full) { throw new QueryCapacityExceededException(); @@ -162,7 +162,7 @@ private void acquireTotal(BulkheadConfig config) private void acquireLane(String lane, BulkheadConfig config) { try { - laneRegistery.bulkhead(lane, config).acquirePermission(); + laneRegistry.bulkhead(lane, config).acquirePermission(); } catch (BulkheadFullException full) { throw new QueryCapacityExceededException(lane); From 87c6cbd1660d4981fc3618feed9210f91a218ca5 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 28 Feb 2020 14:15:22 -0800 Subject: [PATCH 13/37] oops heh --- .../java/org/apache/druid/server/QueryScheduler.java | 12 ++++++++++-- 1 file changed, 10 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/QueryScheduler.java b/server/src/main/java/org/apache/druid/server/QueryScheduler.java index 2bf13f1d79a5..a390d2911037 100644 --- a/server/src/main/java/org/apache/druid/server/QueryScheduler.java +++ b/server/src/main/java/org/apache/druid/server/QueryScheduler.java @@ -51,6 +51,7 @@ public class QueryScheduler implements QueryWatcher { private static final String TOTAL = "default"; + private final int numThreads; private final QueryLaningStrategy laningStrategy; private final BulkheadRegistry laneRegistry; @@ -59,6 +60,7 @@ public class QueryScheduler implements QueryWatcher public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy) { + this.numThreads = totalNumThreads; this.laningStrategy = laningStrategy; this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(totalNumThreads)); this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create()); @@ -107,7 +109,7 @@ public Sequence run(Query query, Sequence resultSequence) final Optional laneConfig = lane == null ? Optional.empty() : laneRegistry.getConfiguration(lane); totalConfig.ifPresent(this::acquireTotal); - laneConfig.ifPresent(config -> acquireLane(lane, config)); + laneConfig.ifPresent(config -> acquireLane(lane, config, totalConfig)); return resultSequence.withBaggage(() -> { totalConfig.ifPresent(config -> laneRegistry.bulkhead(TOTAL, config).releasePermission()); @@ -135,6 +137,11 @@ public Set getQueryDatasources(final String queryId) return queryDatasources.get(queryId); } + public int getNumThreads() + { + return numThreads; + } + public int getTotalAvailableCapacity() { return laneRegistry.getConfiguration(TOTAL) @@ -159,12 +166,13 @@ private void acquireTotal(BulkheadConfig config) } } - private void acquireLane(String lane, BulkheadConfig config) + private void acquireLane(String lane, BulkheadConfig config, Optional totalToReleaseIfFailed) { try { laneRegistry.bulkhead(lane, config).acquirePermission(); } catch (BulkheadFullException full) { + totalToReleaseIfFailed.ifPresent(c -> laneRegistry.bulkhead(TOTAL, c).releasePermission()); throw new QueryCapacityExceededException(lane); } } From 5e91bcba6661334d8b6d6dc745ce2cd68037ca0e Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 28 Feb 2020 14:17:34 -0800 Subject: [PATCH 14/37] remove unused --- .../main/java/org/apache/druid/server/QueryScheduler.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/QueryScheduler.java b/server/src/main/java/org/apache/druid/server/QueryScheduler.java index a390d2911037..76116618b4f0 100644 --- a/server/src/main/java/org/apache/druid/server/QueryScheduler.java +++ b/server/src/main/java/org/apache/druid/server/QueryScheduler.java @@ -51,7 +51,6 @@ public class QueryScheduler implements QueryWatcher { private static final String TOTAL = "default"; - private final int numThreads; private final QueryLaningStrategy laningStrategy; private final BulkheadRegistry laneRegistry; @@ -60,7 +59,6 @@ public class QueryScheduler implements QueryWatcher public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy) { - this.numThreads = totalNumThreads; this.laningStrategy = laningStrategy; this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(totalNumThreads)); this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create()); @@ -137,11 +135,6 @@ public Set getQueryDatasources(final String queryId) return queryDatasources.get(queryId); } - public int getNumThreads() - { - return numThreads; - } - public int getTotalAvailableCapacity() { return laneRegistry.getConfiguration(TOTAL) From 912b7bc43d5406114f85e85918e04b8c01a9bf7f Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 28 Feb 2020 19:07:33 -0800 Subject: [PATCH 15/37] simplify --- .../java/org/apache/druid/server/QueryScheduler.java | 11 ++--------- 1 file changed, 2 insertions(+), 9 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/QueryScheduler.java b/server/src/main/java/org/apache/druid/server/QueryScheduler.java index 76116618b4f0..6a53539b1926 100644 --- a/server/src/main/java/org/apache/druid/server/QueryScheduler.java +++ b/server/src/main/java/org/apache/druid/server/QueryScheduler.java @@ -24,7 +24,6 @@ import com.google.common.collect.SetMultimap; import com.google.common.util.concurrent.ListenableFuture; import io.github.resilience4j.bulkhead.BulkheadConfig; -import io.github.resilience4j.bulkhead.BulkheadFullException; import io.github.resilience4j.bulkhead.BulkheadRegistry; import it.unimi.dsi.fastutil.objects.Object2IntMap; import org.apache.druid.client.SegmentServerSelector; @@ -151,20 +150,14 @@ public int getLaneAvailableCapacity(String lane) private void acquireTotal(BulkheadConfig config) { - try { - laneRegistry.bulkhead(TOTAL, config).acquirePermission(); - } - catch (BulkheadFullException full) { + if (!laneRegistry.bulkhead(TOTAL, config).tryAcquirePermission()) { throw new QueryCapacityExceededException(); } } private void acquireLane(String lane, BulkheadConfig config, Optional totalToReleaseIfFailed) { - try { - laneRegistry.bulkhead(lane, config).acquirePermission(); - } - catch (BulkheadFullException full) { + if (!laneRegistry.bulkhead(lane, config).tryAcquirePermission()) { totalToReleaseIfFailed.ifPresent(c -> laneRegistry.bulkhead(TOTAL, c).releasePermission()); throw new QueryCapacityExceededException(lane); } From 1e384bf9e4d8c2cae1cf12e894aa2afa00e9a9c1 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 2 Mar 2020 03:48:27 -0800 Subject: [PATCH 16/37] concurrency tests --- .../CachingClusteredClientBenchmark.java | 2 +- .../movingaverage/MovingAverageQueryTest.java | 2 +- .../druid/client/SegmentServerSelector.java | 5 +- .../apache/druid/server/QueryScheduler.java | 73 ++++++-- ...chingClusteredClientFunctionalityTest.java | 3 +- .../client/CachingClusteredClientTest.java | 2 +- .../druid/server/QuerySchedulerTest.java | 160 +++++++++++++++--- 7 files changed, 201 insertions(+), 46 deletions(-) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index 84147b8d46cd..506f84556e5d 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -341,7 +341,7 @@ public > QueryToolChest getToolChest new DruidHttpClientConfig(), processingConfig, forkJoinPool, - new QueryScheduler(Integer.MAX_VALUE, NoQueryLaningStrategy.INSTANCE) + new QueryScheduler(0, NoQueryLaningStrategy.INSTANCE) ); } diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index c3b150b30fe4..1378c009d107 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -364,7 +364,7 @@ public String getFormatString() } }, ForkJoinPool.commonPool(), - new QueryScheduler(Integer.MAX_VALUE, NoQueryLaningStrategy.INSTANCE) + new QueryScheduler(0, NoQueryLaningStrategy.INSTANCE) ); ClientQuerySegmentWalker walker = new ClientQuerySegmentWalker( diff --git a/server/src/main/java/org/apache/druid/client/SegmentServerSelector.java b/server/src/main/java/org/apache/druid/client/SegmentServerSelector.java index c5dd11def680..007b7a254591 100644 --- a/server/src/main/java/org/apache/druid/client/SegmentServerSelector.java +++ b/server/src/main/java/org/apache/druid/client/SegmentServerSelector.java @@ -24,10 +24,9 @@ import org.apache.druid.query.SegmentDescriptor; /** - * Given a {@link SegmentDescriptor}, get a {@link ServerSelector} to use to pick a {@link DruidServer} to query the - * segment. + * Given a {@link SegmentDescriptor}, get a {@link ServerSelector} to use to pick a {@link DruidServer} to query. * - * Used by {@link CachingClusteredClient} on the broker to fan out queries to historical and realtime servers + * Used by {@link CachingClusteredClient} on the broker to fan out queries to historical and realtime data */ public class SegmentServerSelector extends Pair { diff --git a/server/src/main/java/org/apache/druid/server/QueryScheduler.java b/server/src/main/java/org/apache/druid/server/QueryScheduler.java index 6a53539b1926..a4ebd6662ccc 100644 --- a/server/src/main/java/org/apache/druid/server/QueryScheduler.java +++ b/server/src/main/java/org/apache/druid/server/QueryScheduler.java @@ -23,6 +23,7 @@ import com.google.common.collect.Multimaps; import com.google.common.collect.SetMultimap; import com.google.common.util.concurrent.ListenableFuture; +import io.github.resilience4j.bulkhead.Bulkhead; import io.github.resilience4j.bulkhead.BulkheadConfig; import io.github.resilience4j.bulkhead.BulkheadRegistry; import it.unimi.dsi.fastutil.objects.Object2IntMap; @@ -35,10 +36,13 @@ import org.apache.druid.query.QueryWatcher; import java.time.Duration; +import java.util.ArrayList; import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; +import java.util.concurrent.atomic.AtomicLong; /** @@ -56,6 +60,9 @@ public class QueryScheduler implements QueryWatcher private final SetMultimap> queryFutures; private final SetMultimap queryDatasources; + private final AtomicLong totalAcquired = new AtomicLong(); + private final AtomicLong totalReleased = new AtomicLong(); + public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy) { this.laningStrategy = laningStrategy; @@ -101,17 +108,8 @@ public Query laneQuery(QueryPlus queryPlus, Set */ public Sequence run(Query query, Sequence resultSequence) { - final String lane = QueryContexts.getLane(query); - final Optional totalConfig = laneRegistry.getConfiguration(TOTAL); - final Optional laneConfig = lane == null ? Optional.empty() : laneRegistry.getConfiguration(lane); - - totalConfig.ifPresent(this::acquireTotal); - laneConfig.ifPresent(config -> acquireLane(lane, config, totalConfig)); - - return resultSequence.withBaggage(() -> { - totalConfig.ifPresent(config -> laneRegistry.bulkhead(TOTAL, config).releasePermission()); - laneConfig.ifPresent(config -> laneRegistry.bulkhead(lane, config).releasePermission()); - }); + List bulkheads = acquireLanes(query); + return resultSequence.withBaggage(() -> releaseLanes(bulkheads)); } /** @@ -148,19 +146,62 @@ public int getLaneAvailableCapacity(String lane) .orElse(-1); } - private void acquireTotal(BulkheadConfig config) + public long getTotalAcquired() + { + return totalAcquired.get(); + } + + public long getTotalReleased() + { + return totalReleased.get(); + } + + private List acquireLanes( + Query query + ) { - if (!laneRegistry.bulkhead(TOTAL, config).tryAcquirePermission()) { + final String lane = QueryContexts.getLane(query); + final Optional laneConfig = lane == null ? Optional.empty() : laneRegistry.getConfiguration(lane); + List hallPasses = new ArrayList<>(2); + // everyone needs to take one from the total lane + final Optional totalConfig = laneRegistry.getConfiguration(TOTAL); + totalConfig.ifPresent(config -> hallPasses.add(acquireTotal(config))); + // catch the 2nd so we can release the first + try { + // if we have a lane, also get it + laneConfig.ifPresent(config -> hallPasses.add(acquireLane(lane, config))); + } + catch (QueryCapacityExceededException ex) { + // release total if couldn't get lane + releaseLanes(hallPasses); + throw ex; + } + return hallPasses; + } + + private Bulkhead acquireTotal(BulkheadConfig config) + { + Bulkhead totalLimiter = laneRegistry.bulkhead(TOTAL, config); + if (!totalLimiter.tryAcquirePermission()) { throw new QueryCapacityExceededException(); } + totalAcquired.incrementAndGet(); + return totalLimiter; } - private void acquireLane(String lane, BulkheadConfig config, Optional totalToReleaseIfFailed) + private Bulkhead acquireLane(String lane, BulkheadConfig config) { - if (!laneRegistry.bulkhead(lane, config).tryAcquirePermission()) { - totalToReleaseIfFailed.ifPresent(c -> laneRegistry.bulkhead(TOTAL, c).releasePermission()); + Bulkhead laneLimiter = laneRegistry.bulkhead(lane, config); + if (!laneLimiter.tryAcquirePermission()) { throw new QueryCapacityExceededException(lane); } + return laneLimiter; + } + + private void releaseLanes(List bulkheads) + { + bulkheads.forEach(Bulkhead::releasePermission); + totalReleased.incrementAndGet(); } private Map getLaneConfigs(int totalNumThreads) diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java index b3540fd56814..987aca2e8b7e 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java @@ -333,8 +333,7 @@ public int getMergePoolParallelism() } }, ForkJoinPool.commonPool(), - // need at least 9 total since runner doesn't actually run queries and release semaphores - new QueryScheduler(Integer.MAX_VALUE, NoQueryLaningStrategy.INSTANCE) + new QueryScheduler(0, NoQueryLaningStrategy.INSTANCE) ); } diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index 83d7a415ec80..422cd1efc60b 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -2476,7 +2476,7 @@ public int getMergePoolParallelism() } }, ForkJoinPool.commonPool(), - new QueryScheduler(Integer.MAX_VALUE, NoQueryLaningStrategy.INSTANCE) + new QueryScheduler(0, NoQueryLaningStrategy.INSTANCE) ); } diff --git a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java index 5e4d756cfa43..89e4a172ce0e 100644 --- a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java +++ b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java @@ -22,6 +22,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.ListenableFuture; +import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; @@ -39,30 +40,53 @@ import org.apache.druid.query.topn.TopNQuery; import org.apache.druid.query.topn.TopNQueryBuilder; import org.apache.druid.server.scheduling.HiLoQueryLaningStrategy; +import org.junit.After; import org.junit.Assert; +import org.junit.Before; import org.junit.Rule; import org.junit.Test; import org.junit.rules.ExpectedException; import java.io.IOException; +import java.util.ArrayList; import java.util.Iterator; +import java.util.List; import java.util.UUID; import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.ThreadLocalRandom; public class QuerySchedulerTest { + private static final int NUM_CONCURRENT_QUERIES = 10000; + private static final int NUM_ROWS = 10000; + @Rule public ExpectedException expected = ExpectedException.none(); + private ListeningExecutorService executorService; + private QueryScheduler scheduler; + + @Before + public void setup() + { + executorService = MoreExecutors.listeningDecorator( + Execs.multiThreaded(8, "test_query_scheduler_%s") + ); + scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)); + } + + @After + public void teardown() + { + executorService.shutdownNow(); + } + @Test public void testHiLoHi() throws ExecutionException, InterruptedException { - QueryScheduler scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)); - TopNQuery interactive = makeInteractiveQuery(); - ListenableFuture future = MoreExecutors.listeningDecorator( - Execs.singleThreaded("test_query_scheduler_%s") - ).submit(() -> { + ListenableFuture future = executorService.submit(() -> { try { Query scheduled = scheduler.laneQuery(QueryPlus.wrap(interactive), ImmutableSet.of()); @@ -87,7 +111,6 @@ public void before() throw new RuntimeException(ex); } }); - scheduler.registerQueryFuture(interactive, future); future.get(); Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); } @@ -95,11 +118,8 @@ public void before() @Test public void testHiLoLo() throws ExecutionException, InterruptedException { - QueryScheduler scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)); TopNQuery report = makeReportQuery(); - ListenableFuture future = MoreExecutors.listeningDecorator( - Execs.singleThreaded("test_query_scheduler_%s") - ).submit(() -> { + ListenableFuture future = executorService.submit(() -> { try { Query scheduledReport = scheduler.laneQuery(QueryPlus.wrap(report), ImmutableSet.of()); Assert.assertNotNull(scheduledReport); @@ -124,25 +144,18 @@ public void before() throw new RuntimeException(ex); } }); - scheduler.registerQueryFuture(report, future); future.get(); Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); Assert.assertEquals(2, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); } - @Test public void testHiLoReleaseSemaphoreWhenSequenceExplodes() throws Exception { expected.expectMessage("exploded"); expected.expect(ExecutionException.class); - - QueryScheduler scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)); - TopNQuery interactive = makeInteractiveQuery(); - ListenableFuture future = MoreExecutors.listeningDecorator( - Execs.singleThreaded("test_query_scheduler_%s") - ).submit(() -> { + ListenableFuture future = executorService.submit(() -> { try { Query scheduled = scheduler.laneQuery(QueryPlus.wrap(interactive), ImmutableSet.of()); @@ -165,7 +178,6 @@ public void before() throw new RuntimeException(ex); } }); - scheduler.registerQueryFuture(interactive, future); future.get(); Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); } @@ -178,8 +190,6 @@ public void testHiLoFailsWhenOutOfLaneCapacity() ); expected.expect(QueryCapacityExceededException.class); - QueryScheduler scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)); - Query report1 = scheduler.laneQuery(QueryPlus.wrap(makeReportQuery()), ImmutableSet.of()); scheduler.run(report1, Sequences.empty()); Assert.assertNotNull(report1); @@ -202,7 +212,6 @@ public void testHiLoFailsWhenOutOfTotalCapacity() expected.expectMessage(QueryCapacityExceededException.ERROR_MESSAGE); expected.expect(QueryCapacityExceededException.class); - QueryScheduler scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)); Query interactive1 = scheduler.laneQuery(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()); scheduler.run(interactive1, Sequences.empty()); Assert.assertNotNull(interactive1); @@ -234,6 +243,50 @@ public void testHiLoFailsWhenOutOfTotalCapacity() scheduler.run(scheduler.laneQuery(QueryPlus.wrap(makeInteractiveQuery()), ImmutableSet.of()), Sequences.empty()); } + @Test + public void testConcurrency() throws Exception + { + List> futures = new ArrayList<>(NUM_CONCURRENT_QUERIES); + for (int i = 0; i < NUM_CONCURRENT_QUERIES; i++) { + futures.add(makeQueryFuture(executorService, scheduler, makeRandomQuery(), NUM_ROWS)); + maybeDelayNextIteration(i); + } + getFuturesAndAssertAftermathIsChill(futures, scheduler, false); + } + + @Test + public void testConcurrencyLo() throws Exception + { + List> futures = new ArrayList<>(NUM_CONCURRENT_QUERIES); + for (int i = 0; i < NUM_CONCURRENT_QUERIES; i++) { + futures.add(makeQueryFuture(executorService, scheduler, makeReportQuery(), NUM_ROWS)); + maybeDelayNextIteration(i); + } + getFuturesAndAssertAftermathIsChill(futures, scheduler, false); + } + + @Test + public void testConcurrencyHi() throws Exception + { + List> futures = new ArrayList<>(NUM_CONCURRENT_QUERIES); + for (int i = 0; i < NUM_CONCURRENT_QUERIES; i++) { + futures.add(makeQueryFuture(executorService, scheduler, makeInteractiveQuery(), NUM_ROWS)); + maybeDelayNextIteration(i); + } + getFuturesAndAssertAftermathIsChill(futures, scheduler, true); + } + + private void maybeDelayNextIteration(int i) throws InterruptedException + { + if (i > 0 && i % 10 == 0) { + Thread.sleep(2); + } + } + + private TopNQuery makeRandomQuery() + { + return ThreadLocalRandom.current().nextBoolean() ? makeInteractiveQuery() : makeReportQuery(); + } private TopNQuery makeInteractiveQuery() { @@ -350,4 +403,67 @@ public void cleanup(Iterator iterFromMake) } ); } + + private ListenableFuture makeQueryFuture( + ListeningExecutorService executorService, + QueryScheduler scheduler, + Query query, + int numRows + ) + { + return executorService.submit(() -> { + try { + Query scheduled = scheduler.laneQuery(QueryPlus.wrap(query), ImmutableSet.of()); + + Assert.assertNotNull(scheduled); + + Sequence underlyingSequence = makeSequence(numRows); + Sequence results = scheduler.run(scheduled, underlyingSequence); + + final int actualNumRows = consumeAndCloseSequence(results); + Assert.assertEquals(actualNumRows, numRows); + } + catch (IOException ex) { + throw new RuntimeException(ex); + } + }); + } + + + private void getFuturesAndAssertAftermathIsChill( + List> futures, + QueryScheduler scheduler, + boolean successEqualsTotal + ) + { + int success = 0; + int denied = 0; + int other = 0; + for (Future f : futures) { + try { + f.get(); + success++; + } + catch (ExecutionException ex) { + if (ex.getCause() instanceof QueryCapacityExceededException) { + denied++; + } else { + other++; + } + } + catch (Exception ex) { + other++; + } + } + Assert.assertEquals(0, other); + if (successEqualsTotal) { + Assert.assertEquals(success, scheduler.getTotalAcquired()); + } else { + Assert.assertTrue(success > 0 && success <= scheduler.getTotalAcquired()); + } + Assert.assertTrue(denied > 0); + Assert.assertEquals(scheduler.getTotalAcquired(), scheduler.getTotalReleased()); + Assert.assertEquals(2, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); + Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); + } } From 60861a43fd25bca15aa3215988252bc392b3ad3f Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 2 Mar 2020 22:02:13 -0800 Subject: [PATCH 17/37] add SqlResource tests, refactor error response --- .../apache/druid/query/QueryException.java | 83 +++++++++++ .../query/QueryInterruptedException.java | 48 +----- .../QueryCapacityExceededException.java | 22 ++- .../apache/druid/server/QueryResource.java | 6 +- .../druid/server/QueryResourceTest.java | 22 ++- .../apache/druid/sql/http/SqlResource.java | 3 +- .../druid/sql/calcite/util/CalciteTests.java | 12 +- .../SpecificSegmentsQuerySegmentWalker.java | 54 ++++++- .../druid/sql/http/SqlResourceTest.java | 138 ++++++++++++++++-- 9 files changed, 317 insertions(+), 71 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/query/QueryException.java diff --git a/processing/src/main/java/org/apache/druid/query/QueryException.java b/processing/src/main/java/org/apache/druid/query/QueryException.java new file mode 100644 index 000000000000..16ab6b147508 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/QueryException.java @@ -0,0 +1,83 @@ +/* + * 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.query; + +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; + +import javax.annotation.Nullable; + +/** + * Base serializable error response + */ +public class QueryException extends RuntimeException +{ + private final String errorCode; + private final String errorClass; + private final String host; + + public QueryException(Throwable cause, String errorCode, String errorClass, String host) + { + super(cause == null ? null : cause.getMessage(), cause); + this.errorCode = errorCode; + this.errorClass = errorClass; + this.host = host; + } + + @JsonCreator + public QueryException( + @JsonProperty("error") @Nullable String errorCode, + @JsonProperty("errorMessage") String errorMessage, + @JsonProperty("errorClass") @Nullable String errorClass, + @JsonProperty("host") @Nullable String host + ) + { + super(errorMessage); + this.errorCode = errorCode; + this.errorClass = errorClass; + this.host = host; + } + + @Nullable + @JsonProperty("error") + public String getErrorCode() + { + return errorCode; + } + + @JsonProperty("errorMessage") + @Override + public String getMessage() + { + return super.getMessage(); + } + + @JsonProperty + public String getErrorClass() + { + return errorClass; + } + + @JsonProperty + public String getHost() + { + return host; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/QueryInterruptedException.java b/processing/src/main/java/org/apache/druid/query/QueryInterruptedException.java index bc2ffeb4c325..acb86a64a956 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryInterruptedException.java +++ b/processing/src/main/java/org/apache/druid/query/QueryInterruptedException.java @@ -42,7 +42,7 @@ * The QueryResource is expected to emit the JSON form of this object when errors happen, and the DirectDruidClient * deserializes and wraps them. */ -public class QueryInterruptedException extends RuntimeException +public class QueryInterruptedException extends QueryException { public static final String QUERY_INTERRUPTED = "Query interrupted"; public static final String QUERY_TIMEOUT = "Query timeout"; @@ -52,10 +52,6 @@ public class QueryInterruptedException extends RuntimeException public static final String UNSUPPORTED_OPERATION = "Unsupported operation"; public static final String UNKNOWN_EXCEPTION = "Unknown exception"; - private final String errorCode; - private final String errorClass; - private final String host; - @JsonCreator public QueryInterruptedException( @JsonProperty("error") @Nullable String errorCode, @@ -64,10 +60,7 @@ public QueryInterruptedException( @JsonProperty("host") @Nullable String host ) { - super(errorMessage); - this.errorCode = errorCode; - this.errorClass = errorClass; - this.host = host; + super(errorCode, errorMessage, errorClass, host); } /** @@ -83,36 +76,7 @@ public QueryInterruptedException(Throwable cause) public QueryInterruptedException(Throwable cause, String host) { - super(cause == null ? null : cause.getMessage(), cause); - this.errorCode = getErrorCodeFromThrowable(cause); - this.errorClass = getErrorClassFromThrowable(cause); - this.host = host; - } - - @Nullable - @JsonProperty("error") - public String getErrorCode() - { - return errorCode; - } - - @JsonProperty("errorMessage") - @Override - public String getMessage() - { - return super.getMessage(); - } - - @JsonProperty - public String getErrorClass() - { - return errorClass; - } - - @JsonProperty - public String getHost() - { - return host; + super(cause, getErrorCodeFromThrowable(cause), getErrorClassFromThrowable(cause), host); } @Override @@ -121,9 +85,9 @@ public String toString() return StringUtils.format( "QueryInterruptedException{msg=%s, code=%s, class=%s, host=%s}", getMessage(), - errorCode, - errorClass, - host + getErrorCode(), + getErrorClass(), + getHost() ); } diff --git a/server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java b/server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java index 687e311b0a15..a9320302b9d9 100644 --- a/server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java +++ b/server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java @@ -19,21 +19,35 @@ package org.apache.druid.server; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.fasterxml.jackson.annotation.JsonProperty; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.query.QueryException; -public class QueryCapacityExceededException extends RuntimeException +public class QueryCapacityExceededException extends QueryException { - public static final String ERROR_MESSAGE = "Query capacity exceeded"; + private static final String ERROR_CLASS = QueryCapacityExceededException.class.getName(); + public static final String ERROR_CODE = "Query capacity exceeded"; + public static final String ERROR_MESSAGE = "Total query capacity exceeded"; public static final String ERROR_MESSAGE_TEMPLATE = "Query capacity exceeded for lane %s"; public static final int STATUS_CODE = 429; public QueryCapacityExceededException() { - super(ERROR_MESSAGE); + super(ERROR_CODE, ERROR_MESSAGE, ERROR_CLASS, null); } public QueryCapacityExceededException(String lane) { - super(StringUtils.format(ERROR_MESSAGE_TEMPLATE, lane)); + super(ERROR_CODE, StringUtils.format(ERROR_MESSAGE_TEMPLATE, lane), ERROR_CLASS, null); + } + + @JsonCreator + public QueryCapacityExceededException( + @JsonProperty("error") String errorCode, + @JsonProperty("errorMessage") String errorMessage, + @JsonProperty("errorClass") String errorClass) + { + super(errorCode, errorMessage, errorClass, null); } } diff --git a/server/src/main/java/org/apache/druid/server/QueryResource.java b/server/src/main/java/org/apache/druid/server/QueryResource.java index 2f17284e288a..d270799b62af 100644 --- a/server/src/main/java/org/apache/druid/server/QueryResource.java +++ b/server/src/main/java/org/apache/druid/server/QueryResource.java @@ -440,9 +440,11 @@ Response gotError(Exception e) throws IOException .build(); } - Response gotLimited(QueryCapacityExceededException e) + Response gotLimited(QueryCapacityExceededException e) throws IOException { - return Response.status(QueryCapacityExceededException.STATUS_CODE).entity(e.getMessage()).build(); + return Response.status(QueryCapacityExceededException.STATUS_CODE) + .entity(newOutputWriter(null, null, false).writeValueAsBytes(e)) + .build(); } } diff --git a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java index 7ca4bda246e5..5233a443f35e 100644 --- a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java @@ -695,7 +695,15 @@ public void testTooManyQuery() throws InterruptedException waitAllFinished, response -> { Assert.assertEquals(QueryCapacityExceededException.STATUS_CODE, response.getStatus()); - Assert.assertEquals(QueryCapacityExceededException.ERROR_MESSAGE, response.getEntity()); + QueryCapacityExceededException ex; + try { + ex = JSON_MAPPER.readValue((byte[]) response.getEntity(), QueryCapacityExceededException.class); + } + catch (IOException e) { + throw new RuntimeException(e); + } + Assert.assertEquals(QueryCapacityExceededException.ERROR_MESSAGE, ex.getMessage()); + Assert.assertEquals(QueryCapacityExceededException.ERROR_CODE, ex.getErrorCode()); } ); waitAllFinished.await(); @@ -723,12 +731,22 @@ public void testTooManyQueryInLane() throws InterruptedException waitAllFinished, response -> { Assert.assertEquals(QueryCapacityExceededException.STATUS_CODE, response.getStatus()); + QueryCapacityExceededException ex; + try { + ex = JSON_MAPPER.readValue((byte[]) response.getEntity(), QueryCapacityExceededException.class); + } + catch (IOException e) { + throw new RuntimeException(e); + } Assert.assertEquals( StringUtils.format( QueryCapacityExceededException.ERROR_MESSAGE_TEMPLATE, HiLoQueryLaningStrategy.LOW ), - response.getEntity()); + ex.getMessage() + ); + Assert.assertEquals(QueryCapacityExceededException.ERROR_CODE, ex.getErrorCode()); + } ); waitTwoStarted.await(); diff --git a/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java b/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java index d96a5ee85e33..c09aa858da6a 100644 --- a/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java +++ b/sql/src/main/java/org/apache/druid/sql/http/SqlResource.java @@ -173,7 +173,8 @@ public Response doPost( } } catch (QueryCapacityExceededException cap) { - return Response.status(429).entity(cap.getMessage()).build(); + lifecycle.emitLogsAndMetrics(cap, remoteAddr, -1); + return Response.status(QueryCapacityExceededException.STATUS_CODE).entity(jsonMapper.writeValueAsBytes(cap)).build(); } catch (ForbiddenException e) { throw e; // let ForbiddenExceptionMapper handle this diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index 132df255d689..fd2532ecddc1 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -106,6 +106,7 @@ import org.apache.druid.segment.writeout.OffHeapMemorySegmentWriteOutMediumFactory; import org.apache.druid.server.DruidNode; import org.apache.druid.server.QueryLifecycleFactory; +import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.coordinator.BytesAccumulatingResponseHandler; import org.apache.druid.server.log.NoopRequestLogger; import org.apache.druid.server.security.Access; @@ -700,6 +701,14 @@ public static SpecificSegmentsQuerySegmentWalker createMockWalker( final QueryRunnerFactoryConglomerate conglomerate, final File tmpDir ) + { + return createMockWalker(conglomerate, tmpDir, null); + } + public static SpecificSegmentsQuerySegmentWalker createMockWalker( + final QueryRunnerFactoryConglomerate conglomerate, + final File tmpDir, + @Nullable final QueryScheduler scheduler + ) { final QueryableIndex index1 = IndexBuilder .create() @@ -753,7 +762,8 @@ public static SpecificSegmentsQuerySegmentWalker createMockWalker( return new SpecificSegmentsQuerySegmentWalker( conglomerate, INJECTOR.getInstance(LookupExtractorFactoryContainerProvider.class), - null + null, + scheduler ).add( DataSegment.builder() .dataSource(DATASOURCE1) diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java index 46f8dad9a231..cc900cb40d0e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/SpecificSegmentsQuerySegmentWalker.java @@ -25,10 +25,12 @@ import com.google.common.collect.Lists; import com.google.common.collect.Ordering; import com.google.common.io.Closeables; +import org.apache.druid.client.SegmentServerSelector; import org.apache.druid.client.cache.CacheConfig; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.FunctionalIterable; +import org.apache.druid.java.util.common.guava.LazySequence; import org.apache.druid.query.DataSource; import org.apache.druid.query.FinalizeResultsQueryRunner; import org.apache.druid.query.InlineDataSource; @@ -63,6 +65,7 @@ import org.apache.druid.segment.join.LookupJoinableFactory; import org.apache.druid.segment.join.MapJoinableFactoryTest; import org.apache.druid.server.ClientQuerySegmentWalker; +import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.timeline.DataSegment; @@ -78,6 +81,7 @@ import java.util.ArrayList; import java.util.Collections; import java.util.HashMap; +import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Optional; @@ -98,18 +102,21 @@ public class SpecificSegmentsQuerySegmentWalker implements QuerySegmentWalker, C private final QueryRunnerFactoryConglomerate conglomerate; private final QuerySegmentWalker walker; private final JoinableFactory joinableFactory; + private final QueryScheduler scheduler; private final Map> timelines = new HashMap<>(); private final List closeables = new ArrayList<>(); private final List segments = new ArrayList<>(); /** * Create an instance using the provided query runner factory conglomerate and lookup provider. - * If a JoinableFactory is provided, it will be used instead of the default. + * If a JoinableFactory is provided, it will be used instead of the default. If a scheduler is included, + * the runner will schedule queries according to the scheduling config. */ public SpecificSegmentsQuerySegmentWalker( final QueryRunnerFactoryConglomerate conglomerate, final LookupExtractorFactoryContainerProvider lookupProvider, - @Nullable final JoinableFactory joinableFactory + @Nullable final JoinableFactory joinableFactory, + @Nullable final QueryScheduler scheduler ) { this.conglomerate = conglomerate; @@ -121,6 +128,7 @@ public SpecificSegmentsQuerySegmentWalker( .build() ) : joinableFactory; + this.scheduler = scheduler; this.walker = new ClientQuerySegmentWalker( new NoopServiceEmitter(), new DataServerLikeWalker(), @@ -165,6 +173,20 @@ public boolean isUseResultLevelCache() ); } + /** + * Create an instance using the provided query runner factory conglomerate and lookup provider. + * If a JoinableFactory is provided, it will be used instead of the default. + */ + public SpecificSegmentsQuerySegmentWalker( + final QueryRunnerFactoryConglomerate conglomerate, + final LookupExtractorFactoryContainerProvider lookupProvider, + @Nullable final JoinableFactory joinableFactory + ) + { + this(conglomerate, lookupProvider, joinableFactory, null); + } + + /** * Create an instance without any lookups, using the default JoinableFactory */ @@ -388,13 +410,33 @@ public QueryRunner getQueryRunnerForSegments(final Query query, final toolChest ); + // Wrap baseRunner in a runner that rewrites the QuerySegmentSpec to mention the specific segments. // This mimics what CachingClusteredClient on the Broker does, and is required for certain queries (like Scan) // to function properly. - return (theQuery, responseContext) -> baseRunner.run( - theQuery.withQuery(Queries.withSpecificSegments(theQuery.getQuery(), ImmutableList.copyOf(specs))), - responseContext - ); + return (theQuery, responseContext) -> { + if (scheduler != null) { + Set segments = new HashSet<>(); + specs.forEach(spec -> segments.add(new SegmentServerSelector(null, spec))); + return scheduler.run( + scheduler.laneQuery(theQuery, segments), + new LazySequence<>( + () -> baseRunner.run( + theQuery.withQuery(Queries.withSpecificSegments( + theQuery.getQuery(), + ImmutableList.copyOf(specs) + )), + responseContext + ) + ) + ); + } else { + return baseRunner.run( + theQuery.withQuery(Queries.withSpecificSegments(theQuery.getQuery(), ImmutableList.copyOf(specs))), + responseContext + ); + } + }; } private QueryRunner makeTableRunner( diff --git a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java index ce925902a575..dc452bbee94b 100644 --- a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java +++ b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java @@ -25,6 +25,8 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.Maps; +import com.google.common.util.concurrent.ListeningExecutorService; +import com.google.common.util.concurrent.MoreExecutors; import org.apache.calcite.avatica.SqlType; import org.apache.calcite.schema.SchemaPlus; import org.apache.calcite.tools.ValidationException; @@ -33,13 +35,18 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; +import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.math.expr.ExprMacroTable; +import org.apache.druid.query.QueryException; import org.apache.druid.query.QueryInterruptedException; import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.ResourceLimitExceededException; +import org.apache.druid.server.QueryCapacityExceededException; +import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.log.TestRequestLogger; import org.apache.druid.server.metrics.NoopServiceEmitter; +import org.apache.druid.server.scheduling.HiLoQueryLaningStrategy; import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.ForbiddenException; import org.apache.druid.sql.SqlLifecycleFactory; @@ -67,9 +74,11 @@ import java.io.ByteArrayOutputStream; import java.io.IOException; import java.nio.charset.StandardCharsets; +import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Map; +import java.util.concurrent.Future; import java.util.function.Function; import java.util.stream.Collectors; @@ -85,9 +94,11 @@ public class SqlResourceTest extends CalciteTestBase @Rule public QueryLogHook queryLogHook = QueryLogHook.create(); private SpecificSegmentsQuerySegmentWalker walker = null; + private QueryScheduler scheduler = null; private TestRequestLogger testRequestLogger; private SqlResource resource; private HttpServletRequest req; + private ListeningExecutorService executorService; @BeforeClass public static void setUpClass() @@ -107,7 +118,11 @@ public static void tearDownClass() throws IOException @Before public void setUp() throws Exception { - walker = CalciteTests.createMockWalker(conglomerate, temporaryFolder.newFolder()); + executorService = MoreExecutors.listeningDecorator( + Execs.multiThreaded(8, "test_sql_resource_%s") + ); + scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)); + walker = CalciteTests.createMockWalker(conglomerate, temporaryFolder.newFolder(), scheduler); final PlannerConfig plannerConfig = new PlannerConfig() { @@ -167,6 +182,7 @@ public void tearDown() throws Exception { walker.close(); walker = null; + executorService.shutdownNow(); } @Test @@ -638,7 +654,7 @@ public void testExplainCountStar() throws Exception @Test public void testCannotValidate() throws Exception { - final QueryInterruptedException exception = doPost( + final QueryException exception = doPost( new SqlQuery( "SELECT dim4 FROM druid.foo", ResultFormat.OBJECT, @@ -659,7 +675,7 @@ public void testCannotValidate() throws Exception public void testCannotConvert() throws Exception { // SELECT + ORDER unsupported - final QueryInterruptedException exception = doPost( + final QueryException exception = doPost( new SqlQuery("SELECT dim1 FROM druid.foo ORDER BY dim1", ResultFormat.OBJECT, false, null, null) ).lhs; @@ -676,7 +692,7 @@ public void testCannotConvert() throws Exception @Test public void testResourceLimitExceeded() throws Exception { - final QueryInterruptedException exception = doPost( + final QueryException exception = doPost( new SqlQuery( "SELECT DISTINCT dim1 FROM foo", ResultFormat.OBJECT, @@ -692,6 +708,56 @@ public void testResourceLimitExceeded() throws Exception checkSqlRequestLog(false); } + @Test + public void testTooManyRequests() throws Exception + { + final int numQueries = 3; + + List>>>> futures = new ArrayList<>(numQueries); + for (int i = 0; i < numQueries; i++) { + futures.add(executorService.submit(() -> { + try { + return doPost( + new SqlQuery( + "SELECT COUNT(*) AS cnt, 'foo' AS TheFoo FROM druid.foo", + null, + false, + ImmutableMap.of("priority", -5), + null + ), + makeExpectedReq() + ); + } + catch (Exception e) { + throw new RuntimeException(e); + } + })); + } + + + int success = 0; + int limited = 0; + for (int i = 0; i < numQueries; i++) { + Pair>> result = futures.get(i).get(); + List> rows = result.rhs; + if (rows != null) { + Assert.assertEquals(ImmutableList.of(ImmutableMap.of("cnt", 6, "TheFoo", "foo")), rows); + success++; + } else { + QueryException interruped = result.lhs; + Assert.assertEquals(QueryCapacityExceededException.ERROR_CODE, interruped.getErrorCode()); + Assert.assertEquals( + StringUtils.format(QueryCapacityExceededException.ERROR_MESSAGE_TEMPLATE, HiLoQueryLaningStrategy.LOW), + interruped.getMessage() + ); + limited++; + } + } + Assert.assertEquals(2, success); + Assert.assertEquals(1, limited); + Assert.assertEquals(3, testRequestLogger.getSqlQueryLogs().size()); + } + @SuppressWarnings("unchecked") private void checkSqlRequestLog(boolean success) { @@ -710,23 +776,53 @@ private void checkSqlRequestLog(boolean success) } } + + private Pair>> doPost(final SqlQuery query) throws Exception + { + return doPost(query, new TypeReference>>() + { + }); + } + + // Returns either an error or a result, assuming the result is a JSON object. + private Pair doPost( + final SqlQuery query, + final TypeReference typeReference + ) throws Exception + { + return doPost(query, req, typeReference); + } + + private Pair doPostRaw(final SqlQuery query) throws Exception + { + return doPostRaw(query, req); + } + + private Pair>> doPost(final SqlQuery query, HttpServletRequest req) throws Exception + { + return doPost(query, req, new TypeReference>>() + { + }); + } + // Returns either an error or a result, assuming the result is a JSON object. - private Pair doPost( + private Pair doPost( final SqlQuery query, + final HttpServletRequest req, final TypeReference typeReference ) throws Exception { - final Pair pair = doPostRaw(query); + final Pair pair = doPostRaw(query, req); if (pair.rhs == null) { //noinspection unchecked - return (Pair) pair; + return (Pair) pair; } else { return Pair.of(pair.lhs, JSON_MAPPER.readValue(pair.rhs, typeReference)); } } // Returns either an error or a result. - private Pair doPostRaw(final SqlQuery query) throws Exception + private Pair doPostRaw(final SqlQuery query, final HttpServletRequest req) throws Exception { final Response response = resource.doPost(query, req); if (response.getStatus() == 200) { @@ -739,16 +835,32 @@ private Pair doPostRaw(final SqlQuery query) ); } else { return Pair.of( - JSON_MAPPER.readValue((byte[]) response.getEntity(), QueryInterruptedException.class), + JSON_MAPPER.readValue((byte[]) response.getEntity(), QueryException.class), null ); } } - private Pair>> doPost(final SqlQuery query) throws Exception + private HttpServletRequest makeExpectedReq() { - return doPost(query, new TypeReference>>() - { - }); + HttpServletRequest req = EasyMock.createStrictMock(HttpServletRequest.class); + EasyMock.expect(req.getRemoteAddr()).andReturn(null).once(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) + .andReturn(CalciteTests.REGULAR_USER_AUTH_RESULT) + .anyTimes(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_ALLOW_UNSECURED_PATH)).andReturn(null).anyTimes(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED)) + .andReturn(null) + .anyTimes(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) + .andReturn(CalciteTests.REGULAR_USER_AUTH_RESULT) + .anyTimes(); + req.setAttribute(AuthConfig.DRUID_AUTHORIZATION_CHECKED, true); + EasyMock.expectLastCall().anyTimes(); + EasyMock.expect(req.getAttribute(AuthConfig.DRUID_AUTHENTICATION_RESULT)) + .andReturn(CalciteTests.REGULAR_USER_AUTH_RESULT) + .anyTimes(); + EasyMock.replay(req); + return req; } } From 9aed16e6ca23d413d30554ebfd8157210849e67b Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 3 Mar 2020 17:13:23 -0800 Subject: [PATCH 18/37] add json config tests --- .../druid/server/QuerySchedulerTest.java | 62 +++++++++++++++++++ 1 file changed, 62 insertions(+) diff --git a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java index 89e4a172ce0e..3c7a0c50612f 100644 --- a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java +++ b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java @@ -19,11 +19,21 @@ package org.apache.druid.server; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; +import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.Module; +import com.google.inject.name.Names; +import org.apache.druid.guice.GuiceInjectors; +import org.apache.druid.guice.JsonConfigProvider; +import org.apache.druid.guice.JsonConfigurator; +import org.apache.druid.guice.annotations.Global; +import org.apache.druid.initialization.Initialization; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.BaseSequence; @@ -51,6 +61,7 @@ import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import java.util.Properties; import java.util.UUID; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; @@ -276,6 +287,42 @@ public void testConcurrencyHi() throws Exception getFuturesAndAssertAftermathIsChill(futures, scheduler, true); } + @Test + public void testConfigNone() + { + final Injector injector = createInjector(); + final String propertyPrefix = "druid.query.scheduler"; + final JsonConfigProvider provider = JsonConfigProvider.of( + propertyPrefix, + QuerySchedulerProvider.class + ); + final Properties properties = new Properties(); + properties.put(propertyPrefix + ".numThreads", "10"); + provider.inject(properties, injector.getInstance(JsonConfigurator.class)); + final QueryScheduler scheduler = provider.get().get().get(); + Assert.assertEquals(10, scheduler.getTotalAvailableCapacity()); + Assert.assertEquals(-1, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); + } + + @Test + public void testConfigHiLo() + { + final Injector injector = createInjector(); + final String propertyPrefix = "druid.query.scheduler"; + final JsonConfigProvider provider = JsonConfigProvider.of( + propertyPrefix, + QuerySchedulerProvider.class + ); + final Properties properties = new Properties(); + properties.put(propertyPrefix + ".numThreads", "10"); + properties.put(propertyPrefix + ".laning.type", "hilo"); + properties.put(propertyPrefix + ".laning.maxLowThreads", "2"); + provider.inject(properties, injector.getInstance(JsonConfigurator.class)); + final QueryScheduler scheduler = provider.get().get().get(); + Assert.assertEquals(10, scheduler.getTotalAvailableCapacity()); + Assert.assertEquals(2, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); + } + private void maybeDelayNextIteration(int i) throws InterruptedException { if (i > 0 && i % 10 == 0) { @@ -466,4 +513,19 @@ private void getFuturesAndAssertAftermathIsChill( Assert.assertEquals(2, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); } + + private Injector createInjector() + { + return Initialization.makeInjectorWithModules( + GuiceInjectors.makeStartupInjector(), + ImmutableList.of( + binder -> { + binder.bind(Key.get(String.class, Names.named("serviceName"))).toInstance("some service"); + binder.bind(Key.get(Integer.class, Names.named("servicePort"))).toInstance(0); + binder.bind(Key.get(Integer.class, Names.named("tlsServicePort"))).toInstance(-1); + JsonConfigProvider.bind(binder, "druid.query.scheduler", QuerySchedulerProvider.class, Global.class); + } + ) + ); + } } From 419ab98ef48eb0a7590bb064393c424a553d3544 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 3 Mar 2020 17:19:13 -0800 Subject: [PATCH 19/37] use LongAdder instead of AtomicLong --- .../org/apache/druid/server/QueryScheduler.java | 14 +++++++------- 1 file changed, 7 insertions(+), 7 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/QueryScheduler.java b/server/src/main/java/org/apache/druid/server/QueryScheduler.java index a4ebd6662ccc..1a5b46f625a6 100644 --- a/server/src/main/java/org/apache/druid/server/QueryScheduler.java +++ b/server/src/main/java/org/apache/druid/server/QueryScheduler.java @@ -42,7 +42,7 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.LongAdder; /** @@ -60,8 +60,8 @@ public class QueryScheduler implements QueryWatcher private final SetMultimap> queryFutures; private final SetMultimap queryDatasources; - private final AtomicLong totalAcquired = new AtomicLong(); - private final AtomicLong totalReleased = new AtomicLong(); + private final LongAdder totalAcquired = new LongAdder(); + private final LongAdder totalReleased = new LongAdder(); public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy) { @@ -148,12 +148,12 @@ public int getLaneAvailableCapacity(String lane) public long getTotalAcquired() { - return totalAcquired.get(); + return totalAcquired.longValue(); } public long getTotalReleased() { - return totalReleased.get(); + return totalReleased.longValue(); } private List acquireLanes( @@ -185,7 +185,7 @@ private Bulkhead acquireTotal(BulkheadConfig config) if (!totalLimiter.tryAcquirePermission()) { throw new QueryCapacityExceededException(); } - totalAcquired.incrementAndGet(); + totalAcquired.increment(); return totalLimiter; } @@ -201,7 +201,7 @@ private Bulkhead acquireLane(String lane, BulkheadConfig config) private void releaseLanes(List bulkheads) { bulkheads.forEach(Bulkhead::releasePermission); - totalReleased.incrementAndGet(); + totalReleased.increment(); } private Map getLaneConfigs(int totalNumThreads) From f0d39e1dd4108c9ee803fd6335c515813a892d14 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Tue, 3 Mar 2020 18:04:05 -0800 Subject: [PATCH 20/37] remove test only stuffs from scheduler --- .../apache/druid/server/QueryScheduler.java | 23 +++---------- .../druid/server/QuerySchedulerTest.java | 32 ++++++++++++++++--- 2 files changed, 33 insertions(+), 22 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/QueryScheduler.java b/server/src/main/java/org/apache/druid/server/QueryScheduler.java index 1a5b46f625a6..9c19bb4f4ed6 100644 --- a/server/src/main/java/org/apache/druid/server/QueryScheduler.java +++ b/server/src/main/java/org/apache/druid/server/QueryScheduler.java @@ -19,6 +19,7 @@ package org.apache.druid.server; +import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.HashMultimap; import com.google.common.collect.Multimaps; import com.google.common.collect.SetMultimap; @@ -42,7 +43,6 @@ import java.util.Map; import java.util.Optional; import java.util.Set; -import java.util.concurrent.atomic.LongAdder; /** @@ -60,9 +60,6 @@ public class QueryScheduler implements QueryWatcher private final SetMultimap> queryFutures; private final SetMultimap queryDatasources; - private final LongAdder totalAcquired = new LongAdder(); - private final LongAdder totalReleased = new LongAdder(); - public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy) { this.laningStrategy = laningStrategy; @@ -146,16 +143,6 @@ public int getLaneAvailableCapacity(String lane) .orElse(-1); } - public long getTotalAcquired() - { - return totalAcquired.longValue(); - } - - public long getTotalReleased() - { - return totalReleased.longValue(); - } - private List acquireLanes( Query query ) @@ -179,13 +166,13 @@ private List acquireLanes( return hallPasses; } - private Bulkhead acquireTotal(BulkheadConfig config) + @VisibleForTesting + protected Bulkhead acquireTotal(BulkheadConfig config) { Bulkhead totalLimiter = laneRegistry.bulkhead(TOTAL, config); if (!totalLimiter.tryAcquirePermission()) { throw new QueryCapacityExceededException(); } - totalAcquired.increment(); return totalLimiter; } @@ -198,10 +185,10 @@ private Bulkhead acquireLane(String lane, BulkheadConfig config) return laneLimiter; } - private void releaseLanes(List bulkheads) + @VisibleForTesting + protected void releaseLanes(List bulkheads) { bulkheads.forEach(Bulkhead::releasePermission); - totalReleased.increment(); } private Map getLaneConfigs(int totalNumThreads) diff --git a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java index 3c7a0c50612f..ac3ff1a62a78 100644 --- a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java +++ b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java @@ -29,6 +29,8 @@ import com.google.inject.Key; import com.google.inject.Module; import com.google.inject.name.Names; +import io.github.resilience4j.bulkhead.Bulkhead; +import io.github.resilience4j.bulkhead.BulkheadConfig; import org.apache.druid.guice.GuiceInjectors; import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.JsonConfigurator; @@ -66,6 +68,7 @@ import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicLong; public class QuerySchedulerTest { @@ -78,13 +81,34 @@ public class QuerySchedulerTest private ListeningExecutorService executorService; private QueryScheduler scheduler; + private AtomicLong totalAcquired; + private AtomicLong totalReleased; @Before public void setup() { executorService = MoreExecutors.listeningDecorator( Execs.multiThreaded(8, "test_query_scheduler_%s") ); - scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)); + totalAcquired = new AtomicLong(); + totalReleased = new AtomicLong(); + scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)){ + @Override + protected Bulkhead acquireTotal(BulkheadConfig config) + { + Bulkhead b = super.acquireTotal(config); + totalAcquired.incrementAndGet(); + return b; + } + + @Override + protected void releaseLanes(List bulkheads) + { + super.releaseLanes(bulkheads); + if (bulkheads.size() > 0) { + totalReleased.incrementAndGet(); + } + } + }; } @After @@ -504,12 +528,12 @@ private void getFuturesAndAssertAftermathIsChill( } Assert.assertEquals(0, other); if (successEqualsTotal) { - Assert.assertEquals(success, scheduler.getTotalAcquired()); + Assert.assertEquals(success, totalAcquired.get()); } else { - Assert.assertTrue(success > 0 && success <= scheduler.getTotalAcquired()); + Assert.assertTrue(success > 0 && success <= totalAcquired.get()); } Assert.assertTrue(denied > 0); - Assert.assertEquals(scheduler.getTotalAcquired(), scheduler.getTotalReleased()); + Assert.assertEquals(totalReleased.get(), totalAcquired.get()); Assert.assertEquals(2, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); } From 2afaaf1150b7808e8543e87eecf45ecd9bf8b9fe Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 4 Mar 2020 03:45:53 -0800 Subject: [PATCH 21/37] javadocs, etc --- .../druid/server/QueryLaningStrategy.java | 6 ++ .../apache/druid/server/QueryScheduler.java | 91 +++++++++++-------- .../druid/server/QuerySchedulerConfig.java | 2 - .../scheduling/HiLoQueryLaningStrategy.java | 3 + .../scheduling/NoQueryLaningStrategy.java | 3 + .../druid/server/QuerySchedulerTest.java | 45 +++++---- 6 files changed, 93 insertions(+), 57 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java index d0245063f232..9a70a902db54 100644 --- a/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java @@ -38,7 +38,13 @@ }) public interface QueryLaningStrategy { + /** + * Provide map of lane names to the limit on the number of concurrent queries for that lane + */ Object2IntMap getLaneLimits(); + /** + * For a given {@link QueryPlus} and set of {@link SegmentServerSelector}, compute if a query belongs to a lane + */ Optional computeLane(QueryPlus query, Set segments); } diff --git a/server/src/main/java/org/apache/druid/server/QueryScheduler.java b/server/src/main/java/org/apache/druid/server/QueryScheduler.java index 9c19bb4f4ed6..38ac3625a621 100644 --- a/server/src/main/java/org/apache/druid/server/QueryScheduler.java +++ b/server/src/main/java/org/apache/druid/server/QueryScheduler.java @@ -44,19 +44,19 @@ import java.util.Optional; import java.util.Set; - /** * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}. * * As a {@link QueryWatcher}, it also provides cancellation facilities. + * + * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe. */ public class QueryScheduler implements QueryWatcher { - private static final String TOTAL = "default"; + static final String TOTAL = "default"; private final QueryLaningStrategy laningStrategy; private final BulkheadRegistry laneRegistry; - private final SetMultimap> queryFutures; private final SetMultimap queryDatasources; @@ -102,6 +102,14 @@ public Query laneQuery(QueryPlus queryPlus, Set /** * Run a query with the scheduler, attempting to acquire a semaphore from the total and lane specific query capacities + * + * Note that {@link #cancelQuery} should not interrupt the thread that calls run, in all current usages it only + * cancels any {@link ListenableFuture} created downstream. If this ever commonly changes, we should add + * synchronization between {@link #cancelQuery} and the acquisition of the {@link Bulkhead} to continue to ensure that + * anything acquired is also released. + * + * In the meantime, if a {@link ListenableFuture} is registered for the query that calls this method, it MUST handle + * this synchronization itself to ensure that no {@link Bulkhead} is acquired without releasing it. */ public Sequence run(Query query, Sequence resultSequence) { @@ -124,11 +132,18 @@ public boolean cancelQuery(String id) return success; } + /** + * Get a {@link Set} of datasource names for a {@link Query} id, used by {@link QueryResource#cancelQuery} to + * authorize that a user may call {@link #cancelQuery} for the given id and datasources + */ public Set getQueryDatasources(final String queryId) { return queryDatasources.get(queryId); } + /** + * Get the maximum number of concurrent queries that {@link #run} can support + */ public int getTotalAvailableCapacity() { return laneRegistry.getConfiguration(TOTAL) @@ -136,6 +151,9 @@ public int getTotalAvailableCapacity() .orElse(-1); } + /** + * Get the maximum number of concurrent queries that {@link #run} can support for a given lane + */ public int getLaneAvailableCapacity(String lane) { return laneRegistry.getConfiguration(lane) @@ -143,54 +161,51 @@ public int getLaneAvailableCapacity(String lane) .orElse(-1); } - private List acquireLanes( - Query query - ) + /** + * Acquire a semaphore for both the 'total' and a lane, if any is associated with a query + */ + @VisibleForTesting + List acquireLanes(Query query) { final String lane = QueryContexts.getLane(query); final Optional laneConfig = lane == null ? Optional.empty() : laneRegistry.getConfiguration(lane); List hallPasses = new ArrayList<>(2); - // everyone needs to take one from the total lane final Optional totalConfig = laneRegistry.getConfiguration(TOTAL); - totalConfig.ifPresent(config -> hallPasses.add(acquireTotal(config))); - // catch the 2nd so we can release the first - try { - // if we have a lane, also get it - laneConfig.ifPresent(config -> hallPasses.add(acquireLane(lane, config))); - } - catch (QueryCapacityExceededException ex) { - // release total if couldn't get lane - releaseLanes(hallPasses); - throw ex; - } + // if we have a lane, get it first + laneConfig.ifPresent(config -> { + Bulkhead laneLimiter = laneRegistry.bulkhead(lane, config); + if (!laneLimiter.tryAcquirePermission()) { + throw new QueryCapacityExceededException(lane); + } + hallPasses.add(laneLimiter); + }); + + // everyone needs to take one from the total lane; to ensure we don't acquire a lane and never release it, we want + // to check for total capacity exceeded and release the lane (if present) before throwing capacity exceeded + totalConfig.ifPresent(config -> { + Bulkhead totalLimiter = laneRegistry.bulkhead(TOTAL, config); + if (!totalLimiter.tryAcquirePermission()) { + releaseLanes(hallPasses); + throw new QueryCapacityExceededException(); + } + hallPasses.add(totalLimiter); + }); return hallPasses; } + /** + * Release all {@link Bulkhead} semaphores in the list + */ @VisibleForTesting - protected Bulkhead acquireTotal(BulkheadConfig config) - { - Bulkhead totalLimiter = laneRegistry.bulkhead(TOTAL, config); - if (!totalLimiter.tryAcquirePermission()) { - throw new QueryCapacityExceededException(); - } - return totalLimiter; - } - - private Bulkhead acquireLane(String lane, BulkheadConfig config) - { - Bulkhead laneLimiter = laneRegistry.bulkhead(lane, config); - if (!laneLimiter.tryAcquirePermission()) { - throw new QueryCapacityExceededException(lane); - } - return laneLimiter; - } - - @VisibleForTesting - protected void releaseLanes(List bulkheads) + void releaseLanes(List bulkheads) { bulkheads.forEach(Bulkhead::releasePermission); } + /** + * With a total thread count and {@link QueryLaningStrategy#getLaneLimits}, create a map of lane name to + * {@link BulkheadConfig} to be used to create the {@link #laneRegistry} + */ private Map getLaneConfigs(int totalNumThreads) { Map configs = new HashMap<>(); diff --git a/server/src/main/java/org/apache/druid/server/QuerySchedulerConfig.java b/server/src/main/java/org/apache/druid/server/QuerySchedulerConfig.java index a787da88ea6b..350d34ae01da 100644 --- a/server/src/main/java/org/apache/druid/server/QuerySchedulerConfig.java +++ b/server/src/main/java/org/apache/druid/server/QuerySchedulerConfig.java @@ -30,13 +30,11 @@ public class QuerySchedulerConfig @JsonProperty("laning") private QueryLaningStrategy laningStrategy = NoQueryLaningStrategy.INSTANCE; - @JsonProperty public int getNumThreads() { return numThreads; } - @JsonProperty("laning") public QueryLaningStrategy getLaningStrategy() { return laningStrategy; diff --git a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java index fdefb68193ef..1964d7893c3d 100644 --- a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java @@ -33,6 +33,9 @@ import java.util.Optional; import java.util.Set; +/** + * Query laning strategy which associates all {@link Query} with priority lower than 0 into a 'low' lane + */ public class HiLoQueryLaningStrategy implements QueryLaningStrategy { public static final String LOW = "low"; diff --git a/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java index 730d049e2875..5df6563757cb 100644 --- a/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java @@ -28,6 +28,9 @@ import java.util.Optional; import java.util.Set; +/** + * Query laning strategy that does nothing and provides the default, unlimited behavior + */ public class NoQueryLaningStrategy implements QueryLaningStrategy { private static final Object2IntMap NONE = new Object2IntArrayMap<>(); diff --git a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java index ac3ff1a62a78..5521844e1e52 100644 --- a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java +++ b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java @@ -26,16 +26,11 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Injector; -import com.google.inject.Key; -import com.google.inject.Module; -import com.google.inject.name.Names; import io.github.resilience4j.bulkhead.Bulkhead; -import io.github.resilience4j.bulkhead.BulkheadConfig; import org.apache.druid.guice.GuiceInjectors; import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.JsonConfigurator; import org.apache.druid.guice.annotations.Global; -import org.apache.druid.initialization.Initialization; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.BaseSequence; @@ -83,6 +78,10 @@ public class QuerySchedulerTest private AtomicLong totalAcquired; private AtomicLong totalReleased; + private AtomicLong laneAcquired; + private AtomicLong laneNotAcquired; + private AtomicLong laneReleased; + @Before public void setup() { @@ -91,22 +90,37 @@ public void setup() ); totalAcquired = new AtomicLong(); totalReleased = new AtomicLong(); + laneAcquired = new AtomicLong(); + laneNotAcquired = new AtomicLong(); + laneReleased = new AtomicLong(); scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)){ @Override - protected Bulkhead acquireTotal(BulkheadConfig config) + List acquireLanes(Query query) { - Bulkhead b = super.acquireTotal(config); - totalAcquired.incrementAndGet(); - return b; + List bulkheads = super.acquireLanes(query); + if (bulkheads.stream().anyMatch(b -> b.getName().equals(QueryScheduler.TOTAL))) { + totalAcquired.incrementAndGet(); + } + if (bulkheads.stream().anyMatch(b -> !b.getName().equals(QueryScheduler.TOTAL))) { + laneAcquired.incrementAndGet(); + } + + return bulkheads; } @Override - protected void releaseLanes(List bulkheads) + void releaseLanes(List bulkheads) { super.releaseLanes(bulkheads); - if (bulkheads.size() > 0) { + if (bulkheads.stream().anyMatch(b -> b.getName().equals(QueryScheduler.TOTAL))) { totalReleased.incrementAndGet(); } + if (bulkheads.stream().anyMatch(b -> !b.getName().equals(QueryScheduler.TOTAL))) { + laneReleased.incrementAndGet(); + if (bulkheads.size() == 1) { + laneNotAcquired.incrementAndGet(); + } + } } }; } @@ -534,19 +548,16 @@ private void getFuturesAndAssertAftermathIsChill( } Assert.assertTrue(denied > 0); Assert.assertEquals(totalReleased.get(), totalAcquired.get()); + Assert.assertEquals(laneReleased.get(), laneAcquired.get() + laneNotAcquired.get()); Assert.assertEquals(2, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); } private Injector createInjector() { - return Initialization.makeInjectorWithModules( - GuiceInjectors.makeStartupInjector(), - ImmutableList.of( + return GuiceInjectors.makeStartupInjectorWithModules( + ImmutableList.of( binder -> { - binder.bind(Key.get(String.class, Names.named("serviceName"))).toInstance("some service"); - binder.bind(Key.get(Integer.class, Names.named("servicePort"))).toInstance(0); - binder.bind(Key.get(Integer.class, Names.named("tlsServicePort"))).toInstance(-1); JsonConfigProvider.bind(binder, "druid.query.scheduler", QuerySchedulerProvider.class, Global.class); } ) From ef029c4a894aafd4b17ca903e77ae497e41034ba Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Wed, 4 Mar 2020 04:13:46 -0800 Subject: [PATCH 22/37] style --- .../test/java/org/apache/druid/server/QuerySchedulerTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java index 5521844e1e52..39a7a6d6e570 100644 --- a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java +++ b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java @@ -93,7 +93,7 @@ public void setup() laneAcquired = new AtomicLong(); laneNotAcquired = new AtomicLong(); laneReleased = new AtomicLong(); - scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)){ + scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)) { @Override List acquireLanes(Query query) { From 059a2d40c9b011068b6aaef93a6dd02f4f372fbd Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 6 Mar 2020 04:59:58 -0800 Subject: [PATCH 23/37] partial review stuffs --- .../CachingClusteredClientBenchmark.java | 3 +- docs/configuration/index.md | 10 ++-- .../movingaverage/MovingAverageQueryTest.java | 2 +- .../druid/guice/QueryRunnerFactoryModule.java | 1 + .../druid/server/QueryLaningStrategy.java | 5 +- .../apache/druid/server/QueryScheduler.java | 33 +++++++++---- .../druid/server/QuerySchedulerProvider.java | 19 +++++++- .../scheduling/HiLoQueryLaningStrategy.java | 15 ++++-- .../scheduling/NoQueryLaningStrategy.java | 2 +- ...chingClusteredClientFunctionalityTest.java | 3 +- .../client/CachingClusteredClientTest.java | 3 +- .../druid/server/QueryResourceTest.java | 7 +-- .../druid/server/QuerySchedulerTest.java | 48 +++++++++++++++---- .../HiLoQueryLaningStrategyTest.java | 26 ++++++++-- .../druid/sql/calcite/util/CalciteTests.java | 1 + .../druid/sql/http/SqlResourceTest.java | 3 +- 16 files changed, 137 insertions(+), 44 deletions(-) diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java index 506f84556e5d..906c8f140a71 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java @@ -106,6 +106,7 @@ import org.apache.druid.segment.QueryableIndexSegment; import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.scheduling.NoQueryLaningStrategy; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment.PruneSpecsHolder; @@ -341,7 +342,7 @@ public > QueryToolChest getToolChest new DruidHttpClientConfig(), processingConfig, forkJoinPool, - new QueryScheduler(0, NoQueryLaningStrategy.INSTANCE) + new QueryScheduler(0, NoQueryLaningStrategy.INSTANCE, new ServerConfig()) ); } diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 6d899786b830..eb6a009b598d 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1483,20 +1483,20 @@ The Broker provides facilities to aid in query capacity reservation for heteroge |Property|Description|Default| |--------|-----------|-------| |`druid.query.scheduler.numThreads`|Maximum number of HTTP threads to dedicate to query processing. To save HTTP thread capacity, this should be lower than `druid.server.http.numThreads`.|Unbounded| -|`druid.query.scheduler.laning.type`|Query laning strategy to use to assign queries to a lane in order to control capacities for certain classes of queries.|`none`| +|`druid.query.scheduler.laning.strategy`|Query laning strategy to use to assign queries to a lane in order to control capacities for certain classes of queries.|`none`| ###### No laning strategy -In this mode, queries are never assigned a lane and only limited by `druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, if set. This is the default Druid query scheduler operating mode. +In this mode, queries are never assigned a lane and only limited by `druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, if set. This is the default Druid query scheduler operating mode, but can be explicitly set with by configuring `druid.query.scheduler.laning.strategy=none`. ###### 'High/Low' laning strategy -This laning strategy splits queries with a `priority` below zero into a `low` query lane, automatically. The limit on `low` queries can be set to some desired fraction of the total capacity (or HTTP thread pool size), reserving capacity for interactive queries. Queries in the `low` lane are _not_ guaranteed their capacity, which may be consumed by interactive queries, but may use up to this limit if total capacity is available. +This laning strategy splits queries with a `priority` below zero into a `low` query lane, automatically. The limit on `low` queries can be set to some desired percentage of the total capacity (or HTTP thread pool size), reserving capacity for interactive queries. Queries in the `low` lane are _not_ guaranteed their capacity, which may be consumed by interactive queries, but may use up to this limit if total capacity is available. -This strategy can be enabled by setting `druid.query.scheduler.laning.type` to `hilo`. +This strategy can be enabled by setting `druid.query.scheduler.laning.strategy=hilo`. |Property|Description|Default| |--------|-----------|-------| -|`druid.query.scheduler.laning.maxLowThreads`|Maximum number of HTTP threads that can be used by queries with a priority lower than 0.|No default, must be set if using this mode| +|`druid.query.scheduler.laning.maxLowPercentage`|Maximum percentage of the smaller number of`druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, defining the number of HTTP threads that can be used by queries with a priority lower than 0. Value must be between 1 and 100, and will be rounded up|No default, must be set if using this mode| ##### Server Configuration diff --git a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java index 1378c009d107..033ffb47f820 100644 --- a/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java +++ b/extensions-contrib/moving-average-query/src/test/java/org/apache/druid/query/movingaverage/MovingAverageQueryTest.java @@ -364,7 +364,7 @@ public String getFormatString() } }, ForkJoinPool.commonPool(), - new QueryScheduler(0, NoQueryLaningStrategy.INSTANCE) + new QueryScheduler(0, NoQueryLaningStrategy.INSTANCE, new ServerConfig()) ); ClientQuerySegmentWalker walker = new ClientQuerySegmentWalker( diff --git a/server/src/main/java/org/apache/druid/guice/QueryRunnerFactoryModule.java b/server/src/main/java/org/apache/druid/guice/QueryRunnerFactoryModule.java index 9570231ab487..102c72d890cc 100644 --- a/server/src/main/java/org/apache/druid/guice/QueryRunnerFactoryModule.java +++ b/server/src/main/java/org/apache/druid/guice/QueryRunnerFactoryModule.java @@ -74,6 +74,7 @@ public void configure(Binder binder) binder.bind(QueryScheduler.class) .toProvider(Key.get(QuerySchedulerProvider.class, Global.class)) .in(LazySingleton.class); + binder.bind(QuerySchedulerProvider.class).in(LazySingleton.class); JsonConfigProvider.bind(binder, "druid.query.scheduler", QuerySchedulerProvider.class, Global.class); final MapBinder, QueryRunnerFactory> queryFactoryBinder = DruidBinders.queryRunnerFactoryBinder( diff --git a/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java index 9a70a902db54..4fc62c9dac59 100644 --- a/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java @@ -31,7 +31,7 @@ import java.util.Set; -@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = NoQueryLaningStrategy.class) +@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "strategy", defaultImpl = NoQueryLaningStrategy.class) @JsonSubTypes(value = { @JsonSubTypes.Type(name = "none", value = NoQueryLaningStrategy.class), @JsonSubTypes.Type(name = "hilo", value = HiLoQueryLaningStrategy.class) @@ -40,8 +40,9 @@ public interface QueryLaningStrategy { /** * Provide map of lane names to the limit on the number of concurrent queries for that lane + * @param totalLimit */ - Object2IntMap getLaneLimits(); + Object2IntMap getLaneLimits(int totalLimit); /** * For a given {@link QueryPlus} and set of {@link SegmentServerSelector}, compute if a query belongs to a lane diff --git a/server/src/main/java/org/apache/druid/server/QueryScheduler.java b/server/src/main/java/org/apache/druid/server/QueryScheduler.java index 38ac3625a621..a943a6750ad6 100644 --- a/server/src/main/java/org/apache/druid/server/QueryScheduler.java +++ b/server/src/main/java/org/apache/druid/server/QueryScheduler.java @@ -35,6 +35,7 @@ import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryPlus; import org.apache.druid.query.QueryWatcher; +import org.apache.druid.server.initialization.ServerConfig; import java.time.Duration; import java.util.ArrayList; @@ -55,17 +56,27 @@ public class QueryScheduler implements QueryWatcher { static final String TOTAL = "default"; + private final int totalCapacity; private final QueryLaningStrategy laningStrategy; private final BulkheadRegistry laneRegistry; private final SetMultimap> queryFutures; private final SetMultimap queryDatasources; - public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy) + public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy, ServerConfig serverConfig) { this.laningStrategy = laningStrategy; - this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(totalNumThreads)); this.queryFutures = Multimaps.synchronizedSetMultimap(HashMultimap.create()); this.queryDatasources = Multimaps.synchronizedSetMultimap(HashMultimap.create()); + // if totalNumThreads is above 0 and less than druid.server.http.numThreads, enforce total limit + final boolean limitTotal; + if (totalNumThreads > 0 && totalNumThreads < serverConfig.getNumThreads()) { + limitTotal = true; + this.totalCapacity = totalNumThreads; + } else { + limitTotal = false; + this.totalCapacity = serverConfig.getNumThreads(); + } + this.laneRegistry = BulkheadRegistry.of(getLaneConfigs(limitTotal)); } @Override @@ -144,7 +155,8 @@ public Set getQueryDatasources(final String queryId) /** * Get the maximum number of concurrent queries that {@link #run} can support */ - public int getTotalAvailableCapacity() + @VisibleForTesting + int getTotalAvailableCapacity() { return laneRegistry.getConfiguration(TOTAL) .map(config -> laneRegistry.bulkhead(TOTAL, config).getMetrics().getAvailableConcurrentCalls()) @@ -154,7 +166,8 @@ public int getTotalAvailableCapacity() /** * Get the maximum number of concurrent queries that {@link #run} can support for a given lane */ - public int getLaneAvailableCapacity(String lane) + @VisibleForTesting + int getLaneAvailableCapacity(String lane) { return laneRegistry.getConfiguration(lane) .map(config -> laneRegistry.bulkhead(lane, config).getMetrics().getAvailableConcurrentCalls()) @@ -204,18 +217,20 @@ void releaseLanes(List bulkheads) /** * With a total thread count and {@link QueryLaningStrategy#getLaneLimits}, create a map of lane name to - * {@link BulkheadConfig} to be used to create the {@link #laneRegistry} + * {@link BulkheadConfig} to be used to create the {@link #laneRegistry}. This accepts the configured value of + * numThreads rather than using {@link #totalCapacity} so that we only have a total {@link Bulkhead} if + * {@link QuerySchedulerConfig#getNumThreads()} is set */ - private Map getLaneConfigs(int totalNumThreads) + private Map getLaneConfigs(boolean hastotalLimit) { Map configs = new HashMap<>(); - if (totalNumThreads > 0) { + if (hastotalLimit) { configs.put( TOTAL, - BulkheadConfig.custom().maxConcurrentCalls(totalNumThreads).maxWaitDuration(Duration.ZERO).build() + BulkheadConfig.custom().maxConcurrentCalls(totalCapacity).maxWaitDuration(Duration.ZERO).build() ); } - for (Object2IntMap.Entry entry : laningStrategy.getLaneLimits().object2IntEntrySet()) { + for (Object2IntMap.Entry entry : laningStrategy.getLaneLimits(totalCapacity).object2IntEntrySet()) { configs.put( entry.getKey(), BulkheadConfig.custom().maxConcurrentCalls(entry.getIntValue()).maxWaitDuration(Duration.ZERO).build() diff --git a/server/src/main/java/org/apache/druid/server/QuerySchedulerProvider.java b/server/src/main/java/org/apache/druid/server/QuerySchedulerProvider.java index bc474a49d532..f73601f69faf 100644 --- a/server/src/main/java/org/apache/druid/server/QuerySchedulerProvider.java +++ b/server/src/main/java/org/apache/druid/server/QuerySchedulerProvider.java @@ -19,14 +19,31 @@ package org.apache.druid.server; +import com.fasterxml.jackson.annotation.JacksonInject; +import com.fasterxml.jackson.annotation.JsonCreator; +import com.google.inject.Inject; import com.google.inject.Provider; +import org.apache.druid.server.initialization.ServerConfig; public class QuerySchedulerProvider extends QuerySchedulerConfig implements Provider { + private final ServerConfig serverConfig; + + /** + * This needs to be both marked as guice injected to be bound correctly, and also marked with json creator and + * jackson inject to work with {@link org.apache.druid.guice.JsonConfigProvider} + */ + @Inject + @JsonCreator + public QuerySchedulerProvider(@JacksonInject ServerConfig serverConfig) + { + this.serverConfig = serverConfig; + } + @Override public QueryScheduler get() { - return new QueryScheduler(getNumThreads(), getLaningStrategy()); + return new QueryScheduler(getNumThreads(), getLaningStrategy(), serverConfig); } } diff --git a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java index 1964d7893c3d..ff8a333b9883 100644 --- a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java @@ -22,6 +22,7 @@ import com.fasterxml.jackson.annotation.JsonCreator; import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.Preconditions; +import com.google.common.primitives.Ints; import it.unimi.dsi.fastutil.objects.Object2IntArrayMap; import it.unimi.dsi.fastutil.objects.Object2IntMap; import org.apache.druid.client.SegmentServerSelector; @@ -41,21 +42,25 @@ public class HiLoQueryLaningStrategy implements QueryLaningStrategy public static final String LOW = "low"; @JsonProperty - private final int maxLowThreads; + private final int maxLowPercentage; @JsonCreator public HiLoQueryLaningStrategy( - @JsonProperty("maxLowThreads") Integer maxLowThreads + @JsonProperty("maxLowPercentage") Integer maxLowPercentage ) { - this.maxLowThreads = Preconditions.checkNotNull(maxLowThreads, "maxLowThreads must be set"); + this.maxLowPercentage = Preconditions.checkNotNull(maxLowPercentage, "maxLowPercentage must be set"); + Preconditions.checkArgument( + 0 < maxLowPercentage && maxLowPercentage < 100, + "maxLowPercentage must be between 0 and 100" + ); } @Override - public Object2IntMap getLaneLimits() + public Object2IntMap getLaneLimits(int totalLimit) { Object2IntMap onlyLow = new Object2IntArrayMap<>(1); - onlyLow.put(LOW, maxLowThreads); + onlyLow.put(LOW, Ints.checkedCast(Math.round(totalLimit * ((double) maxLowPercentage / 100)))); return onlyLow; } diff --git a/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java index 5df6563757cb..1a15d806e61e 100644 --- a/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java @@ -38,7 +38,7 @@ public class NoQueryLaningStrategy implements QueryLaningStrategy public static final NoQueryLaningStrategy INSTANCE = new NoQueryLaningStrategy(); @Override - public Object2IntMap getLaneLimits() + public Object2IntMap getLaneLimits(int totalLimit) { return NONE; } diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java index 987aca2e8b7e..ac983eb3d4cf 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientFunctionalityTest.java @@ -49,6 +49,7 @@ import org.apache.druid.query.planning.DataSourceAnalysis; import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.scheduling.NoQueryLaningStrategy; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.TimelineLookup; @@ -333,7 +334,7 @@ public int getMergePoolParallelism() } }, ForkJoinPool.commonPool(), - new QueryScheduler(0, NoQueryLaningStrategy.INSTANCE) + new QueryScheduler(0, NoQueryLaningStrategy.INSTANCE, new ServerConfig()) ); } diff --git a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java index 422cd1efc60b..25c027e9e56e 100644 --- a/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java +++ b/server/src/test/java/org/apache/druid/client/CachingClusteredClientTest.java @@ -118,6 +118,7 @@ import org.apache.druid.segment.TestHelper; import org.apache.druid.server.QueryScheduler; import org.apache.druid.server.coordination.ServerType; +import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.scheduling.NoQueryLaningStrategy; import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.SegmentId; @@ -2476,7 +2477,7 @@ public int getMergePoolParallelism() } }, ForkJoinPool.commonPool(), - new QueryScheduler(0, NoQueryLaningStrategy.INSTANCE) + new QueryScheduler(0, NoQueryLaningStrategy.INSTANCE, new ServerConfig()) ); } diff --git a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java index 5233a443f35e..722a1655e720 100644 --- a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java @@ -43,6 +43,7 @@ import org.apache.druid.query.Result; import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.timeboundary.TimeBoundaryResultValue; +import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.log.TestRequestLogger; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.server.scheduling.HiLoQueryLaningStrategy; @@ -158,7 +159,7 @@ public void setup() EasyMock.expect(testServletRequest.getHeader("Accept")).andReturn(MediaType.APPLICATION_JSON).anyTimes(); EasyMock.expect(testServletRequest.getHeader(QueryResource.HEADER_IF_NONE_MATCH)).andReturn(null).anyTimes(); EasyMock.expect(testServletRequest.getRemoteAddr()).andReturn("localhost").anyTimes(); - queryScheduler = new QueryScheduler(8, NoQueryLaningStrategy.INSTANCE); + queryScheduler = new QueryScheduler(8, NoQueryLaningStrategy.INSTANCE, new ServerConfig()); testRequestLogger = new TestRequestLogger(); queryResource = new QueryResource( new QueryLifecycleFactory( @@ -676,7 +677,7 @@ public void testTooManyQuery() throws InterruptedException final CountDownLatch waitTwoScheduled = new CountDownLatch(2); final CountDownLatch waitAllFinished = new CountDownLatch(3); - final QueryScheduler laningScheduler = new QueryScheduler(2, NoQueryLaningStrategy.INSTANCE); + final QueryScheduler laningScheduler = new QueryScheduler(2, NoQueryLaningStrategy.INSTANCE, new ServerConfig()); createScheduledQueryResource(laningScheduler, Collections.emptyList(), ImmutableList.of(waitTwoScheduled)); assertResponseAndCountdownOrBlockForever( @@ -716,7 +717,7 @@ public void testTooManyQueryInLane() throws InterruptedException final CountDownLatch waitTwoStarted = new CountDownLatch(2); final CountDownLatch waitOneScheduled = new CountDownLatch(1); final CountDownLatch waitAllFinished = new CountDownLatch(3); - final QueryScheduler scheduler = new QueryScheduler(40, new HiLoQueryLaningStrategy(1)); + final QueryScheduler scheduler = new QueryScheduler(40, new HiLoQueryLaningStrategy(3), new ServerConfig()); createScheduledQueryResource(scheduler, ImmutableList.of(waitTwoStarted), ImmutableList.of(waitOneScheduled)); diff --git a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java index 39a7a6d6e570..c5f367a38151 100644 --- a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java +++ b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java @@ -19,6 +19,8 @@ package org.apache.druid.server; +import com.fasterxml.jackson.databind.InjectableValues; +import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; @@ -26,11 +28,14 @@ import com.google.common.util.concurrent.ListeningExecutorService; import com.google.common.util.concurrent.MoreExecutors; import com.google.inject.Injector; +import com.google.inject.Key; +import com.google.inject.ProvisionException; import io.github.resilience4j.bulkhead.Bulkhead; import org.apache.druid.guice.GuiceInjectors; import org.apache.druid.guice.JsonConfigProvider; import org.apache.druid.guice.JsonConfigurator; import org.apache.druid.guice.annotations.Global; +import org.apache.druid.guice.annotations.Json; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.guava.BaseSequence; @@ -46,6 +51,7 @@ import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.topn.TopNQuery; import org.apache.druid.query.topn.TopNQueryBuilder; +import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.scheduling.HiLoQueryLaningStrategy; import org.junit.After; import org.junit.Assert; @@ -93,7 +99,7 @@ public void setup() laneAcquired = new AtomicLong(); laneNotAcquired = new AtomicLong(); laneReleased = new AtomicLong(); - scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)) { + scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(40), new ServerConfig()) { @Override List acquireLanes(Query query) { @@ -199,7 +205,7 @@ public void before() } @Test - public void testHiLoReleaseSemaphoreWhenSequenceExplodes() throws Exception + public void testHiLoReleaseLaneWhenSequenceExplodes() throws Exception { expected.expectMessage("exploded"); expected.expect(ExecutionException.class); @@ -228,7 +234,6 @@ public void before() } }); future.get(); - Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); } @Test @@ -335,7 +340,7 @@ public void testConfigNone() QuerySchedulerProvider.class ); final Properties properties = new Properties(); - properties.put(propertyPrefix + ".numThreads", "10"); + properties.setProperty(propertyPrefix + ".numThreads", "10"); provider.inject(properties, injector.getInstance(JsonConfigurator.class)); final QueryScheduler scheduler = provider.get().get().get(); Assert.assertEquals(10, scheduler.getTotalAvailableCapacity()); @@ -352,15 +357,36 @@ public void testConfigHiLo() QuerySchedulerProvider.class ); final Properties properties = new Properties(); - properties.put(propertyPrefix + ".numThreads", "10"); - properties.put(propertyPrefix + ".laning.type", "hilo"); - properties.put(propertyPrefix + ".laning.maxLowThreads", "2"); + properties.setProperty(propertyPrefix + ".numThreads", "10"); + properties.setProperty(propertyPrefix + ".laning.strategy", "hilo"); + properties.setProperty(propertyPrefix + ".laning.maxLowPercentage", "20"); + + provider.inject(properties, injector.getInstance(JsonConfigurator.class)); + final QueryScheduler scheduler = provider.get().get().get(); + Assert.assertEquals(10, scheduler.getTotalAvailableCapacity()); + Assert.assertEquals(2, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); + } + + + @Test + public void testMisConfigHiLo() + { + expected.expect(ProvisionException.class); + final Injector injector = createInjector(); + final String propertyPrefix = "druid.query.scheduler"; + final JsonConfigProvider provider = JsonConfigProvider.of( + propertyPrefix, + QuerySchedulerProvider.class + ); + final Properties properties = new Properties(); + properties.setProperty(propertyPrefix + ".laning.strategy", "hilo"); provider.inject(properties, injector.getInstance(JsonConfigurator.class)); final QueryScheduler scheduler = provider.get().get().get(); Assert.assertEquals(10, scheduler.getTotalAvailableCapacity()); Assert.assertEquals(2, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); } + private void maybeDelayNextIteration(int i) throws InterruptedException { if (i > 0 && i % 10 == 0) { @@ -555,12 +581,18 @@ private void getFuturesAndAssertAftermathIsChill( private Injector createInjector() { - return GuiceInjectors.makeStartupInjectorWithModules( + Injector injector = GuiceInjectors.makeStartupInjectorWithModules( ImmutableList.of( binder -> { + binder.bind(ServerConfig.class).toInstance(new ServerConfig()); JsonConfigProvider.bind(binder, "druid.query.scheduler", QuerySchedulerProvider.class, Global.class); } ) ); + ObjectMapper mapper = injector.getInstance(Key.get(ObjectMapper.class, Json.class)); + mapper.setInjectableValues( + new InjectableValues.Std().addValue(ServerConfig.class, injector.getInstance(ServerConfig.class)) + ); + return injector; } } diff --git a/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java b/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java index 10eceac56c96..3d28ec976819 100644 --- a/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java @@ -54,24 +54,40 @@ public void setup() .granularity(Granularities.DAY) .aggregators(new CountAggregatorFactory("count")); - this.strategy = new HiLoQueryLaningStrategy(10); + this.strategy = new HiLoQueryLaningStrategy(40); } @Test - public void testMaxLowThreadsRequired() + public void testMaxPercentageThreadsRequired() { expectedException.expect(NullPointerException.class); - expectedException.expectMessage("maxLowThreads must be set"); + expectedException.expectMessage("maxLowPercentage must be set"); QueryLaningStrategy strategy = new HiLoQueryLaningStrategy(null); } + @Test + public void testMaxLowPercentageMustBeGreaterThanZero() + { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("maxLowPercentage must be between 0 and 100"); + QueryLaningStrategy strategy = new HiLoQueryLaningStrategy(-1); + } + + @Test + public void testMaxLowPercentageMustBeLessThan100() + { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("maxLowPercentage must be between 0 and 100"); + QueryLaningStrategy strategy = new HiLoQueryLaningStrategy(9000); + } + @Test public void testLaneLimits() { - Object2IntMap laneConfig = strategy.getLaneLimits(); + Object2IntMap laneConfig = strategy.getLaneLimits(5); Assert.assertEquals(1, laneConfig.size()); Assert.assertTrue(laneConfig.containsKey(HiLoQueryLaningStrategy.LOW)); - Assert.assertEquals(10, laneConfig.getInt(HiLoQueryLaningStrategy.LOW)); + Assert.assertEquals(2, laneConfig.getInt(HiLoQueryLaningStrategy.LOW)); } @Test diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java index fd2532ecddc1..cbfa5e4953ed 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/util/CalciteTests.java @@ -704,6 +704,7 @@ public static SpecificSegmentsQuerySegmentWalker createMockWalker( { return createMockWalker(conglomerate, tmpDir, null); } + public static SpecificSegmentsQuerySegmentWalker createMockWalker( final QueryRunnerFactoryConglomerate conglomerate, final File tmpDir, diff --git a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java index dc452bbee94b..2fa358285597 100644 --- a/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java +++ b/sql/src/test/java/org/apache/druid/sql/http/SqlResourceTest.java @@ -44,6 +44,7 @@ import org.apache.druid.query.ResourceLimitExceededException; import org.apache.druid.server.QueryCapacityExceededException; import org.apache.druid.server.QueryScheduler; +import org.apache.druid.server.initialization.ServerConfig; import org.apache.druid.server.log.TestRequestLogger; import org.apache.druid.server.metrics.NoopServiceEmitter; import org.apache.druid.server.scheduling.HiLoQueryLaningStrategy; @@ -121,7 +122,7 @@ public void setUp() throws Exception executorService = MoreExecutors.listeningDecorator( Execs.multiThreaded(8, "test_sql_resource_%s") ); - scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)); + scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(40), new ServerConfig()); walker = CalciteTests.createMockWalker(conglomerate, temporaryFolder.newFolder(), scheduler); final PlannerConfig plannerConfig = new PlannerConfig() From 0ec8a2657e9f10e84fcd41cf42fef22063fe89ac Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 6 Mar 2020 13:04:47 -0800 Subject: [PATCH 24/37] adjust --- docs/configuration/index.md | 2 +- .../server/scheduling/HiLoQueryLaningStrategy.java | 12 ++++++------ .../org/apache/druid/server/QuerySchedulerTest.java | 2 +- .../scheduling/HiLoQueryLaningStrategyTest.java | 10 +++++----- 4 files changed, 13 insertions(+), 13 deletions(-) diff --git a/docs/configuration/index.md b/docs/configuration/index.md index eb6a009b598d..0bd46ad2c723 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1496,7 +1496,7 @@ This strategy can be enabled by setting `druid.query.scheduler.laning.strategy=h |Property|Description|Default| |--------|-----------|-------| -|`druid.query.scheduler.laning.maxLowPercentage`|Maximum percentage of the smaller number of`druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, defining the number of HTTP threads that can be used by queries with a priority lower than 0. Value must be between 1 and 100, and will be rounded up|No default, must be set if using this mode| +|`druid.query.scheduler.laning.maxLowPercent`|Maximum percent of the smaller number of`druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, defining the number of HTTP threads that can be used by queries with a priority lower than 0. Value must be between 1 and 100, and will be rounded up|No default, must be set if using this mode| ##### Server Configuration diff --git a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java index ff8a333b9883..2944333b4a18 100644 --- a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java @@ -42,17 +42,17 @@ public class HiLoQueryLaningStrategy implements QueryLaningStrategy public static final String LOW = "low"; @JsonProperty - private final int maxLowPercentage; + private final int maxLowPercent; @JsonCreator public HiLoQueryLaningStrategy( - @JsonProperty("maxLowPercentage") Integer maxLowPercentage + @JsonProperty("maxLowPercent") Integer maxLowPercent ) { - this.maxLowPercentage = Preconditions.checkNotNull(maxLowPercentage, "maxLowPercentage must be set"); + this.maxLowPercent = Preconditions.checkNotNull(maxLowPercent, "maxLowPercent must be set"); Preconditions.checkArgument( - 0 < maxLowPercentage && maxLowPercentage < 100, - "maxLowPercentage must be between 0 and 100" + 0 < maxLowPercent && maxLowPercent < 100, + "maxLowPercent must be between 0 and 100" ); } @@ -60,7 +60,7 @@ public HiLoQueryLaningStrategy( public Object2IntMap getLaneLimits(int totalLimit) { Object2IntMap onlyLow = new Object2IntArrayMap<>(1); - onlyLow.put(LOW, Ints.checkedCast(Math.round(totalLimit * ((double) maxLowPercentage / 100)))); + onlyLow.put(LOW, Ints.checkedCast(Math.round(totalLimit * ((double) maxLowPercent / 100)))); return onlyLow; } diff --git a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java index c5f367a38151..66a1dca5ea32 100644 --- a/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java +++ b/server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java @@ -359,7 +359,7 @@ public void testConfigHiLo() final Properties properties = new Properties(); properties.setProperty(propertyPrefix + ".numThreads", "10"); properties.setProperty(propertyPrefix + ".laning.strategy", "hilo"); - properties.setProperty(propertyPrefix + ".laning.maxLowPercentage", "20"); + properties.setProperty(propertyPrefix + ".laning.maxLowPercent", "20"); provider.inject(properties, injector.getInstance(JsonConfigurator.class)); final QueryScheduler scheduler = provider.get().get().get(); diff --git a/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java b/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java index 3d28ec976819..f3c55cf7b4ea 100644 --- a/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java @@ -61,23 +61,23 @@ public void setup() public void testMaxPercentageThreadsRequired() { expectedException.expect(NullPointerException.class); - expectedException.expectMessage("maxLowPercentage must be set"); + expectedException.expectMessage("maxLowPercent must be set"); QueryLaningStrategy strategy = new HiLoQueryLaningStrategy(null); } @Test - public void testMaxLowPercentageMustBeGreaterThanZero() + public void testmaxLowPercentMustBeGreaterThanZero() { expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("maxLowPercentage must be between 0 and 100"); + expectedException.expectMessage("maxLowPercent must be between 0 and 100"); QueryLaningStrategy strategy = new HiLoQueryLaningStrategy(-1); } @Test - public void testMaxLowPercentageMustBeLessThan100() + public void testmaxLowPercentMustBeLessThan100() { expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("maxLowPercentage must be between 0 and 100"); + expectedException.expectMessage("maxLowPercent must be between 0 and 100"); QueryLaningStrategy strategy = new HiLoQueryLaningStrategy(9000); } From 5711fce706bbd23a5f7b559ff91e4288ea8ad96d Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 6 Mar 2020 17:14:28 -0800 Subject: [PATCH 25/37] review stuffs --- docs/configuration/index.md | 8 +- docs/querying/query-context.md | 1 + .../druid/server/QueryLaningStrategy.java | 4 +- .../apache/druid/server/QueryScheduler.java | 78 +++++++++++-------- .../scheduling/HiLoQueryLaningStrategy.java | 2 +- .../druid/server/QuerySchedulerTest.java | 12 +++ 6 files changed, 70 insertions(+), 35 deletions(-) diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 0bd46ad2c723..9d8be62d29af 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1485,12 +1485,16 @@ The Broker provides facilities to aid in query capacity reservation for heteroge |`druid.query.scheduler.numThreads`|Maximum number of HTTP threads to dedicate to query processing. To save HTTP thread capacity, this should be lower than `druid.server.http.numThreads`.|Unbounded| |`druid.query.scheduler.laning.strategy`|Query laning strategy to use to assign queries to a lane in order to control capacities for certain classes of queries.|`none`| +##### Laning strategies + ###### No laning strategy -In this mode, queries are never assigned a lane and only limited by `druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, if set. This is the default Druid query scheduler operating mode, but can be explicitly set with by configuring `druid.query.scheduler.laning.strategy=none`. +In this mode, queries are never assigned lane, and concurrent query count will only limited by `druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, if set. This is the default Druid query scheduler operating mode. This mode can be explicitly set by setting `druid.query.scheduler.laning.strategy` to `none`. ###### 'High/Low' laning strategy -This laning strategy splits queries with a `priority` below zero into a `low` query lane, automatically. The limit on `low` queries can be set to some desired percentage of the total capacity (or HTTP thread pool size), reserving capacity for interactive queries. Queries in the `low` lane are _not_ guaranteed their capacity, which may be consumed by interactive queries, but may use up to this limit if total capacity is available. +This laning strategy splits queries with a `priority` below zero into a `low` query lane, automatically. Queries with priority of zero (the default) or above are considered 'interactive'. The limit on `low` queries can be set to some desired percentage of the total capacity (or HTTP thread pool size), reserving capacity for interactive queries. Queries in the `low` lane are _not_ guaranteed their capacity, which may be consumed by interactive queries, but may use up to this limit if total capacity is available. + +If the `low` lane is specified in the [query context](../querying/query-context.md) `lane` parameter, this will override the computed lane. This strategy can be enabled by setting `druid.query.scheduler.laning.strategy=hilo`. diff --git a/docs/querying/query-context.md b/docs/querying/query-context.md index 28cfb3bd5228..020527309530 100644 --- a/docs/querying/query-context.md +++ b/docs/querying/query-context.md @@ -29,6 +29,7 @@ The query context is used for various query configuration parameters. The follow |-----------------|----------------------------------------|----------------------| |timeout | `druid.server.http.defaultQueryTimeout`| Query timeout in millis, beyond which unfinished queries will be cancelled. 0 timeout means `no timeout`. To set the default timeout, see [Broker configuration](../configuration/index.html#broker) | |priority | `0` | Query Priority. Queries with higher priority get precedence for computational resources.| +|lane | `null` | Query lane, used to control usage limits on classes of queries. See [Broker configuration](../configuration/index.html#broker) for more details.| |queryId | auto-generated | Unique identifier given to this query. If a query ID is set or known, this can be used to cancel the query | |useCache | `true` | Flag indicating whether to leverage the query cache for this query. When set to false, it disables reading from the query cache for this query. When set to true, Apache Druid uses `druid.broker.cache.useCache` or `druid.historical.cache.useCache` to determine whether or not to read from the query cache | |populateCache | `true` | Flag indicating whether to save the results of the query to the query cache. Primarily used for debugging. When set to false, it disables saving the results of this query to the query cache. When set to true, Druid uses `druid.broker.cache.populateCache` or `druid.historical.cache.populateCache` to determine whether or not to save the results of this query to the query cache | diff --git a/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java index 4fc62c9dac59..a2e922ef48db 100644 --- a/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/QueryLaningStrategy.java @@ -39,13 +39,15 @@ public interface QueryLaningStrategy { /** - * Provide map of lane names to the limit on the number of concurrent queries for that lane + * Provide a map of lane names to the limit on the number of concurrent queries for that lane * @param totalLimit */ Object2IntMap getLaneLimits(int totalLimit); /** * For a given {@link QueryPlus} and set of {@link SegmentServerSelector}, compute if a query belongs to a lane + * + * This method must be thread safe */ Optional computeLane(QueryPlus query, Set segments); } diff --git a/server/src/main/java/org/apache/druid/server/QueryScheduler.java b/server/src/main/java/org/apache/druid/server/QueryScheduler.java index a943a6750ad6..cd9c04bf220b 100644 --- a/server/src/main/java/org/apache/druid/server/QueryScheduler.java +++ b/server/src/main/java/org/apache/druid/server/QueryScheduler.java @@ -49,9 +49,9 @@ * QueryScheduler (potentially) assigns any {@link Query} that is to be executed to a 'query lane' using the * {@link QueryLaningStrategy} that is defined in {@link QuerySchedulerConfig}. * - * As a {@link QueryWatcher}, it also provides cancellation facilities. + * As a {@link QueryWatcher}, it also provides cancellation facilities to brokers, historicals, and realtime tasks. * - * This class is shared by all requests on the Jetty HTTP theadpool and must be thread safe. + * This class is shared by all requests on the HTTP theadpool and must be thread safe. */ public class QueryScheduler implements QueryWatcher { @@ -59,7 +59,13 @@ public class QueryScheduler implements QueryWatcher private final int totalCapacity; private final QueryLaningStrategy laningStrategy; private final BulkheadRegistry laneRegistry; + /** + * mapping of query id to set of futures associated with the query + */ private final SetMultimap> queryFutures; + /** + * mapping of query id to set of datasource names that are being queried, used for authorization + */ private final SetMultimap queryDatasources; public QueryScheduler(int totalNumThreads, QueryLaningStrategy laningStrategy, ServerConfig serverConfig) @@ -103,10 +109,6 @@ public void registerQueryFuture(Query query, ListenableFuture future) public Query laneQuery(QueryPlus queryPlus, Set segments) { Query query = queryPlus.getQuery(); - // man wins over machine.. for now. - if (QueryContexts.getLane(query) != null) { - return query; - } Optional lane = laningStrategy.computeLane(queryPlus, segments); return lane.map(query::withLane).orElse(query); } @@ -125,7 +127,7 @@ public Query laneQuery(QueryPlus queryPlus, Set public Sequence run(Query query, Sequence resultSequence) { List bulkheads = acquireLanes(query); - return resultSequence.withBaggage(() -> releaseLanes(bulkheads)); + return resultSequence.withBaggage(() -> finishLanes(bulkheads)); } /** @@ -133,7 +135,8 @@ public Sequence run(Query query, Sequence resultSequence) */ public boolean cancelQuery(String id) { - // if you re-use queryId and cancel queries... you are going to have a bad time + // if multiple independent queries from the same or different users share a query id, all will be cancelled due + // to the collision queryDatasources.removeAll(id); Set> futures = queryFutures.removeAll(id); boolean success = true; @@ -182,28 +185,35 @@ List acquireLanes(Query query) { final String lane = QueryContexts.getLane(query); final Optional laneConfig = lane == null ? Optional.empty() : laneRegistry.getConfiguration(lane); - List hallPasses = new ArrayList<>(2); final Optional totalConfig = laneRegistry.getConfiguration(TOTAL); - // if we have a lane, get it first - laneConfig.ifPresent(config -> { - Bulkhead laneLimiter = laneRegistry.bulkhead(lane, config); - if (!laneLimiter.tryAcquirePermission()) { - throw new QueryCapacityExceededException(lane); - } - hallPasses.add(laneLimiter); - }); - - // everyone needs to take one from the total lane; to ensure we don't acquire a lane and never release it, we want - // to check for total capacity exceeded and release the lane (if present) before throwing capacity exceeded - totalConfig.ifPresent(config -> { - Bulkhead totalLimiter = laneRegistry.bulkhead(TOTAL, config); - if (!totalLimiter.tryAcquirePermission()) { - releaseLanes(hallPasses); - throw new QueryCapacityExceededException(); - } - hallPasses.add(totalLimiter); - }); - return hallPasses; + List hallPasses = new ArrayList<>(2); + try { + // if we have a lane, get it first + laneConfig.ifPresent(config -> { + Bulkhead laneLimiter = laneRegistry.bulkhead(lane, config); + if (!laneLimiter.tryAcquirePermission()) { + throw new QueryCapacityExceededException(lane); + } + hallPasses.add(laneLimiter); + }); + + // everyone needs to take one from the total lane; to ensure we don't acquire a lane and never release it, we want + // to check for total capacity exceeded and release the lane (if present) before throwing capacity exceeded + // note that this isn't strictly fair: the bulkhead doesn't use a fair semaphore, the first to acquire the lane + // might lose to one that came after it when acquiring the total, or an unlaned query might lose to a laned query + totalConfig.ifPresent(config -> { + Bulkhead totalLimiter = laneRegistry.bulkhead(TOTAL, config); + if (!totalLimiter.tryAcquirePermission()) { + throw new QueryCapacityExceededException(); + } + hallPasses.add(totalLimiter); + }); + return hallPasses; + } + catch (Exception ex) { + releaseLanes(hallPasses); + throw ex; + } } /** @@ -215,16 +225,22 @@ void releaseLanes(List bulkheads) bulkheads.forEach(Bulkhead::releasePermission); } + @VisibleForTesting + void finishLanes(List bulkheads) + { + bulkheads.forEach(Bulkhead::onComplete); + } + /** * With a total thread count and {@link QueryLaningStrategy#getLaneLimits}, create a map of lane name to * {@link BulkheadConfig} to be used to create the {@link #laneRegistry}. This accepts the configured value of * numThreads rather than using {@link #totalCapacity} so that we only have a total {@link Bulkhead} if * {@link QuerySchedulerConfig#getNumThreads()} is set */ - private Map getLaneConfigs(boolean hastotalLimit) + private Map getLaneConfigs(boolean hasTotalLimit) { Map configs = new HashMap<>(); - if (hastotalLimit) { + if (hasTotalLimit) { configs.put( TOTAL, BulkheadConfig.custom().maxConcurrentCalls(totalCapacity).maxWaitDuration(Duration.ZERO).build() diff --git a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java index 2944333b4a18..7e07bd88d52a 100644 --- a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java @@ -74,6 +74,6 @@ public Optional computeLane(QueryPlus query, Set bulkheads) } } } + + @Override + void finishLanes(List bulkheads) + { + super.finishLanes(bulkheads); + if (bulkheads.stream().anyMatch(b -> b.getName().equals(QueryScheduler.TOTAL))) { + totalReleased.incrementAndGet(); + } + if (bulkheads.stream().anyMatch(b -> !b.getName().equals(QueryScheduler.TOTAL))) { + laneReleased.incrementAndGet(); + } + } }; } From aa73a14a056e61f14a0927e8a029da999f95fc1c Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 6 Mar 2020 17:22:25 -0800 Subject: [PATCH 26/37] more javadoc --- .../main/java/org/apache/druid/query/QueryException.java | 2 ++ .../druid/server/QueryCapacityExceededException.java | 7 +++++++ 2 files changed, 9 insertions(+) diff --git a/processing/src/main/java/org/apache/druid/query/QueryException.java b/processing/src/main/java/org/apache/druid/query/QueryException.java index 16ab6b147508..8a835573a2c4 100644 --- a/processing/src/main/java/org/apache/druid/query/QueryException.java +++ b/processing/src/main/java/org/apache/druid/query/QueryException.java @@ -26,6 +26,8 @@ /** * Base serializable error response + * + * QueryResource and SqlResource are expected to emit the JSON form of this object when errors happen. */ public class QueryException extends RuntimeException { diff --git a/server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java b/server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java index a9320302b9d9..b0808f3c059a 100644 --- a/server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java +++ b/server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java @@ -24,6 +24,13 @@ import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.query.QueryException; +/** + * This exception is for {@link QueryResource} and SqlResource to surface when a query is cast away by + * {@link QueryScheduler}. + * + * As a {@link QueryException} it is expected to be serialied to a json response, but will be mapped to + * {@link #STATUS_CODE} instead of the default HTTP 500 status. + */ public class QueryCapacityExceededException extends QueryException { private static final String ERROR_CLASS = QueryCapacityExceededException.class.getName(); From 50847af1a17bb2fa2a8bff12ee32e4476fe67a86 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 6 Mar 2020 18:20:05 -0800 Subject: [PATCH 27/37] error response documentation --- docs/querying/querying.md | 2 ++ .../org/apache/druid/server/QueryCapacityExceededException.java | 2 +- 2 files changed, 3 insertions(+), 1 deletion(-) diff --git a/docs/querying/querying.md b/docs/querying/querying.md index f102173263ab..1734fbaa0c47 100644 --- a/docs/querying/querying.md +++ b/docs/querying/querying.md @@ -109,6 +109,8 @@ If a query fails, you will get an HTTP 500 response containing a JSON object wit } ``` +If a query request fails due to being limited by the [query scheduler laning configuration](../configuration/index.md#broker), an HTTP 429 response with the same JSON object schema as an error response, but with `errorMessage` of the form: "Total query capacity exceeded" or "Query capacity exceeded for lane 'somelane'". + The fields in the response are: |field|description| diff --git a/server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java b/server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java index b0808f3c059a..9085447f2dbe 100644 --- a/server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java +++ b/server/src/main/java/org/apache/druid/server/QueryCapacityExceededException.java @@ -36,7 +36,7 @@ public class QueryCapacityExceededException extends QueryException private static final String ERROR_CLASS = QueryCapacityExceededException.class.getName(); public static final String ERROR_CODE = "Query capacity exceeded"; public static final String ERROR_MESSAGE = "Total query capacity exceeded"; - public static final String ERROR_MESSAGE_TEMPLATE = "Query capacity exceeded for lane %s"; + public static final String ERROR_MESSAGE_TEMPLATE = "Query capacity exceeded for lane '%s'"; public static final int STATUS_CODE = 429; public QueryCapacityExceededException() From abe3631be329e40c5a3fbdbd904f4867a00ec6ee Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 6 Mar 2020 19:00:40 -0800 Subject: [PATCH 28/37] spelling --- docs/querying/querying.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/querying/querying.md b/docs/querying/querying.md index 1734fbaa0c47..20f9feddbdcb 100644 --- a/docs/querying/querying.md +++ b/docs/querying/querying.md @@ -109,7 +109,7 @@ If a query fails, you will get an HTTP 500 response containing a JSON object wit } ``` -If a query request fails due to being limited by the [query scheduler laning configuration](../configuration/index.md#broker), an HTTP 429 response with the same JSON object schema as an error response, but with `errorMessage` of the form: "Total query capacity exceeded" or "Query capacity exceeded for lane 'somelane'". +If a query request fails due to being limited by the [query scheduler laning configuration](../configuration/index.md#broker), an HTTP 429 response with the same JSON object schema as an error response, but with `errorMessage` of the form: "Total query capacity exceeded" or "Query capacity exceeded for lane 'low'". The fields in the response are: From 86501ef4bbe2f1d2b425b75e697b55e32571b2b1 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 6 Mar 2020 19:12:01 -0800 Subject: [PATCH 29/37] preserve user specified lane for NoSchedulingStrategy --- .../scheduling/NoQueryLaningStrategy.java | 3 +- .../HiLoQueryLaningStrategyTest.java | 12 ++++ .../scheduling/NoQueryLaningStrategyTest.java | 67 +++++++++++++++++++ 3 files changed, 81 insertions(+), 1 deletion(-) create mode 100644 server/src/test/java/org/apache/druid/server/scheduling/NoQueryLaningStrategyTest.java diff --git a/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java index 1a15d806e61e..8f830d6b555d 100644 --- a/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/NoQueryLaningStrategy.java @@ -22,6 +22,7 @@ import it.unimi.dsi.fastutil.objects.Object2IntArrayMap; import it.unimi.dsi.fastutil.objects.Object2IntMap; import org.apache.druid.client.SegmentServerSelector; +import org.apache.druid.query.QueryContexts; import org.apache.druid.query.QueryPlus; import org.apache.druid.server.QueryLaningStrategy; @@ -46,6 +47,6 @@ public Object2IntMap getLaneLimits(int totalLimit) @Override public Optional computeLane(QueryPlus query, Set segments) { - return Optional.empty(); + return Optional.ofNullable(QueryContexts.getLane(query.getQuery())); } } diff --git a/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java b/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java index f3c55cf7b4ea..c9f2c5378e13 100644 --- a/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java @@ -121,4 +121,16 @@ public void testLaningLowPriority() strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).get() ); } + + @Test + public void testLaningPreservesManualSetLane() + { + TimeseriesQuery query = queryBuilder.context( + ImmutableMap.of(QueryContexts.PRIORITY_KEY, 100, QueryContexts.LANE_KEY, "low") + ).build(); + Assert.assertEquals( + HiLoQueryLaningStrategy.LOW, + strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).get() + ); + } } diff --git a/server/src/test/java/org/apache/druid/server/scheduling/NoQueryLaningStrategyTest.java b/server/src/test/java/org/apache/druid/server/scheduling/NoQueryLaningStrategyTest.java new file mode 100644 index 000000000000..7f718c1a6d5a --- /dev/null +++ b/server/src/test/java/org/apache/druid/server/scheduling/NoQueryLaningStrategyTest.java @@ -0,0 +1,67 @@ +/* + * 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.scheduling; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import com.google.common.collect.ImmutableSet; +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.QueryContexts; +import org.apache.druid.query.QueryPlus; +import org.apache.druid.query.aggregation.CountAggregatorFactory; +import org.apache.druid.query.timeseries.TimeseriesQuery; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.*; + +public class NoQueryLaningStrategyTest +{ + private Druids.TimeseriesQueryBuilder queryBuilder; + private NoQueryLaningStrategy strategy; + + @Before + public void setup() + { + this.queryBuilder = Druids.newTimeseriesQueryBuilder() + .dataSource("test") + .intervals(ImmutableList.of(Intervals.ETERNITY)) + .granularity(Granularities.DAY) + .aggregators(new CountAggregatorFactory("count")); + + this.strategy = new NoQueryLaningStrategy(); + } + + @Test + public void testPreservesManualLaneFromContext() + { + final String someLane = "some-lane"; + TimeseriesQuery query = queryBuilder.context( + ImmutableMap.of(QueryContexts.PRIORITY_KEY, 100, QueryContexts.LANE_KEY, someLane) + ).build(); + Assert.assertEquals( + someLane, + strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).get() + ); + } +} \ No newline at end of file From 8b7b70d16c6da7347d12718f4db3720114c17fec Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 6 Mar 2020 19:14:09 -0800 Subject: [PATCH 30/37] more test, why not --- .../druid/server/scheduling/NoQueryLaningStrategyTest.java | 7 +++++++ 1 file changed, 7 insertions(+) diff --git a/server/src/test/java/org/apache/druid/server/scheduling/NoQueryLaningStrategyTest.java b/server/src/test/java/org/apache/druid/server/scheduling/NoQueryLaningStrategyTest.java index 7f718c1a6d5a..1d311b0b8316 100644 --- a/server/src/test/java/org/apache/druid/server/scheduling/NoQueryLaningStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/scheduling/NoQueryLaningStrategyTest.java @@ -52,6 +52,13 @@ public void setup() this.strategy = new NoQueryLaningStrategy(); } + @Test + public void testDoesntSetLane() + { + TimeseriesQuery query = queryBuilder.context(ImmutableMap.of()).build(); + Assert.assertFalse(strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).isPresent()); + } + @Test public void testPreservesManualLaneFromContext() { From 91ad9d97fc3d416801204bd3d19f10205d3b6283 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 6 Mar 2020 19:17:21 -0800 Subject: [PATCH 31/37] doc adjustment --- docs/configuration/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 9d8be62d29af..528eb4c14502 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1489,7 +1489,7 @@ The Broker provides facilities to aid in query capacity reservation for heteroge ###### No laning strategy -In this mode, queries are never assigned lane, and concurrent query count will only limited by `druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, if set. This is the default Druid query scheduler operating mode. This mode can be explicitly set by setting `druid.query.scheduler.laning.strategy` to `none`. +In this mode, queries are never assigned lane, and concurrent query count will only limited by `druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, if set. This is the default Druid query scheduler operating mode. This strategy can also be explicitly enabled by setting `druid.query.scheduler.laning.strategy` to `none`. ###### 'High/Low' laning strategy This laning strategy splits queries with a `priority` below zero into a `low` query lane, automatically. Queries with priority of zero (the default) or above are considered 'interactive'. The limit on `low` queries can be set to some desired percentage of the total capacity (or HTTP thread pool size), reserving capacity for interactive queries. Queries in the `low` lane are _not_ guaranteed their capacity, which may be consumed by interactive queries, but may use up to this limit if total capacity is available. From 373fd1158f13c9f7a77ac6ba1748aa2b2cbb8b99 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Fri, 6 Mar 2020 21:08:54 -0800 Subject: [PATCH 32/37] style --- .../druid/server/scheduling/NoQueryLaningStrategyTest.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/server/src/test/java/org/apache/druid/server/scheduling/NoQueryLaningStrategyTest.java b/server/src/test/java/org/apache/druid/server/scheduling/NoQueryLaningStrategyTest.java index 1d311b0b8316..58901df42981 100644 --- a/server/src/test/java/org/apache/druid/server/scheduling/NoQueryLaningStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/scheduling/NoQueryLaningStrategyTest.java @@ -33,8 +33,6 @@ import org.junit.Before; import org.junit.Test; -import static org.junit.Assert.*; - public class NoQueryLaningStrategyTest { private Druids.TimeseriesQueryBuilder queryBuilder; @@ -71,4 +69,4 @@ public void testPreservesManualLaneFromContext() strategy.computeLane(QueryPlus.wrap(query), ImmutableSet.of()).get() ); } -} \ No newline at end of file +} From 274150174fabd07646fe87a01dea67dc714172c7 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 9 Mar 2020 10:12:17 -0700 Subject: [PATCH 33/37] missed review for make a thing a constant --- .../main/java/org/apache/druid/server/QueryScheduler.java | 5 +++-- 1 file changed, 3 insertions(+), 2 deletions(-) diff --git a/server/src/main/java/org/apache/druid/server/QueryScheduler.java b/server/src/main/java/org/apache/druid/server/QueryScheduler.java index cd9c04bf220b..50df2720aaa6 100644 --- a/server/src/main/java/org/apache/druid/server/QueryScheduler.java +++ b/server/src/main/java/org/apache/druid/server/QueryScheduler.java @@ -55,6 +55,7 @@ */ public class QueryScheduler implements QueryWatcher { + private static final int NO_CAPACITY = -1; static final String TOTAL = "default"; private final int totalCapacity; private final QueryLaningStrategy laningStrategy; @@ -163,7 +164,7 @@ int getTotalAvailableCapacity() { return laneRegistry.getConfiguration(TOTAL) .map(config -> laneRegistry.bulkhead(TOTAL, config).getMetrics().getAvailableConcurrentCalls()) - .orElse(-1); + .orElse(NO_CAPACITY); } /** @@ -174,7 +175,7 @@ int getLaneAvailableCapacity(String lane) { return laneRegistry.getConfiguration(lane) .map(config -> laneRegistry.bulkhead(lane, config).getMetrics().getAvailableConcurrentCalls()) - .orElse(-1); + .orElse(NO_CAPACITY); } /** From 8575cf8dc1a3e4fbedb46b1f7d36985e984b991c Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 9 Mar 2020 14:14:59 -0700 Subject: [PATCH 34/37] fixes and tests --- docs/configuration/index.md | 2 +- .../scheduling/HiLoQueryLaningStrategy.java | 6 +-- .../HiLoQueryLaningStrategyTest.java | 52 +++++++++++++++++-- 3 files changed, 52 insertions(+), 8 deletions(-) diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 528eb4c14502..d149f9df470d 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1500,7 +1500,7 @@ This strategy can be enabled by setting `druid.query.scheduler.laning.strategy=h |Property|Description|Default| |--------|-----------|-------| -|`druid.query.scheduler.laning.maxLowPercent`|Maximum percent of the smaller number of`druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, defining the number of HTTP threads that can be used by queries with a priority lower than 0. Value must be between 1 and 100, and will be rounded up|No default, must be set if using this mode| +|`druid.query.scheduler.laning.maxLowPercent`|Maximum percent of the smaller number of`druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, defining the number of HTTP threads that can be used by queries with a priority lower than 0. Value must be in the range 1 to 100, and will be rounded up|No default, must be set if using this mode| ##### Server Configuration diff --git a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java index 7e07bd88d52a..af4e23bddf7f 100644 --- a/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java +++ b/server/src/main/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategy.java @@ -51,8 +51,8 @@ public HiLoQueryLaningStrategy( { this.maxLowPercent = Preconditions.checkNotNull(maxLowPercent, "maxLowPercent must be set"); Preconditions.checkArgument( - 0 < maxLowPercent && maxLowPercent < 100, - "maxLowPercent must be between 0 and 100" + 0 < maxLowPercent && maxLowPercent <= 100, + "maxLowPercent must be in the range 1 to 100" ); } @@ -60,7 +60,7 @@ public HiLoQueryLaningStrategy( public Object2IntMap getLaneLimits(int totalLimit) { Object2IntMap onlyLow = new Object2IntArrayMap<>(1); - onlyLow.put(LOW, Ints.checkedCast(Math.round(totalLimit * ((double) maxLowPercent / 100)))); + onlyLow.put(LOW, Ints.checkedCast((long) Math.ceil(totalLimit * ((double) maxLowPercent / 100)))); return onlyLow; } diff --git a/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java b/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java index c9f2c5378e13..3bc7c3606b1d 100644 --- a/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java +++ b/server/src/test/java/org/apache/druid/server/scheduling/HiLoQueryLaningStrategyTest.java @@ -66,21 +66,65 @@ public void testMaxPercentageThreadsRequired() } @Test - public void testmaxLowPercentMustBeGreaterThanZero() + public void testMaxLowPercentMustBeGreaterThanZero() { expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("maxLowPercent must be between 0 and 100"); + expectedException.expectMessage("maxLowPercent must be in the range 1 to 100"); QueryLaningStrategy strategy = new HiLoQueryLaningStrategy(-1); } + @Test - public void testmaxLowPercentMustBeLessThan100() + public void testMaxLowPercentMustBeLessThanOrEqual100() { expectedException.expect(IllegalArgumentException.class); - expectedException.expectMessage("maxLowPercent must be between 0 and 100"); + expectedException.expectMessage("maxLowPercent must be in the range 1 to 100"); QueryLaningStrategy strategy = new HiLoQueryLaningStrategy(9000); } + @Test + public void testMaxLowPercentZero() + { + expectedException.expect(IllegalArgumentException.class); + expectedException.expectMessage("maxLowPercent must be in the range 1 to 100"); + QueryLaningStrategy strategy = new HiLoQueryLaningStrategy(0); + } + + @Test + public void testMaxLowPercent100() + { + QueryLaningStrategy strategy = new HiLoQueryLaningStrategy(100); + Object2IntMap laneConfig = strategy.getLaneLimits(25); + Assert.assertEquals(1, laneConfig.size()); + Assert.assertTrue(laneConfig.containsKey(HiLoQueryLaningStrategy.LOW)); + Assert.assertEquals(25, laneConfig.getInt(HiLoQueryLaningStrategy.LOW)); + } + + @Test + public void testMaxLowPercentRoundsUp() + { + // will round up to 1 + QueryLaningStrategy strategyRoundLow = new HiLoQueryLaningStrategy(1); + Object2IntMap laneConfigRoundLow = strategyRoundLow.getLaneLimits(25); + Assert.assertEquals(1, laneConfigRoundLow.size()); + Assert.assertTrue(laneConfigRoundLow.containsKey(HiLoQueryLaningStrategy.LOW)); + Assert.assertEquals(1, laneConfigRoundLow.getInt(HiLoQueryLaningStrategy.LOW)); + + // will not round, evenly divides + QueryLaningStrategy strategy = new HiLoQueryLaningStrategy(96); + Object2IntMap laneConfig = strategy.getLaneLimits(25); + Assert.assertEquals(1, laneConfig.size()); + Assert.assertTrue(laneConfig.containsKey(HiLoQueryLaningStrategy.LOW)); + Assert.assertEquals(24, laneConfig.getInt(HiLoQueryLaningStrategy.LOW)); + + // will round up + QueryLaningStrategy strategyRounded = new HiLoQueryLaningStrategy(97); + Object2IntMap laneConfigRounded = strategyRounded.getLaneLimits(25); + Assert.assertEquals(1, laneConfigRounded.size()); + Assert.assertTrue(laneConfigRounded.containsKey(HiLoQueryLaningStrategy.LOW)); + Assert.assertEquals(25, laneConfigRounded.getInt(HiLoQueryLaningStrategy.LOW)); + } + @Test public void testLaneLimits() { From 25a8bda9fa2dd456b23598390c94c1a39e1a70a3 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 9 Mar 2020 17:37:20 -0700 Subject: [PATCH 35/37] fix test --- .../test/java/org/apache/druid/server/QueryResourceTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java index 722a1655e720..a12b69025dd7 100644 --- a/server/src/test/java/org/apache/druid/server/QueryResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/QueryResourceTest.java @@ -717,7 +717,7 @@ public void testTooManyQueryInLane() throws InterruptedException final CountDownLatch waitTwoStarted = new CountDownLatch(2); final CountDownLatch waitOneScheduled = new CountDownLatch(1); final CountDownLatch waitAllFinished = new CountDownLatch(3); - final QueryScheduler scheduler = new QueryScheduler(40, new HiLoQueryLaningStrategy(3), new ServerConfig()); + final QueryScheduler scheduler = new QueryScheduler(40, new HiLoQueryLaningStrategy(1), new ServerConfig()); createScheduledQueryResource(scheduler, ImmutableList.of(waitTwoStarted), ImmutableList.of(waitOneScheduled)); From 32965a4fa92d60c46f4296a5f387b9b29484712b Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 9 Mar 2020 17:43:25 -0700 Subject: [PATCH 36/37] Update docs/configuration/index.md Co-Authored-By: sthetland --- docs/configuration/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration/index.md b/docs/configuration/index.md index d149f9df470d..cee389eb6e8f 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1489,7 +1489,7 @@ The Broker provides facilities to aid in query capacity reservation for heteroge ###### No laning strategy -In this mode, queries are never assigned lane, and concurrent query count will only limited by `druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, if set. This is the default Druid query scheduler operating mode. This strategy can also be explicitly enabled by setting `druid.query.scheduler.laning.strategy` to `none`. +In this mode, queries are never assigned a lane, and the concurrent query count will only be limited by `druid.server.http.numThreads` or `druid.query.scheduler.numThreads`, if set. This is the default Druid query scheduler operating mode. Enable this strategy explicitly by setting `druid.query.scheduler.laning.strategy` to `none`. ###### 'High/Low' laning strategy This laning strategy splits queries with a `priority` below zero into a `low` query lane, automatically. Queries with priority of zero (the default) or above are considered 'interactive'. The limit on `low` queries can be set to some desired percentage of the total capacity (or HTTP thread pool size), reserving capacity for interactive queries. Queries in the `low` lane are _not_ guaranteed their capacity, which may be consumed by interactive queries, but may use up to this limit if total capacity is available. From 3ba7808e32bdeeef603fa3e4b6665bd927ce9bb8 Mon Sep 17 00:00:00 2001 From: Clint Wylie Date: Mon, 9 Mar 2020 17:46:04 -0700 Subject: [PATCH 37/37] doc update --- docs/configuration/index.md | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/docs/configuration/index.md b/docs/configuration/index.md index fabbcefbdfdc..7f95e4b858e5 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -1483,7 +1483,7 @@ These Broker configurations can be defined in the `broker/runtime.properties` fi ##### Query laning -The Broker provides facilities to aid in query capacity reservation for heterogeneous query workloads in the form of 'laning' strategies, which provide a variety of mechanisms to examine and classify a query, assigning it to a 'lane'. Lanes are defined with capacity limits which the broker will enforce, causing requests in excess of the capacity to be discarded with an HTTP 429 status code, reserving resources for other lanes or for interactive queries (with no lane). +*Laning strategies* allow you to control capacity utilization for heterogeneous query workloads. With laning, the broker examines and classifies a query for the purpose of assigning it to a 'lane'. Lanes have capacity limits, enforced by the broker, that can be used to ensure sufficient resources are available for other lanes or for interactive queries (with no lane), or to limit overall throughput for queries within the lane. Requests in excess of the capacity are discarded with an HTTP 429 status code. |Property|Description|Default| |--------|-----------|-------|