From 7fd1a5917cf1246b988f41a5d1c0b9bd9d74fcf9 Mon Sep 17 00:00:00 2001 From: Wu Sheng Date: Fri, 13 Feb 2026 10:25:16 +0800 Subject: [PATCH 01/16] Add library-batch-queue module with partitioned, self-draining queue Introduce BatchQueue as a new queue implementation for metrics aggregation with type-based dispatch, configurable partition selection, and adaptive backoff. Includes comprehensive unit tests, throughput benchmarks, and DataCarrier comparison benchmarks. Also migrates existing microbench benchmarks out of the JMH-based microbench module into standard JUnit tests. --- .../EBPFProfilingAnalyzerBenchmark.java | 177 +++ .../EndpointGrouping4OpenapiBenchmark.java | 81 ++ .../benchmark/RegexVSQuickMatchBenchmark.java | 116 ++ .../library-batch-queue/DESIGN.md | 1205 +++++++++++++++++ .../library-batch-queue/pom.xml | 48 + .../server/library/batchqueue/BatchQueue.java | 516 +++++++ .../library/batchqueue/BatchQueueConfig.java | 110 ++ .../library/batchqueue/BatchQueueManager.java | 179 +++ .../library/batchqueue/BufferStrategy.java | 31 + .../library/batchqueue/HandlerConsumer.java | 39 + .../library/batchqueue/PartitionPolicy.java | 159 +++ .../library/batchqueue/PartitionSelector.java | 54 + .../library/batchqueue/QueueErrorHandler.java | 26 + .../library/batchqueue/ThreadPolicy.java | 83 ++ .../batchqueue/BatchQueueBenchmark.java | 449 ++++++ .../batchqueue/BatchQueueConfigTest.java | 110 ++ .../batchqueue/BatchQueueManagerTest.java | 162 +++ .../library/batchqueue/BatchQueueTest.java | 534 ++++++++ .../batchqueue/BenchmarkMetricTypes.java | 203 +++ .../batchqueue/PartitionPolicyTest.java | 119 ++ .../library/batchqueue/ThreadPolicyTest.java | 78 ++ .../src/test/resources/log4j2-test.xml | 32 + .../datacarrier/DataCarrierBenchmark.java | 192 +++ .../benchmark/StringFormatGroupBenchmark.java | 79 ++ oap-server/server-library/pom.xml | 1 + 25 files changed, 4783 insertions(+) create mode 100644 oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/benchmark/EBPFProfilingAnalyzerBenchmark.java create mode 100644 oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/benchmark/EndpointGrouping4OpenapiBenchmark.java create mode 100644 oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/benchmark/RegexVSQuickMatchBenchmark.java create mode 100644 oap-server/server-library/library-batch-queue/DESIGN.md create mode 100644 oap-server/server-library/library-batch-queue/pom.xml create mode 100644 oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java create mode 100644 oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueConfig.java create mode 100644 oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManager.java create mode 100644 oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BufferStrategy.java create mode 100644 oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/HandlerConsumer.java create mode 100644 oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/PartitionPolicy.java create mode 100644 oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/PartitionSelector.java create mode 100644 oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/QueueErrorHandler.java create mode 100644 oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicy.java create mode 100644 oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueBenchmark.java create mode 100644 oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueConfigTest.java create mode 100644 oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManagerTest.java create mode 100644 oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueTest.java create mode 100644 oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BenchmarkMetricTypes.java create mode 100644 oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/PartitionPolicyTest.java create mode 100644 oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicyTest.java create mode 100644 oap-server/server-library/library-batch-queue/src/test/resources/log4j2-test.xml create mode 100644 oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/DataCarrierBenchmark.java create mode 100644 oap-server/server-library/library-util/src/test/java/org/apache/skywalking/oap/server/library/util/benchmark/StringFormatGroupBenchmark.java diff --git a/oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/benchmark/EBPFProfilingAnalyzerBenchmark.java b/oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/benchmark/EBPFProfilingAnalyzerBenchmark.java new file mode 100644 index 000000000000..4c14aa19c4f9 --- /dev/null +++ b/oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/benchmark/EBPFProfilingAnalyzerBenchmark.java @@ -0,0 +1,177 @@ +/* + * 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.skywalking.oap.server.core.benchmark; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Random; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; +import org.apache.skywalking.oap.server.core.profiling.ebpf.analyze.EBPFProfilingAnalyzer; +import org.apache.skywalking.oap.server.core.profiling.ebpf.analyze.EBPFProfilingStack; +import org.apache.skywalking.oap.server.core.profiling.ebpf.storage.EBPFProfilingStackType; +import org.apache.skywalking.oap.server.core.query.type.EBPFProfilingAnalyzation; +import org.junit.jupiter.api.Test; + +@Slf4j +public class EBPFProfilingAnalyzerBenchmark { + + private static final Random RANDOM = new Random(System.currentTimeMillis()); + private static final int SYMBOL_LENGTH = 10; + private static final char[] SYMBOL_TABLE = "abcdefgABCDEFG1234567890_[]<>.".toCharArray(); + private static final EBPFProfilingStackType[] STACK_TYPES = new EBPFProfilingStackType[] { + EBPFProfilingStackType.KERNEL_SPACE, EBPFProfilingStackType.USER_SPACE}; + + private static final int WARMUP_ITERATIONS = 3; + private static final int MEASURE_ITERATIONS = 5; + + private static List generateStacks(final int totalStackCount, + final int perStackMinDepth, + final int perStackMaxDepth, + final double[] stackSymbolDuplicateRate, + final double stackDuplicateRate) { + final int uniqStackCount = (int) (100 / stackDuplicateRate); + final List stacks = new ArrayList<>(totalStackCount); + final StackSymbolGenerator stackSymbolGenerator = + new StackSymbolGenerator(stackSymbolDuplicateRate, perStackMaxDepth); + for (int inx = 0; inx < uniqStackCount; inx++) { + final EBPFProfilingStack s = generateStack(perStackMinDepth, perStackMaxDepth, stackSymbolGenerator); + stacks.add(s); + } + for (int inx = uniqStackCount; inx < totalStackCount; inx++) { + stacks.add(stacks.get(RANDOM.nextInt(uniqStackCount))); + } + return stacks; + } + + private static class StackSymbolGenerator { + private final Map stackDepthSymbolCount; + private final Map> existingSymbolMap; + + StackSymbolGenerator(final double[] stackSymbolDuplicateRate, final int maxDepth) { + this.stackDepthSymbolCount = new HashMap<>(); + for (int depth = 0; depth < maxDepth; depth++) { + double rate = stackSymbolDuplicateRate[stackSymbolDuplicateRate.length - 1]; + if (stackSymbolDuplicateRate.length > depth) { + rate = stackSymbolDuplicateRate[depth]; + } + final int uniqStackCount = (int) (100 / rate); + stackDepthSymbolCount.put(depth, uniqStackCount); + } + this.existingSymbolMap = new HashMap<>(); + } + + String generate(final int depth) { + List symbols = existingSymbolMap.get(depth); + if (symbols == null) { + symbols = new ArrayList<>(); + existingSymbolMap.put(depth, symbols); + } + final Integer needCount = this.stackDepthSymbolCount.get(depth); + if (symbols.size() < needCount) { + final StringBuilder sb = new StringBuilder(SYMBOL_LENGTH); + for (int j = 0; j < SYMBOL_LENGTH; j++) { + sb.append(SYMBOL_TABLE[RANDOM.nextInt(SYMBOL_TABLE.length)]); + } + final String s = sb.toString(); + symbols.add(s); + return s; + } else { + return symbols.get(RANDOM.nextInt(symbols.size())); + } + } + } + + private static EBPFProfilingStack generateStack(final int stackMinDepth, final int stackMaxDepth, + final StackSymbolGenerator stackSymbolGenerator) { + final int stackDepth = stackMinDepth + RANDOM.nextInt(stackMaxDepth - stackMinDepth); + final List symbols = new ArrayList<>(stackDepth); + for (int i = 0; i < stackDepth; i++) { + final EBPFProfilingStack.Symbol symbol = new EBPFProfilingStack.Symbol( + stackSymbolGenerator.generate(i), buildStackType(i, stackDepth)); + symbols.add(symbol); + } + final EBPFProfilingStack stack = new EBPFProfilingStack(); + stack.setDumpCount(RANDOM.nextInt(100)); + stack.setSymbols(symbols); + return stack; + } + + private static EBPFProfilingStackType buildStackType(final int currentDepth, final int totalDepth) { + final int partition = totalDepth / STACK_TYPES.length; + for (int i = 1; i <= STACK_TYPES.length; i++) { + if (currentDepth < i * partition) { + return STACK_TYPES[i - 1]; + } + } + return STACK_TYPES[STACK_TYPES.length - 1]; + } + + private static int calculateStackCount(final int stackReportPeriodSecond, + final int totalTimeMinute, + final int combineInstanceCount) { + return (int) (TimeUnit.MINUTES.toSeconds(totalTimeMinute) + / stackReportPeriodSecond * combineInstanceCount); + } + + private static void analyze(final List stacks) { + new EBPFProfilingAnalyzer(null, 100, 5) + .generateTrees(new EBPFProfilingAnalyzation(), stacks.parallelStream()); + } + + @Test + public void analyzeLowDataSource() { + final List stacks = generateStacks( + calculateStackCount(5, 60, 10), 15, 30, + new double[] {100, 50, 45, 40, 35, 30, 15, 10, 5}, 5); + runBenchmark("low", stacks); + } + + @Test + public void analyzeMedianDataSource() { + final List stacks = generateStacks( + calculateStackCount(5, 100, 200), 15, 30, + new double[] {50, 40, 35, 30, 20, 10, 7, 5, 2}, 3); + runBenchmark("median", stacks); + } + + @Test + public void analyzeHighDataSource() { + final List stacks = generateStacks( + calculateStackCount(5, 2 * 60, 2000), 15, 40, + new double[] {30, 27, 25, 20, 17, 15, 10, 7, 5, 2, 1}, 1); + runBenchmark("high", stacks); + } + + private void runBenchmark(final String label, final List stacks) { + for (int i = 0; i < WARMUP_ITERATIONS; i++) { + analyze(stacks); + } + final long start = System.nanoTime(); + for (int i = 0; i < MEASURE_ITERATIONS; i++) { + analyze(stacks); + } + final long elapsed = System.nanoTime() - start; + log.info("{}: {} iterations, {} ms/op", + label, MEASURE_ITERATIONS, + elapsed / MEASURE_ITERATIONS / 1_000_000); + } +} diff --git a/oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/benchmark/EndpointGrouping4OpenapiBenchmark.java b/oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/benchmark/EndpointGrouping4OpenapiBenchmark.java new file mode 100644 index 000000000000..d3d0dae01053 --- /dev/null +++ b/oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/benchmark/EndpointGrouping4OpenapiBenchmark.java @@ -0,0 +1,81 @@ +/* + * 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.skywalking.oap.server.core.benchmark; + +import java.util.Collections; +import java.util.Map; +import lombok.extern.slf4j.Slf4j; +import org.apache.skywalking.oap.server.core.config.group.openapi.EndpointGroupingRule4Openapi; +import org.apache.skywalking.oap.server.core.config.group.openapi.EndpointGroupingRuleReader4Openapi; +import org.junit.jupiter.api.Test; + +@Slf4j +public class EndpointGrouping4OpenapiBenchmark { + + private static final String APT_TEST_DATA = " /products1/{id}/%d:\n" + " get:\n" + " post:\n" + + " /products2/{id}/%d:\n" + " get:\n" + " post:\n" + + " /products3/{id}/%d:\n" + " get:\n"; + + private static final int WARMUP_ITERATIONS = 5000; + private static final int MEASURE_ITERATIONS = 100_000; + + private static Map createTestFile(final int size) { + final StringBuilder stringBuilder = new StringBuilder(); + stringBuilder.append("paths:\n"); + for (int i = 0; i <= size; i++) { + stringBuilder.append(String.format(APT_TEST_DATA, i, i, i)); + } + return Collections.singletonMap("whatever", stringBuilder.toString()); + } + + @Test + public void formatEndpointNameMatchedPaths20() { + final EndpointGroupingRule4Openapi rule = + new EndpointGroupingRuleReader4Openapi(createTestFile(3)).read(); + runBenchmark("paths20", rule); + } + + @Test + public void formatEndpointNameMatchedPaths50() { + final EndpointGroupingRule4Openapi rule = + new EndpointGroupingRuleReader4Openapi(createTestFile(9)).read(); + runBenchmark("paths50", rule); + } + + @Test + public void formatEndpointNameMatchedPaths200() { + final EndpointGroupingRule4Openapi rule = + new EndpointGroupingRuleReader4Openapi(createTestFile(39)).read(); + runBenchmark("paths200", rule); + } + + private void runBenchmark(final String label, final EndpointGroupingRule4Openapi rule) { + for (int i = 0; i < WARMUP_ITERATIONS; i++) { + rule.format("serviceA", "GET:/products1/123"); + } + final long start = System.nanoTime(); + for (int i = 0; i < MEASURE_ITERATIONS; i++) { + rule.format("serviceA", "GET:/products1/123"); + } + final long elapsed = System.nanoTime() - start; + log.info("{}: {} ops, {} ns/op, {} ops/s", + label, MEASURE_ITERATIONS, elapsed / MEASURE_ITERATIONS, + MEASURE_ITERATIONS * 1_000_000_000L / elapsed); + } +} diff --git a/oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/benchmark/RegexVSQuickMatchBenchmark.java b/oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/benchmark/RegexVSQuickMatchBenchmark.java new file mode 100644 index 000000000000..0ef96aa5e3f5 --- /dev/null +++ b/oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/benchmark/RegexVSQuickMatchBenchmark.java @@ -0,0 +1,116 @@ +/* + * 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.skywalking.oap.server.core.benchmark; + +import lombok.extern.slf4j.Slf4j; +import org.apache.skywalking.oap.server.core.config.group.EndpointGroupingRule; +import org.apache.skywalking.oap.server.core.config.group.uri.quickmatch.QuickUriGroupingRule; +import org.junit.jupiter.api.Test; + +@Slf4j +public class RegexVSQuickMatchBenchmark { + + private static final int WARMUP_ITERATIONS = 5000; + private static final int MEASURE_ITERATIONS = 100_000; + + private final EndpointGroupingRule regexRule = new EndpointGroupingRule(); + private final QuickUriGroupingRule quickRule = new QuickUriGroupingRule(); + + { + regexRule.addRule("service1", "/products/{var}", "/products/.+"); + regexRule.addRule("service1", "/products/{var}/detail", "/products/.+/detail"); + regexRule.addRule("service1", "/sales/{var}/1", "/sales/.+/1"); + regexRule.addRule("service1", "/sales/{var}/2", "/sales/.+/2"); + regexRule.addRule("service1", "/sales/{var}/3", "/sales/.+/3"); + regexRule.addRule("service1", "/sales/{var}/4", "/sales/.+/4"); + regexRule.addRule("service1", "/sales/{var}/5", "/sales/.+/5"); + regexRule.addRule("service1", "/sales/{var}/6", "/sales/.+/6"); + regexRule.addRule("service1", "/sales/{var}/7", "/sales/.+/7"); + regexRule.addRule("service1", "/sales/{var}/8", "/sales/.+/8"); + regexRule.addRule("service1", "/sales/{var}/9", "/sales/.+/9"); + regexRule.addRule("service1", "/sales/{var}/10", "/sales/.+/10"); + regexRule.addRule("service1", "/sales/{var}/11", "/sales/.+/11"); + regexRule.addRule("service1", "/employees/{var}/profile", "/employees/.+/profile"); + + quickRule.addRule("service1", "/products/{var}"); + quickRule.addRule("service1", "/products/{var}/detail"); + quickRule.addRule("service1", "/sales/{var}/1"); + quickRule.addRule("service1", "/sales/{var}/2"); + quickRule.addRule("service1", "/sales/{var}/3"); + quickRule.addRule("service1", "/sales/{var}/4"); + quickRule.addRule("service1", "/sales/{var}/5"); + quickRule.addRule("service1", "/sales/{var}/6"); + quickRule.addRule("service1", "/sales/{var}/7"); + quickRule.addRule("service1", "/sales/{var}/8"); + quickRule.addRule("service1", "/sales/{var}/9"); + quickRule.addRule("service1", "/sales/{var}/10"); + quickRule.addRule("service1", "/sales/{var}/11"); + quickRule.addRule("service1", "/employees/{var}/profile"); + } + + @Test + public void matchFirstRegex() { + runBenchmark("matchFirstRegex", + () -> regexRule.format("service1", "/products/123")); + } + + @Test + public void matchFirstQuickUriGrouping() { + runBenchmark("matchFirstQuick", + () -> quickRule.format("service1", "/products/123")); + } + + @Test + public void matchFourthRegex() { + runBenchmark("matchFourthRegex", + () -> regexRule.format("service1", "/sales/123/2")); + } + + @Test + public void matchFourthQuickUriGrouping() { + runBenchmark("matchFourthQuick", + () -> quickRule.format("service1", "/sales/123/2")); + } + + @Test + public void notMatchRegex() { + runBenchmark("notMatchRegex", + () -> regexRule.format("service1", "/employees/123")); + } + + @Test + public void notMatchQuickUriGrouping() { + runBenchmark("notMatchQuick", + () -> quickRule.format("service1", "/employees/123")); + } + + private void runBenchmark(final String label, final Runnable action) { + for (int i = 0; i < WARMUP_ITERATIONS; i++) { + action.run(); + } + final long start = System.nanoTime(); + for (int i = 0; i < MEASURE_ITERATIONS; i++) { + action.run(); + } + final long elapsed = System.nanoTime() - start; + log.info("{}: {} ops, {} ns/op, {} ops/s", + label, MEASURE_ITERATIONS, elapsed / MEASURE_ITERATIONS, + MEASURE_ITERATIONS * 1_000_000_000L / elapsed); + } +} diff --git a/oap-server/server-library/library-batch-queue/DESIGN.md b/oap-server/server-library/library-batch-queue/DESIGN.md new file mode 100644 index 000000000000..bc107dbbaffc --- /dev/null +++ b/oap-server/server-library/library-batch-queue/DESIGN.md @@ -0,0 +1,1205 @@ +# library-batch-queue Design Proposal + +## Goal + +Replace `library-datacarrier-queue` with a unified, simpler batch queue library that reduces thread +usage while preserving all required capabilities. + +## Problem Statement + +DataCarrier has two consumption modes with separate code paths: + +1. **Simple mode** (`ConsumeDriver`): Each DataCarrier gets dedicated consumer thread(s). + Used by TopNWorker, GRPCRemoteClient, JDBCBatchDAO, exporters. +2. **Pool mode** (`BulkConsumePool`): Multiple DataCarriers share a thread pool. + Used by MetricsAggregateWorker (L1) and MetricsPersistentMinWorker (L2). + +On an 8-core production machine, this creates **47+ DataCarrier consumer threads**: + +| Source | Threads | Mode | +|--------------------------------|---------|--------| +| L1 OAL aggregation pool | 24 | Pool | +| L2 OAL persistent pool | 2 | Pool | +| L1 MAL aggregation pool | 2 | Pool | +| L2 MAL persistent pool | 1 | Pool | +| TopNWorker (per type) | 5-10 | Simple | +| GRPCRemoteClient (per peer) | 2-4 | Simple | +| JDBCBatchDAO | 2-4 | Simple | +| Exporters (gRPC/Kafka) | 0-3 | Simple | +| **Total** | **~38-48** | | + +Key issues: +- Simple mode wastes threads: each queue gets a dedicated thread even though most are idle + (sleeping in 200ms polling loop). +- Pool mode creates one DataCarrier (with Channels) per metric type. With 100+ metric types, + pool threads must iterate through all assigned channels even when most are empty — wasted CPU. +- Two completely separate code paths for the same produce-consume pattern. +- Pool mode has static assignment — no rebalancing after initial allocation. + +## Design + +### Architecture Overview + +``` + BatchQueueManager (global singleton registry + lazy shared schedulers) + │ + │ Shared schedulers (created lazily on first queue reference): + │ ├── "IO_POOL" ──> ScheduledExecutorService (cpuCores(0.5) → 4 threads on 8-core) + │ │ Created when first queue calls sharedScheduler("IO_POOL", cpuCores(0.5)). + │ │ Shared by all I/O queues: + │ │ - GRPCRemoteClient.* (gRPC to peer OAP nodes) + │ │ - GRPCMetricsExporter (gRPC metrics export) + │ │ - KafkaLogExporter (Kafka log export) + │ │ - KafkaTraceExporter (Kafka trace export) + │ │ - JDBCBatchDAO (JDBC batch writes) + │ │ + │ + │ Queues: + │ + ├── "METRICS_L1_AGGREGATION" ──> BatchQueue + │ │ threads: cpuCores(1.0) → 8 on 8-core + │ │ partitions: threadMultiply(2) → 16 on 8-core + │ │ strategy: IF_POSSIBLE + │ │ handlerMap: { ServiceRespTimeMetrics.class -> handler-A (OAL), + │ │ ServiceCpmMetrics.class -> handler-B (OAL), + │ │ MeterMetrics_xxx.class -> handler-C (MAL), ... } + │ └── OAL and MAL metrics share the same L1 queue and thread pool. + │ + ├── "METRICS_L2_PERSISTENT" ──> BatchQueue + │ │ threads: cpuCores(0.25) → 2 on 8-core + │ │ partitions: threadMultiply(2) → 4 on 8-core + │ │ strategy: BLOCKING + │ │ handlerMap: { ServiceRespTimeMetrics.class -> handler-D (OAL), + │ │ MeterMetrics_xxx.class -> handler-E (MAL), ... } + │ └── OAL and MAL metrics share the same L2 queue and thread pool. + │ + ├── "TOPN_WORKER" ──> BatchQueue + │ │ threads: fixed(1) + │ │ partitions: fixed(1) + │ │ strategy: BLOCKING + │ │ handlerMap: { DatabaseSlowStatement.class -> handler-F, + │ │ DatabaseSlowSql.class -> handler-G, ... } + │ └── drain: drainTo → groupBy(class) → dispatch to handler + │ (TopN is in-memory ranking computation — all types share one thread) + │ + ├── "GRPCRemoteClient.peer1" ──> BatchQueue (shared="IO_POOL", partitions=1, BLOCKING) + │ │ scheduler: shared "IO_POOL" + │ │ consumer: direct consumer for RemoteMessage + │ └── drain: drainTo → direct consumer + │ + └── "JDBCBatchDAO" ──> BatchQueue (shared="IO_POOL", partitions=1, BLOCKING) + │ scheduler: shared "IO_POOL" + │ consumer: direct consumer for PrepareRequest + └── same +``` + +### Core Concept + +**One queue type, one config, two scheduler modes.** + +- **`BatchQueueManager`** is the global singleton registry. It also manages named shared + schedulers for low-throughput queues. Users call `createIfAbsent(name, config)` to + get a named `BatchQueue`. +- **`BatchQueue`** has N partitions (configurable, default 1) and a handler map. + Producers round-robin data into partitions. On drain, each batch is **grouped by + message class** and dispatched to the handler registered for that class. +- **Handler registration** via `queue.addHandler(Class, HandlerConsumer)`. + Each worker provides its own handler instance for its specific type. + +The handler map pattern works the same way regardless of partition count: +- `threadMultiply(2)` with 100+ handlers → metrics aggregation (many types, shared partitions) +- `partitions=1` with N handlers → TopN (multiple types, low throughput, shared 1 thread) +- `partitions=1` with 1 consumer → I/O queue (gRPC client, exporter, JDBC) + +No need for separate queue classes. The difference is just configuration. + +### Why Shared Partitions + Handler Map + +In the old BulkConsumePool model with 100+ metric types: + +``` +Pool Thread-0 assigned: [service_resp_time channels, service_cpm channels, ...] +Pool Thread-1 assigned: [endpoint_resp_time channels, endpoint_cpm channels, ...] +... +Each thread iterates ALL assigned channels per cycle, even if most are empty. +``` + +In the new model: + +``` +Partition-0: mixed data from all metric types (round-robin) +Partition-1: mixed data from all metric types +... +Partition-N: mixed data from all metric types + +On drain of Partition-K: + batch = drainTo(list) // all data, mixed types + grouped = batch.groupBy(item.getClass()) // group by metric class + for each (class, items) in grouped: + handler = handlerMap.get(class) // lookup registered handler + handler.consume(items) // dispatch to the right worker +``` + +Benefits: +- Partitions are created based on parallelism needs, not metric count. + 16 partitions (8 threads * 2) serve 100+ metric types. +- No empty channel iteration — every partition gets data. +- Handlers are registered on-demand. Adding a new metric type is just + `addHandler(NewMetrics.class, handlerInstance)`. +- Each handler still processes only its own metric type's data in isolation. +- I/O queues use the same structure with `partitions=1` and a direct consumer. + +### API + +```java +// ── Metrics aggregation (dedicated pool, many types, handler map dispatch) ── + +BatchQueue l1Queue = BatchQueueManager.createIfAbsent( + "METRICS_L1_AGGREGATION", + BatchQueueConfig.builder() + .threads(ThreadPolicy.cpuCores(1.0)) // 1x CPU cores (e.g. 8 on 8-core) + .partitions(PartitionPolicy.threadMultiply(2)) // 2x resolved threads = 16 on 8-core + .bufferSize(10_000) + .strategy(BufferStrategy.IF_POSSIBLE) + .errorHandler((data, t) -> log.error(t.getMessage(), t)) + .build() +); + +// Each MetricsAggregateWorker registers its inner class handler for its metric class. +// Called per metric type in MetricsStreamProcessor.create() (100+ times): +l1Queue.addHandler(metricsClass, new L1Handler()); // L1Handler is worker's inner class + +// Produce — data goes into a partition by round-robin +// Adaptive backoff ensures fast re-poll (minIdleMs) when data is flowing. +l1Queue.produce(metricsData); + +// ── TopN (shared queue — all TopN types share one thread, handler map dispatch) ── + +BatchQueue topNQueue = BatchQueueManager.createIfAbsent( + "TOPN_WORKER", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) // all TopN types share 1 thread + .partitions(PartitionPolicy.fixed(1)) + .bufferSize(1000) + .strategy(BufferStrategy.BLOCKING) + .errorHandler((data, t) -> log.error(t.getMessage(), t)) + .build() +); + +// Each TopNWorker registers its handler for its specific TopN class (5-10 types): +topNQueue.addHandler(topNClass, new TopNHandler()); // TopNHandler is worker's inner class + +topNQueue.produce(topNData); + +// ── I/O queues (shared scheduler, single consumer) ── +// sharedScheduler() specifies both name and ThreadPolicy. BatchQueueManager creates the +// shared ScheduledExecutorService on first reference, reuses it for subsequent queues. +// No separate createSharedScheduler() call needed — no startup ordering dependency. +// +// All these queues share the same "IO_POOL" scheduler: +// GRPCRemoteClient.* — gRPC streaming to peer OAP nodes +// GRPCMetricsExporter — gRPC metrics export +// KafkaLogExporter — Kafka log export +// KafkaTraceExporter — Kafka trace export +// JDBCBatchDAO — JDBC batch writes to database + +BatchQueue grpcQueue = BatchQueueManager.createIfAbsent( + "GRPCRemoteClient.peer1", + BatchQueueConfig.builder() + .sharedScheduler("IO_POOL", ThreadPolicy.cpuCores(0.5)) // creates IO_POOL on first use + .partitions(PartitionPolicy.fixed(1)) + .bufferSize(10_000) + .strategy(BufferStrategy.BLOCKING) + .consumer(new RemoteMessageHandler()) + .errorHandler((data, t) -> log.error(t.getMessage(), t)) + .build() +); + +// Another queue referencing the same "IO_POOL" — reuses existing scheduler. +// If ThreadPolicy differs from the first creator, logs a warning (first one wins). +BatchQueue jdbcQueue = BatchQueueManager.createIfAbsent( + "JDBCBatchDAO", + BatchQueueConfig.builder() + .sharedScheduler("IO_POOL", ThreadPolicy.cpuCores(0.5)) // reuses existing IO_POOL + .partitions(PartitionPolicy.fixed(1)) + .bufferSize(10_000) + .strategy(BufferStrategy.BLOCKING) + .consumer(new JDBCBatchHandler()) + .errorHandler((data, t) -> log.error(t.getMessage(), t)) + .build() +); + +// ── Lifecycle ── + +BatchQueueManager.shutdown("METRICS_L1_AGGREGATION"); +BatchQueueManager.shutdownAll(); // shuts down all queues + shared schedulers +``` + +### Classes + +``` +library-batch-queue/ + src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ + BatchQueueManager.java — Global singleton registry + named shared schedulers + BatchQueue.java — Partitioned queue with dedicated or shared scheduler + handler map dispatch + BatchQueueConfig.java — Builder: threads/sharedScheduler, partitions, bufferSize, strategy + ThreadPolicy.java — Fixed or CPU-relative thread count: fixed(N) / cpuCores(double) + PartitionPolicy.java — Fixed or thread-relative partition count: fixed(N) / threadMultiply(N) + HandlerConsumer.java — Interface: void consume(List), default void onIdle() + BufferStrategy.java — BLOCKING / IF_POSSIBLE + QueueErrorHandler.java — Functional interface: void onError(List, Throwable) +``` + +### BatchQueueManager + +```java +/** + * Global registry for batch queues and shared schedulers. + * Thread-safe. Queues are created by name and shared across modules. + * + * Two scheduler modes: + * - Dedicated: queue creates its own ScheduledExecutorService (for high-throughput queues). + * Configured via BatchQueueConfig.threads(ThreadPolicy). + * - Shared: queue uses a named shared scheduler managed by this manager (for low-throughput queues). + * Configured via BatchQueueConfig.sharedScheduler(name, ThreadPolicy). + * Multiple queues referencing the same name share one ScheduledExecutorService. + * + * Shared schedulers are created lazily on first queue reference — no separate + * setup step needed. This eliminates startup ordering dependencies. + */ +public class BatchQueueManager { + private static final ConcurrentHashMap> queues = new ConcurrentHashMap<>(); + private static final ConcurrentHashMap sharedSchedulers = new ConcurrentHashMap<>(); + + /** + * Get or create a shared scheduler. Called internally by BatchQueue constructor + * when config specifies sharedScheduler(name, threads). + * + * - First call with a given name: creates the ScheduledExecutorService using + * threads.resolve() and caches it. + * - Subsequent calls with the same name: returns the cached scheduler. + * If the ThreadPolicy differs, logs a warning (first one wins). + * + * Thread-safe (ConcurrentHashMap.computeIfAbsent). + * + * Shared schedulers are owned by BatchQueueManager, NOT by any individual queue. + * They are destroyed only by shutdownAll() — never by individual queue shutdown. + */ + static ScheduledExecutorService getOrCreateSharedScheduler(String name, ThreadPolicy threads); + + /** + * Create a new named queue. Throws if name already exists. + */ + public static BatchQueue create(String name, BatchQueueConfig config); + + /** + * Create if not present. Returns existing queue if name is taken. + * + * If the queue already exists, validates consistency: + * - Consumption mode conflict: throws IllegalStateException if the existing queue + * uses direct consumer mode (config.consumer set) but the new config does not, + * or vice versa. These two modes are mutually exclusive per queue. + * - Infrastructure settings: logs a warning if threads, partitions, + * bufferSize, or strategy differ between the existing and new config. + */ + public static BatchQueue createIfAbsent(String name, BatchQueueConfig config); + + /** + * Get an existing queue by name. Returns null if not found. + */ + public static BatchQueue get(String name); + + /** + * Shutdown and remove a single queue by name. + * - Dedicated scheduler: shut down together with the queue. + * - Shared scheduler: NOT shut down. It is owned by BatchQueueManager + * and may still be used by other queues. + */ + public static void shutdown(String name); + + /** + * Shutdown all queues and all shared schedulers. Called during OAP server shutdown. + * + * Order: + * 1. Set running=false on all queues (stops drain loops from rescheduling) + * 2. Final drain of remaining data in each queue + * 3. Shut down all dedicated schedulers + * 4. Shut down all shared schedulers + * 5. Clear both registries + */ + public static void shutdownAll(); +} +``` + +### BatchQueue + +```java +/** + * A partitioned queue with handler-map-based dispatch. + * + * The scheduler is either dedicated (owned by this queue) or shared + * (managed by BatchQueueManager, shared with other queues). + * + * Partitions are created based on configured parallelism (default 1). + * Producers round-robin data across partitions. + * On drain, each batch is grouped by message class and dispatched to the + * registered handler for that class. + * + * Works uniformly for all use cases: + * - shared scheduler, partitions=1, one consumer → I/O queue (gRPC, Kafka, JDBC) + * - dedicated fixed(1), partitions=1, many handlers → TopN (all types share 1 thread) + * - dedicated cpuCores(1.0), threadMultiply(2), many handlers → metrics aggregation + */ +public class BatchQueue { + private final String name; + private final ScheduledExecutorService scheduler; + private final boolean dedicatedScheduler; // true = owned by this queue, false = shared + private final ArrayBlockingQueue[] partitions; + private final ConcurrentHashMap, HandlerConsumer> handlerMap; + private final BatchQueueConfig config; + private final AtomicInteger roundRobinIndex = new AtomicInteger(0); + private volatile boolean running; + + /** + * Partition assignment per drain task. Each drain task owns a set of partition indices. + * + * Dedicated mode: one drain task per thread, partitions assigned round-robin. + * threads=4, partitions=8: task[0]→[0,4], task[1]→[1,5], task[2]→[2,6], task[3]→[3,7] + * + * Shared mode: single drain task covering ALL partitions (partitions typically = 1). + * assignedPartitions = { [0] } (one task, one partition) + */ + private final int[][] assignedPartitions; + + /** + * Per-task count of consecutive idle cycles (all assigned partitions empty). + * Used to compute adaptive backoff sleep interval. + */ + private final int[] consecutiveIdleCycles; + + BatchQueue(String name, BatchQueueConfig config) { + this.name = name; + this.config = config; + this.handlerMap = new ConcurrentHashMap<>(); + + int taskCount; + if (config.getSharedSchedulerName() != null) { + // Shared scheduler mode: get-or-create shared scheduler from BatchQueueManager. + ScheduledExecutorService sharedScheduler = BatchQueueManager.getOrCreateSharedScheduler( + config.getSharedSchedulerName(), config.getSharedSchedulerThreads()); + + int partitionCount = config.getPartitions().resolve(1, 0); + this.partitions = new ArrayBlockingQueue[partitionCount]; + for (int i = 0; i < partitions.length; i++) { + partitions[i] = new ArrayBlockingQueue<>(config.getBufferSize()); + } + + this.scheduler = sharedScheduler; + this.dedicatedScheduler = false; + taskCount = 1; + this.assignedPartitions = new int[][] { + java.util.stream.IntStream.range(0, partitions.length).toArray() + }; + } else { + // Dedicated scheduler mode: resolve threads and partitions. + int threadCount = config.getThreads().resolve(); // cpuCores(1.0) → 8 on 8-core + int partitionCount = config.getPartitions().resolve(threadCount, 0); + + // Validation: if partitions < threads, cut threads to match and warn. + if (partitionCount < threadCount) { + log.warn("BatchQueue[{}]: partitions({}) < threads({}), " + + "reducing threads to {}", + name, partitionCount, threadCount, partitionCount); + threadCount = partitionCount; + } + + this.partitions = new ArrayBlockingQueue[partitionCount]; + for (int i = 0; i < partitions.length; i++) { + partitions[i] = new ArrayBlockingQueue<>(config.getBufferSize()); + } + + this.scheduler = Executors.newScheduledThreadPool( + threadCount, + new ThreadFactoryBuilder().setNameFormat("BatchQueue-" + name + "-%d").build() + ); + this.dedicatedScheduler = true; + taskCount = threadCount; + + // Assign partitions to threads by round-robin. + // threads=4, partitions=8: task[0]→[0,4], task[1]→[1,5], ... + // threads=4, threadMultiply(2)=8: same + // threads=8, partitions=8: task[0]→[0], task[1]→[1], ... + this.assignedPartitions = new int[taskCount][]; + List> assignment = new ArrayList<>(); + for (int t = 0; t < taskCount; t++) { + assignment.add(new ArrayList<>()); + } + for (int p = 0; p < partitions.length; p++) { + assignment.get(p % taskCount).add(p); + } + for (int t = 0; t < taskCount; t++) { + assignedPartitions[t] = assignment.get(t).stream().mapToInt(Integer::intValue).toArray(); + } + } + + // Kick off one self-rescheduling drain task per assignment. + this.consecutiveIdleCycles = new int[taskCount]; + this.running = true; + for (int t = 0; t < taskCount; t++) { + scheduleDrain(t); + } + } + + /** + * Schedule the next drain with adaptive backoff. + * + * Idle count 0 (just had data): sleep = minIdleMs (e.g. 5ms) + * Idle count 1: sleep = minIdleMs * 2 (e.g. 10ms) + * Idle count 2: sleep = minIdleMs * 4 (e.g. 20ms) + * ... + * Idle count N: sleep = min(minIdleMs * 2^N, maxIdleMs) + */ + private void scheduleDrain(int taskIndex) { + int idleCount = consecutiveIdleCycles[taskIndex]; + long delay = Math.min( + config.getMinIdleMs() * (1L << Math.min(idleCount, 20)), + config.getMaxIdleMs() + ); + scheduler.schedule( + () -> drainLoop(taskIndex), + delay, + TimeUnit.MILLISECONDS + ); + } + + /** + * Register a handler for a specific message class type. + * Multiple metric types can each register their own handler instance. + */ + public void addHandler(Class type, HandlerConsumer handler); + + /** + * Produce data into a partition (round-robin). + * BLOCKING: waits if the selected partition is full (queue.put). + * IF_POSSIBLE: returns false if full (queue.offer). + */ + public boolean produce(T data) { + int index = Math.abs(roundRobinIndex.getAndIncrement() % partitions.length); + if (config.getStrategy() == BufferStrategy.BLOCKING) { + partitions[index].put(data); // blocks until space available + return true; + } else { + return partitions[index].offer(data); // returns false if full + } + } + + /** + * Drain loop for one thread: iterates ALL assigned partitions in a round-robin loop. + * Only stops when ALL assigned partitions are empty in a full cycle. + * + * Each cycle: + * 1. Drain ALL assigned partitions into one combined batch + * 2. If combined batch is empty → all partitions empty → onIdle, break + * 3. dispatch(combined batch) → handlers get ALL data of their type as one list + * 4. Loop back to step 1 (more data may have arrived during dispatch) + * + * Example: Thread-0 owns partitions [0, 4] + * cycle 1: drain(0)→[A1,B1,A2] drain(4)→[C1,A3] + * combined = [A1,B1,A2,C1,A3] + * dispatch → groupBy class: + * handlerA.consume([A1,A2,A3]) ← all A's in one call + * handlerB.consume([B1]) + * handlerC.consume([C1]) + * cycle 2: drain(0)→[A4] drain(4)→[] + * combined = [A4] (not empty → dispatch) + * cycle 3: drain(0)→[] drain(4)→[] + * combined = [] → onIdle, reschedule + */ + void drainLoop(int taskIndex) { + int[] myPartitions = assignedPartitions[taskIndex]; + try { + while (running) { + // Step 1: drain ALL assigned partitions into one batch + List combined = new ArrayList<>(); + for (int partitionIndex : myPartitions) { + partitions[partitionIndex].drainTo(combined); + } + + // Step 2: if nothing across all partitions, we are idle + if (combined.isEmpty()) { + consecutiveIdleCycles[taskIndex]++; + notifyIdle(); + break; // reschedule with backoff + } + + // Had data → reset backoff so next reschedule is fast + consecutiveIdleCycles[taskIndex] = 0; + + // Step 3: dispatch the combined batch + dispatch(combined); + // Step 4: loop immediately — more data may have arrived + } + } finally { + if (running) { + scheduleDrain(taskIndex); + } + } + } + + void shutdown() { + running = false; + // Final drain of remaining data across all partitions + List combined = new ArrayList<>(); + for (int i = 0; i < partitions.length; i++) { + partitions[i].drainTo(combined); + } + if (!combined.isEmpty()) { + dispatch(combined); + } + // Only shut down the scheduler if this queue owns it. + // Shared schedulers are shut down by BatchQueueManager.shutdownAll(). + if (dedicatedScheduler) { + scheduler.shutdown(); + } + } +} +``` + +**Dispatch and idle notification:** + +```java +private void dispatch(List batch) { + // Direct consumer mode: pass the whole batch, no groupBy overhead. + if (config.getConsumer() != null) { + try { + config.getConsumer().consume(batch); + } catch (Throwable t) { + config.getErrorHandler().onError(batch, t); + } + return; + } + + // Handler map mode: group by class type and dispatch to registered handlers. + Map, List> grouped = new HashMap<>(); + for (T item : batch) { + grouped.computeIfAbsent(item.getClass(), k -> new ArrayList<>()).add(item); + } + + for (Map.Entry, List> entry : grouped.entrySet()) { + HandlerConsumer handler = handlerMap.get(entry.getKey()); + if (handler != null) { + try { + handler.consume(entry.getValue()); + } catch (Throwable t) { + config.getErrorHandler().onError(entry.getValue(), t); + } + } + } +} + +private void notifyIdle() { + if (config.getConsumer() != null) { + config.getConsumer().onIdle(); + } else { + handlerMap.values().forEach(HandlerConsumer::onIdle); + } +} +``` + +**Consumer workflow (end-to-end):** + +``` +Producer threads Consumer thread (Thread-0, owns partitions [0, 4]) +────────────── ───────────────────────────────────────────────── + +produce(A1) ──offer/put──► Partition-0: [A1, C1, B1] +produce(B1) ──offer/put──► Partition-4: [A2, A3] +produce(C1) ──┘ (round-robin) + + ┌─── scheduleDrain(0) after adaptive backoff delay + │ + ▼ + drainLoop(taskIndex=0) + │ + ┌────────┴───────────────────────────────────────────┐ + │ while (running): │ + │ │ + │ ── Step 1: drain ALL assigned partitions ── │ + │ combined = [] │ + │ Partition-0.drainTo(combined) → [A1, C1, B1] │ + │ Partition-4.drainTo(combined) → [A1,C1,B1,A2,A3]│ + │ │ + │ ── Step 2: check if empty ── │ + │ combined not empty → dispatch │ + │ │ + │ ── Step 3: dispatch(combined) ── │ + │ ┌─ config.consumer set? │ + │ │ YES → consumer.consume(combined) ─── done │ + │ │ NO → groupBy class: │ + │ │ MetricA.class → [A1, A2, A3] │ + │ │ MetricB.class → [B1] │ + │ │ MetricC.class → [C1] │ + │ │ for each group: │ + │ │ handler = handlerMap.get(class) │ + │ │ handler.consume(group) │ + │ │ ↓ │ + │ │ L1Handler_A.consume([A1, A2, A3]) │ + │ │ → workerA.onWork([A1, A2, A3]) │ + │ │ → mergeDataCache.accept(A1) │ + │ │ → mergeDataCache.accept(A2) │ + │ │ → mergeDataCache.accept(A3) │ + │ │ → flush() if period elapsed │ + │ │ L1Handler_B.consume([B1]) │ + │ │ → workerB.onWork([B1]) │ + │ │ L1Handler_C.consume([C1]) │ + │ │ → workerC.onWork([C1]) │ + │ └───────────────────────────────────────────── │ + │ │ + │ ── Step 4: loop immediately (more may have come)──│ + │ │ + │ ── next cycle ── │ + │ combined = [] │ + │ Partition-0.drainTo(combined) → [] │ + │ Partition-4.drainTo(combined) → [] │ + │ combined is empty │ + │ → notifyIdle() │ + │ handlerMap.values().forEach(::onIdle) │ + │ ↓ │ + │ L1Handler_A.onIdle() │ + │ → workerA.flush() (force flush cache) │ + │ L1Handler_B.onIdle() │ + │ → workerB.flush() │ + │ ... │ + │ consecutiveIdleCycles[0]++ (e.g. now = 1) │ + │ break │ + └────────────────────────────────────────────────────┘ + │ + ▼ (finally block) + scheduleDrain(0) + │ + ├─ idleCount=0 (just had data): wait 5ms ← fast re-poll + ├─ idleCount=1: wait 10ms + ├─ idleCount=2: wait 20ms + ├─ idleCount=3: wait 40ms + ├─ idleCount=4: wait 80ms + ├─ idleCount=5: wait 160ms + ├─ idleCount=6+: wait 200ms ← capped at maxIdleMs + │ + ▼ + drainLoop again + │ + (if data found → idleCount resets to 0 → back to fast polling) +``` + +**Key points:** +- Each cycle drains ALL assigned partitions into one combined batch before dispatching. +- `dispatch()` is called once per cycle with all data from all partitions combined. +- In handler map mode, `groupBy(class)` collects all items of the same type across all + partitions into one list. The handler receives ALL available data of its type in a single + `consume()` call — e.g., `[A1, A2, A3]` not three separate calls. +- The handler's `consume()` runs **synchronously** inside the drain thread. The handler + (an inner class of the worker) directly accesses the worker's fields — merge cache, + counters, flush logic — with no extra threading. +- If any partition had data, loop immediately to check all partitions again. +- `onIdle()` fires only when ALL assigned partitions are empty in a full cycle, giving + handlers a chance to flush periodic caches (e.g., L1 aggregation merge cache → nextWorker). +- **Adaptive backoff**: after data, re-poll in `minIdleMs` (5ms). Each consecutive empty + cycle doubles the sleep, capping at `maxIdleMs` (200ms). Data resets to fast polling. + +**Two consumption modes, same queue class:** +- **Direct consumer** (`config.consumer` set) — whole batch goes to one handler, no groupBy. + Use for I/O queues where all data is the same type (gRPC, Kafka, JDBC). +- **Handler map** (`addHandler` called) — batch grouped by class, dispatched per type. + Use for metrics aggregation (L1/L2) and TopN with many types sharing partitions. + +If both are set, direct consumer takes priority (handler map is ignored). +If neither is set, data is drained but silently dropped. + +### BatchQueueConfig + +```java +@Builder +public class BatchQueueConfig { + /** + * Thread policy for a dedicated ScheduledExecutorService. + * Resolved at queue construction time. Examples: + * ThreadPolicy.fixed(8) → always 8 threads + * ThreadPolicy.cpuCores(1.0) → 1x available CPU cores (8 on 8-core) + * ThreadPolicy.cpuCores(0.25) → 0.25x CPU cores (2 on 8-core, min 1) + * + * When set, the queue creates its own scheduler. + * When null, sharedScheduler must be set — the queue uses a shared scheduler. + * + * Use dedicated pools for high-throughput queues (metrics L1/L2 aggregation) + * where you need guaranteed thread capacity. + */ + private ThreadPolicy threads; + + /** + * Shared scheduler name and its ThreadPolicy. Set via the builder method + * sharedScheduler(name, threads) which populates both fields together. + * Mutually exclusive with the threads field above. + * + * When set, the queue registers its drain tasks on a shared + * ScheduledExecutorService managed by BatchQueueManager. + * The shared scheduler is created lazily on first queue reference — + * no separate setup step needed. Subsequent queues with the same name + * reuse the existing scheduler (ThreadPolicy mismatch logs a warning). + * + * Use for low-throughput I/O queues (gRPC, Kafka, JDBC) to reduce OS thread count. + * Multiple queues sharing the same scheduler name share the same thread pool. + * + * Exactly one of threads or sharedScheduler must be set. + */ + private String sharedSchedulerName; + private ThreadPolicy sharedSchedulerThreads; + + /** + * Number of partitions, or a policy to derive it from resolved thread count. + * + * Can be set as: + * - Absolute: fixed(8) → exactly 8 partitions. + * - Relative: threadMultiply(2) → 2x resolved thread count. + * e.g. cpuCores(0.5) on 8-core = 4 threads, threadMultiply(2) → 8 partitions. + * + * Default: fixed(1). + * + * Validation (applied at construction time for dedicated scheduler mode): + * - If partitions < resolved thread count, thread count is reduced to match + * partitions and a warning is logged. No point having more threads than partitions. + */ + @Builder.Default + private PartitionPolicy partitions = PartitionPolicy.fixed(1); + + /** + * Buffer size per partition. + */ + @Builder.Default + private int bufferSize = 10_000; + + /** + * BLOCKING: producer waits when buffer full. + * IF_POSSIBLE: producer gets false when buffer full. + */ + @Builder.Default + private BufferStrategy strategy = BufferStrategy.BLOCKING; + + /** + * Direct consumer for the whole batch. When set, all drained data goes to this + * handler without class-based grouping. Takes priority over handler map. + * + * Use this for I/O queues where all data is the same type (gRPC, Kafka, JDBC). + * Leave null to use handler map dispatch via addHandler(). + */ + private HandlerConsumer consumer; + + /** + * Called when a handler throws during consume. + */ + private QueueErrorHandler errorHandler; + + /** + * Minimum idle sleep in milliseconds. Default 5ms. + * Used as the base interval when data was recently consumed. + * The thread re-polls quickly to catch new data with low latency. + */ + @Builder.Default + private long minIdleMs = 5; + + /** + * Maximum idle sleep in milliseconds. Default 200ms. + * After several consecutive empty cycles, the sleep interval backs off + * exponentially from minIdleMs up to this cap. + * + * Backoff: sleep = min(minIdleMs * 2^consecutiveIdleCycles, maxIdleMs) + * idle 0: 5ms → idle 1: 10ms → idle 2: 20ms → idle 3: 40ms + * → idle 4: 80ms → idle 5: 160ms → idle 6+: 200ms (capped) + */ + @Builder.Default + private long maxIdleMs = 200; +} +``` + +### HandlerConsumer + +```java +/** + * Handler for processing a batch of data for a specific type. + * Each metric type (or I/O queue user) provides its own handler instance. + */ +public interface HandlerConsumer { + /** + * Process a batch of data belonging to this handler's type. + */ + void consume(List data); + + /** + * Called when there is nothing to consume. Can be used as a timer trigger + * (e.g. flush L1 aggregation cache periodically). + */ + default void onIdle() { + } +} +``` + +### ThreadPolicy + +```java +/** + * Determines the number of threads for a BatchQueue's dedicated scheduler + * or for a shared scheduler created via BatchQueueManager. + * + * Two modes: + * - fixed(N): exactly N threads, regardless of hardware. + * - cpuCores(multiplier): multiplier * Runtime.availableProcessors(), rounded. + * + * Resolved value is always >= 1 — every pool must have at least one thread. + * fixed() requires count >= 1 at construction. cpuCores() applies max(1, ...) at resolution. + */ +public class ThreadPolicy { + private final int fixedCount; // > 0 for fixed mode, 0 for cpuCores mode + private final double cpuMultiplier; // > 0 for cpuCores mode, 0 for fixed mode + + /** + * Fixed number of threads. Count must be >= 1. + * Example: fixed(1) → always 1 thread. + * fixed(8) → always 8 threads. + * Throws IllegalArgumentException if count < 1. + */ + public static ThreadPolicy fixed(int count); + + /** + * Threads = multiplier * available CPU cores, rounded, min 1. + * Multiplier must be > 0. + * Example on 8-core machine: + * cpuCores(1.0) → 8 threads + * cpuCores(0.5) → 4 threads + * cpuCores(0.25) → 2 threads + * cpuCores(2.0) → 16 threads + * Example on 2-core machine: + * cpuCores(0.25) → 1 thread (min 1, never 0) + */ + public static ThreadPolicy cpuCores(double multiplier); + + /** + * Resolve the actual thread count. Always returns >= 1. + * For fixed mode, returns fixedCount. + * For cpuCores mode, returns max(1, round(cpuMultiplier * availableProcessors())). + */ + public int resolve(); +} +``` + +### PartitionPolicy + +```java +/** + * Determines the number of partitions for a BatchQueue. + * + * Two modes: + * - fixed(N): exactly N partitions, regardless of thread count. + * - threadMultiply(N): N * resolved thread count. + * - adaptive(): partition count grows with registered handlers. + * Threshold = threadCount * multiplier (default 25). + * Below threshold: 1:1 (one partition per handler). + * Above threshold: excess at 1:2 ratio. + * + * All policies resolved via resolve(threadCount, handlerCount). + * At queue construction time, if partitions < resolved thread count, + * thread count is reduced to match and a warning is logged. + */ +public class PartitionPolicy { + private final int fixedCount; // > 0 for fixed mode + private final int multiplier; // > 0 for threadMultiply/adaptive + private final boolean adaptive; // true for adaptive mode + + public static PartitionPolicy fixed(int count); + public static PartitionPolicy threadMultiply(int multiplier); + public static PartitionPolicy adaptive(); + public static PartitionPolicy adaptive(int multiplier); + + /** + * Resolve the actual partition count. + * - fixed: returns fixedCount (both params ignored). + * - threadMultiply: returns multiplier * resolvedThreadCount. + * - adaptive: handlerCount == 0 → resolvedThreadCount; + * handlerCount <= threshold → handlerCount (1:1); + * handlerCount > threshold → threshold + (excess / 2). + */ + public int resolve(int resolvedThreadCount, int handlerCount); +} +``` + +### Implementing HandlerConsumer + +Each worker creates a handler as an **inner class** that directly accesses the worker's +fields (merge cache, telemetry counters, flush logic, etc.). The handler instance is +registered per metric class — one handler per worker, one worker per metric type. + +#### L1 Aggregation (MetricsAggregateWorker) + +Current code uses an inner `AggregatorConsumer` that calls `onWork()` and `flush()`: + +```java +// Current: inner class IConsumer accesses outer worker fields +private class AggregatorConsumer implements IConsumer { + public void consume(List data) { + MetricsAggregateWorker.this.onWork(data); // accesses mergeDataCache + } + public void nothingToConsume() { + flush(); // accesses lastSendTime, nextWorker + } +} +``` + +New code — same pattern, just implements `HandlerConsumer` instead of `IConsumer`. +Each `MetricsAggregateWorker` instance creates its own handler and registers it for +its specific metric class: + +```java +public class MetricsAggregateWorker extends AbstractWorker { + private final MergableBufferedData mergeDataCache; + private final AbstractWorker nextWorker; + private final BatchQueue l1Queue; + + MetricsAggregateWorker(ModuleDefineHolder moduleDefineHolder, + AbstractWorker nextWorker, + String modelName, + Class metricsClass, + ...) { + this.nextWorker = nextWorker; + this.mergeDataCache = new MergableBufferedData<>(); + + // Get or create the shared L1 queue (idempotent) + this.l1Queue = BatchQueueManager.createIfAbsent( + "METRICS_L1_AGGREGATION", + BatchQueueConfig.builder() + .threads(ThreadPolicy.cpuCores(1.0)) // 1x CPU cores + .partitions(PartitionPolicy.threadMultiply(2)) // 2x resolved threads + .bufferSize(10_000) + .strategy(BufferStrategy.IF_POSSIBLE) + .errorHandler((data, t) -> log.error(t.getMessage(), t)) + .build() + ); + + // Register this worker's handler for its specific metric class. + // The inner class directly accesses mergeDataCache, nextWorker, etc. + l1Queue.addHandler(metricsClass, new L1Handler()); + } + + @Override + public void in(Metrics metrics) { + l1Queue.produce(metrics); + } + + private void onWork(List data) { + data.forEach(mergeDataCache::accept); + flush(); + } + + private void flush() { + if (System.currentTimeMillis() - lastSendTime > l1FlushPeriod) { + mergeDataCache.read().forEach(nextWorker::in); + lastSendTime = System.currentTimeMillis(); + } + } + + // Inner class handler — accesses worker fields directly + private class L1Handler implements HandlerConsumer { + @Override + public void consume(List data) { + MetricsAggregateWorker.this.onWork(data); + } + + @Override + public void onIdle() { + MetricsAggregateWorker.this.flush(); + } + } +} +``` + +Key point: 100+ `MetricsAggregateWorker` instances are created (one per metric type, +both OAL and MAL), each registers its own `L1Handler` inner class instance on the +same shared queue. Handler map dispatch routes each metric class to its own worker's +`mergeDataCache` — OAL and MAL handlers coexist in the same queue without interference. + +#### L2 Persistent (MetricsPersistentMinWorker) + +Same pattern — inner class handler accesses the worker's `onWork()` method: + +```java +public class MetricsPersistentMinWorker extends MetricsPersistentWorker { + private final BatchQueue l2Queue; + + MetricsPersistentMinWorker(..., Class metricsClass, ...) { + super(...); + + this.l2Queue = BatchQueueManager.createIfAbsent( + "METRICS_L2_PERSISTENT", + BatchQueueConfig.builder() + .threads(ThreadPolicy.cpuCores(0.25)) // 0.25x CPU cores (2 on 8-core) + .partitions(PartitionPolicy.threadMultiply(2)) // 2x resolved threads + .bufferSize(10_000) + .strategy(BufferStrategy.BLOCKING) + .errorHandler((data, t) -> log.error(t.getMessage(), t)) + .build() + ); + + l2Queue.addHandler(metricsClass, new L2Handler()); + } + + @Override + public void in(Metrics metrics) { + l2Queue.produce(metrics); + } + + // Inner class handler — accesses worker's onWork, queuePercentageGauge, etc. + private class L2Handler implements HandlerConsumer { + @Override + public void consume(List data) { + queuePercentageGauge.setValue(...); + MetricsPersistentMinWorker.this.onWork(data); + } + } +} +``` + +#### TopN (TopNWorker — shared queue, handler map dispatch) + +All TopN types share one queue. Same handler map pattern as L1/L2 — each TopNWorker +registers its inner class handler for its specific TopN class: + +```java +public class TopNWorker extends PersistenceWorker { + private final BatchQueue topNQueue; + + TopNWorker(..., Class topNClass, ...) { + // Get or create the shared TopN queue (idempotent) + this.topNQueue = BatchQueueManager.createIfAbsent( + "TOPN_WORKER", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) // all TopN types share 1 thread + .partitions(PartitionPolicy.fixed(1)) + .bufferSize(1000) + .strategy(BufferStrategy.BLOCKING) + .errorHandler((data, t) -> log.error(t.getMessage(), t)) + .build() + ); + + // Register this worker's handler for its specific TopN class + topNQueue.addHandler(topNClass, new TopNHandler()); + } + + @Override + public void in(TopN topN) { + topNQueue.produce(topN); + } + + // Inner class — accesses worker's limitedSizeBufferedData, reportPeriod, etc. + private class TopNHandler implements HandlerConsumer { + @Override + public void consume(List data) { + TopNWorker.this.onWork(data); + } + + @Override + public void onIdle() { + TopNWorker.this.flushIfNeeded(); + } + } +} +``` + +### Thread Reduction + +Thread counts scale with CPU cores via ThreadPolicy. OAL and MAL share the same L1/L2 pools. +Example on 8-core machine: + +| Before (DataCarrier) | Threads | After (BatchQueue) | Threads (8-core) | +|---------------------------------|---------|-------------------------------------------------|------------------| +| L1 OAL pool | 24 | METRICS_L1_AGGREGATION (cpuCores(1.0)) | 8 | +| L1 MAL pool | 2 | *(shared with L1 above)* | *(shared)* | +| L2 OAL pool | 2 | METRICS_L2_PERSISTENT (cpuCores(0.25)) | 2 | +| L2 MAL pool | 1 | *(shared with L2 above)* | *(shared)* | +| TopNWorker (5-10 types) | 5-10 | TOPN_WORKER (fixed(1), handler map) | 1 | +| GRPCRemoteClient (2-4 peers) | 2-4 | GRPCRemoteClient.* (shared IO_POOL) | | +| JDBCBatchDAO | 2-4 | JDBCBatchDAO (shared IO_POOL) | | +| Exporters (gRPC/Kafka) | 0-3 | Exporter.* (shared IO_POOL) | | +| | | **IO_POOL shared scheduler (cpuCores(0.5))** | **4** | +| **Total** | **~38-48** | **Total** | **~15** | + +On different hardware: + +| Machine | L1 (OAL+MAL) | L2 (OAL+MAL) | TopN | IO_POOL | Total | +|-----------|---------------|---------------|------|---------|-------| +| 2-core | 2 | 1 | 1 | 1 | 5 | +| 4-core | 4 | 1 | 1 | 2 | 8 | +| 8-core | 8 | 2 | 1 | 4 | 15 | +| 16-core | 16 | 4 | 1 | 8 | 29 | + +Savings (8-core): +- L1: 24+2 → 8 threads (OAL+MAL share one pool, CPU-relative, no empty channel iteration) +- L2: 2+1 → 2 threads (OAL+MAL share one pool) +- TopN: 5-10 → 1 thread (all types share one queue with handler map dispatch) +- I/O queues: 4-11 → 4 threads (shared IO_POOL for gRPC, Kafka, JDBC) +- Total: from ~38-48 OS threads down to ~15 + +IO_POOL queues (all do network or database I/O, low-throughput, bursty): + +| Queue Name | Current Source | I/O Type | Threads Before | +|-------------------------|------------------------|---------------|----------------| +| GRPCRemoteClient.* | GRPCRemoteClient.java | gRPC network | 2-4 (per peer) | +| GRPCMetricsExporter | GRPCMetricsExporter.java| gRPC network | 1 | +| KafkaLogExporter | KafkaLogExporter.java | Kafka network | 1 | +| KafkaTraceExporter | KafkaTraceExporter.java| Kafka network | 1 | +| JDBCBatchDAO | JDBCBatchDAO.java | JDBC database | 2-4 | +| **Subtotal** | | | **7-14** | +| **After (IO_POOL)** | | | **cpuCores(0.5)** | + +Number of partitions/buffers: + +| Before | Count | After | Count | +|--------------------------------|----------|------------------------------------|-------| +| L1 OAL channels (100+ * 2ch) | 200+ | L1 partitions (8 threads * 2) | 16 | +| L1 MAL channels (N * 1ch) | N | *(shared with L1 above)* | *(0)* | +| L2 OAL channels (100+ * 1ch) | 100+ | L2 partitions (2 threads * 2) | 4 | +| L2 MAL channels (N * 1ch) | N | *(shared with L2 above)* | *(0)* | +| TopN buffers (5-10 types) | 5-10 | TOPN_WORKER partitions | 1 | +| I/O buffers (gRPC, JDBC, etc.) | 5-8 | I/O queue partitions | 5-8 | +| **Total buffers** | **300+** | **Total buffers** | **~28** | + +### What Gets Dropped + +| DataCarrier Feature | Status | Reason | +|-------------------------------------|-----------|-------------------------------------------------------| +| One queue per metric type | Dropped | Shared partitions + handler map instead | +| Separate OAL / MAL pools | Dropped | OAL and MAL share L1/L2 queues (handler map dispatch) | +| One thread per TopN type | Dropped | All TopN types share one TOPN_WORKER queue | +| Multi-channel per DataCarrier | Dropped | Single partition array replaces multi-channel | +| IDataPartitioner | Dropped | Simple round-robin on partition array | +| Consumer instantiation by class | Dropped | All callers use instance-based handlers | +| Consumer init(Properties) | Dropped | Not used by any production consumer | +| EnvUtil override | Dropped | Configuration via application.yml | +| Two separate queue classes | Dropped | One `BatchQueue` with configurable scheduler modes | +| BulkConsumePool / ConsumerPoolFactory | Dropped | Dedicated/shared ScheduledExecutorService replaces pool| +| Fixed thread counts | Dropped | ThreadPolicy: CPU-relative (cpuCores) or fixed | +| Signal-driven consumption | Dropped | Adaptive backoff replaces explicit notify | +| Separate createSharedScheduler step | Dropped | Shared schedulers created lazily on first queue ref | + +### What Gets Preserved + +| Feature | How | +|-----------------------|--------------------------------------------------------------| +| Named queue management| `BatchQueueManager.create/createIfAbsent/get` by name | +| Per-type isolation | `handlerMap` dispatches each class to its own handler | +| Bounded buffer | ArrayBlockingQueue per partition | +| BLOCKING strategy | `queue.put()` — producer blocks when full | +| IF_POSSIBLE strategy | `queue.offer()` — returns false when full, data dropped | +| Batch consumption | `drainTo(list)` — same as current | +| Error handling | `errorHandler.onError(batch, throwable)` | +| Nothing-to-consume | `handler.onIdle()` — called when all partitions empty | +| Fast data response | Adaptive backoff (minIdleMs=5ms) replaces signal-driven mode | +| Drain on shutdown | Manager shutdown drains all queues, then schedulers | +| Produce-gate | `produce()` returns false if queue is shut down | +| Hardware scaling | ThreadPolicy.cpuCores() scales threads with available cores | diff --git a/oap-server/server-library/library-batch-queue/pom.xml b/oap-server/server-library/library-batch-queue/pom.xml new file mode 100644 index 000000000000..455b77f4478a --- /dev/null +++ b/oap-server/server-library/library-batch-queue/pom.xml @@ -0,0 +1,48 @@ + + + + + + server-library + org.apache.skywalking + ${revision} + + 4.0.0 + + library-batch-queue + + + + org.awaitility + awaitility + test + + + org.apache.logging.log4j + log4j-core + test + + + org.apache.logging.log4j + log4j-slf4j-impl + test + + + + diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java new file mode 100644 index 000000000000..f6a117039e8b --- /dev/null +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java @@ -0,0 +1,516 @@ +/* + * 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.skywalking.oap.server.library.batchqueue; + +import java.util.ArrayList; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.ArrayBlockingQueue; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.TimeUnit; +import lombok.extern.slf4j.Slf4j; + +/** + * A partitioned, self-draining queue with type-based dispatch. + * + *

Usage

+ *
+ * BatchQueue queue = BatchQueueManager.create(name, config);
+ * queue.addHandler(TypeA.class, handlerA);   // register metric types
+ * queue.addHandler(TypeB.class, handlerB);   // partitions grow adaptively
+ * queue.produce(data);                        // data flows immediately
+ * 
+ * + *

For adaptive partition policies, each {@link #addHandler} call recalculates + * the partition count from the current number of registered handlers, growing the + * partition array as needed. The thread count is resolved at construction time + * and remains fixed. + * + *

Produce workflow

+ *
+ * produce(data)
+ *   |
+ *   +-- queue stopped?  --> return false
+ *   |
+ *   +-- N == 1?  --> partition[0]  (skip selector)
+ *   +-- N > 1?  --> partition[selector.select(data, N)]
+ *   |
+ *   +-- BLOCKING strategy?
+ *   |     yes --> ArrayBlockingQueue.put(data)   // blocks until space available
+ *   |     no  --> ArrayBlockingQueue.offer(data) // returns false if full (drop)
+ *   |
+ *   +-- return true/false
+ * 
+ * + *

Consume workflow (drain loop, runs on scheduler threads)

+ *
+ * scheduleDrain(taskIndex)                  // schedule with adaptive backoff delay
+ *   |
+ *   +-- scheduler.schedule(drainLoop, delay)
+ *         |
+ *         +-- drainTo(combined) from each assigned partition
+ *         |
+ *         +-- combined is empty?
+ *         |     yes --> consecutiveIdleCycles++, notify onIdle(), break
+ *         |     no  --> consecutiveIdleCycles = 0, dispatch(combined)
+ *         |              |
+ *         |              +-- single consumer set?
+ *         |              |     yes --> consumer.consume(batch)
+ *         |              |     no  --> group by item.getClass()
+ *         |              |              for each (class, subBatch):
+ *         |              |                handlerMap.get(class).consume(subBatch)
+ *         |              |
+ *         |              +-- loop back to drainTo (drain until empty)
+ *         |
+ *         +-- finally: scheduleDrain(taskIndex) // re-schedule self
+ * 
+ * + *

Adaptive backoff

+ * Delay doubles on each consecutive idle cycle: {@code minIdleMs * 2^idleCount}, + * capped at {@code maxIdleMs}. Resets to {@code minIdleMs} on first non-empty drain. + * + *

Scheduler modes

+ *
    + *
  • Dedicated — this queue owns its own ScheduledThreadPool. Each thread + * is assigned a fixed subset of partitions (round-robin). Thread count and + * partition count are configured independently.
  • + *
  • Shared — the queue borrows a scheduler from {@link BatchQueueManager}, + * shared with other queues. Only 1 drain task is submitted (drains all partitions). + * The shared scheduler is reference-counted and shut down when the last queue + * releases it.
  • + *
+ * + *

Use case examples

+ *
+ * shared scheduler, partitions=1, one consumer    --> I/O queue (gRPC, Kafka, JDBC)
+ * dedicated fixed(1), partitions=1, many handlers --> TopN (all types share 1 thread)
+ * dedicated cpuCores(1.0), adaptive(),
+ *           many handlers                         --> metrics aggregation
+ * 
+ */ +@Slf4j +public class BatchQueue { + private final String name; + private final BatchQueueConfig config; + + /** The thread pool that executes drain tasks. Either dedicated or shared. */ + private final ScheduledExecutorService scheduler; + + /** True if this queue owns the scheduler and should shut it down. */ + private final boolean dedicatedScheduler; + + /** Non-null only for shared schedulers; used to release the ref count on shutdown. */ + private final String sharedSchedulerName; + + /** + * Cached partition selector from config. Only used when {@code partitions.length > 1}; + * single-partition queues bypass the selector entirely. + */ + private final PartitionSelector partitionSelector; + + /** + * Type-based handler registry. When no single consumer is configured, + * drained items are grouped by {@code item.getClass()} and dispatched + * to the matching handler. Allows multiple metric/record types to share + * one queue while each type has its own processing logic. + */ + private final ConcurrentHashMap, HandlerConsumer> handlerMap; + + /** + * Tracks unregistered types that have already been warned about, + * to avoid flooding the log with repeated errors. + */ + private final Set> warnedUnregisteredTypes; + + /** Resolved thread count, stored for adaptive partition recalculation. */ + private final int resolvedThreadCount; + + /** Number of drain tasks (equals thread count for dedicated, 1 for shared). */ + private final int taskCount; + + /** + * Partitions. Producers select a partition via {@link PartitionSelector}. + * For adaptive policies, this array grows via {@link #addHandler} as handlers + * are registered. Volatile for visibility to drain loop threads. + */ + private volatile ArrayBlockingQueue[] partitions; + + /** + * Which partitions each drain task is responsible for. + * Volatile for visibility when partitions grow via {@link #addHandler}. + */ + private volatile int[][] assignedPartitions; + + /** + * Per-task count of consecutive idle cycles (no data drained). + * Used for adaptive exponential backoff in {@link #scheduleDrain}. + */ + private final int[] consecutiveIdleCycles; + + /** Set to false on {@link #shutdown()} to stop drain loops and reject new data. */ + private volatile boolean running; + + @SuppressWarnings("unchecked") + BatchQueue(final String name, final BatchQueueConfig config) { + this.name = name; + this.config = config; + this.partitionSelector = config.getPartitionSelector(); + this.handlerMap = new ConcurrentHashMap<>(); + this.warnedUnregisteredTypes = ConcurrentHashMap.newKeySet(); + + if (config.getSharedSchedulerName() != null) { + // ---- Shared scheduler mode ---- + final ScheduledExecutorService sharedScheduler = + BatchQueueManager.getOrCreateSharedScheduler( + config.getSharedSchedulerName(), config.getSharedSchedulerThreads()); + + this.resolvedThreadCount = 1; + final int partitionCount = config.getPartitions().resolve(1, 0); + this.partitions = new ArrayBlockingQueue[partitionCount]; + for (int i = 0; i < partitions.length; i++) { + partitions[i] = new ArrayBlockingQueue<>(config.getBufferSize()); + } + + this.scheduler = sharedScheduler; + this.dedicatedScheduler = false; + this.sharedSchedulerName = config.getSharedSchedulerName(); + this.taskCount = 1; + this.assignedPartitions = buildAssignments(1, partitionCount); + } else { + // ---- Dedicated scheduler mode ---- + int threadCount = config.getThreads().resolve(); + this.resolvedThreadCount = threadCount; + + // For adaptive with 0 handlers, resolve returns threadCount (sensible initial). + // For fixed/threadMultiply, resolve returns the configured count. + final int partitionCount = config.getPartitions().resolve(threadCount, 0); + + if (partitionCount < threadCount) { + log.warn("BatchQueue[{}]: partitions({}) < threads({}), reducing threads to {}", + name, partitionCount, threadCount, partitionCount); + threadCount = partitionCount; + } + + this.partitions = new ArrayBlockingQueue[partitionCount]; + for (int i = 0; i < partitions.length; i++) { + partitions[i] = new ArrayBlockingQueue<>(config.getBufferSize()); + } + + this.scheduler = Executors.newScheduledThreadPool(threadCount, r -> { + final Thread t = new Thread(r); + t.setName("BatchQueue-" + name + "-" + t.getId()); + t.setDaemon(true); + return t; + }); + this.dedicatedScheduler = true; + this.sharedSchedulerName = null; + this.taskCount = threadCount; + this.assignedPartitions = buildAssignments(threadCount, partitionCount); + } + + this.consecutiveIdleCycles = new int[taskCount]; + this.running = true; + // Kick off the drain loop for each task + for (int t = 0; t < taskCount; t++) { + scheduleDrain(t); + } + } + + /** + * Build round-robin partition-to-task assignments. + */ + private static int[][] buildAssignments(final int taskCount, final int partitionCount) { + final int[][] result = new int[taskCount][]; + final List> assignment = new ArrayList<>(); + for (int t = 0; t < taskCount; t++) { + assignment.add(new ArrayList<>()); + } + for (int p = 0; p < partitionCount; p++) { + assignment.get(p % taskCount).add(p); + } + for (int t = 0; t < taskCount; t++) { + final List parts = assignment.get(t); + result[t] = new int[parts.size()]; + for (int i = 0; i < parts.size(); i++) { + result[t][i] = parts.get(i); + } + } + return result; + } + + /** + * Schedule the next drain for the given task. The delay uses adaptive exponential + * backoff: {@code minIdleMs * 2^consecutiveIdleCycles}, capped at maxIdleMs. + * When data is flowing, consecutiveIdleCycles is 0, so delay = minIdleMs. + */ + private void scheduleDrain(final int taskIndex) { + if (!running) { + return; + } + final int idleCount = consecutiveIdleCycles[taskIndex]; + final long delay = Math.min( + config.getMinIdleMs() * (1L << Math.min(idleCount, 20)), + config.getMaxIdleMs() + ); + try { + scheduler.schedule(() -> drainLoop(taskIndex), delay, TimeUnit.MILLISECONDS); + } catch (final Exception e) { + if (running) { + log.error("BatchQueue[{}]: failed to schedule drain task", name, e); + } + } + } + + /** + * Register a type-based handler. Items whose {@code getClass()} matches the given + * type will be batched together and dispatched to this handler. + * + *

For adaptive partition policies, adding a handler recalculates the partition + * count and grows the partition array if needed. For non-adaptive policies the + * resolved count never changes, so this is a no-op beyond the registration. + * Drain loop threads pick up new partitions on their next cycle via volatile reads. + */ + @SuppressWarnings("unchecked") + public void addHandler(final Class type, final HandlerConsumer handler) { + handlerMap.put(type, handler); + + final int newPartitionCount = config.getPartitions() + .resolve(resolvedThreadCount, handlerMap.size()); + final ArrayBlockingQueue[] currentPartitions = this.partitions; + if (newPartitionCount > currentPartitions.length) { + final ArrayBlockingQueue[] grown = new ArrayBlockingQueue[newPartitionCount]; + System.arraycopy(currentPartitions, 0, grown, 0, currentPartitions.length); + for (int i = currentPartitions.length; i < newPartitionCount; i++) { + grown[i] = new ArrayBlockingQueue<>(config.getBufferSize()); + } + // Volatile writes — drain loop threads see the new assignments on next cycle + this.assignedPartitions = buildAssignments(taskCount, newPartitionCount); + this.partitions = grown; + } + } + + /** + * Produce data into a partition selected by the configured {@link PartitionSelector}. + * + *

Single-partition queues bypass the selector entirely (always index 0). + * Multi-partition queues default to {@link PartitionSelector#typeHash()}, which + * uses {@code data.getClass().hashCode()} so all items of the same type land on + * the same partition — the HashMap grouping in {@link #dispatch} becomes + * effectively a no-op. + * + *

Behavior depends on {@link BufferStrategy}: + *

    + *
  • BLOCKING — blocks the caller until space is available
  • + *
  • IF_POSSIBLE — returns false immediately if the partition is full (data dropped)
  • + *
+ * + * @return true if data was accepted, false if dropped or queue is stopped + */ + public boolean produce(final T data) { + if (!running) { + return false; + } + final ArrayBlockingQueue[] currentPartitions = this.partitions; + final int index = currentPartitions.length == 1 + ? 0 : partitionSelector.select(data, currentPartitions.length); + if (config.getStrategy() == BufferStrategy.BLOCKING) { + try { + currentPartitions[index].put(data); + return true; + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + return false; + } + } else { + return currentPartitions[index].offer(data); + } + } + + /** + * Core drain loop. Runs on a scheduler thread. Drains all assigned partitions + * into a single list, then dispatches. Loops until partitions are empty, + * then breaks to re-schedule with backoff. + * + *

Reads volatile references to {@code partitions} and {@code assignedPartitions} + * at the start of each cycle, so it picks up new partitions added via + * {@link #addHandler} on the next iteration. + */ + void drainLoop(final int taskIndex) { + final ArrayBlockingQueue[] currentPartitions = this.partitions; + final int[] myPartitions = this.assignedPartitions[taskIndex]; + try { + while (running) { + // Drain all assigned partitions into one batch + final List combined = new ArrayList<>(); + for (final int partitionIndex : myPartitions) { + if (partitionIndex < currentPartitions.length) { + currentPartitions[partitionIndex].drainTo(combined); + } + } + + if (combined.isEmpty()) { + // Nothing to drain — increase backoff and notify idle + consecutiveIdleCycles[taskIndex]++; + notifyIdle(); + break; + } + + // Data found — reset backoff and dispatch + consecutiveIdleCycles[taskIndex] = 0; + dispatch(combined); + } + } catch (final Throwable t) { + log.error("BatchQueue[{}]: drain loop error", name, t); + } finally { + // Always re-schedule unless shutdown + if (running) { + scheduleDrain(taskIndex); + } + } + } + + /** + * Dispatch a batch to the appropriate consumer(s). + * + *

Two modes: + *

    + *
  1. Single consumer (config.consumer != null): the entire batch goes + * to one consumer, regardless of item types.
  2. + *
  3. Handler map: items are grouped by {@code item.getClass()}, then + * each group is dispatched to its registered handler. Unregistered types + * are logged as errors and dropped.
  4. + *
+ */ + private void dispatch(final List batch) { + if (config.getConsumer() != null) { + try { + config.getConsumer().consume(batch); + } catch (final Throwable t) { + handleError(batch, t); + } + return; + } + + // Group by concrete class, then dispatch each group to its handler + final Map, List> grouped = new HashMap<>(); + for (final T item : batch) { + grouped.computeIfAbsent(item.getClass(), k -> new ArrayList<>()).add(item); + } + + for (final Map.Entry, List> entry : grouped.entrySet()) { + final HandlerConsumer handler = handlerMap.get(entry.getKey()); + if (handler != null) { + try { + handler.consume(entry.getValue()); + } catch (final Throwable t) { + handleError(entry.getValue(), t); + } + } else { + if (warnedUnregisteredTypes.add(entry.getKey())) { + log.error("BatchQueue[{}]: no handler for type {}, {} items abandoned", + name, entry.getKey().getName(), entry.getValue().size()); + } + } + } + } + + /** + * Notify consumer/handlers that a drain cycle found no data. + * Useful for flush-on-idle semantics (e.g. flush partial batches to storage). + */ + private void notifyIdle() { + if (config.getConsumer() != null) { + try { + config.getConsumer().onIdle(); + } catch (final Throwable t) { + log.error("BatchQueue[{}]: onIdle error in consumer", name, t); + } + } else { + for (final HandlerConsumer handler : handlerMap.values()) { + try { + handler.onIdle(); + } catch (final Throwable t) { + log.error("BatchQueue[{}]: onIdle error in handler", name, t); + } + } + } + } + + private void handleError(final List data, final Throwable t) { + if (config.getErrorHandler() != null) { + try { + config.getErrorHandler().onError(data, t); + } catch (final Throwable inner) { + log.error("BatchQueue[{}]: error handler threw", name, inner); + } + } else { + log.error("BatchQueue[{}]: unhandled dispatch error", name, t); + } + } + + /** + * Stop the queue: reject new produces, perform a final drain of all partitions, + * and release the scheduler (dedicated: shutdown; shared: decrement ref count). + */ + void shutdown() { + running = false; + // Final drain — flush any remaining data to consumers + final ArrayBlockingQueue[] currentPartitions = this.partitions; + final List combined = new ArrayList<>(); + for (final ArrayBlockingQueue partition : currentPartitions) { + partition.drainTo(combined); + } + if (!combined.isEmpty()) { + dispatch(combined); + } + if (dedicatedScheduler) { + scheduler.shutdown(); + } else if (sharedSchedulerName != null) { + BatchQueueManager.releaseSharedScheduler(sharedSchedulerName); + } + } + + public String getName() { + return name; + } + + public boolean isRunning() { + return running; + } + + int getPartitionCount() { + return partitions.length; + } + + int getTaskCount() { + return assignedPartitions.length; + } + + int[][] getAssignedPartitions() { + return assignedPartitions; + } + + boolean isDedicatedScheduler() { + return dedicatedScheduler; + } +} diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueConfig.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueConfig.java new file mode 100644 index 000000000000..5de0fd39d2be --- /dev/null +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueConfig.java @@ -0,0 +1,110 @@ +/* + * 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.skywalking.oap.server.library.batchqueue; + +import lombok.Builder; +import lombok.Getter; + +@Getter +@Builder +public class BatchQueueConfig { + /** + * Thread policy for a dedicated ScheduledExecutorService. + * When set, the queue creates its own scheduler. + * When null, sharedSchedulerName/sharedSchedulerThreads must be set. + */ + private ThreadPolicy threads; + + /** + * Shared scheduler name. Set via the builder method + * {@code sharedScheduler(name, threads)}. + */ + private String sharedSchedulerName; + + /** + * Thread policy for the shared scheduler. Set together with sharedSchedulerName. + */ + private ThreadPolicy sharedSchedulerThreads; + + @Builder.Default + private PartitionPolicy partitions = PartitionPolicy.fixed(1); + + /** + * Partition selector for multi-partition queues. Determines which partition + * a produced item is placed into. Ignored when the queue has only 1 partition. + * Defaults to {@link PartitionSelector#typeHash()}. + */ + @Builder.Default + private PartitionSelector partitionSelector = PartitionSelector.typeHash(); + + @Builder.Default + private int bufferSize = 10_000; + + @Builder.Default + private BufferStrategy strategy = BufferStrategy.BLOCKING; + + /** + * Direct consumer for the whole batch. When set, all drained data goes to this + * handler without class-based grouping. Takes priority over handler map. + */ + private HandlerConsumer consumer; + + private QueueErrorHandler errorHandler; + + @Builder.Default + private long minIdleMs = 1; + + @Builder.Default + private long maxIdleMs = 50; + + void validate() { + final boolean hasDedicated = threads != null; + final boolean hasShared = sharedSchedulerName != null; + if (hasDedicated == hasShared) { + throw new IllegalArgumentException( + "Exactly one of threads or sharedScheduler must be set. " + + "threads=" + threads + ", sharedSchedulerName=" + sharedSchedulerName); + } + if (hasShared && sharedSchedulerThreads == null) { + throw new IllegalArgumentException( + "sharedSchedulerThreads must be set when sharedSchedulerName is set"); + } + if (bufferSize < 1) { + throw new IllegalArgumentException("bufferSize must be >= 1, got: " + bufferSize); + } + if (minIdleMs < 1) { + throw new IllegalArgumentException("minIdleMs must be >= 1, got: " + minIdleMs); + } + if (maxIdleMs < minIdleMs) { + throw new IllegalArgumentException( + "maxIdleMs must be >= minIdleMs, got maxIdleMs=" + maxIdleMs + " minIdleMs=" + minIdleMs); + } + } + + /** + * Builder customization: convenience method for setting shared scheduler fields together. + */ + public static class BatchQueueConfigBuilder { + public BatchQueueConfigBuilder sharedScheduler(final String name, final ThreadPolicy threads) { + this.sharedSchedulerName = name; + this.sharedSchedulerThreads = threads; + return this; + } + } +} diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManager.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManager.java new file mode 100644 index 000000000000..061daf6deeb8 --- /dev/null +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManager.java @@ -0,0 +1,179 @@ +/* + * 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.skywalking.oap.server.library.batchqueue; + +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.Executors; +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.atomic.AtomicInteger; +import lombok.extern.slf4j.Slf4j; + +/** + * Global registry for batch queues and shared schedulers. + * Thread-safe. Queues are created by name and shared across modules. + * + *

Shared schedulers are created lazily on first queue reference — no separate + * setup step needed. This eliminates startup ordering dependencies. + * + *

Shared schedulers are reference-counted: each queue that uses a shared scheduler + * increments the count on creation and decrements on shutdown. When the count reaches + * zero, the scheduler is shut down automatically. + * + *

Internal maps: + *

+ * QUEUES:                     queueName     -> BatchQueue instance
+ * SHARED_SCHEDULERS:          schedulerName -> ScheduledExecutorService
+ * SHARED_SCHEDULER_POLICIES:  schedulerName -> ThreadPolicy (first-wins)
+ * SHARED_SCHEDULER_REF_COUNTS: schedulerName -> AtomicInteger (reference count)
+ * 
+ */ +@Slf4j +public class BatchQueueManager { + /** + * queueName -> BatchQueue instance. Each queue has a unique name. + */ + private static final ConcurrentHashMap> QUEUES = new ConcurrentHashMap<>(); + /** + * schedulerName -> ScheduledExecutorService. Multiple queues can share one scheduler by + * referencing the same scheduler name in their config. + */ + private static final ConcurrentHashMap SHARED_SCHEDULERS = + new ConcurrentHashMap<>(); + /** + * schedulerName -> ThreadPolicy. Tracks the first-wins policy for each shared scheduler + * to detect mismatched configs. + */ + private static final ConcurrentHashMap SHARED_SCHEDULER_POLICIES = + new ConcurrentHashMap<>(); + /** + * schedulerName -> reference count. Incremented when a queue acquires the scheduler, + * decremented when a queue releases it. Scheduler is shut down when count reaches 0. + */ + private static final ConcurrentHashMap SHARED_SCHEDULER_REF_COUNTS = + new ConcurrentHashMap<>(); + + /** + * Get or create a shared scheduler and increment its reference count. + * Called internally by BatchQueue constructor. + * First call creates the scheduler; subsequent calls reuse it. + * If ThreadPolicy differs from the first creator, logs a warning (first one wins). + */ + static ScheduledExecutorService getOrCreateSharedScheduler(final String name, + final ThreadPolicy threads) { + SHARED_SCHEDULER_POLICIES.compute(name, (k, existing) -> { + if (existing != null) { + if (!existing.toString().equals(threads.toString())) { + log.warn("Shared scheduler [{}]: ThreadPolicy mismatch. " + + "Existing={}, requested={}. Using existing.", + name, existing, threads); + } + return existing; + } + return threads; + }); + + SHARED_SCHEDULER_REF_COUNTS.computeIfAbsent(name, k -> new AtomicInteger(0)).incrementAndGet(); + + return SHARED_SCHEDULERS.computeIfAbsent(name, k -> { + final int threadCount = threads.resolve(); + log.info("Creating shared scheduler [{}] with {} threads ({})", + name, threadCount, threads); + return Executors.newScheduledThreadPool(threadCount, r -> { + final Thread t = new Thread(r); + t.setName("SharedScheduler-" + name + "-" + t.getId()); + t.setDaemon(true); + return t; + }); + }); + } + + /** + * Decrement the reference count for a shared scheduler. + * When the count reaches zero, the scheduler is shut down and removed. + */ + static void releaseSharedScheduler(final String name) { + final AtomicInteger refCount = SHARED_SCHEDULER_REF_COUNTS.get(name); + if (refCount == null) { + return; + } + if (refCount.decrementAndGet() <= 0) { + SHARED_SCHEDULER_REF_COUNTS.remove(name); + SHARED_SCHEDULER_POLICIES.remove(name); + final ScheduledExecutorService scheduler = SHARED_SCHEDULERS.remove(name); + if (scheduler != null) { + log.info("Shutting down shared scheduler [{}] (ref count reached 0)", name); + scheduler.shutdown(); + } + } + } + + @SuppressWarnings("unchecked") + public static BatchQueue create(final String name, final BatchQueueConfig config) { + config.validate(); + return (BatchQueue) QUEUES.computeIfAbsent(name, k -> new BatchQueue<>(name, config)); + } + + @SuppressWarnings("unchecked") + public static BatchQueue get(final String name) { + return (BatchQueue) QUEUES.get(name); + } + + public static void shutdown(final String name) { + final BatchQueue queue = QUEUES.remove(name); + if (queue != null) { + queue.shutdown(); + } + } + + /** + * Shutdown all queues and all shared schedulers. Called during OAP server shutdown. + */ + public static void shutdownAll() { + final List> allQueues = new ArrayList<>(QUEUES.values()); + QUEUES.clear(); + + for (final BatchQueue queue : allQueues) { + try { + queue.shutdown(); + } catch (final Throwable t) { + log.error("Error shutting down queue: {}", queue.getName(), t); + } + } + + for (final ScheduledExecutorService scheduler : SHARED_SCHEDULERS.values()) { + try { + scheduler.shutdown(); + } catch (final Throwable t) { + log.error("Error shutting down shared scheduler", t); + } + } + SHARED_SCHEDULERS.clear(); + SHARED_SCHEDULER_POLICIES.clear(); + SHARED_SCHEDULER_REF_COUNTS.clear(); + } + + /** + * Reset for testing purposes only. + */ + static void reset() { + shutdownAll(); + } +} diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BufferStrategy.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BufferStrategy.java new file mode 100644 index 000000000000..66a8e5df2a90 --- /dev/null +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BufferStrategy.java @@ -0,0 +1,31 @@ +/* + * 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.skywalking.oap.server.library.batchqueue; + +public enum BufferStrategy { + /** + * Producer blocks when buffer is full (queue.put). + */ + BLOCKING, + + /** + * Producer gets false when buffer is full (queue.offer). Data is dropped. + */ + IF_POSSIBLE +} diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/HandlerConsumer.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/HandlerConsumer.java new file mode 100644 index 000000000000..0b9e583654b3 --- /dev/null +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/HandlerConsumer.java @@ -0,0 +1,39 @@ +/* + * 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.skywalking.oap.server.library.batchqueue; + +import java.util.List; + +/** + * Handler for processing a batch of data for a specific type. + * Each metric type (or I/O queue user) provides its own handler instance. + */ +public interface HandlerConsumer { + /** + * Process a batch of data belonging to this handler's type. + */ + void consume(List data); + + /** + * Called when there is nothing to consume. Can be used as a timer trigger + * (e.g. flush L1 aggregation cache periodically). + */ + default void onIdle() { + } +} diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/PartitionPolicy.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/PartitionPolicy.java new file mode 100644 index 000000000000..6f3428a6f371 --- /dev/null +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/PartitionPolicy.java @@ -0,0 +1,159 @@ +/* + * 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.skywalking.oap.server.library.batchqueue; + +/** + * Determines the number of partitions for a BatchQueue. + * + *
    + *
  • {@link #fixed(int)}: exactly N partitions, regardless of thread count.
  • + *
  • {@link #threadMultiply(int)}: N * resolved thread count.
  • + *
  • {@link #adaptive()}: recommended for metrics aggregation. The partition + * count grows as handlers are registered via {@link BatchQueue#addHandler}. + * Uses {@code threadCount * multiplier} (default 25) as a threshold. + * Below threshold, 1:1 mapping (one partition per handler). Above threshold, + * excess handlers share partitions at 1:2 ratio.
  • + *
+ * + *

All policies are resolved via {@link #resolve(int, int)}. For non-adaptive + * policies the handlerCount parameter is ignored. At queue creation time, if the + * resolved partition count is less than the thread count, the thread count is + * reduced to match and a warning is logged. + */ +public class PartitionPolicy { + private static final int DEFAULT_ADAPTIVE_MULTIPLIER = 25; + + private final int fixedCount; + private final int multiplier; + private final boolean adaptive; + + private PartitionPolicy(final int fixedCount, final int multiplier, + final boolean adaptive) { + this.fixedCount = fixedCount; + this.multiplier = multiplier; + this.adaptive = adaptive; + } + + /** + * Fixed number of partitions. + * + * @throws IllegalArgumentException if count < 1 + */ + public static PartitionPolicy fixed(final int count) { + if (count < 1) { + throw new IllegalArgumentException("Partition count must be >= 1, got: " + count); + } + return new PartitionPolicy(count, 0, false); + } + + /** + * Partitions = multiplier * resolved thread count. + * + * @throws IllegalArgumentException if multiplier < 1 + */ + public static PartitionPolicy threadMultiply(final int multiplier) { + if (multiplier < 1) { + throw new IllegalArgumentException("Partition multiplier must be >= 1, got: " + multiplier); + } + return new PartitionPolicy(0, multiplier, false); + } + + /** + * Adaptive partition count with default threshold multiplier (25). + * + *

The partition count grows as handlers are registered via + * {@link BatchQueue#addHandler}: + *

    + *
  • Threshold = {@code threadCount * 25}
  • + *
  • handlerCount <= threshold: one partition per handler (1:1)
  • + *
  • handlerCount > threshold: {@code threshold + (handlerCount - threshold) / 2}
  • + *
  • handlerCount == 0: returns {@code threadCount} as initial count
  • + *
+ * + *

Examples with 8 threads (threshold = 200): + *

+     *     0 handlers →   8 partitions  (initial = threadCount)
+     *   100 handlers → 100 partitions  (1:1, below threshold)
+     *   200 handlers → 200 partitions  (1:1, at threshold)
+     *   500 handlers → 350 partitions  (200 + 300/2)
+     *  1000 handlers → 600 partitions  (200 + 800/2)
+     *  2000 handlers → 1100 partitions (200 + 1800/2)
+     * 
+ */ + public static PartitionPolicy adaptive() { + return new PartitionPolicy(0, DEFAULT_ADAPTIVE_MULTIPLIER, true); + } + + /** + * Adaptive partition count with custom threshold multiplier. + * + *

Threshold = {@code threadCount * multiplier}. Below threshold, one + * partition per handler (1:1). Above threshold, excess handlers share + * at 1:2 ratio: {@code threshold + (handlerCount - threshold) / 2}. + * + * @param multiplier threshold per thread (default 25) + * @throws IllegalArgumentException if multiplier < 1 + */ + public static PartitionPolicy adaptive(final int multiplier) { + if (multiplier < 1) { + throw new IllegalArgumentException( + "adaptive multiplier must be >= 1, got: " + multiplier); + } + return new PartitionPolicy(0, multiplier, true); + } + + /** + * Resolve the actual partition count. + *

    + *
  • fixed: returns the pre-set count (both parameters ignored).
  • + *
  • threadMultiply: returns multiplier * resolvedThreadCount (handlerCount ignored).
  • + *
  • adaptive: when handlerCount is 0, returns resolvedThreadCount as a sensible + * initial count. Otherwise, threshold = threadCount * multiplier; if handlerCount + * <= threshold, returns handlerCount (1:1). If above, returns + * threshold + (handlerCount - threshold) / 2.
  • + *
+ */ + public int resolve(final int resolvedThreadCount, final int handlerCount) { + if (fixedCount > 0) { + return fixedCount; + } + if (adaptive) { + if (handlerCount == 0) { + return Math.max(1, resolvedThreadCount); + } + final int threshold = Math.max(1, multiplier * resolvedThreadCount); + if (handlerCount <= threshold) { + return handlerCount; + } + return threshold + (handlerCount - threshold) / 2; + } + return Math.max(1, multiplier * resolvedThreadCount); + } + + @Override + public String toString() { + if (fixedCount > 0) { + return "fixed(" + fixedCount + ")"; + } + if (adaptive) { + return "adaptive(multiplier=" + multiplier + ")"; + } + return "threadMultiply(" + multiplier + ")"; + } +} diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/PartitionSelector.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/PartitionSelector.java new file mode 100644 index 000000000000..d30d74a05e1b --- /dev/null +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/PartitionSelector.java @@ -0,0 +1,54 @@ +/* + * 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.skywalking.oap.server.library.batchqueue; + +/** + * Strategy for selecting which partition a produced item should be placed into. + * + *

The default implementation ({@link #typeHash()}) uses the item's class hash, + * ensuring all items of the same type land on the same partition. This eliminates + * the need for HashMap grouping during dispatch and avoids contention on shared + * counters with many concurrent producers. + * + *

Only used when the queue has multiple partitions. Single-partition queues + * bypass the selector entirely. + * + * @param the queue element type + */ +@FunctionalInterface +public interface PartitionSelector { + + /** + * Select a partition index for the given data item. + * + * @param data the item being produced + * @param partitionCount total number of partitions (always > 1) + * @return a partition index in [0, partitionCount) + */ + int select(T data, int partitionCount); + + /** + * Default selector: routes by {@code data.getClass().hashCode()}. + * Same type always hits the same partition, so each consumer thread + * drains pre-grouped batches — dispatch grouping is effectively a no-op. + */ + static PartitionSelector typeHash() { + return (data, count) -> (data.getClass().hashCode() & 0x7FFFFFFF) % count; + } +} diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/QueueErrorHandler.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/QueueErrorHandler.java new file mode 100644 index 000000000000..c122d4c76196 --- /dev/null +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/QueueErrorHandler.java @@ -0,0 +1,26 @@ +/* + * 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.skywalking.oap.server.library.batchqueue; + +import java.util.List; + +@FunctionalInterface +public interface QueueErrorHandler { + void onError(List data, Throwable t); +} diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicy.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicy.java new file mode 100644 index 000000000000..6b6585debe56 --- /dev/null +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicy.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.skywalking.oap.server.library.batchqueue; + +/** + * Determines the number of threads for a BatchQueue's dedicated scheduler + * or for a shared scheduler. + * + * Two modes: + * - fixed(N): exactly N threads, regardless of hardware. + * - cpuCores(multiplier): multiplier * Runtime.availableProcessors(), rounded. + * + * Resolved value is always >= 1 — every pool must have at least one thread. + * fixed() requires count >= 1 at construction. cpuCores() applies max(1, ...) at resolution. + */ +public class ThreadPolicy { + private final int fixedCount; + private final double cpuMultiplier; + + private ThreadPolicy(final int fixedCount, final double cpuMultiplier) { + this.fixedCount = fixedCount; + this.cpuMultiplier = cpuMultiplier; + } + + /** + * Fixed number of threads. Count must be >= 1. + * + * @throws IllegalArgumentException if count < 1 + */ + public static ThreadPolicy fixed(final int count) { + if (count < 1) { + throw new IllegalArgumentException("Thread count must be >= 1, got: " + count); + } + return new ThreadPolicy(count, 0); + } + + /** + * Threads = multiplier * available CPU cores, rounded, min 1. + * Multiplier must be > 0. + * + * @throws IllegalArgumentException if multiplier <= 0 + */ + public static ThreadPolicy cpuCores(final double multiplier) { + if (multiplier <= 0) { + throw new IllegalArgumentException("CPU multiplier must be > 0, got: " + multiplier); + } + return new ThreadPolicy(0, multiplier); + } + + /** + * Resolve the actual thread count. Always returns >= 1. + */ + public int resolve() { + if (fixedCount > 0) { + return fixedCount; + } + return Math.max(1, (int) Math.round(cpuMultiplier * Runtime.getRuntime().availableProcessors())); + } + + @Override + public String toString() { + if (fixedCount > 0) { + return "fixed(" + fixedCount + ")"; + } + return "cpuCores(" + cpuMultiplier + ")"; + } +} diff --git a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueBenchmark.java b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueBenchmark.java new file mode 100644 index 000000000000..115a03fcadeb --- /dev/null +++ b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueBenchmark.java @@ -0,0 +1,449 @@ +/* + * 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.skywalking.oap.server.library.batchqueue; + +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import lombok.extern.slf4j.Slf4j; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; + +/** + * Throughput benchmark for BatchQueue with handler-map dispatch. + * + *

Simulates production OAP metrics aggregation: 500-2000 distinct metric types, + * 32 gRPC producer threads, 8 consumer threads. Tests various partition strategies + * from fixed small counts to 1:1 partition-per-type binding. + * + *

Run with: mvn test -pl oap-server/server-library/library-batch-queue + * -Dtest=BatchQueueBenchmark -DfailIfNoTests=false + * + *

Reference results (Apple M3 Max, 128 GB RAM, macOS 26.2, JDK 17)

+ * + *

Fixed partitions (typeHash selector): + *

+ * Partitions  BufSize   500 types (IF/BLK)      1000 types (IF/BLK)     2000 types (IF/BLK)
+ * ---------- -------- ----------------------- ----------------------- -----------------------
+ * fixed(16)   50,000   ~19.0M / ~12.3M         ~17.9M / ~17.1M         ~15.7M / ~15.2M
+ * fixed(64)  500,000   ~17.0M / ~18.0M         ~17.7M / ~18.4M         ~17.7M / ~18.0M
+ * fixed(128) 500,000   ~24.1M / ~23.2M         ~24.9M / ~26.1M         ~25.3M / ~24.8M
+ * 
+ * + *

Type-aware partitions (typeId selector, 50K buffer each): + *

+ * Ratio       Partitions   500 types (IF/BLK)      1000 types (IF/BLK)     2000 types (IF/BLK)
+ * ---------- ------------ ----------------------- ----------------------- -----------------------
+ * 1:4         types/4      ~32.1M / ~29.9M         ~40.0M / ~40.5M         ~47.1M / ~50.7M
+ * 1:2         types/2      ~38.2M / ~35.9M         ~49.6M / ~35.9M         ~52.6M / ~58.6M
+ * adaptive    350/600/1100 ~45.7M / ~46.3M         ~50.5M / ~54.1M         ~64.0M / ~60.3M
+ * 1:1         types        ~51.3M / ~54.4M         ~61.2M / ~62.5M         ~75.7M / ~67.4M
+ * 
+ * + *

DataCarrier baseline (N independent carriers, raw Long values, pool(8)): + *

+ *   500 types:  ~33.4M IF_POSSIBLE / ~32.5M BLOCKING
+ *  1000 types:  ~37.6M IF_POSSIBLE / ~36.0M BLOCKING
+ *  2000 types:  ~38.0M IF_POSSIBLE / ~42.1M BLOCKING
+ * 
+ * + *

BatchQueue vs DataCarrier (IF_POSSIBLE): + *

+ *              500 types   1000 types   2000 types
+ * 1:4           -4%         +6%          +24%
+ * 1:2          +14%        +32%          +38%
+ * adaptive     +37%        +34%          +68%
+ * 1:1          +53%        +63%          +99%
+ * 
+ * + *

All runs: 32 producers, fixed(8) threads, minIdleMs=1, maxIdleMs=50, 0% drop rate. + * 2000 metric types generated at runtime via bytecode (see {@link BenchmarkMetricTypes}). + * Adaptive policy: {@link PartitionPolicy#adaptive()} with threshold = threadCount * 25. + * Below threshold: 1:1 (one partition per type). Above: excess at 1:2 ratio. + */ +@Slf4j +@SuppressWarnings("all") +public class BatchQueueBenchmark { + + private static final int WARMUP_SECONDS = 2; + private static final int MEASURE_SECONDS = 5; + private static final int PRODUCER_THREADS = 32; + private static final ThreadPolicy THREADS = ThreadPolicy.fixed(8); + + @AfterEach + public void cleanup() { + BatchQueueManager.reset(); + } + + // ---- Fixed partitions, typeHash selector ---- + // fixed(16) 50K: ~19.0M/~12.3M ~17.9M/~17.1M ~15.7M/~15.2M + // fixed(64) 500K: ~17.0M/~18.0M ~17.7M/~18.4M ~17.7M/~18.0M + // fixed(128)500K: ~24.1M/~23.2M ~24.9M/~26.1M ~25.3M/~24.8M + + @Test + public void benchmark500Types() throws Exception { + runBenchmark("500-types", 500, 16, 50_000, BufferStrategy.IF_POSSIBLE); + } + + @Test + public void benchmark1000Types() throws Exception { + runBenchmark("1000-types", 1000, 16, 50_000, BufferStrategy.IF_POSSIBLE); + } + + @Test + public void benchmark2000Types() throws Exception { + runBenchmark("2000-types", 2000, 16, 50_000, BufferStrategy.IF_POSSIBLE); + } + + @Test + public void benchmark500TypesBlocking() throws Exception { + runBenchmark("500-types-blocking", 500, 16, 50_000, BufferStrategy.BLOCKING); + } + + @Test + public void benchmark1000TypesBlocking() throws Exception { + runBenchmark("1000-types-blocking", 1000, 16, 50_000, BufferStrategy.BLOCKING); + } + + @Test + public void benchmark2000TypesBlocking() throws Exception { + runBenchmark("2000-types-blocking", 2000, 16, 50_000, BufferStrategy.BLOCKING); + } + + @Test + public void benchmark500Types_64p() throws Exception { + runBenchmark("500-types-64p", 500, 64, 500_000, BufferStrategy.IF_POSSIBLE); + } + + @Test + public void benchmark1000Types_64p() throws Exception { + runBenchmark("1000-types-64p", 1000, 64, 500_000, BufferStrategy.IF_POSSIBLE); + } + + @Test + public void benchmark2000Types_64p() throws Exception { + runBenchmark("2000-types-64p", 2000, 64, 500_000, BufferStrategy.IF_POSSIBLE); + } + + @Test + public void benchmark500TypesBlocking_64p() throws Exception { + runBenchmark("500-types-blocking-64p", 500, 64, 500_000, BufferStrategy.BLOCKING); + } + + @Test + public void benchmark1000TypesBlocking_64p() throws Exception { + runBenchmark("1000-types-blocking-64p", 1000, 64, 500_000, BufferStrategy.BLOCKING); + } + + @Test + public void benchmark2000TypesBlocking_64p() throws Exception { + runBenchmark("2000-types-blocking-64p", 2000, 64, 500_000, BufferStrategy.BLOCKING); + } + + @Test + public void benchmark500Types_128p() throws Exception { + runBenchmark("500-types-128p", 500, 128, 500_000, BufferStrategy.IF_POSSIBLE); + } + + @Test + public void benchmark1000Types_128p() throws Exception { + runBenchmark("1000-types-128p", 1000, 128, 500_000, BufferStrategy.IF_POSSIBLE); + } + + @Test + public void benchmark2000Types_128p() throws Exception { + runBenchmark("2000-types-128p", 2000, 128, 500_000, BufferStrategy.IF_POSSIBLE); + } + + @Test + public void benchmark500TypesBlocking_128p() throws Exception { + runBenchmark("500-types-blocking-128p", 500, 128, 500_000, BufferStrategy.BLOCKING); + } + + @Test + public void benchmark1000TypesBlocking_128p() throws Exception { + runBenchmark("1000-types-blocking-128p", 1000, 128, 500_000, BufferStrategy.BLOCKING); + } + + @Test + public void benchmark2000TypesBlocking_128p() throws Exception { + runBenchmark("2000-types-blocking-128p", 2000, 128, 500_000, BufferStrategy.BLOCKING); + } + + // ---- Type-aware partitions, typeId selector, 50K buffer each ---- + // 1:4 types/4: ~32.1M/~29.9M ~40.0M/~40.5M ~47.1M/~50.7M + // 1:2 types/2: ~38.2M/~35.9M ~49.6M/~35.9M ~52.6M/~58.6M + // adaptive 350/600/1100: ~45.7M/~46.3M ~50.5M/~54.1M ~64.0M/~60.3M + // 1:1 types: ~51.3M/~54.4M ~61.2M/~62.5M ~75.7M/~67.4M + + private static final PartitionSelector TYPE_ID_SELECTOR = + (data, count) -> data.typeId % count; + + @Test + public void benchmark500Types_quarter() throws Exception { + runBenchmark("500-types-quarter", 500, 125, 50_000, + BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, true); + } + + @Test + public void benchmark1000Types_quarter() throws Exception { + runBenchmark("1000-types-quarter", 1000, 250, 50_000, + BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, true); + } + + @Test + public void benchmark2000Types_quarter() throws Exception { + runBenchmark("2000-types-quarter", 2000, 500, 50_000, + BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, true); + } + + @Test + public void benchmark500TypesBlocking_quarter() throws Exception { + runBenchmark("500-types-blocking-quarter", 500, 125, 50_000, + BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, true); + } + + @Test + public void benchmark1000TypesBlocking_quarter() throws Exception { + runBenchmark("1000-types-blocking-quarter", 1000, 250, 50_000, + BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, true); + } + + @Test + public void benchmark2000TypesBlocking_quarter() throws Exception { + runBenchmark("2000-types-blocking-quarter", 2000, 500, 50_000, + BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, true); + } + + @Test + public void benchmark500Types_half() throws Exception { + runBenchmark("500-types-half", 500, 250, 50_000, + BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, true); + } + + @Test + public void benchmark1000Types_half() throws Exception { + runBenchmark("1000-types-half", 1000, 500, 50_000, + BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, true); + } + + @Test + public void benchmark2000Types_half() throws Exception { + runBenchmark("2000-types-half", 2000, 1000, 50_000, + BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, true); + } + + @Test + public void benchmark500TypesBlocking_half() throws Exception { + runBenchmark("500-types-blocking-half", 500, 250, 50_000, + BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, true); + } + + @Test + public void benchmark1000TypesBlocking_half() throws Exception { + runBenchmark("1000-types-blocking-half", 1000, 500, 50_000, + BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, true); + } + + @Test + public void benchmark2000TypesBlocking_half() throws Exception { + runBenchmark("2000-types-blocking-half", 2000, 1000, 50_000, + BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, true); + } + + @Test + public void benchmark500Types_adaptive() throws Exception { + runAdaptiveBenchmark("500-types-adaptive", 500, BufferStrategy.IF_POSSIBLE); + } + + @Test + public void benchmark1000Types_adaptive() throws Exception { + runAdaptiveBenchmark("1000-types-adaptive", 1000, BufferStrategy.IF_POSSIBLE); + } + + @Test + public void benchmark2000Types_adaptive() throws Exception { + runAdaptiveBenchmark("2000-types-adaptive", 2000, BufferStrategy.IF_POSSIBLE); + } + + @Test + public void benchmark500TypesBlocking_adaptive() throws Exception { + runAdaptiveBenchmark("500-types-blocking-adaptive", 500, BufferStrategy.BLOCKING); + } + + @Test + public void benchmark1000TypesBlocking_adaptive() throws Exception { + runAdaptiveBenchmark("1000-types-blocking-adaptive", 1000, BufferStrategy.BLOCKING); + } + + @Test + public void benchmark2000TypesBlocking_adaptive() throws Exception { + runAdaptiveBenchmark("2000-types-blocking-adaptive", 2000, BufferStrategy.BLOCKING); + } + + @Test + public void benchmark500Types_1to1() throws Exception { + runBenchmark("500-types-1to1", 500, 500, 50_000, + BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, true); + } + + @Test + public void benchmark1000Types_1to1() throws Exception { + runBenchmark("1000-types-1to1", 1000, 1000, 50_000, + BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, true); + } + + @Test + public void benchmark2000Types_1to1() throws Exception { + runBenchmark("2000-types-1to1", 2000, 2000, 50_000, + BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, true); + } + + @Test + public void benchmark500TypesBlocking_1to1() throws Exception { + runBenchmark("500-types-blocking-1to1", 500, 500, 50_000, + BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, true); + } + + @Test + public void benchmark1000TypesBlocking_1to1() throws Exception { + runBenchmark("1000-types-blocking-1to1", 1000, 1000, 50_000, + BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, true); + } + + @Test + public void benchmark2000TypesBlocking_1to1() throws Exception { + runBenchmark("2000-types-blocking-1to1", 2000, 2000, 50_000, + BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, true); + } + + private void runAdaptiveBenchmark(final String label, final int typeCount, + final BufferStrategy strategy) throws Exception { + // adaptive(): threshold = threadCount * 25 = 200 + // 500 types → 350p (200 + 300/2) + // 1000 types → 600p (200 + 800/2) + // 2000 types → 1100p (200 + 1800/2) + final int partitionCount = PartitionPolicy.adaptive() + .resolve(THREADS.resolve(), typeCount); + runBenchmark(label, typeCount, partitionCount, 50_000, + strategy, TYPE_ID_SELECTOR, true); + } + + private void runBenchmark(final String label, final int typeCount, + final int partitionCount, final int bufferSize, + final BufferStrategy strategy) throws Exception { + runBenchmark(label, typeCount, partitionCount, bufferSize, strategy, null, true); + } + + private void runBenchmark(final String label, final int typeCount, + final int partitionCount, final int bufferSize, + final BufferStrategy strategy, + final PartitionSelector selector, + final boolean ignored) throws Exception { + final AtomicLong consumed = new AtomicLong(0); + final PartitionPolicy partitions = PartitionPolicy.fixed(partitionCount); + + final BatchQueueConfig.BatchQueueConfigBuilder configBuilder = + BatchQueueConfig.builder() + .threads(THREADS) + .partitions(partitions) + .bufferSize(bufferSize) + .strategy(strategy) + .minIdleMs(1) + .maxIdleMs(50); + if (selector != null) { + configBuilder.partitionSelector(selector); + } + + final BatchQueue queue = BatchQueueManager.create( + "bench-" + label, configBuilder.build()); + + for (int t = 0; t < typeCount; t++) { + queue.addHandler(BenchmarkMetricTypes.CLASSES[t], + (HandlerConsumer) data -> + consumed.addAndGet(data.size())); + } + + // Warmup + final long warmupEnd = System.currentTimeMillis() + WARMUP_SECONDS * 1000L; + runProducers(queue, typeCount, PRODUCER_THREADS, warmupEnd); + Thread.sleep(200); + consumed.set(0); + + // Measure + final long measureStart = System.currentTimeMillis(); + final long measureEnd = measureStart + MEASURE_SECONDS * 1000L; + final long produced = runProducers(queue, typeCount, PRODUCER_THREADS, measureEnd); + final long measureDuration = System.currentTimeMillis() - measureStart; + + Thread.sleep(500); + final long totalConsumed = consumed.get(); + + log.info("\n=== BatchQueue Benchmark: {} ===\n" + + " Types: {}\n" + + " Threads: {}\n" + + " Partitions: {}\n" + + " BufferSize: {}\n" + + " Strategy: {}\n" + + " Producers: {}\n" + + " Duration: {} ms\n" + + " Produced: {}\n" + + " Consumed: {}\n" + + " Consume rate: {} items/sec\n" + + " Drop rate: {}%\n", + label, typeCount, THREADS, partitions, bufferSize, strategy, PRODUCER_THREADS, + measureDuration, + String.format("%,d", produced), String.format("%,d", totalConsumed), + String.format("%,.0f", totalConsumed * 1000.0 / measureDuration), + String.format("%.2f", produced > 0 + ? (produced - totalConsumed) * 100.0 / produced : 0)); + } + + private long runProducers(final BatchQueue queue, + final int typeCount, final int producerCount, + final long endTimeMs) throws InterruptedException { + final AtomicLong totalProduced = new AtomicLong(0); + final CountDownLatch done = new CountDownLatch(producerCount); + + for (int p = 0; p < producerCount; p++) { + final int producerIndex = p; + final Thread thread = new Thread(() -> { + long count = 0; + int typeIndex = producerIndex; + while (System.currentTimeMillis() < endTimeMs) { + for (int batch = 0; batch < 100; batch++) { + final int type = typeIndex % typeCount; + if (queue.produce(BenchmarkMetricTypes.FACTORIES[type].create(count))) { + count++; + } + typeIndex++; + } + } + totalProduced.addAndGet(count); + done.countDown(); + }); + thread.setName("Producer-" + producerIndex); + thread.setDaemon(true); + thread.start(); + } + + done.await(MEASURE_SECONDS + 10, TimeUnit.SECONDS); + return totalProduced.get(); + } +} diff --git a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueConfigTest.java b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueConfigTest.java new file mode 100644 index 000000000000..f74add79f9d5 --- /dev/null +++ b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueConfigTest.java @@ -0,0 +1,110 @@ +/* + * 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.skywalking.oap.server.library.batchqueue; + +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class BatchQueueConfigTest { + + @Test + public void testDedicatedThreadConfig() { + final BatchQueueConfig config = BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(2)) + .build(); + config.validate(); + assertNotNull(config.getThreads()); + } + + @Test + public void testSharedSchedulerConfig() { + final BatchQueueConfig config = BatchQueueConfig.builder() + .sharedScheduler("IO_POOL", ThreadPolicy.fixed(4)) + .build(); + config.validate(); + assertEquals("IO_POOL", config.getSharedSchedulerName()); + assertNotNull(config.getSharedSchedulerThreads()); + } + + @Test + public void testRejectsBothThreadsAndShared() { + final BatchQueueConfig config = BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(2)) + .sharedScheduler("IO_POOL", ThreadPolicy.fixed(4)) + .build(); + assertThrows(IllegalArgumentException.class, config::validate); + } + + @Test + public void testRejectsNeitherThreadsNorShared() { + final BatchQueueConfig config = BatchQueueConfig.builder() + .build(); + assertThrows(IllegalArgumentException.class, config::validate); + } + + @Test + public void testRejectsSharedWithoutThreadPolicy() { + final BatchQueueConfig config = BatchQueueConfig.builder() + .sharedSchedulerName("IO_POOL") + .build(); + assertThrows(IllegalArgumentException.class, config::validate); + } + + @Test + public void testRejectsZeroBufferSize() { + final BatchQueueConfig config = BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .bufferSize(0) + .build(); + assertThrows(IllegalArgumentException.class, config::validate); + } + + @Test + public void testRejectsZeroMinIdleMs() { + final BatchQueueConfig config = BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .minIdleMs(0) + .build(); + assertThrows(IllegalArgumentException.class, config::validate); + } + + @Test + public void testRejectsMaxIdleLessThanMinIdle() { + final BatchQueueConfig config = BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .minIdleMs(100) + .maxIdleMs(50) + .build(); + assertThrows(IllegalArgumentException.class, config::validate); + } + + @Test + public void testDefaults() { + final BatchQueueConfig config = BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .build(); + assertEquals(10_000, config.getBufferSize()); + assertEquals(BufferStrategy.BLOCKING, config.getStrategy()); + assertEquals(1, config.getMinIdleMs()); + assertEquals(50, config.getMaxIdleMs()); + } +} diff --git a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManagerTest.java b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManagerTest.java new file mode 100644 index 000000000000..17b6226a37ee --- /dev/null +++ b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManagerTest.java @@ -0,0 +1,162 @@ +/* + * 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.skywalking.oap.server.library.batchqueue; + +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; + +import java.util.concurrent.ScheduledExecutorService; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class BatchQueueManagerTest { + + @AfterEach + public void cleanup() { + BatchQueueManager.reset(); + } + + @Test + public void testCreateAndGet() { + final BatchQueue queue = BatchQueueManager.create("test-queue", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .consumer(data -> { }) + .bufferSize(100) + .build()); + + assertNotNull(queue); + assertEquals("test-queue", queue.getName()); + + final BatchQueue retrieved = BatchQueueManager.get("test-queue"); + assertSame(queue, retrieved); + } + + @Test + public void testCreateReturnsExistingIfPresent() { + final BatchQueueConfig config = BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .consumer(data -> { }) + .bufferSize(100) + .build(); + + final BatchQueue first = BatchQueueManager.create("absent-test", config); + final BatchQueue second = BatchQueueManager.create("absent-test", config); + assertSame(first, second); + } + + @Test + public void testGetNonExistentReturnsNull() { + assertNull(BatchQueueManager.get("nonexistent")); + } + + @Test + public void testShutdownRemovesQueue() { + BatchQueueManager.create("shutdown-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .consumer(data -> { }) + .bufferSize(100) + .build()); + + assertNotNull(BatchQueueManager.get("shutdown-test")); + BatchQueueManager.shutdown("shutdown-test"); + assertNull(BatchQueueManager.get("shutdown-test")); + } + + @Test + public void testShutdownAllClearsEverything() { + BatchQueueManager.create("q1", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .consumer(data -> { }) + .bufferSize(100) + .build()); + + BatchQueueManager.create("q2", + BatchQueueConfig.builder() + .sharedScheduler("SHARED", ThreadPolicy.fixed(2)) + .consumer(data -> { }) + .bufferSize(100) + .build()); + + BatchQueueManager.shutdownAll(); + + assertNull(BatchQueueManager.get("q1")); + assertNull(BatchQueueManager.get("q2")); + } + + @Test + public void testSharedSchedulerCreatedLazily() { + // First queue referencing shared scheduler creates it + final BatchQueue q1 = BatchQueueManager.create("lazy1", + BatchQueueConfig.builder() + .sharedScheduler("LAZY_POOL", ThreadPolicy.fixed(2)) + .consumer(data -> { }) + .bufferSize(100) + .build()); + + // Second queue uses the same pool + final BatchQueue q2 = BatchQueueManager.create("lazy2", + BatchQueueConfig.builder() + .sharedScheduler("LAZY_POOL", ThreadPolicy.fixed(2)) + .consumer(data -> { }) + .bufferSize(100) + .build()); + + assertNotNull(q1); + assertNotNull(q2); + // Both should be running on shared scheduler (not dedicated) + assertNotNull(BatchQueueManager.get("lazy1")); + assertNotNull(BatchQueueManager.get("lazy2")); + } + + @Test + public void testSharedSchedulerRefCounting() { + final BatchQueueConfig config = BatchQueueConfig.builder() + .sharedScheduler("REF_POOL", ThreadPolicy.fixed(2)) + .consumer(data -> { }) + .bufferSize(100) + .build(); + + BatchQueueManager.create("ref1", config); + BatchQueueManager.create("ref2", config); + + // Capture the scheduler before any shutdown + final ScheduledExecutorService scheduler = + BatchQueueManager.getOrCreateSharedScheduler("REF_POOL", ThreadPolicy.fixed(2)); + // Release the extra ref from getOrCreateSharedScheduler call above + BatchQueueManager.releaseSharedScheduler("REF_POOL"); + + assertFalse(scheduler.isShutdown()); + + // Shutting down first queue should NOT shut down the shared scheduler + BatchQueueManager.shutdown("ref1"); + assertFalse(scheduler.isShutdown()); + + // Shutting down last queue SHOULD shut down the shared scheduler + BatchQueueManager.shutdown("ref2"); + assertTrue(scheduler.isShutdown()); + } +} diff --git a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueTest.java b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueTest.java new file mode 100644 index 000000000000..a63bbdf400bc --- /dev/null +++ b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueTest.java @@ -0,0 +1,534 @@ +/* + * 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.skywalking.oap.server.library.batchqueue; + +import java.util.ArrayList; +import java.util.Collections; +import java.util.List; +import java.util.concurrent.CopyOnWriteArrayList; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; +import org.awaitility.Awaitility; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class BatchQueueTest { + + @AfterEach + public void cleanup() { + BatchQueueManager.reset(); + } + + // --- Direct consumer mode --- + + @Test + public void testDirectConsumerReceivesAllData() { + final List received = new CopyOnWriteArrayList<>(); + final BatchQueue queue = BatchQueueManager.create("direct-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .consumer(data -> received.addAll(data)) + .bufferSize(1000) + .build()); + + for (int i = 0; i < 100; i++) { + queue.produce("item-" + i); + } + + Awaitility.await().atMost(5, TimeUnit.SECONDS) + .until(() -> received.size() == 100); + + assertEquals(100, received.size()); + } + + @Test + public void testDirectConsumerWithMultipleThreads() { + final List received = Collections.synchronizedList(new ArrayList<>()); + final BatchQueue queue = BatchQueueManager.create("multi-thread-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(4)) + .partitions(PartitionPolicy.fixed(4)) + .consumer(data -> received.addAll(data)) + .bufferSize(1000) + .build()); + + for (int i = 0; i < 1000; i++) { + queue.produce("item-" + i); + } + + Awaitility.await().atMost(5, TimeUnit.SECONDS) + .until(() -> received.size() == 1000); + + assertEquals(1000, received.size()); + } + + // --- Handler map dispatch --- + + static class MetricA { + final String name; + + MetricA(final String name) { + this.name = name; + } + } + + static class MetricB { + final String name; + + MetricB(final String name) { + this.name = name; + } + } + + @Test + public void testHandlerMapDispatch() { + final List receivedA = new CopyOnWriteArrayList<>(); + final List receivedB = new CopyOnWriteArrayList<>(); + + final BatchQueue queue = BatchQueueManager.create("handler-map-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .bufferSize(1000) + .build()); + + queue.addHandler(MetricA.class, data -> receivedA.addAll(data)); + queue.addHandler(MetricB.class, data -> receivedB.addAll(data)); + + for (int i = 0; i < 50; i++) { + queue.produce(new MetricA("a-" + i)); + queue.produce(new MetricB("b-" + i)); + } + + Awaitility.await().atMost(5, TimeUnit.SECONDS) + .until(() -> receivedA.size() == 50 && receivedB.size() == 50); + + assertEquals(50, receivedA.size()); + assertEquals(50, receivedB.size()); + } + + // --- Partition assignment --- + + @Test + public void testPartitionCountMatchesPolicy() { + final BatchQueue queue = BatchQueueManager.create("partition-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(2)) + .partitions(PartitionPolicy.fixed(8)) + .consumer(data -> { }) + .bufferSize(100) + .build()); + + assertEquals(8, queue.getPartitionCount()); + assertEquals(2, queue.getTaskCount()); + } + + @Test + public void testThreadMultiplyPartitions() { + final BatchQueue queue = BatchQueueManager.create("multiply-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(3)) + .partitions(PartitionPolicy.threadMultiply(2)) + .consumer(data -> { }) + .bufferSize(100) + .build()); + + assertEquals(6, queue.getPartitionCount()); + assertEquals(3, queue.getTaskCount()); + } + + @Test + public void testPartitionsLessThanThreadsReducesThreads() { + // 2 partitions but 4 threads → should reduce to 2 tasks + final BatchQueue queue = BatchQueueManager.create("reduce-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(4)) + .partitions(PartitionPolicy.fixed(2)) + .consumer(data -> { }) + .bufferSize(100) + .build()); + + assertEquals(2, queue.getPartitionCount()); + assertEquals(2, queue.getTaskCount()); + } + + @Test + public void testEachPartitionAssignedToExactlyOneTask() { + final BatchQueue queue = BatchQueueManager.create("assign-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(3)) + .partitions(PartitionPolicy.fixed(9)) + .consumer(data -> { }) + .bufferSize(100) + .build()); + + final int[][] assignments = queue.getAssignedPartitions(); + assertEquals(3, assignments.length); + + // Collect all assigned partitions + final boolean[] assigned = new boolean[9]; + for (final int[] taskPartitions : assignments) { + for (final int p : taskPartitions) { + assertFalse(assigned[p], "Partition " + p + " assigned twice"); + assigned[p] = true; + } + } + for (int i = 0; i < 9; i++) { + assertTrue(assigned[i], "Partition " + i + " not assigned"); + } + } + + @Test + public void testAdaptivePartitionGrowsWithHandlers() { + final BatchQueue queue = BatchQueueManager.create("adaptive-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(8)) + .partitions(PartitionPolicy.adaptive()) + .bufferSize(100) + .build()); + + // Initial: 8 partitions (threadCount) + assertEquals(8, queue.getPartitionCount()); + + // Register 500 handlers — threshold = 8*25 = 200, so 200 + 300/2 = 350 + for (int i = 0; i < 500; i++) { + queue.addHandler(BenchmarkMetricTypes.CLASSES[i], data -> { }); + } + + assertEquals(350, queue.getPartitionCount()); + } + + @Test + public void testAdaptiveBelowThresholdIs1to1() { + final BatchQueue queue = BatchQueueManager.create("adaptive-below", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(8)) + .partitions(PartitionPolicy.adaptive()) + .bufferSize(100) + .build()); + + // Register 100 handlers — below threshold (200), so 1:1 + for (int i = 0; i < 100; i++) { + queue.addHandler(BenchmarkMetricTypes.CLASSES[i], data -> { }); + } + + assertEquals(100, queue.getPartitionCount()); + } + + // --- Shared scheduler --- + + @Test + public void testSharedSchedulerQueuesSharePool() { + final List received1 = new CopyOnWriteArrayList<>(); + final List received2 = new CopyOnWriteArrayList<>(); + + final BatchQueue q1 = BatchQueueManager.create("shared-q1", + BatchQueueConfig.builder() + .sharedScheduler("TEST_POOL", ThreadPolicy.fixed(2)) + .consumer(data -> received1.addAll(data)) + .bufferSize(1000) + .build()); + + final BatchQueue q2 = BatchQueueManager.create("shared-q2", + BatchQueueConfig.builder() + .sharedScheduler("TEST_POOL", ThreadPolicy.fixed(2)) + .consumer(data -> received2.addAll(data)) + .bufferSize(1000) + .build()); + + assertFalse(q1.isDedicatedScheduler()); + assertFalse(q2.isDedicatedScheduler()); + + for (int i = 0; i < 50; i++) { + q1.produce("a-" + i); + q2.produce("b-" + i); + } + + Awaitility.await().atMost(5, TimeUnit.SECONDS) + .until(() -> received1.size() == 50 && received2.size() == 50); + } + + @Test + public void testDedicatedSchedulerIsOwned() { + final BatchQueue queue = BatchQueueManager.create("dedicated-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(2)) + .consumer(data -> { }) + .bufferSize(100) + .build()); + + assertTrue(queue.isDedicatedScheduler()); + } + + // --- Produce and buffer strategy --- + + @Test + public void testIfPossibleDropsWhenFull() { + final CountDownLatch blockLatch = new CountDownLatch(1); + final BatchQueue queue = BatchQueueManager.create("drop-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .strategy(BufferStrategy.IF_POSSIBLE) + .consumer(data -> { + try { + blockLatch.await(); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + } + }) + .bufferSize(5) + .build()); + + // Produce enough to fill the buffer while consumer is blocked + Awaitility.await().atMost(2, TimeUnit.SECONDS) + .pollInterval(10, TimeUnit.MILLISECONDS) + .until(() -> { + queue.produce("x"); + return !queue.produce("overflow"); + }); + + blockLatch.countDown(); + } + + @Test + public void testProduceReturnsFalseWhenStopped() { + final BatchQueue queue = BatchQueueManager.create("stopped-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .consumer(data -> { }) + .bufferSize(100) + .build()); + + assertTrue(queue.produce("before-shutdown")); + queue.shutdown(); + assertFalse(queue.produce("after-shutdown")); + } + + // --- Shutdown final drain --- + + @Test + public void testShutdownDrainsRemainingData() throws Exception { + final List received = new CopyOnWriteArrayList<>(); + final CountDownLatch blockLatch = new CountDownLatch(1); + final AtomicInteger consumeCalls = new AtomicInteger(0); + + final BatchQueue queue = BatchQueueManager.create("drain-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .consumer(data -> { + if (consumeCalls.getAndIncrement() == 0) { + try { + blockLatch.await(2, TimeUnit.SECONDS); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + } + } + received.addAll(data); + }) + .bufferSize(10_000) + .strategy(BufferStrategy.IF_POSSIBLE) + .build()); + + int produced = 0; + for (int i = 0; i < 500; i++) { + if (queue.produce("item-" + i)) { + produced++; + } + } + + blockLatch.countDown(); + Thread.sleep(50); + queue.shutdown(); + + assertEquals(produced, received.size()); + } + + // --- Idle callback --- + + @Test + public void testOnIdleCalledWhenEmpty() { + final AtomicInteger idleCalls = new AtomicInteger(0); + + BatchQueueManager.create("idle-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .consumer(new HandlerConsumer() { + @Override + public void consume(final List data) { + } + + @Override + public void onIdle() { + idleCalls.incrementAndGet(); + } + }) + .bufferSize(100) + .build()); + + Awaitility.await().atMost(3, TimeUnit.SECONDS) + .until(() -> idleCalls.get() > 0); + + assertTrue(idleCalls.get() > 0); + } + + @Test + public void testHandlerMapOnIdleCalled() { + final AtomicInteger idleA = new AtomicInteger(0); + final AtomicInteger idleB = new AtomicInteger(0); + + final BatchQueue queue = BatchQueueManager.create("handler-idle-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .bufferSize(100) + .build()); + + queue.addHandler(MetricA.class, new HandlerConsumer() { + @Override + public void consume(final List data) { + } + + @Override + public void onIdle() { + idleA.incrementAndGet(); + } + }); + queue.addHandler(MetricB.class, new HandlerConsumer() { + @Override + public void consume(final List data) { + } + + @Override + public void onIdle() { + idleB.incrementAndGet(); + } + }); + + Awaitility.await().atMost(3, TimeUnit.SECONDS) + .until(() -> idleA.get() > 0 && idleB.get() > 0); + } + + // --- Error handler --- + + @Test + public void testErrorHandlerCalled() { + final AtomicInteger errorCount = new AtomicInteger(0); + + final BatchQueue queue = BatchQueueManager.create("error-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .consumer(data -> { + throw new RuntimeException("intentional"); + }) + .errorHandler((data, t) -> errorCount.incrementAndGet()) + .bufferSize(100) + .build()); + + queue.produce("trigger-error"); + + Awaitility.await().atMost(5, TimeUnit.SECONDS) + .until(() -> errorCount.get() > 0); + } + + // --- Adaptive backoff --- + + @Test + public void testAdaptiveBackoffIncreasesDelay() throws Exception { + final List idleTimestamps = new CopyOnWriteArrayList<>(); + + BatchQueueManager.create("backoff-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .minIdleMs(10) + .maxIdleMs(500) + .consumer(new HandlerConsumer() { + @Override + public void consume(final List data) { + } + + @Override + public void onIdle() { + idleTimestamps.add(System.currentTimeMillis()); + } + }) + .bufferSize(100) + .build()); + + Awaitility.await().atMost(10, TimeUnit.SECONDS) + .until(() -> idleTimestamps.size() >= 8); + + final long earlyGap = idleTimestamps.get(1) - idleTimestamps.get(0); + final long laterGap = idleTimestamps.get(idleTimestamps.size() - 1) + - idleTimestamps.get(idleTimestamps.size() - 2); + + assertTrue(laterGap > earlyGap, + "Later gap (" + laterGap + "ms) should be larger than early gap (" + earlyGap + "ms)"); + } + + @Test + public void testBackoffResetsOnData() throws Exception { + final AtomicInteger consumeCount = new AtomicInteger(0); + final List idleTimestamps = new CopyOnWriteArrayList<>(); + + final BatchQueue queue = BatchQueueManager.create("backoff-reset-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .minIdleMs(10) + .maxIdleMs(2000) + .consumer(new HandlerConsumer() { + @Override + public void consume(final List data) { + consumeCount.addAndGet(data.size()); + } + + @Override + public void onIdle() { + idleTimestamps.add(System.currentTimeMillis()); + } + }) + .bufferSize(100) + .build()); + + Awaitility.await().atMost(5, TimeUnit.SECONDS) + .until(() -> idleTimestamps.size() >= 5); + + final int beforeIdles = idleTimestamps.size(); + for (int i = 0; i < 10; i++) { + queue.produce("reset-" + i); + } + + Awaitility.await().atMost(5, TimeUnit.SECONDS) + .until(() -> consumeCount.get() >= 10); + + final long postDataTime = System.currentTimeMillis(); + Awaitility.await().atMost(3, TimeUnit.SECONDS) + .until(() -> { + for (int i = beforeIdles; i < idleTimestamps.size(); i++) { + if (idleTimestamps.get(i) > postDataTime) { + return true; + } + } + return false; + }); + } + +} diff --git a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BenchmarkMetricTypes.java b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BenchmarkMetricTypes.java new file mode 100644 index 000000000000..910c5b0ccffa --- /dev/null +++ b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BenchmarkMetricTypes.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.skywalking.oap.server.library.batchqueue; + +import java.io.ByteArrayOutputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.lang.invoke.MethodHandle; +import java.lang.invoke.MethodHandles; +import java.lang.invoke.MethodType; + +/** + * 2000 distinct metric subclasses generated at class-load time for benchmark + * handler-map dispatch testing. Each class represents a distinct metric type + * that gets routed to its own handler via {@code data.getClass().hashCode()}. + * + *

Classes are generated as bytecode at runtime using + * {@link MethodHandles.Lookup#defineClass(byte[])} to avoid a 6000+ line + * source file. Each generated class {@code Dyn0..Dyn1999} extends + * {@link TypedMetric} with a constructor that calls {@code super(typeId, v)}. + */ +@SuppressWarnings("all") +class BenchmarkMetricTypes { + + static class TypedMetric { + final int typeId; + final long value; + + TypedMetric(final int typeId, final long value) { + this.typeId = typeId; + this.value = value; + } + } + + @FunctionalInterface + interface MetricFactory { + TypedMetric create(long value); + } + + static final int MAX_TYPES = 2000; + static final Class[] CLASSES = new Class[MAX_TYPES]; + static final MetricFactory[] FACTORIES = new MetricFactory[MAX_TYPES]; + + private static final MethodHandles.Lookup LOOKUP = MethodHandles.lookup(); + private static final String SUPER_INTERNAL = + "org/apache/skywalking/oap/server/library/batchqueue/BenchmarkMetricTypes$TypedMetric"; + + static { + try { + for (int i = 0; i < MAX_TYPES; i++) { + final String name = "org/apache/skywalking/oap/server/library/batchqueue/Dyn" + i; + final byte[] bytes = buildClassBytes(name, SUPER_INTERNAL, i); + final Class cls = + (Class) LOOKUP.defineClass(bytes); + CLASSES[i] = cls; + final MethodHandle mh = LOOKUP.findConstructor( + cls, MethodType.methodType(void.class, long.class)); + FACTORIES[i] = v -> { + try { + return (TypedMetric) mh.invoke(v); + } catch (final Throwable e) { + throw new RuntimeException(e); + } + }; + } + } catch (final Exception e) { + throw new ExceptionInInitializerError(e); + } + } + + /** + * Build minimal class bytecode: + *

+     * class {name} extends TypedMetric {
+     *     {name}(long v) { super(typeId, v); }
+     * }
+     * 
+ */ + private static byte[] buildClassBytes(final String thisClass, final String superClass, + final int typeId) throws IOException { + final ByteArrayOutputStream buf = new ByteArrayOutputStream(256); + final DataOutputStream out = new DataOutputStream(buf); + + // Magic + version (Java 11 = 55) + out.writeInt(0xCAFEBABE); + out.writeShort(0); + out.writeShort(55); + + // Constant pool (10 entries, 1-indexed => count = 11) + out.writeShort(11); + // #1 Methodref -> #3.#7 (superClass.:(IJ)V) + out.writeByte(10); + out.writeShort(3); + out.writeShort(7); + // #2 Class -> #8 (this class) + out.writeByte(7); + out.writeShort(8); + // #3 Class -> #9 (super class) + out.writeByte(7); + out.writeShort(9); + // #4 Utf8 "" + out.writeByte(1); + out.writeUTF(""); + // #5 Utf8 "(J)V" + out.writeByte(1); + out.writeUTF("(J)V"); + // #6 Utf8 "Code" + out.writeByte(1); + out.writeUTF("Code"); + // #7 NameAndType -> #4:#10 (:(IJ)V) + out.writeByte(12); + out.writeShort(4); + out.writeShort(10); + // #8 Utf8 this class name + out.writeByte(1); + out.writeUTF(thisClass); + // #9 Utf8 super class name + out.writeByte(1); + out.writeUTF(superClass); + // #10 Utf8 "(IJ)V" + out.writeByte(1); + out.writeUTF("(IJ)V"); + + // Access flags: ACC_SUPER (0x0020) + out.writeShort(0x0020); + // This class (#2), Super class (#3) + out.writeShort(2); + out.writeShort(3); + // Interfaces: 0 + out.writeShort(0); + // Fields: 0 + out.writeShort(0); + + // Methods: 1 (constructor) + out.writeShort(1); + // Method access_flags: 0 (package-private) + out.writeShort(0); + // Method name: #4 () + out.writeShort(4); + // Method descriptor: #5 ((J)V) + out.writeShort(5); + // Method attributes: 1 (Code) + out.writeShort(1); + + // Code attribute + out.writeShort(6); // attribute_name_index -> "Code" + + // Build bytecode first to know length + final byte[] code = buildConstructorCode(typeId); + final int codeAttrLen = 2 + 2 + 4 + code.length + 2 + 2; // max_stack + max_locals + code_length + code + exception_table_length + attributes_count + out.writeInt(codeAttrLen); + out.writeShort(4); // max_stack (this + int + long[2 slots]) + out.writeShort(3); // max_locals (this + long[2 slots]) + out.writeInt(code.length); + out.write(code); + out.writeShort(0); // exception_table_length + out.writeShort(0); // code attributes_count + + // Class attributes: 0 + out.writeShort(0); + + out.flush(); + return buf.toByteArray(); + } + + /** + * Constructor bytecode: aload_0, sipush typeId, lload_1, invokespecial #1, return + */ + private static byte[] buildConstructorCode(final int typeId) { + final ByteArrayOutputStream code = new ByteArrayOutputStream(16); + // aload_0 + code.write(0x2A); + // sipush typeId (works for 0..32767) + code.write(0x11); + code.write((typeId >> 8) & 0xFF); + code.write(typeId & 0xFF); + // lload_1 + code.write(0x1F); + // invokespecial #1 + code.write(0xB7); + code.write(0x00); + code.write(0x01); + // return + code.write(0xB1); + return code.toByteArray(); + } +} diff --git a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/PartitionPolicyTest.java b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/PartitionPolicyTest.java new file mode 100644 index 000000000000..9cdb5d883b45 --- /dev/null +++ b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/PartitionPolicyTest.java @@ -0,0 +1,119 @@ +/* + * 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.skywalking.oap.server.library.batchqueue; + +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class PartitionPolicyTest { + + @Test + public void testFixedResolve() { + // Returns exact count, ignores both threadCount and handlerCount + assertEquals(1, PartitionPolicy.fixed(1).resolve(4, 0)); + assertEquals(8, PartitionPolicy.fixed(8).resolve(4, 0)); + assertEquals(4, PartitionPolicy.fixed(4).resolve(8, 500)); + + // Same result regardless of threadCount + final PartitionPolicy policy = PartitionPolicy.fixed(5); + assertEquals(5, policy.resolve(1, 0)); + assertEquals(5, policy.resolve(100, 0)); + } + + @Test + public void testFixedRejectsZero() { + assertThrows(IllegalArgumentException.class, () -> PartitionPolicy.fixed(0)); + } + + @Test + public void testFixedRejectsNegative() { + assertThrows(IllegalArgumentException.class, () -> PartitionPolicy.fixed(-1)); + } + + @Test + public void testThreadMultiplyResolve() { + // multiplier * threadCount, ignores handlerCount + assertEquals(8, PartitionPolicy.threadMultiply(2).resolve(4, 0)); + assertEquals(12, PartitionPolicy.threadMultiply(3).resolve(4, 0)); + assertEquals(16, PartitionPolicy.threadMultiply(2).resolve(8, 500)); + + // Even with 0 thread count, should return at least 1 + assertEquals(1, PartitionPolicy.threadMultiply(1).resolve(0, 0)); + } + + @Test + public void testThreadMultiplyRejectsZero() { + assertThrows(IllegalArgumentException.class, () -> PartitionPolicy.threadMultiply(0)); + } + + @Test + public void testAdaptiveZeroHandlersReturnsThreadCount() { + assertEquals(8, PartitionPolicy.adaptive().resolve(8, 0)); + assertEquals(4, PartitionPolicy.adaptive().resolve(4, 0)); + assertEquals(1, PartitionPolicy.adaptive().resolve(0, 0)); + } + + @Test + public void testAdaptiveBelowThreshold() { + // 8 threads * 25 = 200 threshold, handlerCount <= 200 -> 1:1 + assertEquals(50, PartitionPolicy.adaptive().resolve(8, 50)); + assertEquals(100, PartitionPolicy.adaptive().resolve(8, 100)); + assertEquals(200, PartitionPolicy.adaptive().resolve(8, 200)); + } + + @Test + public void testAdaptiveAboveThreshold() { + // 8 threads * 25 = 200 threshold, excess at 1:2 ratio + assertEquals(350, PartitionPolicy.adaptive().resolve(8, 500)); // 200 + 300/2 + assertEquals(600, PartitionPolicy.adaptive().resolve(8, 1000)); // 200 + 800/2 + assertEquals(1100, PartitionPolicy.adaptive().resolve(8, 2000)); // 200 + 1800/2 + } + + @Test + public void testAdaptiveCustomMultiplier() { + // 8 threads * 50 = 400 threshold + assertEquals(100, PartitionPolicy.adaptive(50).resolve(8, 100)); // 1:1 + assertEquals(400, PartitionPolicy.adaptive(50).resolve(8, 400)); // 1:1 at threshold + assertEquals(500, PartitionPolicy.adaptive(50).resolve(8, 600)); // 400 + 200/2 + } + + @Test + public void testAdaptiveWithDifferentThreadCounts() { + // 4 threads * 25 = 100 threshold + assertEquals(50, PartitionPolicy.adaptive().resolve(4, 50)); // 1:1 + assertEquals(100, PartitionPolicy.adaptive().resolve(4, 100)); // 1:1 + assertEquals(350, PartitionPolicy.adaptive().resolve(4, 600)); // 100 + 500/2 + } + + @Test + public void testAdaptiveRejectsInvalidMultiplier() { + assertThrows(IllegalArgumentException.class, + () -> PartitionPolicy.adaptive(0)); + } + + @Test + public void testToString() { + assertEquals("fixed(4)", PartitionPolicy.fixed(4).toString()); + assertEquals("threadMultiply(2)", PartitionPolicy.threadMultiply(2).toString()); + assertEquals("adaptive(multiplier=25)", PartitionPolicy.adaptive().toString()); + assertEquals("adaptive(multiplier=50)", PartitionPolicy.adaptive(50).toString()); + } +} diff --git a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicyTest.java b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicyTest.java new file mode 100644 index 000000000000..4a77f7ad435c --- /dev/null +++ b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicyTest.java @@ -0,0 +1,78 @@ +/* + * 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.skywalking.oap.server.library.batchqueue; + +import org.junit.jupiter.api.Test; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +public class ThreadPolicyTest { + + @Test + public void testFixedReturnsExactCount() { + assertEquals(1, ThreadPolicy.fixed(1).resolve()); + assertEquals(4, ThreadPolicy.fixed(4).resolve()); + assertEquals(100, ThreadPolicy.fixed(100).resolve()); + } + + @Test + public void testFixedRejectsZero() { + assertThrows(IllegalArgumentException.class, () -> ThreadPolicy.fixed(0)); + } + + @Test + public void testFixedRejectsNegative() { + assertThrows(IllegalArgumentException.class, () -> ThreadPolicy.fixed(-1)); + } + + @Test + public void testCpuCoresResolvesAtLeastOne() { + // Even with a tiny multiplier, resolve should return >= 1 + assertTrue(ThreadPolicy.cpuCores(0.001).resolve() >= 1); + } + + @Test + public void testCpuCoresScalesWithProcessors() { + final int cores = Runtime.getRuntime().availableProcessors(); + final int resolved = ThreadPolicy.cpuCores(1.0).resolve(); + assertEquals(cores, resolved); + } + + @Test + public void testCpuCoresRejectsZero() { + assertThrows(IllegalArgumentException.class, () -> ThreadPolicy.cpuCores(0)); + } + + @Test + public void testCpuCoresRejectsNegative() { + assertThrows(IllegalArgumentException.class, () -> ThreadPolicy.cpuCores(-0.5)); + } + + @Test + public void testToStringFixed() { + assertEquals("fixed(4)", ThreadPolicy.fixed(4).toString()); + } + + @Test + public void testToStringCpuCores() { + assertEquals("cpuCores(0.5)", ThreadPolicy.cpuCores(0.5).toString()); + } +} diff --git a/oap-server/server-library/library-batch-queue/src/test/resources/log4j2-test.xml b/oap-server/server-library/library-batch-queue/src/test/resources/log4j2-test.xml new file mode 100644 index 000000000000..6c77b4669ff9 --- /dev/null +++ b/oap-server/server-library/library-batch-queue/src/test/resources/log4j2-test.xml @@ -0,0 +1,32 @@ + + + + + + + + + + + + + + + + diff --git a/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/DataCarrierBenchmark.java b/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/DataCarrierBenchmark.java new file mode 100644 index 000000000000..ad80a9476104 --- /dev/null +++ b/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/DataCarrierBenchmark.java @@ -0,0 +1,192 @@ +/* + * 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.skywalking.oap.server.library.datacarrier; + +import java.util.List; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import org.apache.skywalking.oap.server.library.datacarrier.buffer.BufferStrategy; +import org.apache.skywalking.oap.server.library.datacarrier.consumer.BulkConsumePool; +import org.apache.skywalking.oap.server.library.datacarrier.consumer.IConsumer; +import org.junit.jupiter.api.Test; + +/** + * Throughput benchmark for DataCarrier as a baseline for BatchQueue comparison. + * + *

Simulates the real production pattern: N DataCarrier instances (one per + * metric type), each with 1 channel, all sharing a {@link BulkConsumePool} + * with 8 consumer threads. 32 producer threads simulate gRPC connections. + * + *

Run with: mvn test -pl oap-server/server-library/library-datacarrier-queue + * -Dtest=DataCarrierBenchmark -DfailIfNoTests=false + * + *

Reference results (Apple M3 Max, 128 GB RAM, macOS 26.2, JDK 17)

+ *
+ * Types  Producers  Pool threads    IF_POSSIBLE       BLOCKING
+ * ------ --------- -------------- ------------- -------------
+ *   500      32      pool(8)       ~33,400,000   ~32,500,000
+ *  1000      32      pool(8)       ~37,600,000   ~36,000,000
+ *  2000      32      pool(8)       ~38,000,000   ~42,100,000
+ *
+ * All runs: 1 channel per carrier, bufferSize=50,000, consumeCycle=1ms, 0% drop rate.
+ * 
+ * + *

BatchQueue comparison (type-aware partitions, typed objects): + *

+ *              500 types   1000 types   2000 types
+ * 1:4           -4%         +6%          +24%
+ * 1:2          +14%        +32%          +38%
+ * adaptive     +37%        +34%          +68%
+ * 1:1          +53%        +63%          +99%
+ * 
+ * + *

BatchQueue adaptive() = threshold(threadCount * 25), 1:1 below, 1:2 + * above. Consistently outperforms DataCarrier across all type counts. + * See BatchQueueBenchmark for full details. + */ +@SuppressWarnings("all") +public class DataCarrierBenchmark { + + private static final int WARMUP_SECONDS = 2; + private static final int MEASURE_SECONDS = 5; + private static final int PRODUCER_THREADS = 32; + private static final int POOL_THREADS = 8; + private static final int BUFFER_SIZE = 50_000; + + @Test + public void benchmark500Types() throws Exception { + runSharedPoolBenchmark("500-types", 500, BufferStrategy.IF_POSSIBLE); + } + + @Test + public void benchmark1000Types() throws Exception { + runSharedPoolBenchmark("1000-types", 1000, BufferStrategy.IF_POSSIBLE); + } + + @Test + public void benchmark2000Types() throws Exception { + runSharedPoolBenchmark("2000-types", 2000, BufferStrategy.IF_POSSIBLE); + } + + @Test + public void benchmark500TypesBlocking() throws Exception { + runSharedPoolBenchmark("500-types-blocking", 500, BufferStrategy.BLOCKING); + } + + @Test + public void benchmark1000TypesBlocking() throws Exception { + runSharedPoolBenchmark("1000-types-blocking", 1000, BufferStrategy.BLOCKING); + } + + @Test + public void benchmark2000TypesBlocking() throws Exception { + runSharedPoolBenchmark("2000-types-blocking", 2000, BufferStrategy.BLOCKING); + } + + private void runSharedPoolBenchmark(final String label, final int typeCount, + final BufferStrategy strategy) throws Exception { + final AtomicLong consumed = new AtomicLong(0); + + final BulkConsumePool pool = new BulkConsumePool( + "bench-pool", POOL_THREADS, 1, false); + + final DataCarrier[] carriers = new DataCarrier[typeCount]; + for (int i = 0; i < typeCount; i++) { + carriers[i] = new DataCarrier<>( + "type-" + i, "bench", 1, BUFFER_SIZE, strategy); + carriers[i].consume(pool, new IConsumer() { + @Override + public void consume(final List data) { + consumed.addAndGet(data.size()); + } + + @Override + public void onError(final List data, final Throwable t) { + t.printStackTrace(); + } + }); + } + + // Warmup + final long warmupEnd = System.currentTimeMillis() + WARMUP_SECONDS * 1000L; + runProducers(carriers, warmupEnd); + Thread.sleep(200); + consumed.set(0); + + // Measure + final long measureStart = System.currentTimeMillis(); + final long measureEnd = measureStart + MEASURE_SECONDS * 1000L; + final long produced = runProducers(carriers, measureEnd); + final long measureDuration = System.currentTimeMillis() - measureStart; + + Thread.sleep(500); + final long totalConsumed = consumed.get(); + + pool.close(null); + + System.out.printf("%n=== DataCarrier Benchmark: %s ===%n" + + " Types: %d (1 DataCarrier per type, 1 channel each)%n" + + " Pool threads:%d%n" + + " Strategy: %s%n" + + " Producers: %d%n" + + " Duration: %d ms%n" + + " Produced: %,d%n" + + " Consumed: %,d%n" + + " Consume rate: %,.0f items/sec%n" + + " Drop rate: %.2f%%%n", + label, typeCount, POOL_THREADS, strategy, + PRODUCER_THREADS, measureDuration, + produced, totalConsumed, + totalConsumed * 1000.0 / measureDuration, + produced > 0 ? (produced - totalConsumed) * 100.0 / produced : 0); + } + + private long runProducers(final DataCarrier[] carriers, + final long endTimeMs) throws InterruptedException { + final int carrierCount = carriers.length; + final AtomicLong totalProduced = new AtomicLong(0); + final CountDownLatch done = new CountDownLatch(PRODUCER_THREADS); + + for (int p = 0; p < PRODUCER_THREADS; p++) { + final int producerIndex = p; + final Thread thread = new Thread(() -> { + long count = 0; + int typeIndex = producerIndex; + while (System.currentTimeMillis() < endTimeMs) { + for (int batch = 0; batch < 100; batch++) { + final int type = typeIndex % carrierCount; + if (carriers[type].produce(count)) { + count++; + } + typeIndex++; + } + } + totalProduced.addAndGet(count); + done.countDown(); + }); + thread.setName("Producer-" + producerIndex); + thread.setDaemon(true); + thread.start(); + } + + done.await(MEASURE_SECONDS + 10, TimeUnit.SECONDS); + return totalProduced.get(); + } +} diff --git a/oap-server/server-library/library-util/src/test/java/org/apache/skywalking/oap/server/library/util/benchmark/StringFormatGroupBenchmark.java b/oap-server/server-library/library-util/src/test/java/org/apache/skywalking/oap/server/library/util/benchmark/StringFormatGroupBenchmark.java new file mode 100644 index 000000000000..ad850288deaf --- /dev/null +++ b/oap-server/server-library/library-util/src/test/java/org/apache/skywalking/oap/server/library/util/benchmark/StringFormatGroupBenchmark.java @@ -0,0 +1,79 @@ +/* + * 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.skywalking.oap.server.library.util.benchmark; + +import lombok.extern.slf4j.Slf4j; +import org.apache.skywalking.oap.server.library.util.StringFormatGroup; +import org.junit.jupiter.api.Test; + +@Slf4j +public class StringFormatGroupBenchmark { + + private static final int WARMUP_ITERATIONS = 5000; + private static final int MEASURE_ITERATIONS = 100_000; + + @Test + public void testMatch() { + for (int i = 0; i < WARMUP_ITERATIONS; i++) { + doMatch(); + } + final long start = System.nanoTime(); + for (int i = 0; i < MEASURE_ITERATIONS; i++) { + doMatch(); + } + final long elapsed = System.nanoTime() - start; + log.info("testMatch: {} ops, {} ns/op, {} ops/s", + MEASURE_ITERATIONS, elapsed / MEASURE_ITERATIONS, + MEASURE_ITERATIONS * 1_000_000_000L / elapsed); + } + + @Test + public void test100Rule() { + for (int i = 0; i < WARMUP_ITERATIONS; i++) { + do100Rule(); + } + final long start = System.nanoTime(); + for (int i = 0; i < MEASURE_ITERATIONS; i++) { + do100Rule(); + } + final long elapsed = System.nanoTime() - start; + log.info("test100Rule: {} ops, {} ns/op, {} ops/s", + MEASURE_ITERATIONS, elapsed / MEASURE_ITERATIONS, + MEASURE_ITERATIONS * 1_000_000_000L / elapsed); + } + + private void doMatch() { + StringFormatGroup group = new StringFormatGroup(); + group.addRule("/name/*/add", "/name/.+/add"); + group.format("/name/test/add"); + + group = new StringFormatGroup(); + group.addRule("/name/*/add/{orderId}", "/name/.+/add/.*"); + group.format("/name/test/add/12323"); + } + + private void do100Rule() { + final StringFormatGroup group = new StringFormatGroup(); + group.addRule("/name/*/add/{orderId}", "/name/.+/add/.*"); + for (int i = 0; i < 100; i++) { + group.addRule("/name/*/add/{orderId}" + "/" + 1, "/name/.+/add/.*" + "/abc"); + } + group.format("/name/test/add/12323"); + } +} diff --git a/oap-server/server-library/pom.xml b/oap-server/server-library/pom.xml index 84eeae326074..3505d2289330 100644 --- a/oap-server/server-library/pom.xml +++ b/oap-server/server-library/pom.xml @@ -39,5 +39,6 @@ library-pprof-parser library-integration-test library-banyandb-client + library-batch-queue From f2c5dc9f9110aee931b47f02387b57469c930a40 Mon Sep 17 00:00:00 2001 From: Wu Sheng Date: Sat, 14 Feb 2026 10:25:03 +0800 Subject: [PATCH 02/16] Replace DataCarrier with BatchQueue for metrics pipeline Replace DataCarrier with BatchQueue for L1 metrics aggregation, L2 metrics persistence, TopN persistence, all three exporters (gRPC metrics, Kafka trace, Kafka log), and gRPC remote client. Remove the library-datacarrier-queue module entirely. All metric types (OAL + MAL) now share unified queues instead of separate pools. Thread count reduced from 36 to 15 on 8-core. Co-Authored-By: Claude Opus 4.6 --- docs/en/changes/changes.md | 19 ++ docs/en/setup/backend/grafana-cluster.json | 222 +----------- docs/en/setup/backend/grafana-instance.json | 212 +----------- oap-server/exporter/pom.xml | 2 +- .../exporter/provider/ExporterSetting.java | 3 +- .../provider/grpc/GRPCMetricsExporter.java | 36 +- .../provider/kafka/log/KafkaLogExporter.java | 36 +- .../kafka/trace/KafkaTraceExporter.java | 37 +- .../grpc/GRPCExporterProviderTest.java | 3 +- .../provider/grpc/GRPCExporterTest.java | 31 +- oap-server/microbench/pom.xml | 5 - .../datacarrier/LinkedArrayBenchmark.java | 315 ------------------ .../common/AtomicRangeIntegerBenchmark.java | 166 --------- .../common/AtomicRangeIntegerV1.java | 71 ---- .../common/AtomicRangeIntegerV2.java | 71 ---- oap-server/server-core/pom.xml | 2 +- .../worker/MetricsAggregateMALWorker.java | 65 ---- .../worker/MetricsAggregateOALWorker.java | 48 --- .../worker/MetricsAggregateWorker.java | 178 +++++----- .../worker/MetricsPersistentMinMALWorker.java | 62 ---- .../worker/MetricsPersistentMinOALWorker.java | 52 --- .../worker/MetricsPersistentMinWorker.java | 128 ++++--- .../worker/MetricsStreamProcessor.java | 41 +-- .../analysis/worker/TopNStreamProcessor.java | 2 +- .../core/analysis/worker/TopNWorker.java | 44 ++- .../server/core/exporter/ExporterService.java | 2 - .../core/remote/client/GRPCRemoteClient.java | 49 ++- .../remote/client/RemoteClientManager.java | 2 +- .../client/GRPCRemoteClientRealClient.java | 2 +- .../client/GRPCRemoteClientTestCase.java | 2 +- .../server/library/batchqueue/BatchQueue.java | 12 + .../library/batchqueue/BatchQueueStats.java | 133 ++++++++ .../library/batchqueue/ThreadPolicy.java | 34 +- .../library/batchqueue/BatchQueueTest.java | 190 +++++++++++ .../library/batchqueue/ThreadPolicyTest.java | 32 ++ .../library-datacarrier-queue/pom.xml | 37 -- .../library/datacarrier/DataCarrier.java | 168 ---------- .../server/library/datacarrier/EnvUtil.java | 50 --- .../buffer/ArrayBlockingQueueBuffer.java | 69 ---- .../datacarrier/buffer/BufferStrategy.java | 23 -- .../library/datacarrier/buffer/Channels.java | 91 ----- .../datacarrier/buffer/QueueBuffer.java | 46 --- .../common/AtomicRangeInteger.java | 76 ----- .../datacarrier/consumer/BulkConsumePool.java | 128 ------- .../datacarrier/consumer/ConsumeDriver.java | 137 -------- .../ConsumerCannotBeCreatedException.java | 25 -- .../datacarrier/consumer/ConsumerPool.java | 30 -- .../consumer/ConsumerPoolFactory.java | 50 --- .../datacarrier/consumer/ConsumerThread.java | 101 ------ .../datacarrier/consumer/IConsumer.java | 41 --- .../library/datacarrier/consumer/IDriver.java | 32 -- .../consumer/MultipleChannelsConsumer.java | 145 -------- .../partition/IDataPartitioner.java | 32 -- .../partition/ProducerThreadPartitioner.java | 37 -- .../partition/SimpleRollingPartitioner.java | 37 -- .../datacarrier/DataCarrierBenchmark.java | 192 ----------- .../library/datacarrier/DataCarrierTest.java | 134 -------- .../library/datacarrier/EnvUtilTest.java | 54 --- .../library/datacarrier/SampleData.java | 43 --- .../consumer/ConsumeDriverTest.java | 56 ---- .../consumer/ConsumerPoolFactoryTest.java | 48 --- .../datacarrier/consumer/ConsumerTest.java | 126 ------- .../datacarrier/consumer/SampleConsumer.java | 39 --- .../ProducerThreadPartitionerTest.java | 35 -- .../SimpleRollingPartitionerTest.java | 34 -- oap-server/server-library/pom.xml | 1 - .../src/main/resources/otel-rules/oap.yaml | 2 +- .../so11y_oap/so11y-instance.json | 60 +--- .../storage-banyandb-plugin/pom.xml | 5 - .../storage-jdbc-hikaricp-plugin/pom.xml | 2 +- .../plugin/jdbc/common/dao/JDBCBatchDAO.java | 56 ++-- 71 files changed, 774 insertions(+), 3777 deletions(-) delete mode 100644 oap-server/microbench/src/main/java/org/apache/skywalking/oap/server/microbench/library/datacarrier/LinkedArrayBenchmark.java delete mode 100644 oap-server/microbench/src/main/java/org/apache/skywalking/oap/server/microbench/library/datacarrier/common/AtomicRangeIntegerBenchmark.java delete mode 100644 oap-server/microbench/src/main/java/org/apache/skywalking/oap/server/microbench/library/datacarrier/common/AtomicRangeIntegerV1.java delete mode 100644 oap-server/microbench/src/main/java/org/apache/skywalking/oap/server/microbench/library/datacarrier/common/AtomicRangeIntegerV2.java delete mode 100644 oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsAggregateMALWorker.java delete mode 100644 oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsAggregateOALWorker.java delete mode 100644 oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsPersistentMinMALWorker.java delete mode 100644 oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsPersistentMinOALWorker.java create mode 100644 oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueStats.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/pom.xml delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/DataCarrier.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/EnvUtil.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/buffer/ArrayBlockingQueueBuffer.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/buffer/BufferStrategy.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/buffer/Channels.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/buffer/QueueBuffer.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/common/AtomicRangeInteger.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/BulkConsumePool.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumeDriver.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerCannotBeCreatedException.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerPool.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerPoolFactory.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerThread.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/IConsumer.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/IDriver.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/MultipleChannelsConsumer.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/partition/IDataPartitioner.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/partition/ProducerThreadPartitioner.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/partition/SimpleRollingPartitioner.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/DataCarrierBenchmark.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/DataCarrierTest.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/EnvUtilTest.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/SampleData.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumeDriverTest.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerPoolFactoryTest.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerTest.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/SampleConsumer.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/partition/ProducerThreadPartitionerTest.java delete mode 100644 oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/partition/SimpleRollingPartitionerTest.java diff --git a/docs/en/changes/changes.md b/docs/en/changes/changes.md index ec5d97fd4cad..1b067e364ddd 100644 --- a/docs/en/changes/changes.md +++ b/docs/en/changes/changes.md @@ -12,6 +12,25 @@ * Add `CLAUDE.md` as AI assistant guide for the project. * Upgrade Groovy to 5.0.3 in OAP backend. * Bump up nodejs to v24.13.0 for the latest UI(booster-ui) compiling. +* Add `library-batch-queue` module — a partitioned, self-draining queue with type-based dispatch, + adaptive partitioning, and idle backoff. Designed to replace DataCarrier in high-fan-out scenarios. +* Replace DataCarrier with BatchQueue for L1 metrics aggregation, L2 metrics persistence, TopN persistence, + all three exporters (gRPC metrics, Kafka trace, Kafka log), and gRPC remote client. + All metric types (OAL + MAL) now share unified queues instead of separate OAL/MAL pools. + Each exporter keeps its own dedicated queue with 1 thread, preserving original buffer strategies. + Thread count comparison on an 8-core machine (gRPC remote client excluded — unchanged 1 thread per peer): + + | Queue | Old threads | Old channels | Old buffer slots | New threads | New partitions | New buffer slots | New policy | + |-------|-------------|--------------|------------------|-------------|----------------|------------------|------------| + | L1 Aggregation (OAL) | 24 | ~1,240 | ~12.4M | 8 (unified) | ~460 adaptive | ~9.2M | `cpuCores(1.0)` | + | L1 Aggregation (MAL) | 2 | ~100 | ~100K | (unified above) | | | | + | L2 Persistence (OAL) | 2 | ~620 | ~1.24M | 3 (unified) | ~460 adaptive | ~920K | `cpuCoresWithBase(1, 0.25)` | + | L2 Persistence (MAL) | 1 | ~100 | ~100K | (unified above) | | | | + | TopN Persistence | 4 | 4 | 4K | 1 | 4 adaptive | 4K | `fixed(1)` | + | Exporters (gRPC/Kafka) | 3 | 6 | 120K | 3 (1 per exporter) | — | 60K | `fixed(1)` each | + | **Total** | **36** | **~2,070** | **~13.9M** | **15** | **~924** | **~10.2M** | | + +* Remove `library-datacarrier-queue` module. All usages have been replaced by `library-batch-queue`. #### OAP Server diff --git a/docs/en/setup/backend/grafana-cluster.json b/docs/en/setup/backend/grafana-cluster.json index ea60f3486320..3a61beb9ebf6 100644 --- a/docs/en/setup/backend/grafana-cluster.json +++ b/docs/en/setup/backend/grafana-cluster.json @@ -6693,17 +6693,17 @@ "uid": "$datasource" }, "editorMode": "code", - "expr": "topk(10,avg by(metricName)(metrics_aggregation_queue_used_percentage{job=\"$job\",level=\"1\",kind=\"OAL\"}))", + "expr": "avg by(slot)(metrics_aggregation_queue_used_percentage{job=\"$job\",level=\"1\"})", "format": "time_series", "hide": false, "interval": "", "intervalFactor": 1, - "legendFormat": "{{metricName}}", + "legendFormat": "{{slot}}", "range": true, "refId": "A" } ], - "title": "OAL L1 Aggregation Queue Percentage (%)", + "title": "L1 Aggregation Queue Percentage (%)", "type": "timeseries" }, { @@ -6793,17 +6793,17 @@ "uid": "$datasource" }, "editorMode": "code", - "expr": "topk(10,avg by(metricName)(metrics_aggregation_queue_used_percentage{job=\"$job\",level=\"1\",kind=\"MAL\"}))", + "expr": "avg by(slot)(metrics_aggregation_queue_used_percentage{job=\"$job\",level=\"2\"})", "format": "time_series", "hide": false, "interval": "", "intervalFactor": 1, - "legendFormat": "{{metricName}}", + "legendFormat": "{{slot}}", "range": true, "refId": "A" } ], - "title": "MAL L1 Aggregation Queue Percentage (%)", + "title": "L2 Aggregation Queue Percentage (%)", "type": "timeseries" }, { @@ -6871,206 +6871,6 @@ "x": 8, "y": 122 }, - "id": 150, - "interval": "1m", - "options": { - "dataLinks": [], - "legend": { - "calcs": [], - "displayMode": "list", - "placement": "bottom", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "none" - } - }, - "pluginVersion": "11.3.1", - "targets": [ - { - "datasource": { - "uid": "$datasource" - }, - "editorMode": "code", - "expr": "topk(10,avg by(metricName)(metrics_aggregation_queue_used_percentage{job=\"$job\",level=\"2\",kind=\"OAL\"}))", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{metricName}}", - "range": true, - "refId": "A" - } - ], - "title": "OAL L2 Aggregation Queue Percentage (%)", - "type": "timeseries" - }, - { - "datasource": { - "uid": "$datasource" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisBorderShow": false, - "axisCenteredZero": false, - "axisColorMode": "text", - "axisLabel": "", - "axisPlacement": "auto", - "barAlignment": 0, - "barWidthFactor": 0.6, - "drawStyle": "line", - "fillOpacity": 10, - "gradientMode": "none", - "hideFrom": { - "legend": false, - "tooltip": false, - "viz": false - }, - "insertNulls": false, - "lineInterpolation": "linear", - "lineWidth": 1, - "pointSize": 5, - "scaleDistribution": { - "type": "linear" - }, - "showPoints": "never", - "spanNulls": false, - "stacking": { - "group": "A", - "mode": "none" - }, - "thresholdsStyle": { - "mode": "off" - } - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - }, - { - "color": "red", - "value": 80 - } - ] - } - }, - "overrides": [] - }, - "gridPos": { - "h": 6, - "w": 8, - "x": 16, - "y": 122 - }, - "id": 151, - "interval": "1m", - "options": { - "dataLinks": [], - "legend": { - "calcs": [], - "displayMode": "list", - "placement": "bottom", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "none" - } - }, - "pluginVersion": "11.3.1", - "targets": [ - { - "datasource": { - "uid": "$datasource" - }, - "editorMode": "code", - "expr": "topk(10,avg by(metricName)(metrics_aggregation_queue_used_percentage{job=\"$job\",level=\"2\",kind=\"MAL\"}))", - "format": "time_series", - "hide": false, - "interval": "", - "intervalFactor": 1, - "legendFormat": "{{metricName}}", - "range": true, - "refId": "A" - } - ], - "title": "MAL L2 Aggregation Queue Percentage (%)", - "type": "timeseries" - }, - { - "datasource": { - "uid": "$datasource" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisBorderShow": false, - "axisCenteredZero": false, - "axisColorMode": "text", - "axisLabel": "", - "axisPlacement": "auto", - "barAlignment": 0, - "barWidthFactor": 0.6, - "drawStyle": "line", - "fillOpacity": 10, - "gradientMode": "none", - "hideFrom": { - "legend": false, - "tooltip": false, - "viz": false - }, - "insertNulls": false, - "lineInterpolation": "linear", - "lineWidth": 1, - "pointSize": 5, - "scaleDistribution": { - "type": "linear" - }, - "showPoints": "never", - "spanNulls": false, - "stacking": { - "group": "A", - "mode": "none" - }, - "thresholdsStyle": { - "mode": "off" - } - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - }, - { - "color": "red", - "value": 80 - } - ] - } - }, - "overrides": [] - }, - "gridPos": { - "h": 6, - "w": 8, - "x": 0, - "y": 128 - }, "id": 149, "interval": "1m", "options": { @@ -7168,8 +6968,8 @@ "gridPos": { "h": 6, "w": 8, - "x": 8, - "y": 128 + "x": 16, + "y": 122 }, "id": 152, "interval": "1m", @@ -7269,7 +7069,7 @@ "gridPos": { "h": 6, "w": 8, - "x": 16, + "x": 0, "y": 128 }, "id": 146, @@ -7421,8 +7221,8 @@ "gridPos": { "h": 6, "w": 8, - "x": 0, - "y": 134 + "x": 8, + "y": 128 }, "id": 145, "interval": "1m", diff --git a/docs/en/setup/backend/grafana-instance.json b/docs/en/setup/backend/grafana-instance.json index 4c78f5c31378..ef2070fa1d5d 100644 --- a/docs/en/setup/backend/grafana-instance.json +++ b/docs/en/setup/backend/grafana-instance.json @@ -7518,17 +7518,17 @@ "uid": "$datasource" }, "editorMode": "code", - "expr": "topk(10,metrics_aggregation_queue_used_percentage{instance=\"$instance\",job=\"$job\",level=\"1\",kind=\"OAL\"})", + "expr": "metrics_aggregation_queue_used_percentage{instance=\"$instance\",job=\"$job\",level=\"1\"}", "format": "time_series", "hide": false, "interval": "1m", "intervalFactor": 1, - "legendFormat": "{{metricName}}", + "legendFormat": "{{slot}}", "range": true, "refId": "A" } ], - "title": "OAL L1 Aggregation Queue Percentage (%)", + "title": "L1 Aggregation Queue Percentage (%)", "type": "timeseries" }, { @@ -7618,217 +7618,17 @@ "uid": "$datasource" }, "editorMode": "code", - "expr": "topk(10,metrics_aggregation_queue_used_percentage{instance=\"$instance\",job=\"$job\",level=\"2\",kind=\"OAL\"})", - "format": "time_series", - "hide": false, - "interval": "1m", - "intervalFactor": 1, - "legendFormat": "{{metricName}}", - "range": true, - "refId": "A" - } - ], - "title": "OAL L2 Aggregation Queue Percentage (%)", - "type": "timeseries" - }, - { - "datasource": { - "uid": "$datasource" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisBorderShow": false, - "axisCenteredZero": false, - "axisColorMode": "text", - "axisLabel": "", - "axisPlacement": "auto", - "barAlignment": 0, - "barWidthFactor": 0.6, - "drawStyle": "line", - "fillOpacity": 10, - "gradientMode": "none", - "hideFrom": { - "legend": false, - "tooltip": false, - "viz": false - }, - "insertNulls": false, - "lineInterpolation": "linear", - "lineWidth": 1, - "pointSize": 5, - "scaleDistribution": { - "type": "linear" - }, - "showPoints": "never", - "spanNulls": false, - "stacking": { - "group": "A", - "mode": "none" - }, - "thresholdsStyle": { - "mode": "off" - } - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - }, - { - "color": "red", - "value": 80 - } - ] - } - }, - "overrides": [] - }, - "gridPos": { - "h": 6, - "w": 8, - "x": 0, - "y": 148 - }, - "id": 148, - "interval": "1m", - "options": { - "dataLinks": [], - "legend": { - "calcs": [], - "displayMode": "list", - "placement": "bottom", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "none" - } - }, - "pluginVersion": "11.3.1", - "targets": [ - { - "datasource": { - "uid": "$datasource" - }, - "editorMode": "code", - "expr": "topk(10,metrics_aggregation_queue_used_percentage{instance=\"$instance\",job=\"$job\",level=\"1\",kind=\"MAL\"})", + "expr": "metrics_aggregation_queue_used_percentage{instance=\"$instance\",job=\"$job\",level=\"2\"}", "format": "time_series", "hide": false, "interval": "1m", "intervalFactor": 1, - "legendFormat": "{{metricName}}", - "range": true, - "refId": "A" - } - ], - "title": "MAL L1 Aggregation Queue Percentage (%)", - "type": "timeseries" - }, - { - "datasource": { - "uid": "$datasource" - }, - "fieldConfig": { - "defaults": { - "color": { - "mode": "palette-classic" - }, - "custom": { - "axisBorderShow": false, - "axisCenteredZero": false, - "axisColorMode": "text", - "axisLabel": "", - "axisPlacement": "auto", - "barAlignment": 0, - "barWidthFactor": 0.6, - "drawStyle": "line", - "fillOpacity": 10, - "gradientMode": "none", - "hideFrom": { - "legend": false, - "tooltip": false, - "viz": false - }, - "insertNulls": false, - "lineInterpolation": "linear", - "lineWidth": 1, - "pointSize": 5, - "scaleDistribution": { - "type": "linear" - }, - "showPoints": "never", - "spanNulls": false, - "stacking": { - "group": "A", - "mode": "none" - }, - "thresholdsStyle": { - "mode": "off" - } - }, - "mappings": [], - "thresholds": { - "mode": "absolute", - "steps": [ - { - "color": "green", - "value": null - }, - { - "color": "red", - "value": 80 - } - ] - } - }, - "overrides": [] - }, - "gridPos": { - "h": 6, - "w": 8, - "x": 8, - "y": 148 - }, - "id": 151, - "interval": "1m", - "options": { - "dataLinks": [], - "legend": { - "calcs": [], - "displayMode": "list", - "placement": "bottom", - "showLegend": true - }, - "tooltip": { - "mode": "multi", - "sort": "none" - } - }, - "pluginVersion": "11.3.1", - "targets": [ - { - "datasource": { - "uid": "$datasource" - }, - "editorMode": "code", - "expr": "topk(10,metrics_aggregation_queue_used_percentage{instance=\"$instance\",job=\"$job\",level=\"2\",kind=\"MAL\"})", - "format": "time_series", - "hide": false, - "interval": "1m", - "intervalFactor": 1, - "legendFormat": "{{metricName}}", + "legendFormat": "{{slot}}", "range": true, "refId": "A" } ], - "title": "MAL L2 Aggregation Queue Percentage (%)", + "title": "L2 Aggregation Queue Percentage (%)", "type": "timeseries" }, { diff --git a/oap-server/exporter/pom.xml b/oap-server/exporter/pom.xml index ca75e04522e7..85dc662d5177 100644 --- a/oap-server/exporter/pom.xml +++ b/oap-server/exporter/pom.xml @@ -35,7 +35,7 @@ org.apache.skywalking - library-datacarrier-queue + library-batch-queue ${project.version} diff --git a/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/ExporterSetting.java b/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/ExporterSetting.java index d63be42e87cc..cde8c737cfdf 100644 --- a/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/ExporterSetting.java +++ b/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/ExporterSetting.java @@ -28,8 +28,7 @@ public class ExporterSetting extends ModuleConfig { private boolean enableGRPCMetrics = false; private String gRPCTargetHost; private int gRPCTargetPort; - private int bufferChannelSize = 20000; - private int bufferChannelNum = 2; + private int bufferSize = 20000; //kafka private boolean enableKafkaTrace = false; diff --git a/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCMetricsExporter.java b/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCMetricsExporter.java index d2b6bda6fd24..81691c605c65 100644 --- a/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCMetricsExporter.java +++ b/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCMetricsExporter.java @@ -48,14 +48,17 @@ import org.apache.skywalking.oap.server.exporter.grpc.SubscriptionsResp; import org.apache.skywalking.oap.server.exporter.provider.ExporterSetting; import org.apache.skywalking.oap.server.exporter.provider.MetricFormatter; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueue; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueConfig; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueManager; +import org.apache.skywalking.oap.server.library.batchqueue.BufferStrategy; +import org.apache.skywalking.oap.server.library.batchqueue.ThreadPolicy; import org.apache.skywalking.oap.server.library.client.grpc.GRPCClient; -import org.apache.skywalking.oap.server.library.datacarrier.DataCarrier; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.IConsumer; import org.apache.skywalking.oap.server.library.util.CollectionUtils; import org.apache.skywalking.oap.server.library.util.GRPCStreamStatus; @Slf4j -public class GRPCMetricsExporter extends MetricFormatter implements MetricValuesExportService, IConsumer { +public class GRPCMetricsExporter extends MetricFormatter implements MetricValuesExportService { /** * The period of subscription list fetching is hardcoded as 30s. */ @@ -63,7 +66,7 @@ public class GRPCMetricsExporter extends MetricFormatter implements MetricValues private final ExporterSetting setting; private MetricExportServiceGrpc.MetricExportServiceStub exportServiceFutureStub; private MetricExportServiceGrpc.MetricExportServiceBlockingStub blockingStub; - private DataCarrier exportBuffer; + private BatchQueue queue; private ReentrantLock fetchListLock; private volatile List subscriptionList; private volatile long lastFetchTimestamp = 0; @@ -72,15 +75,22 @@ public GRPCMetricsExporter(ExporterSetting setting) { this.setting = setting; } - @Override public void start() { GRPCClient client = new GRPCClient(setting.getGRPCTargetHost(), setting.getGRPCTargetPort()); client.connect(); ManagedChannel channel = client.getChannel(); exportServiceFutureStub = MetricExportServiceGrpc.newStub(channel); blockingStub = MetricExportServiceGrpc.newBlockingStub(channel); - exportBuffer = new DataCarrier(setting.getBufferChannelNum(), setting.getBufferChannelSize()); - exportBuffer.consume(this, 1, 200); + this.queue = BatchQueueManager.create( + "EXPORTER_GRPC_METRICS", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .bufferSize(setting.getBufferSize()) + .strategy(BufferStrategy.BLOCKING) + .consumer(this::consumeExportData) + .maxIdleMs(200) + .build() + ); subscriptionList = new ArrayList<>(); fetchListLock = new ReentrantLock(); } @@ -91,12 +101,12 @@ public void export(ExportEvent event) { if (metrics instanceof WithMetadata) { MetricsMetaInfo meta = ((WithMetadata) metrics).getMeta(); if (subscriptionList.size() == 0 && ExportEvent.EventType.INCREMENT.equals(event.getType())) { - exportBuffer.produce(new ExportData(meta, metrics, event.getType())); + queue.produce(new ExportData(meta, metrics, event.getType())); } else { subscriptionList.forEach(subscriptionMetric -> { if (subscriptionMetric.getMetricName().equals(meta.getMetricsName()) && eventTypeMatch(event.getType(), subscriptionMetric.getEventType())) { - exportBuffer.produce(new ExportData(meta, metrics, event.getType())); + queue.produce(new ExportData(meta, metrics, event.getType())); } }); } @@ -133,8 +143,7 @@ public void fetchSubscriptionList() { } } - @Override - public void consume(List data) { + void consumeExportData(List data) { if (CollectionUtils.isNotEmpty(data)) { GRPCStreamStatus status = new GRPCStreamStatus(); StreamObserver streamObserver = @@ -247,11 +256,6 @@ public void onCompleted() { fetchSubscriptionList(); } - @Override - public void onError(List data, Throwable t) { - log.error(t.getMessage(), t); - } - private boolean eventTypeMatch(ExportEvent.EventType eventType, org.apache.skywalking.oap.server.exporter.grpc.EventType subscriptionType) { return (ExportEvent.EventType.INCREMENT.equals(eventType) && EventType.INCREMENT.equals(subscriptionType)) diff --git a/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/kafka/log/KafkaLogExporter.java b/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/kafka/log/KafkaLogExporter.java index bc54fd7360b8..0c5f52df3db4 100644 --- a/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/kafka/log/KafkaLogExporter.java +++ b/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/kafka/log/KafkaLogExporter.java @@ -37,9 +37,11 @@ import org.apache.skywalking.oap.server.core.query.type.ContentType; import org.apache.skywalking.oap.server.exporter.provider.ExporterSetting; import org.apache.skywalking.oap.server.exporter.provider.kafka.KafkaExportProducer; -import org.apache.skywalking.oap.server.library.datacarrier.DataCarrier; -import org.apache.skywalking.oap.server.library.datacarrier.buffer.BufferStrategy; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.IConsumer; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueue; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueConfig; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueManager; +import org.apache.skywalking.oap.server.library.batchqueue.BufferStrategy; +import org.apache.skywalking.oap.server.library.batchqueue.ThreadPolicy; import org.apache.skywalking.oap.server.library.module.ModuleManager; import org.apache.skywalking.oap.server.library.util.StringUtil; import org.apache.skywalking.oap.server.telemetry.TelemetryModule; @@ -48,8 +50,8 @@ import org.apache.skywalking.oap.server.telemetry.api.MetricsTag; @Slf4j -public class KafkaLogExporter extends KafkaExportProducer implements LogExportService, IConsumer { - private DataCarrier exportBuffer; +public class KafkaLogExporter extends KafkaExportProducer implements LogExportService { + private BatchQueue queue; private CounterMetrics successCounter; private CounterMetrics errorCounter; private final ModuleManager moduleManager; @@ -59,14 +61,18 @@ public KafkaLogExporter(ModuleManager manager, ExporterSetting setting) { this.moduleManager = manager; } - @Override public void start() { super.getProducer(); - exportBuffer = new DataCarrier<>( - "KafkaLogExporter", "KafkaLogExporter", setting.getBufferChannelNum(), setting.getBufferChannelSize(), - BufferStrategy.IF_POSSIBLE + this.queue = BatchQueueManager.create( + "EXPORTER_KAFKA_LOG", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .bufferSize(setting.getBufferSize()) + .strategy(BufferStrategy.IF_POSSIBLE) + .consumer(this::consumeLogRecords) + .maxIdleMs(200) + .build() ); - exportBuffer.consume(this, 1, 200); MetricsCreator metricsCreator = moduleManager.find(TelemetryModule.NAME) .provider() .getService(MetricsCreator.class); @@ -84,7 +90,7 @@ public void start() { @Override public void export(final LogRecord logRecord) { if (logRecord != null) { - exportBuffer.produce(logRecord); + queue.produce(logRecord); } } @@ -93,8 +99,7 @@ public boolean isEnabled() { return setting.isEnableKafkaLog(); } - @Override - public void consume(final List data) { + private void consumeLogRecords(final List data) { for (LogRecord logRecord : data) { if (logRecord != null) { try { @@ -120,11 +125,6 @@ public void consume(final List data) { } } - @Override - public void onError(final List data, final Throwable t) { - - } - private LogData transLogData(LogRecord logRecord) throws InvalidProtocolBufferException { LogData.Builder builder = LogData.newBuilder(); LogDataBody.Builder bodyBuilder = LogDataBody.newBuilder(); diff --git a/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/kafka/trace/KafkaTraceExporter.java b/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/kafka/trace/KafkaTraceExporter.java index 0c3d90f48202..5902eabc70ef 100644 --- a/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/kafka/trace/KafkaTraceExporter.java +++ b/oap-server/exporter/src/main/java/org/apache/skywalking/oap/server/exporter/provider/kafka/trace/KafkaTraceExporter.java @@ -30,9 +30,11 @@ import org.apache.skywalking.oap.server.core.exporter.TraceExportService; import org.apache.skywalking.oap.server.exporter.provider.ExporterSetting; import org.apache.skywalking.oap.server.exporter.provider.kafka.KafkaExportProducer; -import org.apache.skywalking.oap.server.library.datacarrier.DataCarrier; -import org.apache.skywalking.oap.server.library.datacarrier.buffer.BufferStrategy; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.IConsumer; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueue; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueConfig; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueManager; +import org.apache.skywalking.oap.server.library.batchqueue.BufferStrategy; +import org.apache.skywalking.oap.server.library.batchqueue.ThreadPolicy; import org.apache.skywalking.oap.server.library.module.ModuleManager; import org.apache.skywalking.oap.server.telemetry.TelemetryModule; import org.apache.skywalking.oap.server.telemetry.api.CounterMetrics; @@ -40,8 +42,8 @@ import org.apache.skywalking.oap.server.telemetry.api.MetricsTag; @Slf4j -public class KafkaTraceExporter extends KafkaExportProducer implements TraceExportService, IConsumer { - private DataCarrier exportBuffer; +public class KafkaTraceExporter extends KafkaExportProducer implements TraceExportService { + private BatchQueue queue; private CounterMetrics successCounter; private CounterMetrics errorCounter; private final ModuleManager moduleManager; @@ -51,14 +53,18 @@ public KafkaTraceExporter(ModuleManager manager, ExporterSetting setting) { this.moduleManager = manager; } - @Override public void start() { super.getProducer(); - exportBuffer = new DataCarrier<>( - "KafkaTraceExporter", "KafkaTraceExporter", setting.getBufferChannelNum(), setting.getBufferChannelSize(), - BufferStrategy.IF_POSSIBLE + this.queue = BatchQueueManager.create( + "EXPORTER_KAFKA_TRACE", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .bufferSize(setting.getBufferSize()) + .strategy(BufferStrategy.IF_POSSIBLE) + .consumer(this::consumeSegmentRecords) + .maxIdleMs(200) + .build() ); - exportBuffer.consume(this, 1, 200); MetricsCreator metricsCreator = moduleManager.find(TelemetryModule.NAME) .provider() .getService(MetricsCreator.class); @@ -75,9 +81,8 @@ public void start() { public void export(SegmentRecord segmentRecord) { if (segmentRecord != null) { - exportBuffer.produce(segmentRecord); + queue.produce(segmentRecord); } - } @Override @@ -85,8 +90,7 @@ public boolean isEnabled() { return setting.isEnableKafkaTrace(); } - @Override - public void consume(final List data) { + private void consumeSegmentRecords(final List data) { for (SegmentRecord segmentRecord : data) { if (segmentRecord != null) { try { @@ -124,9 +128,4 @@ private boolean isError(SegmentObject segmentObject) { } return false; } - - @Override - public void onError(final List data, final Throwable t) { - - } } diff --git a/oap-server/exporter/src/test/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCExporterProviderTest.java b/oap-server/exporter/src/test/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCExporterProviderTest.java index 6378093c78ca..d2e473004c11 100644 --- a/oap-server/exporter/src/test/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCExporterProviderTest.java +++ b/oap-server/exporter/src/test/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCExporterProviderTest.java @@ -62,8 +62,7 @@ public void setUp() throws ModuleStartException { assertNotNull(config); assertNull(config.getGRPCTargetHost()); assertEquals(0, config.getGRPCTargetPort()); - assertEquals(20000, config.getBufferChannelSize()); - assertEquals(2, config.getBufferChannelNum()); + assertEquals(20000, config.getBufferSize()); //for test config.setGRPCTargetHost("localhost"); diff --git a/oap-server/exporter/src/test/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCExporterTest.java b/oap-server/exporter/src/test/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCExporterTest.java index 23346d88d645..770c55465a3e 100644 --- a/oap-server/exporter/src/test/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCExporterTest.java +++ b/oap-server/exporter/src/test/java/org/apache/skywalking/oap/server/exporter/provider/grpc/GRPCExporterTest.java @@ -23,6 +23,10 @@ import io.grpc.inprocess.InProcessChannelBuilder; import io.grpc.inprocess.InProcessServerBuilder; import io.grpc.util.MutableHandlerRegistry; +import java.util.Collections; +import java.util.LinkedList; +import java.util.List; +import java.util.UUID; import java.util.concurrent.TimeUnit; import org.apache.skywalking.oap.server.core.analysis.IDManager; import org.apache.skywalking.oap.server.core.analysis.metrics.MetricsMetaInfo; @@ -35,6 +39,7 @@ import org.apache.skywalking.oap.server.exporter.grpc.MetricExportServiceGrpc; import org.apache.skywalking.oap.server.exporter.grpc.SubscriptionMetric; import org.apache.skywalking.oap.server.exporter.provider.ExporterSetting; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueManager; import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; @@ -42,10 +47,6 @@ import org.mockito.MockedStatic; import org.mockito.Mockito; import org.powermock.reflect.Whitebox; -import java.util.Collections; -import java.util.LinkedList; -import java.util.List; -import java.util.UUID; import static org.apache.skywalking.oap.server.core.exporter.ExportEvent.EventType.INCREMENT; import static org.mockito.Mockito.when; @@ -95,6 +96,7 @@ public void setUp() throws Exception { @AfterEach public void after() { + BatchQueueManager.shutdown("EXPORTER_GRPC_METRICS"); channel.shutdown(); server.shutdown(); @@ -143,15 +145,10 @@ public void initSubscriptionList() { Assertions.assertEquals("labeled-mock-metrics", subscriptionList.get(3).getMetricName()); } - @Test - public void init() { - exporter.init(null); - } - @Test public void consume() { - exporter.consume(dataList()); - exporter.consume(Collections.emptyList()); + exporter.consumeExportData(dataList()); + exporter.consumeExportData(Collections.emptyList()); List exportMetricValues = ((MockMetricExportServiceImpl) service).exportMetricValues; Assertions.assertEquals(3, exportMetricValues.size()); Assertions.assertEquals(12, exportMetricValues.get(0).getMetricValues(0).getLongValue()); @@ -160,18 +157,6 @@ public void consume() { Assertions.assertEquals(KeyValue.newBuilder().setKey("labelName").setValue("labelValue").build(), exportMetricValues.get(2).getMetricValues(0).getLabels(0)); } - @Test - public void onError() { - Exception e = new IllegalArgumentException("something wrong"); - exporter.onError(Collections.emptyList(), e); - exporter.onError(dataList(), e); - } - - @Test - public void onExit() { - exporter.onExit(); - } - private List dataList() { List dataList = new LinkedList<>(); dataList.add(new ExportData(new MetricsMetaInfo( diff --git a/oap-server/microbench/pom.xml b/oap-server/microbench/pom.xml index e0fd871dc714..b720d7e84cdc 100644 --- a/oap-server/microbench/pom.xml +++ b/oap-server/microbench/pom.xml @@ -44,11 +44,6 @@ library-util ${project.version} - - org.apache.skywalking - library-datacarrier-queue - ${project.version} - org.openjdk.jmh diff --git a/oap-server/microbench/src/main/java/org/apache/skywalking/oap/server/microbench/library/datacarrier/LinkedArrayBenchmark.java b/oap-server/microbench/src/main/java/org/apache/skywalking/oap/server/microbench/library/datacarrier/LinkedArrayBenchmark.java deleted file mode 100644 index d95703989a5a..000000000000 --- a/oap-server/microbench/src/main/java/org/apache/skywalking/oap/server/microbench/library/datacarrier/LinkedArrayBenchmark.java +++ /dev/null @@ -1,315 +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.skywalking.oap.server.microbench.library.datacarrier; - -import org.apache.skywalking.oap.server.microbench.base.AbstractMicrobenchmark; -import org.openjdk.jmh.annotations.Benchmark; - -import java.util.ArrayList; -import java.util.LinkedList; -import java.util.List; - -/** - * ISSUE-3064 - */ -public class LinkedArrayBenchmark extends AbstractMicrobenchmark { - - @Benchmark - public void testArrayCap1000() { - ArrayList list = new ArrayList(); - for (int i = 0; i < 1000; i++) { - list.add(new SampleData()); - } - } - - @Benchmark - public void testLinkedCap1000() { - LinkedList list = new LinkedList(); - for (int i = 0; i < 1000; i++) { - list.add(new SampleData()); - } - } - - @Benchmark - public void testArrayCap40000() { - ArrayList list = new ArrayList(); - for (int i = 0; i < 40000; i++) { - list.add(new SampleData()); - } - } - - @Benchmark - public void testLinkedCap40000() { - LinkedList list = new LinkedList(); - for (int i = 0; i < 40000; i++) { - list.add(new SampleData()); - } - } - - @Benchmark - public void testArrayStart1() { - List consumerList = new ArrayList(1); - for (int pos = 0; pos < 40000; pos++) { - consumerList.add(new SampleData()); - } - } - - @Benchmark - public void testArrayStart10() { - List consumerList = new ArrayList(10); - for (int pos = 0; pos < 40000; pos++) { - consumerList.add(new SampleData()); - } - } - - @Benchmark - public void testArrayStart8000() { - List consumerList = new ArrayList(8000); - for (int pos = 0; pos < 40000; pos++) { - consumerList.add(new SampleData()); - } - } - - @Benchmark - public void testArrayStart40000() { - List consumerList = new ArrayList(40000); - for (int pos = 0; pos < 40000; pos++) { - consumerList.add(new SampleData()); - } - } - - @Benchmark - public void testReusedArray() { - List consumerList = new ArrayList(); - for (int times = 0; times < 1000; times++) { - for (int pos = 0; pos < 40000; pos++) { - consumerList.add(new SampleData()); - } - consumerList.clear(); - } - } - - @Benchmark - public void testLinked() { - for (int times = 0; times < 1000; times++) { - List consumerList = new LinkedList(); - - for (int pos = 0; pos < 40000; pos++) { - consumerList.add(new SampleData()); - } - } - } - - @Benchmark - public void testReusedLinked() { - List consumerList = new LinkedList(); - for (int times = 0; times < 1000; times++) { - - for (int pos = 0; pos < 40000; pos++) { - consumerList.add(new SampleData()); - } - consumerList.clear(); - } - } - - @Benchmark - public void testArrayList200K() { - ArrayList list = new ArrayList(4000); - for (int times = 0; times < 1000; times++) { - for (int pos = 0; pos < 200000; pos++) { - list.add(new SampleData()); - } - list.clear(); - } - } - - @Benchmark - public void testReusedLinked200K() { - LinkedList list = new LinkedList(); - for (int times = 0; times < 1000; times++) { - for (int pos = 0; pos < 200000; pos++) { - list.add(new SampleData()); - } - list.clear(); - } - } - - @Benchmark - public void testLinked200K() { - for (int times = 0; times < 1000; times++) { - LinkedList list = new LinkedList(); - for (int pos = 0; pos < 200000; pos++) { - list.add(new SampleData()); - } - } - } - - /** - * Test Data - */ - public class SampleData { - - private int intValue; - - private String name; - - } - - /* - Environment: - - # JMH version: 1.21 - # VM version: JDK 1.8.0_121, Java HotSpot(TM) 64-Bit Server VM, 25.121-b13 - # VM invoker: C:\Program Files\Java\jdk1.8.0_121\jre\bin\java.exe - # VM options: -javaagent:C:\Program Files\JetBrains\IntelliJ IDEA Community Edition 2017.2.1\lib\idea_rt.jar=51557:C:\Program Files\JetBrains\IntelliJ IDEA Community Edition 2017.2.1\bin -Dfile.encoding=UTF-8 -Xmx512m -Xms512m - # Warmup: 5 iterations, 10 s each - # Measurement: 5 iterations, 10 s each - # Timeout: 10 min per iteration - # Threads: 1 thread, will synchronize iterations - # Benchmark mode: Throughput, ops/time - - Benchmark Mode Cnt Score Error Units - LinkedArrayBenchmark.testArrayCap1000 thrpt 5 143087.182 ± 3142.078 ops/s - LinkedArrayBenchmark.testArrayCap1000:·gc.alloc.rate thrpt 5 5067.966 ± 111.247 MB/sec - LinkedArrayBenchmark.testArrayCap1000:·gc.alloc.rate.norm thrpt 5 39000.000 ± 0.001 B/op - LinkedArrayBenchmark.testArrayCap1000:·gc.churn.PS_Eden_Space thrpt 5 5067.921 ± 98.198 MB/sec - LinkedArrayBenchmark.testArrayCap1000:·gc.churn.PS_Eden_Space.norm thrpt 5 38999.800 ± 214.267 B/op - LinkedArrayBenchmark.testArrayCap1000:·gc.churn.PS_Survivor_Space thrpt 5 0.649 ± 0.200 MB/sec - LinkedArrayBenchmark.testArrayCap1000:·gc.churn.PS_Survivor_Space.norm thrpt 5 4.993 ± 1.620 B/op - LinkedArrayBenchmark.testArrayCap1000:·gc.count thrpt 5 1570.000 counts - LinkedArrayBenchmark.testArrayCap1000:·gc.time thrpt 5 701.000 ms - LinkedArrayBenchmark.testArrayCap40000 thrpt 5 3765.411 ± 194.475 ops/s - LinkedArrayBenchmark.testArrayCap40000:·gc.alloc.rate thrpt 5 5230.501 ± 270.947 MB/sec - LinkedArrayBenchmark.testArrayCap40000:·gc.alloc.rate.norm thrpt 5 1529496.011 ± 0.001 B/op - LinkedArrayBenchmark.testArrayCap40000:·gc.churn.PS_Eden_Space thrpt 5 5243.183 ± 272.428 MB/sec - LinkedArrayBenchmark.testArrayCap40000:·gc.churn.PS_Eden_Space.norm thrpt 5 1533203.926 ± 3832.510 B/op - LinkedArrayBenchmark.testArrayCap40000:·gc.churn.PS_Survivor_Space thrpt 5 6.820 ± 2.362 MB/sec - LinkedArrayBenchmark.testArrayCap40000:·gc.churn.PS_Survivor_Space.norm thrpt 5 1994.409 ± 698.911 B/op - LinkedArrayBenchmark.testArrayCap40000:·gc.count thrpt 5 1646.000 counts - LinkedArrayBenchmark.testArrayCap40000:·gc.time thrpt 5 1280.000 ms - LinkedArrayBenchmark.testArrayList200K thrpt 5 0.664 ± 0.050 ops/s - LinkedArrayBenchmark.testArrayList200K:·gc.alloc.rate thrpt 5 2903.182 ± 210.494 MB/sec - LinkedArrayBenchmark.testArrayList200K:·gc.alloc.rate.norm thrpt 5 4802736157.714 ± 0.001 B/op - LinkedArrayBenchmark.testArrayList200K:·gc.churn.PS_Eden_Space thrpt 5 2901.983 ± 222.656 MB/sec - LinkedArrayBenchmark.testArrayList200K:·gc.churn.PS_Eden_Space.norm thrpt 5 4800680520.914 ± 39561672.824 B/op - LinkedArrayBenchmark.testArrayList200K:·gc.churn.PS_Survivor_Space thrpt 5 19.788 ± 2.228 MB/sec - LinkedArrayBenchmark.testArrayList200K:·gc.churn.PS_Survivor_Space.norm thrpt 5 32731369.371 ± 1782913.951 B/op - LinkedArrayBenchmark.testArrayList200K:·gc.count thrpt 5 1012.000 counts - LinkedArrayBenchmark.testArrayList200K:·gc.time thrpt 5 9026.000 ms - LinkedArrayBenchmark.testArrayStart1 thrpt 5 3036.206 ± 146.907 ops/s - LinkedArrayBenchmark.testArrayStart1:·gc.alloc.rate thrpt 5 4004.134 ± 193.620 MB/sec - LinkedArrayBenchmark.testArrayStart1:·gc.alloc.rate.norm thrpt 5 1452104.014 ± 0.002 B/op - LinkedArrayBenchmark.testArrayStart1:·gc.churn.PS_Eden_Space thrpt 5 4010.593 ± 201.502 MB/sec - LinkedArrayBenchmark.testArrayStart1:·gc.churn.PS_Eden_Space.norm thrpt 5 1454441.827 ± 12106.958 B/op - LinkedArrayBenchmark.testArrayStart1:·gc.churn.PS_Survivor_Space thrpt 5 4.471 ± 1.039 MB/sec - LinkedArrayBenchmark.testArrayStart1:·gc.churn.PS_Survivor_Space.norm thrpt 5 1621.402 ± 380.693 B/op - LinkedArrayBenchmark.testArrayStart1:·gc.count thrpt 5 1260.000 counts - LinkedArrayBenchmark.testArrayStart1:·gc.time thrpt 5 946.000 ms - LinkedArrayBenchmark.testArrayStart10 thrpt 5 3953.451 ± 124.425 ops/s - LinkedArrayBenchmark.testArrayStart10:·gc.alloc.rate thrpt 5 5491.766 ± 172.901 MB/sec - LinkedArrayBenchmark.testArrayStart10:·gc.alloc.rate.norm thrpt 5 1529496.011 ± 0.001 B/op - LinkedArrayBenchmark.testArrayStart10:·gc.churn.PS_Eden_Space thrpt 5 5506.896 ± 179.841 MB/sec - LinkedArrayBenchmark.testArrayStart10:·gc.churn.PS_Eden_Space.norm thrpt 5 1533707.327 ± 6558.467 B/op - LinkedArrayBenchmark.testArrayStart10:·gc.churn.PS_Survivor_Space thrpt 5 7.319 ± 1.779 MB/sec - LinkedArrayBenchmark.testArrayStart10:·gc.churn.PS_Survivor_Space.norm thrpt 5 2038.423 ± 504.768 B/op - LinkedArrayBenchmark.testArrayStart10:·gc.count thrpt 5 1728.000 counts - LinkedArrayBenchmark.testArrayStart10:·gc.time thrpt 5 1350.000 ms - LinkedArrayBenchmark.testArrayStart40000 thrpt 5 3445.048 ± 38.938 ops/s - LinkedArrayBenchmark.testArrayStart40000:·gc.alloc.rate thrpt 5 3504.290 ± 39.160 MB/sec - LinkedArrayBenchmark.testArrayStart40000:·gc.alloc.rate.norm thrpt 5 1120016.013 ± 0.001 B/op - LinkedArrayBenchmark.testArrayStart40000:·gc.churn.PS_Eden_Space thrpt 5 3506.791 ± 62.456 MB/sec - LinkedArrayBenchmark.testArrayStart40000:·gc.churn.PS_Eden_Space.norm thrpt 5 1120811.902 ± 10367.121 B/op - LinkedArrayBenchmark.testArrayStart40000:·gc.churn.PS_Survivor_Space thrpt 5 4.731 ± 0.275 MB/sec - LinkedArrayBenchmark.testArrayStart40000:·gc.churn.PS_Survivor_Space.norm thrpt 5 1512.123 ± 91.484 B/op - LinkedArrayBenchmark.testArrayStart40000:·gc.count thrpt 5 1100.000 counts - LinkedArrayBenchmark.testArrayStart40000:·gc.time thrpt 5 805.000 ms - LinkedArrayBenchmark.testArrayStart8000 thrpt 5 2940.747 ± 32.257 ops/s - LinkedArrayBenchmark.testArrayStart8000:·gc.alloc.rate thrpt 5 3691.430 ± 39.870 MB/sec - LinkedArrayBenchmark.testArrayStart8000:·gc.alloc.rate.norm thrpt 5 1382080.015 ± 0.001 B/op - LinkedArrayBenchmark.testArrayStart8000:·gc.churn.PS_Eden_Space thrpt 5 3699.920 ± 46.996 MB/sec - LinkedArrayBenchmark.testArrayStart8000:·gc.churn.PS_Eden_Space.norm thrpt 5 1385258.364 ± 7458.176 B/op - LinkedArrayBenchmark.testArrayStart8000:·gc.churn.PS_Survivor_Space thrpt 5 3.228 ± 0.276 MB/sec - LinkedArrayBenchmark.testArrayStart8000:·gc.churn.PS_Survivor_Space.norm thrpt 5 1208.384 ± 102.584 B/op - LinkedArrayBenchmark.testArrayStart8000:·gc.count thrpt 5 1160.000 counts - LinkedArrayBenchmark.testArrayStart8000:·gc.time thrpt 5 776.000 ms - LinkedArrayBenchmark.testLinked thrpt 5 2.145 ± 0.023 ops/s - LinkedArrayBenchmark.testLinked:·gc.alloc.rate thrpt 5 3744.537 ± 38.773 MB/sec - LinkedArrayBenchmark.testLinked:·gc.alloc.rate.norm thrpt 5 1920000019.636 ± 0.001 B/op - LinkedArrayBenchmark.testLinked:·gc.churn.PS_Eden_Space thrpt 5 3743.961 ± 36.688 MB/sec - LinkedArrayBenchmark.testLinked:·gc.churn.PS_Eden_Space.norm thrpt 5 1919709109.527 ± 17177042.598 B/op - LinkedArrayBenchmark.testLinked:·gc.churn.PS_Survivor_Space thrpt 5 9.470 ± 0.430 MB/sec - LinkedArrayBenchmark.testLinked:·gc.churn.PS_Survivor_Space.norm thrpt 5 4855621.818 ± 264728.918 B/op - LinkedArrayBenchmark.testLinked:·gc.count thrpt 5 1217.000 counts - LinkedArrayBenchmark.testLinked:·gc.time thrpt 5 3697.000 ms - LinkedArrayBenchmark.testLinked200K thrpt 5 0.340 ± 0.013 ops/s - LinkedArrayBenchmark.testLinked200K:·gc.alloc.rate thrpt 5 2989.665 ± 108.530 MB/sec - LinkedArrayBenchmark.testLinked200K:·gc.alloc.rate.norm thrpt 5 9600000108.000 ± 0.001 B/op - LinkedArrayBenchmark.testLinked200K:·gc.churn.PS_Eden_Space thrpt 5 2990.920 ± 116.103 MB/sec - LinkedArrayBenchmark.testLinked200K:·gc.churn.PS_Eden_Space.norm thrpt 5 9603986226.400 ± 39954550.430 B/op - LinkedArrayBenchmark.testLinked200K:·gc.churn.PS_Survivor_Space thrpt 5 32.536 ± 4.681 MB/sec - LinkedArrayBenchmark.testLinked200K:·gc.churn.PS_Survivor_Space.norm thrpt 5 104493875.200 ± 16889681.984 B/op - LinkedArrayBenchmark.testLinked200K:·gc.count thrpt 5 1235.000 counts - LinkedArrayBenchmark.testLinked200K:·gc.time thrpt 5 15644.000 ms - LinkedArrayBenchmark.testLinkedCap1000 thrpt 5 84999.730 ± 1164.113 ops/s - LinkedArrayBenchmark.testLinkedCap1000:·gc.alloc.rate thrpt 5 3705.698 ± 50.753 MB/sec - LinkedArrayBenchmark.testLinkedCap1000:·gc.alloc.rate.norm thrpt 5 48000.001 ± 0.001 B/op - LinkedArrayBenchmark.testLinkedCap1000:·gc.churn.PS_Eden_Space thrpt 5 3705.991 ± 71.457 MB/sec - LinkedArrayBenchmark.testLinkedCap1000:·gc.churn.PS_Eden_Space.norm thrpt 5 48003.617 ± 320.127 B/op - LinkedArrayBenchmark.testLinkedCap1000:·gc.churn.PS_Survivor_Space thrpt 5 0.520 ± 0.154 MB/sec - LinkedArrayBenchmark.testLinkedCap1000:·gc.churn.PS_Survivor_Space.norm thrpt 5 6.739 ± 2.066 B/op - LinkedArrayBenchmark.testLinkedCap1000:·gc.count thrpt 5 1148.000 counts - LinkedArrayBenchmark.testLinkedCap1000:·gc.time thrpt 5 515.000 ms - LinkedArrayBenchmark.testLinkedCap40000 thrpt 5 2001.889 ± 58.692 ops/s - LinkedArrayBenchmark.testLinkedCap40000:·gc.alloc.rate thrpt 5 3490.899 ± 102.356 MB/sec - LinkedArrayBenchmark.testLinkedCap40000:·gc.alloc.rate.norm thrpt 5 1920000.022 ± 0.001 B/op - LinkedArrayBenchmark.testLinkedCap40000:·gc.churn.PS_Eden_Space thrpt 5 3491.448 ± 111.952 MB/sec - LinkedArrayBenchmark.testLinkedCap40000:·gc.churn.PS_Eden_Space.norm thrpt 5 1920296.231 ± 15332.688 B/op - LinkedArrayBenchmark.testLinkedCap40000:·gc.churn.PS_Survivor_Space thrpt 5 8.708 ± 0.925 MB/sec - LinkedArrayBenchmark.testLinkedCap40000:·gc.churn.PS_Survivor_Space.norm thrpt 5 4788.927 ± 381.943 B/op - LinkedArrayBenchmark.testLinkedCap40000:·gc.count thrpt 5 1108.000 counts - LinkedArrayBenchmark.testLinkedCap40000:·gc.time thrpt 5 3444.000 ms - LinkedArrayBenchmark.testReusedArray thrpt 5 3.128 ± 0.254 ops/s - LinkedArrayBenchmark.testReusedArray:·gc.alloc.rate thrpt 5 2731.835 ± 222.486 MB/sec - LinkedArrayBenchmark.testReusedArray:·gc.alloc.rate.norm thrpt 5 960569533.505 ± 1.150 B/op - LinkedArrayBenchmark.testReusedArray:·gc.churn.PS_Eden_Space thrpt 5 2730.828 ± 245.487 MB/sec - LinkedArrayBenchmark.testReusedArray:·gc.churn.PS_Eden_Space.norm thrpt 5 960179747.003 ± 8148505.430 B/op - LinkedArrayBenchmark.testReusedArray:·gc.churn.PS_Survivor_Space thrpt 5 1.864 ± 0.329 MB/sec - LinkedArrayBenchmark.testReusedArray:·gc.churn.PS_Survivor_Space.norm thrpt 5 656036.904 ± 159198.847 B/op - LinkedArrayBenchmark.testReusedArray:·gc.count thrpt 5 875.000 counts - LinkedArrayBenchmark.testReusedArray:·gc.time thrpt 5 2103.000 ms - LinkedArrayBenchmark.testReusedLinked thrpt 5 1.574 ± 0.015 ops/s - LinkedArrayBenchmark.testReusedLinked:·gc.alloc.rate thrpt 5 2746.313 ± 25.513 MB/sec - LinkedArrayBenchmark.testReusedLinked:·gc.alloc.rate.norm thrpt 5 1920000059.800 ± 4.218 B/op - LinkedArrayBenchmark.testReusedLinked:·gc.churn.PS_Eden_Space thrpt 5 2745.434 ± 25.184 MB/sec - LinkedArrayBenchmark.testReusedLinked:·gc.churn.PS_Eden_Space.norm thrpt 5 1919385600.000 ± 836969.504 B/op - LinkedArrayBenchmark.testReusedLinked:·gc.churn.PS_Survivor_Space thrpt 5 6.834 ± 0.638 MB/sec - LinkedArrayBenchmark.testReusedLinked:·gc.churn.PS_Survivor_Space.norm thrpt 5 4777984.000 ± 456748.586 B/op - LinkedArrayBenchmark.testReusedLinked:·gc.count thrpt 5 886.000 counts - LinkedArrayBenchmark.testReusedLinked:·gc.time thrpt 5 3041.000 ms - LinkedArrayBenchmark.testReusedLinked200K thrpt 5 0.253 ± 0.009 ops/s - LinkedArrayBenchmark.testReusedLinked200K:·gc.alloc.rate thrpt 5 2226.639 ± 75.414 MB/sec - LinkedArrayBenchmark.testReusedLinked200K:·gc.alloc.rate.norm thrpt 5 9600000178.133 ± 18.369 B/op - LinkedArrayBenchmark.testReusedLinked200K:·gc.churn.PS_Eden_Space thrpt 5 2225.749 ± 77.891 MB/sec - LinkedArrayBenchmark.testReusedLinked200K:·gc.churn.PS_Eden_Space.norm thrpt 5 9596148100.800 ± 50593440.713 B/op - LinkedArrayBenchmark.testReusedLinked200K:·gc.churn.PS_Survivor_Space thrpt 5 22.781 ± 4.309 MB/sec - LinkedArrayBenchmark.testReusedLinked200K:·gc.churn.PS_Survivor_Space.norm thrpt 5 98238464.000 ± 19995139.006 B/op - LinkedArrayBenchmark.testReusedLinked200K:·gc.count thrpt 5 930.000 counts - LinkedArrayBenchmark.testReusedLinked200K:·gc.time thrpt 5 12241.000 ms - */ -} diff --git a/oap-server/microbench/src/main/java/org/apache/skywalking/oap/server/microbench/library/datacarrier/common/AtomicRangeIntegerBenchmark.java b/oap-server/microbench/src/main/java/org/apache/skywalking/oap/server/microbench/library/datacarrier/common/AtomicRangeIntegerBenchmark.java deleted file mode 100644 index a08d7ed16af0..000000000000 --- a/oap-server/microbench/src/main/java/org/apache/skywalking/oap/server/microbench/library/datacarrier/common/AtomicRangeIntegerBenchmark.java +++ /dev/null @@ -1,166 +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.skywalking.oap.server.microbench.library.datacarrier.common; - -import org.apache.skywalking.oap.server.library.datacarrier.common.AtomicRangeInteger; -import org.apache.skywalking.oap.server.microbench.base.AbstractMicrobenchmark; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.openjdk.jmh.annotations.Benchmark; - -public class AtomicRangeIntegerBenchmark extends AbstractMicrobenchmark { - - private static AtomicRangeInteger ATOMIC_V3 = new AtomicRangeInteger(0, 100); - private static AtomicRangeIntegerV1 ATOMIC_V1 = new AtomicRangeIntegerV1(0, 100); - private static AtomicRangeIntegerV2 ATOMIC_V2 = new AtomicRangeIntegerV2(0, 100); - - @Test - public void testGetAndIncrement() { - AtomicRangeInteger atomicI = new AtomicRangeInteger(0, 10); - for (int i = 0; i < 10; i++) { - Assertions.assertEquals(i, atomicI.getAndIncrement()); - } - Assertions.assertEquals(0, atomicI.getAndIncrement()); - Assertions.assertEquals(1, atomicI.get()); - Assertions.assertEquals(1, atomicI.intValue()); - Assertions.assertEquals(1, atomicI.longValue()); - Assertions.assertEquals(1, (int) atomicI.floatValue()); - Assertions.assertEquals(1, (int) atomicI.doubleValue()); - } - - @Test - @Benchmark - public void testGetAndIncrementV1Performance() { - ATOMIC_V1.getAndIncrement(); - } - - @Test - @Benchmark - public void testGetAndIncrementV2Performance() { - ATOMIC_V2.getAndIncrement(); - } - - @Test - @Benchmark - public void testGetAndIncrementV3Performance() { - ATOMIC_V3.getAndIncrement(); - } - - /** - * # JMH version: 1.21 - * # VM version: JDK 1.8.0_111, Java HotSpot(TM) 64-Bit Server VM, 25.111-b14 - * # VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_111.jdk/Contents/Home/jre/bin/java - * # VM options: -XX:-RestrictContended -Dfile.encoding=UTF-8 - * # Warmup: 3 iterations, 10 s each - * # Measurement: 5 iterations, 10 s each - * # Timeout: 10 min per iteration - * # Threads: 128 threads, ***WARNING: Synchronize iterations are disabled!*** - * # Benchmark mode: Throughput, ops/time - * # Benchmark: org.apache.skywalking.apm.commons.datacarrier.common.AtomicRangeIntegerTest.testGetAndIncrementV1Performance - * - * # Run progress: 0.00% complete, ETA 00:04:00 - * # Fork: 1 of 1 - * # Warmup Iteration 1: 14087955.036 ops/s - * # Warmup Iteration 2: 15853193.651 ops/s - * # Warmup Iteration 3: 14242562.576 ops/s - * Iteration 1: 13507077.199 ops/s - * Iteration 2: 13524108.304 ops/s - * Iteration 3: 13428875.424 ops/s - * Iteration 4: 13442334.399 ops/s - * Iteration 5: 13581207.442 ops/s - * - * - * Result "org.apache.skywalking.apm.commons.datacarrier.common.AtomicRangeIntegerTest.testGetAndIncrementV1Performance": - * 13496720.554 ±(99.9%) 240134.803 ops/s [Average] - * (min, avg, max) = (13428875.424, 13496720.554, 13581207.442), stdev = 62362.246 - * CI (99.9%): [13256585.750, 13736855.357] (assumes normal distribution) - * - * - * # JMH version: 1.21 - * # VM version: JDK 1.8.0_111, Java HotSpot(TM) 64-Bit Server VM, 25.111-b14 - * # VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_111.jdk/Contents/Home/jre/bin/java - * # VM options: -XX:-RestrictContended -Dfile.encoding=UTF-8 - * # Warmup: 3 iterations, 10 s each - * # Measurement: 5 iterations, 10 s each - * # Timeout: 10 min per iteration - * # Threads: 128 threads, ***WARNING: Synchronize iterations are disabled!*** - * # Benchmark mode: Throughput, ops/time - * # Benchmark: org.apache.skywalking.apm.commons.datacarrier.common.AtomicRangeIntegerTest.testGetAndIncrementV2Performance - * - * # Run progress: 33.33% complete, ETA 00:02:52 - * # Fork: 1 of 1 - * # Warmup Iteration 1: 38963151.964 ops/s - * # Warmup Iteration 2: 38748023.773 ops/s - * # Warmup Iteration 3: 39049777.582 ops/s - * Iteration 1: 39534928.550 ops/s - * Iteration 2: 39020804.604 ops/s - * Iteration 3: 38991508.452 ops/s - * Iteration 4: 39025237.001 ops/s - * Iteration 5: 39433780.645 ops/s - * - * - * Result "org.apache.skywalking.apm.commons.datacarrier.common.AtomicRangeIntegerTest.testGetAndIncrementV2Performance": - * 39201251.850 ±(99.9%) 1005866.969 ops/s [Average] - * (min, avg, max) = (38991508.452, 39201251.850, 39534928.550), stdev = 261220.458 - * CI (99.9%): [38195384.881, 40207118.820] (assumes normal distribution) - * - * - * # JMH version: 1.21 - * # VM version: JDK 1.8.0_111, Java HotSpot(TM) 64-Bit Server VM, 25.111-b14 - * # VM invoker: /Library/Java/JavaVirtualMachines/jdk1.8.0_111.jdk/Contents/Home/jre/bin/java - * # VM options: -XX:-RestrictContended -Dfile.encoding=UTF-8 - * # Warmup: 3 iterations, 10 s each - * # Measurement: 5 iterations, 10 s each - * # Timeout: 10 min per iteration - * # Threads: 128 threads, ***WARNING: Synchronize iterations are disabled!*** - * # Benchmark mode: Throughput, ops/time - * # Benchmark: org.apache.skywalking.apm.commons.datacarrier.common.AtomicRangeIntegerTest.testGetAndIncrementV3Performance - * - * # Run progress: 66.67% complete, ETA 00:01:25 - * # Fork: 1 of 1 - * # Warmup Iteration 1: 45437159.014 ops/s - * # Warmup Iteration 2: 45253129.637 ops/s - * # Warmup Iteration 3: 45394394.135 ops/s - * Iteration 1: 45434263.958 ops/s - * Iteration 2: 45283522.683 ops/s - * Iteration 3: 47116623.190 ops/s - * Iteration 4: 46012311.703 ops/s - * Iteration 5: 45316353.774 ops/s - * - * - * Result "org.apache.skywalking.apm.commons.datacarrier.common.AtomicRangeIntegerTest.testGetAndIncrementV3Performance": - * 45832615.061 ±(99.9%) 2987464.163 ops/s [Average] - * (min, avg, max) = (45283522.683, 45832615.061, 47116623.190), stdev = 775834.956 - * CI (99.9%): [42845150.898, 48820079.225] (assumes normal distribution) - * - * - * # Run complete. Total time: 00:04:17 - * - * REMEMBER: The numbers below are just data. To gain reusable insights, you need to follow up on - * why the numbers are the way they are. Use profilers (see -prof, -lprof), design factorial - * experiments, perform baseline and negative tests that provide experimental control, make sure - * the benchmarking environment is safe on JVM/OS/HW level, ask for reviews from the domain experts. - * Do not assume the numbers tell you what you want them to tell. - * - * Benchmark Mode Cnt Score Error Units - * AtomicRangeIntegerTest.testGetAndIncrementV1Performance thrpt 5 13496720.554 ± 240134.803 ops/s - * AtomicRangeIntegerTest.testGetAndIncrementV2Performance thrpt 5 39201251.850 ± 1005866.969 ops/s - * AtomicRangeIntegerTest.testGetAndIncrementV3Performance thrpt 5 45832615.061 ± 2987464.163 ops/s - */ -} diff --git a/oap-server/microbench/src/main/java/org/apache/skywalking/oap/server/microbench/library/datacarrier/common/AtomicRangeIntegerV1.java b/oap-server/microbench/src/main/java/org/apache/skywalking/oap/server/microbench/library/datacarrier/common/AtomicRangeIntegerV1.java deleted file mode 100644 index 6a8550d69abc..000000000000 --- a/oap-server/microbench/src/main/java/org/apache/skywalking/oap/server/microbench/library/datacarrier/common/AtomicRangeIntegerV1.java +++ /dev/null @@ -1,71 +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.skywalking.oap.server.microbench.library.datacarrier.common; - -import java.io.Serializable; -import java.util.concurrent.atomic.AtomicInteger; - -/** - * This is moved from SkyWalking 6.1 - */ -public class AtomicRangeIntegerV1 extends Number implements Serializable { - - private static final long serialVersionUID = -4099792402691141643L; - private AtomicInteger value; - private int startValue; - private int endValue; - - public AtomicRangeIntegerV1(int startValue, int maxValue) { - this.value = new AtomicInteger(startValue); - this.startValue = startValue; - this.endValue = maxValue - 1; - } - - public final int getAndIncrement() { - int current; - int next; - do { - current = this.value.get(); - next = current >= this.endValue ? this.startValue : current + 1; - } - while (!this.value.compareAndSet(current, next)); - - return current; - } - - public final int get() { - return this.value.get(); - } - - public int intValue() { - return this.value.intValue(); - } - - public long longValue() { - return this.value.longValue(); - } - - public float floatValue() { - return this.value.floatValue(); - } - - public double doubleValue() { - return this.value.doubleValue(); - } -} diff --git a/oap-server/microbench/src/main/java/org/apache/skywalking/oap/server/microbench/library/datacarrier/common/AtomicRangeIntegerV2.java b/oap-server/microbench/src/main/java/org/apache/skywalking/oap/server/microbench/library/datacarrier/common/AtomicRangeIntegerV2.java deleted file mode 100644 index 725ad59b862b..000000000000 --- a/oap-server/microbench/src/main/java/org/apache/skywalking/oap/server/microbench/library/datacarrier/common/AtomicRangeIntegerV2.java +++ /dev/null @@ -1,71 +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.skywalking.oap.server.microbench.library.datacarrier.common; - -import java.io.Serializable; -import java.util.concurrent.atomic.AtomicInteger; - -/** - * This comes from PR#2874 - */ -public class AtomicRangeIntegerV2 extends Number implements Serializable { - private static final long serialVersionUID = -4099792402691141643L; - private AtomicInteger value; - private int startValue; - private int endValue; - - public AtomicRangeIntegerV2(int startValue, int maxValue) { - this.value = new AtomicInteger(startValue); - this.startValue = startValue; - this.endValue = maxValue - 1; - } - - public final int getAndIncrement() { - int next; - do { - next = this.value.incrementAndGet(); - if (next > endValue && this.value.compareAndSet(next, startValue)) { - return endValue; - } - } - while (next > endValue); - - return next - 1; - } - - public final int get() { - return this.value.get(); - } - - public int intValue() { - return this.value.intValue(); - } - - public long longValue() { - return this.value.longValue(); - } - - public float floatValue() { - return this.value.floatValue(); - } - - public double doubleValue() { - return this.value.doubleValue(); - } -} diff --git a/oap-server/server-core/pom.xml b/oap-server/server-core/pom.xml index f6d233587ad2..056f3c9fa3d0 100644 --- a/oap-server/server-core/pom.xml +++ b/oap-server/server-core/pom.xml @@ -76,7 +76,7 @@ org.apache.skywalking - library-datacarrier-queue + library-batch-queue ${project.version} diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsAggregateMALWorker.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsAggregateMALWorker.java deleted file mode 100644 index 6ce0bb447d3b..000000000000 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsAggregateMALWorker.java +++ /dev/null @@ -1,65 +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.skywalking.oap.server.core.analysis.worker; - -import lombok.extern.slf4j.Slf4j; -import org.apache.skywalking.oap.server.core.analysis.metrics.Metrics; -import org.apache.skywalking.oap.server.core.worker.AbstractWorker; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.BulkConsumePool; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.ConsumerPoolFactory; -import org.apache.skywalking.oap.server.library.module.ModuleDefineHolder; - -/** - * MetricsAggregateMALWorker provides an in-memory metrics merging capability for MAL - */ -@Slf4j -public class MetricsAggregateMALWorker extends MetricsAggregateWorker { - private final static String POOL_NAME = "METRICS_L1_AGGREGATION_MAL"; - private final BulkConsumePool pool; - - MetricsAggregateMALWorker(ModuleDefineHolder moduleDefineHolder, - AbstractWorker nextWorker, - String modelName, - long l1FlushPeriod, - MetricStreamKind kind) { - super( - moduleDefineHolder, nextWorker, modelName, l1FlushPeriod, kind, - POOL_NAME, - calculatePoolSize(), - true, - 1, - 1_000 - ); - this.pool = (BulkConsumePool) ConsumerPoolFactory.INSTANCE.get(POOL_NAME); - } - - /** - * MetricsAggregateWorker#in operation does include enqueue only - */ - @Override - public final void in(Metrics metrics) { - super.in(metrics); - pool.notifyConsumers(); - } - - private static int calculatePoolSize() { - int size = BulkConsumePool.Creator.recommendMaxSize() / 8; - return size == 0 ? 1 : size; - } -} \ No newline at end of file diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsAggregateOALWorker.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsAggregateOALWorker.java deleted file mode 100644 index 833b24419a6f..000000000000 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsAggregateOALWorker.java +++ /dev/null @@ -1,48 +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.skywalking.oap.server.core.analysis.worker; - -import lombok.extern.slf4j.Slf4j; -import org.apache.skywalking.oap.server.core.analysis.metrics.Metrics; -import org.apache.skywalking.oap.server.core.worker.AbstractWorker; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.BulkConsumePool; -import org.apache.skywalking.oap.server.library.module.ModuleDefineHolder; - -/** - * MetricsAggregateOALWorker provides an in-memory metrics merging capability for OAL - */ -@Slf4j -public class MetricsAggregateOALWorker extends MetricsAggregateWorker { - private final static String POOL_NAME = "METRICS_L1_AGGREGATION_OAL"; - - MetricsAggregateOALWorker(ModuleDefineHolder moduleDefineHolder, - AbstractWorker nextWorker, - String modelName, - long l1FlushPeriod, - MetricStreamKind kind) { - super( - moduleDefineHolder, nextWorker, modelName, l1FlushPeriod, kind, - POOL_NAME, - (int) Math.ceil(BulkConsumePool.Creator.recommendMaxSize() * 1.5), - false, - 2, - 10_000 - ); - } -} \ No newline at end of file diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsAggregateWorker.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsAggregateWorker.java index b0a8bffa3430..729834cf6be0 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsAggregateWorker.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsAggregateWorker.java @@ -18,19 +18,21 @@ package org.apache.skywalking.oap.server.core.analysis.worker; -import java.util.Arrays; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; import lombok.extern.slf4j.Slf4j; -import org.apache.skywalking.oap.server.core.UnexpectedException; import org.apache.skywalking.oap.server.core.analysis.data.MergableBufferedData; import org.apache.skywalking.oap.server.core.analysis.metrics.Metrics; import org.apache.skywalking.oap.server.core.worker.AbstractWorker; -import org.apache.skywalking.oap.server.library.datacarrier.DataCarrier; -import org.apache.skywalking.oap.server.library.datacarrier.buffer.BufferStrategy; -import org.apache.skywalking.oap.server.library.datacarrier.buffer.QueueBuffer; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.BulkConsumePool; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.ConsumerPoolFactory; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.IConsumer; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueue; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueConfig; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueManager; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueStats; +import org.apache.skywalking.oap.server.library.batchqueue.BufferStrategy; +import org.apache.skywalking.oap.server.library.batchqueue.HandlerConsumer; +import org.apache.skywalking.oap.server.library.batchqueue.PartitionPolicy; +import org.apache.skywalking.oap.server.library.batchqueue.ThreadPolicy; import org.apache.skywalking.oap.server.library.module.ModuleDefineHolder; import org.apache.skywalking.oap.server.telemetry.TelemetryModule; import org.apache.skywalking.oap.server.telemetry.api.CounterMetrics; @@ -43,46 +45,50 @@ * it merges the data just after the receiver analysis. The metrics belonging to the same entity, metrics type and time * bucket, the L1 aggregation will merge them into one metrics object to reduce the unnecessary memory and network * payload. + * + *

All metric types (OAL and MAL) share a single {@link BatchQueue} with adaptive partitioning. + * The {@code typeHash()} partition selector ensures same metric class lands on the same partition, + * so each handler's {@link MergableBufferedData} is only accessed by one drain thread. */ @Slf4j -public abstract class MetricsAggregateWorker extends AbstractWorker { - public final long l1FlushPeriod; - private AbstractWorker nextWorker; - private final DataCarrier dataCarrier; +public class MetricsAggregateWorker extends AbstractWorker { + private static final String L1_QUEUE_NAME = "METRICS_L1_AGGREGATION"; + private static final BatchQueueConfig L1_QUEUE_CONFIG = + BatchQueueConfig.builder() + .threads(ThreadPolicy.cpuCores(1.0)) + .partitions(PartitionPolicy.adaptive()) + .bufferSize(20_000) + .strategy(BufferStrategy.IF_POSSIBLE) + .minIdleMs(1) + .maxIdleMs(50) + .build(); + + private static final int TOP_N = 10; + /** slot label -> gauge instance. Keys: "total", "top1" .. "top10". */ + private static Map QUEUE_USAGE_GAUGE; + + private final BatchQueue l1Queue; + private final long l1FlushPeriod; + private final AbstractWorker nextWorker; private final MergableBufferedData mergeDataCache; - private CounterMetrics abandonCounter; - private CounterMetrics aggregationCounter; - private GaugeMetrics queuePercentageGauge; + private final CounterMetrics abandonCounter; + private final CounterMetrics aggregationCounter; private long lastSendTime = 0; - private final int queueTotalSize; - MetricsAggregateWorker(ModuleDefineHolder moduleDefineHolder, - AbstractWorker nextWorker, - String modelName, - long l1FlushPeriod, - MetricStreamKind kind, - String poolName, - int poolSize, - boolean isSignalDrivenMode, - int queueChannelSize, - int queueBufferSize - ) { + MetricsAggregateWorker(final ModuleDefineHolder moduleDefineHolder, + final AbstractWorker nextWorker, + final String modelName, + final long l1FlushPeriod, + final Class metricsClass) { super(moduleDefineHolder); this.nextWorker = nextWorker; - this.mergeDataCache = new MergableBufferedData(); - BulkConsumePool.Creator creator = new BulkConsumePool.Creator(poolName, poolSize, 200, isSignalDrivenMode); - this.dataCarrier = new DataCarrier<>( - "MetricsAggregateWorker." + modelName, poolName, queueChannelSize, queueBufferSize, BufferStrategy.IF_POSSIBLE); - try { - ConsumerPoolFactory.INSTANCE.createIfAbsent(poolName, creator); - } catch (Exception e) { - throw new UnexpectedException(e.getMessage(), e); - } - this.dataCarrier.consume(ConsumerPoolFactory.INSTANCE.get(poolName), new AggregatorConsumer()); + this.mergeDataCache = new MergableBufferedData<>(); + this.l1FlushPeriod = l1FlushPeriod; + this.l1Queue = BatchQueueManager.create(L1_QUEUE_NAME, L1_QUEUE_CONFIG); - MetricsCreator metricsCreator = moduleDefineHolder.find(TelemetryModule.NAME) - .provider() - .getService(MetricsCreator.class); + final MetricsCreator metricsCreator = moduleDefineHolder.find(TelemetryModule.NAME) + .provider() + .getService(MetricsCreator.class); abandonCounter = metricsCreator.createCounter( "metrics_aggregator_abandon", "The abandon number of rows received in aggregation.", new MetricsTag.Keys("metricName", "level", "dimensionality"), @@ -93,69 +99,79 @@ public abstract class MetricsAggregateWorker extends AbstractWorker { new MetricsTag.Keys("metricName", "level", "dimensionality"), new MetricsTag.Values(modelName, "1", "minute") ); - queuePercentageGauge = metricsCreator.createGauge( - "metrics_aggregation_queue_used_percentage", "The percentage of queue used in aggregation.", - new MetricsTag.Keys("metricName", "level", "kind"), - new MetricsTag.Values(modelName, "1", kind.name()) - ); - this.l1FlushPeriod = l1FlushPeriod; - queueTotalSize = Arrays.stream(dataCarrier.getChannels().getBufferChannels()) - .mapToInt(QueueBuffer::getBufferSize) - .sum(); + + if (QUEUE_USAGE_GAUGE == null) { + final Map gauge = new LinkedHashMap<>(); + gauge.put("total", metricsCreator.createGauge( + "metrics_aggregation_queue_used_percentage", + "The percentage of queue used in L1 aggregation.", + new MetricsTag.Keys("level", "slot"), + new MetricsTag.Values("1", "total") + )); + for (int i = 1; i <= TOP_N; i++) { + gauge.put("top" + i, metricsCreator.createGauge( + "metrics_aggregation_queue_used_percentage", + "The percentage of queue used in L1 aggregation.", + new MetricsTag.Keys("level", "slot"), + new MetricsTag.Values("1", "top" + i) + )); + } + QUEUE_USAGE_GAUGE = gauge; + } + + l1Queue.addHandler(metricsClass, new L1Handler()); } - /** - * MetricsAggregateWorker#in operation does include enqueue only - */ @Override - public void in(Metrics metrics) { - if (!dataCarrier.produce(metrics)) { + public void in(final Metrics metrics) { + if (!l1Queue.produce(metrics)) { abandonCounter.inc(); } } - /** - * Dequeue consuming. According to {@link IConsumer#consume(List)}, this is a serial operation for every work - * instance. - * - * @param metricsList from the queue. - */ - private void onWork(List metricsList) { - metricsList.forEach(metrics -> { + private void onWork(final List metricsList) { + for (final Metrics metrics : metricsList) { aggregationCounter.inc(); mergeDataCache.accept(metrics); - }); - + } + updateQueueUsageGauges(); flush(); } + private void updateQueueUsageGauges() { + final Map gauge = QUEUE_USAGE_GAUGE; + if (gauge == null) { + return; + } + final BatchQueueStats stats = l1Queue.stats(); + gauge.get("total").setValue(stats.totalUsedPercentage()); + final List topPartitions = stats.topN(TOP_N); + for (int i = 1; i <= TOP_N; i++) { + if (i <= topPartitions.size()) { + gauge.get("top" + i).setValue(topPartitions.get(i - 1).getUsedPercentage()); + } else { + gauge.get("top" + i).setValue(0); + } + } + } + private void flush() { - long currentTime = System.currentTimeMillis(); + final long currentTime = System.currentTimeMillis(); if (currentTime - lastSendTime > l1FlushPeriod) { - mergeDataCache.read().forEach( - data -> { - nextWorker.in(data); - } - ); + mergeDataCache.read().forEach(nextWorker::in); lastSendTime = currentTime; } } - protected class AggregatorConsumer implements IConsumer { - @Override - public void consume(List data) { - queuePercentageGauge.setValue(Math.round(100 * (double) data.size() / queueTotalSize)); - MetricsAggregateWorker.this.onWork(data); - } - + private class L1Handler implements HandlerConsumer { @Override - public void onError(List data, Throwable t) { - log.error(t.getMessage(), t); + public void consume(final List data) { + onWork(data); } @Override - public void nothingToConsume() { + public void onIdle() { flush(); } } -} \ No newline at end of file +} diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsPersistentMinMALWorker.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsPersistentMinMALWorker.java deleted file mode 100644 index 6ced82869427..000000000000 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsPersistentMinMALWorker.java +++ /dev/null @@ -1,62 +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.skywalking.oap.server.core.analysis.worker; - -import lombok.extern.slf4j.Slf4j; -import org.apache.skywalking.oap.server.core.analysis.metrics.Metrics; -import org.apache.skywalking.oap.server.core.exporter.ExportEvent; -import org.apache.skywalking.oap.server.core.storage.IMetricsDAO; -import org.apache.skywalking.oap.server.core.storage.model.Model; -import org.apache.skywalking.oap.server.core.worker.AbstractWorker; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.BulkConsumePool; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.ConsumerPoolFactory; -import org.apache.skywalking.oap.server.library.module.ModuleDefineHolder; - -@Slf4j -public class MetricsPersistentMinMALWorker extends MetricsPersistentMinWorker { - private final static String POOL_NAME = "METRICS_L2_AGGREGATION_MAL"; - private final BulkConsumePool pool; - - MetricsPersistentMinMALWorker(ModuleDefineHolder moduleDefineHolder, Model model, IMetricsDAO metricsDAO, - AbstractWorker nextAlarmWorker, AbstractWorker nextExportWorker, - MetricsTransWorker transWorker, boolean supportUpdate, - long storageSessionTimeout, int metricsDataTTL, MetricStreamKind kind) { - super( - moduleDefineHolder, model, metricsDAO, nextAlarmWorker, nextExportWorker, transWorker, supportUpdate, - storageSessionTimeout, metricsDataTTL, kind, - POOL_NAME, - calculatePoolSize(), - true, - 1, - 1000 - ); - this.pool = (BulkConsumePool) ConsumerPoolFactory.INSTANCE.get(POOL_NAME); - } - - @Override - public void in(Metrics metrics) { - super.in(metrics); - pool.notifyConsumers(); - } - - private static int calculatePoolSize() { - int size = BulkConsumePool.Creator.recommendMaxSize() / 16; - return size == 0 ? 1 : size; - } -} diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsPersistentMinOALWorker.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsPersistentMinOALWorker.java deleted file mode 100644 index 534b50f9f51e..000000000000 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsPersistentMinOALWorker.java +++ /dev/null @@ -1,52 +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.skywalking.oap.server.core.analysis.worker; - -import lombok.extern.slf4j.Slf4j; -import org.apache.skywalking.oap.server.core.analysis.metrics.Metrics; -import org.apache.skywalking.oap.server.core.exporter.ExportEvent; -import org.apache.skywalking.oap.server.core.storage.IMetricsDAO; -import org.apache.skywalking.oap.server.core.storage.model.Model; -import org.apache.skywalking.oap.server.core.worker.AbstractWorker; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.BulkConsumePool; -import org.apache.skywalking.oap.server.library.module.ModuleDefineHolder; - -@Slf4j -public class MetricsPersistentMinOALWorker extends MetricsPersistentMinWorker { - - MetricsPersistentMinOALWorker(ModuleDefineHolder moduleDefineHolder, Model model, IMetricsDAO metricsDAO, - AbstractWorker nextAlarmWorker, AbstractWorker nextExportWorker, - MetricsTransWorker transWorker, boolean supportUpdate, - long storageSessionTimeout, int metricsDataTTL, MetricStreamKind kind) { - super( - moduleDefineHolder, model, metricsDAO, nextAlarmWorker, nextExportWorker, transWorker, supportUpdate, - storageSessionTimeout, metricsDataTTL, kind, - "METRICS_L2_AGGREGATION_OAL", - calculatePoolSize(), - false, - 1, - 2000 - ); - } - - private static int calculatePoolSize() { - int size = BulkConsumePool.Creator.recommendMaxSize() / 8; - return size == 0 ? 1 : size; - } -} diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsPersistentMinWorker.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsPersistentMinWorker.java index 2e2f66704543..7f7e9000f5a3 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsPersistentMinWorker.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsPersistentMinWorker.java @@ -18,22 +18,25 @@ package org.apache.skywalking.oap.server.core.analysis.worker; -import java.util.Arrays; +import java.util.LinkedHashMap; import java.util.List; +import java.util.Map; import lombok.extern.slf4j.Slf4j; import org.apache.skywalking.oap.server.core.CoreModule; -import org.apache.skywalking.oap.server.core.UnexpectedException; import org.apache.skywalking.oap.server.core.analysis.metrics.Metrics; import org.apache.skywalking.oap.server.core.exporter.ExportEvent; import org.apache.skywalking.oap.server.core.status.ServerStatusService; import org.apache.skywalking.oap.server.core.storage.IMetricsDAO; import org.apache.skywalking.oap.server.core.storage.model.Model; import org.apache.skywalking.oap.server.core.worker.AbstractWorker; -import org.apache.skywalking.oap.server.library.datacarrier.DataCarrier; -import org.apache.skywalking.oap.server.library.datacarrier.buffer.QueueBuffer; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.BulkConsumePool; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.ConsumerPoolFactory; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.IConsumer; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueue; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueConfig; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueManager; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueStats; +import org.apache.skywalking.oap.server.library.batchqueue.BufferStrategy; +import org.apache.skywalking.oap.server.library.batchqueue.HandlerConsumer; +import org.apache.skywalking.oap.server.library.batchqueue.PartitionPolicy; +import org.apache.skywalking.oap.server.library.batchqueue.ThreadPolicy; import org.apache.skywalking.oap.server.library.module.ModuleDefineHolder; import org.apache.skywalking.oap.server.telemetry.TelemetryModule; import org.apache.skywalking.oap.server.telemetry.api.GaugeMetrics; @@ -41,16 +44,32 @@ import org.apache.skywalking.oap.server.telemetry.api.MetricsTag; /** - * MetricsPersistentMinWorker is an extension of {@link MetricsPersistentWorker} and focuses on the Minute Metrics data persistent. + * MetricsPersistentMinWorker is an extension of {@link MetricsPersistentWorker} and focuses on the + * Minute Metrics data persistent. + * + *

All metric types (OAL and MAL) share a single {@link BatchQueue} with adaptive partitioning. + * The {@code typeHash()} partition selector ensures same metric class lands on the same partition, + * so each handler's {@link org.apache.skywalking.oap.server.core.analysis.data.ReadWriteSafeCache} + * is only accessed by one drain thread. */ @Slf4j -public abstract class MetricsPersistentMinWorker extends MetricsPersistentWorker { - private final DataCarrier dataCarrier; +public class MetricsPersistentMinWorker extends MetricsPersistentWorker { + private static final String L2_QUEUE_NAME = "METRICS_L2_PERSISTENCE"; + private static final BatchQueueConfig L2_QUEUE_CONFIG = + BatchQueueConfig.builder() + .threads(ThreadPolicy.cpuCoresWithBase(1, 0.25)) + .partitions(PartitionPolicy.adaptive()) + .bufferSize(2_000) + .strategy(BufferStrategy.BLOCKING) + .minIdleMs(1) + .maxIdleMs(50) + .build(); - /** - * The percentage of queue used in aggregation - */ - private final GaugeMetrics queuePercentageGauge; + private static final int TOP_N = 10; + /** slot label -> gauge instance. Keys: "total", "top1" .. "top10". */ + private static Map QUEUE_USAGE_GAUGE; + + private final BatchQueue l2Queue; /** * @since 9.4.0 @@ -59,41 +78,45 @@ public abstract class MetricsPersistentMinWorker extends MetricsPersistentWorker // Not going to expose this as a configuration, only for testing purpose private final boolean isTestingTTL = "true".equalsIgnoreCase(System.getenv("TESTING_TTL")); - private final int queueTotalSize; MetricsPersistentMinWorker(ModuleDefineHolder moduleDefineHolder, Model model, IMetricsDAO metricsDAO, AbstractWorker nextAlarmWorker, AbstractWorker nextExportWorker, MetricsTransWorker transWorker, boolean supportUpdate, long storageSessionTimeout, int metricsDataTTL, MetricStreamKind kind, - String poolName, int poolSize, boolean isSignalDrivenMode, - int queueChannelSize, int queueBufferSize) { + Class metricsClass) { super( moduleDefineHolder, model, metricsDAO, nextAlarmWorker, nextExportWorker, transWorker, supportUpdate, storageSessionTimeout, metricsDataTTL, kind ); - BulkConsumePool.Creator creator = new BulkConsumePool.Creator(poolName, poolSize, 200, isSignalDrivenMode); - try { - ConsumerPoolFactory.INSTANCE.createIfAbsent(poolName, creator); - } catch (Exception e) { - throw new UnexpectedException(e.getMessage(), e); - } - this.dataCarrier = new DataCarrier<>("MetricsPersistentWorker." + model.getName(), poolName, queueChannelSize, queueBufferSize); - this.dataCarrier.consume(ConsumerPoolFactory.INSTANCE.get(poolName), new PersistentConsumer()); + this.l2Queue = BatchQueueManager.create(L2_QUEUE_NAME, L2_QUEUE_CONFIG); - MetricsCreator metricsCreator = moduleDefineHolder.find(TelemetryModule.NAME) - .provider() - .getService(MetricsCreator.class); - queuePercentageGauge = metricsCreator.createGauge( - "metrics_aggregation_queue_used_percentage", "The percentage of queue used in aggregation.", - new MetricsTag.Keys("metricName", "level", "kind"), - new MetricsTag.Values(model.getName(), "2", kind.name()) - ); serverStatusService = moduleDefineHolder.find(CoreModule.NAME).provider().getService(ServerStatusService.class); serverStatusService.registerWatcher(this); - queueTotalSize = Arrays.stream(dataCarrier.getChannels().getBufferChannels()) - .mapToInt(QueueBuffer::getBufferSize) - .sum(); + + if (QUEUE_USAGE_GAUGE == null) { + final MetricsCreator metricsCreator = moduleDefineHolder.find(TelemetryModule.NAME) + .provider() + .getService(MetricsCreator.class); + final Map gauge = new LinkedHashMap<>(); + gauge.put("total", metricsCreator.createGauge( + "metrics_aggregation_queue_used_percentage", + "The percentage of queue used in L2 persistence.", + new MetricsTag.Keys("level", "slot"), + new MetricsTag.Values("2", "total") + )); + for (int i = 1; i <= TOP_N; i++) { + gauge.put("top" + i, metricsCreator.createGauge( + "metrics_aggregation_queue_used_percentage", + "The percentage of queue used in L2 persistence.", + new MetricsTag.Keys("level", "slot"), + new MetricsTag.Values("2", "top" + i) + )); + } + QUEUE_USAGE_GAUGE = gauge; + } + + l2Queue.addHandler(metricsClass, new L2Handler()); } /** @@ -107,24 +130,31 @@ public void in(Metrics metrics) { return; } getAggregationCounter().inc(); - dataCarrier.produce(metrics); + l2Queue.produce(metrics); } - /** - * Metrics queue processor, merge the received metrics if existing one with same ID(s) and time bucket. - * - * ID is declared through {@link Object#hashCode()} and {@link Object#equals(Object)} as usual. - */ - private class PersistentConsumer implements IConsumer { - @Override - public void consume(List data) { - queuePercentageGauge.setValue(Math.round(100 * (double) data.size() / queueTotalSize)); - MetricsPersistentMinWorker.this.onWork(data); + private void updateQueueUsageGauges() { + final Map gauge = QUEUE_USAGE_GAUGE; + if (gauge == null) { + return; + } + final BatchQueueStats stats = l2Queue.stats(); + gauge.get("total").setValue(stats.totalUsedPercentage()); + final List topPartitions = stats.topN(TOP_N); + for (int i = 1; i <= TOP_N; i++) { + if (i <= topPartitions.size()) { + gauge.get("top" + i).setValue(topPartitions.get(i - 1).getUsedPercentage()); + } else { + gauge.get("top" + i).setValue(0); + } } + } + private class L2Handler implements HandlerConsumer { @Override - public void onError(List data, Throwable t) { - log.error(t.getMessage(), t); + public void consume(List data) { + updateQueueUsageGauges(); + onWork(data); } } } diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsStreamProcessor.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsStreamProcessor.java index fe70ebdd7f7c..c405bb5b3714 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsStreamProcessor.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsStreamProcessor.java @@ -185,7 +185,7 @@ private void create(ModuleDefineHolder moduleDefineHolder, metricsClass, stream.getScopeId(), new Storage(stream.getName(), timeRelativeID, DownSampling.Minute) ); MetricsPersistentWorker minutePersistentWorker = minutePersistentWorker( - moduleDefineHolder, metricsDAO, model, transWorker, supportUpdate, kind); + moduleDefineHolder, metricsDAO, model, transWorker, supportUpdate, kind, metricsClass); String remoteReceiverWorkerName = stream.getName() + "_rec"; IWorkerInstanceSetter workerInstanceSetter = moduleDefineHolder.find(CoreModule.NAME) @@ -194,19 +194,8 @@ private void create(ModuleDefineHolder moduleDefineHolder, workerInstanceSetter.put(remoteReceiverWorkerName, minutePersistentWorker, kind, metricsClass); MetricsRemoteWorker remoteWorker = new MetricsRemoteWorker(moduleDefineHolder, remoteReceiverWorkerName); - MetricsAggregateWorker aggregateWorker; - switch (kind) { - case OAL: - aggregateWorker = new MetricsAggregateOALWorker( - moduleDefineHolder, remoteWorker, stream.getName(), l1FlushPeriod, kind); - break; - case MAL: - aggregateWorker = new MetricsAggregateMALWorker( - moduleDefineHolder, remoteWorker, stream.getName(), l1FlushPeriod, kind); - break; - default: - throw new IllegalArgumentException("Unsupported MetricStreamKind: " + kind); - } + MetricsAggregateWorker aggregateWorker = new MetricsAggregateWorker( + moduleDefineHolder, remoteWorker, stream.getName(), l1FlushPeriod, metricsClass); entryWorkers.put(metricsClass, aggregateWorker); } @@ -215,27 +204,15 @@ private MetricsPersistentWorker minutePersistentWorker(ModuleDefineHolder module Model model, MetricsTransWorker transWorker, boolean supportUpdate, - MetricStreamKind kind) { + MetricStreamKind kind, + Class metricsClass) { AlarmNotifyWorker alarmNotifyWorker = new AlarmNotifyWorker(moduleDefineHolder); ExportMetricsWorker exportWorker = new ExportMetricsWorker(moduleDefineHolder); - MetricsPersistentWorker minutePersistentWorker; - switch (kind) { - case OAL: - minutePersistentWorker = new MetricsPersistentMinOALWorker( - moduleDefineHolder, model, metricsDAO, alarmNotifyWorker, exportWorker, transWorker, - supportUpdate, storageSessionTimeout, metricsDataTTL, kind - ); - break; - case MAL: - minutePersistentWorker = new MetricsPersistentMinMALWorker( - moduleDefineHolder, model, metricsDAO, alarmNotifyWorker, exportWorker, transWorker, - supportUpdate, storageSessionTimeout, metricsDataTTL, kind - ); - break; - default: - throw new IllegalArgumentException("Unsupported MetricStreamKind: " + kind); - } + MetricsPersistentMinWorker minutePersistentWorker = new MetricsPersistentMinWorker( + moduleDefineHolder, model, metricsDAO, alarmNotifyWorker, exportWorker, transWorker, + supportUpdate, storageSessionTimeout, metricsDataTTL, kind, metricsClass + ); persistentWorkers.add(minutePersistentWorker); return minutePersistentWorker; diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/TopNStreamProcessor.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/TopNStreamProcessor.java index 1b4a84e9e2af..043f975a5185 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/TopNStreamProcessor.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/TopNStreamProcessor.java @@ -95,7 +95,7 @@ public void create(ModuleDefineHolder moduleDefineHolder, topNClass, stream.scopeId(), new Storage(stream.name(), false, DownSampling.Second)); TopNWorker persistentWorker = new TopNWorker( - moduleDefineHolder, model, topSize, topNWorkerReportCycle * 60 * 1000L, recordDAO); + moduleDefineHolder, model, topSize, topNWorkerReportCycle * 60 * 1000L, recordDAO, topNClass); persistentWorkers.add(persistentWorker); workers.put(topNClass, persistentWorker); } diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/TopNWorker.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/TopNWorker.java index 4c80ef996109..06d4a5acc298 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/TopNWorker.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/TopNWorker.java @@ -27,35 +27,56 @@ import org.apache.skywalking.oap.server.core.analysis.topn.TopN; import org.apache.skywalking.oap.server.core.storage.IRecordDAO; import org.apache.skywalking.oap.server.core.storage.model.Model; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueue; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueConfig; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueManager; +import org.apache.skywalking.oap.server.library.batchqueue.BufferStrategy; +import org.apache.skywalking.oap.server.library.batchqueue.HandlerConsumer; +import org.apache.skywalking.oap.server.library.batchqueue.PartitionPolicy; +import org.apache.skywalking.oap.server.library.batchqueue.ThreadPolicy; import org.apache.skywalking.oap.server.library.client.request.PrepareRequest; -import org.apache.skywalking.oap.server.library.datacarrier.DataCarrier; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.IConsumer; import org.apache.skywalking.oap.server.library.module.ModuleDefineHolder; /** * Top N worker is a persistence worker. Cache and order the data, flush in longer period. + * + *

All TopN types share a single {@link BatchQueue} with fixed threads. + * The {@code typeHash()} partition selector ensures same TopN class lands on the same partition, + * so each worker's {@link LimitedSizeBufferedData} is only accessed by one drain thread. */ @Slf4j public class TopNWorker extends PersistenceWorker { + private static final String TOPN_QUEUE_NAME = "TOPN_PERSISTENCE"; + private static final BatchQueueConfig TOPN_QUEUE_CONFIG = + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .partitions(PartitionPolicy.adaptive()) + .bufferSize(1_000) + .strategy(BufferStrategy.BLOCKING) + .minIdleMs(10) + .maxIdleMs(100) + .build(); + private final IRecordDAO recordDAO; private final Model model; - private final DataCarrier dataCarrier; + private final BatchQueue topNQueue; private long reportPeriod; private volatile long lastReportTimestamp; TopNWorker(ModuleDefineHolder moduleDefineHolder, Model model, int topNSize, long reportPeriod, - IRecordDAO recordDAO) { + IRecordDAO recordDAO, Class topNClass) { super( moduleDefineHolder, new ReadWriteSafeCache<>(new LimitedSizeBufferedData<>(topNSize), new LimitedSizeBufferedData<>(topNSize)) ); this.recordDAO = recordDAO; this.model = model; - this.dataCarrier = new DataCarrier<>("TopNWorker", 1, 1000); - this.dataCarrier.consume(new TopNWorker.TopNConsumer(), 1); + this.topNQueue = BatchQueueManager.create(TOPN_QUEUE_NAME, TOPN_QUEUE_CONFIG); this.lastReportTimestamp = System.currentTimeMillis(); // Top N persistent works per 10 minutes default. this.reportPeriod = reportPeriod; + + topNQueue.addHandler(topNClass, new TopNHandler()); } /** @@ -92,18 +113,13 @@ public void endOfRound() { @Override public void in(TopN n) { - dataCarrier.produce(n); + topNQueue.produce(n); } - private class TopNConsumer implements IConsumer { + private class TopNHandler implements HandlerConsumer { @Override public void consume(List data) { - TopNWorker.this.onWork(data); - } - - @Override - public void onError(List data, Throwable t) { - log.error(t.getMessage(), t); + onWork(data); } } } diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/exporter/ExporterService.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/exporter/ExporterService.java index a322dae22a0f..edf36dc0f6a1 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/exporter/ExporterService.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/exporter/ExporterService.java @@ -22,8 +22,6 @@ public interface ExporterService extends Service { - void start(); - void export(T data); boolean isEnabled(); diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/remote/client/GRPCRemoteClient.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/remote/client/GRPCRemoteClient.java index 3cbc17f16d41..32ca21110760 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/remote/client/GRPCRemoteClient.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/remote/client/GRPCRemoteClient.java @@ -31,9 +31,13 @@ import org.apache.skywalking.oap.server.core.remote.grpc.proto.Empty; import org.apache.skywalking.oap.server.core.remote.grpc.proto.RemoteMessage; import org.apache.skywalking.oap.server.core.remote.grpc.proto.RemoteServiceGrpc; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueue; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueConfig; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueManager; +import org.apache.skywalking.oap.server.library.batchqueue.BufferStrategy; +import org.apache.skywalking.oap.server.library.batchqueue.HandlerConsumer; +import org.apache.skywalking.oap.server.library.batchqueue.ThreadPolicy; import org.apache.skywalking.oap.server.library.client.grpc.GRPCClient; -import org.apache.skywalking.oap.server.library.datacarrier.DataCarrier; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.IConsumer; import org.apache.skywalking.oap.server.library.module.ModuleDefineHolder; import org.apache.skywalking.oap.server.telemetry.TelemetryModule; import org.apache.skywalking.oap.server.telemetry.api.CounterMetrics; @@ -46,13 +50,13 @@ */ @Slf4j public class GRPCRemoteClient implements RemoteClient { - private final int channelSize; private final int bufferSize; private final Address address; private final AtomicInteger concurrentStreamObserverNumber = new AtomicInteger(0); private SslContext sslContext; private GRPCClient client; - private DataCarrier carrier; + private BatchQueue queue; + private String queueName; private boolean isConnect; private CounterMetrics remoteOutCounter; private CounterMetrics remoteOutErrorCounter; @@ -61,13 +65,11 @@ public class GRPCRemoteClient implements RemoteClient { public GRPCRemoteClient(final ModuleDefineHolder moduleDefineHolder, final Address address, - final int channelSize, final int bufferSize, final int remoteTimeout, final SslContext sslContext) { this.address = address; - this.channelSize = channelSize; this.bufferSize = bufferSize; this.remoteTimeout = remoteTimeout; this.sslContext = sslContext; @@ -100,7 +102,16 @@ public GRPCRemoteClient(final ModuleDefineHolder moduleDefineHolder, public void connect() { if (!isConnect) { this.getClient().connect(); - this.getDataCarrier().consume(new RemoteMessageConsumer(), 1); + this.queueName = "GRPC_REMOTE_" + address.getHost() + "_" + address.getPort(); + final BatchQueueConfig config = BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .bufferSize(bufferSize) + .strategy(BufferStrategy.BLOCKING) + .consumer(new RemoteMessageConsumer()) + .minIdleMs(10) + .maxIdleMs(100) + .build(); + this.queue = BatchQueueManager.create(queueName, config); this.isConnect = true; } } @@ -130,17 +141,6 @@ RemoteServiceGrpc.RemoteServiceStub getStub() { return RemoteServiceGrpc.newStub(getChannel()); } - DataCarrier getDataCarrier() { - if (Objects.isNull(this.carrier)) { - synchronized (GRPCRemoteClient.class) { - if (Objects.isNull(this.carrier)) { - this.carrier = new DataCarrier<>("GRPCRemoteClient", channelSize, bufferSize); - } - } - } - return this.carrier; - } - /** * Push stream data which need to send to another OAP server. * @@ -153,10 +153,10 @@ public void push(String nextWorkerName, StreamData streamData) { builder.setNextWorkerName(nextWorkerName); builder.setRemoteData(streamData.serialize()); - this.getDataCarrier().produce(builder.build()); + this.queue.produce(builder.build()); } - class RemoteMessageConsumer implements IConsumer { + class RemoteMessageConsumer implements HandlerConsumer { @Override public void consume(List remoteMessages) { try { @@ -171,11 +171,6 @@ public void consume(List remoteMessages) { log.error(t.getMessage(), t); } } - - @Override - public void onError(List remoteMessages, Throwable t) { - log.error(t.getMessage(), t); - } } /** @@ -247,8 +242,8 @@ public void onCompleted() { @Override public void close() { - if (Objects.nonNull(this.carrier)) { - this.carrier.shutdownConsumers(); + if (queueName != null) { + BatchQueueManager.shutdown(queueName); } if (Objects.nonNull(this.client)) { this.client.shutdown(); diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/remote/client/RemoteClientManager.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/remote/client/RemoteClientManager.java index f541c7edd6dc..f25ad05d29e6 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/remote/client/RemoteClientManager.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/remote/client/RemoteClientManager.java @@ -240,7 +240,7 @@ private void reBuildRemoteClients(List remoteInstances) { newRemoteClients.add(client); } else { RemoteClient client; - client = new GRPCRemoteClient(moduleDefineHolder, address, 1, 3000, remoteTimeout, sslContext); + client = new GRPCRemoteClient(moduleDefineHolder, address, 3000, remoteTimeout, sslContext); client.connect(); newRemoteClients.add(client); } diff --git a/oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/remote/client/GRPCRemoteClientRealClient.java b/oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/remote/client/GRPCRemoteClientRealClient.java index 99a004a4602b..20c929fe2c18 100644 --- a/oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/remote/client/GRPCRemoteClientRealClient.java +++ b/oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/remote/client/GRPCRemoteClientRealClient.java @@ -57,7 +57,7 @@ public void inc(double value) { moduleManager.put(TelemetryModule.NAME, telemetryModuleDefine); telemetryModuleDefine.provider().registerServiceImplementation(MetricsCreator.class, metricsCreator); - GRPCRemoteClient remoteClient = spy(new GRPCRemoteClient(moduleManager, address, 1, 10, 10, null)); + GRPCRemoteClient remoteClient = spy(new GRPCRemoteClient(moduleManager, address, 10, 10, null)); remoteClient.connect(); for (int i = 0; i < 10000; i++) { diff --git a/oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/remote/client/GRPCRemoteClientTestCase.java b/oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/remote/client/GRPCRemoteClientTestCase.java index 0f81d5a23264..b622641a4196 100644 --- a/oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/remote/client/GRPCRemoteClientTestCase.java +++ b/oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/remote/client/GRPCRemoteClientTestCase.java @@ -141,7 +141,7 @@ public void observe(double value) { serviceRegistry.addService(new RemoteServiceHandler(moduleManager)); Address address = new Address("not-important", 11, false); - GRPCRemoteClient remoteClient = spy(new GRPCRemoteClient(moduleManager, address, 1, 10, 10, null)); + GRPCRemoteClient remoteClient = spy(new GRPCRemoteClient(moduleManager, address, 10, 10, null)); remoteClient.connect(); doReturn(channel).when(remoteClient).getChannel(); diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java index f6a117039e8b..1167fc57a58c 100644 --- a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java @@ -513,4 +513,16 @@ int[][] getAssignedPartitions() { boolean isDedicatedScheduler() { return dedicatedScheduler; } + + /** + * Take a point-in-time snapshot of queue usage across all partitions. + */ + public BatchQueueStats stats() { + final ArrayBlockingQueue[] currentPartitions = this.partitions; + final int[] used = new int[currentPartitions.length]; + for (int i = 0; i < currentPartitions.length; i++) { + used[i] = currentPartitions[i].size(); + } + return new BatchQueueStats(currentPartitions.length, config.getBufferSize(), used); + } } diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueStats.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueStats.java new file mode 100644 index 000000000000..2a0b64dff341 --- /dev/null +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueStats.java @@ -0,0 +1,133 @@ +/* + * 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.skywalking.oap.server.library.batchqueue; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import lombok.Getter; + +/** + * Immutable snapshot of a {@link BatchQueue}'s partition usage at a point in time. + * + *

Provides three levels of detail: + *

    + *
  • Total usage — aggregate across all partitions
  • + *
  • Per-partition usage — individual partition sizes
  • + *
  • Top N — the most-loaded partitions, sorted by usage descending
  • + *
+ */ +public class BatchQueueStats { + @Getter + private final int partitionCount; + @Getter + private final int bufferSize; + private final int[] partitionUsed; + + BatchQueueStats(final int partitionCount, final int bufferSize, final int[] partitionUsed) { + this.partitionCount = partitionCount; + this.bufferSize = bufferSize; + this.partitionUsed = Arrays.copyOf(partitionUsed, partitionUsed.length); + } + + /** + * Total capacity across all partitions: {@code partitionCount * bufferSize}. + */ + public long totalCapacity() { + return (long) partitionCount * bufferSize; + } + + /** + * Total number of items currently queued across all partitions. + */ + public int totalUsed() { + int sum = 0; + for (final int used : partitionUsed) { + sum += used; + } + return sum; + } + + /** + * Overall queue usage as a percentage (0.0–100.0). + */ + public double totalUsedPercentage() { + final long capacity = totalCapacity(); + if (capacity == 0) { + return 0.0; + } + return 100.0 * totalUsed() / capacity; + } + + /** + * Number of items currently queued in the given partition. + */ + public int partitionUsed(final int index) { + return partitionUsed[index]; + } + + /** + * Usage of the given partition as a percentage (0.0–100.0). + */ + public double partitionUsedPercentage(final int index) { + if (bufferSize == 0) { + return 0.0; + } + return 100.0 * partitionUsed[index] / bufferSize; + } + + /** + * Return the top {@code n} most-loaded partitions, sorted by usage descending. + * If {@code n >= partitionCount}, all partitions are returned. + */ + public List topN(final int n) { + final Integer[] indices = new Integer[partitionCount]; + for (int i = 0; i < partitionCount; i++) { + indices[i] = i; + } + Arrays.sort(indices, (a, b) -> Integer.compare(partitionUsed[b], partitionUsed[a])); + + final int limit = Math.min(n, partitionCount); + final List result = new ArrayList<>(limit); + for (int i = 0; i < limit; i++) { + final int idx = indices[i]; + result.add(new PartitionUsage( + idx, partitionUsed[idx], + bufferSize == 0 ? 0.0 : 100.0 * partitionUsed[idx] / bufferSize + )); + } + return result; + } + + /** + * Usage snapshot for a single partition. + */ + @Getter + public static class PartitionUsage { + private final int partitionIndex; + private final int used; + private final double usedPercentage; + + PartitionUsage(final int partitionIndex, final int used, final double usedPercentage) { + this.partitionIndex = partitionIndex; + this.used = used; + this.usedPercentage = usedPercentage; + } + } +} diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicy.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicy.java index 6b6585debe56..43e5f77ef190 100644 --- a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicy.java +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicy.java @@ -22,19 +22,22 @@ * Determines the number of threads for a BatchQueue's dedicated scheduler * or for a shared scheduler. * - * Two modes: + * Three modes: * - fixed(N): exactly N threads, regardless of hardware. * - cpuCores(multiplier): multiplier * Runtime.availableProcessors(), rounded. + * - cpuCoresWithBase(base, multiplier): base + multiplier * Runtime.availableProcessors(), rounded. * * Resolved value is always >= 1 — every pool must have at least one thread. * fixed() requires count >= 1 at construction. cpuCores() applies max(1, ...) at resolution. */ public class ThreadPolicy { private final int fixedCount; + private final int base; private final double cpuMultiplier; - private ThreadPolicy(final int fixedCount, final double cpuMultiplier) { + private ThreadPolicy(final int fixedCount, final int base, final double cpuMultiplier) { this.fixedCount = fixedCount; + this.base = base; this.cpuMultiplier = cpuMultiplier; } @@ -47,7 +50,7 @@ public static ThreadPolicy fixed(final int count) { if (count < 1) { throw new IllegalArgumentException("Thread count must be >= 1, got: " + count); } - return new ThreadPolicy(count, 0); + return new ThreadPolicy(count, 0, 0); } /** @@ -60,7 +63,25 @@ public static ThreadPolicy cpuCores(final double multiplier) { if (multiplier <= 0) { throw new IllegalArgumentException("CPU multiplier must be > 0, got: " + multiplier); } - return new ThreadPolicy(0, multiplier); + return new ThreadPolicy(0, 0, multiplier); + } + + /** + * Threads = base + round(multiplier * available CPU cores), min 1. + * Base must be >= 0, multiplier must be > 0. + * + * Example: cpuCoresWithBase(2, 0.25) on 8-core = 2 + 2 = 4, on 16-core = 2 + 4 = 6, on 24-core = 2 + 6 = 8. + * + * @throws IllegalArgumentException if base < 0 or multiplier <= 0 + */ + public static ThreadPolicy cpuCoresWithBase(final int base, final double multiplier) { + if (base < 0) { + throw new IllegalArgumentException("Base must be >= 0, got: " + base); + } + if (multiplier <= 0) { + throw new IllegalArgumentException("CPU multiplier must be > 0, got: " + multiplier); + } + return new ThreadPolicy(0, base, multiplier); } /** @@ -70,7 +91,7 @@ public int resolve() { if (fixedCount > 0) { return fixedCount; } - return Math.max(1, (int) Math.round(cpuMultiplier * Runtime.getRuntime().availableProcessors())); + return Math.max(1, base + (int) Math.round(cpuMultiplier * Runtime.getRuntime().availableProcessors())); } @Override @@ -78,6 +99,9 @@ public String toString() { if (fixedCount > 0) { return "fixed(" + fixedCount + ")"; } + if (base > 0) { + return "cpuCoresWithBase(" + base + ", " + cpuMultiplier + ")"; + } return "cpuCores(" + cpuMultiplier + ")"; } } diff --git a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueTest.java b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueTest.java index a63bbdf400bc..1f3795102fee 100644 --- a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueTest.java +++ b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueTest.java @@ -31,6 +31,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; +import static org.junit.jupiter.api.Assertions.assertNotEquals; import static org.junit.jupiter.api.Assertions.assertTrue; public class BatchQueueTest { @@ -484,6 +485,195 @@ public void onIdle() { "Later gap (" + laterGap + "ms) should be larger than early gap (" + earlyGap + "ms)"); } + // --- Stats --- + + @Test + public void testStatsReflectsQueueUsage() { + final CountDownLatch blockLatch = new CountDownLatch(1); + final BatchQueue queue = BatchQueueManager.create("stats-usage-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .strategy(BufferStrategy.IF_POSSIBLE) + .consumer(data -> { + try { + blockLatch.await(); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + } + }) + .bufferSize(100) + .build()); + + // Wait for the consumer to block on the first item + queue.produce("trigger"); + Awaitility.await().atMost(2, TimeUnit.SECONDS) + .pollInterval(10, TimeUnit.MILLISECONDS) + .until(() -> { + final BatchQueueStats stats = queue.stats(); + return stats.totalUsed() == 0; // first item already drained into blocked consumer + }); + + // Produce 10 items — they'll sit in the partition because consumer is blocked + for (int i = 0; i < 10; i++) { + queue.produce("item-" + i); + } + + final BatchQueueStats stats = queue.stats(); + assertEquals(1, stats.getPartitionCount()); + assertEquals(100, stats.getBufferSize()); + assertEquals(100, stats.totalCapacity()); + assertEquals(10, stats.totalUsed()); + assertEquals(10.0, stats.totalUsedPercentage(), 0.01); + assertEquals(10, stats.partitionUsed(0)); + assertEquals(10.0, stats.partitionUsedPercentage(0), 0.01); + + blockLatch.countDown(); + } + + @Test + public void testStatsWithMultiplePartitions() { + final CountDownLatch blockLatch = new CountDownLatch(1); + final BatchQueue queue = BatchQueueManager.create("stats-multi-partition-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(2)) + .partitions(PartitionPolicy.fixed(4)) + .strategy(BufferStrategy.IF_POSSIBLE) + .partitionSelector((data, count) -> Integer.parseInt(data) % count) + .consumer(data -> { + try { + blockLatch.await(); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + } + }) + .bufferSize(50) + .build()); + + // Wait for drain threads to block + queue.produce("0"); + queue.produce("1"); + Awaitility.await().atMost(2, TimeUnit.SECONDS) + .pollInterval(10, TimeUnit.MILLISECONDS) + .until(() -> queue.stats().totalUsed() == 0); + + // Produce items targeting specific partitions: value % 4 + for (int i = 0; i < 20; i++) { + queue.produce(String.valueOf(i)); + } + + final BatchQueueStats stats = queue.stats(); + assertEquals(4, stats.getPartitionCount()); + assertEquals(50, stats.getBufferSize()); + assertEquals(200, stats.totalCapacity()); + assertEquals(20, stats.totalUsed()); + // Each partition gets 5 items (0,4,8,12,16 / 1,5,9,13,17 / 2,6,10,14,18 / 3,7,11,15,19) + for (int p = 0; p < 4; p++) { + assertEquals(5, stats.partitionUsed(p)); + assertEquals(10.0, stats.partitionUsedPercentage(p), 0.01); + } + + blockLatch.countDown(); + } + + @Test + public void testStatsSnapshotIsImmutable() { + final CountDownLatch blockLatch = new CountDownLatch(1); + final BatchQueue queue = BatchQueueManager.create("stats-immutable-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .strategy(BufferStrategy.IF_POSSIBLE) + .consumer(data -> { + try { + blockLatch.await(); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + } + }) + .bufferSize(100) + .build()); + + queue.produce("trigger"); + Awaitility.await().atMost(2, TimeUnit.SECONDS) + .pollInterval(10, TimeUnit.MILLISECONDS) + .until(() -> queue.stats().totalUsed() == 0); + + for (int i = 0; i < 5; i++) { + queue.produce("item-" + i); + } + + final BatchQueueStats snapshot = queue.stats(); + assertEquals(5, snapshot.totalUsed()); + + // Produce more — the snapshot should not change + for (int i = 0; i < 5; i++) { + queue.produce("more-" + i); + } + + assertEquals(5, snapshot.totalUsed()); + assertNotEquals(5, queue.stats().totalUsed()); + + blockLatch.countDown(); + } + + @Test + public void testStatsTopNReturnsHottestPartitions() { + final CountDownLatch blockLatch = new CountDownLatch(1); + final BatchQueue queue = BatchQueueManager.create("stats-topn-test", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(2)) + .partitions(PartitionPolicy.fixed(4)) + .strategy(BufferStrategy.IF_POSSIBLE) + .partitionSelector((data, count) -> Integer.parseInt(data.split("-")[0]) % count) + .consumer(data -> { + try { + blockLatch.await(); + } catch (final InterruptedException e) { + Thread.currentThread().interrupt(); + } + }) + .bufferSize(100) + .build()); + + // Wait for drain threads to block + queue.produce("0-trigger"); + queue.produce("1-trigger"); + Awaitility.await().atMost(2, TimeUnit.SECONDS) + .pollInterval(10, TimeUnit.MILLISECONDS) + .until(() -> queue.stats().totalUsed() == 0); + + // Load partitions unevenly: p0=20, p1=5, p2=15, p3=10 + for (int i = 0; i < 20; i++) { + queue.produce("0-" + i); + } + for (int i = 0; i < 5; i++) { + queue.produce("1-" + i); + } + for (int i = 0; i < 15; i++) { + queue.produce("2-" + i); + } + for (int i = 0; i < 10; i++) { + queue.produce("3-" + i); + } + + final BatchQueueStats stats = queue.stats(); + final java.util.List top2 = stats.topN(2); + + assertEquals(2, top2.size()); + // Highest: partition 0 (20 items) + assertEquals(0, top2.get(0).getPartitionIndex()); + assertEquals(20, top2.get(0).getUsed()); + assertEquals(20.0, top2.get(0).getUsedPercentage(), 0.01); + // Second: partition 2 (15 items) + assertEquals(2, top2.get(1).getPartitionIndex()); + assertEquals(15, top2.get(1).getUsed()); + + // topN with n > partitionCount returns all + final java.util.List topAll = stats.topN(10); + assertEquals(4, topAll.size()); + + blockLatch.countDown(); + } + @Test public void testBackoffResetsOnData() throws Exception { final AtomicInteger consumeCount = new AtomicInteger(0); diff --git a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicyTest.java b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicyTest.java index 4a77f7ad435c..3cbdeb6c0aaa 100644 --- a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicyTest.java +++ b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicyTest.java @@ -75,4 +75,36 @@ public void testToStringFixed() { public void testToStringCpuCores() { assertEquals("cpuCores(0.5)", ThreadPolicy.cpuCores(0.5).toString()); } + + @Test + public void testCpuCoresWithBaseAddsBaseToScaled() { + final int cores = Runtime.getRuntime().availableProcessors(); + final int resolved = ThreadPolicy.cpuCoresWithBase(2, 0.25).resolve(); + assertEquals(2 + (int) Math.round(0.25 * cores), resolved); + } + + @Test + public void testCpuCoresWithBaseResolvesAtLeastOne() { + assertTrue(ThreadPolicy.cpuCoresWithBase(0, 0.001).resolve() >= 1); + } + + @Test + public void testCpuCoresWithBaseRejectsNegativeBase() { + assertThrows(IllegalArgumentException.class, () -> ThreadPolicy.cpuCoresWithBase(-1, 0.25)); + } + + @Test + public void testCpuCoresWithBaseRejectsZeroMultiplier() { + assertThrows(IllegalArgumentException.class, () -> ThreadPolicy.cpuCoresWithBase(2, 0)); + } + + @Test + public void testCpuCoresWithBaseRejectsNegativeMultiplier() { + assertThrows(IllegalArgumentException.class, () -> ThreadPolicy.cpuCoresWithBase(2, -0.5)); + } + + @Test + public void testToStringCpuCoresWithBase() { + assertEquals("cpuCoresWithBase(2, 0.25)", ThreadPolicy.cpuCoresWithBase(2, 0.25).toString()); + } } diff --git a/oap-server/server-library/library-datacarrier-queue/pom.xml b/oap-server/server-library/library-datacarrier-queue/pom.xml deleted file mode 100644 index dda9b2ffa9d2..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/pom.xml +++ /dev/null @@ -1,37 +0,0 @@ - - - - - - server-library - org.apache.skywalking - ${revision} - - 4.0.0 - - library-datacarrier-queue - - - - uk.org.webcompere - system-stubs-jupiter - test - - - diff --git a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/DataCarrier.java b/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/DataCarrier.java deleted file mode 100644 index d755111ac088..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/DataCarrier.java +++ /dev/null @@ -1,168 +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.skywalking.oap.server.library.datacarrier; - -import java.util.Properties; -import lombok.Getter; -import org.apache.skywalking.oap.server.library.datacarrier.buffer.BufferStrategy; -import org.apache.skywalking.oap.server.library.datacarrier.buffer.Channels; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.ConsumeDriver; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.ConsumerPool; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.IConsumer; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.IDriver; -import org.apache.skywalking.oap.server.library.datacarrier.partition.IDataPartitioner; -import org.apache.skywalking.oap.server.library.datacarrier.partition.SimpleRollingPartitioner; - -/** - * DataCarrier main class. use this instance to set Producer/Consumer Model. - */ -public class DataCarrier { - @Getter - private Channels channels; - private IDriver driver; - private String name; - - public DataCarrier(int channelSize, int bufferSize) { - this("DEFAULT", channelSize, bufferSize); - } - - public DataCarrier(String name, int channelSize, int bufferSize) { - this(name, name, channelSize, bufferSize); - } - - public DataCarrier(String name, String envPrefix, int channelSize, int bufferSize) { - this(name, envPrefix, channelSize, bufferSize, BufferStrategy.BLOCKING); - } - - public DataCarrier(String name, String envPrefix, int channelSize, int bufferSize, BufferStrategy strategy) { - this.name = name; - bufferSize = EnvUtil.getInt(envPrefix + "_BUFFER_SIZE", bufferSize); - channelSize = EnvUtil.getInt(envPrefix + "_CHANNEL_SIZE", channelSize); - channels = new Channels<>(channelSize, bufferSize, new SimpleRollingPartitioner(), strategy); - } - - public DataCarrier(int channelSize, int bufferSize, BufferStrategy strategy) { - this("DEFAULT", "DEFAULT", channelSize, bufferSize, strategy); - } - - /** - * set a new IDataPartitioner. It will cover the current one or default one.(Default is {@link - * SimpleRollingPartitioner} - * - * @param dataPartitioner to partition data into different channel by some rules. - * @return DataCarrier instance for chain - */ - public DataCarrier setPartitioner(IDataPartitioner dataPartitioner) { - this.channels.setPartitioner(dataPartitioner); - return this; - } - - /** - * produce data to buffer, using the given {@link BufferStrategy}. - * - * @return false means produce data failure. The data will not be consumed. - */ - public boolean produce(T data) { - if (driver != null) { - if (!driver.isRunning(channels)) { - return false; - } - } - - return this.channels.save(data); - } - - /** - * set consumeDriver to this Carrier. consumer begin to run when {@link DataCarrier#produce} begin to work. - * - * @param consumerClass class of consumer - * @param num number of consumer threads - * @param properties for initializing consumer. - */ - public DataCarrier consume(Class> consumerClass, - int num, - long consumeCycle, - Properties properties) { - if (driver != null) { - driver.close(channels); - } - driver = new ConsumeDriver(this.name, this.channels, consumerClass, num, consumeCycle, properties); - driver.begin(channels); - return this; - } - - /** - * set consumeDriver to this Carrier. consumer begins to run when {@link DataCarrier#produce} begin to work with 200 - * millis consume cycle. - * - * @param consumerClass class of consumer - * @param num number of consumer threads - */ - public DataCarrier consume(Class> consumerClass, int num) { - return this.consume(consumerClass, num, 200, new Properties()); - } - - /** - * set consumeDriver to this Carrier. consumer begin to run when {@link DataCarrier#produce} begin to work. - * - * @param consumer single instance of consumer, all consumer threads will all use this instance. - * @param num number of consumer threads - */ - public DataCarrier consume(IConsumer consumer, int num, long consumeCycle) { - if (driver != null) { - driver.close(channels); - } - driver = new ConsumeDriver(this.name, this.channels, consumer, num, consumeCycle); - driver.begin(channels); - return this; - } - - /** - * set consumeDriver to this Carrier. consumer begin to run when {@link DataCarrier#produce} begin to work with 200 - * millis consume cycle. - * - * @param consumer single instance of consumer, all consumer threads will all use this instance. - * @param num number of consumer threads - */ - public DataCarrier consume(IConsumer consumer, int num) { - return this.consume(consumer, num, 200); - } - - /** - * Set a consumer pool to manage the channels of this DataCarrier. Then consumerPool could use its own consuming - * model to adjust the consumer thread and throughput. - */ - public DataCarrier consume(ConsumerPool consumerPool, IConsumer consumer) { - driver = consumerPool; - consumerPool.add(this.name, channels, consumer); - driver.begin(channels); - return this; - } - - /** - * shutdown all consumer threads, if consumer threads are running. Notice {@link BufferStrategy}: if {@link - * BufferStrategy} == {@link BufferStrategy#BLOCKING}, shutdown consumeDriver maybe cause blocking when producing. - * Better way to change consumeDriver are use {@link DataCarrier#consume} - */ - public void shutdownConsumers() { - if (driver != null) { - driver.close(channels); - } - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/EnvUtil.java b/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/EnvUtil.java deleted file mode 100644 index 5a672a6895b8..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/EnvUtil.java +++ /dev/null @@ -1,50 +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.skywalking.oap.server.library.datacarrier; - -/** - * Read value from system env. - */ -public class EnvUtil { - public static int getInt(String envName, int defaultValue) { - int value = defaultValue; - String envValue = System.getenv(envName); - if (envValue != null) { - try { - value = Integer.parseInt(envValue); - } catch (NumberFormatException e) { - - } - } - return value; - } - - public static long getLong(String envName, long defaultValue) { - long value = defaultValue; - String envValue = System.getenv(envName); - if (envValue != null) { - try { - value = Long.parseLong(envValue); - } catch (NumberFormatException e) { - - } - } - return value; - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/buffer/ArrayBlockingQueueBuffer.java b/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/buffer/ArrayBlockingQueueBuffer.java deleted file mode 100644 index 502bc2fe3c83..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/buffer/ArrayBlockingQueueBuffer.java +++ /dev/null @@ -1,69 +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.skywalking.oap.server.library.datacarrier.buffer; - -import java.util.List; -import java.util.concurrent.ArrayBlockingQueue; - -/** - * The buffer implementation based on JDK ArrayBlockingQueue. - *

- * This implementation has better performance in server side. We are still trying to research whether this is suitable - * for agent side, which is more sensitive about blocks. - */ -public class ArrayBlockingQueueBuffer implements QueueBuffer { - private BufferStrategy strategy; - private ArrayBlockingQueue queue; - private int bufferSize; - - ArrayBlockingQueueBuffer(int bufferSize, BufferStrategy strategy) { - this.strategy = strategy; - this.queue = new ArrayBlockingQueue(bufferSize); - this.bufferSize = bufferSize; - } - - @Override - public boolean save(T data) { - try { - if (BufferStrategy.IF_POSSIBLE.equals(strategy)) { - return queue.offer(data); - } - queue.put(data); - } catch (InterruptedException e) { - // Ignore the error - return false; - } - return true; - } - - @Override - public void setStrategy(BufferStrategy strategy) { - this.strategy = strategy; - } - - @Override - public void obtain(List consumeList) { - queue.drainTo(consumeList); - } - - @Override - public int getBufferSize() { - return bufferSize; - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/buffer/BufferStrategy.java b/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/buffer/BufferStrategy.java deleted file mode 100644 index bfd36db6406a..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/buffer/BufferStrategy.java +++ /dev/null @@ -1,23 +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.skywalking.oap.server.library.datacarrier.buffer; - -public enum BufferStrategy { - BLOCKING, IF_POSSIBLE -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/buffer/Channels.java b/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/buffer/Channels.java deleted file mode 100644 index 2afa7137925f..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/buffer/Channels.java +++ /dev/null @@ -1,91 +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.skywalking.oap.server.library.datacarrier.buffer; - -import lombok.Getter; -import org.apache.skywalking.oap.server.library.datacarrier.partition.IDataPartitioner; - -/** - * Channels of Buffer It contains all buffer data which belongs to this channel. It supports several strategy when - * buffer is full. The Default is BLOCKING

Created by wusheng on 2016/10/25. - */ -public class Channels { - @Getter - private final QueueBuffer[] bufferChannels; - private IDataPartitioner dataPartitioner; - private final BufferStrategy strategy; - private final long size; - - public Channels(int channelSize, int bufferSize, IDataPartitioner partitioner, BufferStrategy strategy) { - this.dataPartitioner = partitioner; - this.strategy = strategy; - bufferChannels = new QueueBuffer[channelSize]; - for (int i = 0; i < channelSize; i++) { - bufferChannels[i] = new ArrayBlockingQueueBuffer<>(bufferSize, strategy); - } - // noinspection PointlessArithmeticExpression - size = 1L * channelSize * bufferSize; // it's not pointless, it prevents numeric overflow before assigning an integer to a long - } - - public boolean save(T data) { - int index = dataPartitioner.partition(bufferChannels.length, data); - int retryCountDown = 1; - if (BufferStrategy.IF_POSSIBLE.equals(strategy)) { - int maxRetryCount = dataPartitioner.maxRetryCount(); - if (maxRetryCount > 1) { - retryCountDown = maxRetryCount; - } - } - for (; retryCountDown > 0; retryCountDown--) { - if (bufferChannels[index].save(data)) { - return true; - } - } - return false; - } - - public void setPartitioner(IDataPartitioner dataPartitioner) { - this.dataPartitioner = dataPartitioner; - } - - /** - * override the strategy at runtime. Notice, this will override several channels one by one. So, when running - * setStrategy, each channel may use different BufferStrategy - */ - public void setStrategy(BufferStrategy strategy) { - for (QueueBuffer buffer : bufferChannels) { - buffer.setStrategy(strategy); - } - } - - /** - * get channelSize - */ - public int getChannelSize() { - return this.bufferChannels.length; - } - - public long size() { - return size; - } - - public QueueBuffer getBuffer(int index) { - return this.bufferChannels[index]; - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/buffer/QueueBuffer.java b/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/buffer/QueueBuffer.java deleted file mode 100644 index bf24537d3290..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/buffer/QueueBuffer.java +++ /dev/null @@ -1,46 +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.skywalking.oap.server.library.datacarrier.buffer; - -import java.util.List; - -/** - * Queue buffer interface. - */ -public interface QueueBuffer { - /** - * Save data into the queue; - * - * @param data to add. - * @return true if saved - */ - boolean save(T data); - - /** - * Set different strategy when queue is full. - */ - void setStrategy(BufferStrategy strategy); - - /** - * Obtain the existing data from the queue - */ - void obtain(List consumeList); - - int getBufferSize(); -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/common/AtomicRangeInteger.java b/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/common/AtomicRangeInteger.java deleted file mode 100644 index 07eeaec42179..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/common/AtomicRangeInteger.java +++ /dev/null @@ -1,76 +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.skywalking.oap.server.library.datacarrier.common; - -import java.io.Serializable; -import java.util.concurrent.atomic.AtomicIntegerArray; - -public class AtomicRangeInteger extends Number implements Serializable { - private static final long serialVersionUID = -4099792402691141643L; - private AtomicIntegerArray values; - - private static final int VALUE_OFFSET = 15; - - private int startValue; - private int endValue; - - public AtomicRangeInteger(int startValue, int maxValue) { - this.values = new AtomicIntegerArray(31); - this.values.set(VALUE_OFFSET, startValue); - this.startValue = startValue; - this.endValue = maxValue - 1; - } - - public final int getAndIncrement() { - int next; - do { - next = this.values.incrementAndGet(VALUE_OFFSET); - if (next > endValue && this.values.compareAndSet(VALUE_OFFSET, next, startValue)) { - return endValue; - } - } - while (next > endValue); - - return next - 1; - } - - public final int get() { - return this.values.get(VALUE_OFFSET); - } - - @Override - public int intValue() { - return this.values.get(VALUE_OFFSET); - } - - @Override - public long longValue() { - return this.values.get(VALUE_OFFSET); - } - - @Override - public float floatValue() { - return this.values.get(VALUE_OFFSET); - } - - @Override - public double doubleValue() { - return this.values.get(VALUE_OFFSET); - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/BulkConsumePool.java b/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/BulkConsumePool.java deleted file mode 100644 index 3ee33f6581f0..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/BulkConsumePool.java +++ /dev/null @@ -1,128 +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.skywalking.oap.server.library.datacarrier.consumer; - -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.Callable; -import org.apache.skywalking.oap.server.library.datacarrier.EnvUtil; -import org.apache.skywalking.oap.server.library.datacarrier.buffer.Channels; - -/** - * BulkConsumePool works for consuming data from multiple channels(DataCarrier instances), with multiple {@link - * MultipleChannelsConsumer}s. - *

- * In typical case, the number of {@link MultipleChannelsConsumer} should be less than the number of channels. - */ -public class BulkConsumePool implements ConsumerPool { - private List allConsumers; - private volatile boolean isStarted = false; - - public BulkConsumePool(String name, int size, long consumeCycle, boolean isSignalDrivenMode) { - size = EnvUtil.getInt(name + "_THREAD", size); - allConsumers = new ArrayList<>(size); - for (int i = 0; i < size; i++) { - MultipleChannelsConsumer multipleChannelsConsumer = new MultipleChannelsConsumer( - "DataCarrier." + name + ".BulkConsumePool." + i + ".Thread", consumeCycle, isSignalDrivenMode); - multipleChannelsConsumer.setDaemon(true); - allConsumers.add(multipleChannelsConsumer); - } - } - - @Override - synchronized public void add(String name, Channels channels, IConsumer consumer) { - MultipleChannelsConsumer multipleChannelsConsumer = getLowestPayload(); - multipleChannelsConsumer.addNewTarget(channels, consumer); - } - - /** - * Get the lowest payload consumer thread based on current allocate status. - * - * @return the lowest consumer. - */ - private MultipleChannelsConsumer getLowestPayload() { - MultipleChannelsConsumer winner = allConsumers.get(0); - for (int i = 1; i < allConsumers.size(); i++) { - MultipleChannelsConsumer option = allConsumers.get(i); - if (option.size() < winner.size()) { - winner = option; - } - } - return winner; - } - - /** - * - */ - @Override - public boolean isRunning(Channels channels) { - return isStarted; - } - - @Override - public void close(Channels channels) { - for (MultipleChannelsConsumer consumer : allConsumers) { - consumer.shutdown(); - } - } - - @Override - public void begin(Channels channels) { - if (isStarted) { - return; - } - for (MultipleChannelsConsumer consumer : allConsumers) { - consumer.start(); - } - isStarted = true; - } - - public void notifyConsumers() { - for (MultipleChannelsConsumer consumer : allConsumers) { - consumer.setConsumeFlag(true); - } - } - - /** - * The creator for {@link BulkConsumePool}. - */ - public static class Creator implements Callable { - private String name; - private int size; - private long consumeCycle; - // Consumer has two modes to drive consumption. 1. Polling mode. 2. Signal-Driven mode. - private final boolean isSignalDrivenMode; - - public Creator(String name, int poolSize, long consumeCycle, boolean isSignalDrivenMode) { - this.name = name; - this.size = poolSize; - this.consumeCycle = consumeCycle; - this.isSignalDrivenMode = isSignalDrivenMode; - } - - @Override - public ConsumerPool call() { - return new BulkConsumePool(name, size, consumeCycle, isSignalDrivenMode); - } - - public static int recommendMaxSize() { - return Runtime.getRuntime().availableProcessors() * 2; - } - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumeDriver.java b/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumeDriver.java deleted file mode 100644 index d8db3a5f2113..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumeDriver.java +++ /dev/null @@ -1,137 +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.skywalking.oap.server.library.datacarrier.consumer; - -import java.lang.reflect.InvocationTargetException; -import java.util.Properties; -import java.util.concurrent.locks.ReentrantLock; -import org.apache.skywalking.oap.server.library.datacarrier.buffer.Channels; - -/** - * Pool of consumers

Created by wusheng on 2016/10/25. - */ -public class ConsumeDriver implements IDriver { - private boolean running; - private ConsumerThread[] consumerThreads; - private Channels channels; - private ReentrantLock lock; - - public ConsumeDriver(String name, - Channels channels, Class> consumerClass, - int num, - long consumeCycle, - Properties properties) { - this(channels, num); - for (int i = 0; i < num; i++) { - consumerThreads[i] = new ConsumerThread( - "DataCarrier." + name + ".Consumer." + i + ".Thread", getNewConsumerInstance(consumerClass, properties), - consumeCycle - ); - consumerThreads[i].setDaemon(true); - } - } - - public ConsumeDriver(String name, Channels channels, IConsumer prototype, int num, long consumeCycle) { - this(channels, num); - prototype.init(new Properties()); - for (int i = 0; i < num; i++) { - consumerThreads[i] = new ConsumerThread( - "DataCarrier." + name + ".Consumer." + i + ".Thread", prototype, consumeCycle); - consumerThreads[i].setDaemon(true); - } - - } - - private ConsumeDriver(Channels channels, int num) { - running = false; - this.channels = channels; - consumerThreads = new ConsumerThread[num]; - lock = new ReentrantLock(); - } - - private IConsumer getNewConsumerInstance(Class> consumerClass, Properties properties) { - try { - IConsumer inst = consumerClass.getDeclaredConstructor().newInstance(); - inst.init(properties); - return inst; - } catch (InstantiationException e) { - throw new ConsumerCannotBeCreatedException(e); - } catch (IllegalAccessException e) { - throw new ConsumerCannotBeCreatedException(e); - } catch (NoSuchMethodException e) { - throw new ConsumerCannotBeCreatedException(e); - } catch (InvocationTargetException e) { - throw new ConsumerCannotBeCreatedException(e); - } - } - - @Override - public void begin(Channels channels) { - if (running) { - return; - } - lock.lock(); - try { - this.allocateBuffer2Thread(); - for (ConsumerThread consumerThread : consumerThreads) { - consumerThread.start(); - } - running = true; - } finally { - lock.unlock(); - } - } - - @Override - public boolean isRunning(Channels channels) { - return running; - } - - private void allocateBuffer2Thread() { - int channelSize = this.channels.getChannelSize(); - /** - * if consumerThreads.length < channelSize - * each consumer will process several channels. - * - * if consumerThreads.length == channelSize - * each consumer will process one channel. - * - * if consumerThreads.length > channelSize - * there will be some threads do nothing. - */ - for (int channelIndex = 0; channelIndex < channelSize; channelIndex++) { - int consumerIndex = channelIndex % consumerThreads.length; - consumerThreads[consumerIndex].addDataSource(channels.getBuffer(channelIndex)); - } - - } - - @Override - public void close(Channels channels) { - lock.lock(); - try { - this.running = false; - for (ConsumerThread consumerThread : consumerThreads) { - consumerThread.shutdown(); - } - } finally { - lock.unlock(); - } - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerCannotBeCreatedException.java b/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerCannotBeCreatedException.java deleted file mode 100644 index 90640095a7c7..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerCannotBeCreatedException.java +++ /dev/null @@ -1,25 +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.skywalking.oap.server.library.datacarrier.consumer; - -public class ConsumerCannotBeCreatedException extends RuntimeException { - ConsumerCannotBeCreatedException(Throwable t) { - super(t); - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerPool.java b/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerPool.java deleted file mode 100644 index 0b9f34e5094a..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerPool.java +++ /dev/null @@ -1,30 +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.skywalking.oap.server.library.datacarrier.consumer; - -import org.apache.skywalking.oap.server.library.datacarrier.DataCarrier; -import org.apache.skywalking.oap.server.library.datacarrier.buffer.Channels; - -/** - * The Consumer pool could support data consumer from multiple {@link DataCarrier}s, by using different consume thread - * management models. - */ -public interface ConsumerPool extends IDriver { - void add(String name, Channels channels, IConsumer consumer); -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerPoolFactory.java b/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerPoolFactory.java deleted file mode 100644 index fd0455ac1938..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerPoolFactory.java +++ /dev/null @@ -1,50 +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.skywalking.oap.server.library.datacarrier.consumer; - -import java.util.HashMap; -import java.util.Map; -import java.util.concurrent.Callable; - -/** - * Consumer Pool Factory provides global management for all Consumer Pool. - */ -public enum ConsumerPoolFactory { - INSTANCE; - - private final Map pools; - - ConsumerPoolFactory() { - pools = new HashMap<>(); - } - - public synchronized boolean createIfAbsent(String poolName, Callable creator) throws Exception { - if (pools.containsKey(poolName)) { - return false; - } else { - pools.put(poolName, creator.call()); - return true; - } - } - - public ConsumerPool get(String poolName) { - return pools.get(poolName); - } - -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerThread.java b/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerThread.java deleted file mode 100644 index 30caa739b5b7..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerThread.java +++ /dev/null @@ -1,101 +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.skywalking.oap.server.library.datacarrier.consumer; - -import java.util.ArrayList; -import java.util.List; -import org.apache.skywalking.oap.server.library.datacarrier.buffer.QueueBuffer; - -public class ConsumerThread extends Thread { - private volatile boolean running; - private IConsumer consumer; - private List dataSources; - private long consumeCycle; - - ConsumerThread(String threadName, IConsumer consumer, long consumeCycle) { - super(threadName); - this.consumer = consumer; - running = false; - dataSources = new ArrayList(1); - this.consumeCycle = consumeCycle; - } - - /** - * add whole buffer to consume - */ - void addDataSource(QueueBuffer sourceBuffer) { - this.dataSources.add(new DataSource(sourceBuffer)); - } - - @Override - public void run() { - running = true; - - final List consumeList = new ArrayList(1500); - while (running) { - if (!consume(consumeList)) { - try { - Thread.sleep(consumeCycle); - } catch (InterruptedException e) { - } - } - } - - // consumer thread is going to stop - // consume the last time - consume(consumeList); - - consumer.onExit(); - } - - private boolean consume(List consumeList) { - for (DataSource dataSource : dataSources) { - dataSource.obtain(consumeList); - } - - if (!consumeList.isEmpty()) { - try { - consumer.consume(consumeList); - } catch (Throwable t) { - consumer.onError(consumeList, t); - } finally { - consumeList.clear(); - } - return true; - } - consumer.nothingToConsume(); - return false; - } - - void shutdown() { - running = false; - } - - class DataSource { - private QueueBuffer sourceBuffer; - - DataSource(QueueBuffer sourceBuffer) { - this.sourceBuffer = sourceBuffer; - } - - void obtain(List consumeList) { - sourceBuffer.obtain(consumeList); - } - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/IConsumer.java b/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/IConsumer.java deleted file mode 100644 index bb5db59ef740..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/IConsumer.java +++ /dev/null @@ -1,41 +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.skywalking.oap.server.library.datacarrier.consumer; - -import java.util.List; -import java.util.Properties; - -public interface IConsumer { - default void init(final Properties properties) { - } - - void consume(List data); - - void onError(List data, Throwable t); - - default void onExit() { - } - - /** - * Notify the implementation, if there is nothing fetched from the queue. This could be used as a timer to trigger - * reaction if the queue has no element. - */ - default void nothingToConsume() { - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/IDriver.java b/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/IDriver.java deleted file mode 100644 index d08a408c9899..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/IDriver.java +++ /dev/null @@ -1,32 +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.skywalking.oap.server.library.datacarrier.consumer; - -import org.apache.skywalking.oap.server.library.datacarrier.buffer.Channels; - -/** - * The driver of consumer. - */ -public interface IDriver { - boolean isRunning(Channels channels); - - void close(Channels channels); - - void begin(Channels channels); -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/MultipleChannelsConsumer.java b/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/MultipleChannelsConsumer.java deleted file mode 100644 index 1551caca692d..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/MultipleChannelsConsumer.java +++ /dev/null @@ -1,145 +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.skywalking.oap.server.library.datacarrier.consumer; - -import java.util.ArrayList; -import java.util.List; -import lombok.Setter; -import org.apache.skywalking.oap.server.library.datacarrier.buffer.Channels; -import org.apache.skywalking.oap.server.library.datacarrier.buffer.QueueBuffer; - -/** - * MultipleChannelsConsumer represent a single consumer thread, but support multiple channels with their {@link - * IConsumer}s - */ -public class MultipleChannelsConsumer extends Thread { - private volatile boolean running; - private volatile ArrayList consumeTargets; - @SuppressWarnings("NonAtomicVolatileUpdate") - private volatile long size; - private final long consumeCycle; - // The flag to indicate whether the consumer thread should consume data. - @Setter - private volatile boolean consumeFlag = false; - // Consumer has two modes to drive consumption. 1. Polling mode. 2. Signal-Driven mode. - private final boolean isSignalDrivenMode; - - public MultipleChannelsConsumer(String threadName, long consumeCycle, boolean isSignalDrivenMode) { - super(threadName); - this.consumeTargets = new ArrayList<>(); - this.consumeCycle = consumeCycle; - this.isSignalDrivenMode = isSignalDrivenMode; - } - - @Override - public void run() { - running = true; - - final List consumeList = new ArrayList(2000); - while (running) { - boolean hasData = false; - if (!isSignalDrivenMode) { - for (Group target : consumeTargets) { - boolean consumed = consume(target, consumeList); - hasData = hasData || consumed; - } - - if (!hasData) { - try { - Thread.sleep(consumeCycle); - } catch (InterruptedException e) { - } - } - } else { - if (consumeFlag) { - consumeFlag = false; - for (Group target : consumeTargets) { - consume(target, consumeList); - } - } else { - try { - Thread.sleep(consumeCycle); - } catch (InterruptedException e) { - } - } - } - } - - // consumer thread is going to stop - // consume the last time - for (Group target : consumeTargets) { - consume(target, consumeList); - - target.consumer.onExit(); - } - } - - private boolean consume(Group target, List consumeList) { - for (int i = 0; i < target.channels.getChannelSize(); i++) { - QueueBuffer buffer = target.channels.getBuffer(i); - buffer.obtain(consumeList); - } - - if (!consumeList.isEmpty()) { - try { - target.consumer.consume(consumeList); - } catch (Throwable t) { - target.consumer.onError(consumeList, t); - } finally { - consumeList.clear(); - } - return true; - } - target.consumer.nothingToConsume(); - return false; - } - - /** - * Add a new target channels. - */ - public void addNewTarget(Channels channels, IConsumer consumer) { - Group group = new Group(channels, consumer); - // Recreate the new list to avoid change list while the list is used in consuming. - ArrayList newList = new ArrayList<>(); - for (Group target : consumeTargets) { - newList.add(target); - } - newList.add(group); - consumeTargets = newList; - size += channels.size(); - } - - public long size() { - return size; - } - - void shutdown() { - running = false; - } - - private static class Group { - private Channels channels; - private IConsumer consumer; - - public Group(Channels channels, IConsumer consumer) { - this.channels = channels; - this.consumer = consumer; - } - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/partition/IDataPartitioner.java b/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/partition/IDataPartitioner.java deleted file mode 100644 index 5f033ff3c67c..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/partition/IDataPartitioner.java +++ /dev/null @@ -1,32 +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.skywalking.oap.server.library.datacarrier.partition; - -import org.apache.skywalking.oap.server.library.datacarrier.buffer.BufferStrategy; - -public interface IDataPartitioner { - int partition(int total, T data); - - /** - * @return an integer represents how many times should retry when {@link BufferStrategy#IF_POSSIBLE}. - *

- * Less or equal 1, means not support retry. - */ - int maxRetryCount(); -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/partition/ProducerThreadPartitioner.java b/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/partition/ProducerThreadPartitioner.java deleted file mode 100644 index ee647ed05747..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/partition/ProducerThreadPartitioner.java +++ /dev/null @@ -1,37 +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.skywalking.oap.server.library.datacarrier.partition; - -/** - * use threadid % total to partition - */ -public class ProducerThreadPartitioner implements IDataPartitioner { - public ProducerThreadPartitioner() { - } - - @Override - public int partition(int total, T data) { - return (int) Thread.currentThread().getId() % total; - } - - @Override - public int maxRetryCount() { - return 1; - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/partition/SimpleRollingPartitioner.java b/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/partition/SimpleRollingPartitioner.java deleted file mode 100644 index 9c9d61ddb8a0..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/main/java/org/apache/skywalking/oap/server/library/datacarrier/partition/SimpleRollingPartitioner.java +++ /dev/null @@ -1,37 +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.skywalking.oap.server.library.datacarrier.partition; - -/** - * use normal int to rolling. - */ -public class SimpleRollingPartitioner implements IDataPartitioner { - @SuppressWarnings("NonAtomicVolatileUpdate") - private volatile int i = 0; - - @Override - public int partition(int total, T data) { - return Math.abs(i++ % total); - } - - @Override - public int maxRetryCount() { - return 3; - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/DataCarrierBenchmark.java b/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/DataCarrierBenchmark.java deleted file mode 100644 index ad80a9476104..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/DataCarrierBenchmark.java +++ /dev/null @@ -1,192 +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.skywalking.oap.server.library.datacarrier; - -import java.util.List; -import java.util.concurrent.CountDownLatch; -import java.util.concurrent.TimeUnit; -import java.util.concurrent.atomic.AtomicLong; -import org.apache.skywalking.oap.server.library.datacarrier.buffer.BufferStrategy; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.BulkConsumePool; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.IConsumer; -import org.junit.jupiter.api.Test; - -/** - * Throughput benchmark for DataCarrier as a baseline for BatchQueue comparison. - * - *

Simulates the real production pattern: N DataCarrier instances (one per - * metric type), each with 1 channel, all sharing a {@link BulkConsumePool} - * with 8 consumer threads. 32 producer threads simulate gRPC connections. - * - *

Run with: mvn test -pl oap-server/server-library/library-datacarrier-queue - * -Dtest=DataCarrierBenchmark -DfailIfNoTests=false - * - *

Reference results (Apple M3 Max, 128 GB RAM, macOS 26.2, JDK 17)

- *
- * Types  Producers  Pool threads    IF_POSSIBLE       BLOCKING
- * ------ --------- -------------- ------------- -------------
- *   500      32      pool(8)       ~33,400,000   ~32,500,000
- *  1000      32      pool(8)       ~37,600,000   ~36,000,000
- *  2000      32      pool(8)       ~38,000,000   ~42,100,000
- *
- * All runs: 1 channel per carrier, bufferSize=50,000, consumeCycle=1ms, 0% drop rate.
- * 
- * - *

BatchQueue comparison (type-aware partitions, typed objects): - *

- *              500 types   1000 types   2000 types
- * 1:4           -4%         +6%          +24%
- * 1:2          +14%        +32%          +38%
- * adaptive     +37%        +34%          +68%
- * 1:1          +53%        +63%          +99%
- * 
- * - *

BatchQueue adaptive() = threshold(threadCount * 25), 1:1 below, 1:2 - * above. Consistently outperforms DataCarrier across all type counts. - * See BatchQueueBenchmark for full details. - */ -@SuppressWarnings("all") -public class DataCarrierBenchmark { - - private static final int WARMUP_SECONDS = 2; - private static final int MEASURE_SECONDS = 5; - private static final int PRODUCER_THREADS = 32; - private static final int POOL_THREADS = 8; - private static final int BUFFER_SIZE = 50_000; - - @Test - public void benchmark500Types() throws Exception { - runSharedPoolBenchmark("500-types", 500, BufferStrategy.IF_POSSIBLE); - } - - @Test - public void benchmark1000Types() throws Exception { - runSharedPoolBenchmark("1000-types", 1000, BufferStrategy.IF_POSSIBLE); - } - - @Test - public void benchmark2000Types() throws Exception { - runSharedPoolBenchmark("2000-types", 2000, BufferStrategy.IF_POSSIBLE); - } - - @Test - public void benchmark500TypesBlocking() throws Exception { - runSharedPoolBenchmark("500-types-blocking", 500, BufferStrategy.BLOCKING); - } - - @Test - public void benchmark1000TypesBlocking() throws Exception { - runSharedPoolBenchmark("1000-types-blocking", 1000, BufferStrategy.BLOCKING); - } - - @Test - public void benchmark2000TypesBlocking() throws Exception { - runSharedPoolBenchmark("2000-types-blocking", 2000, BufferStrategy.BLOCKING); - } - - private void runSharedPoolBenchmark(final String label, final int typeCount, - final BufferStrategy strategy) throws Exception { - final AtomicLong consumed = new AtomicLong(0); - - final BulkConsumePool pool = new BulkConsumePool( - "bench-pool", POOL_THREADS, 1, false); - - final DataCarrier[] carriers = new DataCarrier[typeCount]; - for (int i = 0; i < typeCount; i++) { - carriers[i] = new DataCarrier<>( - "type-" + i, "bench", 1, BUFFER_SIZE, strategy); - carriers[i].consume(pool, new IConsumer() { - @Override - public void consume(final List data) { - consumed.addAndGet(data.size()); - } - - @Override - public void onError(final List data, final Throwable t) { - t.printStackTrace(); - } - }); - } - - // Warmup - final long warmupEnd = System.currentTimeMillis() + WARMUP_SECONDS * 1000L; - runProducers(carriers, warmupEnd); - Thread.sleep(200); - consumed.set(0); - - // Measure - final long measureStart = System.currentTimeMillis(); - final long measureEnd = measureStart + MEASURE_SECONDS * 1000L; - final long produced = runProducers(carriers, measureEnd); - final long measureDuration = System.currentTimeMillis() - measureStart; - - Thread.sleep(500); - final long totalConsumed = consumed.get(); - - pool.close(null); - - System.out.printf("%n=== DataCarrier Benchmark: %s ===%n" - + " Types: %d (1 DataCarrier per type, 1 channel each)%n" - + " Pool threads:%d%n" - + " Strategy: %s%n" - + " Producers: %d%n" - + " Duration: %d ms%n" - + " Produced: %,d%n" - + " Consumed: %,d%n" - + " Consume rate: %,.0f items/sec%n" - + " Drop rate: %.2f%%%n", - label, typeCount, POOL_THREADS, strategy, - PRODUCER_THREADS, measureDuration, - produced, totalConsumed, - totalConsumed * 1000.0 / measureDuration, - produced > 0 ? (produced - totalConsumed) * 100.0 / produced : 0); - } - - private long runProducers(final DataCarrier[] carriers, - final long endTimeMs) throws InterruptedException { - final int carrierCount = carriers.length; - final AtomicLong totalProduced = new AtomicLong(0); - final CountDownLatch done = new CountDownLatch(PRODUCER_THREADS); - - for (int p = 0; p < PRODUCER_THREADS; p++) { - final int producerIndex = p; - final Thread thread = new Thread(() -> { - long count = 0; - int typeIndex = producerIndex; - while (System.currentTimeMillis() < endTimeMs) { - for (int batch = 0; batch < 100; batch++) { - final int type = typeIndex % carrierCount; - if (carriers[type].produce(count)) { - count++; - } - typeIndex++; - } - } - totalProduced.addAndGet(count); - done.countDown(); - }); - thread.setName("Producer-" + producerIndex); - thread.setDaemon(true); - thread.start(); - } - - done.await(MEASURE_SECONDS + 10, TimeUnit.SECONDS); - return totalProduced.get(); - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/DataCarrierTest.java b/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/DataCarrierTest.java deleted file mode 100644 index 25f9633acd02..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/DataCarrierTest.java +++ /dev/null @@ -1,134 +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.skywalking.oap.server.library.datacarrier; - -import org.apache.skywalking.oap.server.library.datacarrier.buffer.BufferStrategy; -import org.apache.skywalking.oap.server.library.datacarrier.buffer.Channels; -import org.apache.skywalking.oap.server.library.datacarrier.buffer.QueueBuffer; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.IConsumer; -import org.apache.skywalking.oap.server.library.datacarrier.partition.ProducerThreadPartitioner; -import org.apache.skywalking.oap.server.library.datacarrier.partition.SimpleRollingPartitioner; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.powermock.reflect.Whitebox; - -import java.util.ArrayList; -import java.util.List; - -import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertTrue; - -public class DataCarrierTest { - @Test - public void testCreateDataCarrier() { - DataCarrier carrier = new DataCarrier<>(5, 100, BufferStrategy.IF_POSSIBLE); - - Channels channels = Whitebox.getInternalState(carrier, "channels"); - assertEquals(5, channels.getChannelSize()); - - QueueBuffer buffer = channels.getBuffer(0); - assertEquals(100, buffer.getBufferSize()); - - assertEquals(Whitebox.getInternalState(buffer, "strategy"), BufferStrategy.IF_POSSIBLE); - assertEquals(Whitebox.getInternalState(buffer, "strategy"), BufferStrategy.IF_POSSIBLE); - - assertEquals(Whitebox.getInternalState(channels, "dataPartitioner").getClass(), SimpleRollingPartitioner.class); - carrier.setPartitioner(new ProducerThreadPartitioner<>()); - assertEquals(Whitebox.getInternalState(channels, "dataPartitioner").getClass(), ProducerThreadPartitioner.class); - } - - @Test - public void testProduce() throws IllegalAccessException { - DataCarrier carrier = new DataCarrier<>(2, 100); - assertTrue(carrier.produce(new SampleData().setName("a"))); - assertTrue(carrier.produce(new SampleData().setName("b"))); - assertTrue(carrier.produce(new SampleData().setName("c"))); - assertTrue(carrier.produce(new SampleData().setName("d"))); - - Channels channels = Whitebox.getInternalState(carrier, "channels"); - QueueBuffer buffer1 = channels.getBuffer(0); - - List result = new ArrayList(); - buffer1.obtain(result); - assertEquals(2, result.size()); - - QueueBuffer buffer2 = channels.getBuffer(1); - buffer2.obtain(result); - - assertEquals(4, result.size()); - - } - - @Test - public void testIfPossibleProduce() { - DataCarrier carrier = new DataCarrier<>(2, 100, BufferStrategy.IF_POSSIBLE); - - for (int i = 0; i < 200; i++) { - assertTrue(carrier.produce(new SampleData().setName("d" + i))); - } - - for (int i = 0; i < 200; i++) { - Assertions.assertFalse(carrier.produce(new SampleData().setName("d" + i + "_2"))); - } - - Channels channels = Whitebox.getInternalState(carrier, "channels"); - QueueBuffer buffer1 = channels.getBuffer(0); - List result = new ArrayList<>(); - buffer1.obtain(result); - - QueueBuffer buffer2 = channels.getBuffer(1); - buffer2.obtain(result); - assertEquals(200, result.size()); - } - - @Test - public void testBlockingProduce() { - final DataCarrier carrier = new DataCarrier<>(2, 100); - - for (int i = 0; i < 200; i++) { - assertTrue(carrier.produce(new SampleData().setName("d" + i))); - } - - long time1 = System.currentTimeMillis(); - new Thread(() -> { - try { - Thread.sleep(3000); - } catch (InterruptedException e) { - e.printStackTrace(); - } - IConsumer consumer = new IConsumer() { - @Override - public void consume(List data) { - - } - - @Override - public void onError(List data, Throwable t) { - - } - }; - carrier.consume(consumer, 1); - }).start(); - - carrier.produce(new SampleData().setName("blocking-data")); - long time2 = System.currentTimeMillis(); - - assertTrue(time2 - time1 > 2000); - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/EnvUtilTest.java b/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/EnvUtilTest.java deleted file mode 100644 index a432df78fec3..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/EnvUtilTest.java +++ /dev/null @@ -1,54 +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.skywalking.oap.server.library.datacarrier; - -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; -import org.junit.jupiter.api.extension.ExtendWith; -import uk.org.webcompere.systemstubs.environment.EnvironmentVariables; -import uk.org.webcompere.systemstubs.jupiter.SystemStub; -import uk.org.webcompere.systemstubs.jupiter.SystemStubsExtension; - -import static org.junit.jupiter.api.Assertions.assertEquals; - -@ExtendWith(SystemStubsExtension.class) -public class EnvUtilTest { - @SystemStub - private final EnvironmentVariables environmentVariables = new EnvironmentVariables(); - - @BeforeEach - public void before() { - environmentVariables.set("myInt", "123"); - environmentVariables.set("wrongInt", "wrong123"); - environmentVariables.set("myLong", "12345678901234567"); - environmentVariables.set("wrongLong", "wrong123"); - } - - @Test - public void getInt() { - assertEquals(123, EnvUtil.getInt("myInt", 234)); - assertEquals(234, EnvUtil.getLong("wrongInt", 234)); - } - - @Test - public void getLong() { - assertEquals(12345678901234567L, EnvUtil.getLong("myLong", 123L)); - assertEquals(987654321987654321L, EnvUtil.getLong("wrongLong", 987654321987654321L)); - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/SampleData.java b/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/SampleData.java deleted file mode 100644 index 58611f06a408..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/SampleData.java +++ /dev/null @@ -1,43 +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.skywalking.oap.server.library.datacarrier; - -public class SampleData { - private int intValue; - - private String name; - - public int getIntValue() { - return intValue; - } - - public String getName() { - return name; - } - - public SampleData setIntValue(int intValue) { - this.intValue = intValue; - return this; - } - - public SampleData setName(String name) { - this.name = name; - return this; - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumeDriverTest.java b/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumeDriverTest.java deleted file mode 100644 index 9ccf0fd15acc..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumeDriverTest.java +++ /dev/null @@ -1,56 +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.skywalking.oap.server.library.datacarrier.consumer; - -import org.apache.skywalking.oap.server.library.datacarrier.SampleData; -import org.apache.skywalking.oap.server.library.datacarrier.buffer.BufferStrategy; -import org.apache.skywalking.oap.server.library.datacarrier.buffer.Channels; -import org.apache.skywalking.oap.server.library.datacarrier.partition.SimpleRollingPartitioner; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.powermock.reflect.Whitebox; - -public class ConsumeDriverTest { - @Test - public void testBeginConsumeDriver() { - Channels channels = new Channels(2, 100, new SimpleRollingPartitioner(), BufferStrategy.BLOCKING); - ConsumeDriver pool = new ConsumeDriver("default", channels, new SampleConsumer(), 2, 20); - pool.begin(channels); - - ConsumerThread[] threads = Whitebox.getInternalState(pool, "consumerThreads"); - Assertions.assertEquals(2, threads.length); - Assertions.assertTrue(threads[0].isAlive()); - Assertions.assertTrue(threads[1].isAlive()); - } - - @Test - public void testCloseConsumeDriver() throws InterruptedException, IllegalAccessException { - Channels channels = new Channels(2, 100, new SimpleRollingPartitioner(), BufferStrategy.BLOCKING); - ConsumeDriver pool = new ConsumeDriver("default", channels, new SampleConsumer(), 2, 20); - pool.begin(channels); - - Thread.sleep(5000); - pool.close(channels); - ConsumerThread[] threads = Whitebox.getInternalState(pool, "consumerThreads"); - - Assertions.assertEquals(2, threads.length); - Assertions.assertFalse((Boolean) Whitebox.getInternalState(threads[0], "running")); - Assertions.assertFalse((Boolean) Whitebox.getInternalState(threads[1], "running")); - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerPoolFactoryTest.java b/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerPoolFactoryTest.java deleted file mode 100644 index ef4b2e7e62ce..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerPoolFactoryTest.java +++ /dev/null @@ -1,48 +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.skywalking.oap.server.library.datacarrier.consumer; - -import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.Test; - -import static org.junit.jupiter.api.Assertions.assertNotNull; -import static org.junit.jupiter.api.Assertions.assertNull; -import static org.junit.jupiter.api.Assertions.assertTrue; - -public class ConsumerPoolFactoryTest { - - @BeforeEach - public void createIfAbsent() throws Exception { - BulkConsumePool.Creator creator = new BulkConsumePool.Creator("my-test-pool", 10, 20, false); - boolean firstCreated = ConsumerPoolFactory.INSTANCE.createIfAbsent("my-test-pool", creator); - assertTrue(firstCreated); - - boolean secondCreated = ConsumerPoolFactory.INSTANCE.createIfAbsent("my-test-pool", creator); - assertTrue(!secondCreated); - } - - @Test - public void get() { - ConsumerPool consumerPool = ConsumerPoolFactory.INSTANCE.get("my-test-pool"); - assertNotNull(consumerPool); - - ConsumerPool notExist = ConsumerPoolFactory.INSTANCE.get("not-exists-pool"); - assertNull(notExist); - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerTest.java b/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerTest.java deleted file mode 100644 index 193221c57313..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/ConsumerTest.java +++ /dev/null @@ -1,126 +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.skywalking.oap.server.library.datacarrier.consumer; - -import org.apache.skywalking.oap.server.library.datacarrier.DataCarrier; -import org.apache.skywalking.oap.server.library.datacarrier.SampleData; -import org.junit.jupiter.api.Assertions; -import org.junit.jupiter.api.Test; -import org.powermock.reflect.Whitebox; - -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.concurrent.LinkedBlockingQueue; - -public class ConsumerTest { - public static LinkedBlockingQueue BUFFER = new LinkedBlockingQueue<>(); - - public static boolean IS_OCCUR_ERROR = false; - - @Test - public void testConsumerLessThanChannel() throws IllegalAccessException { - - final DataCarrier carrier = new DataCarrier<>(2, 100); - - for (int i = 0; i < 100; i++) { - Assertions.assertTrue(carrier.produce(new SampleData().setName("data" + i))); - } - SampleConsumer consumer = new SampleConsumer(); - - consumer.i = 100; - carrier.consume(SampleConsumer.class, 1); - Assertions.assertEquals(1, ((SampleConsumer) getConsumer(carrier)).i); - - SampleConsumer2 consumer2 = new SampleConsumer2(); - consumer2.i = 100; - carrier.consume(consumer2, 1); - Assertions.assertEquals(100, ((SampleConsumer2) getConsumer(carrier)).i); - - carrier.shutdownConsumers(); - } - - @Test - public void testConsumerMoreThanChannel() throws InterruptedException { - BUFFER.drainTo(new ArrayList()); - final DataCarrier carrier = new DataCarrier(2, 100); - - for (int i = 0; i < 200; i++) { - Assertions.assertTrue(carrier.produce(new SampleData().setName("data" + i))); - } - SampleConsumer consumer = new SampleConsumer(); - - carrier.consume(SampleConsumer.class, 5); - - Thread.sleep(2000); - - List result = new ArrayList(); - BUFFER.drainTo(result); - - Assertions.assertEquals(200, result.size()); - - HashSet consumerCounter = new HashSet(); - for (SampleData data : result) { - consumerCounter.add(data.getIntValue()); - } - Assertions.assertEquals(2, consumerCounter.size()); - } - - @Test - public void testConsumerOnError() throws InterruptedException { - final DataCarrier carrier = new DataCarrier(2, 100); - - for (int i = 0; i < 200; i++) { - Assertions.assertTrue(carrier.produce(new SampleData().setName("data" + i))); - } - SampleConsumer2 consumer = new SampleConsumer2(); - - consumer.onError = true; - carrier.consume(consumer, 5); - - Thread.sleep(3 * 1000L); - - Assertions.assertTrue(IS_OCCUR_ERROR); - } - - class SampleConsumer2 implements IConsumer { - public int i = 1; - - public boolean onError = false; - - @Override - public void consume(List data) { - if (onError) { - throw new RuntimeException("consume exception"); - } - } - - @Override - public void onError(List data, Throwable t) { - IS_OCCUR_ERROR = true; - } - } - - private IConsumer getConsumer(DataCarrier carrier) throws IllegalAccessException { - ConsumeDriver pool = Whitebox.getInternalState(carrier, "driver"); - ConsumerThread[] threads = Whitebox.getInternalState(pool, "consumerThreads"); - - return Whitebox.getInternalState(threads[0], "consumer"); - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/SampleConsumer.java b/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/SampleConsumer.java deleted file mode 100644 index 4d433fc713ef..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/consumer/SampleConsumer.java +++ /dev/null @@ -1,39 +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.skywalking.oap.server.library.datacarrier.consumer; - -import java.util.List; -import org.apache.skywalking.oap.server.library.datacarrier.SampleData; - -public class SampleConsumer implements IConsumer { - public int i = 1; - - @Override - public void consume(List data) { - for (SampleData one : data) { - one.setIntValue(this.hashCode()); - ConsumerTest.BUFFER.offer(one); - } - } - - @Override - public void onError(List data, Throwable t) { - - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/partition/ProducerThreadPartitionerTest.java b/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/partition/ProducerThreadPartitionerTest.java deleted file mode 100644 index 845e4b20071c..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/partition/ProducerThreadPartitionerTest.java +++ /dev/null @@ -1,35 +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.skywalking.oap.server.library.datacarrier.partition; - -import org.apache.skywalking.oap.server.library.datacarrier.SampleData; -import org.junit.jupiter.api.Test; - -import static org.junit.jupiter.api.Assertions.assertEquals; - -public class ProducerThreadPartitionerTest { - @Test - public void testPartition() { - int partitionNum = (int) Thread.currentThread().getId() % 10; - ProducerThreadPartitioner partitioner = new ProducerThreadPartitioner(); - assertEquals(partitioner.partition(10, new SampleData()), partitionNum); - assertEquals(partitioner.partition(10, new SampleData()), partitionNum); - assertEquals(partitioner.partition(10, new SampleData()), partitionNum); - } -} diff --git a/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/partition/SimpleRollingPartitionerTest.java b/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/partition/SimpleRollingPartitionerTest.java deleted file mode 100644 index 1f8304bc208a..000000000000 --- a/oap-server/server-library/library-datacarrier-queue/src/test/java/org/apache/skywalking/oap/server/library/datacarrier/partition/SimpleRollingPartitionerTest.java +++ /dev/null @@ -1,34 +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.skywalking.oap.server.library.datacarrier.partition; - -import org.apache.skywalking.oap.server.library.datacarrier.SampleData; -import org.junit.jupiter.api.Test; - -import static org.junit.jupiter.api.Assertions.assertEquals; - -public class SimpleRollingPartitionerTest { - @Test - public void testPartition() { - SimpleRollingPartitioner partitioner = new SimpleRollingPartitioner(); - assertEquals(partitioner.partition(10, new SampleData()), 0); - assertEquals(partitioner.partition(10, new SampleData()), 1); - assertEquals(partitioner.partition(10, new SampleData()), 2); - } -} diff --git a/oap-server/server-library/pom.xml b/oap-server/server-library/pom.xml index 3505d2289330..ba375584b9c6 100644 --- a/oap-server/server-library/pom.xml +++ b/oap-server/server-library/pom.xml @@ -33,7 +33,6 @@ library-util library-client library-elasticsearch-client - library-datacarrier-queue library-kubernetes-support library-async-profiler-jfr-parser library-pprof-parser diff --git a/oap-server/server-starter/src/main/resources/otel-rules/oap.yaml b/oap-server/server-starter/src/main/resources/otel-rules/oap.yaml index 56b1cf9f71f6..223c6e0cf450 100644 --- a/oap-server/server-starter/src/main/resources/otel-rules/oap.yaml +++ b/oap-server/server-starter/src/main/resources/otel-rules/oap.yaml @@ -69,7 +69,7 @@ metricsRules: metrics_aggregation.tagEqual('dimensionality', 'minute').sum(['service', 'host_name', 'level']).increase('PT1M') .tag({tags -> if (tags['level'] == '1') {tags.level = 'L1 aggregation'} }).tag({tags -> if (tags['level'] == '2') {tags.level = 'L2 aggregation'} }) - name: instance_metrics_aggregation_queue_used_percentage - exp: metrics_aggregation_queue_used_percentage.sum(['service', 'host_name', 'level', 'kind', 'metricName']) + exp: metrics_aggregation_queue_used_percentage.sum(['service', 'host_name', 'level', 'slot']) - name: instance_persistence_execute_percentile exp: persistence_timer_bulk_execute_latency.sum(['le', 'service', 'host_name']).increase('PT1M').histogram().histogram_percentile([50,75,90,95,99]) - name: instance_persistence_prepare_percentile diff --git a/oap-server/server-starter/src/main/resources/ui-initialized-templates/so11y_oap/so11y-instance.json b/oap-server/server-starter/src/main/resources/ui-initialized-templates/so11y_oap/so11y-instance.json index 0e630c917a1a..cfba96f7c0be 100644 --- a/oap-server/server-starter/src/main/resources/ui-initialized-templates/so11y_oap/so11y-instance.json +++ b/oap-server/server-starter/src/main/resources/ui-initialized-templates/so11y_oap/so11y-instance.json @@ -470,7 +470,7 @@ "i": "22", "type": "Widget", "expressions": [ - "sort_values(meter_oap_instance_metrics_aggregation_queue_used_percentage{level='1',kind='OAL'},10,des,avg)" + "sort_values(meter_oap_instance_metrics_aggregation_queue_used_percentage{level='1'},10,des,avg)" ], "graph": { "type": "Line", @@ -481,7 +481,7 @@ "showYAxis": true }, "widget": { - "title": "OAL L1 Aggregation Queue Percentage (%)" + "title": "L1 Aggregation Queue Percentage (%)" } }, { @@ -492,51 +492,7 @@ "i": "23", "type": "Widget", "expressions": [ - "sort_values(meter_oap_instance_metrics_aggregation_queue_used_percentage{level='1',kind='MAL'},10,des,avg)" - ], - "widget": { - "title": "MAL L1 Aggregation Queue Percentage (%)" - }, - "graph": { - "type": "Line", - "step": false, - "smooth": false, - "showSymbol": true, - "showXAxis": true, - "showYAxis": true - } - }, - { - "x": 12, - "y": 65, - "w": 12, - "h": 13, - "i": "24", - "type": "Widget", - "graph": { - "type": "Line", - "step": false, - "smooth": false, - "showSymbol": true, - "showXAxis": true, - "showYAxis": true - }, - "expressions": [ - "sort_values(meter_oap_instance_metrics_aggregation_queue_used_percentage{level='2',kind='OAL'},10,des,avg)" - ], - "widget": { - "title": "OAL L2 Aggregation Queue Percentage (%)" - } - }, - { - "x": 0, - "y": 78, - "w": 12, - "h": 13, - "i": "25", - "type": "Widget", - "expressions": [ - "sort_values(meter_oap_instance_metrics_aggregation_queue_used_percentage{level='2',kind='MAL'},10,des,avg)" + "sort_values(meter_oap_instance_metrics_aggregation_queue_used_percentage{level='2'},10,des,avg)" ], "graph": { "type": "Line", @@ -547,15 +503,15 @@ "showYAxis": true }, "widget": { - "title": "MAL L2 Aggregation Queue Percentage (%)" + "title": "L2 Aggregation Queue Percentage (%)" } }, { "x": 12, - "y": 78, + "y": 65, "w": 12, "h": 13, - "i": "26", + "i": "24", "type": "Widget", "graph": { "type": "Line", @@ -574,10 +530,10 @@ }, { "x": 0, - "y": 91, + "y": 78, "w": 12, "h": 13, - "i": "27", + "i": "25", "type": "Widget", "expressions": [ "sort_values(meter_oap_instance_metrics_persistent_collection_cached_size{dimensionality='minute',kind='MAL'},10,des,avg)" diff --git a/oap-server/server-storage-plugin/storage-banyandb-plugin/pom.xml b/oap-server/server-storage-plugin/storage-banyandb-plugin/pom.xml index 7dbb49b705e9..14558eca719d 100644 --- a/oap-server/server-storage-plugin/storage-banyandb-plugin/pom.xml +++ b/oap-server/server-storage-plugin/storage-banyandb-plugin/pom.xml @@ -39,11 +39,6 @@ library-client ${project.version} - - org.apache.skywalking - library-datacarrier-queue - ${project.version} - org.apache.skywalking library-banyandb-client diff --git a/oap-server/server-storage-plugin/storage-jdbc-hikaricp-plugin/pom.xml b/oap-server/server-storage-plugin/storage-jdbc-hikaricp-plugin/pom.xml index 3285d110a4e1..17d6c7fdb763 100644 --- a/oap-server/server-storage-plugin/storage-jdbc-hikaricp-plugin/pom.xml +++ b/oap-server/server-storage-plugin/storage-jdbc-hikaricp-plugin/pom.xml @@ -41,7 +41,7 @@ org.apache.skywalking - library-datacarrier-queue + library-batch-queue ${project.version} diff --git a/oap-server/server-storage-plugin/storage-jdbc-hikaricp-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/jdbc/common/dao/JDBCBatchDAO.java b/oap-server/server-storage-plugin/storage-jdbc-hikaricp-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/jdbc/common/dao/JDBCBatchDAO.java index 563859eb89d7..5896ed98bed4 100644 --- a/oap-server/server-storage-plugin/storage-jdbc-hikaricp-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/jdbc/common/dao/JDBCBatchDAO.java +++ b/oap-server/server-storage-plugin/storage-jdbc-hikaricp-plugin/src/main/java/org/apache/skywalking/oap/server/storage/plugin/jdbc/common/dao/JDBCBatchDAO.java @@ -18,38 +18,47 @@ package org.apache.skywalking.oap.server.storage.plugin.jdbc.common.dao; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.CompletableFuture; +import java.util.function.Function; +import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; import org.apache.skywalking.oap.server.core.storage.IBatchDAO; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueue; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueConfig; +import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueManager; +import org.apache.skywalking.oap.server.library.batchqueue.PartitionPolicy; +import org.apache.skywalking.oap.server.library.batchqueue.ThreadPolicy; import org.apache.skywalking.oap.server.library.client.jdbc.hikaricp.JDBCClient; import org.apache.skywalking.oap.server.library.client.request.InsertRequest; import org.apache.skywalking.oap.server.library.client.request.PrepareRequest; -import org.apache.skywalking.oap.server.library.datacarrier.DataCarrier; -import org.apache.skywalking.oap.server.library.datacarrier.consumer.IConsumer; import org.apache.skywalking.oap.server.library.util.CollectionUtils; import org.apache.skywalking.oap.server.storage.plugin.jdbc.BatchSQLExecutor; import org.apache.skywalking.oap.server.storage.plugin.jdbc.SQLExecutor; -import java.util.ArrayList; -import java.util.List; -import java.util.concurrent.CompletableFuture; -import java.util.function.Function; -import java.util.stream.Collectors; - @Slf4j public class JDBCBatchDAO implements IBatchDAO { private final JDBCClient jdbcClient; - private final DataCarrier dataCarrier; + private final BatchQueue queue; private final int maxBatchSqlSize; public JDBCBatchDAO(JDBCClient jdbcClient, int maxBatchSqlSize, int asyncBatchPersistentPoolSize) { this.jdbcClient = jdbcClient; - String name = "H2_ASYNCHRONOUS_BATCH_PERSISTENT"; if (log.isDebugEnabled()) { - log.debug("H2_ASYNCHRONOUS_BATCH_PERSISTENT poolSize: {}, maxBatchSqlSize:{}", asyncBatchPersistentPoolSize, maxBatchSqlSize); + log.debug("JDBC_ASYNC_BATCH_PERSISTENT poolSize: {}, maxBatchSqlSize:{}", asyncBatchPersistentPoolSize, maxBatchSqlSize); } this.maxBatchSqlSize = maxBatchSqlSize; - this.dataCarrier = new DataCarrier<>(name, asyncBatchPersistentPoolSize, 10000); - this.dataCarrier.consume(new H2BatchConsumer(this), asyncBatchPersistentPoolSize, 20); + this.queue = BatchQueueManager.create( + "JDBC_ASYNC_BATCH_PERSISTENT", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(asyncBatchPersistentPoolSize)) + .partitions(PartitionPolicy.fixed(asyncBatchPersistentPoolSize)) + .bufferSize(10_000) + .consumer(batch -> flush(batch)) + .maxIdleMs(20) + .build() + ); } @Override @@ -89,25 +98,6 @@ public CompletableFuture flush(List prepareRequests) { @Override public void insert(InsertRequest insertRequest) { - this.dataCarrier.produce(insertRequest); - } - - private static class H2BatchConsumer implements IConsumer { - - private final JDBCBatchDAO h2BatchDAO; - - private H2BatchConsumer(JDBCBatchDAO h2BatchDAO) { - this.h2BatchDAO = h2BatchDAO; - } - - @Override - public void consume(List prepareRequests) { - h2BatchDAO.flush(prepareRequests); - } - - @Override - public void onError(List prepareRequests, Throwable t) { - log.error(t.getMessage(), t); - } + this.queue.produce(insertRequest); } } From e720cf21038e191ba6ca8faa0cf1eecf131c2a3c Mon Sep 17 00:00:00 2001 From: Wu Sheng Date: Sun, 15 Feb 2026 09:21:07 +0800 Subject: [PATCH 03/16] Add CLAUDE.md and DESIGN.md for library-batch-queue module CLAUDE.md documents the module's design principles, architecture, dispatch modes, scheduler modes, key classes, and current usage across the codebase. DESIGN.md describes the throughput-weighted partition rebalancing feature: two-phase handoff protocol with cycle-count fencing for safe concurrent-free reassignment, targeting L2 persistence (primary) and L1 aggregation (secondary) queues. Co-Authored-By: Claude Opus 4.6 --- .../library-batch-queue/CLAUDE.md | 166 +++ .../library-batch-queue/DESIGN.md | 1318 +++-------------- 2 files changed, 352 insertions(+), 1132 deletions(-) create mode 100644 oap-server/server-library/library-batch-queue/CLAUDE.md diff --git a/oap-server/server-library/library-batch-queue/CLAUDE.md b/oap-server/server-library/library-batch-queue/CLAUDE.md new file mode 100644 index 000000000000..4b13a2999a0a --- /dev/null +++ b/oap-server/server-library/library-batch-queue/CLAUDE.md @@ -0,0 +1,166 @@ +# library-batch-queue + +A partitioned, self-draining queue with type-based dispatch. Replaces the legacy `DataCarrier` across the OAP server. + +## Core Design Principles + +1. **Describe workload, not threads.** Callers declare intent (`cpuCores(1.0)`, `adaptive()`) and the queue resolves concrete thread/partition counts at runtime. +2. **One queue per concern, many types per queue.** Metrics aggregation, persistence, and export each get one shared queue. Individual metric types register handlers and share the queue's thread pool. +3. **Partition-level isolation.** Each partition is an independent `ArrayBlockingQueue`. The default `typeHash` selector routes all items of the same class to the same partition, so drain-loop dispatch grouping is effectively free. +4. **Adaptive backoff.** Idle drain loops double their sleep interval (`minIdleMs * 2^idleCount`, capped at `maxIdleMs`), resetting on first non-empty drain. No busy-waiting. + +## Architecture + +``` +Producer threads Drain threads (scheduler) + | | + | produce(data) | drainLoop(taskIndex) + | | | | + | +-- select partition (typeHash) | +-- drainTo(combined) from assigned partitions + | +-- put/offer into partition | +-- dispatch(combined) + | | | + | | +-- single consumer? -> consumer.consume(batch) + | | +-- handler map? -> group by class, handler.consume(subBatch) + | | +-- loop until empty, then re-schedule with backoff +``` + +## Two Dispatch Modes + +### Single consumer mode +Set `config.consumer(handler)`. The entire drained batch goes to one callback. No class-based grouping. +Use for: homogeneous queues where all items are the same type (JDBC batch, single exporter). + +### Handler map mode +Call `queue.addHandler(TypeA.class, handlerA)` per type. Drained items are grouped by `getClass()` and dispatched to matching handlers. Unregistered types are logged and dropped. +Use for: shared queues where many metric types coexist (L1 aggregation, L2 persistence, TopN). + +## Scheduler Modes + +### Dedicated scheduler +The queue owns a `ScheduledThreadPool`. Each thread is assigned a fixed subset of partitions (round-robin). Multiple threads drain concurrently. + +```java +BatchQueueConfig.builder() + .threads(ThreadPolicy.cpuCores(1.0)) // own thread pool + .partitions(PartitionPolicy.adaptive()) + ... +``` + +### Shared scheduler +Multiple queues share one `ScheduledThreadPool` (reference-counted, auto-shutdown). Each queue gets 1 drain task on the shared pool. Useful for low-throughput I/O queues. + +```java +BatchQueueConfig.builder() + .sharedScheduler("exporter", ThreadPolicy.fixed(1)) // shared pool + .partitions(PartitionPolicy.fixed(1)) + ... +``` + +## Key Classes + +| Class | Role | +|-------|------| +| `BatchQueue` | The queue itself. Holds partitions, runs drain loops, dispatches to consumers/handlers. | +| `BatchQueueManager` | Global registry. Creates queues by name, manages shared schedulers with ref-counting. | +| `BatchQueueConfig` | Builder for queue configuration (threads, partitions, buffer, strategy, consumer). | +| `ThreadPolicy` | Resolves thread count: `fixed(N)`, `cpuCores(mult)`, `cpuCoresWithBase(base, mult)`. | +| `PartitionPolicy` | Resolves partition count: `fixed(N)`, `threadMultiply(N)`, `adaptive()`. | +| `PartitionSelector` | Routes items to partitions. Default `typeHash()` groups by class. | +| `HandlerConsumer` | Callback for processing a batch. Has optional `onIdle()` for flush-on-idle. | +| `BufferStrategy` | `BLOCKING` (put, waits) or `IF_POSSIBLE` (offer, drops if full). | +| `BatchQueueStats` | Point-in-time snapshot of queue usage. `totalUsed()`, `topN(n)`, per-partition stats. | +| `QueueErrorHandler` | Optional error callback. If absent, errors are logged. | + +## ThreadPolicy + +```java +ThreadPolicy.fixed(4) // exactly 4 threads +ThreadPolicy.cpuCores(1.0) // 1 thread per CPU core +ThreadPolicy.cpuCoresWithBase(1, 0.25) // 1 + 0.25 * cores (e.g., 3 on 8-core) +``` + +Always resolves to >= 1. + +## PartitionPolicy + +```java +PartitionPolicy.fixed(4) // exactly 4 partitions +PartitionPolicy.threadMultiply(2) // 2 * thread count +PartitionPolicy.adaptive() // grows with addHandler() calls +``` + +Adaptive growth (default multiplier 25, with 8 threads -> threshold 200): +- 0 handlers -> 8 partitions (= thread count) +- 100 handlers -> 100 partitions (1:1) +- 500 handlers -> 350 partitions (200 + 300/2) + +## Usage in the Codebase + +### L1 Metrics Aggregation (`MetricsAggregateWorker`) +``` +threads: cpuCores(1.0) -- 8 threads on 8-core +partitions: adaptive() -- grows with metric types (~460 for typical OAL+MAL) +bufferSize: 20,000 per partition +strategy: IF_POSSIBLE +idleMs: 1..50 +mode: handler map (one handler per metric class) +``` + +### L2 Metrics Persistence (`MetricsPersistentMinWorker`) +``` +threads: cpuCoresWithBase(1, 0.25) -- 3 threads on 8-core +partitions: adaptive() -- grows with metric types +bufferSize: 2,000 per partition +strategy: IF_POSSIBLE +idleMs: 1..50 +mode: handler map (one handler per metric class) +``` + +### TopN Persistence (`TopNWorker`) +``` +threads: fixed(1) +partitions: adaptive() -- grows with TopN types +bufferSize: 1,000 per partition +strategy: IF_POSSIBLE +idleMs: 10..100 +mode: handler map (one handler per TopN class) +``` + +### gRPC Remote Client (`GRPCRemoteClient`) +``` +threads: fixed(1) +partitions: fixed(1) +bufferSize: configurable (channelSize * bufferSize) +strategy: BLOCKING +idleMs: 1..50 +mode: single consumer (sends over gRPC stream) +``` + +### Exporters (gRPC metrics, Kafka trace, Kafka log) +``` +threads: fixed(1) each +partitions: fixed(1) each +bufferSize: configurable (default 20,000) +strategy: BLOCKING (gRPC) / IF_POSSIBLE (Kafka) +idleMs: 1..200 +mode: single consumer +``` + +### JDBC Batch DAO (`JDBCBatchDAO`) +``` +threads: fixed(N) where N = asyncBatchPersistentPoolSize (default 4) +partitions: fixed(N) (1 partition per thread) +bufferSize: 10,000 per partition +strategy: BLOCKING +idleMs: 1..20 +mode: single consumer (JDBC batch flush) +``` + +## Lifecycle + +1. `BatchQueueManager.create(name, config)` -- creates and starts drain loops immediately +2. `queue.addHandler(type, handler)` -- registers type handler (adaptive: may grow partitions) +3. `queue.produce(data)` -- routes to partition, blocks or drops per strategy +4. Drain loops run continuously, dispatching batches to consumers/handlers +5. `BatchQueueManager.shutdown(name)` -- stops drain, final flush, releases scheduler +6. `BatchQueueManager.shutdownAll()` -- called during OAP server shutdown diff --git a/oap-server/server-library/library-batch-queue/DESIGN.md b/oap-server/server-library/library-batch-queue/DESIGN.md index bc107dbbaffc..e5525c9d0b41 100644 --- a/oap-server/server-library/library-batch-queue/DESIGN.md +++ b/oap-server/server-library/library-batch-queue/DESIGN.md @@ -1,1205 +1,259 @@ -# library-batch-queue Design Proposal +# Throughput-Weighted Partition Rebalancing -## Goal +## Problem -Replace `library-datacarrier-queue` with a unified, simpler batch queue library that reduces thread -usage while preserving all required capabilities. +`BatchQueue` assigns partitions to drain threads with a static round-robin mapping +(`buildAssignments`). Combined with `typeHash()` partition selection, each metric +class is pinned to exactly one partition and one drain thread. -## Problem Statement +In a typical SkyWalking deployment, OAL metrics generate far more data than MAL +metrics. With hundreds of metric types of varying throughput, the static assignment +creates **unbalanced drain threads**: some threads are overloaded with hot OAL +partitions while others are nearly idle draining cold MAL partitions. -DataCarrier has two consumption modes with separate code paths: +The imbalance is invisible for low-throughput queues (exporters, TopN, JDBC) but +significant for **L1 aggregation** and **L2 persistence** queues, which have +`cpuCores(1.0)` or more threads and `adaptive()` partitions scaling to hundreds. -1. **Simple mode** (`ConsumeDriver`): Each DataCarrier gets dedicated consumer thread(s). - Used by TopNWorker, GRPCRemoteClient, JDBCBatchDAO, exporters. -2. **Pool mode** (`BulkConsumePool`): Multiple DataCarriers share a thread pool. - Used by MetricsAggregateWorker (L1) and MetricsPersistentMinWorker (L2). +## Design: Periodic Throughput-Weighted Reassignment -On an 8-core production machine, this creates **47+ DataCarrier consumer threads**: +A background rebalancer periodically measures per-partition throughput, then +reassigns partitions to threads to equalize total load per thread. -| Source | Threads | Mode | -|--------------------------------|---------|--------| -| L1 OAL aggregation pool | 24 | Pool | -| L2 OAL persistent pool | 2 | Pool | -| L1 MAL aggregation pool | 2 | Pool | -| L2 MAL persistent pool | 1 | Pool | -| TopNWorker (per type) | 5-10 | Simple | -| GRPCRemoteClient (per peer) | 2-4 | Simple | -| JDBCBatchDAO | 2-4 | Simple | -| Exporters (gRPC/Kafka) | 0-3 | Simple | -| **Total** | **~38-48** | | - -Key issues: -- Simple mode wastes threads: each queue gets a dedicated thread even though most are idle - (sleeping in 200ms polling loop). -- Pool mode creates one DataCarrier (with Channels) per metric type. With 100+ metric types, - pool threads must iterate through all assigned channels even when most are empty — wasted CPU. -- Two completely separate code paths for the same produce-consume pattern. -- Pool mode has static assignment — no rebalancing after initial allocation. - -## Design - -### Architecture Overview +### Data flow overview ``` - BatchQueueManager (global singleton registry + lazy shared schedulers) - │ - │ Shared schedulers (created lazily on first queue reference): - │ ├── "IO_POOL" ──> ScheduledExecutorService (cpuCores(0.5) → 4 threads on 8-core) - │ │ Created when first queue calls sharedScheduler("IO_POOL", cpuCores(0.5)). - │ │ Shared by all I/O queues: - │ │ - GRPCRemoteClient.* (gRPC to peer OAP nodes) - │ │ - GRPCMetricsExporter (gRPC metrics export) - │ │ - KafkaLogExporter (Kafka log export) - │ │ - KafkaTraceExporter (Kafka trace export) - │ │ - JDBCBatchDAO (JDBC batch writes) - │ │ - │ - │ Queues: - │ - ├── "METRICS_L1_AGGREGATION" ──> BatchQueue - │ │ threads: cpuCores(1.0) → 8 on 8-core - │ │ partitions: threadMultiply(2) → 16 on 8-core - │ │ strategy: IF_POSSIBLE - │ │ handlerMap: { ServiceRespTimeMetrics.class -> handler-A (OAL), - │ │ ServiceCpmMetrics.class -> handler-B (OAL), - │ │ MeterMetrics_xxx.class -> handler-C (MAL), ... } - │ └── OAL and MAL metrics share the same L1 queue and thread pool. - │ - ├── "METRICS_L2_PERSISTENT" ──> BatchQueue - │ │ threads: cpuCores(0.25) → 2 on 8-core - │ │ partitions: threadMultiply(2) → 4 on 8-core - │ │ strategy: BLOCKING - │ │ handlerMap: { ServiceRespTimeMetrics.class -> handler-D (OAL), - │ │ MeterMetrics_xxx.class -> handler-E (MAL), ... } - │ └── OAL and MAL metrics share the same L2 queue and thread pool. - │ - ├── "TOPN_WORKER" ──> BatchQueue - │ │ threads: fixed(1) - │ │ partitions: fixed(1) - │ │ strategy: BLOCKING - │ │ handlerMap: { DatabaseSlowStatement.class -> handler-F, - │ │ DatabaseSlowSql.class -> handler-G, ... } - │ └── drain: drainTo → groupBy(class) → dispatch to handler - │ (TopN is in-memory ranking computation — all types share one thread) - │ - ├── "GRPCRemoteClient.peer1" ──> BatchQueue (shared="IO_POOL", partitions=1, BLOCKING) - │ │ scheduler: shared "IO_POOL" - │ │ consumer: direct consumer for RemoteMessage - │ └── drain: drainTo → direct consumer - │ - └── "JDBCBatchDAO" ──> BatchQueue (shared="IO_POOL", partitions=1, BLOCKING) - │ scheduler: shared "IO_POOL" - │ consumer: direct consumer for PrepareRequest - └── same +Producer threads Drain threads Rebalancer (periodic) + | | | + | produce(data) | drainLoop(taskIndex) | every rebalanceIntervalMs: + | | | | | 1. snapshot throughput counters + | +-- typeHash() | +-- read partitionOwner 2. reset counters + | +-- put/offer | | skip if != me | 3. LPT assign partitions + | +-- bump counter | +-- drainTo + dispatch | 4. two-phase handoff + | | +-- bump cycleCount | ``` -### Core Concept - -**One queue type, one config, two scheduler modes.** - -- **`BatchQueueManager`** is the global singleton registry. It also manages named shared - schedulers for low-throughput queues. Users call `createIfAbsent(name, config)` to - get a named `BatchQueue`. -- **`BatchQueue`** has N partitions (configurable, default 1) and a handler map. - Producers round-robin data into partitions. On drain, each batch is **grouped by - message class** and dispatched to the handler registered for that class. -- **Handler registration** via `queue.addHandler(Class, HandlerConsumer)`. - Each worker provides its own handler instance for its specific type. - -The handler map pattern works the same way regardless of partition count: -- `threadMultiply(2)` with 100+ handlers → metrics aggregation (many types, shared partitions) -- `partitions=1` with N handlers → TopN (multiple types, low throughput, shared 1 thread) -- `partitions=1` with 1 consumer → I/O queue (gRPC client, exporter, JDBC) +### Throughput counters -No need for separate queue classes. The difference is just configuration. - -### Why Shared Partitions + Handler Map - -In the old BulkConsumePool model with 100+ metric types: - -``` -Pool Thread-0 assigned: [service_resp_time channels, service_cpm channels, ...] -Pool Thread-1 assigned: [endpoint_resp_time channels, endpoint_cpm channels, ...] -... -Each thread iterates ALL assigned channels per cycle, even if most are empty. -``` - -In the new model: - -``` -Partition-0: mixed data from all metric types (round-robin) -Partition-1: mixed data from all metric types -... -Partition-N: mixed data from all metric types - -On drain of Partition-K: - batch = drainTo(list) // all data, mixed types - grouped = batch.groupBy(item.getClass()) // group by metric class - for each (class, items) in grouped: - handler = handlerMap.get(class) // lookup registered handler - handler.consume(items) // dispatch to the right worker -``` - -Benefits: -- Partitions are created based on parallelism needs, not metric count. - 16 partitions (8 threads * 2) serve 100+ metric types. -- No empty channel iteration — every partition gets data. -- Handlers are registered on-demand. Adding a new metric type is just - `addHandler(NewMetrics.class, handlerInstance)`. -- Each handler still processes only its own metric type's data in isolation. -- I/O queues use the same structure with `partitions=1` and a direct consumer. - -### API +Each partition has an `AtomicLong` counter, incremented on every `produce()` call. +The rebalancer snapshots and resets all counters each interval. ```java -// ── Metrics aggregation (dedicated pool, many types, handler map dispatch) ── - -BatchQueue l1Queue = BatchQueueManager.createIfAbsent( - "METRICS_L1_AGGREGATION", - BatchQueueConfig.builder() - .threads(ThreadPolicy.cpuCores(1.0)) // 1x CPU cores (e.g. 8 on 8-core) - .partitions(PartitionPolicy.threadMultiply(2)) // 2x resolved threads = 16 on 8-core - .bufferSize(10_000) - .strategy(BufferStrategy.IF_POSSIBLE) - .errorHandler((data, t) -> log.error(t.getMessage(), t)) - .build() -); - -// Each MetricsAggregateWorker registers its inner class handler for its metric class. -// Called per metric type in MetricsStreamProcessor.create() (100+ times): -l1Queue.addHandler(metricsClass, new L1Handler()); // L1Handler is worker's inner class - -// Produce — data goes into a partition by round-robin -// Adaptive backoff ensures fast re-poll (minIdleMs) when data is flowing. -l1Queue.produce(metricsData); +AtomicLongArray partitionThroughput; // one slot per partition -// ── TopN (shared queue — all TopN types share one thread, handler map dispatch) ── - -BatchQueue topNQueue = BatchQueueManager.createIfAbsent( - "TOPN_WORKER", - BatchQueueConfig.builder() - .threads(ThreadPolicy.fixed(1)) // all TopN types share 1 thread - .partitions(PartitionPolicy.fixed(1)) - .bufferSize(1000) - .strategy(BufferStrategy.BLOCKING) - .errorHandler((data, t) -> log.error(t.getMessage(), t)) - .build() -); - -// Each TopNWorker registers its handler for its specific TopN class (5-10 types): -topNQueue.addHandler(topNClass, new TopNHandler()); // TopNHandler is worker's inner class - -topNQueue.produce(topNData); - -// ── I/O queues (shared scheduler, single consumer) ── -// sharedScheduler() specifies both name and ThreadPolicy. BatchQueueManager creates the -// shared ScheduledExecutorService on first reference, reuses it for subsequent queues. -// No separate createSharedScheduler() call needed — no startup ordering dependency. -// -// All these queues share the same "IO_POOL" scheduler: -// GRPCRemoteClient.* — gRPC streaming to peer OAP nodes -// GRPCMetricsExporter — gRPC metrics export -// KafkaLogExporter — Kafka log export -// KafkaTraceExporter — Kafka trace export -// JDBCBatchDAO — JDBC batch writes to database - -BatchQueue grpcQueue = BatchQueueManager.createIfAbsent( - "GRPCRemoteClient.peer1", - BatchQueueConfig.builder() - .sharedScheduler("IO_POOL", ThreadPolicy.cpuCores(0.5)) // creates IO_POOL on first use - .partitions(PartitionPolicy.fixed(1)) - .bufferSize(10_000) - .strategy(BufferStrategy.BLOCKING) - .consumer(new RemoteMessageHandler()) - .errorHandler((data, t) -> log.error(t.getMessage(), t)) - .build() -); - -// Another queue referencing the same "IO_POOL" — reuses existing scheduler. -// If ThreadPolicy differs from the first creator, logs a warning (first one wins). -BatchQueue jdbcQueue = BatchQueueManager.createIfAbsent( - "JDBCBatchDAO", - BatchQueueConfig.builder() - .sharedScheduler("IO_POOL", ThreadPolicy.cpuCores(0.5)) // reuses existing IO_POOL - .partitions(PartitionPolicy.fixed(1)) - .bufferSize(10_000) - .strategy(BufferStrategy.BLOCKING) - .consumer(new JDBCBatchHandler()) - .errorHandler((data, t) -> log.error(t.getMessage(), t)) - .build() -); +// In produce(), after the put/offer: +partitionThroughput.incrementAndGet(index); +``` -// ── Lifecycle ── +The counter is on the produce path, which is already doing an `ArrayBlockingQueue.put/offer`. +A single `incrementAndGet` adds negligible overhead (no contention — each metric type +hashes to a fixed partition, so each partition's counter is written by a predictable +set of producer threads). -BatchQueueManager.shutdown("METRICS_L1_AGGREGATION"); -BatchQueueManager.shutdownAll(); // shuts down all queues + shared schedulers -``` +### Rebalance algorithm (LPT — Longest Processing Time) -### Classes +The rebalancer runs on the queue's scheduler (one extra scheduled task). It uses the +classic **LPT multiprocessor scheduling** heuristic: ``` -library-batch-queue/ - src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ - BatchQueueManager.java — Global singleton registry + named shared schedulers - BatchQueue.java — Partitioned queue with dedicated or shared scheduler + handler map dispatch - BatchQueueConfig.java — Builder: threads/sharedScheduler, partitions, bufferSize, strategy - ThreadPolicy.java — Fixed or CPU-relative thread count: fixed(N) / cpuCores(double) - PartitionPolicy.java — Fixed or thread-relative partition count: fixed(N) / threadMultiply(N) - HandlerConsumer.java — Interface: void consume(List), default void onIdle() - BufferStrategy.java — BLOCKING / IF_POSSIBLE - QueueErrorHandler.java — Functional interface: void onError(List, Throwable) +1. snapshot = partitionThroughput[0..N-1] +2. reset all counters to 0 +3. sort partitions by snapshot[p] descending +4. threadLoad = long[taskCount], all zeros +5. newAssignment = List[taskCount] +6. for each partition p in sorted order: + t = argmin(threadLoad) // thread with least total load + newAssignment[t].add(p) + threadLoad[t] += snapshot[p] +7. two-phase handoff (see below) ``` -### BatchQueueManager - -```java -/** - * Global registry for batch queues and shared schedulers. - * Thread-safe. Queues are created by name and shared across modules. - * - * Two scheduler modes: - * - Dedicated: queue creates its own ScheduledExecutorService (for high-throughput queues). - * Configured via BatchQueueConfig.threads(ThreadPolicy). - * - Shared: queue uses a named shared scheduler managed by this manager (for low-throughput queues). - * Configured via BatchQueueConfig.sharedScheduler(name, ThreadPolicy). - * Multiple queues referencing the same name share one ScheduledExecutorService. - * - * Shared schedulers are created lazily on first queue reference — no separate - * setup step needed. This eliminates startup ordering dependencies. - */ -public class BatchQueueManager { - private static final ConcurrentHashMap> queues = new ConcurrentHashMap<>(); - private static final ConcurrentHashMap sharedSchedulers = new ConcurrentHashMap<>(); - - /** - * Get or create a shared scheduler. Called internally by BatchQueue constructor - * when config specifies sharedScheduler(name, threads). - * - * - First call with a given name: creates the ScheduledExecutorService using - * threads.resolve() and caches it. - * - Subsequent calls with the same name: returns the cached scheduler. - * If the ThreadPolicy differs, logs a warning (first one wins). - * - * Thread-safe (ConcurrentHashMap.computeIfAbsent). - * - * Shared schedulers are owned by BatchQueueManager, NOT by any individual queue. - * They are destroyed only by shutdownAll() — never by individual queue shutdown. - */ - static ScheduledExecutorService getOrCreateSharedScheduler(String name, ThreadPolicy threads); +LPT is O(P log P) for sorting + O(P log T) for assignment (with a min-heap for +threadLoad). For 500 partitions and 8 threads, this is sub-millisecond. - /** - * Create a new named queue. Throws if name already exists. - */ - public static BatchQueue create(String name, BatchQueueConfig config); +If a partition has zero throughput in the last interval, it keeps its previous +assignment (no unnecessary moves). - /** - * Create if not present. Returns existing queue if name is taken. - * - * If the queue already exists, validates consistency: - * - Consumption mode conflict: throws IllegalStateException if the existing queue - * uses direct consumer mode (config.consumer set) but the new config does not, - * or vice versa. These two modes are mutually exclusive per queue. - * - Infrastructure settings: logs a warning if threads, partitions, - * bufferSize, or strategy differ between the existing and new config. - */ - public static BatchQueue createIfAbsent(String name, BatchQueueConfig config); +### Two-phase handoff protocol - /** - * Get an existing queue by name. Returns null if not found. - */ - public static BatchQueue get(String name); +Reassigning a partition from Thread-A to Thread-B while Thread-A is mid-dispatch +creates a **concurrent handler invocation** — two threads calling the same +`HandlerConsumer.consume()` on different batches simultaneously. For L1 aggregation, +`MergableBufferedData` is not thread-safe, so this corrupts state. - /** - * Shutdown and remove a single queue by name. - * - Dedicated scheduler: shut down together with the queue. - * - Shared scheduler: NOT shut down. It is owned by BatchQueueManager - * and may still be used by other queues. - */ - public static void shutdown(String name); +The race condition: - /** - * Shutdown all queues and all shared schedulers. Called during OAP server shutdown. - * - * Order: - * 1. Set running=false on all queues (stops drain loops from rescheduling) - * 2. Final drain of remaining data in each queue - * 3. Shut down all dedicated schedulers - * 4. Shut down all shared schedulers - * 5. Clear both registries - */ - public static void shutdownAll(); -} ``` - -### BatchQueue - -```java -/** - * A partitioned queue with handler-map-based dispatch. - * - * The scheduler is either dedicated (owned by this queue) or shared - * (managed by BatchQueueManager, shared with other queues). - * - * Partitions are created based on configured parallelism (default 1). - * Producers round-robin data across partitions. - * On drain, each batch is grouped by message class and dispatched to the - * registered handler for that class. - * - * Works uniformly for all use cases: - * - shared scheduler, partitions=1, one consumer → I/O queue (gRPC, Kafka, JDBC) - * - dedicated fixed(1), partitions=1, many handlers → TopN (all types share 1 thread) - * - dedicated cpuCores(1.0), threadMultiply(2), many handlers → metrics aggregation - */ -public class BatchQueue { - private final String name; - private final ScheduledExecutorService scheduler; - private final boolean dedicatedScheduler; // true = owned by this queue, false = shared - private final ArrayBlockingQueue[] partitions; - private final ConcurrentHashMap, HandlerConsumer> handlerMap; - private final BatchQueueConfig config; - private final AtomicInteger roundRobinIndex = new AtomicInteger(0); - private volatile boolean running; - - /** - * Partition assignment per drain task. Each drain task owns a set of partition indices. - * - * Dedicated mode: one drain task per thread, partitions assigned round-robin. - * threads=4, partitions=8: task[0]→[0,4], task[1]→[1,5], task[2]→[2,6], task[3]→[3,7] - * - * Shared mode: single drain task covering ALL partitions (partitions typically = 1). - * assignedPartitions = { [0] } (one task, one partition) - */ - private final int[][] assignedPartitions; - - /** - * Per-task count of consecutive idle cycles (all assigned partitions empty). - * Used to compute adaptive backoff sleep interval. - */ - private final int[] consecutiveIdleCycles; - - BatchQueue(String name, BatchQueueConfig config) { - this.name = name; - this.config = config; - this.handlerMap = new ConcurrentHashMap<>(); - - int taskCount; - if (config.getSharedSchedulerName() != null) { - // Shared scheduler mode: get-or-create shared scheduler from BatchQueueManager. - ScheduledExecutorService sharedScheduler = BatchQueueManager.getOrCreateSharedScheduler( - config.getSharedSchedulerName(), config.getSharedSchedulerThreads()); - - int partitionCount = config.getPartitions().resolve(1, 0); - this.partitions = new ArrayBlockingQueue[partitionCount]; - for (int i = 0; i < partitions.length; i++) { - partitions[i] = new ArrayBlockingQueue<>(config.getBufferSize()); - } - - this.scheduler = sharedScheduler; - this.dedicatedScheduler = false; - taskCount = 1; - this.assignedPartitions = new int[][] { - java.util.stream.IntStream.range(0, partitions.length).toArray() - }; - } else { - // Dedicated scheduler mode: resolve threads and partitions. - int threadCount = config.getThreads().resolve(); // cpuCores(1.0) → 8 on 8-core - int partitionCount = config.getPartitions().resolve(threadCount, 0); - - // Validation: if partitions < threads, cut threads to match and warn. - if (partitionCount < threadCount) { - log.warn("BatchQueue[{}]: partitions({}) < threads({}), " - + "reducing threads to {}", - name, partitionCount, threadCount, partitionCount); - threadCount = partitionCount; - } - - this.partitions = new ArrayBlockingQueue[partitionCount]; - for (int i = 0; i < partitions.length; i++) { - partitions[i] = new ArrayBlockingQueue<>(config.getBufferSize()); - } - - this.scheduler = Executors.newScheduledThreadPool( - threadCount, - new ThreadFactoryBuilder().setNameFormat("BatchQueue-" + name + "-%d").build() - ); - this.dedicatedScheduler = true; - taskCount = threadCount; - - // Assign partitions to threads by round-robin. - // threads=4, partitions=8: task[0]→[0,4], task[1]→[1,5], ... - // threads=4, threadMultiply(2)=8: same - // threads=8, partitions=8: task[0]→[0], task[1]→[1], ... - this.assignedPartitions = new int[taskCount][]; - List> assignment = new ArrayList<>(); - for (int t = 0; t < taskCount; t++) { - assignment.add(new ArrayList<>()); - } - for (int p = 0; p < partitions.length; p++) { - assignment.get(p % taskCount).add(p); - } - for (int t = 0; t < taskCount; t++) { - assignedPartitions[t] = assignment.get(t).stream().mapToInt(Integer::intValue).toArray(); - } - } - - // Kick off one self-rescheduling drain task per assignment. - this.consecutiveIdleCycles = new int[taskCount]; - this.running = true; - for (int t = 0; t < taskCount; t++) { - scheduleDrain(t); - } - } - - /** - * Schedule the next drain with adaptive backoff. - * - * Idle count 0 (just had data): sleep = minIdleMs (e.g. 5ms) - * Idle count 1: sleep = minIdleMs * 2 (e.g. 10ms) - * Idle count 2: sleep = minIdleMs * 4 (e.g. 20ms) - * ... - * Idle count N: sleep = min(minIdleMs * 2^N, maxIdleMs) - */ - private void scheduleDrain(int taskIndex) { - int idleCount = consecutiveIdleCycles[taskIndex]; - long delay = Math.min( - config.getMinIdleMs() * (1L << Math.min(idleCount, 20)), - config.getMaxIdleMs() - ); - scheduler.schedule( - () -> drainLoop(taskIndex), - delay, - TimeUnit.MILLISECONDS - ); - } - - /** - * Register a handler for a specific message class type. - * Multiple metric types can each register their own handler instance. - */ - public void addHandler(Class type, HandlerConsumer handler); - - /** - * Produce data into a partition (round-robin). - * BLOCKING: waits if the selected partition is full (queue.put). - * IF_POSSIBLE: returns false if full (queue.offer). - */ - public boolean produce(T data) { - int index = Math.abs(roundRobinIndex.getAndIncrement() % partitions.length); - if (config.getStrategy() == BufferStrategy.BLOCKING) { - partitions[index].put(data); // blocks until space available - return true; - } else { - return partitions[index].offer(data); // returns false if full - } - } - - /** - * Drain loop for one thread: iterates ALL assigned partitions in a round-robin loop. - * Only stops when ALL assigned partitions are empty in a full cycle. - * - * Each cycle: - * 1. Drain ALL assigned partitions into one combined batch - * 2. If combined batch is empty → all partitions empty → onIdle, break - * 3. dispatch(combined batch) → handlers get ALL data of their type as one list - * 4. Loop back to step 1 (more data may have arrived during dispatch) - * - * Example: Thread-0 owns partitions [0, 4] - * cycle 1: drain(0)→[A1,B1,A2] drain(4)→[C1,A3] - * combined = [A1,B1,A2,C1,A3] - * dispatch → groupBy class: - * handlerA.consume([A1,A2,A3]) ← all A's in one call - * handlerB.consume([B1]) - * handlerC.consume([C1]) - * cycle 2: drain(0)→[A4] drain(4)→[] - * combined = [A4] (not empty → dispatch) - * cycle 3: drain(0)→[] drain(4)→[] - * combined = [] → onIdle, reschedule - */ - void drainLoop(int taskIndex) { - int[] myPartitions = assignedPartitions[taskIndex]; - try { - while (running) { - // Step 1: drain ALL assigned partitions into one batch - List combined = new ArrayList<>(); - for (int partitionIndex : myPartitions) { - partitions[partitionIndex].drainTo(combined); - } - - // Step 2: if nothing across all partitions, we are idle - if (combined.isEmpty()) { - consecutiveIdleCycles[taskIndex]++; - notifyIdle(); - break; // reschedule with backoff - } - - // Had data → reset backoff so next reschedule is fast - consecutiveIdleCycles[taskIndex] = 0; - - // Step 3: dispatch the combined batch - dispatch(combined); - // Step 4: loop immediately — more data may have arrived - } - } finally { - if (running) { - scheduleDrain(taskIndex); - } - } - } - - void shutdown() { - running = false; - // Final drain of remaining data across all partitions - List combined = new ArrayList<>(); - for (int i = 0; i < partitions.length; i++) { - partitions[i].drainTo(combined); - } - if (!combined.isEmpty()) { - dispatch(combined); - } - // Only shut down the scheduler if this queue owns it. - // Shared schedulers are shut down by BatchQueueManager.shutdownAll(). - if (dedicatedScheduler) { - scheduler.shutdown(); - } - } -} +Thread-A Rebalancer Thread-B +──────── ────────── ──────── +drainTo(P3) → 500 items +dispatch(batch): + handler_X.consume(500) owner[P3] = B + ← still running drainTo(P3) → 200 new items + dispatch(batch): + handler_X.consume(200) + ← CONCURRENT! handler_X corrupted ``` -**Dispatch and idle notification:** - -```java -private void dispatch(List batch) { - // Direct consumer mode: pass the whole batch, no groupBy overhead. - if (config.getConsumer() != null) { - try { - config.getConsumer().consume(batch); - } catch (Throwable t) { - config.getErrorHandler().onError(batch, t); - } - return; - } - - // Handler map mode: group by class type and dispatch to registered handlers. - Map, List> grouped = new HashMap<>(); - for (T item : batch) { - grouped.computeIfAbsent(item.getClass(), k -> new ArrayList<>()).add(item); - } +A simple ownership gate (`if partitionOwner[p] != me: skip`) prevents the new owner +from **draining** the partition. But the old owner already drained the data and is +still **dispatching** to the handler. The new owner would drain new items and call +the same handler concurrently. - for (Map.Entry, List> entry : grouped.entrySet()) { - HandlerConsumer handler = handlerMap.get(entry.getKey()); - if (handler != null) { - try { - handler.consume(entry.getValue()); - } catch (Throwable t) { - config.getErrorHandler().onError(entry.getValue(), t); - } - } - } -} +The fix is a two-phase handoff with a cycle-count fence: -private void notifyIdle() { - if (config.getConsumer() != null) { - config.getConsumer().onIdle(); - } else { - handlerMap.values().forEach(HandlerConsumer::onIdle); - } -} ``` - -**Consumer workflow (end-to-end):** - +Phase 1 — Revoke: + partitionOwner[p] = UNOWNED (-1) + // Thread-A sees UNOWNED on next drainTo check, skips P3. + // But Thread-A may be mid-dispatch right now — handler still in use. + + Wait: spin until cycleCount[oldTask] > snapshot + // Once the counter increments, Thread-A has finished its current + // drain+dispatch cycle. The handler is no longer being called. + +Phase 2 — Assign: + partitionOwner[p] = newTaskIndex + // Thread-B picks up P3 on its next drain cycle. Safe — no concurrent handler call. ``` -Producer threads Consumer thread (Thread-0, owns partitions [0, 4]) -────────────── ───────────────────────────────────────────────── -produce(A1) ──offer/put──► Partition-0: [A1, C1, B1] -produce(B1) ──offer/put──► Partition-4: [A2, A3] -produce(C1) ──┘ (round-robin) +During the gap between phases, new items accumulate in partition P3 but are not lost. +Thread-B drains them once Phase 2 completes. - ┌─── scheduleDrain(0) after adaptive backoff delay - │ - ▼ - drainLoop(taskIndex=0) - │ - ┌────────┴───────────────────────────────────────────┐ - │ while (running): │ - │ │ - │ ── Step 1: drain ALL assigned partitions ── │ - │ combined = [] │ - │ Partition-0.drainTo(combined) → [A1, C1, B1] │ - │ Partition-4.drainTo(combined) → [A1,C1,B1,A2,A3]│ - │ │ - │ ── Step 2: check if empty ── │ - │ combined not empty → dispatch │ - │ │ - │ ── Step 3: dispatch(combined) ── │ - │ ┌─ config.consumer set? │ - │ │ YES → consumer.consume(combined) ─── done │ - │ │ NO → groupBy class: │ - │ │ MetricA.class → [A1, A2, A3] │ - │ │ MetricB.class → [B1] │ - │ │ MetricC.class → [C1] │ - │ │ for each group: │ - │ │ handler = handlerMap.get(class) │ - │ │ handler.consume(group) │ - │ │ ↓ │ - │ │ L1Handler_A.consume([A1, A2, A3]) │ - │ │ → workerA.onWork([A1, A2, A3]) │ - │ │ → mergeDataCache.accept(A1) │ - │ │ → mergeDataCache.accept(A2) │ - │ │ → mergeDataCache.accept(A3) │ - │ │ → flush() if period elapsed │ - │ │ L1Handler_B.consume([B1]) │ - │ │ → workerB.onWork([B1]) │ - │ │ L1Handler_C.consume([C1]) │ - │ │ → workerC.onWork([C1]) │ - │ └───────────────────────────────────────────── │ - │ │ - │ ── Step 4: loop immediately (more may have come)──│ - │ │ - │ ── next cycle ── │ - │ combined = [] │ - │ Partition-0.drainTo(combined) → [] │ - │ Partition-4.drainTo(combined) → [] │ - │ combined is empty │ - │ → notifyIdle() │ - │ handlerMap.values().forEach(::onIdle) │ - │ ↓ │ - │ L1Handler_A.onIdle() │ - │ → workerA.flush() (force flush cache) │ - │ L1Handler_B.onIdle() │ - │ → workerB.flush() │ - │ ... │ - │ consecutiveIdleCycles[0]++ (e.g. now = 1) │ - │ break │ - └────────────────────────────────────────────────────┘ - │ - ▼ (finally block) - scheduleDrain(0) - │ - ├─ idleCount=0 (just had data): wait 5ms ← fast re-poll - ├─ idleCount=1: wait 10ms - ├─ idleCount=2: wait 20ms - ├─ idleCount=3: wait 40ms - ├─ idleCount=4: wait 80ms - ├─ idleCount=5: wait 160ms - ├─ idleCount=6+: wait 200ms ← capped at maxIdleMs - │ - ▼ - drainLoop again - │ - (if data found → idleCount resets to 0 → back to fast polling) -``` - -**Key points:** -- Each cycle drains ALL assigned partitions into one combined batch before dispatching. -- `dispatch()` is called once per cycle with all data from all partitions combined. -- In handler map mode, `groupBy(class)` collects all items of the same type across all - partitions into one list. The handler receives ALL available data of its type in a single - `consume()` call — e.g., `[A1, A2, A3]` not three separate calls. -- The handler's `consume()` runs **synchronously** inside the drain thread. The handler - (an inner class of the worker) directly accesses the worker's fields — merge cache, - counters, flush logic — with no extra threading. -- If any partition had data, loop immediately to check all partitions again. -- `onIdle()` fires only when ALL assigned partitions are empty in a full cycle, giving - handlers a chance to flush periodic caches (e.g., L1 aggregation merge cache → nextWorker). -- **Adaptive backoff**: after data, re-poll in `minIdleMs` (5ms). Each consecutive empty - cycle doubles the sleep, capping at `maxIdleMs` (200ms). Data resets to fast polling. - -**Two consumption modes, same queue class:** -- **Direct consumer** (`config.consumer` set) — whole batch goes to one handler, no groupBy. - Use for I/O queues where all data is the same type (gRPC, Kafka, JDBC). -- **Handler map** (`addHandler` called) — batch grouped by class, dispatched per type. - Use for metrics aggregation (L1/L2) and TopN with many types sharing partitions. +### Cycle counter -If both are set, direct consumer takes priority (handler map is ignored). -If neither is set, data is drained but silently dropped. - -### BatchQueueConfig +Each drain task increments its cycle counter at the end of every drain cycle (after +dispatch completes, before re-scheduling). The rebalancer reads this counter to know +when a task has finished any in-flight work. ```java -@Builder -public class BatchQueueConfig { - /** - * Thread policy for a dedicated ScheduledExecutorService. - * Resolved at queue construction time. Examples: - * ThreadPolicy.fixed(8) → always 8 threads - * ThreadPolicy.cpuCores(1.0) → 1x available CPU cores (8 on 8-core) - * ThreadPolicy.cpuCores(0.25) → 0.25x CPU cores (2 on 8-core, min 1) - * - * When set, the queue creates its own scheduler. - * When null, sharedScheduler must be set — the queue uses a shared scheduler. - * - * Use dedicated pools for high-throughput queues (metrics L1/L2 aggregation) - * where you need guaranteed thread capacity. - */ - private ThreadPolicy threads; - - /** - * Shared scheduler name and its ThreadPolicy. Set via the builder method - * sharedScheduler(name, threads) which populates both fields together. - * Mutually exclusive with the threads field above. - * - * When set, the queue registers its drain tasks on a shared - * ScheduledExecutorService managed by BatchQueueManager. - * The shared scheduler is created lazily on first queue reference — - * no separate setup step needed. Subsequent queues with the same name - * reuse the existing scheduler (ThreadPolicy mismatch logs a warning). - * - * Use for low-throughput I/O queues (gRPC, Kafka, JDBC) to reduce OS thread count. - * Multiple queues sharing the same scheduler name share the same thread pool. - * - * Exactly one of threads or sharedScheduler must be set. - */ - private String sharedSchedulerName; - private ThreadPolicy sharedSchedulerThreads; - - /** - * Number of partitions, or a policy to derive it from resolved thread count. - * - * Can be set as: - * - Absolute: fixed(8) → exactly 8 partitions. - * - Relative: threadMultiply(2) → 2x resolved thread count. - * e.g. cpuCores(0.5) on 8-core = 4 threads, threadMultiply(2) → 8 partitions. - * - * Default: fixed(1). - * - * Validation (applied at construction time for dedicated scheduler mode): - * - If partitions < resolved thread count, thread count is reduced to match - * partitions and a warning is logged. No point having more threads than partitions. - */ - @Builder.Default - private PartitionPolicy partitions = PartitionPolicy.fixed(1); - - /** - * Buffer size per partition. - */ - @Builder.Default - private int bufferSize = 10_000; - - /** - * BLOCKING: producer waits when buffer full. - * IF_POSSIBLE: producer gets false when buffer full. - */ - @Builder.Default - private BufferStrategy strategy = BufferStrategy.BLOCKING; - - /** - * Direct consumer for the whole batch. When set, all drained data goes to this - * handler without class-based grouping. Takes priority over handler map. - * - * Use this for I/O queues where all data is the same type (gRPC, Kafka, JDBC). - * Leave null to use handler map dispatch via addHandler(). - */ - private HandlerConsumer consumer; - - /** - * Called when a handler throws during consume. - */ - private QueueErrorHandler errorHandler; +AtomicLongArray cycleCount; // one slot per drain task - /** - * Minimum idle sleep in milliseconds. Default 5ms. - * Used as the base interval when data was recently consumed. - * The thread re-polls quickly to catch new data with low latency. - */ - @Builder.Default - private long minIdleMs = 5; - - /** - * Maximum idle sleep in milliseconds. Default 200ms. - * After several consecutive empty cycles, the sleep interval backs off - * exponentially from minIdleMs up to this cap. - * - * Backoff: sleep = min(minIdleMs * 2^consecutiveIdleCycles, maxIdleMs) - * idle 0: 5ms → idle 1: 10ms → idle 2: 20ms → idle 3: 40ms - * → idle 4: 80ms → idle 5: 160ms → idle 6+: 200ms (capped) - */ - @Builder.Default - private long maxIdleMs = 200; +void drainLoop(int taskIndex) { + // ... drain assigned partitions, dispatch ... + cycleCount.incrementAndGet(taskIndex); + // re-schedule } ``` -### HandlerConsumer +The rebalancer uses it in the revoke phase: ```java -/** - * Handler for processing a batch of data for a specific type. - * Each metric type (or I/O queue user) provides its own handler instance. - */ -public interface HandlerConsumer { - /** - * Process a batch of data belonging to this handler's type. - */ - void consume(List data); +void movePartition(int p, int oldTask, int newTask) { + partitionOwner.set(p, UNOWNED); - /** - * Called when there is nothing to consume. Can be used as a timer trigger - * (e.g. flush L1 aggregation cache periodically). - */ - default void onIdle() { + long snapshot = cycleCount.get(oldTask); + while (cycleCount.get(oldTask) <= snapshot) { + LockSupport.parkNanos(TimeUnit.MILLISECONDS.toNanos(1)); } -} -``` - -### ThreadPolicy - -```java -/** - * Determines the number of threads for a BatchQueue's dedicated scheduler - * or for a shared scheduler created via BatchQueueManager. - * - * Two modes: - * - fixed(N): exactly N threads, regardless of hardware. - * - cpuCores(multiplier): multiplier * Runtime.availableProcessors(), rounded. - * - * Resolved value is always >= 1 — every pool must have at least one thread. - * fixed() requires count >= 1 at construction. cpuCores() applies max(1, ...) at resolution. - */ -public class ThreadPolicy { - private final int fixedCount; // > 0 for fixed mode, 0 for cpuCores mode - private final double cpuMultiplier; // > 0 for cpuCores mode, 0 for fixed mode - - /** - * Fixed number of threads. Count must be >= 1. - * Example: fixed(1) → always 1 thread. - * fixed(8) → always 8 threads. - * Throws IllegalArgumentException if count < 1. - */ - public static ThreadPolicy fixed(int count); - - /** - * Threads = multiplier * available CPU cores, rounded, min 1. - * Multiplier must be > 0. - * Example on 8-core machine: - * cpuCores(1.0) → 8 threads - * cpuCores(0.5) → 4 threads - * cpuCores(0.25) → 2 threads - * cpuCores(2.0) → 16 threads - * Example on 2-core machine: - * cpuCores(0.25) → 1 thread (min 1, never 0) - */ - public static ThreadPolicy cpuCores(double multiplier); - - /** - * Resolve the actual thread count. Always returns >= 1. - * For fixed mode, returns fixedCount. - * For cpuCores mode, returns max(1, round(cpuMultiplier * availableProcessors())). - */ - public int resolve(); -} -``` - -### PartitionPolicy - -```java -/** - * Determines the number of partitions for a BatchQueue. - * - * Two modes: - * - fixed(N): exactly N partitions, regardless of thread count. - * - threadMultiply(N): N * resolved thread count. - * - adaptive(): partition count grows with registered handlers. - * Threshold = threadCount * multiplier (default 25). - * Below threshold: 1:1 (one partition per handler). - * Above threshold: excess at 1:2 ratio. - * - * All policies resolved via resolve(threadCount, handlerCount). - * At queue construction time, if partitions < resolved thread count, - * thread count is reduced to match and a warning is logged. - */ -public class PartitionPolicy { - private final int fixedCount; // > 0 for fixed mode - private final int multiplier; // > 0 for threadMultiply/adaptive - private final boolean adaptive; // true for adaptive mode - - public static PartitionPolicy fixed(int count); - public static PartitionPolicy threadMultiply(int multiplier); - public static PartitionPolicy adaptive(); - public static PartitionPolicy adaptive(int multiplier); - /** - * Resolve the actual partition count. - * - fixed: returns fixedCount (both params ignored). - * - threadMultiply: returns multiplier * resolvedThreadCount. - * - adaptive: handlerCount == 0 → resolvedThreadCount; - * handlerCount <= threshold → handlerCount (1:1); - * handlerCount > threshold → threshold + (excess / 2). - */ - public int resolve(int resolvedThreadCount, int handlerCount); + partitionOwner.set(p, newTask); } ``` -### Implementing HandlerConsumer +**Worst-case wait:** one `maxIdleMs` (the drain task may be sleeping in backoff). +For L1 aggregation this is 50ms, for L2 persistence 50ms. Since rebalancing runs +every few minutes, this latency is negligible. -Each worker creates a handler as an **inner class** that directly accesses the worker's -fields (merge cache, telemetry counters, flush logic, etc.). The handler instance is -registered per metric class — one handler per worker, one worker per metric type. - -#### L1 Aggregation (MetricsAggregateWorker) - -Current code uses an inner `AggregatorConsumer` that calls `onWork()` and `flush()`: +### Partition ownership array ```java -// Current: inner class IConsumer accesses outer worker fields -private class AggregatorConsumer implements IConsumer { - public void consume(List data) { - MetricsAggregateWorker.this.onWork(data); // accesses mergeDataCache - } - public void nothingToConsume() { - flush(); // accesses lastSendTime, nextWorker - } -} +AtomicIntegerArray partitionOwner; // partitionOwner[p] = taskIndex that owns it, or -1 ``` -New code — same pattern, just implements `HandlerConsumer` instead of `IConsumer`. -Each `MetricsAggregateWorker` instance creates its own handler and registers it for -its specific metric class: +The drain loop checks ownership before draining each partition: ```java -public class MetricsAggregateWorker extends AbstractWorker { - private final MergableBufferedData mergeDataCache; - private final AbstractWorker nextWorker; - private final BatchQueue l1Queue; - - MetricsAggregateWorker(ModuleDefineHolder moduleDefineHolder, - AbstractWorker nextWorker, - String modelName, - Class metricsClass, - ...) { - this.nextWorker = nextWorker; - this.mergeDataCache = new MergableBufferedData<>(); - - // Get or create the shared L1 queue (idempotent) - this.l1Queue = BatchQueueManager.createIfAbsent( - "METRICS_L1_AGGREGATION", - BatchQueueConfig.builder() - .threads(ThreadPolicy.cpuCores(1.0)) // 1x CPU cores - .partitions(PartitionPolicy.threadMultiply(2)) // 2x resolved threads - .bufferSize(10_000) - .strategy(BufferStrategy.IF_POSSIBLE) - .errorHandler((data, t) -> log.error(t.getMessage(), t)) - .build() - ); - - // Register this worker's handler for its specific metric class. - // The inner class directly accesses mergeDataCache, nextWorker, etc. - l1Queue.addHandler(metricsClass, new L1Handler()); - } - - @Override - public void in(Metrics metrics) { - l1Queue.produce(metrics); - } - - private void onWork(List data) { - data.forEach(mergeDataCache::accept); - flush(); - } - - private void flush() { - if (System.currentTimeMillis() - lastSendTime > l1FlushPeriod) { - mergeDataCache.read().forEach(nextWorker::in); - lastSendTime = System.currentTimeMillis(); - } - } - - // Inner class handler — accesses worker fields directly - private class L1Handler implements HandlerConsumer { - @Override - public void consume(List data) { - MetricsAggregateWorker.this.onWork(data); - } - - @Override - public void onIdle() { - MetricsAggregateWorker.this.flush(); +void drainLoop(int taskIndex) { + for (int p : assignedPartitions[taskIndex]) { + if (partitionOwner.get(p) != taskIndex) { + continue; // revoked or not yet assigned } + partitions[p].drainTo(combined); } + dispatch(combined); + cycleCount.incrementAndGet(taskIndex); } ``` -Key point: 100+ `MetricsAggregateWorker` instances are created (one per metric type, -both OAL and MAL), each registers its own `L1Handler` inner class instance on the -same shared queue. Handler map dispatch routes each metric class to its own worker's -`mergeDataCache` — OAL and MAL handlers coexist in the same queue without interference. +The `partitionOwner` check is one `AtomicIntegerArray.get()` per partition per drain +cycle — a volatile read with no CAS. This is the only overhead on the hot path. -#### L2 Persistent (MetricsPersistentMinWorker) +### Configuration -Same pattern — inner class handler accesses the worker's `onWork()` method: +Rebalancing is opt-in via `BatchQueueConfig`: ```java -public class MetricsPersistentMinWorker extends MetricsPersistentWorker { - private final BatchQueue l2Queue; - - MetricsPersistentMinWorker(..., Class metricsClass, ...) { - super(...); - - this.l2Queue = BatchQueueManager.createIfAbsent( - "METRICS_L2_PERSISTENT", - BatchQueueConfig.builder() - .threads(ThreadPolicy.cpuCores(0.25)) // 0.25x CPU cores (2 on 8-core) - .partitions(PartitionPolicy.threadMultiply(2)) // 2x resolved threads - .bufferSize(10_000) - .strategy(BufferStrategy.BLOCKING) - .errorHandler((data, t) -> log.error(t.getMessage(), t)) - .build() - ); - - l2Queue.addHandler(metricsClass, new L2Handler()); - } - - @Override - public void in(Metrics metrics) { - l2Queue.produce(metrics); - } - - // Inner class handler — accesses worker's onWork, queuePercentageGauge, etc. - private class L2Handler implements HandlerConsumer { - @Override - public void consume(List data) { - queuePercentageGauge.setValue(...); - MetricsPersistentMinWorker.this.onWork(data); - } - } -} +BatchQueueConfig.builder() + .threads(ThreadPolicy.cpuCores(1.0)) + .partitions(PartitionPolicy.adaptive()) + .rebalanceIntervalMs(300_000) // every 5 minutes, 0 = disabled (default) + .build(); ``` -#### TopN (TopNWorker — shared queue, handler map dispatch) - -All TopN types share one queue. Same handler map pattern as L1/L2 — each TopNWorker -registers its inner class handler for its specific TopN class: - -```java -public class TopNWorker extends PersistenceWorker { - private final BatchQueue topNQueue; - - TopNWorker(..., Class topNClass, ...) { - // Get or create the shared TopN queue (idempotent) - this.topNQueue = BatchQueueManager.createIfAbsent( - "TOPN_WORKER", - BatchQueueConfig.builder() - .threads(ThreadPolicy.fixed(1)) // all TopN types share 1 thread - .partitions(PartitionPolicy.fixed(1)) - .bufferSize(1000) - .strategy(BufferStrategy.BLOCKING) - .errorHandler((data, t) -> log.error(t.getMessage(), t)) - .build() - ); - - // Register this worker's handler for its specific TopN class - topNQueue.addHandler(topNClass, new TopNHandler()); - } - - @Override - public void in(TopN topN) { - topNQueue.produce(topN); - } +Only meaningful for dedicated-scheduler queues with multiple threads and partitions. +Silently ignored for single-thread or shared-scheduler queues. - // Inner class — accesses worker's limitedSizeBufferedData, reportPeriod, etc. - private class TopNHandler implements HandlerConsumer { - @Override - public void consume(List data) { - TopNWorker.this.onWork(data); - } +### Full rebalance cycle - @Override - public void onIdle() { - TopNWorker.this.flushIfNeeded(); - } - } -} +``` +1. rebalanceTask fires (scheduled every rebalanceIntervalMs) + | +2. snapshot all partitionThroughput[p], reset to 0 + | +3. skip rebalance if throughput is uniform (max/min ratio < 1.5) + | +4. LPT assignment: sort partitions by throughput desc, + assign each to the least-loaded thread + | +5. diff against current assignedPartitions + | only partitions that changed owner need handoff + | +6. for each moved partition: + | Phase 1: partitionOwner[p] = UNOWNED + | +7. for each moved partition: + | wait: cycleCount[oldTask] > snapshot_before_revoke + | +8. for each moved partition: + | Phase 2: partitionOwner[p] = newTask + | +9. update assignedPartitions (volatile write) + | +10. log summary: "rebalanced N partitions, max thread load delta: X%" ``` -### Thread Reduction - -Thread counts scale with CPU cores via ThreadPolicy. OAL and MAL share the same L1/L2 pools. -Example on 8-core machine: - -| Before (DataCarrier) | Threads | After (BatchQueue) | Threads (8-core) | -|---------------------------------|---------|-------------------------------------------------|------------------| -| L1 OAL pool | 24 | METRICS_L1_AGGREGATION (cpuCores(1.0)) | 8 | -| L1 MAL pool | 2 | *(shared with L1 above)* | *(shared)* | -| L2 OAL pool | 2 | METRICS_L2_PERSISTENT (cpuCores(0.25)) | 2 | -| L2 MAL pool | 1 | *(shared with L2 above)* | *(shared)* | -| TopNWorker (5-10 types) | 5-10 | TOPN_WORKER (fixed(1), handler map) | 1 | -| GRPCRemoteClient (2-4 peers) | 2-4 | GRPCRemoteClient.* (shared IO_POOL) | | -| JDBCBatchDAO | 2-4 | JDBCBatchDAO (shared IO_POOL) | | -| Exporters (gRPC/Kafka) | 0-3 | Exporter.* (shared IO_POOL) | | -| | | **IO_POOL shared scheduler (cpuCores(0.5))** | **4** | -| **Total** | **~38-48** | **Total** | **~15** | - -On different hardware: - -| Machine | L1 (OAL+MAL) | L2 (OAL+MAL) | TopN | IO_POOL | Total | -|-----------|---------------|---------------|------|---------|-------| -| 2-core | 2 | 1 | 1 | 1 | 5 | -| 4-core | 4 | 1 | 1 | 2 | 8 | -| 8-core | 8 | 2 | 1 | 4 | 15 | -| 16-core | 16 | 4 | 1 | 8 | 29 | - -Savings (8-core): -- L1: 24+2 → 8 threads (OAL+MAL share one pool, CPU-relative, no empty channel iteration) -- L2: 2+1 → 2 threads (OAL+MAL share one pool) -- TopN: 5-10 → 1 thread (all types share one queue with handler map dispatch) -- I/O queues: 4-11 → 4 threads (shared IO_POOL for gRPC, Kafka, JDBC) -- Total: from ~38-48 OS threads down to ~15 - -IO_POOL queues (all do network or database I/O, low-throughput, bursty): - -| Queue Name | Current Source | I/O Type | Threads Before | -|-------------------------|------------------------|---------------|----------------| -| GRPCRemoteClient.* | GRPCRemoteClient.java | gRPC network | 2-4 (per peer) | -| GRPCMetricsExporter | GRPCMetricsExporter.java| gRPC network | 1 | -| KafkaLogExporter | KafkaLogExporter.java | Kafka network | 1 | -| KafkaTraceExporter | KafkaTraceExporter.java| Kafka network | 1 | -| JDBCBatchDAO | JDBCBatchDAO.java | JDBC database | 2-4 | -| **Subtotal** | | | **7-14** | -| **After (IO_POOL)** | | | **cpuCores(0.5)** | - -Number of partitions/buffers: - -| Before | Count | After | Count | -|--------------------------------|----------|------------------------------------|-------| -| L1 OAL channels (100+ * 2ch) | 200+ | L1 partitions (8 threads * 2) | 16 | -| L1 MAL channels (N * 1ch) | N | *(shared with L1 above)* | *(0)* | -| L2 OAL channels (100+ * 1ch) | 100+ | L2 partitions (2 threads * 2) | 4 | -| L2 MAL channels (N * 1ch) | N | *(shared with L2 above)* | *(0)* | -| TopN buffers (5-10 types) | 5-10 | TOPN_WORKER partitions | 1 | -| I/O buffers (gRPC, JDBC, etc.) | 5-8 | I/O queue partitions | 5-8 | -| **Total buffers** | **300+** | **Total buffers** | **~28** | - -### What Gets Dropped - -| DataCarrier Feature | Status | Reason | -|-------------------------------------|-----------|-------------------------------------------------------| -| One queue per metric type | Dropped | Shared partitions + handler map instead | -| Separate OAL / MAL pools | Dropped | OAL and MAL share L1/L2 queues (handler map dispatch) | -| One thread per TopN type | Dropped | All TopN types share one TOPN_WORKER queue | -| Multi-channel per DataCarrier | Dropped | Single partition array replaces multi-channel | -| IDataPartitioner | Dropped | Simple round-robin on partition array | -| Consumer instantiation by class | Dropped | All callers use instance-based handlers | -| Consumer init(Properties) | Dropped | Not used by any production consumer | -| EnvUtil override | Dropped | Configuration via application.yml | -| Two separate queue classes | Dropped | One `BatchQueue` with configurable scheduler modes | -| BulkConsumePool / ConsumerPoolFactory | Dropped | Dedicated/shared ScheduledExecutorService replaces pool| -| Fixed thread counts | Dropped | ThreadPolicy: CPU-relative (cpuCores) or fixed | -| Signal-driven consumption | Dropped | Adaptive backoff replaces explicit notify | -| Separate createSharedScheduler step | Dropped | Shared schedulers created lazily on first queue ref | - -### What Gets Preserved - -| Feature | How | -|-----------------------|--------------------------------------------------------------| -| Named queue management| `BatchQueueManager.create/createIfAbsent/get` by name | -| Per-type isolation | `handlerMap` dispatches each class to its own handler | -| Bounded buffer | ArrayBlockingQueue per partition | -| BLOCKING strategy | `queue.put()` — producer blocks when full | -| IF_POSSIBLE strategy | `queue.offer()` — returns false when full, data dropped | -| Batch consumption | `drainTo(list)` — same as current | -| Error handling | `errorHandler.onError(batch, throwable)` | -| Nothing-to-consume | `handler.onIdle()` — called when all partitions empty | -| Fast data response | Adaptive backoff (minIdleMs=5ms) replaces signal-driven mode | -| Drain on shutdown | Manager shutdown drains all queues, then schedulers | -| Produce-gate | `produce()` returns false if queue is shut down | -| Hardware scaling | ThreadPolicy.cpuCores() scales threads with available cores | +Steps 6 and 7 batch all revocations first, then wait for all old owners in parallel. +This bounds the total handoff latency to one drain cycle (the slowest old owner), +rather than one cycle per moved partition sequentially. + +### Safety guarantees + +| Property | Mechanism | +|----------|-----------| +| No concurrent handler calls | Two-phase handoff: revoke + cycle-count fence + assign | +| No data loss | Items stay in `ArrayBlockingQueue` during the UNOWNED gap | +| No data duplication | `drainTo` atomically moves items out of the queue | +| Lock-free hot path | Only `AtomicIntegerArray.get()` added to drain loop | +| Lock-free produce path | Only `AtomicLongArray.incrementAndGet()` added | +| Bounded handoff latency | At most one `maxIdleMs` wait per rebalance | + +### Scope + +| Queue | Rebalance? | Reason | +|-------|------------|--------| +| L1 Aggregation (`METRICS_L1_AGGREGATION`) | Yes | Hundreds of metric types, cpuCores threads, high throughput variance | +| L2 Persistence (`METRICS_L2_PERSISTENCE`) | Yes | Same type distribution, fewer threads but still benefits | +| TopN (`TOPN_PERSISTENCE`) | No | Single thread, nothing to rebalance | +| Exporters / gRPC Remote / JDBC | No | Single thread or fixed(1) partition, nothing to rebalance | + +### Complexity budget + +| Component | Lines (est.) | Hot-path cost | +|-----------|-------------|---------------| +| `partitionThroughput` counter | ~5 | 1 `AtomicLong.incrementAndGet` per produce | +| `partitionOwner` check in drain loop | ~5 | 1 `AtomicInteger.get` per partition per cycle | +| `cycleCount` bump | ~2 | 1 `AtomicLong.incrementAndGet` per drain cycle | +| Rebalance task (LPT + handoff) | ~80 | 0 (runs on scheduler, not on hot path) | +| Config field + validation | ~10 | 0 | +| **Total** | **~100** | **2 atomic ops per produce+drain** | From 4bc0ac60fe610e0d810a7ede7a493010965dffb9 Mon Sep 17 00:00:00 2001 From: Wu Sheng Date: Sun, 15 Feb 2026 10:23:37 +0800 Subject: [PATCH 04/16] Add named ThreadFactory to all anonymous Executors pool threads Replace default pool-N-thread-M naming with descriptive thread names across all Executors.newXxx() calls for easier thread dump analysis. Co-Authored-By: Claude Opus 4.6 --- docs/en/changes/changes.md | 31 +- .../server/core/alarm/provider/AlarmCore.java | 2 +- .../server/core/cache/CacheUpdateTimer.java | 2 +- .../config/group/EndpointNameGrouping.java | 2 +- .../core/hierarchy/HierarchyService.java | 2 +- .../ebpf/analyze/EBPFProfilingAnalyzer.java | 6 +- .../remote/client/RemoteClientManager.java | 2 +- .../server/core/storage/PersistenceTimer.java | 8 +- .../core/storage/ttl/DataTTLKeeperTimer.java | 2 +- .../core/watermark/WatermarkWatcher.java | 2 +- .../provider/HealthCheckerProvider.java | 2 +- .../client/grpc/channel/ChannelManager.java | 3 +- .../server/library/batchqueue/BatchQueue.java | 392 ++++++++++++++++-- .../library/batchqueue/BatchQueueManager.java | 16 +- .../library/batchqueue/ThreadPolicy.java | 12 +- .../library/client/grpc/GRPCClient.java | 3 +- .../util/MultipleFilesChangeMonitor.java | 3 +- 17 files changed, 429 insertions(+), 61 deletions(-) diff --git a/docs/en/changes/changes.md b/docs/en/changes/changes.md index 1b067e364ddd..2eddb03c256e 100644 --- a/docs/en/changes/changes.md +++ b/docs/en/changes/changes.md @@ -13,7 +13,8 @@ * Upgrade Groovy to 5.0.3 in OAP backend. * Bump up nodejs to v24.13.0 for the latest UI(booster-ui) compiling. * Add `library-batch-queue` module — a partitioned, self-draining queue with type-based dispatch, - adaptive partitioning, and idle backoff. Designed to replace DataCarrier in high-fan-out scenarios. + adaptive partitioning, idle backoff, and throughput-weighted drain rebalancing (`DrainBalancer`). + Designed to replace DataCarrier in high-fan-out scenarios. * Replace DataCarrier with BatchQueue for L1 metrics aggregation, L2 metrics persistence, TopN persistence, all three exporters (gRPC metrics, Kafka trace, Kafka log), and gRPC remote client. All metric types (OAL + MAL) now share unified queues instead of separate OAL/MAL pools. @@ -48,6 +49,34 @@ * Replace BanyanDB Java client with native implementation. * Remove `bydb.dependencies.properties` and set the compatible BanyanDB API version number in `${SW_STORAGE_BANYANDB_COMPATIBLE_SERVER_API_VERSIONS}`. * Fix trace profiling query time range condition. +* Add named ThreadFactory to all `Executors.newXxx()` calls to replace anonymous `pool-N-thread-M` thread names + with meaningful names for easier thread dump analysis. Complete OAP server thread inventory + (counts on an 8-core machine, exporters and JDBC are optional): + + | Catalog | Thread Name | Count | Policy | Partitions | + |---------|-------------|-------|--------|------------| + | Data Pipeline | `BatchQueue-METRICS_L1_AGGREGATION-N` | 8 | `cpuCores(1.0)` | ~460 adaptive | + | Data Pipeline | `BatchQueue-METRICS_L2_PERSISTENCE-N` | 3 | `cpuCoresWithBase(1, 0.25)` | ~460 adaptive | + | Data Pipeline | `BatchQueue-TOPN_PERSISTENCE-N` | 1 | `fixed(1)` | ~4 adaptive | + | Data Pipeline | `BatchQueue-GRPC_REMOTE_{host}_{port}-N` | 1 per peer | `fixed(1)` | `fixed(1)` | + | Data Pipeline | `BatchQueue-EXPORTER_GRPC_METRICS-N` | 1 | `fixed(1)` | `fixed(1)` | + | Data Pipeline | `BatchQueue-EXPORTER_KAFKA_TRACE-N` | 1 | `fixed(1)` | `fixed(1)` | + | Data Pipeline | `BatchQueue-EXPORTER_KAFKA_LOG-N` | 1 | `fixed(1)` | `fixed(1)` | + | Data Pipeline | `BatchQueue-JDBC_ASYNC_BATCH_PERSISTENT-N` | 4 (configurable) | `fixed(N)` | `fixed(N)` | + | Scheduler | `RemoteClientManager` | 1 | scheduled | — | + | Scheduler | `PersistenceTimer` | 1 | scheduled | — | + | Scheduler | `PersistenceTimer-prepare-N` | 2 (configurable) | fixed pool | — | + | Scheduler | `DataTTLKeeper` | 1 | scheduled | — | + | Scheduler | `CacheUpdateTimer` | 1 | scheduled | — | + | Scheduler | `HierarchyAutoMatching` | 1 | scheduled | — | + | Scheduler | `WatermarkWatcher` | 1 | scheduled | — | + | Scheduler | `AlarmCore` | 1 | scheduled | — | + | Scheduler | `HealthChecker` | 1 | scheduled | — | + | Scheduler | `EndpointUriRecognition` | 1 (conditional) | scheduled | — | + | Scheduler | `FileChangeMonitor` | 1 | scheduled | — | + | Scheduler | `BanyanDB-ChannelManager` | 1 | scheduled | — | + | Scheduler | `GRPCClient-HealthCheck-{host}:{port}` | 1 per client | scheduled | — | + | Scheduler | `EBPFProfiling-N` | configurable | fixed pool | — | * Fix BanyanDB time range overflow in profile thread snapshot query. * `BrowserErrorLog`, OAP Server generated UUID to replace the original client side ID, because Browser scripts can't guarantee generated IDs are globally unique. * MQE: fix multiple labeled metric query and ensure no results are returned if no label value combinations match. diff --git a/oap-server/server-alarm-plugin/src/main/java/org/apache/skywalking/oap/server/core/alarm/provider/AlarmCore.java b/oap-server/server-alarm-plugin/src/main/java/org/apache/skywalking/oap/server/core/alarm/provider/AlarmCore.java index 610ca3f30a46..e20c14ec42bb 100644 --- a/oap-server/server-alarm-plugin/src/main/java/org/apache/skywalking/oap/server/core/alarm/provider/AlarmCore.java +++ b/oap-server/server-alarm-plugin/src/main/java/org/apache/skywalking/oap/server/core/alarm/provider/AlarmCore.java @@ -70,7 +70,7 @@ public List findRunningRule(String metricsName) { public void start(List allCallbacks) { LocalDateTime now = LocalDateTime.now(); lastExecuteTime = now; - Executors.newSingleThreadScheduledExecutor().scheduleAtFixedRate(() -> { + Executors.newSingleThreadScheduledExecutor(r -> new Thread(r, "AlarmCore")).scheduleAtFixedRate(() -> { try { final List alarmMessageList = new ArrayList<>(30); LocalDateTime checkTime = LocalDateTime.now(); diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/cache/CacheUpdateTimer.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/cache/CacheUpdateTimer.java index 1344cf7f22fe..06eb42e2d911 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/cache/CacheUpdateTimer.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/cache/CacheUpdateTimer.java @@ -59,7 +59,7 @@ public void start(ModuleDefineHolder moduleDefineHolder, int ttl) { final long timeInterval = 10; - Executors.newSingleThreadScheduledExecutor() + Executors.newSingleThreadScheduledExecutor(r -> new Thread(r, "CacheUpdateTimer")) .scheduleAtFixedRate( new RunnableWithExceptionProtection(() -> update(moduleDefineHolder), t -> log .error("Cache update failure.", t)), 1, timeInterval, TimeUnit.SECONDS); diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/config/group/EndpointNameGrouping.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/config/group/EndpointNameGrouping.java index 81de84d04f02..c4ff5e714a5b 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/config/group/EndpointNameGrouping.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/config/group/EndpointNameGrouping.java @@ -210,7 +210,7 @@ public void startHttpUriRecognitionSvr(HttpUriRecognition httpUriRecognitionSvr) } this.quickUriGroupingRule = new QuickUriGroupingRule(); HTTPUrlRecognitionConfig config = this.httpUrlRecognitionConfig; - Executors.newSingleThreadScheduledExecutor() + Executors.newSingleThreadScheduledExecutor(r -> new Thread(r, "EndpointUriRecognition")) .scheduleWithFixedDelay( new RunnableWithExceptionProtection( () -> { diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/hierarchy/HierarchyService.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/hierarchy/HierarchyService.java index f1ea6b17debd..783d5bc3fb85 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/hierarchy/HierarchyService.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/hierarchy/HierarchyService.java @@ -136,7 +136,7 @@ public void startAutoMatchingServiceHierarchy() { if (!this.isEnableHierarchy) { return; } - Executors.newSingleThreadScheduledExecutor() + Executors.newSingleThreadScheduledExecutor(r -> new Thread(r, "HierarchyAutoMatching")) .scheduleWithFixedDelay( new RunnableWithExceptionProtection(this::autoMatchingServiceRelation, t -> log.error( "Scheduled auto matching service hierarchy from service traffic failure.", t)), 30, 20, TimeUnit.SECONDS); diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/profiling/ebpf/analyze/EBPFProfilingAnalyzer.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/profiling/ebpf/analyze/EBPFProfilingAnalyzer.java index 379330cd4057..496f1b9b8f94 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/profiling/ebpf/analyze/EBPFProfilingAnalyzer.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/profiling/ebpf/analyze/EBPFProfilingAnalyzer.java @@ -40,6 +40,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -60,7 +61,10 @@ public class EBPFProfilingAnalyzer { public EBPFProfilingAnalyzer(ModuleManager moduleManager, int maxDurationOfQuery, int fetchDataThreadPoolSize) { this.moduleManager = moduleManager; this.maxQueryTimeoutInSecond = maxDurationOfQuery; - this.fetchDataThreadPool = Executors.newFixedThreadPool(fetchDataThreadPoolSize); + final AtomicInteger fetchThreadSeq = new AtomicInteger(0); + this.fetchDataThreadPool = Executors.newFixedThreadPool( + fetchDataThreadPoolSize, + r -> new Thread(r, "EBPFProfiling-" + fetchThreadSeq.incrementAndGet())); } /** diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/remote/client/RemoteClientManager.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/remote/client/RemoteClientManager.java index f25ad05d29e6..3f633f6bb547 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/remote/client/RemoteClientManager.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/remote/client/RemoteClientManager.java @@ -94,7 +94,7 @@ public RemoteClientManager(final ModuleDefineHolder moduleDefineHolder, final in public void start() { Optional.ofNullable(sslContext).ifPresent(DynamicSslContext::start); - Executors.newSingleThreadScheduledExecutor() + Executors.newSingleThreadScheduledExecutor(r -> new Thread(r, "RemoteClientManager")) .scheduleWithFixedDelay(new RunnableWithExceptionProtection(this::refresh, t -> log.error( "Scheduled refresh Remote Clients failure.", t)), 1, 10, TimeUnit.SECONDS); } diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/PersistenceTimer.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/PersistenceTimer.java index f191057370e5..33460f4153a1 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/PersistenceTimer.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/PersistenceTimer.java @@ -25,6 +25,7 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; import lombok.extern.slf4j.Slf4j; import org.apache.skywalking.oap.server.core.CoreModuleConfig; import org.apache.skywalking.oap.server.core.analysis.worker.MetricsStreamProcessor; @@ -89,9 +90,12 @@ public void start(ModuleManager moduleManager, CoreModuleConfig moduleConfig) { 0.5, 1, 3, 5, 10, 15, 20, 25, 50, 120 ); - prepareExecutorService = Executors.newFixedThreadPool(moduleConfig.getPrepareThreads()); + final AtomicInteger prepareThreadSeq = new AtomicInteger(0); + prepareExecutorService = Executors.newFixedThreadPool( + moduleConfig.getPrepareThreads(), + r -> new Thread(r, "PersistenceTimer-prepare-" + prepareThreadSeq.incrementAndGet())); if (!isStarted) { - Executors.newSingleThreadScheduledExecutor() + Executors.newSingleThreadScheduledExecutor(r -> new Thread(r, "PersistenceTimer")) .scheduleWithFixedDelay( new RunnableWithExceptionProtection( () -> extractDataAndSave(batchDAO).join(), diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/ttl/DataTTLKeeperTimer.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/ttl/DataTTLKeeperTimer.java index 9d7d3627ef10..27dc5bef2f96 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/ttl/DataTTLKeeperTimer.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/ttl/DataTTLKeeperTimer.java @@ -60,7 +60,7 @@ public void start(ModuleManager moduleManager, CoreModuleConfig moduleConfig) { this.clusterNodesQuery = moduleManager.find(ClusterModule.NAME).provider().getService(ClusterNodesQuery.class); this.moduleConfig = moduleConfig; - Executors.newSingleThreadScheduledExecutor() + Executors.newSingleThreadScheduledExecutor(r -> new Thread(r, "DataTTLKeeper")) .scheduleAtFixedRate( new RunnableWithExceptionProtection( this::delete, diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/watermark/WatermarkWatcher.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/watermark/WatermarkWatcher.java index db5c75a718f7..8f8104f7bdf3 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/watermark/WatermarkWatcher.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/watermark/WatermarkWatcher.java @@ -80,7 +80,7 @@ public void start(MetricsCollector so11yCollector) { .getService(MetricsCreator.class); this.addListener(WatermarkGRPCInterceptor.INSTANCE); - Executors.newSingleThreadScheduledExecutor() + Executors.newSingleThreadScheduledExecutor(r -> new Thread(r, "WatermarkWatcher")) .scheduleWithFixedDelay(this::watch, 0, 10, TimeUnit.SECONDS); } diff --git a/oap-server/server-health-checker/src/main/java/org/apache/skywalking/oap/server/health/checker/provider/HealthCheckerProvider.java b/oap-server/server-health-checker/src/main/java/org/apache/skywalking/oap/server/health/checker/provider/HealthCheckerProvider.java index 6bec3853ad07..13239c2475b7 100644 --- a/oap-server/server-health-checker/src/main/java/org/apache/skywalking/oap/server/health/checker/provider/HealthCheckerProvider.java +++ b/oap-server/server-health-checker/src/main/java/org/apache/skywalking/oap/server/health/checker/provider/HealthCheckerProvider.java @@ -80,7 +80,7 @@ public void onInitialized(final HealthCheckerConfig initialized) { @Override public void prepare() throws ServiceNotProvidedException, ModuleStartException { score.set(-1); - ses = Executors.newSingleThreadScheduledExecutor(); + ses = Executors.newSingleThreadScheduledExecutor(r -> new Thread(r, "HealthChecker")); healthQueryService = new HealthQueryService(score, details); this.registerServiceImplementation(HealthQueryService.class, healthQueryService); } diff --git a/oap-server/server-library/library-banyandb-client/src/main/java/org/apache/skywalking/library/banyandb/v1/client/grpc/channel/ChannelManager.java b/oap-server/server-library/library-banyandb-client/src/main/java/org/apache/skywalking/library/banyandb/v1/client/grpc/channel/ChannelManager.java index 89a8d7568561..7ffbbb50427c 100644 --- a/oap-server/server-library/library-banyandb-client/src/main/java/org/apache/skywalking/library/banyandb/v1/client/grpc/channel/ChannelManager.java +++ b/oap-server/server-library/library-banyandb-client/src/main/java/org/apache/skywalking/library/banyandb/v1/client/grpc/channel/ChannelManager.java @@ -57,7 +57,8 @@ public class ChannelManager extends ManagedChannel { public static ChannelManager create(ChannelManagerSettings settings, ChannelFactory channelFactory) throws IOException { - return new ChannelManager(settings, channelFactory, Executors.newSingleThreadScheduledExecutor()); + return new ChannelManager(settings, channelFactory, + Executors.newSingleThreadScheduledExecutor(r -> new Thread(r, "BanyanDB-ChannelManager"))); } ChannelManager(ChannelManagerSettings settings, ChannelFactory channelFactory, ScheduledExecutorService executor) throws IOException { diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java index 1167fc57a58c..d86d5d05aa9f 100644 --- a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java @@ -28,6 +28,11 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicIntegerArray; +import java.util.concurrent.atomic.AtomicLongArray; +import java.util.concurrent.locks.LockSupport; +import lombok.AccessLevel; +import lombok.Getter; import lombok.extern.slf4j.Slf4j; /** @@ -50,14 +55,14 @@ *

  * produce(data)
  *   |
- *   +-- queue stopped?  --> return false
+ *   +-- queue stopped?  --> return false
  *   |
- *   +-- N == 1?  --> partition[0]  (skip selector)
- *   +-- N > 1?  --> partition[selector.select(data, N)]
+ *   +-- N == 1?  --> partition[0]  (skip selector)
+ *   +-- N > 1?  --> partition[selector.select(data, N)]
  *   |
  *   +-- BLOCKING strategy?
- *   |     yes --> ArrayBlockingQueue.put(data)   // blocks until space available
- *   |     no  --> ArrayBlockingQueue.offer(data) // returns false if full (drop)
+ *   |     yes --> ArrayBlockingQueue.put(data)   // blocks until space available
+ *   |     no  --> ArrayBlockingQueue.offer(data) // returns false if full (drop)
  *   |
  *   +-- return true/false
  * 
@@ -71,12 +76,12 @@ * +-- drainTo(combined) from each assigned partition * | * +-- combined is empty? - * | yes --> consecutiveIdleCycles++, notify onIdle(), break - * | no --> consecutiveIdleCycles = 0, dispatch(combined) + * | yes --> consecutiveIdleCycles++, notify onIdle(), break + * | no --> consecutiveIdleCycles = 0, dispatch(combined) * | | * | +-- single consumer set? - * | | yes --> consumer.consume(batch) - * | | no --> group by item.getClass() + * | | yes --> consumer.consume(batch) + * | | no --> group by item.getClass() * | | for each (class, subBatch): * | | handlerMap.get(class).consume(subBatch) * | | @@ -102,14 +107,17 @@ * *

Use case examples

*
- * shared scheduler, partitions=1, one consumer    --> I/O queue (gRPC, Kafka, JDBC)
- * dedicated fixed(1), partitions=1, many handlers --> TopN (all types share 1 thread)
+ * shared scheduler, partitions=1, one consumer    --> I/O queue (gRPC, Kafka, JDBC)
+ * dedicated fixed(1), partitions=1, many handlers --> TopN (all types share 1 thread)
  * dedicated cpuCores(1.0), adaptive(),
- *           many handlers                         --> metrics aggregation
+ *           many handlers                         --> metrics aggregation
  * 
*/ @Slf4j public class BatchQueue { + private static final int UNOWNED = -1; + + @Getter private final String name; private final BatchQueueConfig config; @@ -117,6 +125,7 @@ public class BatchQueue { private final ScheduledExecutorService scheduler; /** True if this queue owns the scheduler and should shut it down. */ + @Getter(AccessLevel.PACKAGE) private final boolean dedicatedScheduler; /** Non-null only for shared schedulers; used to release the ref count on shutdown. */ @@ -152,24 +161,145 @@ public class BatchQueue { * Partitions. Producers select a partition via {@link PartitionSelector}. * For adaptive policies, this array grows via {@link #addHandler} as handlers * are registered. Volatile for visibility to drain loop threads. + * + *

Layout: {@code partitions[partitionIndex]} is an {@link ArrayBlockingQueue} + * holding items routed to that partition. Length equals the resolved partition count. + * + *

Example — 8 partitions, bufferSize=2000, 5 metric types: + *

+     *   TypeA.hashCode() % 8 = 2  -->  partitions[2].put(typeA_item)
+     *   TypeB.hashCode() % 8 = 5  -->  partitions[5].put(typeB_item)
+     *   TypeC.hashCode() % 8 = 2  -->  partitions[2].put(typeC_item)  (same partition as TypeA)
+     *   TypeD.hashCode() % 8 = 7  -->  partitions[7].put(typeD_item)
+     *   TypeE.hashCode() % 8 = 0  -->  partitions[0].put(typeE_item)
+     * 
*/ private volatile ArrayBlockingQueue[] partitions; /** - * Which partitions each drain task is responsible for. - * Volatile for visibility when partitions grow via {@link #addHandler}. + * Per-task partition assignments. Each drain task drains only its assigned partitions. + * Built by round-robin initially, rebuilt by the rebalancer when partitions are moved. + * Volatile for visibility when partitions grow via {@link #addHandler} or rebalance. + * + *

Example — 8 partitions, 4 tasks, initial round-robin: + *

+     *   assignedPartitions[0] = {0, 4}   -- task 0 drains partitions 0,4
+     *   assignedPartitions[1] = {1, 5}   -- task 1 drains partitions 1,5
+     *   assignedPartitions[2] = {2, 6}   -- task 2 drains partitions 2,6
+     *   assignedPartitions[3] = {3, 7}   -- task 3 drains partitions 3,7
+     * 
+ * After rebalancing moves partition 4 from task 0 to task 2 (to equalize load): + *
+     *   assignedPartitions[0] = {0}      -- task 0 lost partition 4
+     *   assignedPartitions[1] = {1, 5}   -- unchanged
+     *   assignedPartitions[2] = {2, 6, 4}-- task 2 gained partition 4
+     *   assignedPartitions[3] = {3, 7}   -- unchanged
+     * 
*/ + @Getter(AccessLevel.PACKAGE) private volatile int[][] assignedPartitions; /** * Per-task count of consecutive idle cycles (no data drained). * Used for adaptive exponential backoff in {@link #scheduleDrain}. + * + *

Example — 4 tasks, task 0 busy, task 3 idle for 5 cycles: + *

+     *   consecutiveIdleCycles = {0, 0, 1, 5}
+     *   delay for task 0: minIdleMs * 2^0 = 1ms  (data flowing)
+     *   delay for task 3: minIdleMs * 2^5 = 32ms (backing off, capped at maxIdleMs)
+     * 
*/ private final int[] consecutiveIdleCycles; /** Set to false on {@link #shutdown()} to stop drain loops and reject new data. */ + @Getter private volatile boolean running; + // ---- Rebalancing fields (only allocated when enableRebalancing() is called) ---- + + /** + * Per-partition produce counter, incremented in {@link #produce} before {@code put/offer}. + * The rebalancer snapshots and resets all counters each interval to measure throughput. + * Null until {@link #enableRebalancing} is called. + * + *

Example — 8 partitions, after 2 seconds of production with skewed types: + *

+     *   partitionThroughput = [1200, 300, 4800, 150, 3600, 300, 900, 2400]
+     *                           ^                ^         ^
+     *                      TypeE(1x)     TypeA+C(24x)  TypeD(12x)
+     * 
+ * The rebalancer snapshots these values and resets all to 0: + *
+     *   snapshot             = [1200, 300, 4800, 150, 3600, 300, 900, 2400]
+     *   partitionThroughput  = [   0,   0,    0,   0,    0,   0,   0,    0]  (reset)
+     * 
+ */ + private volatile AtomicLongArray partitionThroughput; + + /** + * Partition-to-task ownership map, authoritative source for which task drains which + * partition. The drain loop checks this before draining each partition, skipping + * partitions it no longer owns. Null until {@link #enableRebalancing} is called. + * + *

Example — 8 partitions, 4 tasks, initial round-robin ownership: + *

+     *   partitionOwner = [0, 1, 2, 3, 0, 1, 2, 3]
+     *                     ^           ^
+     *               partition 0    partition 4
+     *               owned by       owned by
+     *               task 0         task 0
+     * 
+ * During two-phase handoff (moving partition 4 from task 0 to task 2): + *
+     *   Phase 1 — revoke:  partitionOwner = [0, 1, 2, 3, -1, 1, 2, 3]
+     *                                                      ^
+     *                                                  UNOWNED (-1)
+     *                                                  task 0 skips it on next drain
+     *
+     *   (rebalancer waits for task 0's cycleCount to advance)
+     *
+     *   Phase 2 — assign:  partitionOwner = [0, 1, 2, 3, 2, 1, 2, 3]
+     *                                                     ^
+     *                                                 now owned by task 2
+     * 
+ */ + private volatile AtomicIntegerArray partitionOwner; + + /** + * Per-task drain cycle counter, monotonically increasing. Incremented once in the + * {@code finally} block of {@link #drainLoop} when the entire scheduled invocation + * exits — which may have drained and dispatched multiple batches in its inner + * {@code while} loop before finding an empty drain. + * + *

The rebalancer uses this as a fence: after revoking a partition from a task, + * it waits for that task's cycle count to advance, which proves the task has exited + * {@code drainLoop} and will not touch the revoked partition again until the next + * scheduled invocation (which will re-read the updated {@code partitionOwner}). + * Null until {@link #enableRebalancing} is called. + * + *

Example — 4 tasks, rebalancer revoking partition 4 from task 0: + *

+     *   cycleCount = [142, 138, 145, 140]
+     *                  ^
+     *            snapshot task 0's count = 142
+     *
+     *   task 0 is still inside drainLoop:
+     *     while(running) { drain --> dispatch --> drain --> empty --> break }
+     *     finally { cycleCount[0]++ }      // increments once for the whole invocation
+     *     scheduleDrain(0)                 // next invocation re-reads partitionOwner
+     *
+     *   cycleCount = [143, 138, 145, 140]
+     *                  ^
+     *            143 > 142 --> task 0 exited drainLoop, safe to reassign
+     * 
+ */ + private volatile AtomicLongArray cycleCount; + + /** Whether rebalancing is active. Gates all hot-path additions. */ + @Getter(AccessLevel.PACKAGE) + private volatile boolean rebalancingEnabled; + @SuppressWarnings("unchecked") BatchQueue(final String name, final BatchQueueConfig config) { this.name = name; @@ -234,6 +364,11 @@ public class BatchQueue { for (int t = 0; t < taskCount; t++) { scheduleDrain(t); } + + // Enable rebalancing if configured + if (config.getBalancer() != null && config.getRebalanceIntervalMs() > 0) { + enableRebalancing(config.getRebalanceIntervalMs()); + } } /** @@ -298,17 +433,80 @@ public void addHandler(final Class type, final HandlerConsumer h .resolve(resolvedThreadCount, handlerMap.size()); final ArrayBlockingQueue[] currentPartitions = this.partitions; if (newPartitionCount > currentPartitions.length) { + final int oldCount = currentPartitions.length; final ArrayBlockingQueue[] grown = new ArrayBlockingQueue[newPartitionCount]; - System.arraycopy(currentPartitions, 0, grown, 0, currentPartitions.length); - for (int i = currentPartitions.length; i < newPartitionCount; i++) { + System.arraycopy(currentPartitions, 0, grown, 0, oldCount); + for (int i = oldCount; i < newPartitionCount; i++) { grown[i] = new ArrayBlockingQueue<>(config.getBufferSize()); } + + if (rebalancingEnabled) { + // Grow atomic arrays to cover new partitions + final AtomicLongArray newThroughput = new AtomicLongArray(newPartitionCount); + for (int i = 0; i < oldCount; i++) { + newThroughput.set(i, partitionThroughput.get(i)); + } + this.partitionThroughput = newThroughput; + + final AtomicIntegerArray newOwner = new AtomicIntegerArray(newPartitionCount); + for (int i = 0; i < oldCount; i++) { + newOwner.set(i, partitionOwner.get(i)); + } + // Assign new partitions round-robin + for (int i = oldCount; i < newPartitionCount; i++) { + newOwner.set(i, i % taskCount); + } + this.partitionOwner = newOwner; + } + // Volatile writes — drain loop threads see the new assignments on next cycle this.assignedPartitions = buildAssignments(taskCount, newPartitionCount); this.partitions = grown; } } + /** + * Initialize rebalancing infrastructure and schedule periodic rebalance task. + * Called from constructor when {@code .balancer(DrainBalancer, intervalMs)} is + * configured. Silently returns if {@code taskCount <= 1} (nothing to rebalance). + */ + private void enableRebalancing(final long intervalMs) { + if (taskCount <= 1) { + return; + } + + final int partitionCount = partitions.length; + + // Allocate atomic arrays + this.partitionThroughput = new AtomicLongArray(partitionCount); + this.cycleCount = new AtomicLongArray(taskCount); + + // Initialize ownership from current assignedPartitions + final AtomicIntegerArray owner = new AtomicIntegerArray(partitionCount); + // Default all to UNOWNED, then set from assignments + for (int p = 0; p < partitionCount; p++) { + owner.set(p, UNOWNED); + } + final int[][] currentAssignments = this.assignedPartitions; + for (int t = 0; t < currentAssignments.length; t++) { + for (final int p : currentAssignments[t]) { + owner.set(p, t); + } + } + this.partitionOwner = owner; + + // Enable the flag — gates hot-path additions in produce() and drainLoop() + this.rebalancingEnabled = true; + + // Schedule periodic rebalancing on the queue's scheduler + scheduler.scheduleAtFixedRate( + this::rebalance, intervalMs, intervalMs, TimeUnit.MILLISECONDS + ); + + log.info("BatchQueue[{}]: rebalancing enabled, interval={}ms, tasks={}, partitions={}", + name, intervalMs, taskCount, partitionCount); + } + /** * Produce data into a partition selected by the configured {@link PartitionSelector}. * @@ -333,6 +531,13 @@ public boolean produce(final T data) { final ArrayBlockingQueue[] currentPartitions = this.partitions; final int index = currentPartitions.length == 1 ? 0 : partitionSelector.select(data, currentPartitions.length); + // Increment throughput counter BEFORE put/offer so the rebalancer sees + // true produce demand, not the drain-throttled rate. With BLOCKING, the + // post-put counter would reflect drain rate (already equalized by + // backpressure), hiding the imbalance the rebalancer needs to detect. + if (rebalancingEnabled) { + partitionThroughput.incrementAndGet(index); + } if (config.getStrategy() == BufferStrategy.BLOCKING) { try { currentPartitions[index].put(data); @@ -358,12 +563,17 @@ public boolean produce(final T data) { void drainLoop(final int taskIndex) { final ArrayBlockingQueue[] currentPartitions = this.partitions; final int[] myPartitions = this.assignedPartitions[taskIndex]; + final boolean checkOwnership = rebalancingEnabled; try { while (running) { // Drain all assigned partitions into one batch final List combined = new ArrayList<>(); for (final int partitionIndex : myPartitions) { if (partitionIndex < currentPartitions.length) { + // Skip partitions revoked by the rebalancer + if (checkOwnership && partitionOwner.get(partitionIndex) != taskIndex) { + continue; + } currentPartitions[partitionIndex].drainTo(combined); } } @@ -382,6 +592,10 @@ void drainLoop(final int taskIndex) { } catch (final Throwable t) { log.error("BatchQueue[{}]: drain loop error", name, t); } finally { + // Bump cycle count so the rebalancer knows this task finished its dispatch + if (checkOwnership) { + cycleCount.incrementAndGet(taskIndex); + } // Always re-schedule unless shutdown if (running) { scheduleDrain(taskIndex); @@ -468,6 +682,136 @@ private void handleError(final List data, final Throwable t) { } } + /** + * Periodic rebalance task. Snapshots throughput counters, runs LPT assignment, + * and performs two-phase handoff for moved partitions. + */ + private void rebalance() { + if (!running) { + return; + } + try { + final int partitionCount = partitions.length; + final AtomicLongArray throughput = this.partitionThroughput; + final AtomicIntegerArray owner = this.partitionOwner; + + // Step 1: Snapshot and reset throughput counters + final long[] snapshot = new long[partitionCount]; + for (int p = 0; p < partitionCount; p++) { + snapshot[p] = throughput.getAndSet(p, 0); + } + + // Step 2: Snapshot current ownership + final int[] currentOwner = new int[partitionCount]; + for (int p = 0; p < partitionCount; p++) { + currentOwner[p] = owner.get(p); + } + + // Log per-thread load before balancing (debug) + if (log.isDebugEnabled()) { + final long[] threadLoad = new long[taskCount]; + for (int p = 0; p < partitionCount; p++) { + final int t = currentOwner[p]; + if (t >= 0 && t < taskCount) { + threadLoad[t] += snapshot[p]; + } + } + final StringBuilder sb = new StringBuilder(); + for (int t = 0; t < taskCount; t++) { + if (t > 0) { + sb.append(", "); + } + sb.append(threadLoad[t]); + } + log.debug("BatchQueue[{}]: rebalance check — thread loads: [{}]", name, sb); + } + + // Step 3: Delegate to the configured balancer + final int[] newOwner = config.getBalancer().assign(snapshot, currentOwner, taskCount); + if (newOwner == null) { + return; // Balancer decided to skip this cycle + } + + // Step 4: Diff — find partitions that changed owner + final List moves = new ArrayList<>(); // [partition, oldTask, newTask] + for (int p = 0; p < partitionCount; p++) { + final int oldTask = currentOwner[p]; + final int newTask = newOwner[p]; + if (oldTask != newTask && oldTask >= 0 && newTask >= 0) { + moves.add(new int[]{p, oldTask, newTask}); + } + } + + if (moves.isEmpty()) { + return; + } + + // Step 5: Two-phase handoff + // Phase 1 — Revoke: set all moved partitions to UNOWNED + final long[] cycleSnapshots = new long[taskCount]; + final boolean[] needWait = new boolean[taskCount]; + for (final int[] move : moves) { + owner.set(move[0], UNOWNED); + needWait[move[1]] = true; + } + + // Snapshot cycle counts for old owners that lost partitions + for (int t = 0; t < taskCount; t++) { + if (needWait[t]) { + cycleSnapshots[t] = cycleCount.get(t); + } + } + + // Wait for all old owners to complete their current drain+dispatch cycle + for (int t = 0; t < taskCount; t++) { + if (needWait[t]) { + final long snap = cycleSnapshots[t]; + while (cycleCount.get(t) <= snap && running) { + LockSupport.parkNanos(TimeUnit.MILLISECONDS.toNanos(1)); + } + } + } + + // Phase 2 — Assign: set new owners + for (final int[] move : moves) { + owner.set(move[0], move[2]); + } + + // Rebuild assignedPartitions from ownership array + this.assignedPartitions = buildAssignmentsFromOwner(owner, taskCount, partitionCount); + + log.info("BatchQueue[{}]: rebalanced {} partitions", name, moves.size()); + } catch (final Throwable t) { + log.error("BatchQueue[{}]: rebalance error", name, t); + } + } + + /** + * Build partition-to-task assignments from the ownership array. + */ + private static int[][] buildAssignmentsFromOwner( + final AtomicIntegerArray owner, final int taskCount, final int partitionCount) { + final List> assignment = new ArrayList<>(); + for (int t = 0; t < taskCount; t++) { + assignment.add(new ArrayList<>()); + } + for (int p = 0; p < partitionCount; p++) { + final int t = owner.get(p); + if (t >= 0 && t < taskCount) { + assignment.get(t).add(p); + } + } + final int[][] result = new int[taskCount][]; + for (int t = 0; t < taskCount; t++) { + final List parts = assignment.get(t); + result[t] = new int[parts.size()]; + for (int i = 0; i < parts.size(); i++) { + result[t][i] = parts.get(i); + } + } + return result; + } + /** * Stop the queue: reject new produces, perform a final drain of all partitions, * and release the scheduler (dedicated: shutdown; shared: decrement ref count). @@ -490,14 +834,6 @@ void shutdown() { } } - public String getName() { - return name; - } - - public boolean isRunning() { - return running; - } - int getPartitionCount() { return partitions.length; } @@ -506,14 +842,6 @@ int getTaskCount() { return assignedPartitions.length; } - int[][] getAssignedPartitions() { - return assignedPartitions; - } - - boolean isDedicatedScheduler() { - return dedicatedScheduler; - } - /** * Take a point-in-time snapshot of queue usage across all partitions. */ diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManager.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManager.java index 061daf6deeb8..af75eb841bf9 100644 --- a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManager.java +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManager.java @@ -39,32 +39,32 @@ * *

Internal maps: *

- * QUEUES:                     queueName     -> BatchQueue instance
- * SHARED_SCHEDULERS:          schedulerName -> ScheduledExecutorService
- * SHARED_SCHEDULER_POLICIES:  schedulerName -> ThreadPolicy (first-wins)
- * SHARED_SCHEDULER_REF_COUNTS: schedulerName -> AtomicInteger (reference count)
+ * QUEUES:                     queueName     -> BatchQueue instance
+ * SHARED_SCHEDULERS:          schedulerName -> ScheduledExecutorService
+ * SHARED_SCHEDULER_POLICIES:  schedulerName -> ThreadPolicy (first-wins)
+ * SHARED_SCHEDULER_REF_COUNTS: schedulerName -> AtomicInteger (reference count)
  * 
*/ @Slf4j public class BatchQueueManager { /** - * queueName -> BatchQueue instance. Each queue has a unique name. + * queueName -> BatchQueue instance. Each queue has a unique name. */ private static final ConcurrentHashMap> QUEUES = new ConcurrentHashMap<>(); /** - * schedulerName -> ScheduledExecutorService. Multiple queues can share one scheduler by + * schedulerName -> ScheduledExecutorService. Multiple queues can share one scheduler by * referencing the same scheduler name in their config. */ private static final ConcurrentHashMap SHARED_SCHEDULERS = new ConcurrentHashMap<>(); /** - * schedulerName -> ThreadPolicy. Tracks the first-wins policy for each shared scheduler + * schedulerName -> ThreadPolicy. Tracks the first-wins policy for each shared scheduler * to detect mismatched configs. */ private static final ConcurrentHashMap SHARED_SCHEDULER_POLICIES = new ConcurrentHashMap<>(); /** - * schedulerName -> reference count. Incremented when a queue acquires the scheduler, + * schedulerName -> reference count. Incremented when a queue acquires the scheduler, * decremented when a queue releases it. Scheduler is shut down when count reaches 0. */ private static final ConcurrentHashMap SHARED_SCHEDULER_REF_COUNTS = diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicy.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicy.java index 43e5f77ef190..e38d5cfc87e7 100644 --- a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicy.java +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicy.java @@ -27,8 +27,8 @@ * - cpuCores(multiplier): multiplier * Runtime.availableProcessors(), rounded. * - cpuCoresWithBase(base, multiplier): base + multiplier * Runtime.availableProcessors(), rounded. * - * Resolved value is always >= 1 — every pool must have at least one thread. - * fixed() requires count >= 1 at construction. cpuCores() applies max(1, ...) at resolution. + * Resolved value is always >= 1 — every pool must have at least one thread. + * fixed() requires count >= 1 at construction. cpuCores() applies max(1, ...) at resolution. */ public class ThreadPolicy { private final int fixedCount; @@ -42,7 +42,7 @@ private ThreadPolicy(final int fixedCount, final int base, final double cpuMulti } /** - * Fixed number of threads. Count must be >= 1. + * Fixed number of threads. Count must be >= 1. * * @throws IllegalArgumentException if count < 1 */ @@ -55,7 +55,7 @@ public static ThreadPolicy fixed(final int count) { /** * Threads = multiplier * available CPU cores, rounded, min 1. - * Multiplier must be > 0. + * Multiplier must be > 0. * * @throws IllegalArgumentException if multiplier <= 0 */ @@ -68,7 +68,7 @@ public static ThreadPolicy cpuCores(final double multiplier) { /** * Threads = base + round(multiplier * available CPU cores), min 1. - * Base must be >= 0, multiplier must be > 0. + * Base must be >= 0, multiplier must be > 0. * * Example: cpuCoresWithBase(2, 0.25) on 8-core = 2 + 2 = 4, on 16-core = 2 + 4 = 6, on 24-core = 2 + 6 = 8. * @@ -85,7 +85,7 @@ public static ThreadPolicy cpuCoresWithBase(final int base, final double multipl } /** - * Resolve the actual thread count. Always returns >= 1. + * Resolve the actual thread count. Always returns >= 1. */ public int resolve() { if (fixedCount > 0) { diff --git a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/grpc/GRPCClient.java b/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/grpc/GRPCClient.java index 1ab0ee3e338c..679a7e47d3d9 100644 --- a/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/grpc/GRPCClient.java +++ b/oap-server/server-library/library-client/src/main/java/org/apache/skywalking/oap/server/library/client/grpc/GRPCClient.java @@ -167,7 +167,8 @@ public void overrideCheckerRunnable(final Runnable healthCheckRunnable, final lo private void checkHealth() { if (healthCheckExecutor == null) { - healthCheckExecutor = Executors.newSingleThreadScheduledExecutor(); + healthCheckExecutor = Executors.newSingleThreadScheduledExecutor( + r -> new Thread(r, "GRPCClient-HealthCheck-" + host + ":" + port)); healthCheckExecutor.scheduleAtFixedRate(healthCheckRunnable, initialDelay, period, TimeUnit.SECONDS ); } diff --git a/oap-server/server-library/library-util/src/main/java/org/apache/skywalking/oap/server/library/util/MultipleFilesChangeMonitor.java b/oap-server/server-library/library-util/src/main/java/org/apache/skywalking/oap/server/library/util/MultipleFilesChangeMonitor.java index ddf375d6c351..202ac9b0e091 100644 --- a/oap-server/server-library/library-util/src/main/java/org/apache/skywalking/oap/server/library/util/MultipleFilesChangeMonitor.java +++ b/oap-server/server-library/library-util/src/main/java/org/apache/skywalking/oap/server/library/util/MultipleFilesChangeMonitor.java @@ -134,7 +134,8 @@ public void start() { SCHEDULER_CHANGE_LOCK.lock(); try { if (FILE_MONITOR_TASK_SCHEDULER == null) { - FILE_MONITOR_TASK_SCHEDULER = Executors.newSingleThreadScheduledExecutor() + FILE_MONITOR_TASK_SCHEDULER = Executors.newSingleThreadScheduledExecutor( + r -> new Thread(r, "FileChangeMonitor")) .scheduleAtFixedRate( MultipleFilesChangeMonitor::scanChanges, 1, 200, TimeUnit.MILLISECONDS From 7d69bbff5b43f7e470dc0df30cc5f2c366abcd82 Mon Sep 17 00:00:00 2001 From: Wu Sheng Date: Sun, 15 Feb 2026 21:16:08 +0800 Subject: [PATCH 05/16] Add throughput-weighted drain rebalancing for L1 and L2 queues Enable periodic partition-to-thread reassignment based on drain throughput to equalize load when metric types have skewed volume. Co-Authored-By: Claude Opus 4.6 --- docs/en/changes/changes.md | 12 +- .../worker/MetricsAggregateWorker.java | 2 + .../worker/MetricsPersistentMinWorker.java | 2 + .../client/RemoteClientManagerTestCase.java | 9 +- .../library-batch-queue/CLAUDE.md | 51 +- .../library-batch-queue/DESIGN.md | 259 -------- .../server/library/batchqueue/BatchQueue.java | 11 + .../library/batchqueue/BatchQueueConfig.java | 37 +- .../library/batchqueue/BatchQueueManager.java | 11 +- .../library/batchqueue/BatchQueueStats.java | 17 +- .../library/batchqueue/DrainBalancer.java | 58 ++ .../library/batchqueue/HandlerConsumer.java | 2 + .../library/batchqueue/PartitionPolicy.java | 11 + .../library/batchqueue/PartitionSelector.java | 3 + .../library/batchqueue/ThreadPolicy.java | 38 +- .../ThroughputWeightedBalancer.java | 102 ++++ .../batchqueue/BatchQueueBenchmark.java | 154 +++-- .../batchqueue/BatchQueueManagerTest.java | 9 +- .../library/batchqueue/BatchQueueTest.java | 136 +++++ .../batchqueue/RebalanceBenchmark.java | 553 ++++++++++++++++++ 20 files changed, 1160 insertions(+), 317 deletions(-) delete mode 100644 oap-server/server-library/library-batch-queue/DESIGN.md create mode 100644 oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/DrainBalancer.java create mode 100644 oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThroughputWeightedBalancer.java create mode 100644 oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/RebalanceBenchmark.java diff --git a/docs/en/changes/changes.md b/docs/en/changes/changes.md index 2eddb03c256e..7bc5b2632cc2 100644 --- a/docs/en/changes/changes.md +++ b/docs/en/changes/changes.md @@ -23,15 +23,17 @@ | Queue | Old threads | Old channels | Old buffer slots | New threads | New partitions | New buffer slots | New policy | |-------|-------------|--------------|------------------|-------------|----------------|------------------|------------| - | L1 Aggregation (OAL) | 24 | ~1,240 | ~12.4M | 8 (unified) | ~460 adaptive | ~9.2M | `cpuCores(1.0)` | + | L1 Aggregation (OAL) | 24 | ~1,240 | ~12.4M | 8 (unified) | ~330 adaptive | ~6.6M | `cpuCores(1.0)` | | L1 Aggregation (MAL) | 2 | ~100 | ~100K | (unified above) | | | | - | L2 Persistence (OAL) | 2 | ~620 | ~1.24M | 3 (unified) | ~460 adaptive | ~920K | `cpuCoresWithBase(1, 0.25)` | + | L2 Persistence (OAL) | 2 | ~620 | ~1.24M | 3 (unified) | ~330 adaptive | ~660K | `cpuCoresWithBase(1, 0.25)` | | L2 Persistence (MAL) | 1 | ~100 | ~100K | (unified above) | | | | | TopN Persistence | 4 | 4 | 4K | 1 | 4 adaptive | 4K | `fixed(1)` | | Exporters (gRPC/Kafka) | 3 | 6 | 120K | 3 (1 per exporter) | — | 60K | `fixed(1)` each | - | **Total** | **36** | **~2,070** | **~13.9M** | **15** | **~924** | **~10.2M** | | + | **Total** | **36** | **~2,070** | **~13.9M** | **15** | **~664** | **~7.3M** | | * Remove `library-datacarrier-queue` module. All usages have been replaced by `library-batch-queue`. +* Enable throughput-weighted drain rebalancing for L1 aggregation and L2 persistence queues (10s interval). + Periodically reassigns partitions across drain threads to equalize load when metric types have skewed throughput. #### OAP Server @@ -55,8 +57,8 @@ | Catalog | Thread Name | Count | Policy | Partitions | |---------|-------------|-------|--------|------------| - | Data Pipeline | `BatchQueue-METRICS_L1_AGGREGATION-N` | 8 | `cpuCores(1.0)` | ~460 adaptive | - | Data Pipeline | `BatchQueue-METRICS_L2_PERSISTENCE-N` | 3 | `cpuCoresWithBase(1, 0.25)` | ~460 adaptive | + | Data Pipeline | `BatchQueue-METRICS_L1_AGGREGATION-N` | 8 | `cpuCores(1.0)` | ~330 adaptive | + | Data Pipeline | `BatchQueue-METRICS_L2_PERSISTENCE-N` | 3 | `cpuCoresWithBase(1, 0.25)` | ~330 adaptive | | Data Pipeline | `BatchQueue-TOPN_PERSISTENCE-N` | 1 | `fixed(1)` | ~4 adaptive | | Data Pipeline | `BatchQueue-GRPC_REMOTE_{host}_{port}-N` | 1 per peer | `fixed(1)` | `fixed(1)` | | Data Pipeline | `BatchQueue-EXPORTER_GRPC_METRICS-N` | 1 | `fixed(1)` | `fixed(1)` | diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsAggregateWorker.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsAggregateWorker.java index 729834cf6be0..c86557fb8f3d 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsAggregateWorker.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsAggregateWorker.java @@ -30,6 +30,7 @@ import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueManager; import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueStats; import org.apache.skywalking.oap.server.library.batchqueue.BufferStrategy; +import org.apache.skywalking.oap.server.library.batchqueue.DrainBalancer; import org.apache.skywalking.oap.server.library.batchqueue.HandlerConsumer; import org.apache.skywalking.oap.server.library.batchqueue.PartitionPolicy; import org.apache.skywalking.oap.server.library.batchqueue.ThreadPolicy; @@ -57,6 +58,7 @@ public class MetricsAggregateWorker extends AbstractWorker { BatchQueueConfig.builder() .threads(ThreadPolicy.cpuCores(1.0)) .partitions(PartitionPolicy.adaptive()) + .balancer(DrainBalancer.throughputWeighted(), 10_000) .bufferSize(20_000) .strategy(BufferStrategy.IF_POSSIBLE) .minIdleMs(1) diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsPersistentMinWorker.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsPersistentMinWorker.java index 7f7e9000f5a3..e121a9827d5c 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsPersistentMinWorker.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsPersistentMinWorker.java @@ -34,6 +34,7 @@ import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueManager; import org.apache.skywalking.oap.server.library.batchqueue.BatchQueueStats; import org.apache.skywalking.oap.server.library.batchqueue.BufferStrategy; +import org.apache.skywalking.oap.server.library.batchqueue.DrainBalancer; import org.apache.skywalking.oap.server.library.batchqueue.HandlerConsumer; import org.apache.skywalking.oap.server.library.batchqueue.PartitionPolicy; import org.apache.skywalking.oap.server.library.batchqueue.ThreadPolicy; @@ -59,6 +60,7 @@ public class MetricsPersistentMinWorker extends MetricsPersistentWorker { BatchQueueConfig.builder() .threads(ThreadPolicy.cpuCoresWithBase(1, 0.25)) .partitions(PartitionPolicy.adaptive()) + .balancer(DrainBalancer.throughputWeighted(), 10_000) .bufferSize(2_000) .strategy(BufferStrategy.BLOCKING) .minIdleMs(1) diff --git a/oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/remote/client/RemoteClientManagerTestCase.java b/oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/remote/client/RemoteClientManagerTestCase.java index 9ee9544a55fb..0d0239ba2dde 100644 --- a/oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/remote/client/RemoteClientManagerTestCase.java +++ b/oap-server/server-core/src/test/java/org/apache/skywalking/oap/server/core/remote/client/RemoteClientManagerTestCase.java @@ -29,6 +29,7 @@ import org.apache.skywalking.oap.server.testing.module.ModuleDefineTesting; import org.apache.skywalking.oap.server.testing.module.ModuleManagerTesting; import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Test; import org.mockito.internal.verification.AtLeast; @@ -40,7 +41,6 @@ import java.util.concurrent.ExecutorService; import java.util.concurrent.Executors; import java.util.concurrent.Future; - import static org.mockito.Mockito.any; import static org.mockito.Mockito.mock; import static org.mockito.Mockito.spy; @@ -53,6 +53,13 @@ public class RemoteClientManagerTestCase { private RemoteClientManager clientManager; private ClusterNodesQuery clusterNodesQuery; + @AfterEach + public void tearDown() { + clientManager.getRemoteClient().stream() + .filter(c -> !c.getAddress().isSelf()) + .forEach(RemoteClient::close); + } + @BeforeEach public void setup() { ModuleManagerTesting moduleManager = new ModuleManagerTesting(); diff --git a/oap-server/server-library/library-batch-queue/CLAUDE.md b/oap-server/server-library/library-batch-queue/CLAUDE.md index 4b13a2999a0a..fe798380119f 100644 --- a/oap-server/server-library/library-batch-queue/CLAUDE.md +++ b/oap-server/server-library/library-batch-queue/CLAUDE.md @@ -62,7 +62,7 @@ BatchQueueConfig.builder() |-------|------| | `BatchQueue` | The queue itself. Holds partitions, runs drain loops, dispatches to consumers/handlers. | | `BatchQueueManager` | Global registry. Creates queues by name, manages shared schedulers with ref-counting. | -| `BatchQueueConfig` | Builder for queue configuration (threads, partitions, buffer, strategy, consumer). | +| `BatchQueueConfig` | Builder for queue configuration (threads, partitions, buffer, strategy, consumer, balancer). | | `ThreadPolicy` | Resolves thread count: `fixed(N)`, `cpuCores(mult)`, `cpuCoresWithBase(base, mult)`. | | `PartitionPolicy` | Resolves partition count: `fixed(N)`, `threadMultiply(N)`, `adaptive()`. | | `PartitionSelector` | Routes items to partitions. Default `typeHash()` groups by class. | @@ -70,6 +70,7 @@ BatchQueueConfig.builder() | `BufferStrategy` | `BLOCKING` (put, waits) or `IF_POSSIBLE` (offer, drops if full). | | `BatchQueueStats` | Point-in-time snapshot of queue usage. `totalUsed()`, `topN(n)`, per-partition stats. | | `QueueErrorHandler` | Optional error callback. If absent, errors are logged. | +| `DrainBalancer` | Strategy for periodic partition-to-thread rebalancing. Default `throughputWeighted()`. | ## ThreadPolicy @@ -94,12 +95,58 @@ Adaptive growth (default multiplier 25, with 8 threads -> threshold 200): - 100 handlers -> 100 partitions (1:1) - 500 handlers -> 350 partitions (200 + 300/2) +## Drain Rebalancing + +Static round-robin partition assignment creates thread imbalance when metric types have varying +throughput (e.g., endpoint-scoped OAL >> service-scoped OAL). The `DrainBalancer` periodically +reassigns partitions to equalize per-thread load. + +### Configuration + +Opt-in via the builder's `.balancer(strategy, intervalMs)` method: + +```java +BatchQueueConfig.builder() + .threads(ThreadPolicy.cpuCores(1.0)) + .partitions(PartitionPolicy.adaptive()) + .balancer(DrainBalancer.throughputWeighted(), 300_000) // rebalance every 5 min + ... +``` + +Silently ignored for single-thread queues (nothing to rebalance). + +### How it works + +1. **Throughput counters** — `produce()` increments a per-partition `AtomicLong` counter before `put/offer`. +2. **LPT assignment** — The rebalancer snapshots and resets counters, sorts partitions by throughput descending, assigns each to the least-loaded thread (Longest Processing Time heuristic). +3. **Two-phase handoff** — Moved partitions go through revoke (UNOWNED) → wait for old owner's drain cycle to finish (cycleCount fence) → assign to new owner. This prevents concurrent handler invocations. +4. **Skip threshold** — Rebalancing is skipped when max/min thread load ratio < 1.15 (BLOCKING backpressure compresses observed ratios). + +### Safety guarantees + +| Property | Mechanism | +|----------|-----------| +| No concurrent handler calls | Two-phase handoff: revoke + cycle-count fence + assign | +| No data loss | Items stay in `ArrayBlockingQueue` during the UNOWNED gap | +| No data duplication | `drainTo` atomically moves items out of the queue | +| Lock-free hot path | Only `AtomicIntegerArray.get()` added to drain loop | +| Lock-free produce path | Only `AtomicLongArray.incrementAndGet()` added | + +### Benchmark results (4 drain threads, 16 producers, 100 types, skewed load) + +``` + Static Rebalanced + Throughput: 7,211,794 8,729,310 items/sec + Load ratio: 1.30x 1.04x (max/min thread) + Improvement: +21.0% +``` + ## Usage in the Codebase ### L1 Metrics Aggregation (`MetricsAggregateWorker`) ``` threads: cpuCores(1.0) -- 8 threads on 8-core -partitions: adaptive() -- grows with metric types (~460 for typical OAL+MAL) +partitions: adaptive() -- grows with metric types (~330 for typical OAL+MAL on 8 threads) bufferSize: 20,000 per partition strategy: IF_POSSIBLE idleMs: 1..50 diff --git a/oap-server/server-library/library-batch-queue/DESIGN.md b/oap-server/server-library/library-batch-queue/DESIGN.md deleted file mode 100644 index e5525c9d0b41..000000000000 --- a/oap-server/server-library/library-batch-queue/DESIGN.md +++ /dev/null @@ -1,259 +0,0 @@ -# Throughput-Weighted Partition Rebalancing - -## Problem - -`BatchQueue` assigns partitions to drain threads with a static round-robin mapping -(`buildAssignments`). Combined with `typeHash()` partition selection, each metric -class is pinned to exactly one partition and one drain thread. - -In a typical SkyWalking deployment, OAL metrics generate far more data than MAL -metrics. With hundreds of metric types of varying throughput, the static assignment -creates **unbalanced drain threads**: some threads are overloaded with hot OAL -partitions while others are nearly idle draining cold MAL partitions. - -The imbalance is invisible for low-throughput queues (exporters, TopN, JDBC) but -significant for **L1 aggregation** and **L2 persistence** queues, which have -`cpuCores(1.0)` or more threads and `adaptive()` partitions scaling to hundreds. - -## Design: Periodic Throughput-Weighted Reassignment - -A background rebalancer periodically measures per-partition throughput, then -reassigns partitions to threads to equalize total load per thread. - -### Data flow overview - -``` -Producer threads Drain threads Rebalancer (periodic) - | | | - | produce(data) | drainLoop(taskIndex) | every rebalanceIntervalMs: - | | | | | 1. snapshot throughput counters - | +-- typeHash() | +-- read partitionOwner 2. reset counters - | +-- put/offer | | skip if != me | 3. LPT assign partitions - | +-- bump counter | +-- drainTo + dispatch | 4. two-phase handoff - | | +-- bump cycleCount | -``` - -### Throughput counters - -Each partition has an `AtomicLong` counter, incremented on every `produce()` call. -The rebalancer snapshots and resets all counters each interval. - -```java -AtomicLongArray partitionThroughput; // one slot per partition - -// In produce(), after the put/offer: -partitionThroughput.incrementAndGet(index); -``` - -The counter is on the produce path, which is already doing an `ArrayBlockingQueue.put/offer`. -A single `incrementAndGet` adds negligible overhead (no contention — each metric type -hashes to a fixed partition, so each partition's counter is written by a predictable -set of producer threads). - -### Rebalance algorithm (LPT — Longest Processing Time) - -The rebalancer runs on the queue's scheduler (one extra scheduled task). It uses the -classic **LPT multiprocessor scheduling** heuristic: - -``` -1. snapshot = partitionThroughput[0..N-1] -2. reset all counters to 0 -3. sort partitions by snapshot[p] descending -4. threadLoad = long[taskCount], all zeros -5. newAssignment = List[taskCount] -6. for each partition p in sorted order: - t = argmin(threadLoad) // thread with least total load - newAssignment[t].add(p) - threadLoad[t] += snapshot[p] -7. two-phase handoff (see below) -``` - -LPT is O(P log P) for sorting + O(P log T) for assignment (with a min-heap for -threadLoad). For 500 partitions and 8 threads, this is sub-millisecond. - -If a partition has zero throughput in the last interval, it keeps its previous -assignment (no unnecessary moves). - -### Two-phase handoff protocol - -Reassigning a partition from Thread-A to Thread-B while Thread-A is mid-dispatch -creates a **concurrent handler invocation** — two threads calling the same -`HandlerConsumer.consume()` on different batches simultaneously. For L1 aggregation, -`MergableBufferedData` is not thread-safe, so this corrupts state. - -The race condition: - -``` -Thread-A Rebalancer Thread-B -──────── ────────── ──────── -drainTo(P3) → 500 items -dispatch(batch): - handler_X.consume(500) owner[P3] = B - ← still running drainTo(P3) → 200 new items - dispatch(batch): - handler_X.consume(200) - ← CONCURRENT! handler_X corrupted -``` - -A simple ownership gate (`if partitionOwner[p] != me: skip`) prevents the new owner -from **draining** the partition. But the old owner already drained the data and is -still **dispatching** to the handler. The new owner would drain new items and call -the same handler concurrently. - -The fix is a two-phase handoff with a cycle-count fence: - -``` -Phase 1 — Revoke: - partitionOwner[p] = UNOWNED (-1) - // Thread-A sees UNOWNED on next drainTo check, skips P3. - // But Thread-A may be mid-dispatch right now — handler still in use. - - Wait: spin until cycleCount[oldTask] > snapshot - // Once the counter increments, Thread-A has finished its current - // drain+dispatch cycle. The handler is no longer being called. - -Phase 2 — Assign: - partitionOwner[p] = newTaskIndex - // Thread-B picks up P3 on its next drain cycle. Safe — no concurrent handler call. -``` - -During the gap between phases, new items accumulate in partition P3 but are not lost. -Thread-B drains them once Phase 2 completes. - -### Cycle counter - -Each drain task increments its cycle counter at the end of every drain cycle (after -dispatch completes, before re-scheduling). The rebalancer reads this counter to know -when a task has finished any in-flight work. - -```java -AtomicLongArray cycleCount; // one slot per drain task - -void drainLoop(int taskIndex) { - // ... drain assigned partitions, dispatch ... - cycleCount.incrementAndGet(taskIndex); - // re-schedule -} -``` - -The rebalancer uses it in the revoke phase: - -```java -void movePartition(int p, int oldTask, int newTask) { - partitionOwner.set(p, UNOWNED); - - long snapshot = cycleCount.get(oldTask); - while (cycleCount.get(oldTask) <= snapshot) { - LockSupport.parkNanos(TimeUnit.MILLISECONDS.toNanos(1)); - } - - partitionOwner.set(p, newTask); -} -``` - -**Worst-case wait:** one `maxIdleMs` (the drain task may be sleeping in backoff). -For L1 aggregation this is 50ms, for L2 persistence 50ms. Since rebalancing runs -every few minutes, this latency is negligible. - -### Partition ownership array - -```java -AtomicIntegerArray partitionOwner; // partitionOwner[p] = taskIndex that owns it, or -1 -``` - -The drain loop checks ownership before draining each partition: - -```java -void drainLoop(int taskIndex) { - for (int p : assignedPartitions[taskIndex]) { - if (partitionOwner.get(p) != taskIndex) { - continue; // revoked or not yet assigned - } - partitions[p].drainTo(combined); - } - dispatch(combined); - cycleCount.incrementAndGet(taskIndex); -} -``` - -The `partitionOwner` check is one `AtomicIntegerArray.get()` per partition per drain -cycle — a volatile read with no CAS. This is the only overhead on the hot path. - -### Configuration - -Rebalancing is opt-in via `BatchQueueConfig`: - -```java -BatchQueueConfig.builder() - .threads(ThreadPolicy.cpuCores(1.0)) - .partitions(PartitionPolicy.adaptive()) - .rebalanceIntervalMs(300_000) // every 5 minutes, 0 = disabled (default) - .build(); -``` - -Only meaningful for dedicated-scheduler queues with multiple threads and partitions. -Silently ignored for single-thread or shared-scheduler queues. - -### Full rebalance cycle - -``` -1. rebalanceTask fires (scheduled every rebalanceIntervalMs) - | -2. snapshot all partitionThroughput[p], reset to 0 - | -3. skip rebalance if throughput is uniform (max/min ratio < 1.5) - | -4. LPT assignment: sort partitions by throughput desc, - assign each to the least-loaded thread - | -5. diff against current assignedPartitions - | only partitions that changed owner need handoff - | -6. for each moved partition: - | Phase 1: partitionOwner[p] = UNOWNED - | -7. for each moved partition: - | wait: cycleCount[oldTask] > snapshot_before_revoke - | -8. for each moved partition: - | Phase 2: partitionOwner[p] = newTask - | -9. update assignedPartitions (volatile write) - | -10. log summary: "rebalanced N partitions, max thread load delta: X%" -``` - -Steps 6 and 7 batch all revocations first, then wait for all old owners in parallel. -This bounds the total handoff latency to one drain cycle (the slowest old owner), -rather than one cycle per moved partition sequentially. - -### Safety guarantees - -| Property | Mechanism | -|----------|-----------| -| No concurrent handler calls | Two-phase handoff: revoke + cycle-count fence + assign | -| No data loss | Items stay in `ArrayBlockingQueue` during the UNOWNED gap | -| No data duplication | `drainTo` atomically moves items out of the queue | -| Lock-free hot path | Only `AtomicIntegerArray.get()` added to drain loop | -| Lock-free produce path | Only `AtomicLongArray.incrementAndGet()` added | -| Bounded handoff latency | At most one `maxIdleMs` wait per rebalance | - -### Scope - -| Queue | Rebalance? | Reason | -|-------|------------|--------| -| L1 Aggregation (`METRICS_L1_AGGREGATION`) | Yes | Hundreds of metric types, cpuCores threads, high throughput variance | -| L2 Persistence (`METRICS_L2_PERSISTENCE`) | Yes | Same type distribution, fewer threads but still benefits | -| TopN (`TOPN_PERSISTENCE`) | No | Single thread, nothing to rebalance | -| Exporters / gRPC Remote / JDBC | No | Single thread or fixed(1) partition, nothing to rebalance | - -### Complexity budget - -| Component | Lines (est.) | Hot-path cost | -|-----------|-------------|---------------| -| `partitionThroughput` counter | ~5 | 1 `AtomicLong.incrementAndGet` per produce | -| `partitionOwner` check in drain loop | ~5 | 1 `AtomicInteger.get` per partition per cycle | -| `cycleCount` bump | ~2 | 1 `AtomicLong.incrementAndGet` per drain cycle | -| Rebalance task (LPT + handoff) | ~80 | 0 (runs on scheduler, not on hot path) | -| Config field + validation | ~10 | 0 | -| **Total** | **~100** | **2 atomic ops per produce+drain** | diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java index d86d5d05aa9f..7cc0e599c5ab 100644 --- a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java @@ -213,6 +213,11 @@ public class BatchQueue { private final int[] consecutiveIdleCycles; /** Set to false on {@link #shutdown()} to stop drain loops and reject new data. */ + /** + * Whether the queue is currently accepting produces and running drain loops. + * + * @return true if the queue is running + */ @Getter private volatile boolean running; @@ -424,6 +429,9 @@ private void scheduleDrain(final int taskIndex) { * count and grows the partition array if needed. For non-adaptive policies the * resolved count never changes, so this is a no-op beyond the registration. * Drain loop threads pick up new partitions on their next cycle via volatile reads. + * + * @param type the class of items to route to this handler + * @param handler the consumer that processes batches of the given type */ @SuppressWarnings("unchecked") public void addHandler(final Class type, final HandlerConsumer handler) { @@ -522,6 +530,7 @@ private void enableRebalancing(final long intervalMs) { *
  • IF_POSSIBLE — returns false immediately if the partition is full (data dropped)
  • * * + * @param data the item to enqueue * @return true if data was accepted, false if dropped or queue is stopped */ public boolean produce(final T data) { @@ -844,6 +853,8 @@ int getTaskCount() { /** * Take a point-in-time snapshot of queue usage across all partitions. + * + * @return a stats snapshot containing per-partition usage and capacity */ public BatchQueueStats stats() { final ArrayBlockingQueue[] currentPartitions = this.partitions; diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueConfig.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueConfig.java index 5de0fd39d2be..dea210ec45e9 100644 --- a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueConfig.java +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueConfig.java @@ -73,6 +73,21 @@ public class BatchQueueConfig { @Builder.Default private long maxIdleMs = 50; + /** + * Drain balancer for periodic rebalancing of partition-to-thread assignments. + * Set via {@code .balancer(DrainBalancer, intervalMs)} on the builder. + * When null (default), rebalancing is disabled. + * + * @see DrainBalancer#throughputWeighted() + */ + private DrainBalancer balancer; + + /** + * Rebalance interval in milliseconds. Set together with {@link #balancer} + * via {@code .balancer(DrainBalancer, intervalMs)} on the builder. + */ + private long rebalanceIntervalMs; + void validate() { final boolean hasDedicated = threads != null; final boolean hasShared = sharedSchedulerName != null; @@ -98,13 +113,33 @@ void validate() { } /** - * Builder customization: convenience method for setting shared scheduler fields together. + * Builder customizations: convenience methods for setting paired fields together. */ public static class BatchQueueConfigBuilder { + /** + * Configure the queue to use a shared scheduler instead of a dedicated one. + * + * @param name the shared scheduler name (queues with the same name share a pool) + * @param threads the thread policy for the shared scheduler + * @return this builder + */ public BatchQueueConfigBuilder sharedScheduler(final String name, final ThreadPolicy threads) { this.sharedSchedulerName = name; this.sharedSchedulerThreads = threads; return this; } + + /** + * Enable periodic drain rebalancing with the given strategy and interval. + * + * @param balancer rebalancing strategy (e.g. {@link DrainBalancer#throughputWeighted()}) + * @param intervalMs rebalance interval in milliseconds + * @return this builder + */ + public BatchQueueConfigBuilder balancer(final DrainBalancer balancer, final long intervalMs) { + this.balancer = balancer; + this.rebalanceIntervalMs = intervalMs; + return this; + } } } diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManager.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManager.java index af75eb841bf9..063bc9673a42 100644 --- a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManager.java +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManager.java @@ -80,7 +80,7 @@ static ScheduledExecutorService getOrCreateSharedScheduler(final String name, final ThreadPolicy threads) { SHARED_SCHEDULER_POLICIES.compute(name, (k, existing) -> { if (existing != null) { - if (!existing.toString().equals(threads.toString())) { + if (!existing.equals(threads)) { log.warn("Shared scheduler [{}]: ThreadPolicy mismatch. " + "Existing={}, requested={}. Using existing.", name, existing, threads); @@ -128,7 +128,14 @@ static void releaseSharedScheduler(final String name) { @SuppressWarnings("unchecked") public static BatchQueue create(final String name, final BatchQueueConfig config) { config.validate(); - return (BatchQueue) QUEUES.computeIfAbsent(name, k -> new BatchQueue<>(name, config)); + final BatchQueue queue = new BatchQueue<>(name, config); + final BatchQueue existing = QUEUES.putIfAbsent(name, queue); + if (existing != null) { + queue.shutdown(); + throw new IllegalStateException( + "BatchQueue [" + name + "] already exists. Each queue name must be unique."); + } + return queue; } @SuppressWarnings("unchecked") diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueStats.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueStats.java index 2a0b64dff341..1a18da9c9fc6 100644 --- a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueStats.java +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueStats.java @@ -48,6 +48,8 @@ public class BatchQueueStats { /** * Total capacity across all partitions: {@code partitionCount * bufferSize}. + * + * @return total capacity in item slots */ public long totalCapacity() { return (long) partitionCount * bufferSize; @@ -55,6 +57,8 @@ public long totalCapacity() { /** * Total number of items currently queued across all partitions. + * + * @return sum of items across all partitions */ public int totalUsed() { int sum = 0; @@ -66,6 +70,8 @@ public int totalUsed() { /** * Overall queue usage as a percentage (0.0–100.0). + * + * @return usage percentage across all partitions */ public double totalUsedPercentage() { final long capacity = totalCapacity(); @@ -77,6 +83,9 @@ public double totalUsedPercentage() { /** * Number of items currently queued in the given partition. + * + * @param index the partition index + * @return number of items in the partition */ public int partitionUsed(final int index) { return partitionUsed[index]; @@ -84,6 +93,9 @@ public int partitionUsed(final int index) { /** * Usage of the given partition as a percentage (0.0–100.0). + * + * @param index the partition index + * @return usage percentage for the partition */ public double partitionUsedPercentage(final int index) { if (bufferSize == 0) { @@ -94,7 +106,10 @@ public double partitionUsedPercentage(final int index) { /** * Return the top {@code n} most-loaded partitions, sorted by usage descending. - * If {@code n >= partitionCount}, all partitions are returned. + * If {@code n >= partitionCount}, all partitions are returned. + * + * @param n the maximum number of partitions to return + * @return list of partition usage snapshots sorted by usage descending */ public List topN(final int n) { final Integer[] indices = new Integer[partitionCount]; diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/DrainBalancer.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/DrainBalancer.java new file mode 100644 index 000000000000..5d5cd99933d8 --- /dev/null +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/DrainBalancer.java @@ -0,0 +1,58 @@ +/* + * 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.skywalking.oap.server.library.batchqueue; + +/** + * Strategy for rebalancing partition-to-thread assignments based on observed throughput. + * + *

    The balancer is invoked periodically by the queue's rebalance task. It receives + * a throughput snapshot and current ownership, and returns a new assignment. The queue + * infrastructure handles the two-phase handoff protocol to ensure safe reassignment. + * + *

    Implementations only need to solve the assignment problem — they do not need to + * worry about concurrent handler invocations or data loss, which are handled by + * {@link BatchQueue}. + * + * @see ThroughputWeightedBalancer + */ +public interface DrainBalancer { + + /** + * Compute new partition-to-task assignments. + * + * @param throughput per-partition throughput since last rebalance (snapshot, already reset) + * @param currentOwner current partition-to-task mapping ({@code currentOwner[p] = taskIndex}) + * @param taskCount number of drain tasks/threads + * @return new owner for each partition ({@code result[p] = taskIndex}), + * or {@code null} to skip rebalancing this cycle + */ + int[] assign(long[] throughput, int[] currentOwner, int taskCount); + + /** + * Throughput-weighted balancer using the LPT (Longest Processing Time) heuristic. + * Sorts partitions by throughput descending, assigns each to the least-loaded thread. + * Zero-throughput partitions keep their current owner to avoid unnecessary moves. + * Skips rebalancing when load is already balanced (max/min ratio < 1.15). + * + * @return a throughput-weighted drain balancer + */ + static DrainBalancer throughputWeighted() { + return new ThroughputWeightedBalancer(); + } +} diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/HandlerConsumer.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/HandlerConsumer.java index 0b9e583654b3..8ab5f8c729e7 100644 --- a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/HandlerConsumer.java +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/HandlerConsumer.java @@ -27,6 +27,8 @@ public interface HandlerConsumer { /** * Process a batch of data belonging to this handler's type. + * + * @param data the batch of items to process */ void consume(List data); diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/PartitionPolicy.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/PartitionPolicy.java index 6f3428a6f371..8ec0a0c4cf50 100644 --- a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/PartitionPolicy.java +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/PartitionPolicy.java @@ -53,6 +53,8 @@ private PartitionPolicy(final int fixedCount, final int multiplier, /** * Fixed number of partitions. * + * @param count the exact number of partitions + * @return a PartitionPolicy with a fixed partition count * @throws IllegalArgumentException if count < 1 */ public static PartitionPolicy fixed(final int count) { @@ -65,6 +67,8 @@ public static PartitionPolicy fixed(final int count) { /** * Partitions = multiplier * resolved thread count. * + * @param multiplier factor applied to thread count + * @return a PartitionPolicy that scales with thread count * @throws IllegalArgumentException if multiplier < 1 */ public static PartitionPolicy threadMultiply(final int multiplier) { @@ -95,6 +99,8 @@ public static PartitionPolicy threadMultiply(final int multiplier) { * 1000 handlers → 600 partitions (200 + 800/2) * 2000 handlers → 1100 partitions (200 + 1800/2) * + * + * @return an adaptive PartitionPolicy with default threshold multiplier */ public static PartitionPolicy adaptive() { return new PartitionPolicy(0, DEFAULT_ADAPTIVE_MULTIPLIER, true); @@ -108,6 +114,7 @@ public static PartitionPolicy adaptive() { * at 1:2 ratio: {@code threshold + (handlerCount - threshold) / 2}. * * @param multiplier threshold per thread (default 25) + * @return a PartitionPolicy that grows with handler registrations * @throws IllegalArgumentException if multiplier < 1 */ public static PartitionPolicy adaptive(final int multiplier) { @@ -128,6 +135,10 @@ public static PartitionPolicy adaptive(final int multiplier) { * <= threshold, returns handlerCount (1:1). If above, returns * threshold + (handlerCount - threshold) / 2. * + * + * @param resolvedThreadCount the resolved number of drain threads + * @param handlerCount the current number of registered type handlers + * @return the resolved partition count, always >= 1 */ public int resolve(final int resolvedThreadCount, final int handlerCount) { if (fixedCount > 0) { diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/PartitionSelector.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/PartitionSelector.java index d30d74a05e1b..a1b6de472469 100644 --- a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/PartitionSelector.java +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/PartitionSelector.java @@ -47,6 +47,9 @@ public interface PartitionSelector { * Default selector: routes by {@code data.getClass().hashCode()}. * Same type always hits the same partition, so each consumer thread * drains pre-grouped batches — dispatch grouping is effectively a no-op. + * + * @param the queue element type + * @return a selector that partitions by item class hash */ static PartitionSelector typeHash() { return (data, count) -> (data.getClass().hashCode() & 0x7FFFFFFF) % count; diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicy.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicy.java index e38d5cfc87e7..33fde8dfef21 100644 --- a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicy.java +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicy.java @@ -44,7 +44,9 @@ private ThreadPolicy(final int fixedCount, final int base, final double cpuMulti /** * Fixed number of threads. Count must be >= 1. * - * @throws IllegalArgumentException if count < 1 + * @param count the exact number of threads + * @return a ThreadPolicy with a fixed thread count + * @throws IllegalArgumentException if count < 1 */ public static ThreadPolicy fixed(final int count) { if (count < 1) { @@ -57,7 +59,9 @@ public static ThreadPolicy fixed(final int count) { * Threads = multiplier * available CPU cores, rounded, min 1. * Multiplier must be > 0. * - * @throws IllegalArgumentException if multiplier <= 0 + * @param multiplier factor applied to available CPU core count + * @return a ThreadPolicy proportional to CPU cores + * @throws IllegalArgumentException if multiplier <= 0 */ public static ThreadPolicy cpuCores(final double multiplier) { if (multiplier <= 0) { @@ -72,7 +76,10 @@ public static ThreadPolicy cpuCores(final double multiplier) { * * Example: cpuCoresWithBase(2, 0.25) on 8-core = 2 + 2 = 4, on 16-core = 2 + 4 = 6, on 24-core = 2 + 6 = 8. * - * @throws IllegalArgumentException if base < 0 or multiplier <= 0 + * @param base fixed base thread count added to the CPU-proportional portion + * @param multiplier factor applied to available CPU core count + * @return a ThreadPolicy that combines a fixed base with a CPU-proportional count + * @throws IllegalArgumentException if base < 0 or multiplier <= 0 */ public static ThreadPolicy cpuCoresWithBase(final int base, final double multiplier) { if (base < 0) { @@ -86,6 +93,8 @@ public static ThreadPolicy cpuCoresWithBase(final int base, final double multipl /** * Resolve the actual thread count. Always returns >= 1. + * + * @return the resolved thread count, at least 1 */ public int resolve() { if (fixedCount > 0) { @@ -94,6 +103,29 @@ public int resolve() { return Math.max(1, base + (int) Math.round(cpuMultiplier * Runtime.getRuntime().availableProcessors())); } + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final ThreadPolicy that = (ThreadPolicy) o; + return fixedCount == that.fixedCount + && base == that.base + && Double.compare(that.cpuMultiplier, cpuMultiplier) == 0; + } + + @Override + public int hashCode() { + int result = fixedCount; + result = 31 * result + base; + final long temp = Double.doubleToLongBits(cpuMultiplier); + result = 31 * result + (int) (temp ^ (temp >>> 32)); + return result; + } + @Override public String toString() { if (fixedCount > 0) { diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThroughputWeightedBalancer.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThroughputWeightedBalancer.java new file mode 100644 index 000000000000..2a2fab05ee9d --- /dev/null +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThroughputWeightedBalancer.java @@ -0,0 +1,102 @@ +/* + * 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.skywalking.oap.server.library.batchqueue; + +import java.util.Arrays; + +/** + * LPT (Longest Processing Time) drain balancer. + * + *

    Sorts partitions by observed throughput in descending order, then assigns each + * partition to the thread with the least total load. This is the classic multiprocessor + * scheduling heuristic — O(P log P) for sorting + O(P * T) for assignment. + * + *

    Zero-throughput partitions keep their current owner to avoid unnecessary moves. + * Returns {@code null} (skip) when load is already balanced (max/min thread load ratio < 1.15) + * or when there is no throughput. The 1.15 threshold is intentionally low because BLOCKING + * backpressure compresses the observed throughput ratio — a true 2x imbalance may appear + * as only ~1.3x in the counters. + */ +class ThroughputWeightedBalancer implements DrainBalancer { + + @Override + public int[] assign(final long[] throughput, final int[] currentOwner, final int taskCount) { + final int partitionCount = throughput.length; + + // Compute per-thread load under current assignment to check imbalance + final long[] threadLoad = new long[taskCount]; + for (int p = 0; p < partitionCount; p++) { + final int t = currentOwner[p]; + if (t >= 0 && t < taskCount) { + threadLoad[t] += throughput[p]; + } + } + long maxLoad = 0; + long minLoad = Long.MAX_VALUE; + for (int t = 0; t < taskCount; t++) { + if (threadLoad[t] > maxLoad) { + maxLoad = threadLoad[t]; + } + if (threadLoad[t] < minLoad) { + minLoad = threadLoad[t]; + } + } + // Skip if load is already balanced (max/min ratio < 1.15, or no throughput) + if (maxLoad == 0 || (minLoad > 0 && maxLoad * 100 / minLoad < 115)) { + return null; + } + + // Sort partitions by throughput descending + final int[][] sorted = new int[partitionCount][2]; + for (int p = 0; p < partitionCount; p++) { + sorted[p][0] = p; + sorted[p][1] = (int) Math.min(throughput[p], Integer.MAX_VALUE); + } + Arrays.sort(sorted, (a, b) -> Integer.compare(b[1], a[1])); + + // Assign each partition to the least-loaded thread + final long[] newThreadLoad = new long[taskCount]; + final int[] newOwner = new int[partitionCount]; + for (final int[] entry : sorted) { + final int p = entry[0]; + final long tp = throughput[p]; + + if (tp == 0) { + // Zero-throughput partition keeps current owner (avoid unnecessary moves) + newOwner[p] = currentOwner[p]; + if (newOwner[p] >= 0) { + newThreadLoad[newOwner[p]] += tp; + } + continue; + } + + // Find thread with least total load (linear scan — taskCount is small) + int bestThread = 0; + for (int t = 1; t < taskCount; t++) { + if (newThreadLoad[t] < newThreadLoad[bestThread]) { + bestThread = t; + } + } + newOwner[p] = bestThread; + newThreadLoad[bestThread] += tp; + } + + return newOwner; + } +} diff --git a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueBenchmark.java b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueBenchmark.java index 115a03fcadeb..db3cbff07623 100644 --- a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueBenchmark.java +++ b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueBenchmark.java @@ -198,164 +198,232 @@ public void benchmark2000TypesBlocking_128p() throws Exception { @Test public void benchmark500Types_quarter() throws Exception { runBenchmark("500-types-quarter", 500, 125, 50_000, - BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, true); + BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, false); } @Test public void benchmark1000Types_quarter() throws Exception { runBenchmark("1000-types-quarter", 1000, 250, 50_000, - BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, true); + BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, false); } @Test public void benchmark2000Types_quarter() throws Exception { runBenchmark("2000-types-quarter", 2000, 500, 50_000, - BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, true); + BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, false); } @Test public void benchmark500TypesBlocking_quarter() throws Exception { runBenchmark("500-types-blocking-quarter", 500, 125, 50_000, - BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, true); + BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, false); } @Test public void benchmark1000TypesBlocking_quarter() throws Exception { runBenchmark("1000-types-blocking-quarter", 1000, 250, 50_000, - BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, true); + BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, false); } @Test public void benchmark2000TypesBlocking_quarter() throws Exception { runBenchmark("2000-types-blocking-quarter", 2000, 500, 50_000, - BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, true); + BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, false); } @Test public void benchmark500Types_half() throws Exception { runBenchmark("500-types-half", 500, 250, 50_000, - BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, true); + BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, false); } @Test public void benchmark1000Types_half() throws Exception { runBenchmark("1000-types-half", 1000, 500, 50_000, - BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, true); + BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, false); } @Test public void benchmark2000Types_half() throws Exception { runBenchmark("2000-types-half", 2000, 1000, 50_000, - BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, true); + BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, false); } @Test public void benchmark500TypesBlocking_half() throws Exception { runBenchmark("500-types-blocking-half", 500, 250, 50_000, - BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, true); + BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, false); } @Test public void benchmark1000TypesBlocking_half() throws Exception { runBenchmark("1000-types-blocking-half", 1000, 500, 50_000, - BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, true); + BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, false); } @Test public void benchmark2000TypesBlocking_half() throws Exception { runBenchmark("2000-types-blocking-half", 2000, 1000, 50_000, - BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, true); + BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, false); } @Test public void benchmark500Types_adaptive() throws Exception { - runAdaptiveBenchmark("500-types-adaptive", 500, BufferStrategy.IF_POSSIBLE); + runAdaptiveBenchmark("500-types-adaptive", 500, BufferStrategy.IF_POSSIBLE, false); } @Test public void benchmark1000Types_adaptive() throws Exception { - runAdaptiveBenchmark("1000-types-adaptive", 1000, BufferStrategy.IF_POSSIBLE); + runAdaptiveBenchmark("1000-types-adaptive", 1000, BufferStrategy.IF_POSSIBLE, false); } @Test public void benchmark2000Types_adaptive() throws Exception { - runAdaptiveBenchmark("2000-types-adaptive", 2000, BufferStrategy.IF_POSSIBLE); + runAdaptiveBenchmark("2000-types-adaptive", 2000, BufferStrategy.IF_POSSIBLE, false); } @Test public void benchmark500TypesBlocking_adaptive() throws Exception { - runAdaptiveBenchmark("500-types-blocking-adaptive", 500, BufferStrategy.BLOCKING); + runAdaptiveBenchmark("500-types-blocking-adaptive", 500, BufferStrategy.BLOCKING, false); } @Test public void benchmark1000TypesBlocking_adaptive() throws Exception { - runAdaptiveBenchmark("1000-types-blocking-adaptive", 1000, BufferStrategy.BLOCKING); + runAdaptiveBenchmark("1000-types-blocking-adaptive", 1000, BufferStrategy.BLOCKING, false); } @Test public void benchmark2000TypesBlocking_adaptive() throws Exception { - runAdaptiveBenchmark("2000-types-blocking-adaptive", 2000, BufferStrategy.BLOCKING); + runAdaptiveBenchmark("2000-types-blocking-adaptive", 2000, BufferStrategy.BLOCKING, false); } @Test public void benchmark500Types_1to1() throws Exception { runBenchmark("500-types-1to1", 500, 500, 50_000, - BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, true); + BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, false); } @Test public void benchmark1000Types_1to1() throws Exception { runBenchmark("1000-types-1to1", 1000, 1000, 50_000, - BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, true); + BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, false); } @Test public void benchmark2000Types_1to1() throws Exception { runBenchmark("2000-types-1to1", 2000, 2000, 50_000, - BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, true); + BufferStrategy.IF_POSSIBLE, TYPE_ID_SELECTOR, false); } @Test public void benchmark500TypesBlocking_1to1() throws Exception { runBenchmark("500-types-blocking-1to1", 500, 500, 50_000, - BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, true); + BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, false); } @Test public void benchmark1000TypesBlocking_1to1() throws Exception { runBenchmark("1000-types-blocking-1to1", 1000, 1000, 50_000, - BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, true); + BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, false); } @Test public void benchmark2000TypesBlocking_1to1() throws Exception { runBenchmark("2000-types-blocking-1to1", 2000, 2000, 50_000, - BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, true); + BufferStrategy.BLOCKING, TYPE_ID_SELECTOR, false); } - private void runAdaptiveBenchmark(final String label, final int typeCount, - final BufferStrategy strategy) throws Exception { + /** + * Run the same adaptive benchmark 3 times each (static vs rebalanced) + * to distinguish real overhead from run-to-run variance. + */ + @Test + public void benchmarkRebalanceOverhead() throws Exception { + log.info("\n========================================"); + log.info(" Rebalance overhead: 1000 types, BLOCKING"); + log.info("========================================\n"); + + final int typeCount = 1000; + final BufferStrategy strategy = BufferStrategy.BLOCKING; + final int runs = 3; + + final double[] staticRates = new double[runs]; + final double[] rebalRates = new double[runs]; + + for (int i = 0; i < runs; i++) { + staticRates[i] = runAdaptiveBenchmark( + "overhead-static-" + i, typeCount, strategy, false); + cleanup(); + rebalRates[i] = runAdaptiveBenchmark( + "overhead-rebal-" + i, typeCount, strategy, true); + cleanup(); + } + + // Compute stats + double staticSum = 0; + double rebalSum = 0; + for (int i = 0; i < runs; i++) { + staticSum += staticRates[i]; + rebalSum += rebalRates[i]; + } + final double staticAvg = staticSum / runs; + final double rebalAvg = rebalSum / runs; + + double staticVarSum = 0; + double rebalVarSum = 0; + for (int i = 0; i < runs; i++) { + staticVarSum += (staticRates[i] - staticAvg) * (staticRates[i] - staticAvg); + rebalVarSum += (rebalRates[i] - rebalAvg) * (rebalRates[i] - rebalAvg); + } + final double staticStddev = Math.sqrt(staticVarSum / runs); + final double rebalStddev = Math.sqrt(rebalVarSum / runs); + + log.info("\n--- OVERHEAD ANALYSIS ---"); + log.info(" Run Static (items/s) Rebalanced (items/s)"); + log.info(" --- ---------------- --------------------"); + for (int i = 0; i < runs; i++) { + log.info(" {} {} {}", + i + 1, + String.format("%16s", String.format("%,.0f", staticRates[i])), + String.format("%16s", String.format("%,.0f", rebalRates[i]))); + } + log.info(" Avg {} {}", + String.format("%16s", String.format("%,.0f", staticAvg)), + String.format("%16s", String.format("%,.0f", rebalAvg))); + log.info(" Std {} {}", + String.format("%16s", String.format("%,.0f", staticStddev)), + String.format("%16s", String.format("%,.0f", rebalStddev))); + final double delta = staticAvg > 0 ? (rebalAvg - staticAvg) / staticAvg * 100 : 0; + final double noiseRange = staticAvg > 0 ? staticStddev / staticAvg * 100 : 0; + log.info(" Delta: {}% (noise range: +/-{}%)", + String.format("%+.1f", delta), String.format("%.1f", noiseRange)); + log.info(""); + } + + private double runAdaptiveBenchmark(final String label, final int typeCount, + final BufferStrategy strategy, + final boolean rebalance) throws Exception { // adaptive(): threshold = threadCount * 25 = 200 // 500 types → 350p (200 + 300/2) // 1000 types → 600p (200 + 800/2) // 2000 types → 1100p (200 + 1800/2) final int partitionCount = PartitionPolicy.adaptive() .resolve(THREADS.resolve(), typeCount); - runBenchmark(label, typeCount, partitionCount, 50_000, - strategy, TYPE_ID_SELECTOR, true); + return runBenchmark(label, typeCount, partitionCount, 50_000, + strategy, TYPE_ID_SELECTOR, rebalance); } - private void runBenchmark(final String label, final int typeCount, - final int partitionCount, final int bufferSize, - final BufferStrategy strategy) throws Exception { - runBenchmark(label, typeCount, partitionCount, bufferSize, strategy, null, true); + private double runBenchmark(final String label, final int typeCount, + final int partitionCount, final int bufferSize, + final BufferStrategy strategy) throws Exception { + return runBenchmark(label, typeCount, partitionCount, bufferSize, strategy, null, false); } - private void runBenchmark(final String label, final int typeCount, - final int partitionCount, final int bufferSize, - final BufferStrategy strategy, - final PartitionSelector selector, - final boolean ignored) throws Exception { + private double runBenchmark(final String label, final int typeCount, + final int partitionCount, final int bufferSize, + final BufferStrategy strategy, + final PartitionSelector selector, + final boolean rebalance) throws Exception { final AtomicLong consumed = new AtomicLong(0); final PartitionPolicy partitions = PartitionPolicy.fixed(partitionCount); @@ -370,6 +438,9 @@ private void runBenchmark(final String label, final int typeCount, if (selector != null) { configBuilder.partitionSelector(selector); } + if (rebalance) { + configBuilder.balancer(DrainBalancer.throughputWeighted(), 2000); + } final BatchQueue queue = BatchQueueManager.create( "bench-" + label, configBuilder.build()); @@ -394,6 +465,7 @@ private void runBenchmark(final String label, final int typeCount, Thread.sleep(500); final long totalConsumed = consumed.get(); + final double consumeRate = totalConsumed * 1000.0 / measureDuration; log.info("\n=== BatchQueue Benchmark: {} ===\n" + " Types: {}\n" @@ -401,18 +473,22 @@ private void runBenchmark(final String label, final int typeCount, + " Partitions: {}\n" + " BufferSize: {}\n" + " Strategy: {}\n" + + " Rebalance: {}\n" + " Producers: {}\n" + " Duration: {} ms\n" + " Produced: {}\n" + " Consumed: {}\n" + " Consume rate: {} items/sec\n" + " Drop rate: {}%\n", - label, typeCount, THREADS, partitions, bufferSize, strategy, PRODUCER_THREADS, + label, typeCount, THREADS, partitions, bufferSize, strategy, + rebalance, PRODUCER_THREADS, measureDuration, String.format("%,d", produced), String.format("%,d", totalConsumed), - String.format("%,.0f", totalConsumed * 1000.0 / measureDuration), + String.format("%,.0f", consumeRate), String.format("%.2f", produced > 0 ? (produced - totalConsumed) * 100.0 / produced : 0)); + + return consumeRate; } private long runProducers(final BatchQueue queue, diff --git a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManagerTest.java b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManagerTest.java index 17b6226a37ee..738bb83ca6cb 100644 --- a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManagerTest.java +++ b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManagerTest.java @@ -28,6 +28,7 @@ import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertSame; +import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; public class BatchQueueManagerTest { @@ -54,16 +55,16 @@ public void testCreateAndGet() { } @Test - public void testCreateReturnsExistingIfPresent() { + public void testCreateThrowsOnDuplicateName() { final BatchQueueConfig config = BatchQueueConfig.builder() .threads(ThreadPolicy.fixed(1)) .consumer(data -> { }) .bufferSize(100) .build(); - final BatchQueue first = BatchQueueManager.create("absent-test", config); - final BatchQueue second = BatchQueueManager.create("absent-test", config); - assertSame(first, second); + BatchQueueManager.create("absent-test", config); + assertThrows(IllegalStateException.class, + () -> BatchQueueManager.create("absent-test", config)); } @Test diff --git a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueTest.java b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueTest.java index 1f3795102fee..c6a2aa18c927 100644 --- a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueTest.java +++ b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueTest.java @@ -674,6 +674,142 @@ public void testStatsTopNReturnsHottestPartitions() { blockLatch.countDown(); } + // --- Rebalancing --- + + @Test + public void testRebalancingIgnoredForSingleThread() { + final BatchQueue queue = BatchQueueManager.create("rebal-single", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .partitions(PartitionPolicy.fixed(4)) + .balancer(DrainBalancer.throughputWeighted(), 1000) + .consumer(data -> { }) + .bufferSize(100) + .build()); + + // Single thread — nothing to rebalance, silently skipped + assertFalse(queue.isRebalancingEnabled()); + } + + @Test + public void testRebalancingIgnoredWithoutBalancer() { + final BatchQueue queue = BatchQueueManager.create("rebal-no-balancer", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(2)) + .partitions(PartitionPolicy.fixed(4)) + .consumer(data -> { }) + .bufferSize(100) + .build()); + + // No balancer configured — rebalancing not enabled + assertFalse(queue.isRebalancingEnabled()); + } + + @Test + public void testRebalancingNoDataLoss() throws Exception { + final AtomicInteger totalReceived = new AtomicInteger(0); + final int itemCount = 5000; + + final BatchQueue queue = BatchQueueManager.create("rebal-no-loss", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(2)) + .partitions(PartitionPolicy.fixed(8)) + .balancer(DrainBalancer.throughputWeighted(), 100) + .bufferSize(2000) + .strategy(BufferStrategy.BLOCKING) + .build()); + + // Register 8 handler types, each just counts + for (int i = 0; i < 8; i++) { + queue.addHandler(BenchmarkMetricTypes.CLASSES[i], + data -> totalReceived.addAndGet(data.size())); + } + + assertTrue(queue.isRebalancingEnabled()); + + // Produce items across all 8 types + for (int i = 0; i < itemCount; i++) { + queue.produce(BenchmarkMetricTypes.FACTORIES[i % 8].create(i)); + } + + Awaitility.await().atMost(10, TimeUnit.SECONDS) + .until(() -> totalReceived.get() == itemCount); + + assertEquals(itemCount, totalReceived.get()); + } + + @Test + public void testRebalancingRedistributesPartitions() throws Exception { + final AtomicInteger totalReceived = new AtomicInteger(0); + + final BatchQueue queue = BatchQueueManager.create("rebal-redistribute", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(2)) + .partitions(PartitionPolicy.fixed(4)) + .balancer(DrainBalancer.throughputWeighted(), 200) + .bufferSize(2000) + .strategy(BufferStrategy.IF_POSSIBLE) + .build()); + + // Register 4 handler types + for (int i = 0; i < 4; i++) { + queue.addHandler(BenchmarkMetricTypes.CLASSES[i], + data -> totalReceived.addAndGet(data.size())); + } + + // Produce heavily skewed load: type 0 gets 90% of items + for (int round = 0; round < 5; round++) { + for (int i = 0; i < 900; i++) { + queue.produce(BenchmarkMetricTypes.FACTORIES[0].create(i)); + } + for (int i = 0; i < 34; i++) { + queue.produce(BenchmarkMetricTypes.FACTORIES[1].create(i)); + queue.produce(BenchmarkMetricTypes.FACTORIES[2].create(i)); + queue.produce(BenchmarkMetricTypes.FACTORIES[3].create(i)); + } + Thread.sleep(50); + } + + // Wait for all data to be consumed + final int expectedTotal = 5 * (900 + 34 * 3); + Awaitility.await().atMost(10, TimeUnit.SECONDS) + .until(() -> totalReceived.get() == expectedTotal); + + assertEquals(expectedTotal, totalReceived.get()); + } + + @Test + public void testRebalancingWithPartitionGrowth() { + final AtomicInteger totalReceived = new AtomicInteger(0); + + final BatchQueue queue = BatchQueueManager.create("rebal-growth", + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(2)) + .partitions(PartitionPolicy.adaptive()) + .balancer(DrainBalancer.throughputWeighted(), 200) + .bufferSize(1000) + .build()); + + assertTrue(queue.isRebalancingEnabled()); + + // Add handlers — this grows partitions while rebalancing is active + for (int i = 0; i < 20; i++) { + queue.addHandler(BenchmarkMetricTypes.CLASSES[i], + data -> totalReceived.addAndGet(data.size())); + } + + // Produce items across all 20 types + final int itemCount = 2000; + for (int i = 0; i < itemCount; i++) { + queue.produce(BenchmarkMetricTypes.FACTORIES[i % 20].create(i)); + } + + Awaitility.await().atMost(10, TimeUnit.SECONDS) + .until(() -> totalReceived.get() == itemCount); + + assertEquals(itemCount, totalReceived.get()); + } + @Test public void testBackoffResetsOnData() throws Exception { final AtomicInteger consumeCount = new AtomicInteger(0); diff --git a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/RebalanceBenchmark.java b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/RebalanceBenchmark.java new file mode 100644 index 000000000000..ef3ee33c5b7f --- /dev/null +++ b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/RebalanceBenchmark.java @@ -0,0 +1,553 @@ +/* + * 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.skywalking.oap.server.library.batchqueue; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.TreeMap; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.CountDownLatch; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicLong; +import lombok.extern.slf4j.Slf4j; +import org.junit.jupiter.api.AfterEach; +import org.junit.jupiter.api.Test; + +/** + * Benchmark comparing throughput with and without partition rebalancing + * under skewed load simulating OAP L2 persistence. + * + *

    Scenario: L2 entity-count-driven imbalance

    + * After L1 merge, each metric type produces one item per unique entity per minute. + * Endpoint-scoped metrics see many more entities than service-scoped metrics: + *
    + *   endpoint-level OAL:  ~24 entities (6 endpoints × 4 services)
    + *   instance-level OAL:  ~8 entities  (2 instances × 4 services)
    + *   service-level OAL:   ~4 entities  (4 services)
    + *   MAL / cold:          ~1 entity    (periodic scrape)
    + * 
    + * + * With {@code typeHash()} partition selection and round-robin thread assignment, + * some threads get more endpoint-scoped types by chance, creating load imbalance. + * The throughput-weighted rebalancer fixes this by reassigning partitions based + * on observed throughput. + * + *

    What this benchmark measures

    + *
      + *
    1. Static vs rebalanced throughput: total consumed items/sec with + * BLOCKING strategy and simulated consumer work (~500ns/item). With imbalance, + * producers block on full partitions of the overloaded thread; rebalancing + * equalizes thread load and reduces blocking.
    2. + *
    3. Rebalance stability: once throughput distribution is stable, the + * rebalancer should converge to a fixed assignment. Measured by sampling + * per-thread load ratio over multiple intervals.
    4. + *
    + * + *

    Results (4 drain threads, 16 producers, 100 types, 500 LCG iters/item)

    + *
    + *                     Static          Rebalanced
    + *   Throughput:    7,211,794         8,729,310  items/sec
    + *   Load ratio:       1.30x             1.04x  (max/min thread)
    + *   Improvement:                       +21.0%
    + * 
    + * + *

    Stability (20 sec, sampled every 2 sec after initial rebalance)

    + *
    + *   Interval    Throughput      Ratio
    + *    0- 2s     8,915,955       1.00x
    + *    2- 4s     8,956,595       1.01x
    + *    4- 6s     8,934,778       1.00x
    + *    6- 8s     8,838,461       1.01x
    + *    8-10s     8,887,092       1.00x
    + *   10-12s     8,844,614       1.00x
    + *   12-14s     8,877,651       1.00x
    + *   14-16s     8,851,595       1.01x
    + *   16-18s     8,639,045       1.01x
    + *   18-20s     8,708,210       1.01x
    + *   Stable: YES (avg ratio 1.01x)
    + * 
    + * + *

    Run with: {@code mvn test -pl oap-server/server-library/library-batch-queue + * -Dtest=RebalanceBenchmark -DfailIfNoTests=false} + */ +@Slf4j +@SuppressWarnings("all") +public class RebalanceBenchmark { + + // --- Configuration --- + + /** Number of metric types. */ + private static final int TYPE_COUNT = 100; + + /** Throughput tiers (simulate entity count variance after L1 merge). */ + private static final int ENDPOINT_TYPES = 20; // types 0-19, weight 24 + private static final int INSTANCE_TYPES = 30; // types 20-49, weight 8 + private static final int SERVICE_TYPES = 20; // types 50-69, weight 4 + private static final int COLD_TYPES = 30; // types 70-99, weight 1 + + private static final int ENDPOINT_WEIGHT = 24; + private static final int INSTANCE_WEIGHT = 8; + private static final int SERVICE_WEIGHT = 4; + private static final int COLD_WEIGHT = 1; + + /** Consumer busy-spin iterations per item. Tuned so drain threads are the bottleneck. */ + private static final int WORK_ITERATIONS = 500; + + private static final int DRAIN_THREADS = 4; + private static final int PRODUCER_THREADS = 16; + private static final int BUFFER_SIZE = 2000; + private static final int WARMUP_SECONDS = 3; + private static final int MEASURE_SECONDS = 10; + + /** Rebalance interval for rebalanced scenario. */ + private static final long REBALANCE_INTERVAL_MS = 2000; + + /** Weighted type selection array. Producers cycle through this to create skewed throughput. */ + private static final int[] WEIGHTED_TYPES = buildWeightedTypes(); + + /** Volatile sink to prevent JIT from optimizing away consumer work. */ + private volatile long sink; + + @AfterEach + public void cleanup() { + BatchQueueManager.reset(); + } + + /** + * Compare throughput: static round-robin assignment vs rebalanced. + */ + @Test + public void benchmarkStaticVsRebalanced() throws Exception { + // Print workload description + final int totalWeight = ENDPOINT_TYPES * ENDPOINT_WEIGHT + + INSTANCE_TYPES * INSTANCE_WEIGHT + + SERVICE_TYPES * SERVICE_WEIGHT + + COLD_TYPES * COLD_WEIGHT; + log.info("\n========================================"); + log.info(" RebalanceBenchmark: Static vs Rebalanced"); + log.info("========================================"); + log.info("Types: {} ({} endpoint@{}x, {} instance@{}x, {} service@{}x, {} cold@{}x)", + TYPE_COUNT, + ENDPOINT_TYPES, ENDPOINT_WEIGHT, + INSTANCE_TYPES, INSTANCE_WEIGHT, + SERVICE_TYPES, SERVICE_WEIGHT, + COLD_TYPES, COLD_WEIGHT); + log.info("Total weighted units per cycle: {}", totalWeight); + log.info("Drain threads: {}, Producers: {}, Buffer: {}, Work: {} iters/item", + DRAIN_THREADS, PRODUCER_THREADS, BUFFER_SIZE, WORK_ITERATIONS); + log.info(""); + + // Show initial type-to-thread distribution (based on typeHash) + printTypeDistribution(); + + // Run static scenario + log.info("--- Running STATIC scenario ({} sec) ---", MEASURE_SECONDS); + final ScenarioResult staticResult = runScenario("static", false); + cleanup(); + + // Run rebalanced scenario + log.info("--- Running REBALANCED scenario ({} sec) ---", MEASURE_SECONDS); + final ScenarioResult rebalancedResult = runScenario("rebalanced", true); + + // Comparison + log.info("\n--- COMPARISON ---"); + log.info(" Static Rebalanced"); + log.info(" Throughput: {} {} items/sec", + String.format("%12s", String.format("%,.0f", staticResult.throughputPerSec)), + String.format("%12s", String.format("%,.0f", rebalancedResult.throughputPerSec))); + log.info(" Load ratio: {}x {}x (max/min thread)", + String.format("%.2f", staticResult.loadRatio), + String.format("%.2f", rebalancedResult.loadRatio)); + if (staticResult.throughputPerSec > 0) { + final double improvement = (rebalancedResult.throughputPerSec - staticResult.throughputPerSec) + / staticResult.throughputPerSec * 100; + log.info(" Improvement: {}%", String.format("%+.1f", improvement)); + } + log.info(""); + } + + /** + * Verify rebalance stability: once the throughput distribution is stable, + * the per-thread load ratio should converge and stay low across intervals. + */ + @Test + public void benchmarkRebalanceStability() throws Exception { + log.info("\n========================================"); + log.info(" RebalanceBenchmark: Stability Check"); + log.info("========================================"); + + final ConcurrentHashMap perThreadConsumed = new ConcurrentHashMap<>(); + final AtomicLong totalConsumed = new AtomicLong(0); + + final BatchQueue queue = createQueue("stability", true); + registerHandlers(queue, perThreadConsumed, totalConsumed); + + // Warmup + log.info("Warming up ({} sec)...", WARMUP_SECONDS); + final long warmupEnd = System.currentTimeMillis() + WARMUP_SECONDS * 1000L; + runProducers(queue, warmupEnd); + Thread.sleep(500); + resetCounters(perThreadConsumed, totalConsumed); + + // Measure stability over multiple intervals + final int totalSeconds = 20; + final int sampleIntervalSec = 2; + final int samples = totalSeconds / sampleIntervalSec; + + log.info("Measuring stability ({} sec, sample every {} sec)...\n", totalSeconds, sampleIntervalSec); + log.info(" Interval Throughput Ratio Per-thread consumed"); + log.info(" -------- ---------- ----- -------------------"); + + final long measureEnd = System.currentTimeMillis() + totalSeconds * 1000L; + + // Start producers in background + final CountDownLatch producersDone = new CountDownLatch(1); + final Thread producerManager = new Thread(() -> { + try { + runProducers(queue, measureEnd); + } catch (final Exception e) { + log.error("Producer error", e); + } finally { + producersDone.countDown(); + } + }); + producerManager.setDaemon(true); + producerManager.start(); + + // Sample at each interval + final List ratios = new ArrayList<>(); + for (int s = 0; s < samples; s++) { + // Snapshot current values + final Map before = snapshotCounters(perThreadConsumed); + final long consumedBefore = totalConsumed.get(); + + Thread.sleep(sampleIntervalSec * 1000L); + + // Compute deltas + final Map after = snapshotCounters(perThreadConsumed); + final long consumedAfter = totalConsumed.get(); + final long intervalConsumed = consumedAfter - consumedBefore; + final double throughput = intervalConsumed * 1000.0 / (sampleIntervalSec * 1000); + + // Per-thread deltas + final List threadDeltas = new ArrayList<>(); + for (final Map.Entry entry : after.entrySet()) { + final long delta = entry.getValue() - before.getOrDefault(entry.getKey(), 0L); + threadDeltas.add(delta); + } + + long maxDelta = 0; + long minDelta = Long.MAX_VALUE; + for (final long d : threadDeltas) { + if (d > maxDelta) { + maxDelta = d; + } + if (d < minDelta) { + minDelta = d; + } + } + final double ratio = minDelta > 0 ? (double) maxDelta / minDelta : 0; + ratios.add(ratio); + + log.info(" {}-{}s {} {}x {}", + String.format("%3d", s * sampleIntervalSec), + String.format("%3d", (s + 1) * sampleIntervalSec), + String.format("%10s", String.format("%,.0f", throughput)), + String.format("%5.2f", ratio), + formatThreadDeltas(threadDeltas)); + } + + producersDone.await(5, TimeUnit.SECONDS); + + // Summary + log.info(""); + final double firstRatio = ratios.get(0); + final double avgLaterRatio = ratios.subList(Math.min(2, ratios.size()), ratios.size()) + .stream().mapToDouble(d -> d).average().orElse(0); + log.info(" First interval ratio: {}x", String.format("%.2f", firstRatio)); + log.info(" Avg later ratio (4s+): {}x", String.format("%.2f", avgLaterRatio)); + log.info(" Stable: {}", avgLaterRatio < 1.3 ? "YES" : "NO (ratio > 1.3)"); + log.info(""); + } + + // ========== Helpers ========== + + private BatchQueue createQueue( + final String label, final boolean withBalancer) { + final BatchQueueConfig.BatchQueueConfigBuilder builder = + BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(DRAIN_THREADS)) + .partitions(PartitionPolicy.adaptive()) + .bufferSize(BUFFER_SIZE) + .strategy(BufferStrategy.BLOCKING) + .minIdleMs(1) + .maxIdleMs(50); + if (withBalancer) { + builder.balancer(DrainBalancer.throughputWeighted(), REBALANCE_INTERVAL_MS); + } + return BatchQueueManager.create("rebal-" + label, builder.build()); + } + + private void registerHandlers( + final BatchQueue queue, + final ConcurrentHashMap perThreadConsumed, + final AtomicLong totalConsumed) { + for (int t = 0; t < TYPE_COUNT; t++) { + queue.addHandler(BenchmarkMetricTypes.CLASSES[t], + (HandlerConsumer) data -> { + // Track per-thread consumed + final long tid = Thread.currentThread().getId(); + perThreadConsumed.computeIfAbsent(tid, k -> new AtomicLong()) + .addAndGet(data.size()); + totalConsumed.addAndGet(data.size()); + + // Simulate consumer work per item. + // Uses item data to prevent JIT from collapsing the loop. + long s = tid; + for (int i = 0; i < data.size(); i++) { + long v = data.get(i).value; + for (int w = 0; w < WORK_ITERATIONS; w++) { + v = v * 6364136223846793005L + s; + } + s += v; + } + sink = s; + }); + } + } + + private ScenarioResult runScenario(final String label, final boolean rebalance) throws Exception { + final ConcurrentHashMap perThreadConsumed = new ConcurrentHashMap<>(); + final AtomicLong totalConsumed = new AtomicLong(0); + + final BatchQueue queue = createQueue(label, rebalance); + registerHandlers(queue, perThreadConsumed, totalConsumed); + + // Warmup + final long warmupEnd = System.currentTimeMillis() + WARMUP_SECONDS * 1000L; + runProducers(queue, warmupEnd); + Thread.sleep(500); + resetCounters(perThreadConsumed, totalConsumed); + + // Measure + final long measureStart = System.currentTimeMillis(); + final long measureEnd = measureStart + MEASURE_SECONDS * 1000L; + runProducers(queue, measureEnd); + final long elapsed = System.currentTimeMillis() - measureStart; + Thread.sleep(500); + + final long total = totalConsumed.get(); + final double throughput = total * 1000.0 / elapsed; + + // Per-thread stats + long maxThread = 0; + long minThread = Long.MAX_VALUE; + final StringBuilder threadDetail = new StringBuilder(); + for (final Map.Entry entry : perThreadConsumed.entrySet()) { + final long val = entry.getValue().get(); + if (val > maxThread) { + maxThread = val; + } + if (val < minThread) { + minThread = val; + } + if (threadDetail.length() > 0) { + threadDetail.append(", "); + } + threadDetail.append(String.format("%,d", val)); + } + final double ratio = minThread > 0 ? (double) maxThread / minThread : 0; + + log.info(" {} result: {} items/sec, ratio={}x, threads=[{}]", + label, + String.format("%,.0f", throughput), + String.format("%.2f", ratio), + threadDetail); + + return new ScenarioResult(throughput, ratio); + } + + /** + * Produce items with weighted type distribution until endTimeMs. + * Each producer cycles through {@link #WEIGHTED_TYPES}, producing one item per slot. + */ + private void runProducers( + final BatchQueue queue, + final long endTimeMs) throws InterruptedException { + final CountDownLatch done = new CountDownLatch(PRODUCER_THREADS); + + for (int p = 0; p < PRODUCER_THREADS; p++) { + final int producerIndex = p; + final Thread thread = new Thread(() -> { + long count = 0; + // Start at different offset per producer to spread contention + int slotIndex = producerIndex * (WEIGHTED_TYPES.length / PRODUCER_THREADS); + while (System.currentTimeMillis() < endTimeMs) { + for (int batch = 0; batch < 50; batch++) { + final int typeId = WEIGHTED_TYPES[slotIndex % WEIGHTED_TYPES.length]; + queue.produce(BenchmarkMetricTypes.FACTORIES[typeId].create(count)); + count++; + slotIndex++; + } + } + done.countDown(); + }); + thread.setName("RebalProducer-" + producerIndex); + thread.setDaemon(true); + thread.start(); + } + + done.await(MEASURE_SECONDS + WARMUP_SECONDS + 30, TimeUnit.SECONDS); + } + + /** + * Show the initial type-to-thread mapping based on typeHash and round-robin. + * This reveals the static imbalance before rebalancing. + */ + private void printTypeDistribution() { + // Resolve partition count using adaptive policy + final int partitionCount = PartitionPolicy.adaptive().resolve(DRAIN_THREADS, TYPE_COUNT); + final int[][] assignments = buildAssignments(DRAIN_THREADS, partitionCount); + + // Map partition -> thread + final int[] partitionToThread = new int[partitionCount]; + for (int t = 0; t < assignments.length; t++) { + for (final int p : assignments[t]) { + partitionToThread[p] = t; + } + } + + // Compute per-thread weighted load + final long[] threadLoad = new long[DRAIN_THREADS]; + final int[] threadTypeCount = new int[DRAIN_THREADS]; + final int[] threadEndpointTypes = new int[DRAIN_THREADS]; + for (int typeId = 0; typeId < TYPE_COUNT; typeId++) { + final int hash = BenchmarkMetricTypes.CLASSES[typeId].hashCode() & 0x7FFFFFFF; + final int partition = hash % partitionCount; + final int thread = partitionToThread[partition]; + threadLoad[thread] += weightOf(typeId); + threadTypeCount[thread]++; + if (typeId < ENDPOINT_TYPES) { + threadEndpointTypes[thread]++; + } + } + + final long maxLoad = Arrays.stream(threadLoad).max().orElse(0); + final long minLoad = Arrays.stream(threadLoad).min().orElse(0); + final double ratio = minLoad > 0 ? (double) maxLoad / minLoad : 0; + + log.info("Initial type distribution (typeHash + round-robin):"); + for (int t = 0; t < DRAIN_THREADS; t++) { + log.info(" Thread {}: {} types ({} endpoint), weighted load = {}", + t, threadTypeCount[t], threadEndpointTypes[t], threadLoad[t]); + } + log.info(" Static load ratio: {}x (max={}, min={})\n", + String.format("%.2f", ratio), maxLoad, minLoad); + } + + private static int weightOf(final int typeId) { + if (typeId < ENDPOINT_TYPES) { + return ENDPOINT_WEIGHT; + } + if (typeId < ENDPOINT_TYPES + INSTANCE_TYPES) { + return INSTANCE_WEIGHT; + } + if (typeId < ENDPOINT_TYPES + INSTANCE_TYPES + SERVICE_TYPES) { + return SERVICE_WEIGHT; + } + return COLD_WEIGHT; + } + + /** + * Build a weighted type selection array. Producers cycle through this to + * produce items at rates proportional to each type's entity count. + */ + private static int[] buildWeightedTypes() { + final List slots = new ArrayList<>(); + for (int t = 0; t < TYPE_COUNT; t++) { + final int weight = weightOf(t); + for (int w = 0; w < weight; w++) { + slots.add(t); + } + } + // Shuffle to avoid sequential clustering of hot types in the produce cycle + java.util.Collections.shuffle(slots, new java.util.Random(42)); + return slots.stream().mapToInt(i -> i).toArray(); + } + + private static int[][] buildAssignments(final int taskCount, final int partitionCount) { + final int[][] result = new int[taskCount][]; + final List> assignment = new ArrayList<>(); + for (int t = 0; t < taskCount; t++) { + assignment.add(new ArrayList<>()); + } + for (int p = 0; p < partitionCount; p++) { + assignment.get(p % taskCount).add(p); + } + for (int t = 0; t < taskCount; t++) { + final List parts = assignment.get(t); + result[t] = new int[parts.size()]; + for (int i = 0; i < parts.size(); i++) { + result[t][i] = parts.get(i); + } + } + return result; + } + + private static Map snapshotCounters( + final ConcurrentHashMap perThread) { + final Map snapshot = new TreeMap<>(); + for (final Map.Entry entry : perThread.entrySet()) { + snapshot.put(entry.getKey(), entry.getValue().get()); + } + return snapshot; + } + + private static void resetCounters( + final ConcurrentHashMap perThread, final AtomicLong total) { + for (final AtomicLong v : perThread.values()) { + v.set(0); + } + total.set(0); + } + + private static String formatThreadDeltas(final List deltas) { + final StringBuilder sb = new StringBuilder("["); + for (int i = 0; i < deltas.size(); i++) { + if (i > 0) { + sb.append(", "); + } + sb.append(String.format("%,d", deltas.get(i))); + } + sb.append("]"); + return sb.toString(); + } + + private static class ScenarioResult { + final double throughputPerSec; + final double loadRatio; + + ScenarioResult(final double throughputPerSec, final double loadRatio) { + this.throughputPerSec = throughputPerSec; + this.loadRatio = loadRatio; + } + } +} From cba568778b4aee7bd1bccf49ac578a96b4f7bfca Mon Sep 17 00:00:00 2001 From: Wu Sheng Date: Sat, 21 Feb 2026 19:44:15 +0800 Subject: [PATCH 06/16] Fix shared queue startup crash: add getOrCreate(), remove sharedScheduler OAP server crashed at startup with "BatchQueue [TOPN_PERSISTENCE] already exists" because TopNWorker, MetricsAggregateWorker, and MetricsPersistentMinWorker each called BatchQueueManager.create() with the same queue name for every metric/TopN type, but create() throws on duplicate names. Add BatchQueueManager.getOrCreate() for shared queues where multiple workers register handlers on the same queue. The first caller creates the queue; subsequent callers get the existing instance. Remove the sharedScheduler infrastructure (shared ScheduledExecutorService with reference counting) since all queues now use dedicated schedulers. The getOrCreate() pattern with addHandler() provides the same sharing semantics with better isolation (DrainBalancer, adaptive partitions). Co-Authored-By: Claude Opus 4.6 --- .../worker/MetricsAggregateWorker.java | 6 +- .../worker/MetricsPersistentMinWorker.java | 2 +- .../core/analysis/worker/TopNWorker.java | 2 +- .../library-batch-queue/CLAUDE.md | 52 ++++---- .../server/library/batchqueue/BatchQueue.java | 97 ++++---------- .../library/batchqueue/BatchQueueConfig.java | 40 +----- .../library/batchqueue/BatchQueueManager.java | 125 ++++-------------- .../library/batchqueue/ThreadPolicy.java | 3 +- .../batchqueue/BatchQueueConfigTest.java | 29 +--- .../batchqueue/BatchQueueManagerTest.java | 72 ++-------- .../library/batchqueue/BatchQueueTest.java | 45 ------- 11 files changed, 99 insertions(+), 374 deletions(-) diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsAggregateWorker.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsAggregateWorker.java index c86557fb8f3d..0361e0711f31 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsAggregateWorker.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsAggregateWorker.java @@ -47,9 +47,9 @@ * bucket, the L1 aggregation will merge them into one metrics object to reduce the unnecessary memory and network * payload. * - *

    All metric types (OAL and MAL) share a single {@link BatchQueue} with adaptive partitioning. + *

    All metric types share a single {@link BatchQueue} with adaptive partitioning. * The {@code typeHash()} partition selector ensures same metric class lands on the same partition, - * so each handler's {@link MergableBufferedData} is only accessed by one drain thread. + * so each worker's {@link MergableBufferedData} is only accessed by one drain thread. */ @Slf4j public class MetricsAggregateWorker extends AbstractWorker { @@ -86,7 +86,7 @@ public class MetricsAggregateWorker extends AbstractWorker { this.nextWorker = nextWorker; this.mergeDataCache = new MergableBufferedData<>(); this.l1FlushPeriod = l1FlushPeriod; - this.l1Queue = BatchQueueManager.create(L1_QUEUE_NAME, L1_QUEUE_CONFIG); + this.l1Queue = BatchQueueManager.getOrCreate(L1_QUEUE_NAME, L1_QUEUE_CONFIG); final MetricsCreator metricsCreator = moduleDefineHolder.find(TelemetryModule.NAME) .provider() diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsPersistentMinWorker.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsPersistentMinWorker.java index e121a9827d5c..197cdab9faa0 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsPersistentMinWorker.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/MetricsPersistentMinWorker.java @@ -91,7 +91,7 @@ public class MetricsPersistentMinWorker extends MetricsPersistentWorker { storageSessionTimeout, metricsDataTTL, kind ); - this.l2Queue = BatchQueueManager.create(L2_QUEUE_NAME, L2_QUEUE_CONFIG); + this.l2Queue = BatchQueueManager.getOrCreate(L2_QUEUE_NAME, L2_QUEUE_CONFIG); serverStatusService = moduleDefineHolder.find(CoreModule.NAME).provider().getService(ServerStatusService.class); serverStatusService.registerWatcher(this); diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/TopNWorker.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/TopNWorker.java index 06d4a5acc298..702224fe8c76 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/TopNWorker.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/analysis/worker/TopNWorker.java @@ -71,7 +71,7 @@ public class TopNWorker extends PersistenceWorker { ); this.recordDAO = recordDAO; this.model = model; - this.topNQueue = BatchQueueManager.create(TOPN_QUEUE_NAME, TOPN_QUEUE_CONFIG); + this.topNQueue = BatchQueueManager.getOrCreate(TOPN_QUEUE_NAME, TOPN_QUEUE_CONFIG); this.lastReportTimestamp = System.currentTimeMillis(); // Top N persistent works per 10 minutes default. this.reportPeriod = reportPeriod; diff --git a/oap-server/server-library/library-batch-queue/CLAUDE.md b/oap-server/server-library/library-batch-queue/CLAUDE.md index fe798380119f..09e727875285 100644 --- a/oap-server/server-library/library-batch-queue/CLAUDE.md +++ b/oap-server/server-library/library-batch-queue/CLAUDE.md @@ -34,34 +34,19 @@ Use for: homogeneous queues where all items are the same type (JDBC batch, singl Call `queue.addHandler(TypeA.class, handlerA)` per type. Drained items are grouped by `getClass()` and dispatched to matching handlers. Unregistered types are logged and dropped. Use for: shared queues where many metric types coexist (L1 aggregation, L2 persistence, TopN). -## Scheduler Modes +## Queue Sharing -### Dedicated scheduler -The queue owns a `ScheduledThreadPool`. Each thread is assigned a fixed subset of partitions (round-robin). Multiple threads drain concurrently. - -```java -BatchQueueConfig.builder() - .threads(ThreadPolicy.cpuCores(1.0)) // own thread pool - .partitions(PartitionPolicy.adaptive()) - ... -``` - -### Shared scheduler -Multiple queues share one `ScheduledThreadPool` (reference-counted, auto-shutdown). Each queue gets 1 drain task on the shared pool. Useful for low-throughput I/O queues. - -```java -BatchQueueConfig.builder() - .sharedScheduler("exporter", ThreadPolicy.fixed(1)) // shared pool - .partitions(PartitionPolicy.fixed(1)) - ... -``` +Multiple workers of the same concern share a single queue via `BatchQueueManager.getOrCreate(name, config)`. +The first caller creates the queue; subsequent callers with the same name get the existing instance. +Each worker registers its type handler via `addHandler()`. For strict unique-name enforcement, +use `BatchQueueManager.create(name, config)` which throws on duplicate names. ## Key Classes | Class | Role | |-------|------| | `BatchQueue` | The queue itself. Holds partitions, runs drain loops, dispatches to consumers/handlers. | -| `BatchQueueManager` | Global registry. Creates queues by name, manages shared schedulers with ref-counting. | +| `BatchQueueManager` | Global registry. Creates/retrieves queues by name. `create()` for unique, `getOrCreate()` for shared. | | `BatchQueueConfig` | Builder for queue configuration (threads, partitions, buffer, strategy, consumer, balancer). | | `ThreadPolicy` | Resolves thread count: `fixed(N)`, `cpuCores(mult)`, `cpuCoresWithBase(base, mult)`. | | `PartitionPolicy` | Resolves partition count: `fixed(N)`, `threadMultiply(N)`, `adaptive()`. | @@ -145,8 +130,10 @@ Silently ignored for single-thread queues (nothing to rebalance). ### L1 Metrics Aggregation (`MetricsAggregateWorker`) ``` +queue: getOrCreate("METRICS_L1_AGGREGATION", ...) threads: cpuCores(1.0) -- 8 threads on 8-core partitions: adaptive() -- grows with metric types (~330 for typical OAL+MAL on 8 threads) +balancer: throughputWeighted(), 10s bufferSize: 20,000 per partition strategy: IF_POSSIBLE idleMs: 1..50 @@ -155,26 +142,30 @@ mode: handler map (one handler per metric class) ### L2 Metrics Persistence (`MetricsPersistentMinWorker`) ``` +queue: getOrCreate("METRICS_L2_PERSISTENCE", ...) threads: cpuCoresWithBase(1, 0.25) -- 3 threads on 8-core partitions: adaptive() -- grows with metric types +balancer: throughputWeighted(), 10s bufferSize: 2,000 per partition -strategy: IF_POSSIBLE +strategy: BLOCKING idleMs: 1..50 mode: handler map (one handler per metric class) ``` ### TopN Persistence (`TopNWorker`) ``` +queue: getOrCreate("TOPN_PERSISTENCE", ...) threads: fixed(1) partitions: adaptive() -- grows with TopN types bufferSize: 1,000 per partition -strategy: IF_POSSIBLE +strategy: BLOCKING idleMs: 10..100 mode: handler map (one handler per TopN class) ``` ### gRPC Remote Client (`GRPCRemoteClient`) ``` +queue: create(unique name per client, ...) threads: fixed(1) partitions: fixed(1) bufferSize: configurable (channelSize * bufferSize) @@ -185,6 +176,7 @@ mode: single consumer (sends over gRPC stream) ### Exporters (gRPC metrics, Kafka trace, Kafka log) ``` +queue: create(unique name per exporter, ...) threads: fixed(1) each partitions: fixed(1) each bufferSize: configurable (default 20,000) @@ -195,6 +187,7 @@ mode: single consumer ### JDBC Batch DAO (`JDBCBatchDAO`) ``` +queue: create("JDBC_BATCH_PERSISTENCE", ...) threads: fixed(N) where N = asyncBatchPersistentPoolSize (default 4) partitions: fixed(N) (1 partition per thread) bufferSize: 10,000 per partition @@ -205,9 +198,10 @@ mode: single consumer (JDBC batch flush) ## Lifecycle -1. `BatchQueueManager.create(name, config)` -- creates and starts drain loops immediately -2. `queue.addHandler(type, handler)` -- registers type handler (adaptive: may grow partitions) -3. `queue.produce(data)` -- routes to partition, blocks or drops per strategy -4. Drain loops run continuously, dispatching batches to consumers/handlers -5. `BatchQueueManager.shutdown(name)` -- stops drain, final flush, releases scheduler -6. `BatchQueueManager.shutdownAll()` -- called during OAP server shutdown +1. `BatchQueueManager.getOrCreate(name, config)` -- gets existing or creates new queue, starts drain loops +2. `BatchQueueManager.create(name, config)` -- creates queue (throws if name already exists) +3. `queue.addHandler(type, handler)` -- registers type handler (adaptive: may grow partitions) +4. `queue.produce(data)` -- routes to partition, blocks or drops per strategy +5. Drain loops run continuously, dispatching batches to consumers/handlers +6. `BatchQueueManager.shutdown(name)` -- stops drain, final flush +7. `BatchQueueManager.shutdownAll()` -- called during OAP server shutdown diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java index 7cc0e599c5ab..ca0e976057be 100644 --- a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java @@ -94,20 +94,9 @@ * Delay doubles on each consecutive idle cycle: {@code minIdleMs * 2^idleCount}, * capped at {@code maxIdleMs}. Resets to {@code minIdleMs} on first non-empty drain. * - *

    Scheduler modes

    - *
      - *
    • Dedicated — this queue owns its own ScheduledThreadPool. Each thread - * is assigned a fixed subset of partitions (round-robin). Thread count and - * partition count are configured independently.
    • - *
    • Shared — the queue borrows a scheduler from {@link BatchQueueManager}, - * shared with other queues. Only 1 drain task is submitted (drains all partitions). - * The shared scheduler is reference-counted and shut down when the last queue - * releases it.
    • - *
    - * *

    Use case examples

    *
    - * shared scheduler, partitions=1, one consumer    --> I/O queue (gRPC, Kafka, JDBC)
    + * dedicated fixed(1), partitions=1, one consumer  --> I/O queue (gRPC, Kafka, JDBC)
      * dedicated fixed(1), partitions=1, many handlers --> TopN (all types share 1 thread)
      * dedicated cpuCores(1.0), adaptive(),
      *           many handlers                         --> metrics aggregation
    @@ -121,16 +110,9 @@ public class BatchQueue {
         private final String name;
         private final BatchQueueConfig config;
     
    -    /** The thread pool that executes drain tasks. Either dedicated or shared. */
    +    /** The thread pool that executes drain tasks. */
         private final ScheduledExecutorService scheduler;
     
    -    /** True if this queue owns the scheduler and should shut it down. */
    -    @Getter(AccessLevel.PACKAGE)
    -    private final boolean dedicatedScheduler;
    -
    -    /** Non-null only for shared schedulers; used to release the ref count on shutdown. */
    -    private final String sharedSchedulerName;
    -
         /**
          * Cached partition selector from config. Only used when {@code partitions.length > 1};
          * single-partition queues bypass the selector entirely.
    @@ -313,56 +295,33 @@ public class BatchQueue {
             this.handlerMap = new ConcurrentHashMap<>();
             this.warnedUnregisteredTypes = ConcurrentHashMap.newKeySet();
     
    -        if (config.getSharedSchedulerName() != null) {
    -            // ---- Shared scheduler mode ----
    -            final ScheduledExecutorService sharedScheduler =
    -                BatchQueueManager.getOrCreateSharedScheduler(
    -                    config.getSharedSchedulerName(), config.getSharedSchedulerThreads());
    -
    -            this.resolvedThreadCount = 1;
    -            final int partitionCount = config.getPartitions().resolve(1, 0);
    -            this.partitions = new ArrayBlockingQueue[partitionCount];
    -            for (int i = 0; i < partitions.length; i++) {
    -                partitions[i] = new ArrayBlockingQueue<>(config.getBufferSize());
    -            }
    +        int threadCount = config.getThreads().resolve();
    +        this.resolvedThreadCount = threadCount;
     
    -            this.scheduler = sharedScheduler;
    -            this.dedicatedScheduler = false;
    -            this.sharedSchedulerName = config.getSharedSchedulerName();
    -            this.taskCount = 1;
    -            this.assignedPartitions = buildAssignments(1, partitionCount);
    -        } else {
    -            // ---- Dedicated scheduler mode ----
    -            int threadCount = config.getThreads().resolve();
    -            this.resolvedThreadCount = threadCount;
    -
    -            // For adaptive with 0 handlers, resolve returns threadCount (sensible initial).
    -            // For fixed/threadMultiply, resolve returns the configured count.
    -            final int partitionCount = config.getPartitions().resolve(threadCount, 0);
    -
    -            if (partitionCount < threadCount) {
    -                log.warn("BatchQueue[{}]: partitions({}) < threads({}), reducing threads to {}",
    -                    name, partitionCount, threadCount, partitionCount);
    -                threadCount = partitionCount;
    -            }
    +        // For adaptive with 0 handlers, resolve returns threadCount (sensible initial).
    +        // For fixed/threadMultiply, resolve returns the configured count.
    +        final int partitionCount = config.getPartitions().resolve(threadCount, 0);
     
    -            this.partitions = new ArrayBlockingQueue[partitionCount];
    -            for (int i = 0; i < partitions.length; i++) {
    -                partitions[i] = new ArrayBlockingQueue<>(config.getBufferSize());
    -            }
    +        if (partitionCount < threadCount) {
    +            log.warn("BatchQueue[{}]: partitions({}) < threads({}), reducing threads to {}",
    +                name, partitionCount, threadCount, partitionCount);
    +            threadCount = partitionCount;
    +        }
     
    -            this.scheduler = Executors.newScheduledThreadPool(threadCount, r -> {
    -                final Thread t = new Thread(r);
    -                t.setName("BatchQueue-" + name + "-" + t.getId());
    -                t.setDaemon(true);
    -                return t;
    -            });
    -            this.dedicatedScheduler = true;
    -            this.sharedSchedulerName = null;
    -            this.taskCount = threadCount;
    -            this.assignedPartitions = buildAssignments(threadCount, partitionCount);
    +        this.partitions = new ArrayBlockingQueue[partitionCount];
    +        for (int i = 0; i < partitions.length; i++) {
    +            partitions[i] = new ArrayBlockingQueue<>(config.getBufferSize());
             }
     
    +        this.scheduler = Executors.newScheduledThreadPool(threadCount, r -> {
    +            final Thread t = new Thread(r);
    +            t.setName("BatchQueue-" + name + "-" + t.getId());
    +            t.setDaemon(true);
    +            return t;
    +        });
    +        this.taskCount = threadCount;
    +        this.assignedPartitions = buildAssignments(threadCount, partitionCount);
    +
             this.consecutiveIdleCycles = new int[taskCount];
             this.running = true;
             // Kick off the drain loop for each task
    @@ -823,7 +782,7 @@ private static int[][] buildAssignmentsFromOwner(
     
         /**
          * Stop the queue: reject new produces, perform a final drain of all partitions,
    -     * and release the scheduler (dedicated: shutdown; shared: decrement ref count).
    +     * and shut down the scheduler.
          */
         void shutdown() {
             running = false;
    @@ -836,11 +795,7 @@ void shutdown() {
             if (!combined.isEmpty()) {
                 dispatch(combined);
             }
    -        if (dedicatedScheduler) {
    -            scheduler.shutdown();
    -        } else if (sharedSchedulerName != null) {
    -            BatchQueueManager.releaseSharedScheduler(sharedSchedulerName);
    -        }
    +        scheduler.shutdown();
         }
     
         int getPartitionCount() {
    diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueConfig.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueConfig.java
    index dea210ec45e9..b47e154acb80 100644
    --- a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueConfig.java
    +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueConfig.java
    @@ -25,23 +25,10 @@
     @Builder
     public class BatchQueueConfig {
         /**
    -     * Thread policy for a dedicated ScheduledExecutorService.
    -     * When set, the queue creates its own scheduler.
    -     * When null, sharedSchedulerName/sharedSchedulerThreads must be set.
    +     * Thread policy for the ScheduledExecutorService.
          */
         private ThreadPolicy threads;
     
    -    /**
    -     * Shared scheduler name. Set via the builder method
    -     * {@code sharedScheduler(name, threads)}.
    -     */
    -    private String sharedSchedulerName;
    -
    -    /**
    -     * Thread policy for the shared scheduler. Set together with sharedSchedulerName.
    -     */
    -    private ThreadPolicy sharedSchedulerThreads;
    -
         @Builder.Default
         private PartitionPolicy partitions = PartitionPolicy.fixed(1);
     
    @@ -89,16 +76,8 @@ public class BatchQueueConfig {
         private long rebalanceIntervalMs;
     
         void validate() {
    -        final boolean hasDedicated = threads != null;
    -        final boolean hasShared = sharedSchedulerName != null;
    -        if (hasDedicated == hasShared) {
    -            throw new IllegalArgumentException(
    -                "Exactly one of threads or sharedScheduler must be set. " +
    -                    "threads=" + threads + ", sharedSchedulerName=" + sharedSchedulerName);
    -        }
    -        if (hasShared && sharedSchedulerThreads == null) {
    -            throw new IllegalArgumentException(
    -                "sharedSchedulerThreads must be set when sharedSchedulerName is set");
    +        if (threads == null) {
    +            throw new IllegalArgumentException("threads must be set.");
             }
             if (bufferSize < 1) {
                 throw new IllegalArgumentException("bufferSize must be >= 1, got: " + bufferSize);
    @@ -116,19 +95,6 @@ void validate() {
          * Builder customizations: convenience methods for setting paired fields together.
          */
         public static class BatchQueueConfigBuilder {
    -        /**
    -         * Configure the queue to use a shared scheduler instead of a dedicated one.
    -         *
    -         * @param name the shared scheduler name (queues with the same name share a pool)
    -         * @param threads the thread policy for the shared scheduler
    -         * @return this builder
    -         */
    -        public BatchQueueConfigBuilder sharedScheduler(final String name, final ThreadPolicy threads) {
    -            this.sharedSchedulerName = name;
    -            this.sharedSchedulerThreads = threads;
    -            return this;
    -        }
    -
             /**
              * Enable periodic drain rebalancing with the given strategy and interval.
              *
    diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManager.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManager.java
    index 063bc9673a42..1ba24aac8b24 100644
    --- a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManager.java
    +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManager.java
    @@ -21,28 +21,15 @@
     import java.util.ArrayList;
     import java.util.List;
     import java.util.concurrent.ConcurrentHashMap;
    -import java.util.concurrent.Executors;
    -import java.util.concurrent.ScheduledExecutorService;
    -import java.util.concurrent.atomic.AtomicInteger;
     import lombok.extern.slf4j.Slf4j;
     
     /**
    - * Global registry for batch queues and shared schedulers.
    + * Global registry for batch queues.
      * Thread-safe. Queues are created by name and shared across modules.
      *
    - * 

    Shared schedulers are created lazily on first queue reference — no separate - * setup step needed. This eliminates startup ordering dependencies. - * - *

    Shared schedulers are reference-counted: each queue that uses a shared scheduler - * increments the count on creation and decrements on shutdown. When the count reaches - * zero, the scheduler is shut down automatically. - * *

    Internal maps: *

    - * QUEUES:                     queueName     -> BatchQueue instance
    - * SHARED_SCHEDULERS:          schedulerName -> ScheduledExecutorService
    - * SHARED_SCHEDULER_POLICIES:  schedulerName -> ThreadPolicy (first-wins)
    - * SHARED_SCHEDULER_REF_COUNTS: schedulerName -> AtomicInteger (reference count)
    + * QUEUES: queueName -> BatchQueue instance
      * 
    */ @Slf4j @@ -51,81 +38,11 @@ public class BatchQueueManager { * queueName -> BatchQueue instance. Each queue has a unique name. */ private static final ConcurrentHashMap> QUEUES = new ConcurrentHashMap<>(); - /** - * schedulerName -> ScheduledExecutorService. Multiple queues can share one scheduler by - * referencing the same scheduler name in their config. - */ - private static final ConcurrentHashMap SHARED_SCHEDULERS = - new ConcurrentHashMap<>(); - /** - * schedulerName -> ThreadPolicy. Tracks the first-wins policy for each shared scheduler - * to detect mismatched configs. - */ - private static final ConcurrentHashMap SHARED_SCHEDULER_POLICIES = - new ConcurrentHashMap<>(); - /** - * schedulerName -> reference count. Incremented when a queue acquires the scheduler, - * decremented when a queue releases it. Scheduler is shut down when count reaches 0. - */ - private static final ConcurrentHashMap SHARED_SCHEDULER_REF_COUNTS = - new ConcurrentHashMap<>(); /** - * Get or create a shared scheduler and increment its reference count. - * Called internally by BatchQueue constructor. - * First call creates the scheduler; subsequent calls reuse it. - * If ThreadPolicy differs from the first creator, logs a warning (first one wins). + * Create a new queue with the given name and config. Throws if a queue with the same name + * already exists. Use {@link #getOrCreate} when multiple callers share a single queue. */ - static ScheduledExecutorService getOrCreateSharedScheduler(final String name, - final ThreadPolicy threads) { - SHARED_SCHEDULER_POLICIES.compute(name, (k, existing) -> { - if (existing != null) { - if (!existing.equals(threads)) { - log.warn("Shared scheduler [{}]: ThreadPolicy mismatch. " - + "Existing={}, requested={}. Using existing.", - name, existing, threads); - } - return existing; - } - return threads; - }); - - SHARED_SCHEDULER_REF_COUNTS.computeIfAbsent(name, k -> new AtomicInteger(0)).incrementAndGet(); - - return SHARED_SCHEDULERS.computeIfAbsent(name, k -> { - final int threadCount = threads.resolve(); - log.info("Creating shared scheduler [{}] with {} threads ({})", - name, threadCount, threads); - return Executors.newScheduledThreadPool(threadCount, r -> { - final Thread t = new Thread(r); - t.setName("SharedScheduler-" + name + "-" + t.getId()); - t.setDaemon(true); - return t; - }); - }); - } - - /** - * Decrement the reference count for a shared scheduler. - * When the count reaches zero, the scheduler is shut down and removed. - */ - static void releaseSharedScheduler(final String name) { - final AtomicInteger refCount = SHARED_SCHEDULER_REF_COUNTS.get(name); - if (refCount == null) { - return; - } - if (refCount.decrementAndGet() <= 0) { - SHARED_SCHEDULER_REF_COUNTS.remove(name); - SHARED_SCHEDULER_POLICIES.remove(name); - final ScheduledExecutorService scheduler = SHARED_SCHEDULERS.remove(name); - if (scheduler != null) { - log.info("Shutting down shared scheduler [{}] (ref count reached 0)", name); - scheduler.shutdown(); - } - } - } - - @SuppressWarnings("unchecked") public static BatchQueue create(final String name, final BatchQueueConfig config) { config.validate(); final BatchQueue queue = new BatchQueue<>(name, config); @@ -138,6 +55,27 @@ public static BatchQueue create(final String name, final BatchQueueConfig return queue; } + /** + * Get an existing queue or create a new one. The first caller creates the queue; + * subsequent callers with the same name get the existing instance (config is ignored + * for them). Thread-safe via CAS on the internal map. + */ + @SuppressWarnings("unchecked") + public static BatchQueue getOrCreate(final String name, final BatchQueueConfig config) { + final BatchQueue existing = QUEUES.get(name); + if (existing != null) { + return (BatchQueue) existing; + } + config.validate(); + final BatchQueue queue = new BatchQueue<>(name, config); + final BatchQueue prev = QUEUES.putIfAbsent(name, queue); + if (prev != null) { + queue.shutdown(); + return (BatchQueue) prev; + } + return queue; + } + @SuppressWarnings("unchecked") public static BatchQueue get(final String name) { return (BatchQueue) QUEUES.get(name); @@ -151,7 +89,7 @@ public static void shutdown(final String name) { } /** - * Shutdown all queues and all shared schedulers. Called during OAP server shutdown. + * Shutdown all queues. Called during OAP server shutdown. */ public static void shutdownAll() { final List> allQueues = new ArrayList<>(QUEUES.values()); @@ -164,17 +102,6 @@ public static void shutdownAll() { log.error("Error shutting down queue: {}", queue.getName(), t); } } - - for (final ScheduledExecutorService scheduler : SHARED_SCHEDULERS.values()) { - try { - scheduler.shutdown(); - } catch (final Throwable t) { - log.error("Error shutting down shared scheduler", t); - } - } - SHARED_SCHEDULERS.clear(); - SHARED_SCHEDULER_POLICIES.clear(); - SHARED_SCHEDULER_REF_COUNTS.clear(); } /** diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicy.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicy.java index 33fde8dfef21..b1d50f4dda9f 100644 --- a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicy.java +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/ThreadPolicy.java @@ -19,8 +19,7 @@ package org.apache.skywalking.oap.server.library.batchqueue; /** - * Determines the number of threads for a BatchQueue's dedicated scheduler - * or for a shared scheduler. + * Determines the number of threads for a BatchQueue's scheduler. * * Three modes: * - fixed(N): exactly N threads, regardless of hardware. diff --git a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueConfigTest.java b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueConfigTest.java index f74add79f9d5..165058f5e5a3 100644 --- a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueConfigTest.java +++ b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueConfigTest.java @@ -36,35 +36,8 @@ public void testDedicatedThreadConfig() { } @Test - public void testSharedSchedulerConfig() { + public void testRejectsNullThreads() { final BatchQueueConfig config = BatchQueueConfig.builder() - .sharedScheduler("IO_POOL", ThreadPolicy.fixed(4)) - .build(); - config.validate(); - assertEquals("IO_POOL", config.getSharedSchedulerName()); - assertNotNull(config.getSharedSchedulerThreads()); - } - - @Test - public void testRejectsBothThreadsAndShared() { - final BatchQueueConfig config = BatchQueueConfig.builder() - .threads(ThreadPolicy.fixed(2)) - .sharedScheduler("IO_POOL", ThreadPolicy.fixed(4)) - .build(); - assertThrows(IllegalArgumentException.class, config::validate); - } - - @Test - public void testRejectsNeitherThreadsNorShared() { - final BatchQueueConfig config = BatchQueueConfig.builder() - .build(); - assertThrows(IllegalArgumentException.class, config::validate); - } - - @Test - public void testRejectsSharedWithoutThreadPolicy() { - final BatchQueueConfig config = BatchQueueConfig.builder() - .sharedSchedulerName("IO_POOL") .build(); assertThrows(IllegalArgumentException.class, config::validate); } diff --git a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManagerTest.java b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManagerTest.java index 738bb83ca6cb..6d836c349914 100644 --- a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManagerTest.java +++ b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueManagerTest.java @@ -21,15 +21,11 @@ import org.junit.jupiter.api.AfterEach; import org.junit.jupiter.api.Test; -import java.util.concurrent.ScheduledExecutorService; - import static org.junit.jupiter.api.Assertions.assertEquals; -import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertSame; import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; public class BatchQueueManagerTest { @@ -67,6 +63,19 @@ public void testCreateThrowsOnDuplicateName() { () -> BatchQueueManager.create("absent-test", config)); } + @Test + public void testGetOrCreateReturnsSameInstance() { + final BatchQueueConfig config = BatchQueueConfig.builder() + .threads(ThreadPolicy.fixed(1)) + .consumer(data -> { }) + .bufferSize(100) + .build(); + + final BatchQueue first = BatchQueueManager.getOrCreate("shared-test", config); + final BatchQueue second = BatchQueueManager.getOrCreate("shared-test", config); + assertSame(first, second); + } + @Test public void testGetNonExistentReturnsNull() { assertNull(BatchQueueManager.get("nonexistent")); @@ -97,7 +106,7 @@ public void testShutdownAllClearsEverything() { BatchQueueManager.create("q2", BatchQueueConfig.builder() - .sharedScheduler("SHARED", ThreadPolicy.fixed(2)) + .threads(ThreadPolicy.fixed(1)) .consumer(data -> { }) .bufferSize(100) .build()); @@ -107,57 +116,4 @@ public void testShutdownAllClearsEverything() { assertNull(BatchQueueManager.get("q1")); assertNull(BatchQueueManager.get("q2")); } - - @Test - public void testSharedSchedulerCreatedLazily() { - // First queue referencing shared scheduler creates it - final BatchQueue q1 = BatchQueueManager.create("lazy1", - BatchQueueConfig.builder() - .sharedScheduler("LAZY_POOL", ThreadPolicy.fixed(2)) - .consumer(data -> { }) - .bufferSize(100) - .build()); - - // Second queue uses the same pool - final BatchQueue q2 = BatchQueueManager.create("lazy2", - BatchQueueConfig.builder() - .sharedScheduler("LAZY_POOL", ThreadPolicy.fixed(2)) - .consumer(data -> { }) - .bufferSize(100) - .build()); - - assertNotNull(q1); - assertNotNull(q2); - // Both should be running on shared scheduler (not dedicated) - assertNotNull(BatchQueueManager.get("lazy1")); - assertNotNull(BatchQueueManager.get("lazy2")); - } - - @Test - public void testSharedSchedulerRefCounting() { - final BatchQueueConfig config = BatchQueueConfig.builder() - .sharedScheduler("REF_POOL", ThreadPolicy.fixed(2)) - .consumer(data -> { }) - .bufferSize(100) - .build(); - - BatchQueueManager.create("ref1", config); - BatchQueueManager.create("ref2", config); - - // Capture the scheduler before any shutdown - final ScheduledExecutorService scheduler = - BatchQueueManager.getOrCreateSharedScheduler("REF_POOL", ThreadPolicy.fixed(2)); - // Release the extra ref from getOrCreateSharedScheduler call above - BatchQueueManager.releaseSharedScheduler("REF_POOL"); - - assertFalse(scheduler.isShutdown()); - - // Shutting down first queue should NOT shut down the shared scheduler - BatchQueueManager.shutdown("ref1"); - assertFalse(scheduler.isShutdown()); - - // Shutting down last queue SHOULD shut down the shared scheduler - BatchQueueManager.shutdown("ref2"); - assertTrue(scheduler.isShutdown()); - } } diff --git a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueTest.java b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueTest.java index c6a2aa18c927..182d8b95dfcc 100644 --- a/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueTest.java +++ b/oap-server/server-library/library-batch-queue/src/test/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueueTest.java @@ -236,51 +236,6 @@ public void testAdaptiveBelowThresholdIs1to1() { assertEquals(100, queue.getPartitionCount()); } - // --- Shared scheduler --- - - @Test - public void testSharedSchedulerQueuesSharePool() { - final List received1 = new CopyOnWriteArrayList<>(); - final List received2 = new CopyOnWriteArrayList<>(); - - final BatchQueue q1 = BatchQueueManager.create("shared-q1", - BatchQueueConfig.builder() - .sharedScheduler("TEST_POOL", ThreadPolicy.fixed(2)) - .consumer(data -> received1.addAll(data)) - .bufferSize(1000) - .build()); - - final BatchQueue q2 = BatchQueueManager.create("shared-q2", - BatchQueueConfig.builder() - .sharedScheduler("TEST_POOL", ThreadPolicy.fixed(2)) - .consumer(data -> received2.addAll(data)) - .bufferSize(1000) - .build()); - - assertFalse(q1.isDedicatedScheduler()); - assertFalse(q2.isDedicatedScheduler()); - - for (int i = 0; i < 50; i++) { - q1.produce("a-" + i); - q2.produce("b-" + i); - } - - Awaitility.await().atMost(5, TimeUnit.SECONDS) - .until(() -> received1.size() == 50 && received2.size() == 50); - } - - @Test - public void testDedicatedSchedulerIsOwned() { - final BatchQueue queue = BatchQueueManager.create("dedicated-test", - BatchQueueConfig.builder() - .threads(ThreadPolicy.fixed(2)) - .consumer(data -> { }) - .bufferSize(100) - .build()); - - assertTrue(queue.isDedicatedScheduler()); - } - // --- Produce and buffer strategy --- @Test From 5b4a4c9d934d7042aff604842343100352e4963a Mon Sep 17 00:00:00 2001 From: Wu Sheng Date: Sat, 21 Feb 2026 21:53:49 +0800 Subject: [PATCH 07/16] Add cluster e2e test with ZK + BanyanDB storage --- .github/workflows/skywalking.yaml | 2 + .../cluster/zk/banyandb/docker-compose.yml | 149 ++++++++++++++++++ .../e2e-v2/cases/cluster/zk/banyandb/e2e.yaml | 40 +++++ 3 files changed, 191 insertions(+) create mode 100644 test/e2e-v2/cases/cluster/zk/banyandb/docker-compose.yml create mode 100644 test/e2e-v2/cases/cluster/zk/banyandb/e2e.yaml diff --git a/.github/workflows/skywalking.yaml b/.github/workflows/skywalking.yaml index d763ef2584d7..1b49f2fae8b7 100644 --- a/.github/workflows/skywalking.yaml +++ b/.github/workflows/skywalking.yaml @@ -346,6 +346,8 @@ jobs: test: - name: Cluster ZK/ES config: test/e2e-v2/cases/cluster/zk/es/e2e.yaml + - name: Cluster ZK/BanyanDB + config: test/e2e-v2/cases/cluster/zk/banyandb/e2e.yaml - name: Agent NodeJS Backend config: test/e2e-v2/cases/nodejs/e2e.yaml diff --git a/test/e2e-v2/cases/cluster/zk/banyandb/docker-compose.yml b/test/e2e-v2/cases/cluster/zk/banyandb/docker-compose.yml new file mode 100644 index 000000000000..b17906e41a2c --- /dev/null +++ b/test/e2e-v2/cases/cluster/zk/banyandb/docker-compose.yml @@ -0,0 +1,149 @@ +# 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. + +version: '2.1' + +services: + zk: + image: zookeeper:3.5 + expose: + - 2181 + networks: + - e2e + healthcheck: + test: [ "CMD", "bash", "-c", "cat < /dev/null > /dev/tcp/127.0.0.1/2181" ] + interval: 5s + timeout: 60s + retries: 120 + + banyandb: + extends: + file: ../../../../script/docker-compose/base-compose.yml + service: banyandb + + oap1: + extends: + file: ../../../../script/docker-compose/base-compose.yml + service: oap + volumes: + - ../../test-downsampling.yaml:/skywalking/config/meter-analyzer-config/test-downsampling.yaml + environment: + SW_CLUSTER: zookeeper + SW_STORAGE: banyandb + SW_CORE_PERSISTENT_PERIOD: 5 + SW_METER_ANALYZER_ACTIVE_FILES: test-downsampling + SW_CLUSTER_ZK_HOST_PORT: zk:2181 + SW_CLUSTER_INTERNAL_COM_HOST: oap1 + SW_CLUSTER_INTERNAL_COM_PORT: 11800 + SW_CORE_DATA_KEEPER_EXECUTE_PERIOD: 100 + SW_TELEMETRY: prometheus + TESTING_TTL: "true" + ports: + - 1234 + depends_on: + zk: + condition: service_healthy + banyandb: + condition: service_healthy + + oap2: + extends: + file: ../../../../script/docker-compose/base-compose.yml + service: oap + volumes: + - ../../test-downsampling.yaml:/skywalking/config/meter-analyzer-config/test-downsampling.yaml + environment: + SW_CLUSTER: zookeeper + SW_STORAGE: banyandb + SW_CORE_PERSISTENT_PERIOD: 5 + SW_METER_ANALYZER_ACTIVE_FILES: test-downsampling + SW_CLUSTER_ZK_HOST_PORT: zk:2181 + SW_CLUSTER_INTERNAL_COM_HOST: oap2 + SW_CLUSTER_INTERNAL_COM_PORT: 11800 + SW_CORE_DATA_KEEPER_EXECUTE_PERIOD: 100 + SW_TELEMETRY: prometheus + TESTING_TTL: "true" + ports: + - 1234 + depends_on: + zk: + condition: service_healthy + banyandb: + condition: service_healthy + oap1: + condition: service_healthy + + ui: + extends: + file: ../../../../script/docker-compose/base-compose.yml + service: ui + environment: + - SW_OAP_ADDRESS=http://oap1:12800,http://oap2:12800 + - SW_ZIPKIN_ADDRESS=http://oap1:9412,http://oap2:9412 + depends_on: + oap1: + condition: service_healthy + oap2: + condition: service_healthy + ports: + - 8080 + + sender1: + image: "eclipse-temurin:8-jre" + volumes: + - ../../../../java-test-service/e2e-mock-sender/target/e2e-mock-sender-2.0.0.jar:/e2e-mock-sender-2.0.0.jar + command: [ "java", "-jar", "/e2e-mock-sender-2.0.0.jar" ] + environment: + OAP_HOST: oap1 + OAP_GRPC_PORT: 11800 + networks: + - e2e + ports: + - 9093 + healthcheck: + test: [ "CMD", "sh", "-c", "nc -nz 127.0.0.1 9093" ] + interval: 5s + timeout: 60s + retries: 120 + depends_on: + oap1: + condition: service_healthy + oap2: + condition: service_healthy + + sender2: + image: "eclipse-temurin:8-jre" + volumes: + - ../../../../java-test-service/e2e-mock-sender/target/e2e-mock-sender-2.0.0.jar:/e2e-mock-sender-2.0.0.jar + command: [ "java", "-jar", "/e2e-mock-sender-2.0.0.jar" ] + environment: + OAP_HOST: oap2 + OAP_GRPC_PORT: 11800 + networks: + - e2e + ports: + - 9093 + healthcheck: + test: [ "CMD", "sh", "-c", "nc -nz 127.0.0.1 9093" ] + interval: 5s + timeout: 60s + retries: 120 + depends_on: + oap1: + condition: service_healthy + oap2: + condition: service_healthy +networks: + e2e: diff --git a/test/e2e-v2/cases/cluster/zk/banyandb/e2e.yaml b/test/e2e-v2/cases/cluster/zk/banyandb/e2e.yaml new file mode 100644 index 000000000000..4ac22598c772 --- /dev/null +++ b/test/e2e-v2/cases/cluster/zk/banyandb/e2e.yaml @@ -0,0 +1,40 @@ +# 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. + +# This file is used to show how to write configuration files and can be used to test. + +setup: + env: compose + file: docker-compose.yml + timeout: 20m + init-system-environment: ../../../../script/env + steps: + - name: set PATH + command: export PATH=/tmp/skywalking-infra-e2e/bin:$PATH + - name: install yq + command: bash test/e2e-v2/script/prepare/setup-e2e-shell/install.sh yq + - name: install swctl + command: bash test/e2e-v2/script/prepare/setup-e2e-shell/install.sh swctl + +verify: + # verify with retry strategy + retry: + # max retry count + count: 20 + # the interval between two retries, in millisecond. + interval: 3s + cases: + - includes: + - ../../cluster-cases.yaml From 54968bafeb6fb0f8343760f48eccd5138bdf6c44 Mon Sep 17 00:00:00 2001 From: Wu Sheng Date: Sat, 21 Feb 2026 22:32:27 +0800 Subject: [PATCH 08/16] Fix onIdle() dispatch to respect partition ownership in BatchQueue onIdle() was previously broadcast to all handlers from every drain thread, causing concurrent onIdle() calls on the same handler. Now only handlers whose partition is owned by the calling drain task receive onIdle(), consistent with the consume() ownership model. Co-Authored-By: Claude Opus 4.6 --- .../core/remote/client/GRPCRemoteClient.java | 1 + .../server/library/batchqueue/BatchQueue.java | 88 ++++++++++++++++--- 2 files changed, 75 insertions(+), 14 deletions(-) diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/remote/client/GRPCRemoteClient.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/remote/client/GRPCRemoteClient.java index 32ca21110760..3917d2403673 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/remote/client/GRPCRemoteClient.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/remote/client/GRPCRemoteClient.java @@ -111,6 +111,7 @@ public void connect() { .minIdleMs(10) .maxIdleMs(100) .build(); + this.queue = BatchQueueManager.create(queueName, config); this.isConnect = true; } diff --git a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java index ca0e976057be..f0b3ff560d08 100644 --- a/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java +++ b/oap-server/server-library/library-batch-queue/src/main/java/org/apache/skywalking/oap/server/library/batchqueue/BatchQueue.java @@ -19,6 +19,7 @@ package org.apache.skywalking.oap.server.library.batchqueue; import java.util.ArrayList; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -549,13 +550,13 @@ void drainLoop(final int taskIndex) { if (combined.isEmpty()) { // Nothing to drain — increase backoff and notify idle consecutiveIdleCycles[taskIndex]++; - notifyIdle(); + notifyIdle(taskIndex, myPartitions); break; } // Data found — reset backoff and dispatch consecutiveIdleCycles[taskIndex] = 0; - dispatch(combined); + dispatch(combined, taskIndex, myPartitions); } } catch (final Throwable t) { log.error("BatchQueue[{}]: drain loop error", name, t); @@ -580,10 +581,17 @@ void drainLoop(final int taskIndex) { * to one consumer, regardless of item types. *
  • Handler map: items are grouped by {@code item.getClass()}, then * each group is dispatched to its registered handler. Unregistered types - * are logged as errors and dropped.
  • + * are logged as errors and dropped. After dispatch, owned handlers whose + * type was not in the batch receive an {@code onIdle()} call so + * they can flush any accumulated data. * + * + * @param batch the drained items + * @param taskIndex the drain task index + * @param myPartitions the partitions assigned to this task */ - private void dispatch(final List batch) { + private void dispatch(final List batch, final int taskIndex, + final int[] myPartitions) { if (config.getConsumer() != null) { try { config.getConsumer().consume(batch); @@ -614,13 +622,24 @@ private void dispatch(final List batch) { } } } + + // Notify idle for owned handlers whose type had no data in this cycle. + // myPartitions is null during shutdown — skip idle notification. + if (myPartitions != null) { + notifyIdleAbsentTypes(taskIndex, myPartitions, grouped.keySet()); + } } /** - * Notify consumer/handlers that a drain cycle found no data. - * Useful for flush-on-idle semantics (e.g. flush partial batches to storage). + * Notify consumer/handlers that a drain cycle found no data at all. + * For single-consumer mode, calls {@code onIdle()} directly. + * For handler-map mode, delegates to {@link #notifyIdleAbsentTypes} with + * an empty dispatched set (all handlers are idle). + * + * @param taskIndex the drain task that is idle + * @param myPartitions the partitions assigned to this task */ - private void notifyIdle() { + private void notifyIdle(final int taskIndex, final int[] myPartitions) { if (config.getConsumer() != null) { try { config.getConsumer().onIdle(); @@ -628,14 +647,54 @@ private void notifyIdle() { log.error("BatchQueue[{}]: onIdle error in consumer", name, t); } } else { - for (final HandlerConsumer handler : handlerMap.values()) { - try { - handler.onIdle(); - } catch (final Throwable t) { - log.error("BatchQueue[{}]: onIdle error in handler", name, t); - } + notifyIdleAbsentTypes(taskIndex, myPartitions, Collections.emptySet()); + } + } + + /** + * Notify owned handlers whose type was not in the dispatched set. + * Each handler's type is hashed to a partition; only handlers whose partition + * is owned by the calling task are notified, preventing concurrent + * {@code onIdle()} calls on the same handler from different drain threads. + * + * @param taskIndex the drain task index + * @param myPartitions the partitions assigned to this task + * @param dispatchedTypes types that had data in this cycle (skip these) + */ + private void notifyIdleAbsentTypes(final int taskIndex, final int[] myPartitions, + final Set> dispatchedTypes) { + final int partitionCount = partitions.length; + final boolean checkOwnership = rebalancingEnabled; + for (final Map.Entry, HandlerConsumer> entry : handlerMap.entrySet()) { + if (dispatchedTypes.contains(entry.getKey())) { + continue; + } + final int partition = (entry.getKey().hashCode() & 0x7FFFFFFF) % partitionCount; + if (!isOwnedByTask(partition, taskIndex, myPartitions, checkOwnership)) { + continue; + } + try { + entry.getValue().onIdle(); + } catch (final Throwable t) { + log.error("BatchQueue[{}]: onIdle error in handler", name, t); + } + } + } + + /** + * Check whether a partition is owned by the given drain task. + */ + private boolean isOwnedByTask(final int partition, final int taskIndex, + final int[] myPartitions, final boolean checkOwnership) { + if (checkOwnership) { + return partitionOwner.get(partition) == taskIndex; + } + for (final int p : myPartitions) { + if (p == partition) { + return true; } } + return false; } private void handleError(final List data, final Throwable t) { @@ -793,7 +852,8 @@ void shutdown() { partition.drainTo(combined); } if (!combined.isEmpty()) { - dispatch(combined); + // Shutdown dispatch — no idle notification needed + dispatch(combined, -1, null); } scheduler.shutdown(); } From 0518b557dbd9fc58b315a33879d5a199154862c6 Mon Sep 17 00:00:00 2001 From: Wu Sheng Date: Sun, 22 Feb 2026 11:31:45 +0800 Subject: [PATCH 09/16] Add benchmark framework: Istio ALS + BanyanDB cluster with thread analysis Introduce a benchmarks/ directory with reusable environment setup and benchmark case infrastructure. The framework separates environment provisioning from benchmark execution, allowing the same case to run against different topologies. Environment (cluster_oap-banyandb): - 2-node OAP cluster with BanyanDB on Kind (K8s 1.34) - Istio ALS (Access Log Service) for telemetry collection - Bookinfo sample app with Envoy sidecars at ~5 RPS - Deployed via SkyWalking Helm chart with automatic image pre-loading - Cluster health check via remote_out_count Prometheus metric - Centralized version config in benchmarks/env Case (thread-analysis): - Collects periodic OAP thread dumps via kill -3 (SIGQUIT) - Monitors metrics via swctl in the background - Produces per-pod thread pool trend tables (OAP threads only) - Highlights differences between pods in a comparison section - Normalizes pool names (IPs, hashcodes, HttpClient instances) Runner (run.sh): - Two modes: setup-only and full run (setup + case) - Automatic cleanup after run mode (Kind cluster + Docker prune) - Error-only cleanup in setup mode Co-Authored-By: Claude Opus 4.6 --- .gitignore | 4 + benchmarks/cases/thread-analysis/run.sh | 502 ++++++++++++++++++ benchmarks/env | 52 ++ .../envs-setup/cluster_oap-banyandb/kind.yaml | 20 + .../envs-setup/cluster_oap-banyandb/setup.sh | 444 ++++++++++++++++ .../cluster_oap-banyandb/traffic-gen.yaml | 53 ++ .../cluster_oap-banyandb/values.yaml | 23 + benchmarks/run.sh | 168 ++++++ docs/en/guides/benchmark.md | 151 +++++- 9 files changed, 1407 insertions(+), 10 deletions(-) create mode 100755 benchmarks/cases/thread-analysis/run.sh create mode 100644 benchmarks/env create mode 100644 benchmarks/envs-setup/cluster_oap-banyandb/kind.yaml create mode 100755 benchmarks/envs-setup/cluster_oap-banyandb/setup.sh create mode 100644 benchmarks/envs-setup/cluster_oap-banyandb/traffic-gen.yaml create mode 100644 benchmarks/envs-setup/cluster_oap-banyandb/values.yaml create mode 100755 benchmarks/run.sh diff --git a/.gitignore b/.gitignore index 35217509472d..38241c2e1b25 100644 --- a/.gitignore +++ b/.gitignore @@ -29,3 +29,7 @@ oap-server/server-query-plugin/logql-plugin/gen/ # This serves as a template but will ONLY be updated when building a source release tar, # so we don't track future updates of this file. oap-server/server-starter/src/main/resources/version.properties + +# Benchmark reports and downloaded tools +benchmarks/reports/ +benchmarks/.istio/ diff --git a/benchmarks/cases/thread-analysis/run.sh b/benchmarks/cases/thread-analysis/run.sh new file mode 100755 index 000000000000..ed85e35c8550 --- /dev/null +++ b/benchmarks/cases/thread-analysis/run.sh @@ -0,0 +1,502 @@ +#!/usr/bin/env bash +# 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. + +# Benchmark case: Thread dump analysis. +# +# Collects periodic thread dumps from OAP pods, monitors metrics in the +# background, and produces a thread pool analysis report. +# +# Usage: +# ./run.sh +# +# The env-context file is produced by an env-setup script (e.g., +# envs-setup/cluster_oap-banyandb/setup.sh). + +set -euo pipefail + +SCRIPT_DIR="$(cd "$(dirname "$0")" && pwd)" + +if [ $# -lt 1 ] || [ ! -f "$1" ]; then + echo "Usage: $0 " + echo " env-context-file is produced by an envs-setup/*/setup.sh script." + exit 1 +fi + +# Source environment context +source "$1" + +# Configurable via env vars +DUMP_COUNT="${DUMP_COUNT:-5}" +DUMP_INTERVAL="${DUMP_INTERVAL:-60}" + +log() { echo "[$(date +%H:%M:%S)] $*"; } + +cleanup_pids() { + for pid in "${BG_PIDS[@]:-}"; do + kill "$pid" 2>/dev/null || true + done +} +trap cleanup_pids EXIT +BG_PIDS=() + +# Verify we can reach OAP +if ! command -v swctl &>/dev/null; then + echo "ERROR: swctl not found. Please install it first." + exit 1 +fi + +log "=== Thread Analysis Benchmark ===" +log "Environment: $ENV_NAME" +log "OAP: ${OAP_HOST}:${OAP_PORT}" +log "Namespace: $NAMESPACE" +log "Dump config: $DUMP_COUNT rounds, ${DUMP_INTERVAL}s apart" +log "Report dir: $REPORT_DIR" + +############################################################################# +# Metrics monitor (background) +############################################################################# +log "--- Starting metrics monitor (every 10s) ---" + +OAP_BASE_URL="http://${OAP_HOST}:${OAP_PORT}/graphql" + +metrics_monitor() { + local round=0 + while true; do + round=$((round + 1)) + local out="$REPORT_DIR/metrics-round-${round}.yaml" + { + echo "--- round: $round time: $(date -u +%Y-%m-%dT%H:%M:%SZ) ---" + echo "" + echo "# services" + swctl --display yaml --base-url="$OAP_BASE_URL" service ls 2>/dev/null || echo "ERROR" + echo "" + # Query the first discovered service for metrics + FIRST_SVC=$(swctl --display yaml --base-url="$OAP_BASE_URL" service ls 2>/dev/null \ + | grep ' name:' | head -1 | sed 's/.*name: //' || echo "") + if [ -n "$FIRST_SVC" ]; then + echo "# instances ($FIRST_SVC)" + swctl --display yaml --base-url="$OAP_BASE_URL" instance list --service-name="$FIRST_SVC" 2>/dev/null || echo "ERROR" + echo "" + echo "# topology ($FIRST_SVC)" + swctl --display yaml --base-url="$OAP_BASE_URL" dependency service --service-name="$FIRST_SVC" 2>/dev/null || echo "ERROR" + echo "" + echo "# service_cpm ($FIRST_SVC)" + swctl --display yaml --base-url="$OAP_BASE_URL" metrics exec --expression=service_cpm --service-name="$FIRST_SVC" 2>/dev/null || echo "ERROR" + echo "" + echo "# service_resp_time ($FIRST_SVC)" + swctl --display yaml --base-url="$OAP_BASE_URL" metrics exec --expression=service_resp_time --service-name="$FIRST_SVC" 2>/dev/null || echo "ERROR" + fi + } > "$out" 2>&1 + + # Print summary to stdout + local svc_count + svc_count=$(grep -c "^- id:" "$out" 2>/dev/null || echo 0) + local has_cpm + has_cpm=$(grep -c 'value: "[1-9]' "$out" 2>/dev/null || echo 0) + log " metrics round $round: services=$svc_count, has_values=$has_cpm" + + sleep 10 + done +} +metrics_monitor & +BG_PIDS+=($!) + +############################################################################# +# Wait for initial data +############################################################################# +log "--- Waiting 30s for initial data ---" +sleep 30 + +############################################################################# +# Thread dump collection +############################################################################# +log "--- Collecting $DUMP_COUNT thread dumps (${DUMP_INTERVAL}s apart) ---" + +OAP_PODS=($(kubectl -n "$NAMESPACE" get pods -l "$OAP_SELECTOR" -o jsonpath='{.items[*].metadata.name}')) +log "OAP pods: ${OAP_PODS[*]}" + +for i in $(seq 1 "$DUMP_COUNT"); do + log "Thread dump round $i/$DUMP_COUNT..." + for pod in "${OAP_PODS[@]}"; do + safe_name="${pod//skywalking-oap-/oap-}" + outfile="$REPORT_DIR/${safe_name}-dump-${i}.txt" + echo "# Pod: $pod Time: $(date -u +%Y-%m-%dT%H:%M:%SZ)" > "$outfile" + + if kubectl -n "$NAMESPACE" exec "$pod" -c oap -- jstack 1 >> "$outfile" 2>&1; then + log " $pod: jstack OK ($(wc -l < "$outfile") lines)" + elif kubectl -n "$NAMESPACE" exec "$pod" -c oap -- jcmd 1 Thread.print >> "$outfile" 2>&1; then + log " $pod: jcmd OK ($(wc -l < "$outfile") lines)" + else + # JRE-only images lack jstack/jcmd. Use kill -3 (SIGQUIT) which + # triggers the JVM's built-in thread dump to stderr (container logs). + log " $pod: jstack/jcmd unavailable, using kill -3 (SIGQUIT)..." + ts_before=$(date -u +%Y-%m-%dT%H:%M:%SZ) + kubectl -n "$NAMESPACE" exec "$pod" -c oap -- kill -3 1 2>/dev/null || true + sleep 3 + # Capture the thread dump from container logs since the signal + kubectl -n "$NAMESPACE" logs "$pod" -c oap --since-time="$ts_before" >> "$outfile" 2>&1 + lines=$(wc -l < "$outfile") + if [ "$lines" -gt 5 ]; then + log " $pod: kill -3 OK ($lines lines)" + else + log " $pod: kill -3 produced only $lines lines (may have failed)" + fi + fi + done + if [ "$i" -lt "$DUMP_COUNT" ]; then + log " Sleeping ${DUMP_INTERVAL}s..." + sleep "$DUMP_INTERVAL" + fi +done + +############################################################################# +# Thread dump analysis +############################################################################# +log "--- Analyzing thread dumps ---" + +analyze_dumps() { + local analysis="$REPORT_DIR/thread-analysis.txt" + : > "$analysis" + + # JVM internal threads to exclude — not OAP application threads. + local JVM_FILTER='(^C[12] CompilerThread|^Common-Cleaner$|^DestroyJavaVM$|^Finalizer$|^G1 |^GC Thread|^PIC-Cleaner$|^Reference Handler$|^Service Thread$|^Signal Dispatcher$|^Sweeper thread$|^VM |^startstop-support$)' + + # Normalize a pool name: strip numeric thread suffixes, replace embedded + # IPs, hashcodes, and per-instance numbers with wildcards. + normalize_pool() { + sed -E 's/[-#][0-9]+$//' | sed -E 's/[-#][0-9]+$//' | sed -E 's/ [0-9]+$//' | \ + sed -E 's/[0-9]+\.[0-9]+\.[0-9]+\.[0-9]+_[0-9]+/*/g' | \ + sed -E 's/(HttpClient)-[0-9]+-(SelectorManager)/\1-*-\2/g' | \ + sed -E 's/(JdkHttpClientFactory)-[0-9]+-(pool)/\1-*-\2/g' | \ + sed -E 's/(CachedSingleThreadScheduler)-[0-9]+-(pool)/\1-*-\2/g' | \ + sed -E 's/^-?[0-9]{6,}-pool/scheduled-pool/g' + } + + # Helper: extract and normalize pool names from a dump file (OAP threads only). + extract_pools() { + awk '/^"/ { s=$0; sub(/^"/, "", s); sub(/".*/, "", s); print s }' "$1" | \ + normalize_pool | grep -vE "$JVM_FILTER" + } + + # Helper: count threads matching a pool name in a dump file. + count_pool() { + extract_pools "$1" | grep -cF -- "$2" || true + } + + # Find all dump file prefixes (e.g., oap-0, oap-1) + local prefixes=() + for f in "$REPORT_DIR"/oap-*-dump-1.txt; do + [ -f "$f" ] || continue + local base + base=$(basename "$f") + prefixes+=("${base%-dump-1.txt}") + done + + echo "================================================================" >> "$analysis" + echo " OAP Thread Analysis Report" >> "$analysis" + echo " Generated: $(date -u +%Y-%m-%dT%H:%M:%SZ)" >> "$analysis" + echo " Dump rounds: $DUMP_COUNT x ${DUMP_INTERVAL}s apart" >> "$analysis" + echo " OAP pods: ${prefixes[*]}" >> "$analysis" + echo " Note: JVM internal threads (GC, compiler, VM) are excluded." >> "$analysis" + echo "================================================================" >> "$analysis" + echo "" >> "$analysis" + + # ── Per-pod sections ────────────────────────────────────────────── + for pod_prefix in "${prefixes[@]}"; do + echo "================================================================" >> "$analysis" + echo " $pod_prefix — Thread Count Trend" >> "$analysis" + echo "================================================================" >> "$analysis" + echo "" >> "$analysis" + + # Header + printf "%-50s" "Pool Name" >> "$analysis" + for d in $(seq 1 "$DUMP_COUNT"); do + printf " %5s" "#$d" >> "$analysis" + done + printf " %s\n" "States (latest)" >> "$analysis" + printf "%-50s" "$(printf '%0.s-' {1..50})" >> "$analysis" + for d in $(seq 1 "$DUMP_COUNT"); do + printf " %5s" "-----" >> "$analysis" + done + printf " %s\n" "---------------" >> "$analysis" + + # Collect all OAP pool names across all dumps for this pod + local all_pools + all_pools=$(mktemp) + for d in $(seq 1 "$DUMP_COUNT"); do + local dumpfile="$REPORT_DIR/${pod_prefix}-dump-${d}.txt" + [ -f "$dumpfile" ] || continue + extract_pools "$dumpfile" | sort -u + done | sort -u > "$all_pools" + + # Parse states from the latest dump for this pod + local latest_dump="$REPORT_DIR/${pod_prefix}-dump-${DUMP_COUNT}.txt" + local tmp_states + tmp_states=$(mktemp) + if [ -f "$latest_dump" ]; then + awk ' + /^"/ { + tname = $0 + sub(/^"/, "", tname) + sub(/".*/, "", tname) + state = "UNKNOWN" + } + /java\.lang\.Thread\.State:/ { + state = $0 + sub(/.*State: /, "", state) + sub(/[^A-Z_].*/, "", state) + print tname "\t" state + } + ' "$latest_dump" | while IFS=$'\t' read -r tname tstate; do + local pool + pool=$(echo "$tname" | normalize_pool) + echo "$pool $tstate" + done | grep -vE "$JVM_FILTER" | sort | awk -F'\t' ' + { + pool = $1; state = $2 + states[pool, state]++ + if (!(pool in seen)) { seen[pool] = 1 } + } + END { + for (key in states) { + split(key, parts, SUBSEP) + print parts[1] "\t" parts[2] "\t" states[key] + } + }' > "$tmp_states" + fi + + local total_latest=0 + while IFS= read -r pool; do + printf "%-50s" "$pool" >> "$analysis" + local last_cnt=0 + for d in $(seq 1 "$DUMP_COUNT"); do + local dumpfile="$REPORT_DIR/${pod_prefix}-dump-${d}.txt" + local cnt=0 + if [ -f "$dumpfile" ]; then + cnt=$(count_pool "$dumpfile" "$pool") + fi + printf " %5d" "$cnt" >> "$analysis" + if [ "$d" -eq "$DUMP_COUNT" ]; then last_cnt=$cnt; fi + done + total_latest=$((total_latest + last_cnt)) + # Append states from latest dump + local state_str="" + if [ -f "$tmp_states" ]; then + state_str=$(awk -F'\t' -v p="$pool" '$1 == p { printf "%s(%s) ", $2, $3 }' "$tmp_states") + fi + printf " %s\n" "$state_str" >> "$analysis" + done < "$all_pools" + + echo "" >> "$analysis" + printf "%-50s" "TOTAL (OAP threads)" >> "$analysis" + for d in $(seq 1 "$DUMP_COUNT"); do + local dumpfile="$REPORT_DIR/${pod_prefix}-dump-${d}.txt" + local t=0 + if [ -f "$dumpfile" ]; then + t=$(extract_pools "$dumpfile" | wc -l | tr -d ' ') + fi + printf " %5d" "$t" >> "$analysis" + done + echo "" >> "$analysis" + echo "" >> "$analysis" + + rm -f "$all_pools" "$tmp_states" + done + + # ── Pod Comparison (latest dump) ────────────────────────────────── + if [ "${#prefixes[@]}" -ge 2 ]; then + echo "================================================================" >> "$analysis" + echo " Pod Comparison (dump #$DUMP_COUNT)" >> "$analysis" + echo "================================================================" >> "$analysis" + echo "" >> "$analysis" + + # Collect all OAP pool names across both pods' latest dumps + local cmp_pools + cmp_pools=$(mktemp) + for pod_prefix in "${prefixes[@]}"; do + local dumpfile="$REPORT_DIR/${pod_prefix}-dump-${DUMP_COUNT}.txt" + [ -f "$dumpfile" ] || continue + extract_pools "$dumpfile" | sort -u + done | sort -u > "$cmp_pools" + + # Header + printf "%-50s" "Pool Name" >> "$analysis" + for pod_prefix in "${prefixes[@]}"; do + printf " %8s" "$pod_prefix" >> "$analysis" + done + printf " %s\n" "Diff" >> "$analysis" + printf "%-50s" "$(printf '%0.s-' {1..50})" >> "$analysis" + for pod_prefix in "${prefixes[@]}"; do + printf " %8s" "--------" >> "$analysis" + done + printf " %s\n" "----" >> "$analysis" + + local diff_count=0 + while IFS= read -r pool; do + local counts=() + for pod_prefix in "${prefixes[@]}"; do + local dumpfile="$REPORT_DIR/${pod_prefix}-dump-${DUMP_COUNT}.txt" + local cnt=0 + if [ -f "$dumpfile" ]; then + cnt=$(count_pool "$dumpfile" "$pool") + fi + counts+=("$cnt") + done + + # Check if counts differ across pods + local has_diff=false + for c in "${counts[@]}"; do + if [ "$c" != "${counts[0]}" ]; then + has_diff=true + break + fi + done + + printf "%-50s" "$pool" >> "$analysis" + for c in "${counts[@]}"; do + printf " %8d" "$c" >> "$analysis" + done + if [ "$has_diff" = true ]; then + printf " %s\n" "<--" >> "$analysis" + diff_count=$((diff_count + 1)) + else + echo "" >> "$analysis" + fi + done < "$cmp_pools" + + echo "" >> "$analysis" + # Totals + printf "%-50s" "TOTAL (OAP threads)" >> "$analysis" + for pod_prefix in "${prefixes[@]}"; do + local dumpfile="$REPORT_DIR/${pod_prefix}-dump-${DUMP_COUNT}.txt" + local t=0 + if [ -f "$dumpfile" ]; then + t=$(extract_pools "$dumpfile" | wc -l | tr -d ' ') + fi + printf " %8d" "$t" >> "$analysis" + done + echo "" >> "$analysis" + echo "" >> "$analysis" + if [ "$diff_count" -gt 0 ]; then + echo "$diff_count pool(s) differ between pods (marked with <--)." >> "$analysis" + else + echo "All OAP thread pools are identical across pods." >> "$analysis" + fi + echo "" >> "$analysis" + + rm -f "$cmp_pools" + fi + + log "Analysis written to: $analysis" +} + +analyze_dumps + +############################################################################# +# Environment summary report +############################################################################# +log "--- Writing environment summary ---" + +ENV_REPORT="$REPORT_DIR/environment.txt" +{ + echo "================================================================" + echo " Benchmark Report: Thread Analysis" + echo " Environment: $ENV_NAME" + echo " Generated: $(date -u +%Y-%m-%dT%H:%M:%SZ)" + echo "================================================================" + echo "" + echo "--- Host ---" + echo " OS: $(uname -s) $(uname -r)" + echo " Arch: $(uname -m)" + echo "" + echo "--- Docker ---" + echo " Server: $DOCKER_SERVER_VERSION" + echo " OS: $DOCKER_OS" + echo " Driver: $DOCKER_STORAGE_DRIVER" + echo " CPUs: $DOCKER_CPUS" + echo " Memory: ${DOCKER_MEM_GB} GB" + echo "" + echo "--- Tool Versions ---" + echo " kind: $KIND_VERSION" + echo " kubectl: ${KUBECTL_CLIENT_VERSION:-unknown}" + echo " Helm: $HELM_VERSION" + echo " istioctl: ${ISTIOCTL_VERSION:-unknown}" + echo " swctl: $(swctl --version 2>/dev/null | head -1 || echo unknown)" + echo "" + echo "--- OAP JRE ---" + OAP_FIRST_POD=$(kubectl -n "$NAMESPACE" get pods -l "$OAP_SELECTOR" -o jsonpath='{.items[0].metadata.name}' 2>/dev/null) + if [ -n "$OAP_FIRST_POD" ]; then + OAP_JAVA_VERSION=$(kubectl -n "$NAMESPACE" exec "$OAP_FIRST_POD" -c oap -- java -version 2>&1 || echo "unknown") + OAP_OS_INFO=$(kubectl -n "$NAMESPACE" exec "$OAP_FIRST_POD" -c oap -- cat /etc/os-release 2>/dev/null \ + | grep -E '^(PRETTY_NAME|ID|VERSION_ID)=' | head -3 || echo "unknown") + OAP_ARCH=$(kubectl -n "$NAMESPACE" exec "$OAP_FIRST_POD" -c oap -- uname -m 2>/dev/null || echo "unknown") + echo "$OAP_JAVA_VERSION" | sed 's/^/ /' + echo " Arch: $OAP_ARCH" + echo "$OAP_OS_INFO" | sed 's/^/ /' + else + echo " (could not query OAP pod)" + fi + echo "" + echo "--- Kubernetes ---" + echo " Node image: K8s $K8S_NODE_MINOR (from kind.yaml)" + echo " Cluster: $CLUSTER_NAME" + echo " Namespace: $NAMESPACE" + echo "" + echo "--- K8s Node Resources ---" + if [ -f "$REPORT_DIR/node-resources.txt" ]; then + cat "$REPORT_DIR/node-resources.txt" + else + echo " (not captured)" + fi + echo "" + echo "--- Benchmark Config ---" + echo " OAP replicas: 2" + echo " Storage: BanyanDB (standalone)" + echo " Istio: ${ISTIO_VERSION:-N/A}" + echo " ALS analyzer: ${ALS_ANALYZER:-N/A}" + echo " Traffic rate: ~5 RPS" + echo " Thread dumps: $DUMP_COUNT x ${DUMP_INTERVAL}s apart" + echo "" + echo "--- Pod Status (at completion) ---" + kubectl -n "$NAMESPACE" get pods -o wide 2>/dev/null || echo " (could not query)" + echo "" + echo "--- Bookinfo Pods (default namespace) ---" + kubectl -n default get pods -o wide 2>/dev/null || echo " (could not query)" + echo "" + echo "--- Pod Resource Usage (requests) ---" + kubectl -n "$NAMESPACE" describe node 2>/dev/null \ + | sed -n '/Non-terminated Pods/,/Allocated resources/p' || echo " (could not query)" + echo "" +} > "$ENV_REPORT" + +log "Environment summary: $ENV_REPORT" + +############################################################################# +# Done +############################################################################# +log "=== Thread analysis complete ===" +log "Reports in: $REPORT_DIR" +log " environment.txt - Host, Docker, K8s resources, tool versions" +log " node-resources.txt - K8s node capacity and allocatable" +log " thread-analysis.txt - Thread pool summary and trends" +log " metrics-round-*.yaml - Periodic swctl query results" +log " oap-*-dump-*.txt - Raw thread dumps" + +# Stop background metrics monitor +cleanup_pids +BG_PIDS=() + +log "Done. Environment is still running." diff --git a/benchmarks/env b/benchmarks/env new file mode 100644 index 000000000000..60b011299cf8 --- /dev/null +++ b/benchmarks/env @@ -0,0 +1,52 @@ +# 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. + +# Benchmark environment configuration. +# All environment setup scripts source this file for image repos and versions. +# +# To use local builds, set *_IMAGE_MODE to "local" and keep *_IMAGE_TAG as "latest". +# To use remote images, set *_IMAGE_MODE to "remote" and configure the full registry path. + +############################################################################## +# SkyWalking OAP +############################################################################## +SW_OAP_IMAGE_MODE="local" +SW_OAP_IMAGE_REPO="skywalking/oap" +SW_OAP_IMAGE_TAG="latest" + +############################################################################## +# SkyWalking UI +############################################################################## +SW_UI_IMAGE_MODE="local" +SW_UI_IMAGE_REPO="skywalking/ui" +SW_UI_IMAGE_TAG="latest" + +############################################################################## +# BanyanDB +############################################################################## +SW_BANYANDB_IMAGE_REPO="ghcr.io/apache/skywalking-banyandb" +SW_BANYANDB_IMAGE_TAG="208982aaa11092bc38018a9e1b24eda67e829312" + +############################################################################## +# SkyWalking Helm Chart +############################################################################## +SW_HELM_CHART="oci://ghcr.io/apache/skywalking-helm/skywalking-helm" +SW_KUBERNETES_COMMIT_SHA="6fe5e6f0d3b7686c6be0457733e825ee68cb9b35" + +############################################################################## +# Istio +############################################################################## +ISTIO_VERSION="1.25.2" +ALS_ANALYZER="k8s-mesh" diff --git a/benchmarks/envs-setup/cluster_oap-banyandb/kind.yaml b/benchmarks/envs-setup/cluster_oap-banyandb/kind.yaml new file mode 100644 index 000000000000..dc96ee5e50bf --- /dev/null +++ b/benchmarks/envs-setup/cluster_oap-banyandb/kind.yaml @@ -0,0 +1,20 @@ +# 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. + +kind: Cluster +apiVersion: kind.x-k8s.io/v1alpha4 +nodes: + - role: control-plane + image: kindest/node:v1.34.3@sha256:08497ee19eace7b4b5348db5c6a1591d7752b164530a36f855cb0f2bdcbadd48 diff --git a/benchmarks/envs-setup/cluster_oap-banyandb/setup.sh b/benchmarks/envs-setup/cluster_oap-banyandb/setup.sh new file mode 100755 index 000000000000..a97dce8295fc --- /dev/null +++ b/benchmarks/envs-setup/cluster_oap-banyandb/setup.sh @@ -0,0 +1,444 @@ +#!/usr/bin/env bash +# 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. + +# Environment setup: 2-node OAP cluster + BanyanDB + Istio ALS on Kind. +# +# Deploys Istio with Access Log Service (ALS) enabled, SkyWalking OAP +# cluster receiving telemetry from Envoy sidecars, and Istio Bookinfo +# sample app as the workload. +# +# The context file is written to: $REPORT_DIR/env-context.sh + +set -euo pipefail + +SCRIPT_DIR="$(cd "$(dirname "$0")" && pwd)" +ROOT_DIR="$(cd "$SCRIPT_DIR/../../.." && pwd)" + +NAMESPACE="istio-system" +CLUSTER_NAME="benchmark-cluster" + +# Load benchmark environment configuration (image repos, versions) +BENCHMARKS_DIR="$(cd "$SCRIPT_DIR/../.." && pwd)" +source "$BENCHMARKS_DIR/env" + +# Kind ↔ Kubernetes node image compatibility table. +# Source: https://github.com/kubernetes-sigs/kind/releases +KIND_MIN_VERSION="0.25.0" +HELM_MIN_VERSION="3.12.0" +# K8s minor version from the node image in kind.yaml +K8S_NODE_MINOR="1.34" + +log() { echo "[$(date +%H:%M:%S)] $*"; } + +# Compare two semver strings: returns 0 if $1 >= $2, 1 otherwise. +version_gte() { + local IFS=. + local i a=($1) b=($2) + for ((i = 0; i < ${#b[@]}; i++)); do + local va=${a[i]:-0} + local vb=${b[i]:-0} + if ((va > vb)); then return 0; fi + if ((va < vb)); then return 1; fi + done + return 0 +} + +# Map K8s minor version to the minimum kind version that ships a node image. +min_kind_for_k8s() { + case "$1" in + 1.28) echo "0.25.0" ;; + 1.29) echo "0.25.0" ;; + 1.30) echo "0.27.0" ;; + 1.31) echo "0.25.0" ;; + 1.32) echo "0.26.0" ;; + 1.33) echo "0.27.0" ;; + 1.34) echo "0.30.0" ;; + 1.35) echo "0.31.0" ;; + *) echo "unknown" ;; + esac +} + +# Report dir is passed in by the caller (benchmark case), or defaults. +REPORT_DIR="${REPORT_DIR:?ERROR: REPORT_DIR must be set by the caller}" +mkdir -p "$REPORT_DIR" + +############################################################################# +# Pre-checks +############################################################################# +log "=== Pre-checks ===" + +for cmd in kind kubectl helm docker; do + if ! command -v "$cmd" &>/dev/null; then + echo "ERROR: $cmd not found. Please install it first." + exit 1 + fi +done + +# --- kind version check --- +KIND_VERSION=$(kind version | grep -oE '[0-9]+\.[0-9]+\.[0-9]+' | head -1) +log "kind version: $KIND_VERSION (minimum: $KIND_MIN_VERSION)" +if ! version_gte "$KIND_VERSION" "$KIND_MIN_VERSION"; then + echo "ERROR: kind >= $KIND_MIN_VERSION is required, found $KIND_VERSION" + exit 1 +fi + +# --- kind ↔ K8s node image compatibility --- +REQUIRED_KIND_FOR_NODE=$(min_kind_for_k8s "$K8S_NODE_MINOR") +if [ "$REQUIRED_KIND_FOR_NODE" = "unknown" ]; then + echo "WARNING: No known kind compatibility data for K8s $K8S_NODE_MINOR. Proceeding anyway." +elif ! version_gte "$KIND_VERSION" "$REQUIRED_KIND_FOR_NODE"; then + echo "ERROR: K8s $K8S_NODE_MINOR node image requires kind >= $REQUIRED_KIND_FOR_NODE, found $KIND_VERSION" + echo " Either upgrade kind or change the node image in kind.yaml." + exit 1 +fi +log "kind $KIND_VERSION is compatible with K8s $K8S_NODE_MINOR node image." + +# --- kubectl version check (must be within ±1 minor of K8s node) --- +KUBECTL_CLIENT_VERSION=$(kubectl version --client -o json 2>/dev/null \ + | grep -oE '"gitVersion":\s*"v([0-9]+\.[0-9]+)' | head -1 | grep -oE '[0-9]+\.[0-9]+') +if [ -n "$KUBECTL_CLIENT_VERSION" ]; then + KUBECTL_MINOR=$(echo "$KUBECTL_CLIENT_VERSION" | cut -d. -f2) + NODE_MINOR=$(echo "$K8S_NODE_MINOR" | cut -d. -f2) + SKEW=$((KUBECTL_MINOR - NODE_MINOR)) + if [ "$SKEW" -lt 0 ]; then SKEW=$((-SKEW)); fi + log "kubectl client: $KUBECTL_CLIENT_VERSION, K8s node: $K8S_NODE_MINOR (skew: $SKEW)" + if [ "$SKEW" -gt 1 ]; then + echo "ERROR: kubectl version $KUBECTL_CLIENT_VERSION is too far from K8s $K8S_NODE_MINOR (max ±1 minor)." + echo " See https://kubernetes.io/releases/version-skew-policy/" + exit 1 + fi +else + echo "WARNING: Could not determine kubectl client version, skipping skew check." +fi + +# --- Helm version check --- +HELM_VERSION=$(helm version --short 2>/dev/null | grep -oE '[0-9]+\.[0-9]+\.[0-9]+' | head -1) +log "Helm version: $HELM_VERSION (minimum: $HELM_MIN_VERSION)" +if ! version_gte "$HELM_VERSION" "$HELM_MIN_VERSION"; then + echo "ERROR: Helm >= $HELM_MIN_VERSION is required, found $HELM_VERSION" + exit 1 +fi + +# --- istioctl version check / download --- +# Download the correct istioctl version if not available or version mismatch. +ISTIOCTL_LOCAL_VERSION=$(istioctl version --remote=false 2>/dev/null | head -1 || echo "none") +if [ "$ISTIOCTL_LOCAL_VERSION" != "$ISTIO_VERSION" ]; then + log "istioctl version mismatch: have $ISTIOCTL_LOCAL_VERSION, need $ISTIO_VERSION. Downloading..." + ISTIO_DOWNLOAD_DIR="$BENCHMARKS_DIR/.istio" + mkdir -p "$ISTIO_DOWNLOAD_DIR" + if [ ! -f "$ISTIO_DOWNLOAD_DIR/istio-${ISTIO_VERSION}/bin/istioctl" ]; then + (cd "$ISTIO_DOWNLOAD_DIR" && export ISTIO_VERSION && curl -sL https://istio.io/downloadIstio | sh -) + fi + if [ ! -f "$ISTIO_DOWNLOAD_DIR/istio-${ISTIO_VERSION}/bin/istioctl" ]; then + echo "ERROR: Failed to download istioctl $ISTIO_VERSION" + echo " Check available versions at https://github.com/istio/istio/releases" + exit 1 + fi + export PATH="$ISTIO_DOWNLOAD_DIR/istio-${ISTIO_VERSION}/bin:$PATH" + ISTIOCTL_VERSION=$(istioctl version --remote=false 2>/dev/null | head -1 || echo "unknown") + log "Using downloaded istioctl: $ISTIOCTL_VERSION" +else + ISTIOCTL_VERSION="$ISTIOCTL_LOCAL_VERSION" + log "istioctl version: $ISTIOCTL_VERSION" +fi + +log "All version checks passed." + +# --- Docker daemon resources (caps what Kind can use) --- +DOCKER_CPUS=$(docker info --format '{{.NCPU}}' 2>/dev/null || echo "unknown") +DOCKER_MEM_BYTES=$(docker info --format '{{.MemTotal}}' 2>/dev/null || echo "0") +if [ "$DOCKER_MEM_BYTES" -gt 0 ] 2>/dev/null; then + DOCKER_MEM_GB=$(awk "BEGIN {printf \"%.1f\", $DOCKER_MEM_BYTES / 1073741824}") +else + DOCKER_MEM_GB="unknown" +fi +DOCKER_SERVER_VERSION=$(docker info --format '{{.ServerVersion}}' 2>/dev/null || echo "unknown") +DOCKER_OS=$(docker info --format '{{.OperatingSystem}}' 2>/dev/null || echo "unknown") +DOCKER_STORAGE_DRIVER=$(docker info --format '{{.Driver}}' 2>/dev/null || echo "unknown") +log "Docker: ${DOCKER_CPUS} CPUs, ${DOCKER_MEM_GB} GB memory (server: ${DOCKER_SERVER_VERSION}, ${DOCKER_OS})" + +if [ "$DOCKER_MEM_BYTES" -gt 0 ] 2>/dev/null; then + MIN_MEM_BYTES=$((4 * 1073741824)) # 4 GB + if [ "$DOCKER_MEM_BYTES" -lt "$MIN_MEM_BYTES" ]; then + echo "WARNING: Docker has only ${DOCKER_MEM_GB} GB memory. Recommend >= 4 GB for this benchmark." + fi +fi +if [ "$DOCKER_CPUS" != "unknown" ] && [ "$DOCKER_CPUS" -lt 2 ] 2>/dev/null; then + echo "WARNING: Docker has only ${DOCKER_CPUS} CPU(s). Recommend >= 2 CPUs for this benchmark." +fi + +############################################################################# +# Boot Kind cluster +############################################################################# +log "=== Booting Kind cluster ===" + +if kind get clusters 2>/dev/null | grep -q "^${CLUSTER_NAME}$"; then + log "Kind cluster '$CLUSTER_NAME' already exists, reusing." +else + log "Creating Kind cluster '$CLUSTER_NAME'..." + kind create cluster --name "$CLUSTER_NAME" --config "$SCRIPT_DIR/kind.yaml" +fi + +# Pre-pull and load ALL required images into Kind. +# Kind nodes may not have access to Docker Hub (proxy or network issues), +# so we pull on the host first and then load into the cluster. +BANYANDB_IMAGE="${SW_BANYANDB_IMAGE_REPO}:${SW_BANYANDB_IMAGE_TAG}" +OAP_IMAGE="${SW_OAP_IMAGE_REPO}:${SW_OAP_IMAGE_TAG}" +UI_IMAGE="${SW_UI_IMAGE_REPO}:${SW_UI_IMAGE_TAG}" +IMAGES=( + # SkyWalking + "$OAP_IMAGE" + "$UI_IMAGE" + "$BANYANDB_IMAGE" + # Istio core + "docker.io/istio/pilot:${ISTIO_VERSION}" + "docker.io/istio/proxyv2:${ISTIO_VERSION}" + # Bookinfo sample app + "docker.io/istio/examples-bookinfo-productpage-v1:1.20.2" + "docker.io/istio/examples-bookinfo-details-v1:1.20.2" + "docker.io/istio/examples-bookinfo-reviews-v1:1.20.2" + "docker.io/istio/examples-bookinfo-reviews-v2:1.20.2" + "docker.io/istio/examples-bookinfo-reviews-v3:1.20.2" + "docker.io/istio/examples-bookinfo-ratings-v1:1.20.2" + "docker.io/istio/examples-bookinfo-ratings-v2:1.20.2" + "docker.io/istio/examples-bookinfo-mongodb:1.20.2" + # Helm chart init containers + traffic generator + "curlimages/curl:latest" +) +log "Pulling images on host (if not cached)..." +for img in "${IMAGES[@]}"; do + if [ "$SW_OAP_IMAGE_MODE" = "local" ] && [ "$img" = "$OAP_IMAGE" ]; then continue; fi + if [ "$SW_UI_IMAGE_MODE" = "local" ] && [ "$img" = "$UI_IMAGE" ]; then continue; fi + pulled=false + for attempt in 1 2 3; do + if docker pull "$img" -q 2>/dev/null; then + pulled=true + break + fi + [ "$attempt" -lt 3 ] && sleep 5 + done + if [ "$pulled" = false ]; then + log " WARNING: failed to pull $img after 3 attempts" + fi +done + +log "Loading images into Kind..." +for img in "${IMAGES[@]}"; do + kind load docker-image "$img" --name "$CLUSTER_NAME" 2>/dev/null || log " WARNING: failed to load $img" +done + +############################################################################# +# Install Istio +############################################################################# +log "=== Installing Istio $ISTIO_VERSION ===" + +istioctl install -y --set profile=demo \ + --set meshConfig.defaultConfig.envoyAccessLogService.address=skywalking-oap.${NAMESPACE}:11800 \ + --set meshConfig.enableEnvoyAccessLogService=true + +# Enable sidecar injection in default namespace +kubectl label namespace default istio-injection=enabled --overwrite + +############################################################################# +# Deploy SkyWalking via Helm (in istio-system namespace) +############################################################################# +log "=== Deploying SkyWalking (OAP x2 + BanyanDB + UI) via Helm ===" + +helm -n "$NAMESPACE" upgrade --install skywalking \ + "$SW_HELM_CHART" \ + --version "0.0.0-${SW_KUBERNETES_COMMIT_SHA}" \ + --set fullnameOverride=skywalking \ + --set oap.replicas=2 \ + --set oap.image.repository="$SW_OAP_IMAGE_REPO" \ + --set oap.image.tag="$SW_OAP_IMAGE_TAG" \ + --set oap.storageType=banyandb \ + --set oap.env.SW_ENVOY_METRIC_ALS_HTTP_ANALYSIS="$ALS_ANALYZER" \ + --set oap.env.SW_ENVOY_METRIC_ALS_TCP_ANALYSIS="$ALS_ANALYZER" \ + --set oap.env.SW_HEALTH_CHECKER=default \ + --set oap.env.SW_TELEMETRY=prometheus \ + --set oap.envoy.als.enabled=true \ + --set ui.image.repository="$SW_UI_IMAGE_REPO" \ + --set ui.image.tag="$SW_UI_IMAGE_TAG" \ + --set elasticsearch.enabled=false \ + --set banyandb.enabled=true \ + --set banyandb.image.repository="$SW_BANYANDB_IMAGE_REPO" \ + --set banyandb.image.tag="$SW_BANYANDB_IMAGE_TAG" \ + --set banyandb.standalone.enabled=true \ + --timeout 1200s \ + -f "$SCRIPT_DIR/values.yaml" + +# Wait for BanyanDB → init job → OAP pods +log "Waiting for BanyanDB to be ready..." +kubectl -n "$NAMESPACE" wait --for=condition=ready pod -l app.kubernetes.io/name=banyandb --timeout=300s + +log "Waiting for OAP init job to complete..." +for i in $(seq 1 60); do + if kubectl -n "$NAMESPACE" get jobs -l component=skywalking-job -o jsonpath='{.items[0].status.succeeded}' 2>/dev/null | grep -q '1'; then + log "OAP init job succeeded." + break + fi + if [ "$i" -eq 60 ]; then + echo "ERROR: OAP init job did not complete within 300s." + kubectl -n "$NAMESPACE" get pods -l component=skywalking-job 2>/dev/null + exit 1 + fi + sleep 5 +done + +log "Waiting for OAP pods to be ready..." +kubectl -n "$NAMESPACE" wait --for=condition=ready pod -l app=skywalking,component=oap --timeout=300s + +############################################################################# +# Capture K8s node resources +############################################################################# +log "Capturing node resource info..." +kubectl get nodes -o json | awk ' + BEGIN { print "--- K8s Node Resources ---" } + /"capacity":/ { cap=1 } /"allocatable":/ { alloc=1 } + cap && /"cpu":/ { gsub(/[",]/, ""); printf " capacity.cpu: %s\n", $2; cap=0 } + cap && /"memory":/ { gsub(/[",]/, ""); printf " capacity.memory: %s\n", $2; cap=0 } + cap && /"ephemeral-storage":/ { gsub(/[",]/, ""); printf " capacity.storage: %s\n", $2; cap=0 } + cap && /"pods":/ { gsub(/[",]/, ""); printf " capacity.pods: %s\n", $2; cap=0 } + alloc && /"cpu":/ { gsub(/[",]/, ""); printf " allocatable.cpu: %s\n", $2; alloc=0 } + alloc && /"memory":/ { gsub(/[",]/, ""); printf " allocatable.memory: %s\n", $2; alloc=0 } +' > "$REPORT_DIR/node-resources.txt" +kubectl describe node | sed -n '/Allocated resources/,/Events/p' \ + >> "$REPORT_DIR/node-resources.txt" 2>/dev/null || true + +############################################################################# +# Deploy Istio Bookinfo sample app +############################################################################# +log "=== Deploying Bookinfo sample app ===" + +BOOKINFO_BASE="https://raw.githubusercontent.com/istio/istio/${ISTIO_VERSION}/samples/bookinfo" + +kubectl apply -f "${BOOKINFO_BASE}/platform/kube/bookinfo.yaml" +kubectl apply -f "${BOOKINFO_BASE}/networking/bookinfo-gateway.yaml" +# Enable TCP services (ratings-v2 + MongoDB) +kubectl apply -f "${BOOKINFO_BASE}/platform/kube/bookinfo-ratings-v2.yaml" +kubectl apply -f "${BOOKINFO_BASE}/platform/kube/bookinfo-db.yaml" +kubectl apply -f "${BOOKINFO_BASE}/networking/destination-rule-all.yaml" +kubectl apply -f "${BOOKINFO_BASE}/networking/virtual-service-ratings-db.yaml" + +log "Waiting for Bookinfo pods to be ready..." +# Wait for critical Bookinfo pods (reviews-v3 is optional — may fail if image pull fails). +kubectl -n default wait --for=condition=ready pod -l app=productpage --timeout=300s +kubectl -n default wait --for=condition=ready pod -l app=details --timeout=300s +kubectl -n default wait --for=condition=ready pod -l app=ratings,version=v1 --timeout=300s +kubectl -n default wait --for=condition=ready pod -l app=reviews,version=v1 --timeout=300s +kubectl -n default wait --for=condition=ready pod -l app=reviews,version=v2 --timeout=300s + +log "Deploying traffic generator..." +kubectl apply -f "$SCRIPT_DIR/traffic-gen.yaml" +kubectl -n default wait --for=condition=ready pod -l app=traffic-gen --timeout=60s + +############################################################################# +# Cluster health check — verify OAP inter-node communication +############################################################################# +log "=== Cluster health check (remote_out_count) ===" +log "Waiting 30s for traffic to flow..." +sleep 30 + +OAP_PODS_CHECK=($(kubectl -n "$NAMESPACE" get pods -l app=skywalking,component=oap -o jsonpath='{.items[*].metadata.name}')) +EXPECTED_NODES=${#OAP_PODS_CHECK[@]} +CLUSTER_HEALTHY=true + +for pod in "${OAP_PODS_CHECK[@]}"; do + log " Checking $pod..." + METRICS=$(kubectl -n "$NAMESPACE" exec "$pod" -c oap -- curl -s http://localhost:1234/metrics 2>/dev/null) + REMOTE_OUT=$(echo "$METRICS" | grep '^remote_out_count{' || true) + + if [ -z "$REMOTE_OUT" ]; then + REMOTE_IN=$(echo "$METRICS" | grep '^remote_in_count{' || true) + if [ -n "$REMOTE_IN" ]; then + log " $pod: no remote_out_count but has remote_in_count (receiver-only node)" + echo "$REMOTE_IN" | sed 's/^/ /' + else + log " WARNING: $pod has no remote_out_count or remote_in_count" + CLUSTER_HEALTHY=false + fi + else + DEST_COUNT=$(echo "$REMOTE_OUT" | wc -l | tr -d ' ') + SELF_COUNT=$(echo "$REMOTE_OUT" | grep 'self="Y"' | wc -l | tr -d ' ') + REMOTE_COUNT=$(echo "$REMOTE_OUT" | grep 'self="N"' | wc -l | tr -d ' ') + log " $pod: $DEST_COUNT dest(s) — self=$SELF_COUNT, remote=$REMOTE_COUNT" + echo "$REMOTE_OUT" | sed 's/^/ /' + if [ "$DEST_COUNT" -lt "$EXPECTED_NODES" ]; then + log " WARNING: Expected $EXPECTED_NODES destinations, found $DEST_COUNT" + fi + fi +done + +if [ "$CLUSTER_HEALTHY" = true ]; then + log "Cluster health check passed." +else + log "WARNING: Cluster health check has issues. Proceeding anyway." +fi + +############################################################################# +# Port-forward OAP for local queries +############################################################################# +log "Setting up port-forwards..." +kubectl -n "$NAMESPACE" port-forward svc/skywalking-oap 12800:12800 & +SETUP_BG_PIDS=($!) +kubectl -n "$NAMESPACE" port-forward svc/skywalking-ui 8080:80 & +SETUP_BG_PIDS+=($!) +sleep 3 + +log "Environment is up. OAP at localhost:12800, UI at localhost:8080" + +############################################################################# +# Write context file for benchmark cases +############################################################################# +CONTEXT_FILE="$REPORT_DIR/env-context.sh" +cat > "$CONTEXT_FILE" < +# +# Mode 2: Setup environment + run benchmark case +# ./benchmarks/run.sh run +# +# Available environments: (ls benchmarks/envs-setup/) +# Available cases: (ls benchmarks/cases/) +# +# Examples: +# ./benchmarks/run.sh setup cluster_oap-banyandb +# ./benchmarks/run.sh run cluster_oap-banyandb thread-analysis + +set -euo pipefail + +BENCHMARKS_DIR="$(cd "$(dirname "$0")" && pwd)" +TIMESTAMP="$(date +%Y%m%d-%H%M%S)" + +# Load cluster name from env config (for cleanup) +source "$BENCHMARKS_DIR/env" 2>/dev/null || true +CLUSTER_NAME="${CLUSTER_NAME:-benchmark-cluster}" + +# Cleanup: delete Kind cluster and prune Docker resources. +# Called automatically after 'run' mode completes (success or failure). +# Deleting the Kind cluster reclaims the largest chunk of disk (copied images). +# We only prune dangling images (not -a) to preserve locally built images. +cleanup_cluster() { + echo "" + echo ">>> Cleaning up..." + if kind get clusters 2>/dev/null | grep -q "^${CLUSTER_NAME}$"; then + echo " Deleting Kind cluster '${CLUSTER_NAME}'..." + kind delete cluster --name "$CLUSTER_NAME" 2>&1 || true + fi + echo " Pruning dangling Docker resources..." + docker image prune -f 2>&1 | tail -1 || true + docker volume prune -f 2>&1 | tail -1 || true + echo ">>> Cleanup complete." +} + +usage() { + echo "Usage:" + echo " $0 setup Setup environment only" + echo " $0 run Setup environment + run benchmark case" + echo "" + echo "Available environments:" + for d in "$BENCHMARKS_DIR"/envs-setup/*/; do + [ -d "$d" ] && echo " $(basename "$d")" + done + echo "" + echo "Available cases:" + for d in "$BENCHMARKS_DIR"/cases/*/; do + [ -d "$d" ] && echo " $(basename "$d")" + done + exit 1 +} + +if [ $# -lt 2 ]; then + usage +fi + +MODE="$1" +ENV_NAME="$2" + +ENV_DIR="$BENCHMARKS_DIR/envs-setup/$ENV_NAME" +if [ ! -d "$ENV_DIR" ] || [ ! -f "$ENV_DIR/setup.sh" ]; then + echo "ERROR: Environment '$ENV_NAME' not found." + echo " Expected: $ENV_DIR/setup.sh" + exit 1 +fi + +case "$MODE" in + setup) + export REPORT_DIR="$BENCHMARKS_DIR/reports/$ENV_NAME/$TIMESTAMP" + mkdir -p "$REPORT_DIR" + + # Clean up on failure only (user inspects the env on success) + setup_cleanup_on_error() { + local rc=$? + if [ $rc -ne 0 ]; then + cleanup_cluster + fi + } + trap setup_cleanup_on_error EXIT + + echo "=== Setting up environment: $ENV_NAME ===" + echo " Report dir: $REPORT_DIR" + echo "" + + "$ENV_DIR/setup.sh" + + echo "" + echo "=== Environment ready ===" + echo " Context file: $REPORT_DIR/env-context.sh" + echo "" + echo "To run a benchmark case against this environment:" + echo " $0 run $ENV_NAME " + echo " — or directly —" + echo " benchmarks/cases//run.sh $REPORT_DIR/env-context.sh" + echo "" + echo "To tear down when done:" + echo " kind delete cluster --name $CLUSTER_NAME" + ;; + + run) + if [ $# -lt 3 ]; then + echo "ERROR: 'run' mode requires both and ." + echo "" + usage + fi + CASE_NAME="$3" + + CASE_DIR="$BENCHMARKS_DIR/cases/$CASE_NAME" + if [ ! -d "$CASE_DIR" ] || [ ! -f "$CASE_DIR/run.sh" ]; then + echo "ERROR: Case '$CASE_NAME' not found." + echo " Expected: $CASE_DIR/run.sh" + exit 1 + fi + + export REPORT_DIR="$BENCHMARKS_DIR/reports/$ENV_NAME/$CASE_NAME/$TIMESTAMP" + mkdir -p "$REPORT_DIR" + + # Always clean up after run mode (success or failure) + trap cleanup_cluster EXIT + + echo "=== Benchmark: $CASE_NAME on $ENV_NAME ===" + echo " Report dir: $REPORT_DIR" + echo "" + + # Phase 1: Setup environment + echo ">>> Setting up environment: $ENV_NAME" + "$ENV_DIR/setup.sh" + + CONTEXT_FILE="$REPORT_DIR/env-context.sh" + if [ ! -f "$CONTEXT_FILE" ]; then + echo "ERROR: setup.sh did not produce $CONTEXT_FILE" + exit 1 + fi + + # Phase 2: Run benchmark case + echo "" + echo ">>> Running case: $CASE_NAME" + "$CASE_DIR/run.sh" "$CONTEXT_FILE" + ;; + + *) + echo "ERROR: Unknown mode '$MODE'." + echo "" + usage + ;; +esac diff --git a/docs/en/guides/benchmark.md b/docs/en/guides/benchmark.md index 5f11314bbc9a..ef38df757f71 100644 --- a/docs/en/guides/benchmark.md +++ b/docs/en/guides/benchmark.md @@ -1,15 +1,146 @@ -# Java Microbenchmark Harness (JMH) -JMH is a Java harness for building, running, and analysing nano/micro/milli/macro benchmarks written in Java and other languages targeting the JVM. +# Benchmark Tests -We have a module called `microbench` which performs a series of micro-benchmark tests for JMH testing. -Make new JMH tests extend the `org.apache.skywalking.oap.server.microbench.base.AbstractMicrobenchmark` -to customize runtime conditions (Measurement, Fork, Warmup, etc.). +The `benchmarks/` directory provides a framework for running system-level benchmarks against +SkyWalking deployments. It separates **environment setup** from **benchmark cases** so that the +same case can run against different topologies, and the same environment can serve multiple cases. -You can build the jar with command `./mvnw -Dmaven.test.skip -DskipTests -pl :microbench package -am -Pbenchmark`. +## Directory Structure -JMH tests could run as a normal unit test. And they could run as an independent uber jar via `java -jar benchmark.jar` for all benchmarks, -or via `java -jar /benchmarks.jar exampleClassName` for a specific test. +``` +benchmarks/ +├── run.sh # Entry point (two modes) +├── env # Image repos and versions (local/remote) +├── envs-setup/ # Reusable environment definitions +│ └── / +│ ├── setup.sh # Boots the environment, writes env-context.sh +│ └── ... # K8s manifests, Helm values, etc. +├── cases/ # Reusable benchmark cases +│ └── / +│ └── run.sh # Reads env-context.sh, runs the benchmark +└── reports/ # Git-ignored, timestamped output +``` -Output test results in JSON format, you can add `-rf json` like `java -jar benchmarks.jar -rf json`, if you run through the IDE, you can configure the `-DperfReportDir=savePath` parameter to set the JMH report result save path, a report results in JSON format will be generated when the run ends. +## Quick Start -More information about JMH can be found here: [jmh docs](https://openjdk.java.net/projects/code-tools/jmh/). +### Prerequisites + +- Docker +- [Kind](https://kind.sigs.k8s.io/) (version checked automatically against the K8s node image) +- kubectl (version skew checked against the K8s node — max ±1 minor) +- [Helm](https://helm.sh/) >= 3.12.0 +- [istioctl](https://istio.io/latest/docs/setup/getting-started/#download) (for Istio-based environments) +- [swctl](https://github.com/apache/skywalking-cli) (SkyWalking CLI) +- Pre-built OAP and UI Docker images + +```bash +# Build OAP/UI images (from repo root) +./mvnw clean package -Pall -Dmaven.test.skip +make docker +``` + +### Two Modes + +**Mode 1 — Setup environment only.** Useful for manual inspection or running cases later. + +```bash +./benchmarks/run.sh setup +``` + +The script boots the environment (Kind cluster, Helm deploy, traffic generator, port-forwards) +and prints the path to the generated `env-context.sh` file. You can then attach any case to it: + +```bash +./benchmarks/cases//run.sh +``` + +**Mode 2 — Setup + run case end-to-end.** + +```bash +./benchmarks/run.sh run +``` + +This chains environment setup and benchmark execution in one command. + +### Example + +```bash +# Full run: cluster OAP + BanyanDB environment, thread analysis case +./benchmarks/run.sh run cluster_oap-banyandb thread-analysis +``` + +## Available Environments + +| Name | Description | +|------|-------------| +| `cluster_oap-banyandb` | 2-node OAP cluster with BanyanDB on Kind. Istio ALS (Access Log Service) for telemetry. Bookinfo sample app with Envoy sidecars at ~5 RPS. Deployed via SkyWalking Helm chart. | + +### Environment Pre-checks + +Each environment setup script validates before proceeding: + +- Required CLI tools are installed (`kind`, `kubectl`, `helm`, `docker`) +- **Kind version** meets the minimum for the K8s node image (compatibility table in the script) +- **kubectl version skew** is within ±1 minor of the K8s node version +- **Helm version** meets the minimum +- **Docker resources** (CPUs, memory) are reported and warned if below recommended thresholds + +## Available Cases + +| Name | Description | +|------|-------------| +| `thread-analysis` | Collects periodic OAP thread dumps, monitors metrics via swctl, and produces a thread pool analysis report with per-pool state breakdown and count trends. | + +### thread-analysis + +Configurable via environment variables: + +| Variable | Default | Description | +|----------|---------|-------------| +| `DUMP_COUNT` | 5 | Number of thread dump rounds | +| `DUMP_INTERVAL` | 60 | Seconds between dump rounds | + +## Reports + +Reports are saved under `benchmarks/reports/` (git-ignored) with a timestamped directory: + +``` +benchmarks/reports//// +├── environment.txt # Host, Docker, K8s resources, tool versions +├── node-resources.txt # K8s node capacity and allocatable +├── thread-analysis.txt # Thread pool summary and trend table +├── metrics-round-*.yaml # Periodic swctl query results +├── oap-*-dump-*.txt # Raw jstack thread dumps +└── env-context.sh # Environment variables for the case +``` + +## Adding a New Environment + +1. Create `benchmarks/envs-setup//` with a `setup.sh`. +2. `setup.sh` must read `$REPORT_DIR` (set by the caller) and write `$REPORT_DIR/env-context.sh` + exporting at least: + - `ENV_NAME`, `NAMESPACE`, `CLUSTER_NAME` + - `OAP_HOST`, `OAP_PORT`, `OAP_SELECTOR` + - `REPORT_DIR` + - Resource info (`DOCKER_CPUS`, `DOCKER_MEM_GB`, tool versions, etc.) +3. Add the environment to the table above. + +## Adding a New Case + +1. Create `benchmarks/cases//` with a `run.sh`. +2. `run.sh` takes the env-context file as its first argument and sources it. +3. The case uses `$NAMESPACE`, `$OAP_HOST`, `$OAP_PORT`, `$OAP_SELECTOR`, `$REPORT_DIR`, etc. + from the context. +4. Add the case to the table above. + +## Cleanup + +**`run` mode** automatically deletes the Kind cluster and prunes Docker resources when the +benchmark finishes (whether it succeeds or fails). + +**`setup` mode** leaves the environment running for manual inspection. If setup fails, the +cluster and Docker resources are cleaned up automatically. To tear down a successfully set up +environment manually: + +```bash +kind delete cluster --name benchmark-cluster +``` From 6f4f661f68812ddbfa0adb9d0f297dbdd163d499 Mon Sep 17 00:00:00 2001 From: Wu Sheng Date: Sun, 22 Feb 2026 12:09:25 +0800 Subject: [PATCH 10/16] Fix HttpAlarmCallback thread leak; rename benchmark env to istio-cluster_oap-banyandb - Use a shared static HttpClient singleton instead of creating a new instance per alarm post() call, which leaked NIO selector threads. - Rename benchmark environment from cluster_oap-banyandb to istio-cluster_oap-banyandb to reflect its Istio-based setup. - Add changelog entries for both the fix and the benchmark framework. Co-Authored-By: Claude Opus 4.6 --- benchmarks/cases/thread-analysis/run.sh | 2 +- .../kind.yaml | 0 .../setup.sh | 4 ++-- .../traffic-gen.yaml | 0 .../values.yaml | 0 benchmarks/run.sh | 4 ++-- docs/en/changes/changes.md | 5 +++++ docs/en/guides/benchmark.md | 4 ++-- .../oap/server/core/alarm/HttpAlarmCallback.java | 10 ++++++---- 9 files changed, 18 insertions(+), 11 deletions(-) rename benchmarks/envs-setup/{cluster_oap-banyandb => istio-cluster_oap-banyandb}/kind.yaml (100%) rename benchmarks/envs-setup/{cluster_oap-banyandb => istio-cluster_oap-banyandb}/setup.sh (99%) rename benchmarks/envs-setup/{cluster_oap-banyandb => istio-cluster_oap-banyandb}/traffic-gen.yaml (100%) rename benchmarks/envs-setup/{cluster_oap-banyandb => istio-cluster_oap-banyandb}/values.yaml (100%) diff --git a/benchmarks/cases/thread-analysis/run.sh b/benchmarks/cases/thread-analysis/run.sh index ed85e35c8550..3076a9fc2844 100755 --- a/benchmarks/cases/thread-analysis/run.sh +++ b/benchmarks/cases/thread-analysis/run.sh @@ -23,7 +23,7 @@ # ./run.sh # # The env-context file is produced by an env-setup script (e.g., -# envs-setup/cluster_oap-banyandb/setup.sh). +# envs-setup/istio-cluster_oap-banyandb/setup.sh). set -euo pipefail diff --git a/benchmarks/envs-setup/cluster_oap-banyandb/kind.yaml b/benchmarks/envs-setup/istio-cluster_oap-banyandb/kind.yaml similarity index 100% rename from benchmarks/envs-setup/cluster_oap-banyandb/kind.yaml rename to benchmarks/envs-setup/istio-cluster_oap-banyandb/kind.yaml diff --git a/benchmarks/envs-setup/cluster_oap-banyandb/setup.sh b/benchmarks/envs-setup/istio-cluster_oap-banyandb/setup.sh similarity index 99% rename from benchmarks/envs-setup/cluster_oap-banyandb/setup.sh rename to benchmarks/envs-setup/istio-cluster_oap-banyandb/setup.sh index a97dce8295fc..0e67f9bc7ab3 100755 --- a/benchmarks/envs-setup/cluster_oap-banyandb/setup.sh +++ b/benchmarks/envs-setup/istio-cluster_oap-banyandb/setup.sh @@ -405,9 +405,9 @@ log "Environment is up. OAP at localhost:12800, UI at localhost:8080" ############################################################################# CONTEXT_FILE="$REPORT_DIR/env-context.sh" cat > "$CONTEXT_FILE" < Date: Sun, 22 Feb 2026 12:10:03 +0800 Subject: [PATCH 11/16] Move HttpAlarmCallback changelog entry to end of OAP Server section Co-Authored-By: Claude Opus 4.6 --- docs/en/changes/changes.md | 5 ++--- 1 file changed, 2 insertions(+), 3 deletions(-) diff --git a/docs/en/changes/changes.md b/docs/en/changes/changes.md index f7c9157b1d48..1de5ebb37571 100644 --- a/docs/en/changes/changes.md +++ b/docs/en/changes/changes.md @@ -39,9 +39,6 @@ #### OAP Server -* Fix `HttpAlarmCallback` creating a new `HttpClient` on every alarm `post()` call, leaking NIO selector threads. - Replace with a shared static singleton. - * KubernetesCoordinator: make self instance return real pod IP address instead of `127.0.0.1`. * Enhance the alarm kernel with recovered status notification capability * Fix BrowserWebVitalsPerfData `clsTime` to `cls` and make it double type. @@ -93,6 +90,8 @@ * PromQL service: fix time parse issue when using RFC3339 time format for querying. * Envoy metrics service receiver: support adapter listener metrics. * Envoy metrics service receiver: support config MAL rules files. +* Fix `HttpAlarmCallback` creating a new `HttpClient` on every alarm `post()` call, leaking NIO selector threads. + Replace with a shared static singleton. #### UI * Fix the missing icon in new native trace view. From 7b6fdc70c10dbb14d4f6aae3c6e36c1cfb55e5d6 Mon Sep 17 00:00:00 2001 From: Wu Sheng Date: Sun, 22 Feb 2026 21:01:00 +0800 Subject: [PATCH 12/16] Shared KubernetesClient singleton to fix thread leak and churn - Add SharedKubernetesClient enum singleton with KubernetesHttpClientFactory that uses virtual threads on JDK 25+ or a single fixed thread on JDK <25. - Replace all KubernetesClientBuilder().build() calls (9 sites across 7 files) with the shared instance to eliminate per-call thread churn. - Fix KubernetesCoordinator client leak (never closed, selector thread persisted). - Consolidate kubernetes-client dependencies in query-graphql-plugin and configuration-k8s-configmap to use library-kubernetes-support. - Fix benchmark health check to use curlimages/curl pod (OAP JRE 25 image does not include curl). Co-Authored-By: Claude Opus 4.6 --- .../istio-cluster_oap-banyandb/setup.sh | 5 +- .../kubernetes/KubernetesCoordinator.java | 6 +-- .../configuration-k8s-configmap/pom.xml | 9 ++-- .../ConfigurationConfigmapInformer.java | 10 +--- .../library-kubernetes-support/pom.xml | 5 ++ .../kubernetes/KubernetesEndpoints.java | 5 +- .../KubernetesHttpClientFactory.java | 54 +++++++++++++++++++ .../library/kubernetes/KubernetesPods.java | 33 +++++------- .../kubernetes/KubernetesServices.java | 19 +++---- .../kubernetes/SharedKubernetesClient.java | 54 +++++++++++++++++++ .../query-graphql-plugin/pom.xml | 9 ++-- .../graphql/resolver/OndemandLogQuery.java | 16 +++--- .../envoy/als/k8s/K8SServiceRegistry.java | 6 +-- 13 files changed, 164 insertions(+), 67 deletions(-) create mode 100644 oap-server/server-library/library-kubernetes-support/src/main/java/org/apache/skywalking/library/kubernetes/KubernetesHttpClientFactory.java create mode 100644 oap-server/server-library/library-kubernetes-support/src/main/java/org/apache/skywalking/library/kubernetes/SharedKubernetesClient.java diff --git a/benchmarks/envs-setup/istio-cluster_oap-banyandb/setup.sh b/benchmarks/envs-setup/istio-cluster_oap-banyandb/setup.sh index 0e67f9bc7ab3..773c6268b5c4 100755 --- a/benchmarks/envs-setup/istio-cluster_oap-banyandb/setup.sh +++ b/benchmarks/envs-setup/istio-cluster_oap-banyandb/setup.sh @@ -356,9 +356,12 @@ OAP_PODS_CHECK=($(kubectl -n "$NAMESPACE" get pods -l app=skywalking,component=o EXPECTED_NODES=${#OAP_PODS_CHECK[@]} CLUSTER_HEALTHY=true +CURL_IMAGE="curlimages/curl:latest" for pod in "${OAP_PODS_CHECK[@]}"; do log " Checking $pod..." - METRICS=$(kubectl -n "$NAMESPACE" exec "$pod" -c oap -- curl -s http://localhost:1234/metrics 2>/dev/null) + POD_IP=$(kubectl -n "$NAMESPACE" get pod "$pod" -o jsonpath='{.status.podIP}') + METRICS=$(kubectl -n "$NAMESPACE" run "health-check-${pod##*-}" --rm -i --restart=Never \ + --image="$CURL_IMAGE" -- curl -s "http://${POD_IP}:1234/metrics" 2>/dev/null) || METRICS="" REMOTE_OUT=$(echo "$METRICS" | grep '^remote_out_count{' || true) if [ -z "$REMOTE_OUT" ]; then diff --git a/oap-server/server-cluster-plugin/cluster-kubernetes-plugin/src/main/java/org/apache/skywalking/oap/server/cluster/plugin/kubernetes/KubernetesCoordinator.java b/oap-server/server-cluster-plugin/cluster-kubernetes-plugin/src/main/java/org/apache/skywalking/oap/server/cluster/plugin/kubernetes/KubernetesCoordinator.java index 16fbecfbc39c..e2d9ec6b2923 100644 --- a/oap-server/server-cluster-plugin/cluster-kubernetes-plugin/src/main/java/org/apache/skywalking/oap/server/cluster/plugin/kubernetes/KubernetesCoordinator.java +++ b/oap-server/server-cluster-plugin/cluster-kubernetes-plugin/src/main/java/org/apache/skywalking/oap/server/cluster/plugin/kubernetes/KubernetesCoordinator.java @@ -19,8 +19,8 @@ package org.apache.skywalking.oap.server.cluster.plugin.kubernetes; import com.linecorp.armeria.client.Endpoint; -import io.fabric8.kubernetes.client.KubernetesClientBuilder; import lombok.extern.slf4j.Slf4j; +import org.apache.skywalking.library.kubernetes.SharedKubernetesClient; import org.apache.skywalking.oap.server.core.CoreModule; import org.apache.skywalking.oap.server.core.cluster.ClusterCoordinator; import org.apache.skywalking.oap.server.core.cluster.RemoteInstance; @@ -72,8 +72,8 @@ private EndpointGroup createEndpointGroup() { if (port == -1) { port = manager.find(CoreModule.NAME).provider().getService(ConfigService.class).getGRPCPort(); } - final var kubernetesClient = new KubernetesClientBuilder().build(); - final var builder = KubernetesLabelSelectorEndpointGroup.builder(kubernetesClient); + final var builder = KubernetesLabelSelectorEndpointGroup.builder( + SharedKubernetesClient.INSTANCE.get()); if (StringUtil.isNotBlank(config.getNamespace())) { builder.namespace(config.getNamespace()); diff --git a/oap-server/server-configuration/configuration-k8s-configmap/pom.xml b/oap-server/server-configuration/configuration-k8s-configmap/pom.xml index d21cb3f827ee..8b0c3127ca6d 100644 --- a/oap-server/server-configuration/configuration-k8s-configmap/pom.xml +++ b/oap-server/server-configuration/configuration-k8s-configmap/pom.xml @@ -37,12 +37,9 @@ ${project.version} - io.fabric8 - kubernetes-client - - - io.fabric8 - kubernetes-httpclient-jdk + org.apache.skywalking + library-kubernetes-support + ${project.version} org.apache.skywalking diff --git a/oap-server/server-configuration/configuration-k8s-configmap/src/main/java/org/apache/skywalking/oap/server/configuration/configmap/ConfigurationConfigmapInformer.java b/oap-server/server-configuration/configuration-k8s-configmap/src/main/java/org/apache/skywalking/oap/server/configuration/configmap/ConfigurationConfigmapInformer.java index 11a7282b30fa..4dcc366da372 100644 --- a/oap-server/server-configuration/configuration-k8s-configmap/src/main/java/org/apache/skywalking/oap/server/configuration/configmap/ConfigurationConfigmapInformer.java +++ b/oap-server/server-configuration/configuration-k8s-configmap/src/main/java/org/apache/skywalking/oap/server/configuration/configmap/ConfigurationConfigmapInformer.java @@ -19,8 +19,8 @@ package org.apache.skywalking.oap.server.configuration.configmap; import io.fabric8.kubernetes.api.model.ConfigMap; -import io.fabric8.kubernetes.client.KubernetesClientBuilder; import io.fabric8.kubernetes.client.informers.cache.Lister; +import org.apache.skywalking.library.kubernetes.SharedKubernetesClient; import lombok.extern.slf4j.Slf4j; import java.util.HashMap; @@ -32,19 +32,13 @@ public class ConfigurationConfigmapInformer { private final Lister configMapLister; public ConfigurationConfigmapInformer(ConfigmapConfigurationSettings settings) { - final var client = new KubernetesClientBuilder().build(); - final var informer = client + final var informer = SharedKubernetesClient.INSTANCE.get() .configMaps() .inNamespace(settings.getNamespace()) .withLabelSelector(settings.getLabelSelector()) .inform(); configMapLister = new Lister<>(informer.getIndexer()); - - Runtime.getRuntime().addShutdownHook(new Thread(() -> { - informer.stop(); - client.close(); - })); } public Map configMapData() { diff --git a/oap-server/server-library/library-kubernetes-support/pom.xml b/oap-server/server-library/library-kubernetes-support/pom.xml index 39de0cf5a712..d36c15137427 100644 --- a/oap-server/server-library/library-kubernetes-support/pom.xml +++ b/oap-server/server-library/library-kubernetes-support/pom.xml @@ -28,6 +28,11 @@ library-kubernetes-support + + org.apache.skywalking + library-util + ${project.version} + io.fabric8 kubernetes-client diff --git a/oap-server/server-library/library-kubernetes-support/src/main/java/org/apache/skywalking/library/kubernetes/KubernetesEndpoints.java b/oap-server/server-library/library-kubernetes-support/src/main/java/org/apache/skywalking/library/kubernetes/KubernetesEndpoints.java index 3a00a64e75c4..88f2a9c2e812 100644 --- a/oap-server/server-library/library-kubernetes-support/src/main/java/org/apache/skywalking/library/kubernetes/KubernetesEndpoints.java +++ b/oap-server/server-library/library-kubernetes-support/src/main/java/org/apache/skywalking/library/kubernetes/KubernetesEndpoints.java @@ -23,7 +23,6 @@ import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; import io.fabric8.kubernetes.api.model.Endpoints; -import io.fabric8.kubernetes.client.KubernetesClientBuilder; import lombok.SneakyThrows; import org.slf4j.LoggerFactory; @@ -43,8 +42,8 @@ public enum KubernetesEndpoints { .expireAfterWrite(Duration.ofMinutes(3)); endpoints = cacheBuilder.build(CacheLoader.from(() -> { - try (final var kubernetesClient = new KubernetesClientBuilder().build()) { - return kubernetesClient + try { + return SharedKubernetesClient.INSTANCE.get() .endpoints() .inAnyNamespace() .list() diff --git a/oap-server/server-library/library-kubernetes-support/src/main/java/org/apache/skywalking/library/kubernetes/KubernetesHttpClientFactory.java b/oap-server/server-library/library-kubernetes-support/src/main/java/org/apache/skywalking/library/kubernetes/KubernetesHttpClientFactory.java new file mode 100644 index 000000000000..71e5d34f3358 --- /dev/null +++ b/oap-server/server-library/library-kubernetes-support/src/main/java/org/apache/skywalking/library/kubernetes/KubernetesHttpClientFactory.java @@ -0,0 +1,54 @@ +/* + * 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.skywalking.library.kubernetes; + +import io.fabric8.kubernetes.client.jdkhttp.JdkHttpClientFactory; +import org.apache.skywalking.oap.server.library.util.VirtualThreads; + +import java.net.http.HttpClient; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; + +/** + * Custom {@link JdkHttpClientFactory} that configures the JDK {@link HttpClient} + * with a minimal executor to reduce thread usage. + * + *
      + *
    • JDK 25+: virtual-thread-per-task executor (0 platform threads)
    • + *
    • JDK < 25: single-thread fixed pool named {@code K8sClient-executor-0}
    • + *
    + * + *

    The JDK {@code HttpClient} always creates 1 internal {@code SelectorManager} + * thread regardless. This factory controls only the executor threads. + */ +final class KubernetesHttpClientFactory extends JdkHttpClientFactory { + + @Override + protected void additionalConfig(final HttpClient.Builder builder) { + final ExecutorService executor = VirtualThreads.createExecutor( + "K8sClient-executor", + () -> Executors.newFixedThreadPool(1, r -> { + final Thread t = new Thread(r, "K8sClient-executor-0"); + t.setDaemon(true); + return t; + }) + ); + builder.executor(executor); + } +} diff --git a/oap-server/server-library/library-kubernetes-support/src/main/java/org/apache/skywalking/library/kubernetes/KubernetesPods.java b/oap-server/server-library/library-kubernetes-support/src/main/java/org/apache/skywalking/library/kubernetes/KubernetesPods.java index 5cf1bcaada53..16c9cdea2882 100644 --- a/oap-server/server-library/library-kubernetes-support/src/main/java/org/apache/skywalking/library/kubernetes/KubernetesPods.java +++ b/oap-server/server-library/library-kubernetes-support/src/main/java/org/apache/skywalking/library/kubernetes/KubernetesPods.java @@ -23,7 +23,6 @@ import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; import io.fabric8.kubernetes.api.model.Pod; -import io.fabric8.kubernetes.client.KubernetesClientBuilder; import lombok.SneakyThrows; import java.time.Duration; @@ -44,30 +43,26 @@ public enum KubernetesPods { podByIP = cacheBuilder.build(new CacheLoader<>() { @Override public Optional load(String ip) { - try (final var kubernetesClient = new KubernetesClientBuilder().build()) { - return kubernetesClient - .pods() - .inAnyNamespace() - .withField("status.podIP", ip) - .list() - .getItems() - .stream() - .findFirst(); - } + return SharedKubernetesClient.INSTANCE.get() + .pods() + .inAnyNamespace() + .withField("status.podIP", ip) + .list() + .getItems() + .stream() + .findFirst(); } }); podByObjectID = cacheBuilder.build(new CacheLoader<>() { @Override public Optional load(ObjectID objectID) { - try (final var kubernetesClient = new KubernetesClientBuilder().build()) { - return Optional.ofNullable( - kubernetesClient - .pods() - .inNamespace(objectID.namespace()) - .withName(objectID.name()) - .get()); - } + return Optional.ofNullable( + SharedKubernetesClient.INSTANCE.get() + .pods() + .inNamespace(objectID.namespace()) + .withName(objectID.name()) + .get()); } }); } diff --git a/oap-server/server-library/library-kubernetes-support/src/main/java/org/apache/skywalking/library/kubernetes/KubernetesServices.java b/oap-server/server-library/library-kubernetes-support/src/main/java/org/apache/skywalking/library/kubernetes/KubernetesServices.java index a6bb8c02ab85..e116ce5b5a67 100644 --- a/oap-server/server-library/library-kubernetes-support/src/main/java/org/apache/skywalking/library/kubernetes/KubernetesServices.java +++ b/oap-server/server-library/library-kubernetes-support/src/main/java/org/apache/skywalking/library/kubernetes/KubernetesServices.java @@ -23,7 +23,6 @@ import com.google.common.cache.CacheLoader; import com.google.common.cache.LoadingCache; import io.fabric8.kubernetes.api.model.Service; -import io.fabric8.kubernetes.client.KubernetesClientBuilder; import lombok.SneakyThrows; import org.slf4j.LoggerFactory; @@ -45,8 +44,8 @@ public enum KubernetesServices { .expireAfterWrite(Duration.ofMinutes(3)); services = cacheBuilder.build(CacheLoader.from(() -> { - try (final var kubernetesClient = new KubernetesClientBuilder().build()) { - return kubernetesClient + try { + return SharedKubernetesClient.INSTANCE.get() .services() .inAnyNamespace() .list() @@ -60,14 +59,12 @@ public enum KubernetesServices { serviceByID = cacheBuilder.build(new CacheLoader<>() { @Override public Optional load(ObjectID id) { - try (final var kubernetesClient = new KubernetesClientBuilder().build()) { - return Optional.ofNullable( - kubernetesClient - .services() - .inNamespace(id.namespace()) - .withName(id.name()) - .get()); - } + return Optional.ofNullable( + SharedKubernetesClient.INSTANCE.get() + .services() + .inNamespace(id.namespace()) + .withName(id.name()) + .get()); } }); } diff --git a/oap-server/server-library/library-kubernetes-support/src/main/java/org/apache/skywalking/library/kubernetes/SharedKubernetesClient.java b/oap-server/server-library/library-kubernetes-support/src/main/java/org/apache/skywalking/library/kubernetes/SharedKubernetesClient.java new file mode 100644 index 000000000000..95749a8cc0ac --- /dev/null +++ b/oap-server/server-library/library-kubernetes-support/src/main/java/org/apache/skywalking/library/kubernetes/SharedKubernetesClient.java @@ -0,0 +1,54 @@ +/* + * 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.skywalking.library.kubernetes; + +import io.fabric8.kubernetes.client.KubernetesClient; +import io.fabric8.kubernetes.client.KubernetesClientBuilder; + +/** + * Shared {@link KubernetesClient} singleton. All modules that need Kubernetes API + * access should use this instead of creating their own client instances. + * + *

    Each {@code KubernetesClient} spawns internal JDK {@code HttpClient} threads + * (NIO selector, executor pool). Sharing a single client eliminates thread churn + * from repeated client creation in Guava cache loaders. + * + *

    Thread footprint (per JDK version): + *

      + *
    • JDK 25+: 1 SelectorManager + virtual thread executor = ~1 platform thread
    • + *
    • JDK < 25: 1 SelectorManager + 1 fixed executor thread = 2 platform threads
    • + *
    + */ +public enum SharedKubernetesClient { + INSTANCE; + + private final KubernetesClient client; + + SharedKubernetesClient() { + client = new KubernetesClientBuilder() + .withHttpClientFactory(new KubernetesHttpClientFactory()) + .build(); + Runtime.getRuntime().addShutdownHook( + new Thread(client::close, "K8sClient-shutdown")); + } + + public KubernetesClient get() { + return client; + } +} diff --git a/oap-server/server-query-plugin/query-graphql-plugin/pom.xml b/oap-server/server-query-plugin/query-graphql-plugin/pom.xml index 18b98fb329e7..82ed6c794949 100644 --- a/oap-server/server-query-plugin/query-graphql-plugin/pom.xml +++ b/oap-server/server-query-plugin/query-graphql-plugin/pom.xml @@ -66,12 +66,9 @@
    - io.fabric8 - kubernetes-client - - - io.fabric8 - kubernetes-httpclient-jdk + org.apache.skywalking + library-kubernetes-support + ${project.version}
    diff --git a/oap-server/server-query-plugin/query-graphql-plugin/src/main/java/org/apache/skywalking/oap/query/graphql/resolver/OndemandLogQuery.java b/oap-server/server-query-plugin/query-graphql-plugin/src/main/java/org/apache/skywalking/oap/query/graphql/resolver/OndemandLogQuery.java index 0e89f17a092d..4db3fe4ecca6 100644 --- a/oap-server/server-query-plugin/query-graphql-plugin/src/main/java/org/apache/skywalking/oap/query/graphql/resolver/OndemandLogQuery.java +++ b/oap-server/server-query-plugin/query-graphql-plugin/src/main/java/org/apache/skywalking/oap/query/graphql/resolver/OndemandLogQuery.java @@ -26,8 +26,8 @@ import io.fabric8.kubernetes.api.model.ObjectMeta; import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.PodSpec; -import io.fabric8.kubernetes.client.KubernetesClientBuilder; import io.fabric8.kubernetes.client.KubernetesClientException; +import org.apache.skywalking.library.kubernetes.SharedKubernetesClient; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; import org.apache.skywalking.oap.query.graphql.type.InternalLog; @@ -100,11 +100,12 @@ protected Map convertInstancePropertiesToMap(final ServiceInstan public PodContainers listContainers( final String namespace, final String podName) { - try (final var client = new KubernetesClientBuilder().build()) { - if (Strings.isNullOrEmpty(namespace) || Strings.isNullOrEmpty(podName)) { - return new PodContainers() - .setErrorReason("namespace and podName can't be null or empty"); - } + if (Strings.isNullOrEmpty(namespace) || Strings.isNullOrEmpty(podName)) { + return new PodContainers() + .setErrorReason("namespace and podName can't be null or empty"); + } + try { + final var client = SharedKubernetesClient.INSTANCE.get(); final var pod = client.pods().inNamespace(namespace).withName(podName).get(); if (isNull(pod)) { return new PodContainers().setErrorReason("No pod can be found"); @@ -138,7 +139,8 @@ public Logs ondemandPodLogs( if (Strings.isNullOrEmpty(namespace) || Strings.isNullOrEmpty(podName)) { return new Logs().setErrorReason("namespace and podName can't be null or empty"); } - try (final var client = new KubernetesClientBuilder().build()) { + try { + final var client = SharedKubernetesClient.INSTANCE.get(); final Pod pod = client.pods().inNamespace(namespace).withName(podName).get(); final ObjectMeta podMetadata = pod.getMetadata(); if (isNull(podMetadata)) { diff --git a/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/envoy/als/k8s/K8SServiceRegistry.java b/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/envoy/als/k8s/K8SServiceRegistry.java index 68ac0b5131fe..0ab1a2c8ffa9 100644 --- a/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/envoy/als/k8s/K8SServiceRegistry.java +++ b/oap-server/server-receiver-plugin/envoy-metrics-receiver-plugin/src/main/java/org/apache/skywalking/oap/server/receiver/envoy/als/k8s/K8SServiceRegistry.java @@ -28,13 +28,13 @@ import io.fabric8.kubernetes.api.model.ObjectMeta; import io.fabric8.kubernetes.api.model.Pod; import io.fabric8.kubernetes.api.model.Service; -import io.fabric8.kubernetes.client.KubernetesClientBuilder; import lombok.SneakyThrows; import lombok.extern.slf4j.Slf4j; import org.apache.skywalking.library.kubernetes.KubernetesEndpoints; import org.apache.skywalking.library.kubernetes.KubernetesPods; import org.apache.skywalking.library.kubernetes.KubernetesServices; import org.apache.skywalking.library.kubernetes.ObjectID; +import org.apache.skywalking.library.kubernetes.SharedKubernetesClient; import org.apache.skywalking.oap.server.library.util.StringUtil; import org.apache.skywalking.oap.server.receiver.envoy.EnvoyMetricReceiverConfig; import org.apache.skywalking.oap.server.receiver.envoy.als.ServiceMetaInfo; @@ -71,8 +71,8 @@ public K8SServiceRegistry(final EnvoyMetricReceiverConfig config) { .expireAfterWrite(Duration.ofMinutes(3)); nodeIPs = cacheBuilder.build(CacheLoader.from(() -> { - try (final var kubernetesClient = new KubernetesClientBuilder().build()) { - return kubernetesClient + try { + return SharedKubernetesClient.INSTANCE.get() .nodes() .list() .getItems() From 40c515c618001cf09a5a37bfcd54daf8c8618c11 Mon Sep 17 00:00:00 2001 From: Wu Sheng Date: Sun, 22 Feb 2026 21:06:16 +0800 Subject: [PATCH 13/16] Add SharedKubernetesClient changelog entry --- docs/en/changes/changes.md | 4 ++++ 1 file changed, 4 insertions(+) diff --git a/docs/en/changes/changes.md b/docs/en/changes/changes.md index 47895227e6f5..24d283de5f09 100644 --- a/docs/en/changes/changes.md +++ b/docs/en/changes/changes.md @@ -103,6 +103,10 @@ * Envoy metrics service receiver: support config MAL rules files. * Fix `HttpAlarmCallback` creating a new `HttpClient` on every alarm `post()` call, leaking NIO selector threads. Replace with a shared static singleton. +* Add `SharedKubernetesClient` singleton in `library-kubernetes-support` to replace 9 separate + `KubernetesClientBuilder().build()` calls across 7 files. Fixes `KubernetesCoordinator` client leak + (never closed, NIO selector thread persisted). Uses `KubernetesHttpClientFactory` with virtual threads + on JDK 25+ or a single fixed executor thread on JDK <25. #### UI * Fix the missing icon in new native trace view. From d10868c2b2ce0237a567f19aa7dbf22bfd1a4142 Mon Sep 17 00:00:00 2001 From: Wu Sheng Date: Sun, 22 Feb 2026 22:23:46 +0800 Subject: [PATCH 14/16] Reduce Armeria HTTP event loop threads and add thread benchmark comparison Reduce Armeria HTTP server event loop threads from default (cores*2=20) to max(5, cores/4) with a shared EventLoopGroup across all 4 HTTP servers. HTTP traffic (UI, PromQL, LogQL) is much lighter than gRPC. Add thread count benchmark comparison table (v10.3.0 vs v10.4.0) in changelog showing ~50% reduction from 150+ to ~72 OAP threads on a 10-core machine. Co-Authored-By: Claude Opus 4.6 --- docs/en/changes/changes.md | 20 +++++++++++++++++++ .../library/server/http/HTTPServer.java | 15 ++++++++++++++ 2 files changed, 35 insertions(+) diff --git a/docs/en/changes/changes.md b/docs/en/changes/changes.md index 24d283de5f09..93fa958b0ff8 100644 --- a/docs/en/changes/changes.md +++ b/docs/en/changes/changes.md @@ -47,6 +47,23 @@ On JDK 25+, all 11 thread pools above share ~9 carrier threads instead of up to 1,400+ platform threads. * Change default Docker base image to JDK 25 (`eclipse-temurin:25-jre`). JDK 11 kept as `-java11` variant. +* Thread count benchmark comparison — 2-node OAP cluster on JDK 25 with BanyanDB, Istio bookinfo traffic + (10-core machine, JVM-internal threads excluded): + + | Pool | v10.3.0 threads | v10.4.0 threads | Notes | + |---------------------------------------|--------------------|-----------------|---------------------------------------------| + | L1 Aggregation (OAL + MAL) | 26 (DataCarrier) | 10 (BatchQueue) | Unified OAL + MAL | + | L2 Persistence (OAL + MAL) | 3 (DataCarrier) | 4 (BatchQueue) | Unified OAL + MAL | + | TopN Persistence | 4 (DataCarrier) | 1 (BatchQueue) | | + | gRPC Remote Client | 1 (DataCarrier) | 1 (BatchQueue) | Per peer | + | Armeria HTTP event loop | 20 | 5 | `max(5, cores/4)` shared group | + | Armeria HTTP handler | on-demand platform | - | Virtual threads on JDK 25+ | + | gRPC event loop | 10 | 10 | Unchanged | + | gRPC handler | ~9 (platform) | - | Virtual threads on JDK 25+ | + | ForkJoinPool (Virtual Thread carrier) | 0 | ~10 | JDK 25+ virtual thread scheduler | + | HttpClient-SelectorManager | 4 | 2 | SharedKubernetesClient | + | Schedulers + others | ~24 | ~24 | Mostly unchanged | + | **Total (OAP threads)** | **150+** | **~72** | **~50% reduction, stable in high payload.** | #### OAP Server @@ -107,6 +124,9 @@ `KubernetesClientBuilder().build()` calls across 7 files. Fixes `KubernetesCoordinator` client leak (never closed, NIO selector thread persisted). Uses `KubernetesHttpClientFactory` with virtual threads on JDK 25+ or a single fixed executor thread on JDK <25. +* Reduce Armeria HTTP server event loop threads from Armeria's default (`availableProcessors * 2`) + to `max(5, availableProcessors / 4)`. All HTTP servers share a single worker group. HTTP traffic + (UI queries, PromQL, LogQL) is much lighter than gRPC and does not need as many I/O threads. #### UI * Fix the missing icon in new native trace view. diff --git a/oap-server/server-library/library-server/src/main/java/org/apache/skywalking/oap/server/library/server/http/HTTPServer.java b/oap-server/server-library/library-server/src/main/java/org/apache/skywalking/oap/server/library/server/http/HTTPServer.java index 1a352818fa27..dad4536fb55d 100644 --- a/oap-server/server-library/library-server/src/main/java/org/apache/skywalking/oap/server/library/server/http/HTTPServer.java +++ b/oap-server/server-library/library-server/src/main/java/org/apache/skywalking/oap/server/library/server/http/HTTPServer.java @@ -22,11 +22,13 @@ import com.linecorp.armeria.common.HttpMethod; import com.linecorp.armeria.common.HttpResponse; import com.linecorp.armeria.common.HttpStatus; +import com.linecorp.armeria.common.util.EventLoopGroups; import com.linecorp.armeria.server.Route; import com.linecorp.armeria.server.ServerBuilder; import com.linecorp.armeria.server.docs.DocService; import com.linecorp.armeria.server.encoding.DecodingService; import com.linecorp.armeria.server.logging.LoggingService; +import io.netty.channel.EventLoopGroup; import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; @@ -46,6 +48,18 @@ @Slf4j public class HTTPServer implements Server { + /** + * Shared event loop group for all HTTP servers. HTTP traffic (UI queries, + * PromQL, LogQL) is much lighter than gRPC, so we use a smaller pool + * instead of Armeria's default (availableProcessors * 2). + */ + private static final EventLoopGroup SHARED_WORKER_GROUP; + + static { + final int threads = Math.max(5, Runtime.getRuntime().availableProcessors() / 4); + SHARED_WORKER_GROUP = EventLoopGroups.newEventLoopGroup(threads); + } + private final HTTPServerConfig config; protected ServerBuilder sb; // Health check service, supports HEAD, GET method. @@ -64,6 +78,7 @@ public void setBlockingTaskName(final String blockingTaskName) { public void initialize() { sb = com.linecorp.armeria.server.Server .builder() + .workerGroup(SHARED_WORKER_GROUP, false) .baseContextPath(config.getContextPath()) .serviceUnder("/docs", DocService.builder().build()) .http1MaxHeaderSize(config.getMaxRequestHeaderSize()) From 9c5fa6793f9a61f603f88713ddd04098290c1845 Mon Sep 17 00:00:00 2001 From: Wu Sheng Date: Sun, 22 Feb 2026 22:28:48 +0800 Subject: [PATCH 15/16] Update. --- docs/en/changes/changes.md | 4 ++-- 1 file changed, 2 insertions(+), 2 deletions(-) diff --git a/docs/en/changes/changes.md b/docs/en/changes/changes.md index 93fa958b0ff8..2726ff0e9935 100644 --- a/docs/en/changes/changes.md +++ b/docs/en/changes/changes.md @@ -57,9 +57,9 @@ | TopN Persistence | 4 (DataCarrier) | 1 (BatchQueue) | | | gRPC Remote Client | 1 (DataCarrier) | 1 (BatchQueue) | Per peer | | Armeria HTTP event loop | 20 | 5 | `max(5, cores/4)` shared group | - | Armeria HTTP handler | on-demand platform | - | Virtual threads on JDK 25+ | + | Armeria HTTP handler | on-demand platform(increasing with payload) | - | Virtual threads on JDK 25+ | | gRPC event loop | 10 | 10 | Unchanged | - | gRPC handler | ~9 (platform) | - | Virtual threads on JDK 25+ | + | gRPC handler | on-demand platform(increasing with payload)| - | Virtual threads on JDK 25+ | | ForkJoinPool (Virtual Thread carrier) | 0 | ~10 | JDK 25+ virtual thread scheduler | | HttpClient-SelectorManager | 4 | 2 | SharedKubernetesClient | | Schedulers + others | ~24 | ~24 | Mostly unchanged | From 33f529bc16d5ae2669883583e569442938b7f597 Mon Sep 17 00:00:00 2001 From: Wu Sheng Date: Sun, 22 Feb 2026 23:54:40 +0800 Subject: [PATCH 16/16] Replace 6 scheduler threads with virtual threads on JDK 25+ Use VirtualThreads.createScheduledExecutor() for CacheUpdateTimer, DataTTLKeeper, HealthChecker, WatermarkWatcher, HierarchyAutoMatching, and FetchingConfigWatcherRegister. On JDK 25+ these run as virtual threads instead of dedicated platform threads. On JDK <25 they fall back to named daemon platform threads. Benchmark confirmed: 5 named scheduler threads disappear from thread dump on JRE 25, total OAP threads drop from ~72 to ~67-69. Co-Authored-By: Claude Opus 4.6 --- .../api/FetchingConfigWatcherRegister.java | 19 +++++++++++------ .../server/core/cache/CacheUpdateTimer.java | 15 +++++++++---- .../core/hierarchy/HierarchyService.java | 15 +++++++++---- .../core/storage/ttl/DataTTLKeeperTimer.java | 21 ++++++++++++------- .../core/watermark/WatermarkWatcher.java | 11 ++++++++-- .../provider/HealthCheckerProvider.java | 9 +++++++- 6 files changed, 66 insertions(+), 24 deletions(-) diff --git a/oap-server/server-configuration/configuration-api/src/main/java/org/apache/skywalking/oap/server/configuration/api/FetchingConfigWatcherRegister.java b/oap-server/server-configuration/configuration-api/src/main/java/org/apache/skywalking/oap/server/configuration/api/FetchingConfigWatcherRegister.java index 9fe403927a89..99cda5143ddf 100644 --- a/oap-server/server-configuration/configuration-api/src/main/java/org/apache/skywalking/oap/server/configuration/api/FetchingConfigWatcherRegister.java +++ b/oap-server/server-configuration/configuration-api/src/main/java/org/apache/skywalking/oap/server/configuration/api/FetchingConfigWatcherRegister.java @@ -27,6 +27,7 @@ import lombok.Getter; import lombok.extern.slf4j.Slf4j; import org.apache.skywalking.oap.server.library.util.RunnableWithExceptionProtection; +import org.apache.skywalking.oap.server.library.util.VirtualThreads; /** * Implement Config Watcher register using a periodic sync task. @@ -81,12 +82,18 @@ public void start() { log.info( "Current configurations after the bootstrap sync." + LINE_SEPARATOR + singleConfigChangeWatcherRegister.toString()); - Executors.newSingleThreadScheduledExecutor() - .scheduleAtFixedRate( - new RunnableWithExceptionProtection( - this::configSync, - t -> log.error("Sync config center error.", t) - ), 0, syncPeriod, TimeUnit.SECONDS); + VirtualThreads.createScheduledExecutor( + "ConfigWatcherSync", + () -> Executors.newSingleThreadScheduledExecutor(r -> { + final Thread t = new Thread(r, "ConfigWatcherSync"); + t.setDaemon(true); + return t; + })) + .scheduleAtFixedRate( + new RunnableWithExceptionProtection( + this::configSync, + t -> log.error("Sync config center error.", t) + ), 0, syncPeriod, TimeUnit.SECONDS); } void configSync() { diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/cache/CacheUpdateTimer.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/cache/CacheUpdateTimer.java index 06eb42e2d911..cb2d5fefc321 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/cache/CacheUpdateTimer.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/cache/CacheUpdateTimer.java @@ -22,6 +22,7 @@ import java.util.List; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import org.apache.skywalking.oap.server.library.util.VirtualThreads; import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; import org.apache.skywalking.oap.server.core.profiling.asyncprofiler.storage.AsyncProfilerTaskRecord; @@ -59,10 +60,16 @@ public void start(ModuleDefineHolder moduleDefineHolder, int ttl) { final long timeInterval = 10; - Executors.newSingleThreadScheduledExecutor(r -> new Thread(r, "CacheUpdateTimer")) - .scheduleAtFixedRate( - new RunnableWithExceptionProtection(() -> update(moduleDefineHolder), t -> log - .error("Cache update failure.", t)), 1, timeInterval, TimeUnit.SECONDS); + VirtualThreads.createScheduledExecutor( + "CacheUpdateTimer", + () -> Executors.newSingleThreadScheduledExecutor(r -> { + final Thread t = new Thread(r, "CacheUpdateTimer"); + t.setDaemon(true); + return t; + })) + .scheduleAtFixedRate( + new RunnableWithExceptionProtection(() -> update(moduleDefineHolder), t -> log + .error("Cache update failure.", t)), 1, timeInterval, TimeUnit.SECONDS); this.ttl = ttl; } diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/hierarchy/HierarchyService.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/hierarchy/HierarchyService.java index 783d5bc3fb85..02e4014229ae 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/hierarchy/HierarchyService.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/hierarchy/HierarchyService.java @@ -22,6 +22,7 @@ import java.util.Map; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import org.apache.skywalking.oap.server.library.util.VirtualThreads; import java.util.stream.Collectors; import lombok.extern.slf4j.Slf4j; import org.apache.skywalking.oap.server.core.CoreModule; @@ -136,10 +137,16 @@ public void startAutoMatchingServiceHierarchy() { if (!this.isEnableHierarchy) { return; } - Executors.newSingleThreadScheduledExecutor(r -> new Thread(r, "HierarchyAutoMatching")) - .scheduleWithFixedDelay( - new RunnableWithExceptionProtection(this::autoMatchingServiceRelation, t -> log.error( - "Scheduled auto matching service hierarchy from service traffic failure.", t)), 30, 20, TimeUnit.SECONDS); + VirtualThreads.createScheduledExecutor( + "HierarchyAutoMatching", + () -> Executors.newSingleThreadScheduledExecutor(r -> { + final Thread t = new Thread(r, "HierarchyAutoMatching"); + t.setDaemon(true); + return t; + })) + .scheduleWithFixedDelay( + new RunnableWithExceptionProtection(this::autoMatchingServiceRelation, t -> log.error( + "Scheduled auto matching service hierarchy from service traffic failure.", t)), 30, 20, TimeUnit.SECONDS); } private void autoMatchingServiceRelation(String upperServiceName, diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/ttl/DataTTLKeeperTimer.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/ttl/DataTTLKeeperTimer.java index 27dc5bef2f96..6c6388a3f123 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/ttl/DataTTLKeeperTimer.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/storage/ttl/DataTTLKeeperTimer.java @@ -28,6 +28,7 @@ import org.apache.skywalking.oap.server.core.storage.IHistoryDeleteDAO; import org.apache.skywalking.oap.server.core.storage.StorageModule; import org.apache.skywalking.oap.server.core.storage.model.IModelManager; +import org.apache.skywalking.oap.server.library.util.VirtualThreads; import org.apache.skywalking.oap.server.core.storage.model.Model; import org.apache.skywalking.oap.server.library.module.ModuleManager; import org.apache.skywalking.oap.server.library.util.CollectionUtils; @@ -60,13 +61,19 @@ public void start(ModuleManager moduleManager, CoreModuleConfig moduleConfig) { this.clusterNodesQuery = moduleManager.find(ClusterModule.NAME).provider().getService(ClusterNodesQuery.class); this.moduleConfig = moduleConfig; - Executors.newSingleThreadScheduledExecutor(r -> new Thread(r, "DataTTLKeeper")) - .scheduleAtFixedRate( - new RunnableWithExceptionProtection( - this::delete, - t -> log.error("Remove data in background failure.", t) - ), moduleConfig - .getDataKeeperExecutePeriod(), moduleConfig.getDataKeeperExecutePeriod(), TimeUnit.MINUTES); + VirtualThreads.createScheduledExecutor( + "DataTTLKeeper", + () -> Executors.newSingleThreadScheduledExecutor(r -> { + final Thread t = new Thread(r, "DataTTLKeeper"); + t.setDaemon(true); + return t; + })) + .scheduleAtFixedRate( + new RunnableWithExceptionProtection( + this::delete, + t -> log.error("Remove data in background failure.", t) + ), moduleConfig + .getDataKeeperExecutePeriod(), moduleConfig.getDataKeeperExecutePeriod(), TimeUnit.MINUTES); } /** diff --git a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/watermark/WatermarkWatcher.java b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/watermark/WatermarkWatcher.java index 8f8104f7bdf3..3841f2a429ca 100644 --- a/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/watermark/WatermarkWatcher.java +++ b/oap-server/server-core/src/main/java/org/apache/skywalking/oap/server/core/watermark/WatermarkWatcher.java @@ -24,6 +24,7 @@ import java.util.Map; import java.util.concurrent.Executors; import java.util.concurrent.TimeUnit; +import org.apache.skywalking.oap.server.library.util.VirtualThreads; import java.util.concurrent.locks.ReentrantLock; import lombok.RequiredArgsConstructor; import lombok.extern.slf4j.Slf4j; @@ -80,8 +81,14 @@ public void start(MetricsCollector so11yCollector) { .getService(MetricsCreator.class); this.addListener(WatermarkGRPCInterceptor.INSTANCE); - Executors.newSingleThreadScheduledExecutor(r -> new Thread(r, "WatermarkWatcher")) - .scheduleWithFixedDelay(this::watch, 0, 10, TimeUnit.SECONDS); + VirtualThreads.createScheduledExecutor( + "WatermarkWatcher", + () -> Executors.newSingleThreadScheduledExecutor(r -> { + final Thread t = new Thread(r, "WatermarkWatcher"); + t.setDaemon(true); + return t; + })) + .scheduleWithFixedDelay(this::watch, 0, 10, TimeUnit.SECONDS); } private void watch() { diff --git a/oap-server/server-health-checker/src/main/java/org/apache/skywalking/oap/server/health/checker/provider/HealthCheckerProvider.java b/oap-server/server-health-checker/src/main/java/org/apache/skywalking/oap/server/health/checker/provider/HealthCheckerProvider.java index 13239c2475b7..238179726e9e 100644 --- a/oap-server/server-health-checker/src/main/java/org/apache/skywalking/oap/server/health/checker/provider/HealthCheckerProvider.java +++ b/oap-server/server-health-checker/src/main/java/org/apache/skywalking/oap/server/health/checker/provider/HealthCheckerProvider.java @@ -25,6 +25,7 @@ import java.util.concurrent.Executors; import java.util.concurrent.ScheduledExecutorService; import java.util.concurrent.TimeUnit; +import org.apache.skywalking.oap.server.library.util.VirtualThreads; import java.util.concurrent.atomic.AtomicReference; import lombok.extern.slf4j.Slf4j; import org.apache.skywalking.oap.server.core.CoreModule; @@ -80,7 +81,13 @@ public void onInitialized(final HealthCheckerConfig initialized) { @Override public void prepare() throws ServiceNotProvidedException, ModuleStartException { score.set(-1); - ses = Executors.newSingleThreadScheduledExecutor(r -> new Thread(r, "HealthChecker")); + ses = VirtualThreads.createScheduledExecutor( + "HealthChecker", + () -> Executors.newSingleThreadScheduledExecutor(r -> { + final Thread t = new Thread(r, "HealthChecker"); + t.setDaemon(true); + return t; + })); healthQueryService = new HealthQueryService(score, details); this.registerServiceImplementation(HealthQueryService.class, healthQueryService); }