DefaultDeclarativeSlotPoolTest.java 31.8 KB
Newer Older
1 2 3 4 5 6 7 8 9 10 11 12 13 14 15 16 17 18 19 20 21 22 23 24 25 26 27 28 29 30 31 32 33 34 35 36 37 38 39 40 41 42 43 44 45 46 47 48 49 50 51 52 53 54 55 56 57 58 59 60 61 62 63 64 65 66 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.flink.runtime.jobmaster.slotpool;

import org.apache.flink.api.common.time.Time;
import org.apache.flink.runtime.clusterframework.types.AllocationID;
import org.apache.flink.runtime.clusterframework.types.ResourceProfile;
import org.apache.flink.runtime.jobmanager.slots.TaskManagerGateway;
import org.apache.flink.runtime.jobmaster.JobMasterId;
import org.apache.flink.runtime.jobmaster.RpcTaskManagerGateway;
import org.apache.flink.runtime.messages.Acknowledge;
import org.apache.flink.runtime.slots.ResourceRequirement;
import org.apache.flink.runtime.taskexecutor.TaskExecutorGateway;
import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGateway;
import org.apache.flink.runtime.taskexecutor.TestingTaskExecutorGatewayBuilder;
import org.apache.flink.runtime.taskexecutor.slot.SlotOffer;
import org.apache.flink.runtime.taskmanager.LocalTaskManagerLocation;
import org.apache.flink.util.FlinkException;
import org.apache.flink.util.TestLogger;

import org.apache.flink.shaded.guava18.com.google.common.collect.Iterables;

import org.hamcrest.Description;
import org.hamcrest.TypeSafeMatcher;
import org.junit.Test;

import javax.annotation.Nonnull;
import javax.annotation.Nullable;

import java.util.ArrayList;
import java.util.Collection;
import java.util.HashMap;
import java.util.Map;
import java.util.concurrent.ArrayBlockingQueue;
import java.util.concurrent.BlockingQueue;
import java.util.concurrent.CompletableFuture;
import java.util.function.BiFunction;
import java.util.function.Consumer;
import java.util.function.Function;
import java.util.stream.Collectors;

import static org.hamcrest.CoreMatchers.hasItems;
import static org.hamcrest.CoreMatchers.is;
import static org.hamcrest.CoreMatchers.not;
import static org.hamcrest.CoreMatchers.sameInstance;
import static org.hamcrest.Matchers.containsInAnyOrder;
import static org.hamcrest.Matchers.empty;
import static org.junit.Assert.assertFalse;
import static org.junit.Assert.assertThat;
import static org.junit.Assert.assertTrue;

68
/** Tests for the {@link DefaultDeclarativeSlotPool}. */
69 70
public class DefaultDeclarativeSlotPoolTest extends TestLogger {

71 72 73 74 75 76 77 78 79 80 81 82 83 84 85 86
    private static final ResourceProfile RESOURCE_PROFILE_1 =
            ResourceProfile.newBuilder().setCpuCores(1.7).build();
    private static final ResourceProfile RESOURCE_PROFILE_2 =
            ResourceProfile.newBuilder().setManagedMemoryMB(100).build();

