|
72 | 72 | import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager.SeekableStreamExceptionEvent; |
73 | 73 | import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorStateManager.SeekableStreamState; |
74 | 74 | import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.AutoScalerConfig; |
| 75 | +import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.LagBasedAutoScalerConfig; |
75 | 76 | import org.apache.druid.java.util.common.DateTimes; |
76 | 77 | import org.apache.druid.java.util.common.ISE; |
77 | 78 | import org.apache.druid.java.util.common.Intervals; |
@@ -135,6 +136,8 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport |
135 | 136 | private static final StreamPartition<String> SHARD0_PARTITION = StreamPartition.of(STREAM, SHARD_ID); |
136 | 137 | private static final String EXCEPTION_MSG = "I had an exception"; |
137 | 138 | private static final Map<String, Object> METRIC_TAGS = ImmutableMap.of("k1", "v1", "k2", 20); |
| 139 | + private static final int DEFAULT_WORKER_THREADS = 2; |
| 140 | + private static final int DEFAULT_TASKS_PER_WORKER_THREAD = 4; |
138 | 141 |
|
139 | 142 | private TaskStorage taskStorage; |
140 | 143 | private TaskMaster taskMaster; |
@@ -177,7 +180,7 @@ public void setupTest() |
177 | 180 | EasyMock.expect(spec.getSupervisorStateManagerConfig()).andReturn(supervisorConfig).anyTimes(); |
178 | 181 |
|
179 | 182 | EasyMock.expect(spec.getDataSchema()).andReturn(getDataSchema()).anyTimes(); |
180 | | - EasyMock.expect(spec.getIoConfig()).andReturn(getIOConfig()).anyTimes(); |
| 183 | + EasyMock.expect(spec.getIoConfig()).andReturn(createSupervisorIOConfig()).anyTimes(); |
181 | 184 | EasyMock.expect(spec.getTuningConfig()).andReturn(getTuningConfig()).anyTimes(); |
182 | 185 | EasyMock.expect(spec.getEmitter()).andReturn(emitter).anyTimes(); |
183 | 186 | EasyMock.expect(spec.getContextValue(DruidMetrics.TAGS)).andReturn(METRIC_TAGS).anyTimes(); |
@@ -2563,6 +2566,83 @@ public LagStats computeLagStats() |
2563 | 2566 | EasyMock.verify(executorService, spec); |
2564 | 2567 | } |
2565 | 2568 |
|
| 2569 | + @Test |
| 2570 | + public void test_calculateWorkerThreads_shouldHonourWorkerConfig() |
| 2571 | + { |
| 2572 | + final int numWorkerThreads = 5; |
| 2573 | + final int taskCount = 1; |
| 2574 | + SeekableStreamSupervisorTuningConfig tuningConfig = createSupervisorTuningConfigWithWorkerThreads(numWorkerThreads); |
| 2575 | + SeekableStreamSupervisorIOConfig ioConfig = createSupervisorIOConfig(taskCount, null); |
| 2576 | + Assert.assertEquals(numWorkerThreads, SeekableStreamSupervisor.calculateWorkerThreads(tuningConfig, ioConfig)); |
| 2577 | + } |
| 2578 | + |
| 2579 | + @Test |
| 2580 | + public void test_calculateWorkerThreads_shouldUseDefaultWorkerThreads() |
| 2581 | + { |
| 2582 | + final int taskCount = 1; |
| 2583 | + SeekableStreamSupervisorTuningConfig tuningConfig = createSupervisorTuningConfig(); |
| 2584 | + SeekableStreamSupervisorIOConfig ioConfig = createSupervisorIOConfig(taskCount, null); |
| 2585 | + Assert.assertEquals( |
| 2586 | + DEFAULT_WORKER_THREADS, |
| 2587 | + SeekableStreamSupervisor.calculateWorkerThreads(tuningConfig, ioConfig) |
| 2588 | + ); |
| 2589 | + } |
| 2590 | + |
| 2591 | + @Test |
| 2592 | + public void test_calculateWorkerThreads_shouldUseMinimumWorkerThreadstWithTasks() |
| 2593 | + { |
| 2594 | + final int taskCount = 7; |
| 2595 | + SeekableStreamSupervisorTuningConfig tuningConfig = createSupervisorTuningConfig(); |
| 2596 | + SeekableStreamSupervisorIOConfig ioConfig = createSupervisorIOConfig(taskCount, null); |
| 2597 | + Assert.assertEquals( |
| 2598 | + DEFAULT_WORKER_THREADS, |
| 2599 | + SeekableStreamSupervisor.calculateWorkerThreads(tuningConfig, ioConfig) |
| 2600 | + ); |
| 2601 | + } |
| 2602 | + |
| 2603 | + @Test |
| 2604 | + public void test_calculateWorkerThreads_shouldUseFactorOfTaskCount() |
| 2605 | + { |
| 2606 | + final int taskCount = 18; |
| 2607 | + SeekableStreamSupervisorTuningConfig tuningConfig = createSupervisorTuningConfig(); |
| 2608 | + SeekableStreamSupervisorIOConfig ioConfig = createSupervisorIOConfig(taskCount, null); |
| 2609 | + Assert.assertEquals( |
| 2610 | + taskCount / DEFAULT_TASKS_PER_WORKER_THREAD, |
| 2611 | + SeekableStreamSupervisor.calculateWorkerThreads(tuningConfig, ioConfig) |
| 2612 | + ); |
| 2613 | + } |
| 2614 | + |
| 2615 | + @Test |
| 2616 | + public void test_calculateWorkerThreads_shouldUseAutoScalerConfig() |
| 2617 | + { |
| 2618 | + final int taskCountMax = 21; |
| 2619 | + final int taskCountMin = 5; |
| 2620 | + SeekableStreamSupervisorTuningConfig tuningConfig = createSupervisorTuningConfig(); |
| 2621 | + AutoScalerConfig autoScalerConfig = new LagBasedAutoScalerConfig( |
| 2622 | + null, |
| 2623 | + null, |
| 2624 | + null, |
| 2625 | + null, |
| 2626 | + null, |
| 2627 | + null, |
| 2628 | + null, |
| 2629 | + null, |
| 2630 | + taskCountMax, |
| 2631 | + null, |
| 2632 | + taskCountMin, |
| 2633 | + null, |
| 2634 | + null, |
| 2635 | + true, |
| 2636 | + null, |
| 2637 | + null |
| 2638 | + ); |
| 2639 | + SeekableStreamSupervisorIOConfig ioConfig = createSupervisorIOConfig(1, autoScalerConfig); |
| 2640 | + Assert.assertEquals( |
| 2641 | + taskCountMax / DEFAULT_TASKS_PER_WORKER_THREAD, |
| 2642 | + SeekableStreamSupervisor.calculateWorkerThreads(tuningConfig, ioConfig) |
| 2643 | + ); |
| 2644 | + } |
| 2645 | + |
2566 | 2646 | private void expectEmitterSupervisor(boolean suspended) |
2567 | 2647 | { |
2568 | 2648 | spec = createMock(SeekableStreamSupervisorSpec.class); |
@@ -2630,21 +2710,29 @@ private static DataSchema getDataSchema() |
2630 | 2710 | .build(); |
2631 | 2711 | } |
2632 | 2712 |
|
2633 | | - private static SeekableStreamSupervisorIOConfig getIOConfig() |
| 2713 | + private static SeekableStreamSupervisorIOConfig createSupervisorIOConfig() |
| 2714 | + { |
| 2715 | + return createSupervisorIOConfig(1, OBJECT_MAPPER.convertValue(getProperties(), AutoScalerConfig.class)); |
| 2716 | + } |
| 2717 | + |
| 2718 | + private static SeekableStreamSupervisorIOConfig createSupervisorIOConfig( |
| 2719 | + int taskCount, |
| 2720 | + @Nullable AutoScalerConfig autoScalerConfig |
| 2721 | + ) |
2634 | 2722 | { |
2635 | 2723 | return new SeekableStreamSupervisorIOConfig( |
2636 | 2724 | "stream", |
2637 | 2725 | new JsonInputFormat(new JSONPathSpec(true, ImmutableList.of()), ImmutableMap.of(), false, false, false), |
2638 | 2726 | 1, |
2639 | | - 1, |
| 2727 | + taskCount, |
2640 | 2728 | new Period("PT1H"), |
2641 | 2729 | new Period("P1D"), |
2642 | 2730 | new Period("PT30S"), |
2643 | 2731 | false, |
2644 | 2732 | new Period("PT30M"), |
2645 | 2733 | null, |
2646 | 2734 | null, |
2647 | | - OBJECT_MAPPER.convertValue(getProperties(), AutoScalerConfig.class), |
| 2735 | + autoScalerConfig, |
2648 | 2736 | LagAggregator.DEFAULT, |
2649 | 2737 | null, |
2650 | 2738 | null, |
@@ -2675,13 +2763,23 @@ private static Map<String, Object> getProperties() |
2675 | 2763 | } |
2676 | 2764 |
|
2677 | 2765 | private static SeekableStreamSupervisorTuningConfig getTuningConfig() |
| 2766 | + { |
| 2767 | + return createSupervisorTuningConfigWithWorkerThreads(1); |
| 2768 | + } |
| 2769 | + |
| 2770 | + private static SeekableStreamSupervisorTuningConfig createSupervisorTuningConfig() |
| 2771 | + { |
| 2772 | + return createSupervisorTuningConfigWithWorkerThreads(null); |
| 2773 | + } |
| 2774 | + |
| 2775 | + private static SeekableStreamSupervisorTuningConfig createSupervisorTuningConfigWithWorkerThreads(@Nullable Integer workerThreads) |
2678 | 2776 | { |
2679 | 2777 | return new SeekableStreamSupervisorTuningConfig() |
2680 | 2778 | { |
2681 | 2779 | @Override |
2682 | 2780 | public Integer getWorkerThreads() |
2683 | 2781 | { |
2684 | | - return 1; |
| 2782 | + return workerThreads; |
2685 | 2783 | } |
2686 | 2784 |
|
2687 | 2785 | @Override |
|
0 commit comments