|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one or more |
| 3 | + * contributor license agreements. See the NOTICE file distributed with |
| 4 | + * this work for additional information regarding copyright ownership. |
| 5 | + * The ASF licenses this file to You under the Apache License, Version 2.0 |
| 6 | + * (the "License"); you may not use this file except in compliance with |
| 7 | + * the License. You may obtain a copy of the License at |
| 8 | + * |
| 9 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 10 | + * |
| 11 | + * Unless required by applicable law or agreed to in writing, software |
| 12 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 13 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 14 | + * See the License for the specific language governing permissions and |
| 15 | + * limitations under the License. |
| 16 | + */ |
| 17 | + |
| 18 | +package org.apache.seatunnel.connectors.seatunnel.fake.source; |
| 19 | + |
| 20 | +import org.apache.seatunnel.shade.com.typesafe.config.Config; |
| 21 | +import org.apache.seatunnel.shade.com.typesafe.config.ConfigFactory; |
| 22 | + |
| 23 | +import org.apache.seatunnel.api.common.metrics.AbstractMetricsContext; |
| 24 | +import org.apache.seatunnel.api.common.metrics.MetricsContext; |
| 25 | +import org.apache.seatunnel.api.configuration.ReadonlyConfig; |
| 26 | +import org.apache.seatunnel.api.event.Event; |
| 27 | +import org.apache.seatunnel.api.event.EventListener; |
| 28 | +import org.apache.seatunnel.api.source.SourceEvent; |
| 29 | +import org.apache.seatunnel.api.source.SourceSplitEnumerator; |
| 30 | +import org.apache.seatunnel.connectors.seatunnel.fake.config.MultipleTableFakeSourceConfig; |
| 31 | + |
| 32 | +import org.junit.jupiter.api.Assertions; |
| 33 | +import org.junit.jupiter.api.Test; |
| 34 | + |
| 35 | +import java.io.File; |
| 36 | +import java.net.URISyntaxException; |
| 37 | +import java.net.URL; |
| 38 | +import java.nio.file.Paths; |
| 39 | +import java.util.ArrayList; |
| 40 | +import java.util.Arrays; |
| 41 | +import java.util.Collections; |
| 42 | +import java.util.HashMap; |
| 43 | +import java.util.HashSet; |
| 44 | +import java.util.List; |
| 45 | +import java.util.Map; |
| 46 | +import java.util.Set; |
| 47 | +import java.util.stream.Collectors; |
| 48 | + |
| 49 | +class FakeSourceSplitEnumeratorTest { |
| 50 | + |
| 51 | + @Test |
| 52 | + void signalNoMoreSplitsAfterRestoreWhenNoPendingSplits() throws Exception { |
| 53 | + MultipleTableFakeSourceConfig sourceConfig = loadSingleTableFakeSourceConfig(); |
| 54 | + |
| 55 | + TestingEnumeratorContext firstContext = |
| 56 | + new TestingEnumeratorContext(2, new HashSet<>(Arrays.asList(0, 1))); |
| 57 | + FakeSourceSplitEnumerator firstRunEnumerator = |
| 58 | + new FakeSourceSplitEnumerator(firstContext, sourceConfig, Collections.emptySet()); |
| 59 | + firstRunEnumerator.run(); |
| 60 | + |
| 61 | + Set<FakeSourceSplit> assignedSplits = new HashSet<>(firstContext.getAllAssignedSplits()); |
| 62 | + Assertions.assertFalse(assignedSplits.isEmpty(), "Expected assigned splits in first run"); |
| 63 | + |
| 64 | + TestingEnumeratorContext restoredContext = |
| 65 | + new TestingEnumeratorContext(2, new HashSet<>(Arrays.asList(0, 1))); |
| 66 | + FakeSourceSplitEnumerator restoredEnumerator = |
| 67 | + new FakeSourceSplitEnumerator(restoredContext, sourceConfig, assignedSplits); |
| 68 | + restoredEnumerator.run(); |
| 69 | + |
| 70 | + Assertions.assertTrue( |
| 71 | + restoredContext.getAllAssignedSplits().isEmpty(), |
| 72 | + "Expected no split assignments on restore when all splits were already assigned"); |
| 73 | + Assertions.assertEquals( |
| 74 | + new HashSet<>(Arrays.asList(0, 1)), |
| 75 | + restoredContext.getNoMoreSplitsReaders(), |
| 76 | + "Expected signalNoMoreSplits for all registered readers"); |
| 77 | + } |
| 78 | + |
| 79 | + @Test |
| 80 | + void assignAndSignalOnLateRegisterReaderAfterDiscovery() throws Exception { |
| 81 | + MultipleTableFakeSourceConfig sourceConfig = loadSingleTableFakeSourceConfig(); |
| 82 | + |
| 83 | + TestingEnumeratorContext context = new TestingEnumeratorContext(2, new HashSet<>()); |
| 84 | + FakeSourceSplitEnumerator enumerator = |
| 85 | + new FakeSourceSplitEnumerator(context, sourceConfig, Collections.emptySet()); |
| 86 | + |
| 87 | + enumerator.run(); |
| 88 | + Assertions.assertTrue( |
| 89 | + context.getAllAssignedSplits().isEmpty(), |
| 90 | + "Expected no split assignments when no readers are registered during run()"); |
| 91 | + |
| 92 | + enumerator.registerReader(0); |
| 93 | + enumerator.registerReader(1); |
| 94 | + |
| 95 | + Assertions.assertFalse( |
| 96 | + context.getAllAssignedSplits().isEmpty(), |
| 97 | + "Expected split assignments after late reader registration"); |
| 98 | + Assertions.assertEquals( |
| 99 | + new HashSet<>(Arrays.asList(0, 1)), |
| 100 | + context.getNoMoreSplitsReaders(), |
| 101 | + "Expected signalNoMoreSplits for late registered readers"); |
| 102 | + } |
| 103 | + |
| 104 | + private static MultipleTableFakeSourceConfig loadSingleTableFakeSourceConfig() |
| 105 | + throws URISyntaxException { |
| 106 | + URL resource = FakeSourceSplitEnumeratorTest.class.getResource("/simple.schema.conf"); |
| 107 | + Config config = ConfigFactory.parseFile(new File(Paths.get(resource.toURI()).toString())); |
| 108 | + ReadonlyConfig readonlyConfig = ReadonlyConfig.fromConfig(config.getConfig("FakeSource")); |
| 109 | + return new MultipleTableFakeSourceConfig(readonlyConfig); |
| 110 | + } |
| 111 | + |
| 112 | + private static final class TestingEnumeratorContext |
| 113 | + implements SourceSplitEnumerator.Context<FakeSourceSplit> { |
| 114 | + private final int parallelism; |
| 115 | + private final Set<Integer> registeredReaders; |
| 116 | + private final Map<Integer, List<FakeSourceSplit>> assignedSplitsByReader = new HashMap<>(); |
| 117 | + private final Set<Integer> noMoreSplitsReaders = new HashSet<>(); |
| 118 | + private final MetricsContext metricsContext = new AbstractMetricsContext() {}; |
| 119 | + private final EventListener eventListener = |
| 120 | + new EventListener() { |
| 121 | + @Override |
| 122 | + public void onEvent(Event event) { |
| 123 | + // no-op |
| 124 | + } |
| 125 | + }; |
| 126 | + |
| 127 | + private TestingEnumeratorContext(int parallelism, Set<Integer> registeredReaders) { |
| 128 | + this.parallelism = parallelism; |
| 129 | + this.registeredReaders = registeredReaders; |
| 130 | + } |
| 131 | + |
| 132 | + @Override |
| 133 | + public int currentParallelism() { |
| 134 | + return parallelism; |
| 135 | + } |
| 136 | + |
| 137 | + @Override |
| 138 | + public Set<Integer> registeredReaders() { |
| 139 | + return registeredReaders; |
| 140 | + } |
| 141 | + |
| 142 | + @Override |
| 143 | + public void assignSplit(int subtaskId, List<FakeSourceSplit> splits) { |
| 144 | + assignedSplitsByReader |
| 145 | + .computeIfAbsent(subtaskId, ignored -> new ArrayList<>()) |
| 146 | + .addAll(splits); |
| 147 | + } |
| 148 | + |
| 149 | + @Override |
| 150 | + public void signalNoMoreSplits(int subtask) { |
| 151 | + noMoreSplitsReaders.add(subtask); |
| 152 | + } |
| 153 | + |
| 154 | + @Override |
| 155 | + public void sendEventToSourceReader(int subtaskId, SourceEvent event) { |
| 156 | + // no-op |
| 157 | + } |
| 158 | + |
| 159 | + @Override |
| 160 | + public MetricsContext getMetricsContext() { |
| 161 | + return metricsContext; |
| 162 | + } |
| 163 | + |
| 164 | + @Override |
| 165 | + public EventListener getEventListener() { |
| 166 | + return eventListener; |
| 167 | + } |
| 168 | + |
| 169 | + private List<FakeSourceSplit> getAllAssignedSplits() { |
| 170 | + return assignedSplitsByReader.values().stream() |
| 171 | + .flatMap(List::stream) |
| 172 | + .collect(Collectors.toList()); |
| 173 | + } |
| 174 | + |
| 175 | + private Set<Integer> getNoMoreSplitsReaders() { |
| 176 | + return noMoreSplitsReaders; |
| 177 | + } |
| 178 | + } |
| 179 | +} |
0 commit comments