    @Test
    public void testIncreasingResourceRequirementsWillSendResourceRequirementNotification()
            throws InterruptedException {
        final NewResourceRequirementsService requirementsListener =
                new NewResourceRequirementsService();
        final DeclarativeSlotPool slotPool = createDefaultDeclarativeSlotPool(requirementsListener);

        final ResourceCounter increment1 = ResourceCounter.withResource(RESOURCE_PROFILE_1, 1);
        final ResourceCounter increment2 = createResourceRequirements();
        slotPool.increaseResourceRequirementsBy(increment1);
        slotPool.increaseResourceRequirementsBy(increment2);
87

88 89 90
        assertThat(
                requirementsListener.takeResourceRequirements(),
                is(toResourceRequirements(increment1)));
91

92 93 94 95 96 97
        final ResourceCounter totalResources = increment1.add(increment2);
        assertThat(
                requirementsListener.takeResourceRequirements(),
                is(toResourceRequirements(totalResources)));
        assertThat(requirementsListener.hasNextResourceRequirements(), is(false));
    }
98

99 100 101 102 103 104 105
    @Test
    public void testDecreasingResourceRequirementsWillSendResourceRequirementNotification()
            throws InterruptedException {
        final NewResourceRequirementsService requirementsListener =
                new NewResourceRequirementsService();
        final DefaultDeclarativeSlotPool slotPool =
                createDefaultDeclarativeSlotPool(requirementsListener);
106

107 108
        final ResourceCounter increment = ResourceCounter.withResource(RESOURCE_PROFILE_1, 3);
        slotPool.increaseResourceRequirementsBy(increment);
109

110
        requirementsListener.takeResourceRequirements();
111

112 113
        final ResourceCounter decrement = ResourceCounter.withResource(RESOURCE_PROFILE_1, 2);
        slotPool.decreaseResourceRequirementsBy(decrement);
114

115 116 117 118 119 120
        final ResourceCounter totalResources = increment.subtract(decrement);
        assertThat(
                requirementsListener.takeResourceRequirements(),
                is(toResourceRequirements(totalResources)));
        assertThat(requirementsListener.hasNextResourceRequirements(), is(false));
    }
121

122 123 124 125
    @Test
    public void testGetResourceRequirements() {
        final DefaultDeclarativeSlotPool slotPool =
                DefaultDeclarativeSlotPoolBuilder.builder().build();
126

127 128 129
        assertThat(
                slotPool.getResourceRequirements(),
                is(toResourceRequirements(ResourceCounter.empty())));
130

131
        final ResourceCounter resourceRequirements = createResourceRequirements();
132

133
        slotPool.increaseResourceRequirementsBy(resourceRequirements);
134

135 136 137 138
        assertThat(
                slotPool.getResourceRequirements(),
                is(toResourceRequirements(resourceRequirements)));
    }
139

140 141 142 143
    @Test
    public void testOfferSlots() throws InterruptedException {
        final NewSlotsService notifyNewSlots = new NewSlotsService();
        final DefaultDeclarativeSlotPool slotPool =
144
                createDefaultDeclarativeSlotPoolWithNewSlotsListener(notifyNewSlots);
145

146
        final ResourceCounter resourceRequirements = createResourceRequirements();
147

148 149 150 151
        slotPool.increaseResourceRequirementsBy(resourceRequirements);

        Collection<SlotOffer> slotOffers =
                createSlotOffersForResourceRequirements(resourceRequirements);
152

153
        final Collection<SlotOffer> acceptedSlots = offerSlots(slotPool, slotOffers);
154

155
        assertThat(acceptedSlots, containsInAnyOrder(slotOffers.toArray()));
156

157 158 159 160 161 162 163 164 165 166 167 168 169 170 171 172 173 174 175 176
        final Collection<PhysicalSlot> newSlots = drainNewSlotService(notifyNewSlots);

        assertThat(
                newSlots,
                containsInAnyOrder(
                        slotOffers.stream()
                                .map(DefaultDeclarativeSlotPoolTest::matchesSlotOffer)
                                .collect(Collectors.toList())));
        assertThat(
                slotPool.getAllSlotsInformation(),
                containsInAnyOrder(
                        newSlots.stream()
                                .map(DefaultAllocatedSlotPoolTest::matchesPhysicalSlot)
                                .collect(Collectors.toList())));
    }

