clintropolis commented on a change in pull request #9407: query laning and load shedding URL: https://github.com/apache/druid/pull/9407#discussion_r388761914
########## File path: server/src/test/java/org/apache/druid/server/QuerySchedulerTest.java ########## @@ -0,0 +1,566 @@ +/* + * 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.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 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.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; +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.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.Properties; +import java.util.UUID; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.atomic.AtomicLong; + +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; + + private AtomicLong totalAcquired; + private AtomicLong totalReleased; + private AtomicLong laneAcquired; + private AtomicLong laneNotAcquired; + private AtomicLong laneReleased; + + @Before + public void setup() + { + executorService = MoreExecutors.listeningDecorator( + Execs.multiThreaded(8, "test_query_scheduler_%s") + ); + totalAcquired = new AtomicLong(); + totalReleased = new AtomicLong(); + laneAcquired = new AtomicLong(); + laneNotAcquired = new AtomicLong(); + laneReleased = new AtomicLong(); + scheduler = new QueryScheduler(5, new HiLoQueryLaningStrategy(2)) { + @Override + List<Bulkhead> acquireLanes(Query<?> query) + { + List<Bulkhead> 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 + void releaseLanes(List<Bulkhead> bulkheads) + { + super.releaseLanes(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(); + if (bulkheads.size() == 1) { + laneNotAcquired.incrementAndGet(); + } + } + } + }; + } + + @After + public void teardown() + { + executorService.shutdownNow(); + } + + @Test + public void testHiLoHi() throws ExecutionException, InterruptedException + { + TopNQuery interactive = makeInteractiveQuery(); + ListenableFuture<?> future = executorService.submit(() -> { + try { + Query<?> scheduled = scheduler.laneQuery(QueryPlus.wrap(interactive), ImmutableSet.of()); + + Assert.assertNotNull(scheduled); + + Sequence<Integer> underlyingSequence = makeSequence(10); + underlyingSequence = Sequences.wrap(underlyingSequence, new SequenceWrapper() + { + @Override + public void before() + { + Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); + Assert.assertEquals(2, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); + } + }); + Sequence<Integer> results = scheduler.run(scheduled, underlyingSequence); + int rowCount = consumeAndCloseSequence(results); + + Assert.assertEquals(10, rowCount); + } + catch (IOException ex) { + throw new RuntimeException(ex); + } + }); + future.get(); + Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); + } + + @Test + public void testHiLoLo() throws ExecutionException, InterruptedException + { + TopNQuery report = makeReportQuery(); + ListenableFuture<?> future = executorService.submit(() -> { + try { + Query<?> scheduledReport = scheduler.laneQuery(QueryPlus.wrap(report), ImmutableSet.of()); + Assert.assertNotNull(scheduledReport); + Assert.assertEquals(HiLoQueryLaningStrategy.LOW, QueryContexts.getLane(scheduledReport)); + + Sequence<Integer> underlyingSequence = makeSequence(10); + underlyingSequence = Sequences.wrap(underlyingSequence, new SequenceWrapper() + { + @Override + public void before() + { + Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); + Assert.assertEquals(1, scheduler.getLaneAvailableCapacity(HiLoQueryLaningStrategy.LOW)); + } + }); + Sequence<Integer> results = scheduler.run(scheduledReport, underlyingSequence); + + int rowCount = consumeAndCloseSequence(results); + Assert.assertEquals(10, rowCount); + } + catch (IOException ex) { + throw new RuntimeException(ex); + } + }); + 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); + TopNQuery interactive = makeInteractiveQuery(); + ListenableFuture<?> future = executorService.submit(() -> { + try { + Query<?> scheduled = scheduler.laneQuery(QueryPlus.wrap(interactive), ImmutableSet.of()); + + Assert.assertNotNull(scheduled); + + Sequence<Integer> underlyingSequence = makeExplodingSequence(10); + underlyingSequence = Sequences.wrap(underlyingSequence, new SequenceWrapper() + { + @Override + public void before() + { + Assert.assertEquals(4, scheduler.getTotalAvailableCapacity()); + } + }); + Sequence<Integer> results = scheduler.run(scheduled, underlyingSequence); + + consumeAndCloseSequence(results); + } + catch (IOException ex) { + throw new RuntimeException(ex); + } + }); + future.get(); + Assert.assertEquals(5, scheduler.getTotalAvailableCapacity()); Review comment: Ah, i think that was a leftover from copying another test to make this one, but I think it does seem useful to assert that the token was released after the exception so will change ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on to GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: [email protected] With regards, Apache Git Services --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
