-
Notifications
You must be signed in to change notification settings - Fork 14k
Commit
This commit does not belong to any branch on this repository, and may belong to a fork outside of the repository.
KAFKA-9668: Iterating over KafkaStreams.getAllMetadata() results in C…
…oncurrentModificationException (#8233) `KafkaStreams.getAllMetadata()` returns `StreamsMetadataState.getAllMetadata()`. All the latter methods is `synchronized` it returns a reference to internal mutable state. Not only does this break encapsulation, but it means any thread iterating over the returned collection when the metadata gets rebuilt will encounter a `ConcurrentModificationException`. This change: * switches from clearing and rebuild `allMetadata` when `onChange` is called to building a new list and swapping this in. This is thread safe and has the benefit that the returned list is not empty during a rebuild: you either get the old or the new list. * removes synchronisation from `getAllMetadata` and `getLocalMetadata`. These are returning member variables. Synchronisation adds nothing. * changes `getAllMetadata` to wrap its return value in an unmodifiable wrapper to avoid breaking encapsulation. * changes the getters in `StreamsMetadata` to wrap their return values in unmodifiable wrapper to avoid breaking encapsulation. Co-authored-by: Andy Coates <[email protected]> Reviewers: Guozhang Wang <[email protected]>
- Loading branch information
1 parent
6b41993
commit ca90a84
Showing
4 changed files
with
106 additions
and
13 deletions.
There are no files selected for viewing
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
64 changes: 64 additions & 0 deletions
64
streams/src/test/java/org/apache/kafka/streams/state/StreamsMetadataTest.java
This file contains bidirectional Unicode text that may be interpreted or compiled differently than what appears below. To review, open the file in an editor that reveals hidden Unicode characters.
Learn more about bidirectional Unicode characters
Original file line number | Diff line number | Diff line change |
---|---|---|
@@ -0,0 +1,64 @@ | ||
/* | ||
* 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.kafka.streams.state; | ||
|
||
import org.apache.kafka.common.TopicPartition; | ||
import org.junit.Before; | ||
import org.junit.Test; | ||
|
||
import java.util.Collection; | ||
|
||
import static org.apache.kafka.common.utils.Utils.mkSet; | ||
import static org.junit.Assert.assertTrue; | ||
|
||
public class StreamsMetadataTest { | ||
|
||
private static final HostInfo HOST_INFO = new HostInfo("local", 12); | ||
private static final TopicPartition TP_0 = new TopicPartition("t", 0); | ||
private static final TopicPartition TP_1 = new TopicPartition("t", 1); | ||
|
||
private StreamsMetadata streamsMetadata; | ||
|
||
@Before | ||
public void setUp() { | ||
streamsMetadata = new StreamsMetadata( | ||
HOST_INFO, | ||
mkSet("store1", "store2"), | ||
mkSet(TP_0, TP_1), | ||
mkSet("store2"), | ||
mkSet(TP_1) | ||
); | ||
} | ||
|
||
@Test | ||
public void shouldNotAllowModificationOfInternalStateViaGetters() { | ||
assertTrue(isUnmodifiable(streamsMetadata.stateStoreNames())); | ||
assertTrue(isUnmodifiable(streamsMetadata.topicPartitions())); | ||
assertTrue(isUnmodifiable(streamsMetadata.standbyTopicPartitions())); | ||
assertTrue(isUnmodifiable(streamsMetadata.standbyStateStoreNames())); | ||
} | ||
|
||
private static boolean isUnmodifiable(final Collection<?> collection) { | ||
try { | ||
collection.clear(); | ||
return false; | ||
} catch (final UnsupportedOperationException e) { | ||
return true; | ||
} | ||
} | ||
} |