    @Test
    public void testDuplicateSlotOfferings() throws InterruptedException {
        final NewSlotsService notifyNewSlots = new NewSlotsService();
        final DefaultDeclarativeSlotPool slotPool =
177
                createDefaultDeclarativeSlotPoolWithNewSlotsListener(notifyNewSlots);
178 179 180 181 182 183 184 185 186

        final ResourceCounter resourceRequirements = createResourceRequirements();

        slotPool.increaseResourceRequirementsBy(resourceRequirements);

        final Collection<SlotOffer> slotOffers =
                createSlotOffersForResourceRequirements(resourceRequirements);

        offerSlots(slotPool, slotOffers);
187

188
        drainNewSlotService(notifyNewSlots);
189

190
        final Collection<SlotOffer> acceptedSlots = offerSlots(slotPool, slotOffers);
191

192 193 194 195
        assertThat(acceptedSlots, containsInAnyOrder(slotOffers.toArray()));
        // duplicate slots should not trigger notify new slots
        assertFalse(notifyNewSlots.hasNextNewSlots());
    }
196

197 198 199 200
    @Test
    public void testOfferingTooManySlots() {
        final NewSlotsService notifyNewSlots = new NewSlotsService();
        final DefaultDeclarativeSlotPool slotPool =
201
                createDefaultDeclarativeSlotPoolWithNewSlotsListener(notifyNewSlots);
202

203
        final ResourceCounter resourceRequirements = createResourceRequirements();
204

205
        slotPool.increaseResourceRequirementsBy(resourceRequirements);
206

207 208
        final ResourceCounter increasedRequirements =
                resourceRequirements.add(RESOURCE_PROFILE_1, 2);
209

210 211
        final Collection<SlotOffer> slotOffers =
                createSlotOffersForResourceRequirements(increasedRequirements);
212

213
        final Collection<SlotOffer> acceptedSlots = offerSlots(slotPool, slotOffers);
214

215 216 217 218
        final Map<ResourceProfile, Long> resourceProfileCount =
                acceptedSlots.stream()
                        .map(SlotOffer::getResourceProfile)
                        .collect(Collectors.groupingBy(Function.identity(), Collectors.counting()));
219

220 221 222 223 224 225 226
        for (Map.Entry<ResourceProfile, Integer> resourceCount :
                resourceRequirements.getResourcesWithCount()) {
            assertThat(
                    resourceProfileCount.getOrDefault(resourceCount.getKey(), 0L),
                    is((long) resourceCount.getValue()));
        }
    }
227

228 229 230 231 232 233
    @Test
    public void testReleaseSlotsRemovesSlots() throws InterruptedException {
        final NewResourceRequirementsService notifyNewResourceRequirements =
                new NewResourceRequirementsService();
        final DefaultDeclarativeSlotPool slotPool =
                createDefaultDeclarativeSlotPool(notifyNewResourceRequirements);
234

235 236 237
        final LocalTaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation();
        increaseRequirementsAndOfferSlotsToSlotPool(
                slotPool, createResourceRequirements(), taskManagerLocation);
238

239
        notifyNewResourceRequirements.takeResourceRequirements();
240

241 242 243 244
        slotPool.releaseSlots(
                taskManagerLocation.getResourceID(), new FlinkException("Test failure"));
        assertThat(slotPool.getAllSlotsInformation(), is(empty()));
    }
245

246 247 248 249 250 251 252 253 254 255 256 257 258 259 260 261 262 263 264 265 266 267 268 269 270
    @Test
    public void testReleaseSlotsReturnsSlot() {
        final DefaultDeclarativeSlotPool slotPool =
                DefaultDeclarativeSlotPoolBuilder.builder().build();

        final ResourceCounter resourceRequirements = createResourceRequirements();

        final LocalTaskManagerLocation taskManagerLocation = new LocalTaskManagerLocation();
        final FreeSlotConsumer freeSlotConsumer = new FreeSlotConsumer();
        final TestingTaskExecutorGateway testingTaskExecutorGateway =
                new TestingTaskExecutorGatewayBuilder()
                        .setFreeSlotFunction(freeSlotConsumer)
                        .createTestingTaskExecutorGateway();

        final Collection<SlotOffer> slotOffers =
                increaseRequirementsAndOfferSlotsToSlotPool(
                        slotPool,
                        resourceRequirements,
                        taskManagerLocation,
                        testingTaskExecutorGateway);

        slotPool.releaseSlots(
                taskManagerLocation.getResourceID(), new FlinkException("Test failure"));

        final Collection<AllocationID> freedSlots = freeSlotConsumer.drainFreedSlots();
271

272 273 274 275 276 277 278 279 280 281
        assertThat(
                freedSlots,
                containsInAnyOrder(slotOffers.stream().map(SlotOffer::getAllocationId).toArray()));
    }

    @Test
    public void testReleaseSlotDecreasesFulfilledResourceRequirements()
            throws InterruptedException {
        final NewSlotsService notifyNewSlots = new NewSlotsService();
        final DefaultDeclarativeSlotPool slotPool =
282
                createDefaultDeclarativeSlotPoolWithNewSlotsListener(notifyNewSlots);
283 284 285 286 287 288 289 290 291 292 293 294 295 296 297 298 299 300 301

        final ResourceCounter resourceRequirements = createResourceRequirements();
        increaseRequirementsAndOfferSlotsToSlotPool(slotPool, resourceRequirements, null);

        final Collection<? extends PhysicalSlot> physicalSlots = notifyNewSlots.takeNewSlots();

        final PhysicalSlot physicalSlot = physicalSlots.iterator().next();

        slotPool.releaseSlot(physicalSlot.getAllocationId(), new FlinkException("Test failure"));

        final ResourceCounter finalResourceRequirements =
                resourceRequirements.subtract(physicalSlot.getResourceProfile(), 1);
        assertThat(slotPool.getFulfilledResourceRequirements(), is(finalResourceRequirements));
    }

