forked from apache/flink
-
Notifications
You must be signed in to change notification settings - Fork 0
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
[FLINK-16439][runtime] Introduce PendingWorkerCounter for counting pe…
…nding workers per WorkerResourceSpec in ActiveResourceManager.
- Loading branch information
1 parent
83b7138
commit 78603b3
Showing
2 changed files
with
154 additions
and
0 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
75 changes: 75 additions & 0 deletions
75
...ime/src/test/java/org/apache/flink/runtime/resourcemanager/ActiveResourceManagerTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,75 @@ | ||
/* | ||
* 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.resourcemanager; | ||
|
||
import org.apache.flink.util.TestLogger; | ||
|
||
import org.junit.Test; | ||
|
||
import static org.hamcrest.MatcherAssert.assertThat; | ||
import static org.hamcrest.core.Is.is; | ||
|
||
/** | ||
* Tests for {@link ActiveResourceManager}. | ||
*/ | ||
public class ActiveResourceManagerTest extends TestLogger { | ||
|
||
@Test | ||
public void testPendingWorkerCounterIncreaseAndDecrease() { | ||
final WorkerResourceSpec spec1 = new WorkerResourceSpec.Builder().setCpuCores(1.0).build(); | ||
final WorkerResourceSpec spec2 = new WorkerResourceSpec.Builder().setCpuCores(2.0).build(); | ||
|
||
final ActiveResourceManager.PendingWorkerCounter counter = new ActiveResourceManager.PendingWorkerCounter(); | ||
assertThat(counter.getTotalNum(), is(0)); | ||
assertThat(counter.getNum(spec1), is(0)); | ||
assertThat(counter.getNum(spec2), is(0)); | ||
|
||
assertThat(counter.increaseAndGet(spec1), is(1)); | ||
assertThat(counter.getTotalNum(), is(1)); | ||
assertThat(counter.getNum(spec1), is(1)); | ||
assertThat(counter.getNum(spec2), is(0)); | ||
|
||
assertThat(counter.increaseAndGet(spec1), is(2)); | ||
assertThat(counter.getTotalNum(), is(2)); | ||
assertThat(counter.getNum(spec1), is(2)); | ||
assertThat(counter.getNum(spec2), is(0)); | ||
|
||
assertThat(counter.increaseAndGet(spec2), is(1)); | ||
assertThat(counter.getTotalNum(), is(3)); | ||
assertThat(counter.getNum(spec1), is(2)); | ||
assertThat(counter.getNum(spec2), is(1)); | ||
|
||
assertThat(counter.decreaseAndGet(spec1), is(1)); | ||
assertThat(counter.getTotalNum(), is(2)); | ||
assertThat(counter.getNum(spec1), is(1)); | ||
assertThat(counter.getNum(spec2), is(1)); | ||
|
||
assertThat(counter.decreaseAndGet(spec2), is(0)); | ||
assertThat(counter.getTotalNum(), is(1)); | ||
assertThat(counter.getNum(spec1), is(1)); | ||
assertThat(counter.getNum(spec2), is(0)); | ||
} | ||
|
||
@Test(expected = IllegalStateException.class) | ||
public void testPendingWorkerCounterDecreaseOnZero() { | ||
final WorkerResourceSpec spec = new WorkerResourceSpec.Builder().build(); | ||
final ActiveResourceManager.PendingWorkerCounter counter = new ActiveResourceManager.PendingWorkerCounter(); | ||
counter.decreaseAndGet(spec); | ||
} | ||
} |