Skip to content
New issue

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

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

Already on GitHub? Sign in to your account

Report total active threads #28513

Merged
merged 16 commits into from
Oct 2, 2023
Merged
Show file tree
Hide file tree
Changes from 13 commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,8 @@ public enum StreamingSystemCounterNames {
JAVA_HARNESS_MAX_MEMORY("dataflow_java_harness_max_memory"),
JAVA_HARNESS_RESTARTS("dataflow_java_harness_restarts"),
TIME_AT_MAX_ACTIVE_THREADS("dataflow_time_at_max_active_threads"),
ACTIVE_THREADS("dataflow_active_threads"),
TOTAL_ALLOCATED_THREADS("dataflow_total_allocated_threads"),
WINDMILL_QUOTA_THROTTLING("dataflow_streaming_engine_throttled_msecs"),
MEMORY_THRASHING("dataflow_streaming_engine_user_worker_thrashing");

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -246,6 +246,8 @@ public class StreamingDataflowWorker {
private final Counter<Long, Long> javaHarnessUsedMemory;
private final Counter<Long, Long> javaHarnessMaxMemory;
private final Counter<Long, Long> timeAtMaxActiveThreads;
private final Counter<Integer, Integer> activeThreads;
private final Counter<Integer, Integer> totalAllocatedThreads;
private final Counter<Integer, Integer> windmillMaxObservedWorkItemCommitBytes;
private final Counter<Integer, Integer> memoryThrashing;
private final boolean publishCounters;
Expand Down Expand Up @@ -330,6 +332,11 @@ public class StreamingDataflowWorker {
this.timeAtMaxActiveThreads =
pendingCumulativeCounters.longSum(
StreamingSystemCounterNames.TIME_AT_MAX_ACTIVE_THREADS.counterName());
this.activeThreads =
pendingCumulativeCounters.intSum(StreamingSystemCounterNames.ACTIVE_THREADS.counterName());
this.totalAllocatedThreads =
pendingCumulativeCounters.intSum(
edman124 marked this conversation as resolved.
Show resolved Hide resolved
StreamingSystemCounterNames.TOTAL_ALLOCATED_THREADS.counterName());
this.windmillMaxObservedWorkItemCommitBytes =
pendingCumulativeCounters.intMax(
StreamingSystemCounterNames.WINDMILL_MAX_WORK_ITEM_COMMIT_BYTES.counterName());
Expand Down Expand Up @@ -1702,6 +1709,10 @@ private void updateVMMetrics() {
private void updateThreadMetrics() {
timeAtMaxActiveThreads.getAndReset();
timeAtMaxActiveThreads.addValue(workUnitExecutor.allThreadsActiveTime());
activeThreads.getAndReset();
activeThreads.addValue(workUnitExecutor.activeCount());
totalAllocatedThreads.getAndReset();
totalAllocatedThreads.addValue(chooseMaximumNumberOfThreads());
}

@VisibleForTesting
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -119,6 +119,10 @@ public long allThreadsActiveTime() {
return totalTimeMaxActiveThreadsUsed;
}

public int activeCount() {
return activeCount.intValue();
}

public String summaryHtml() {
monitor.enter();
try {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -2645,6 +2645,38 @@ public void testUnboundedSourceWorkRetry() throws Exception {
assertThat(finalizeTracker, contains(0));
}

private static class MockWork extends StreamingDataflowWorker.Work {

public MockWork(long workToken) {
super(
Windmill.WorkItem.newBuilder().setKey(ByteString.EMPTY).setWorkToken(workToken).build(),
Instant::now,
Collections.emptyList());
}

@Override
public void run() {}
}

private static class MockActiveWork extends StreamingDataflowWorker.Work {
// exit must be volatile so changes to it are reflected in the run function
public static volatile boolean exit;
edman124 marked this conversation as resolved.
Show resolved Hide resolved

public MockActiveWork(long workToken) {
super(
Windmill.WorkItem.newBuilder().setKey(ByteString.EMPTY).setWorkToken(workToken).build(),
Instant::now,
Collections.emptyList());
exit = false;
}

@Override
public void run() {}

public void stop() {
exit = true;
}
}
@Test
public void testActiveWork() throws Exception {
BoundedQueueExecutor mockExecutor = Mockito.mock(BoundedQueueExecutor.class);
Expand Down Expand Up @@ -2793,6 +2825,107 @@ public void testMaxThreadMetric() throws Exception {
executor.shutdown();
}

@Test
public void testActiveThreadMetric() throws Exception {
int maxThreads = 5;
int threadExpirationSec = 60;
// setting up actual implementation of executor instead of mocking to keep track of
// active thread count.
BoundedQueueExecutor executor =
new BoundedQueueExecutor(
maxThreads,
threadExpirationSec,
TimeUnit.SECONDS,
maxThreads,
10000000,
new ThreadFactoryBuilder()
.setNameFormat("DataflowWorkUnits-%d")
.setDaemon(true)
.build());

StreamingDataflowWorker.ComputationState computationState =
new StreamingDataflowWorker.ComputationState(
"computation",
defaultMapTask(Arrays.asList(makeSourceInstruction(StringUtf8Coder.of()))),
executor,
ImmutableMap.of(),
null);

ShardedKey key1Shard1 = ShardedKey.create(ByteString.copyFromUtf8("key1"), 1);

// real work
MockActiveWork m1 =
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The test is now failing to build, please address the precommit failures (e.g. https://github.com/apache/beam/actions/runs/6344373926/job/17234456430?pr=28513)

new MockActiveWork(1) {
@Override
public void run() {
synchronized (this) {
this.notify();
}
int count = 0;
while (!exit) {
count += 1;
}
Thread.currentThread().interrupt();
}
};

// idle work
MockWork m2 =
new MockWork(2) {
@Override
public void run() {
synchronized (this) {
this.notify();
}
try {
Thread.sleep(2000);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
}
};

// idle work
MockWork m3 =
new MockWork(3) {
@Override
public void run() {
synchronized (this) {
this.notify();
}
try {
Thread.sleep(2000);
} catch (InterruptedException e) {
Thread.currentThread().interrupt();
}
}
};
assertEquals(0, executor.activeCount());
edman124 marked this conversation as resolved.
Show resolved Hide resolved

assertTrue(computationState.activateWork(key1Shard1, m1));
synchronized (m1) {
executor.execute(m1, m1.getWorkItem().getSerializedSize());
m1.wait();
}
assertEquals(2, executor.activeCount());

assertTrue(computationState.activateWork(key1Shard1, m2));
assertTrue(computationState.activateWork(key1Shard1, m3));
synchronized (m2) {
executor.execute(m2, m2.getWorkItem().getSerializedSize());
m2.wait();
}
synchronized (m3) {
executor.execute(m3, m3.getWorkItem().getSerializedSize());
m3.wait();
}
// this.wait();
assertEquals(4, executor.activeCount());

m1.stop();
executor.shutdown();
}

@Test
public void testExceptionInvalidatesCache() throws Exception {
// We'll need to force the system to limit bundles to one message at a time.
Expand Down
Loading