    @Test
    public void testReleaseSlotReturnsSlot() throws InterruptedException {
        final NewSlotsService notifyNewSlots = new NewSlotsService();
        final DefaultDeclarativeSlotPool slotPool =
302
                createDefaultDeclarativeSlotPoolWithNewSlotsListener(notifyNewSlots);
303 304 305 306 307 308 309 310 311 312 313 314 315 316 317 318 319 320 321 322 323 324 325 326 327 328 329 330 331 332 333 334 335 336 337 338 339 340 341 342 343 344 345 346 347 348 349 350 351 352 353 354 355 356 357 358 359 360 361 362 363 364 365 366 367 368 369 370 371 372 373 374 375 376 377 378 379 380 381 382 383 384 385 386 387 388 389 390 391 392 393 394 395 396 397 398 399 400 401 402 403

        final ResourceCounter resourceRequirements = createResourceRequirements();
        final FreeSlotConsumer freeSlotConsumer = new FreeSlotConsumer();
        final TestingTaskExecutorGateway testingTaskExecutorGateway =
                new TestingTaskExecutorGatewayBuilder()
                        .setFreeSlotFunction(freeSlotConsumer)
                        .createTestingTaskExecutorGateway();

        increaseRequirementsAndOfferSlotsToSlotPool(
                slotPool,
                resourceRequirements,
                new LocalTaskManagerLocation(),
                testingTaskExecutorGateway);

        final Collection<? extends PhysicalSlot> physicalSlots = notifyNewSlots.takeNewSlots();

        final PhysicalSlot physicalSlot = physicalSlots.iterator().next();

        slotPool.releaseSlot(physicalSlot.getAllocationId(), new FlinkException("Test failure"));

        final AllocationID freedSlot = Iterables.getOnlyElement(freeSlotConsumer.drainFreedSlots());

        assertThat(freedSlot, is(physicalSlot.getAllocationId()));
    }

    @Test
    public void testReturnIdleSlotsAfterTimeout() {
        final Time idleSlotTimeout = Time.seconds(10);
        final long offerTime = 0;
        final DefaultDeclarativeSlotPool slotPool =
                DefaultDeclarativeSlotPoolBuilder.builder()
                        .setIdleSlotTimeout(idleSlotTimeout)
                        .build();

        final ResourceCounter resourceRequirements = createResourceRequirements();
        final FreeSlotConsumer freeSlotConsumer = new FreeSlotConsumer();
        final TestingTaskExecutorGateway testingTaskExecutorGateway =
                new TestingTaskExecutorGatewayBuilder()
                        .setFreeSlotFunction(freeSlotConsumer)
                        .createTestingTaskExecutorGateway();

        final Collection<SlotOffer> acceptedSlots =
                increaseRequirementsAndOfferSlotsToSlotPool(
                        slotPool,
                        resourceRequirements,
                        new LocalTaskManagerLocation(),
                        testingTaskExecutorGateway);

        // decrease the resource requirements so that slots are no longer needed
        slotPool.decreaseResourceRequirementsBy(resourceRequirements);

        slotPool.releaseIdleSlots(offerTime + idleSlotTimeout.toMilliseconds());

        final Collection<AllocationID> freedSlots = freeSlotConsumer.drainFreedSlots();

        assertThat(acceptedSlots, is(not(empty())));
        assertThat(
                freedSlots,
                containsInAnyOrder(
                        acceptedSlots.stream().map(SlotOffer::getAllocationId).toArray()));
        assertNoAvailableAndRequiredResources(slotPool);
    }

    private void assertNoAvailableAndRequiredResources(DefaultDeclarativeSlotPool slotPool) {
        assertTrue(slotPool.getFulfilledResourceRequirements().isEmpty());
        assertTrue(slotPool.getResourceRequirements().isEmpty());
        assertThat(slotPool.getAllSlotsInformation(), is(empty()));
    }

