|
63 | 63 | import org.slf4j.LoggerFactory;
|
64 | 64 |
|
65 | 65 | import java.util.ArrayList;
|
66 |
| -import java.util.Collections; |
67 | 66 | import java.util.HashMap;
|
68 | 67 | import java.util.List;
|
69 | 68 | import java.util.Map;
|
@@ -175,18 +174,13 @@ public RecordState validateTransition(RecordState newState) throws IllegalStateE
|
175 | 174 | }
|
176 | 175 |
|
177 | 176 | public static RecordState forId(byte id) {
|
178 |
| - switch (id) { |
179 |
| - case 0: |
180 |
| - return AVAILABLE; |
181 |
| - case 1: |
182 |
| - return ACQUIRED; |
183 |
| - case 2: |
184 |
| - return ACKNOWLEDGED; |
185 |
| - case 4: |
186 |
| - return ARCHIVED; |
187 |
| - default: |
188 |
| - throw new IllegalArgumentException("Unknown record state id: " + id); |
189 |
| - } |
| 177 | + return switch (id) { |
| 178 | + case 0 -> AVAILABLE; |
| 179 | + case 1 -> ACQUIRED; |
| 180 | + case 2 -> ACKNOWLEDGED; |
| 181 | + case 4 -> ARCHIVED; |
| 182 | + default -> throw new IllegalArgumentException("Unknown record state id: " + id); |
| 183 | + }; |
190 | 184 | }
|
191 | 185 |
|
192 | 186 | public byte id() {
|
@@ -440,8 +434,8 @@ public CompletableFuture<Void> maybeInitialize() {
|
440 | 434 | persister.readState(new ReadShareGroupStateParameters.Builder()
|
441 | 435 | .setGroupTopicPartitionData(new GroupTopicPartitionData.Builder<PartitionIdLeaderEpochData>()
|
442 | 436 | .setGroupId(this.groupId)
|
443 |
| - .setTopicsData(Collections.singletonList(new TopicData<>(topicIdPartition.topicId(), |
444 |
| - Collections.singletonList(PartitionFactory.newPartitionIdLeaderEpochData(topicIdPartition.partition(), leaderEpoch))))) |
| 437 | + .setTopicsData(List.of(new TopicData<>(topicIdPartition.topicId(), |
| 438 | + List.of(PartitionFactory.newPartitionIdLeaderEpochData(topicIdPartition.partition(), leaderEpoch))))) |
445 | 439 | .build())
|
446 | 440 | .build()
|
447 | 441 | ).whenComplete((result, exception) -> {
|
@@ -1675,17 +1669,12 @@ private Map<Long, RecordState> fetchRecordStateMapForAcknowledgementBatch(
|
1675 | 1669 | }
|
1676 | 1670 |
|
1677 | 1671 | private static RecordState fetchRecordState(byte acknowledgeType) {
|
1678 |
| - switch (acknowledgeType) { |
1679 |
| - case 1 /* ACCEPT */: |
1680 |
| - return RecordState.ACKNOWLEDGED; |
1681 |
| - case 2 /* RELEASE */: |
1682 |
| - return RecordState.AVAILABLE; |
1683 |
| - case 3 /* REJECT */: |
1684 |
| - case 0 /* GAP */: |
1685 |
| - return RecordState.ARCHIVED; |
1686 |
| - default: |
1687 |
| - throw new IllegalArgumentException("Invalid acknowledge type: " + acknowledgeType); |
1688 |
| - } |
| 1672 | + return switch (acknowledgeType) { |
| 1673 | + case 1 /* ACCEPT */ -> RecordState.ACKNOWLEDGED; |
| 1674 | + case 2 /* RELEASE */ -> RecordState.AVAILABLE; |
| 1675 | + case 3, 0 /* REJECT / GAP */ -> RecordState.ARCHIVED; |
| 1676 | + default -> throw new IllegalArgumentException("Invalid acknowledge type: " + acknowledgeType); |
| 1677 | + }; |
1689 | 1678 | }
|
1690 | 1679 |
|
1691 | 1680 | private NavigableMap<Long, InFlightBatch> fetchSubMapForAcknowledgementBatch(
|
@@ -2239,8 +2228,8 @@ CompletableFuture<Void> writeShareGroupState(List<PersisterStateBatch> stateBatc
|
2239 | 2228 | persister.writeState(new WriteShareGroupStateParameters.Builder()
|
2240 | 2229 | .setGroupTopicPartitionData(new GroupTopicPartitionData.Builder<PartitionStateBatchData>()
|
2241 | 2230 | .setGroupId(this.groupId)
|
2242 |
| - .setTopicsData(Collections.singletonList(new TopicData<>(topicIdPartition.topicId(), |
2243 |
| - Collections.singletonList(PartitionFactory.newPartitionStateBatchData( |
| 2231 | + .setTopicsData(List.of(new TopicData<>(topicIdPartition.topicId(), |
| 2232 | + List.of(PartitionFactory.newPartitionStateBatchData( |
2244 | 2233 | topicIdPartition.partition(), stateEpoch, startOffset, leaderEpoch, stateBatches))))
|
2245 | 2234 | ).build()).build())
|
2246 | 2235 | .whenComplete((result, exception) -> {
|
@@ -2284,22 +2273,20 @@ CompletableFuture<Void> writeShareGroupState(List<PersisterStateBatch> stateBatc
|
2284 | 2273 |
|
2285 | 2274 | private KafkaException fetchPersisterError(short errorCode, String errorMessage) {
|
2286 | 2275 | Errors error = Errors.forCode(errorCode);
|
2287 |
| - switch (error) { |
2288 |
| - case NOT_COORDINATOR: |
2289 |
| - case COORDINATOR_NOT_AVAILABLE: |
2290 |
| - case COORDINATOR_LOAD_IN_PROGRESS: |
2291 |
| - return new CoordinatorNotAvailableException(errorMessage); |
2292 |
| - case GROUP_ID_NOT_FOUND: |
2293 |
| - return new GroupIdNotFoundException(errorMessage); |
2294 |
| - case UNKNOWN_TOPIC_OR_PARTITION: |
2295 |
| - return new UnknownTopicOrPartitionException(errorMessage); |
2296 |
| - case FENCED_STATE_EPOCH: |
2297 |
| - return new FencedStateEpochException(errorMessage); |
2298 |
| - case FENCED_LEADER_EPOCH: |
2299 |
| - return new NotLeaderOrFollowerException(errorMessage); |
2300 |
| - default: |
2301 |
| - return new UnknownServerException(errorMessage); |
2302 |
| - } |
| 2276 | + return switch (error) { |
| 2277 | + case NOT_COORDINATOR, COORDINATOR_NOT_AVAILABLE, COORDINATOR_LOAD_IN_PROGRESS -> |
| 2278 | + new CoordinatorNotAvailableException(errorMessage); |
| 2279 | + case GROUP_ID_NOT_FOUND -> |
| 2280 | + new GroupIdNotFoundException(errorMessage); |
| 2281 | + case UNKNOWN_TOPIC_OR_PARTITION -> |
| 2282 | + new UnknownTopicOrPartitionException(errorMessage); |
| 2283 | + case FENCED_STATE_EPOCH -> |
| 2284 | + new FencedStateEpochException(errorMessage); |
| 2285 | + case FENCED_LEADER_EPOCH -> |
| 2286 | + new NotLeaderOrFollowerException(errorMessage); |
| 2287 | + default -> |
| 2288 | + new UnknownServerException(errorMessage); |
| 2289 | + }; |
2303 | 2290 | }
|
2304 | 2291 |
|
2305 | 2292 | // Visible for testing
|
@@ -2466,7 +2453,7 @@ private void releaseAcquisitionLockOnTimeoutForPerOffsetBatch(InFlightBatch inFl
|
2466 | 2453 | }
|
2467 | 2454 | }
|
2468 | 2455 |
|
2469 |
| - private long startOffsetDuringInitialization(long partitionDataStartOffset) throws Exception { |
| 2456 | + private long startOffsetDuringInitialization(long partitionDataStartOffset) { |
2470 | 2457 | // Set the state epoch and end offset from the persisted state.
|
2471 | 2458 | if (partitionDataStartOffset != PartitionFactory.UNINITIALIZED_START_OFFSET) {
|
2472 | 2459 | return partitionDataStartOffset;
|
|
0 commit comments