    @Test
    public void testOnlyReturnExcessIdleSlots() {
        final Time idleSlotTimeout = Time.seconds(10);
        final long offerTime = 0;
        final DefaultDeclarativeSlotPool slotPool =
                DefaultDeclarativeSlotPoolBuilder.builder()
                        .setIdleSlotTimeout(idleSlotTimeout)
                        .build();

        final ResourceCounter resourceRequirements = createResourceRequirements();
        final Collection<SlotOffer> slotOffers =
                createSlotOffersForResourceRequirements(resourceRequirements);

        slotPool.increaseResourceRequirementsBy(resourceRequirements);
        final Collection<SlotOffer> acceptedSlots = offerSlots(slotPool, slotOffers);

        final ResourceCounter requiredResources =
                ResourceCounter.withResource(RESOURCE_PROFILE_1, 1);
        final ResourceCounter excessRequirements = resourceRequirements.subtract(requiredResources);
        slotPool.decreaseResourceRequirementsBy(excessRequirements);

        slotPool.releaseIdleSlots(offerTime + idleSlotTimeout.toMilliseconds());

        assertThat(acceptedSlots, is(not(empty())));
        assertThat(slotPool.getFulfilledResourceRequirements(), is(requiredResources));
    }

    @Test
    public void testFreedSlotWillBeUsedToFulfillOutstandingResourceRequirementsOfSameProfile()
            throws InterruptedException {
        final NewSlotsService notifyNewSlots = new NewSlotsService();
        final DefaultDeclarativeSlotPool slotPool =
404
                createDefaultDeclarativeSlotPoolWithNewSlotsListener(notifyNewSlots);
405 406 407 408 409 410 411 412 413 414 415 416 417 418 419 420 421 422 423 424 425 426 427 428 429 430 431 432 433 434 435 436 437 438 439 440 441 442 443 444 445 446 447 448 449 450 451 452 453 454 455 456 457 458 459 460 461 462 463 464 465 466 467 468 469 470 471 472 473 474 475 476 477 478 479 480 481 482 483 484 485 486 487 488 489 490 491 492 493 494 495 496 497 498 499 500 501 502 503 504 505 506 507 508 509 510 511 512 513 514 515 516 517

        final ResourceCounter initialRequirements =
                ResourceCounter.withResource(RESOURCE_PROFILE_1, 1);

        increaseRequirementsAndOfferSlotsToSlotPool(slotPool, initialRequirements, null);

        final Collection<PhysicalSlot> newSlots = drainNewSlotService(notifyNewSlots);
        final PhysicalSlot newSlot = Iterables.getOnlyElement(newSlots);

        slotPool.reserveFreeSlot(newSlot.getAllocationId(), RESOURCE_PROFILE_1);
        slotPool.freeReservedSlot(newSlot.getAllocationId(), null, 0);

        final Collection<PhysicalSlot> recycledSlots = drainNewSlotService(notifyNewSlots);

        assertThat(Iterables.getOnlyElement(recycledSlots), sameInstance(newSlot));

        final Collection<SlotOffer> newSlotOffers =
                createSlotOffersForResourceRequirements(initialRequirements);

        // the pending requirement should be fulfilled by the freed slot --> rejecting new slot
        // offers
        final Collection<SlotOffer> acceptedSlots =
                slotPool.offerSlots(
                        newSlotOffers,
                        new LocalTaskManagerLocation(),
                        createTaskManagerGateway(null),
                        0);

        assertThat(acceptedSlots, is(empty()));
        assertTrue(slotPool.calculateUnfulfilledResources().isEmpty());
    }

    @Test
    public void testFreedSlotWillRemainAssignedToMatchedResourceProfile() {
        final DefaultDeclarativeSlotPool slotPool = new DefaultDeclarativeSlotPoolBuilder().build();

        final ResourceProfile largeResourceProfile =
                ResourceProfile.newBuilder().setManagedMemoryMB(1024).build();
        final ResourceProfile smallResourceProfile =
                ResourceProfile.newBuilder().setManagedMemoryMB(512).build();

        slotPool.increaseResourceRequirementsBy(
                ResourceCounter.withResource(largeResourceProfile, 1));
        offerSlots(
                slotPool,
                createSlotOffersForResourceRequirements(
                        ResourceCounter.withResource(ResourceProfile.ANY, 1)));

        final SlotInfoWithUtilization slot = slotPool.getFreeSlotsInformation().iterator().next();

        slotPool.reserveFreeSlot(slot.getAllocationId(), largeResourceProfile);
        assertThat(
                slotPool.getFulfilledResourceRequirements().getResourceCount(largeResourceProfile),
                is(1));

        slotPool.increaseResourceRequirementsBy(
                ResourceCounter.withResource(smallResourceProfile, 1));
        slotPool.decreaseResourceRequirementsBy(
                ResourceCounter.withResource(largeResourceProfile, 1));

        // free the slot; this should not cause the slot to be automatically re-matched to the small
        // resource profile
        // this is currently the responsibility of the user, by reserving the slot for a different
        // profile
        slotPool.freeReservedSlot(slot.getAllocationId(), null, 1);
        assertThat(
                slotPool.getFulfilledResourceRequirements().getResourceCount(largeResourceProfile),
                is(1));
        assertThat(
                slotPool.getFulfilledResourceRequirements().getResourceCount(smallResourceProfile),
                is(0));
    }

    @Test
    public void testReserveFreeSlotForResourceUpdatesAvailableResourcesAndRequirements() {
        final DefaultDeclarativeSlotPool slotPool = new DefaultDeclarativeSlotPoolBuilder().build();

        final ResourceProfile largeResourceProfile =
                ResourceProfile.newBuilder().setManagedMemoryMB(1024).build();
        final ResourceProfile smallResourceProfile =
                ResourceProfile.newBuilder().setManagedMemoryMB(512).build();

        slotPool.increaseResourceRequirementsBy(
                ResourceCounter.withResource(largeResourceProfile, 1));
        offerSlots(
                slotPool,
                createSlotOffersForResourceRequirements(
                        ResourceCounter.withResource(largeResourceProfile, 1)));
        slotPool.increaseResourceRequirementsBy(
                ResourceCounter.withResource(smallResourceProfile, 1));

        final SlotInfoWithUtilization largeSlot =
                slotPool.getFreeSlotsInformation().stream()
                        .filter(slot -> slot.getResourceProfile().equals(largeResourceProfile))
                        .findFirst()
                        .get();

        slotPool.reserveFreeSlot(largeSlot.getAllocationId(), smallResourceProfile);

        ResourceCounter availableResources = slotPool.getFulfilledResourceRequirements();
        assertThat(availableResources.getResourceCount(smallResourceProfile), is(1));
        assertThat(availableResources.getResourceCount(largeResourceProfile), is(0));

        Collection<ResourceRequirement> currentResourceRequirements =
                slotPool.getResourceRequirements();
        // since we used one of the large slots for fulfilling another profile, we now need another
        // large slot for fulfill the original requirement
        // conversely we no longer need the small slot, because we are now using another slot for it
        assertThat(
                currentResourceRequirements,
                hasItems(ResourceRequirement.create(largeResourceProfile, 2)));
    }

518 519 520 521 522 523 524 525 526 527 528 529 530 531 532 533 534 535 536 537 538 539 540 541 542 543 544 545 546
    @Test
    public void testSetResourceRequirementsForInitialResourceRequirements() {
        final DefaultDeclarativeSlotPool slotPool = new DefaultDeclarativeSlotPoolBuilder().build();

        final ResourceCounter resourceRequirements =
                ResourceCounter.withResource(RESOURCE_PROFILE_1, 2);

        slotPool.setResourceRequirements(resourceRequirements);

        assertThat(
                slotPool.getResourceRequirements(),
                is(toResourceRequirements(resourceRequirements)));
    }

    @Test
    public void testSetResourceRequirementsOverwritesPreviousValue() {
        final DefaultDeclarativeSlotPool slotPool = new DefaultDeclarativeSlotPoolBuilder().build();

        slotPool.setResourceRequirements(ResourceCounter.withResource(RESOURCE_PROFILE_1, 1));

        final ResourceCounter resourceRequirements =
                ResourceCounter.withResource(RESOURCE_PROFILE_2, 1);
        slotPool.setResourceRequirements(resourceRequirements);

        assertThat(
                slotPool.getResourceRequirements(),
                is(toResourceRequirements(resourceRequirements)));
    }

547 548 549 550 551 552 553 554 555 556 557 558 559 560 561 562 563 564 565 566 567 568 569 570 571 572 573 574 575 576 577 578 579 580 581 582 583 584 585 586 587 588 589 590 591 592 593 594 595 596 597
    @Nonnull
    private static ResourceCounter createResourceRequirements() {
        final Map<ResourceProfile, Integer> requirements = new HashMap<>();
        requirements.put(RESOURCE_PROFILE_1, 2);
        requirements.put(RESOURCE_PROFILE_2, 1);

        return ResourceCounter.withResources(requirements);
    }

    @Nonnull
    private static Collection<SlotOffer> createSlotOffersForResourceRequirements(
            ResourceCounter resourceRequirements) {
        Collection<SlotOffer> slotOffers = new ArrayList<>();
        int slotIndex = 0;

        for (Map.Entry<ResourceProfile, Integer> resourceWithCount :
                resourceRequirements.getResourcesWithCount()) {
            for (int i = 0; i < resourceWithCount.getValue(); i++) {
                ResourceProfile slotProfile = resourceWithCount.getKey();
                slotOffers.add(
                        new SlotOffer(
                                new AllocationID(),
                                slotIndex++,
                                slotProfile == ResourceProfile.UNKNOWN
                                        ? ResourceProfile.ANY
                                        : slotProfile));
            }
        }
        return slotOffers;
    }

    @Nonnull
    private static Collection<ResourceRequirement> toResourceRequirements(
            ResourceCounter resourceCounter) {
        return resourceCounter.getResourcesWithCount().stream()
                .map(
                        resourceCount ->
                                ResourceRequirement.create(
                                        resourceCount.getKey(), resourceCount.getValue()))
                .collect(Collectors.toList());
    }

    @Nonnull
    private static DefaultDeclarativeSlotPool createDefaultDeclarativeSlotPool(
            NewResourceRequirementsService requirementsListener) {
        return DefaultDeclarativeSlotPoolBuilder.builder()
                .setNotifyNewResourceRequirements(requirementsListener)
                .build();
    }

    @Nonnull
598 599 600 601 602 603 604 605 606 607 608
    private static DefaultDeclarativeSlotPool createDefaultDeclarativeSlotPoolWithNewSlotsListener(
            DeclarativeSlotPool.NewSlotsListener newSlotsListener) {
        final DefaultDeclarativeSlotPool declarativeSlotPool = createDefaultDeclarativeSlotPool();

        declarativeSlotPool.registerNewSlotsListener(newSlotsListener);
        return declarativeSlotPool;
    }

    @Nonnull
    private static DefaultDeclarativeSlotPool createDefaultDeclarativeSlotPool() {
        return DefaultDeclarativeSlotPoolBuilder.builder().build();
609 610 611 612 613 614 615 616 617 618 619 620 621 622 623 624 625 626 627 628 629 630 631 632 633 634 635 636 637 638 639 640 641 642 643 644 645 646 647 648 649 650 651 652 653 654 655 656 657 658 659 660 661 662 663 664 665 666 667 668 669 670 671 672 673 674 675 676 677 678 679 680 681 682 683 684 685 686 687 688 689
    }

    @Nonnull
    static Collection<SlotOffer> offerSlots(
            DeclarativeSlotPool slotPool, Collection<SlotOffer> slotOffers) {
        return slotPool.offerSlots(
                slotOffers, new LocalTaskManagerLocation(), createTaskManagerGateway(null), 0);
    }

    @Nonnull
    private static Collection<SlotOffer> increaseRequirementsAndOfferSlotsToSlotPool(
            DefaultDeclarativeSlotPool slotPool,
            ResourceCounter resourceRequirements,
            @Nullable LocalTaskManagerLocation taskManagerLocation) {
        return increaseRequirementsAndOfferSlotsToSlotPool(
                slotPool, resourceRequirements, taskManagerLocation, null);
    }

    @Nonnull
    private static Collection<SlotOffer> increaseRequirementsAndOfferSlotsToSlotPool(
            DefaultDeclarativeSlotPool slotPool,
            ResourceCounter resourceRequirements,
            @Nullable LocalTaskManagerLocation taskManagerLocation,
            @Nullable TaskExecutorGateway taskExecutorGateway) {
        final Collection<SlotOffer> slotOffers =
                createSlotOffersForResourceRequirements(resourceRequirements);

        slotPool.increaseResourceRequirementsBy(resourceRequirements);

        return slotPool.offerSlots(
                slotOffers,
                taskManagerLocation == null ? new LocalTaskManagerLocation() : taskManagerLocation,
                createTaskManagerGateway(taskExecutorGateway),
                0);
    }

    @Nonnull
    private static Collection<PhysicalSlot> drainNewSlotService(NewSlotsService notifyNewSlots)
            throws InterruptedException {
        final Collection<PhysicalSlot> newSlots = new ArrayList<>();

        while (notifyNewSlots.hasNextNewSlots()) {
            newSlots.addAll(notifyNewSlots.takeNewSlots());
        }
        return newSlots;
    }

    private static TypeSafeMatcher<PhysicalSlot> matchesSlotOffer(SlotOffer slotOffer) {
        return new PhysicalSlotSlotOfferMatcher(slotOffer);
    }

    private static TaskManagerGateway createTaskManagerGateway(
            @Nullable TaskExecutorGateway taskExecutorGateway) {
        return new RpcTaskManagerGateway(
                taskExecutorGateway == null
                        ? new TestingTaskExecutorGatewayBuilder().createTestingTaskExecutorGateway()
                        : taskExecutorGateway,
                JobMasterId.generate());
    }

    private static final class NewResourceRequirementsService
            implements Consumer<Collection<ResourceRequirement>> {

        private final BlockingQueue<Collection<ResourceRequirement>> resourceRequirementsQueue =
                new ArrayBlockingQueue<>(2);

        @Override
        public void accept(Collection<ResourceRequirement> resourceRequirements) {
            resourceRequirementsQueue.offer(resourceRequirements);
        }

        private Collection<ResourceRequirement> takeResourceRequirements()
                throws InterruptedException {
            return resourceRequirementsQueue.take();
        }

        public boolean hasNextResourceRequirements() {
            return !resourceRequirementsQueue.isEmpty();
        }
    }

690
    private static final class NewSlotsService implements DeclarativeSlotPool.NewSlotsListener {
691 692 693 694 695 696 697 698 699 700 701

        private final BlockingQueue<Collection<? extends PhysicalSlot>> physicalSlotsQueue =
                new ArrayBlockingQueue<>(2);

        private Collection<? extends PhysicalSlot> takeNewSlots() throws InterruptedException {
            return physicalSlotsQueue.take();
        }

        private boolean hasNextNewSlots() {
            return !physicalSlotsQueue.isEmpty();
        }
702 703 704 705 706 707

        @Override
        public void notifyNewSlotsAreAvailable(
                Collection<? extends PhysicalSlot> newlyAvailableSlots) {
            physicalSlotsQueue.offer(newlyAvailableSlots);
        }
708 709 710 711 712 713 714 715 716 717 718 719 720 721 722 723 724 725 726 727 728 729 730 731 732 733 734 735 736 737 738 739 740 741 742 743 744 745 746 747 748 749 750 751 752 753 754 755 756
    }

    private static class PhysicalSlotSlotOfferMatcher extends TypeSafeMatcher<PhysicalSlot> {
        private final SlotOffer slotOffer;

        public PhysicalSlotSlotOfferMatcher(SlotOffer slotOffer) {
            this.slotOffer = slotOffer;
        }

        @Override
        protected boolean matchesSafely(PhysicalSlot item) {
            return item.getAllocationId().equals(slotOffer.getAllocationId())
                    && item.getResourceProfile().equals(slotOffer.getResourceProfile())
                    && item.getPhysicalSlotNumber() == slotOffer.getSlotIndex();
        }

        @Override
        public void describeTo(Description description) {
            description.appendText("SlotOffer: ");
            description.appendValueList(
                    "{",
                    ",",
                    "}",
                    slotOffer.getAllocationId(),
                    slotOffer.getResourceProfile(),
                    slotOffer.getSlotIndex());
        }
    }

    private static class FreeSlotConsumer
            implements BiFunction<AllocationID, Throwable, CompletableFuture<Acknowledge>> {

        final BlockingQueue<AllocationID> freedSlots = new ArrayBlockingQueue<>(10);

        @Override
        public CompletableFuture<Acknowledge> apply(
                AllocationID allocationID, Throwable throwable) {
            freedSlots.offer(allocationID);
            return CompletableFuture.completedFuture(Acknowledge.get());
        }

        private Collection<AllocationID> drainFreedSlots() {
            final Collection<AllocationID> result = new ArrayList<>();

            freedSlots.drainTo(result);

            return result;
        }
    }
757
}