rondagostino commented on code in PR #13437:
URL: https://github.com/apache/kafka/pull/13437#discussion_r1172661731


##########
server-common/src/main/java/org/apache/kafka/server/immutable/ImmutableNavigableSet.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.server.immutable;
+
+import 
org.apache.kafka.server.immutable.pcollections.PCollectionsImmutableNavigableSet;
+
+import java.util.NavigableSet;
+
+/**
+ * A persistent Tree-based NavigableSet wrapper
+ * java.util.Set methods that mutate in-place will throw 
UnsupportedOperationException
+ *
+ * @param <E> the element type
+ */
+public interface ImmutableNavigableSet<E> extends ImmutableSet<E>, 
NavigableSet<E> {
+    /**
+     * @return a wrapped tree-based persistent navigable set that is empty
+     * @param <E> the element type
+     */
+    static <E extends Comparable<? super E>> ImmutableNavigableSet<E> empty() {
+        return PCollectionsImmutableNavigableSet.empty();
+    }
+
+    /**
+     * @param e the element
+     * @return a wrapped hash-based persistent set that is empty
+     * @param <E> the element type
+     */
+    static <E extends Comparable<? super E>> ImmutableNavigableSet<E> 
singleton(E e) {
+        return PCollectionsImmutableNavigableSet.singleton(e);
+    }
+
+    /**
+     * @param e the element
+     * @return a wrapped persistent sorted set that differs from this one in 
that the given element is added (if necessary)
+     */
+    ImmutableNavigableSet<E> added(E e);
+
+    /**
+     * @param e the element
+     * @return a wrapped persistent sorted set that differs from this one in 
that the given element is added (if necessary)

Review Comment:
   s/sorted/navigable/



##########
server-common/src/main/java/org/apache/kafka/server/immutable/ImmutableNavigableSet.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.server.immutable;
+
+import 
org.apache.kafka.server.immutable.pcollections.PCollectionsImmutableNavigableSet;
+
+import java.util.NavigableSet;
+
+/**
+ * A persistent Tree-based NavigableSet wrapper
+ * java.util.Set methods that mutate in-place will throw 
UnsupportedOperationException
+ *
+ * @param <E> the element type
+ */
+public interface ImmutableNavigableSet<E> extends ImmutableSet<E>, 
NavigableSet<E> {
+    /**
+     * @return a wrapped tree-based persistent navigable set that is empty
+     * @param <E> the element type
+     */
+    static <E extends Comparable<? super E>> ImmutableNavigableSet<E> empty() {
+        return PCollectionsImmutableNavigableSet.empty();
+    }
+
+    /**
+     * @param e the element
+     * @return a wrapped hash-based persistent set that is empty

Review Comment:
   Looks like copy/paste error -- not hash-based.



##########
server-common/src/test/java/org/apache/kafka/server/immutable/pcollections/PCollectionsImmutableNavigableSetTest.java:
##########
@@ -0,0 +1,422 @@
+/*
+ * 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.server.immutable.pcollections;
+
+import org.apache.kafka.server.immutable.DelegationChecker;
+import org.apache.kafka.server.immutable.ImmutableNavigableSet;
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.ValueSource;
+import org.pcollections.HashTreePSet;
+import org.pcollections.TreePSet;
+
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Iterator;
+import java.util.Random;
+import java.util.Spliterator;
+import java.util.function.Consumer;
+import java.util.function.Predicate;
+import java.util.stream.Stream;
+
+import static java.util.function.Function.identity;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.mockito.ArgumentMatchers.eq;
+import static org.mockito.Mockito.mock;
+
+@SuppressWarnings({"unchecked", "deprecation"})
+public class PCollectionsImmutableNavigableSetTest {
+
+    private static final TreePSet<Integer> SINGLETON_SET = 
TreePSet.singleton(new Random().nextInt());
+
+    private static final class PCollectionsTreeSetWrapperDelegationChecker<R> 
extends DelegationChecker<TreePSet<Object>, 
PCollectionsImmutableNavigableSet<Object>, R> {
+        public PCollectionsTreeSetWrapperDelegationChecker() {
+            super(mock(TreePSet.class), 
PCollectionsImmutableNavigableSet::new);
+        }
+
+        public TreePSet<Object> 
unwrap(PCollectionsImmutableNavigableSet<Object> wrapper) {
+            return wrapper.underlying();
+        }
+    }
+
+    @Test
+    public void testEmptySet() {
+        Assertions.assertEquals(HashTreePSet.empty(), 
((PCollectionsImmutableNavigableSet<?>) 
ImmutableNavigableSet.empty()).underlying());
+    }
+
+    @Test
+    public void testSingletonSet() {
+        Assertions.assertEquals(HashTreePSet.singleton(1), 
((PCollectionsImmutableNavigableSet<?>) 
ImmutableNavigableSet.singleton(1)).underlying());
+    }
+
+    @Test
+    public void testUnderlying() {
+        assertSame(SINGLETON_SET, new 
PCollectionsImmutableNavigableSet<>(SINGLETON_SET).underlying());
+    }
+
+    @Test
+    public void testDelegationOfAfterAdding() {

Review Comment:
   My oversight; I forgot to rename these methods in the test classes when we 
renamed the methods on the interfaces.
   
   Can you rename this `testDelegationOfAdded()` and rename 
`testDelegationOfAfterRemoving()` to `testDelegationOfRemoved()` -- both here 
and in `PCollectionsImmutableSetTest`?  And then do a similar rename in 
`PCollectionsImmutableMapTest`?



##########
server-common/src/test/java/org/apache/kafka/server/immutable/pcollections/PCollectionsImmutableMapTest.java:
##########
@@ -225,77 +225,59 @@ public void testDelegationOfReplaceAll() {
         new PCollectionsHashMapWrapperDelegationChecker<>()
             .defineMockConfigurationForVoidMethodInvocation(mock -> 
mock.replaceAll(eq(mockBiFunction)))
             .defineWrapperVoidMethodInvocation(wrapper -> 
wrapper.replaceAll(mockBiFunction))
-            .doVoidMethodDelegationCheck();
+            .doUnsupportedVoidFunctionDelegrationCheck();
     }
 
     @Test
     public void testDelegationOfPutIfAbsent() {
         new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForFunctionInvocation(mock -> 
mock.putIfAbsent(eq(this), eq(this)), this)
-            
.defineWrapperFunctionInvocationAndMockReturnValueTransformation(wrapper -> 
wrapper.putIfAbsent(this, this), identity())
-            .doFunctionDelegationCheck();
-    }
-
-    @ParameterizedTest
-    @ValueSource(booleans = {true, false})
-    public void testDelegationOfRemoveByKeyAndValue(boolean 
mockFunctionReturnValue) {
-        new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForFunctionInvocation(mock -> 
mock.remove(eq(this), eq(this)), mockFunctionReturnValue)
-            
.defineWrapperFunctionInvocationAndMockReturnValueTransformation(wrapper -> 
wrapper.remove(this, this), identity())
-            .doFunctionDelegationCheck();
-    }
-
-    @ParameterizedTest
-    @ValueSource(booleans = {true, false})
-    public void testDelegationOfReplaceWhenMappedToSpecificValue(boolean 
mockFunctionReturnValue) {
-        new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForFunctionInvocation(mock -> 
mock.replace(eq(this), eq(this), eq(this)), mockFunctionReturnValue)
-            
.defineWrapperFunctionInvocationAndMockReturnValueTransformation(wrapper -> 
wrapper.replace(this, this, this), identity())
-            .doFunctionDelegationCheck();
+            .defineMockConfigurationForUnsupportedFunction(mock -> 
mock.putIfAbsent(eq(this), eq(this)))
+            .defineWrapperUnsupportedFunctionInvocation(wrapper -> 
wrapper.putIfAbsent(this, this))
+            .doUnsupportedFunctionDelegrationCheck();
     }
 
     @Test
     public void testDelegationOfReplaceWhenMappedToAnyValue() {
         new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForFunctionInvocation(mock -> 
mock.replace(eq(this), eq(this)), this)
-            
.defineWrapperFunctionInvocationAndMockReturnValueTransformation(wrapper -> 
wrapper.replace(this, this), identity())
-            .doFunctionDelegationCheck();
+            .defineMockConfigurationForUnsupportedFunction(mock -> 
mock.replace(eq(this), eq(this)))
+            .defineWrapperUnsupportedFunctionInvocation(wrapper -> 
wrapper.replace(this, this))
+            .doUnsupportedFunctionDelegrationCheck();
     }
 
     @Test
     public void testDelegationOfComputeIfAbsent() {
         final Function<Object, Object> mockFunction = mock(Function.class);
         new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForVoidMethodInvocation(mock -> 
mock.computeIfAbsent(eq(this), eq(mockFunction)))
-            .defineWrapperVoidMethodInvocation(wrapper -> 
wrapper.computeIfAbsent(this, mockFunction))
-            .doVoidMethodDelegationCheck();
+            .defineMockConfigurationForUnsupportedFunction(mock -> 
mock.computeIfAbsent(eq(this), eq(mockFunction)))
+            .defineWrapperUnsupportedFunctionInvocation(wrapper -> 
wrapper.computeIfAbsent(this, mockFunction))
+            .doUnsupportedFunctionDelegrationCheck();
     }
 
     @Test
     public void testDelegationOfComputeIfPresent() {
         final BiFunction<Object, Object, Object> mockBiFunction = 
mock(BiFunction.class);
         new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForVoidMethodInvocation(mock -> 
mock.computeIfPresent(eq(this), eq(mockBiFunction)))
-            .defineWrapperVoidMethodInvocation(wrapper -> 
wrapper.computeIfPresent(this, mockBiFunction))
-            .doVoidMethodDelegationCheck();
+            .defineMockConfigurationForUnsupportedFunction(mock -> 
mock.computeIfPresent(eq(this), eq(mockBiFunction)))
+            .defineWrapperUnsupportedFunctionInvocation(wrapper -> 
wrapper.computeIfPresent(this, mockBiFunction))
+            .doUnsupportedFunctionDelegrationCheck();

Review Comment:
   Nice catch on fixing this; computeIfPresent() is a function rather than a 
void method.



##########
server-common/src/test/java/org/apache/kafka/server/immutable/pcollections/PCollectionsImmutableMapTest.java:
##########
@@ -225,77 +225,59 @@ public void testDelegationOfReplaceAll() {
         new PCollectionsHashMapWrapperDelegationChecker<>()
             .defineMockConfigurationForVoidMethodInvocation(mock -> 
mock.replaceAll(eq(mockBiFunction)))
             .defineWrapperVoidMethodInvocation(wrapper -> 
wrapper.replaceAll(mockBiFunction))
-            .doVoidMethodDelegationCheck();
+            .doUnsupportedVoidFunctionDelegrationCheck();
     }
 
     @Test
     public void testDelegationOfPutIfAbsent() {
         new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForFunctionInvocation(mock -> 
mock.putIfAbsent(eq(this), eq(this)), this)
-            
.defineWrapperFunctionInvocationAndMockReturnValueTransformation(wrapper -> 
wrapper.putIfAbsent(this, this), identity())
-            .doFunctionDelegationCheck();
-    }
-
-    @ParameterizedTest
-    @ValueSource(booleans = {true, false})
-    public void testDelegationOfRemoveByKeyAndValue(boolean 
mockFunctionReturnValue) {
-        new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForFunctionInvocation(mock -> 
mock.remove(eq(this), eq(this)), mockFunctionReturnValue)
-            
.defineWrapperFunctionInvocationAndMockReturnValueTransformation(wrapper -> 
wrapper.remove(this, this), identity())
-            .doFunctionDelegationCheck();
-    }
-
-    @ParameterizedTest
-    @ValueSource(booleans = {true, false})
-    public void testDelegationOfReplaceWhenMappedToSpecificValue(boolean 
mockFunctionReturnValue) {
-        new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForFunctionInvocation(mock -> 
mock.replace(eq(this), eq(this), eq(this)), mockFunctionReturnValue)
-            
.defineWrapperFunctionInvocationAndMockReturnValueTransformation(wrapper -> 
wrapper.replace(this, this, this), identity())
-            .doFunctionDelegationCheck();
+            .defineMockConfigurationForUnsupportedFunction(mock -> 
mock.putIfAbsent(eq(this), eq(this)))
+            .defineWrapperUnsupportedFunctionInvocation(wrapper -> 
wrapper.putIfAbsent(this, this))
+            .doUnsupportedFunctionDelegrationCheck();
     }
 
     @Test
     public void testDelegationOfReplaceWhenMappedToAnyValue() {
         new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForFunctionInvocation(mock -> 
mock.replace(eq(this), eq(this)), this)
-            
.defineWrapperFunctionInvocationAndMockReturnValueTransformation(wrapper -> 
wrapper.replace(this, this), identity())
-            .doFunctionDelegationCheck();
+            .defineMockConfigurationForUnsupportedFunction(mock -> 
mock.replace(eq(this), eq(this)))
+            .defineWrapperUnsupportedFunctionInvocation(wrapper -> 
wrapper.replace(this, this))
+            .doUnsupportedFunctionDelegrationCheck();
     }
 
     @Test
     public void testDelegationOfComputeIfAbsent() {
         final Function<Object, Object> mockFunction = mock(Function.class);
         new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForVoidMethodInvocation(mock -> 
mock.computeIfAbsent(eq(this), eq(mockFunction)))
-            .defineWrapperVoidMethodInvocation(wrapper -> 
wrapper.computeIfAbsent(this, mockFunction))
-            .doVoidMethodDelegationCheck();
+            .defineMockConfigurationForUnsupportedFunction(mock -> 
mock.computeIfAbsent(eq(this), eq(mockFunction)))
+            .defineWrapperUnsupportedFunctionInvocation(wrapper -> 
wrapper.computeIfAbsent(this, mockFunction))
+            .doUnsupportedFunctionDelegrationCheck();
     }
 
     @Test
     public void testDelegationOfComputeIfPresent() {
         final BiFunction<Object, Object, Object> mockBiFunction = 
mock(BiFunction.class);
         new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForVoidMethodInvocation(mock -> 
mock.computeIfPresent(eq(this), eq(mockBiFunction)))
-            .defineWrapperVoidMethodInvocation(wrapper -> 
wrapper.computeIfPresent(this, mockBiFunction))
-            .doVoidMethodDelegationCheck();
+            .defineMockConfigurationForUnsupportedFunction(mock -> 
mock.computeIfPresent(eq(this), eq(mockBiFunction)))
+            .defineWrapperUnsupportedFunctionInvocation(wrapper -> 
wrapper.computeIfPresent(this, mockBiFunction))
+            .doUnsupportedFunctionDelegrationCheck();
     }
 
     @Test
     public void testDelegationOfCompute() {
         final BiFunction<Object, Object, Object> mockBiFunction = 
mock(BiFunction.class);
         new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForVoidMethodInvocation(mock -> 
mock.compute(eq(this), eq(mockBiFunction)))
-            .defineWrapperVoidMethodInvocation(wrapper -> 
wrapper.compute(this, mockBiFunction))
-            .doVoidMethodDelegationCheck();
+            .defineMockConfigurationForUnsupportedFunction(mock -> 
mock.compute(eq(this), eq(mockBiFunction)))
+            .defineWrapperUnsupportedFunctionInvocation(wrapper -> 
wrapper.compute(this, mockBiFunction))
+            .doUnsupportedFunctionDelegrationCheck();

Review Comment:
   Nice catch on fixing this; compute() is a function rather than a void method.



##########
server-common/src/test/java/org/apache/kafka/server/immutable/pcollections/PCollectionsImmutableMapTest.java:
##########
@@ -225,77 +225,59 @@ public void testDelegationOfReplaceAll() {
         new PCollectionsHashMapWrapperDelegationChecker<>()
             .defineMockConfigurationForVoidMethodInvocation(mock -> 
mock.replaceAll(eq(mockBiFunction)))
             .defineWrapperVoidMethodInvocation(wrapper -> 
wrapper.replaceAll(mockBiFunction))
-            .doVoidMethodDelegationCheck();
+            .doUnsupportedVoidFunctionDelegrationCheck();
     }
 
     @Test
     public void testDelegationOfPutIfAbsent() {
         new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForFunctionInvocation(mock -> 
mock.putIfAbsent(eq(this), eq(this)), this)
-            
.defineWrapperFunctionInvocationAndMockReturnValueTransformation(wrapper -> 
wrapper.putIfAbsent(this, this), identity())
-            .doFunctionDelegationCheck();
-    }
-
-    @ParameterizedTest
-    @ValueSource(booleans = {true, false})
-    public void testDelegationOfRemoveByKeyAndValue(boolean 
mockFunctionReturnValue) {
-        new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForFunctionInvocation(mock -> 
mock.remove(eq(this), eq(this)), mockFunctionReturnValue)
-            
.defineWrapperFunctionInvocationAndMockReturnValueTransformation(wrapper -> 
wrapper.remove(this, this), identity())
-            .doFunctionDelegationCheck();
-    }
-
-    @ParameterizedTest
-    @ValueSource(booleans = {true, false})
-    public void testDelegationOfReplaceWhenMappedToSpecificValue(boolean 
mockFunctionReturnValue) {
-        new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForFunctionInvocation(mock -> 
mock.replace(eq(this), eq(this), eq(this)), mockFunctionReturnValue)
-            
.defineWrapperFunctionInvocationAndMockReturnValueTransformation(wrapper -> 
wrapper.replace(this, this, this), identity())
-            .doFunctionDelegationCheck();
+            .defineMockConfigurationForUnsupportedFunction(mock -> 
mock.putIfAbsent(eq(this), eq(this)))
+            .defineWrapperUnsupportedFunctionInvocation(wrapper -> 
wrapper.putIfAbsent(this, this))
+            .doUnsupportedFunctionDelegrationCheck();
     }
 
     @Test
     public void testDelegationOfReplaceWhenMappedToAnyValue() {
         new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForFunctionInvocation(mock -> 
mock.replace(eq(this), eq(this)), this)
-            
.defineWrapperFunctionInvocationAndMockReturnValueTransformation(wrapper -> 
wrapper.replace(this, this), identity())
-            .doFunctionDelegationCheck();
+            .defineMockConfigurationForUnsupportedFunction(mock -> 
mock.replace(eq(this), eq(this)))
+            .defineWrapperUnsupportedFunctionInvocation(wrapper -> 
wrapper.replace(this, this))
+            .doUnsupportedFunctionDelegrationCheck();
     }
 
     @Test
     public void testDelegationOfComputeIfAbsent() {
         final Function<Object, Object> mockFunction = mock(Function.class);
         new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForVoidMethodInvocation(mock -> 
mock.computeIfAbsent(eq(this), eq(mockFunction)))
-            .defineWrapperVoidMethodInvocation(wrapper -> 
wrapper.computeIfAbsent(this, mockFunction))
-            .doVoidMethodDelegationCheck();
+            .defineMockConfigurationForUnsupportedFunction(mock -> 
mock.computeIfAbsent(eq(this), eq(mockFunction)))
+            .defineWrapperUnsupportedFunctionInvocation(wrapper -> 
wrapper.computeIfAbsent(this, mockFunction))
+            .doUnsupportedFunctionDelegrationCheck();

Review Comment:
   Nice catch on fixing this; computeIfAbsent() is a function rather than a 
void method.



##########
server-common/src/test/java/org/apache/kafka/server/immutable/DelegationChecker.java:
##########
@@ -139,6 +163,16 @@ public void doFunctionDelegationCheck() {
         }
     }
 
+    public void doUnsupportedFunctionDelegrationCheck() {

Review Comment:
   `s/Delegration/Delegation/` (and same above in 
`doUnsupportedVoidFunctionDelegrationCheck()`).
   
   I think what is going on with these methods you added is that you are 
configuring the mock to invoke the real `TreePSet` method and then confirming 
that the invocation of the real method throws `UnsupportedOperationException`.  
The methods that (for example) 
`PCollectionsImmutableMapTest.testDelegationOfPut()` were previously invoking 
were not invoking the underlying `HashPMap` method but were instead configuring 
the mock to flag the fact that the delegation occurred as expected (by setting 
`persistentCollectionMethodInvokedCorrectly = true`).
   
   I think on the one hand we don't have to test that the real method throws 
the exception -- that's up to the library itself to test in its test suite.  
All we have to test is that we delegate correctly.  This is why I implemented 
it this way.  However, I think it is useful to explicitly call out the fact 
that these methods will throw that exception.  So I'm fine with this change in 
general.  We just have to correct the `Delegration` misspelling.  Thanks for 
the improvement.
   
   What do you think about adjusting the test method names where we expect the 
method to throw the exception?  So a test method might be 
`testDelegationOfUnsupportedOperationAdd()`?  I think this might be a good 
change.
   



##########
server-common/src/main/java/org/apache/kafka/server/immutable/pcollections/PCollectionsImmutableNavigableSet.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.server.immutable.pcollections;
+
+import org.apache.kafka.server.immutable.ImmutableNavigableSet;
+import org.pcollections.TreePSet;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.Objects;
+import java.util.Spliterator;
+import java.util.function.Consumer;
+import java.util.function.Predicate;
+import java.util.stream.Stream;
+
+@SuppressWarnings("deprecation")
+public class PCollectionsImmutableNavigableSet<E> implements 
ImmutableNavigableSet<E> {
+    private final TreePSet<E> underlying;
+
+    /**
+     * @return a wrapped tree-based persistent navigable set that is empty
+     * @param <E> the element type
+     */
+    public static <E extends Comparable<? super E>> 
PCollectionsImmutableNavigableSet<E> empty() {
+        return new PCollectionsImmutableNavigableSet<>(TreePSet.<E>empty());
+    }
+
+    /**
+     * @param e the element
+     * @return a wrapped hash-based persistent set that is empty
+     * @param <E> the element type
+     */
+    public static <E extends Comparable<? super E>> 
PCollectionsImmutableNavigableSet<E> singleton(E e) {
+        return new PCollectionsImmutableNavigableSet<>(TreePSet.singleton(e));
+    }
+
+    public PCollectionsImmutableNavigableSet(TreePSet<E> underlying) {
+        this.underlying = underlying;
+    }
+
+    @Override
+    public PCollectionsImmutableNavigableSet<E> added(E e) {
+        return new PCollectionsImmutableNavigableSet<>(underlying.plus(e));
+    }
+
+    @Override
+    public PCollectionsImmutableNavigableSet<E> removed(E e) {
+        return new PCollectionsImmutableNavigableSet<>(underlying.minus(e));
+    }
+
+    @Override
+    public E lower(E e) {
+        return underlying.lower(e);
+    }
+
+    @Override
+    public E floor(E e) {
+        return underlying.floor(e);
+    }
+
+    @Override
+    public E ceiling(E e) {
+        return underlying.ceiling(e);
+    }
+
+    @Override
+    public E higher(E e) {
+        return underlying.higher(e);
+    }
+
+    @Override
+    public E pollFirst() {
+        // will throw UnsupportedOperationException
+        return underlying.pollFirst();
+    }
+
+    @Override
+    public E pollLast() {
+        // will throw UnsupportedOperationException
+        return underlying.pollLast();
+    }
+
+    @Override
+    public PCollectionsImmutableNavigableSet<E> descendingSet() {
+        return new 
PCollectionsImmutableNavigableSet<>(underlying.descendingSet());
+    }
+
+    @Override
+    public Iterator<E> descendingIterator() {
+        return underlying.descendingIterator();
+    }
+
+    @Override
+    public PCollectionsImmutableNavigableSet<E> subSet(E fromElement, boolean 
fromInclusive, E toElement, boolean toInclusive) {
+        return new 
PCollectionsImmutableNavigableSet<>(underlying.subSet(fromElement, 
fromInclusive, toElement, toInclusive));
+    }
+
+    @Override
+    public PCollectionsImmutableNavigableSet<E> headSet(E toElement, boolean 
inclusive) {
+        return new 
PCollectionsImmutableNavigableSet<>(underlying.headSet(toElement, inclusive));
+    }
+
+    @Override
+    public PCollectionsImmutableNavigableSet<E> tailSet(E fromElement, boolean 
inclusive) {
+        return new 
PCollectionsImmutableNavigableSet<>(underlying.tailSet(fromElement, inclusive));
+    }
+
+    @Override
+    public Comparator<? super E> comparator() {
+        return underlying.comparator();
+    }
+
+    @Override
+    public PCollectionsImmutableNavigableSet<E> subSet(E fromElement, E 
toElement) {
+        return new 
PCollectionsImmutableNavigableSet<>(underlying.subSet(fromElement, toElement));
+    }
+
+    @Override
+    public PCollectionsImmutableNavigableSet<E> headSet(E toElement) {
+        return new 
PCollectionsImmutableNavigableSet<>(underlying.headSet(toElement));
+    }
+
+    @Override
+    public PCollectionsImmutableNavigableSet<E> tailSet(E fromElement) {
+        return new 
PCollectionsImmutableNavigableSet<>(underlying.tailSet(fromElement));
+    }
+
+    @Override
+    public E first() {
+        return underlying.first();
+    }
+
+    @Override
+    public E last() {
+        return underlying.last();
+    }
+
+    @Override
+    public int size() {
+        return underlying.size();
+    }
+
+    @Override
+    public boolean isEmpty() {
+        return underlying.isEmpty();
+    }
+
+    @Override
+    public boolean contains(Object o) {
+        return underlying.contains(o);
+    }
+
+    @Override
+    public Iterator<E> iterator() {
+        return underlying.iterator();
+    }
+
+    @Override
+    public void forEach(Consumer<? super E> action) {
+        underlying.forEach(action);
+    }
+
+    @Override
+    public Object[] toArray() {
+        return underlying.toArray();
+    }
+
+    @Override
+    public <T> T[] toArray(T[] a) {
+        return underlying.toArray(a);
+    }
+
+    @Override
+    public boolean add(E e) {
+        // will throw UnsupportedOperationException
+        return underlying.add(e);
+    }
+
+    @Override
+    public boolean remove(Object o) {
+        // will throw UnsupportedOperationException
+        return underlying.remove(o);
+    }
+
+    @Override
+    public boolean containsAll(Collection<?> c) {
+        return underlying.containsAll(c);
+    }
+
+    @Override
+    public boolean addAll(Collection<? extends E> c) {
+        // will throw UnsupportedOperationException
+        return underlying.addAll(c);
+    }
+
+    @Override
+    public boolean retainAll(Collection<?> c) {
+        // will throw UnsupportedOperationException
+        return underlying.retainAll(c);
+    }
+
+    @Override
+    public boolean removeAll(Collection<?> c) {
+        // will throw UnsupportedOperationException
+        return underlying.removeAll(c);
+    }
+
+    @Override
+    public boolean removeIf(Predicate<? super E> filter) {
+        // will throw UnsupportedOperationException
+        return underlying.removeIf(filter);
+    }
+
+    @Override
+    public void clear() {
+        // will throw UnsupportedOperationException
+        underlying.clear();
+    }
+
+    @Override
+    public Spliterator<E> spliterator() {
+        return underlying().spliterator();
+    }
+
+    @Override
+    public Stream<E> stream() {
+        return underlying().stream();
+    }
+
+    @Override
+    public Stream<E> parallelStream() {
+        return underlying().parallelStream();
+    }
+
+    @Override
+    public boolean equals(Object o) {
+        if (this == o) return true;
+        if (o == null || getClass() != o.getClass()) return false;
+        PCollectionsImmutableNavigableSet<?> that = 
(PCollectionsImmutableNavigableSet<?>) o;
+        return Objects.equals(underlying(), that.underlying());
+    }
+
+    @Override
+    public int hashCode() {
+        return underlying().hashCode();
+    }
+
+    @Override
+    public String toString() {
+        return "PCollectionsImmutableNavigableSet{" +
+                "underlying=" + underlying() +
+                '}';
+    }
+    TreePSet<E> underlying() {

Review Comment:
   Add a separator line and the comment `// package-private for testing`
   



##########
server-common/src/test/java/org/apache/kafka/server/immutable/pcollections/PCollectionsImmutableMapTest.java:
##########
@@ -225,77 +225,59 @@ public void testDelegationOfReplaceAll() {
         new PCollectionsHashMapWrapperDelegationChecker<>()
             .defineMockConfigurationForVoidMethodInvocation(mock -> 
mock.replaceAll(eq(mockBiFunction)))
             .defineWrapperVoidMethodInvocation(wrapper -> 
wrapper.replaceAll(mockBiFunction))
-            .doVoidMethodDelegationCheck();
+            .doUnsupportedVoidFunctionDelegrationCheck();
     }
 
     @Test
     public void testDelegationOfPutIfAbsent() {
         new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForFunctionInvocation(mock -> 
mock.putIfAbsent(eq(this), eq(this)), this)
-            
.defineWrapperFunctionInvocationAndMockReturnValueTransformation(wrapper -> 
wrapper.putIfAbsent(this, this), identity())
-            .doFunctionDelegationCheck();
-    }
-
-    @ParameterizedTest
-    @ValueSource(booleans = {true, false})
-    public void testDelegationOfRemoveByKeyAndValue(boolean 
mockFunctionReturnValue) {
-        new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForFunctionInvocation(mock -> 
mock.remove(eq(this), eq(this)), mockFunctionReturnValue)
-            
.defineWrapperFunctionInvocationAndMockReturnValueTransformation(wrapper -> 
wrapper.remove(this, this), identity())
-            .doFunctionDelegationCheck();
-    }
-
-    @ParameterizedTest
-    @ValueSource(booleans = {true, false})
-    public void testDelegationOfReplaceWhenMappedToSpecificValue(boolean 
mockFunctionReturnValue) {
-        new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForFunctionInvocation(mock -> 
mock.replace(eq(this), eq(this), eq(this)), mockFunctionReturnValue)
-            
.defineWrapperFunctionInvocationAndMockReturnValueTransformation(wrapper -> 
wrapper.replace(this, this, this), identity())
-            .doFunctionDelegationCheck();

Review Comment:
   Looks like we lost these two test methods



##########
server-common/src/main/java/org/apache/kafka/server/immutable/ImmutableNavigableSet.java:
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.server.immutable;
+
+import 
org.apache.kafka.server.immutable.pcollections.PCollectionsImmutableNavigableSet;
+
+import java.util.NavigableSet;
+
+/**
+ * A persistent Tree-based NavigableSet wrapper
+ * java.util.Set methods that mutate in-place will throw 
UnsupportedOperationException
+ *
+ * @param <E> the element type
+ */
+public interface ImmutableNavigableSet<E> extends ImmutableSet<E>, 
NavigableSet<E> {
+    /**
+     * @return a wrapped tree-based persistent navigable set that is empty
+     * @param <E> the element type
+     */
+    static <E extends Comparable<? super E>> ImmutableNavigableSet<E> empty() {
+        return PCollectionsImmutableNavigableSet.empty();
+    }
+
+    /**
+     * @param e the element
+     * @return a wrapped hash-based persistent set that is empty
+     * @param <E> the element type
+     */
+    static <E extends Comparable<? super E>> ImmutableNavigableSet<E> 
singleton(E e) {
+        return PCollectionsImmutableNavigableSet.singleton(e);
+    }
+
+    /**
+     * @param e the element
+     * @return a wrapped persistent sorted set that differs from this one in 
that the given element is added (if necessary)

Review Comment:
   s/sorted/navigable/



##########
server-common/src/test/java/org/apache/kafka/server/immutable/pcollections/PCollectionsImmutableMapTest.java:
##########
@@ -225,77 +225,59 @@ public void testDelegationOfReplaceAll() {
         new PCollectionsHashMapWrapperDelegationChecker<>()
             .defineMockConfigurationForVoidMethodInvocation(mock -> 
mock.replaceAll(eq(mockBiFunction)))
             .defineWrapperVoidMethodInvocation(wrapper -> 
wrapper.replaceAll(mockBiFunction))
-            .doVoidMethodDelegationCheck();
+            .doUnsupportedVoidFunctionDelegrationCheck();
     }
 
     @Test
     public void testDelegationOfPutIfAbsent() {
         new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForFunctionInvocation(mock -> 
mock.putIfAbsent(eq(this), eq(this)), this)
-            
.defineWrapperFunctionInvocationAndMockReturnValueTransformation(wrapper -> 
wrapper.putIfAbsent(this, this), identity())
-            .doFunctionDelegationCheck();
-    }
-
-    @ParameterizedTest
-    @ValueSource(booleans = {true, false})
-    public void testDelegationOfRemoveByKeyAndValue(boolean 
mockFunctionReturnValue) {
-        new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForFunctionInvocation(mock -> 
mock.remove(eq(this), eq(this)), mockFunctionReturnValue)
-            
.defineWrapperFunctionInvocationAndMockReturnValueTransformation(wrapper -> 
wrapper.remove(this, this), identity())
-            .doFunctionDelegationCheck();
-    }
-
-    @ParameterizedTest
-    @ValueSource(booleans = {true, false})
-    public void testDelegationOfReplaceWhenMappedToSpecificValue(boolean 
mockFunctionReturnValue) {
-        new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForFunctionInvocation(mock -> 
mock.replace(eq(this), eq(this), eq(this)), mockFunctionReturnValue)
-            
.defineWrapperFunctionInvocationAndMockReturnValueTransformation(wrapper -> 
wrapper.replace(this, this, this), identity())
-            .doFunctionDelegationCheck();
+            .defineMockConfigurationForUnsupportedFunction(mock -> 
mock.putIfAbsent(eq(this), eq(this)))
+            .defineWrapperUnsupportedFunctionInvocation(wrapper -> 
wrapper.putIfAbsent(this, this))
+            .doUnsupportedFunctionDelegrationCheck();
     }
 
     @Test
     public void testDelegationOfReplaceWhenMappedToAnyValue() {
         new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForFunctionInvocation(mock -> 
mock.replace(eq(this), eq(this)), this)
-            
.defineWrapperFunctionInvocationAndMockReturnValueTransformation(wrapper -> 
wrapper.replace(this, this), identity())
-            .doFunctionDelegationCheck();
+            .defineMockConfigurationForUnsupportedFunction(mock -> 
mock.replace(eq(this), eq(this)))
+            .defineWrapperUnsupportedFunctionInvocation(wrapper -> 
wrapper.replace(this, this))
+            .doUnsupportedFunctionDelegrationCheck();
     }
 
     @Test
     public void testDelegationOfComputeIfAbsent() {
         final Function<Object, Object> mockFunction = mock(Function.class);
         new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForVoidMethodInvocation(mock -> 
mock.computeIfAbsent(eq(this), eq(mockFunction)))
-            .defineWrapperVoidMethodInvocation(wrapper -> 
wrapper.computeIfAbsent(this, mockFunction))
-            .doVoidMethodDelegationCheck();
+            .defineMockConfigurationForUnsupportedFunction(mock -> 
mock.computeIfAbsent(eq(this), eq(mockFunction)))
+            .defineWrapperUnsupportedFunctionInvocation(wrapper -> 
wrapper.computeIfAbsent(this, mockFunction))
+            .doUnsupportedFunctionDelegrationCheck();
     }
 
     @Test
     public void testDelegationOfComputeIfPresent() {
         final BiFunction<Object, Object, Object> mockBiFunction = 
mock(BiFunction.class);
         new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForVoidMethodInvocation(mock -> 
mock.computeIfPresent(eq(this), eq(mockBiFunction)))
-            .defineWrapperVoidMethodInvocation(wrapper -> 
wrapper.computeIfPresent(this, mockBiFunction))
-            .doVoidMethodDelegationCheck();
+            .defineMockConfigurationForUnsupportedFunction(mock -> 
mock.computeIfPresent(eq(this), eq(mockBiFunction)))
+            .defineWrapperUnsupportedFunctionInvocation(wrapper -> 
wrapper.computeIfPresent(this, mockBiFunction))
+            .doUnsupportedFunctionDelegrationCheck();
     }
 
     @Test
     public void testDelegationOfCompute() {
         final BiFunction<Object, Object, Object> mockBiFunction = 
mock(BiFunction.class);
         new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForVoidMethodInvocation(mock -> 
mock.compute(eq(this), eq(mockBiFunction)))
-            .defineWrapperVoidMethodInvocation(wrapper -> 
wrapper.compute(this, mockBiFunction))
-            .doVoidMethodDelegationCheck();
+            .defineMockConfigurationForUnsupportedFunction(mock -> 
mock.compute(eq(this), eq(mockBiFunction)))
+            .defineWrapperUnsupportedFunctionInvocation(wrapper -> 
wrapper.compute(this, mockBiFunction))
+            .doUnsupportedFunctionDelegrationCheck();
     }
 
     @Test
     public void testDelegationOfMerge() {
         final BiFunction<Object, Object, Object> mockBiFunction = 
mock(BiFunction.class);
         new PCollectionsHashMapWrapperDelegationChecker<>()
-            .defineMockConfigurationForVoidMethodInvocation(mock -> 
mock.merge(eq(this), eq(this), eq(mockBiFunction)))
-            .defineWrapperVoidMethodInvocation(wrapper -> wrapper.merge(this, 
this, mockBiFunction))
-            .doVoidMethodDelegationCheck();
+            .defineMockConfigurationForUnsupportedFunction(mock -> 
mock.merge(eq(this), eq(this), eq(mockBiFunction)))
+            .defineWrapperUnsupportedFunctionInvocation(wrapper -> 
wrapper.merge(this, this, mockBiFunction))
+            .doUnsupportedFunctionDelegrationCheck();

Review Comment:
   Nice catch on fixing this; merge() is a function rather than a void method.



##########
server-common/src/main/java/org/apache/kafka/server/immutable/pcollections/PCollectionsImmutableNavigableSet.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.server.immutable.pcollections;
+
+import org.apache.kafka.server.immutable.ImmutableNavigableSet;
+import org.pcollections.TreePSet;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.Objects;
+import java.util.Spliterator;
+import java.util.function.Consumer;
+import java.util.function.Predicate;
+import java.util.stream.Stream;
+
+@SuppressWarnings("deprecation")
+public class PCollectionsImmutableNavigableSet<E> implements 
ImmutableNavigableSet<E> {
+    private final TreePSet<E> underlying;
+
+    /**
+     * @return a wrapped tree-based persistent navigable set that is empty
+     * @param <E> the element type
+     */
+    public static <E extends Comparable<? super E>> 
PCollectionsImmutableNavigableSet<E> empty() {
+        return new PCollectionsImmutableNavigableSet<>(TreePSet.<E>empty());
+    }
+
+    /**
+     * @param e the element
+     * @return a wrapped hash-based persistent set that is empty
+     * @param <E> the element type
+     */
+    public static <E extends Comparable<? super E>> 
PCollectionsImmutableNavigableSet<E> singleton(E e) {
+        return new PCollectionsImmutableNavigableSet<>(TreePSet.singleton(e));
+    }
+
+    public PCollectionsImmutableNavigableSet(TreePSet<E> underlying) {
+        this.underlying = underlying;
+    }
+
+    @Override
+    public PCollectionsImmutableNavigableSet<E> added(E e) {
+        return new PCollectionsImmutableNavigableSet<>(underlying.plus(e));
+    }
+
+    @Override
+    public PCollectionsImmutableNavigableSet<E> removed(E e) {
+        return new PCollectionsImmutableNavigableSet<>(underlying.minus(e));
+    }
+
+    @Override
+    public E lower(E e) {
+        return underlying.lower(e);
+    }
+
+    @Override
+    public E floor(E e) {
+        return underlying.floor(e);
+    }
+
+    @Override
+    public E ceiling(E e) {
+        return underlying.ceiling(e);
+    }
+
+    @Override
+    public E higher(E e) {
+        return underlying.higher(e);
+    }
+
+    @Override
+    public E pollFirst() {
+        // will throw UnsupportedOperationException
+        return underlying.pollFirst();
+    }
+
+    @Override
+    public E pollLast() {
+        // will throw UnsupportedOperationException
+        return underlying.pollLast();
+    }
+
+    @Override
+    public PCollectionsImmutableNavigableSet<E> descendingSet() {
+        return new 
PCollectionsImmutableNavigableSet<>(underlying.descendingSet());
+    }
+
+    @Override
+    public Iterator<E> descendingIterator() {
+        return underlying.descendingIterator();
+    }
+
+    @Override
+    public PCollectionsImmutableNavigableSet<E> subSet(E fromElement, boolean 
fromInclusive, E toElement, boolean toInclusive) {
+        return new 
PCollectionsImmutableNavigableSet<>(underlying.subSet(fromElement, 
fromInclusive, toElement, toInclusive));
+    }
+
+    @Override
+    public PCollectionsImmutableNavigableSet<E> headSet(E toElement, boolean 
inclusive) {
+        return new 
PCollectionsImmutableNavigableSet<>(underlying.headSet(toElement, inclusive));
+    }
+
+    @Override
+    public PCollectionsImmutableNavigableSet<E> tailSet(E fromElement, boolean 
inclusive) {
+        return new 
PCollectionsImmutableNavigableSet<>(underlying.tailSet(fromElement, inclusive));
+    }
+
+    @Override
+    public Comparator<? super E> comparator() {
+        return underlying.comparator();
+    }
+
+    @Override
+    public PCollectionsImmutableNavigableSet<E> subSet(E fromElement, E 
toElement) {
+        return new 
PCollectionsImmutableNavigableSet<>(underlying.subSet(fromElement, toElement));
+    }
+
+    @Override
+    public PCollectionsImmutableNavigableSet<E> headSet(E toElement) {
+        return new 
PCollectionsImmutableNavigableSet<>(underlying.headSet(toElement));
+    }
+
+    @Override
+    public PCollectionsImmutableNavigableSet<E> tailSet(E fromElement) {
+        return new 
PCollectionsImmutableNavigableSet<>(underlying.tailSet(fromElement));
+    }
+
+    @Override
+    public E first() {
+        return underlying.first();
+    }
+
+    @Override
+    public E last() {
+        return underlying.last();
+    }
+
+    @Override
+    public int size() {
+        return underlying.size();
+    }
+
+    @Override
+    public boolean isEmpty() {
+        return underlying.isEmpty();
+    }
+
+    @Override
+    public boolean contains(Object o) {
+        return underlying.contains(o);
+    }
+
+    @Override
+    public Iterator<E> iterator() {
+        return underlying.iterator();
+    }
+
+    @Override
+    public void forEach(Consumer<? super E> action) {
+        underlying.forEach(action);
+    }
+
+    @Override
+    public Object[] toArray() {
+        return underlying.toArray();
+    }
+
+    @Override
+    public <T> T[] toArray(T[] a) {
+        return underlying.toArray(a);
+    }
+
+    @Override
+    public boolean add(E e) {
+        // will throw UnsupportedOperationException
+        return underlying.add(e);
+    }
+
+    @Override
+    public boolean remove(Object o) {
+        // will throw UnsupportedOperationException
+        return underlying.remove(o);
+    }
+
+    @Override
+    public boolean containsAll(Collection<?> c) {
+        return underlying.containsAll(c);
+    }
+
+    @Override
+    public boolean addAll(Collection<? extends E> c) {
+        // will throw UnsupportedOperationException
+        return underlying.addAll(c);
+    }
+
+    @Override
+    public boolean retainAll(Collection<?> c) {
+        // will throw UnsupportedOperationException
+        return underlying.retainAll(c);
+    }
+
+    @Override
+    public boolean removeAll(Collection<?> c) {
+        // will throw UnsupportedOperationException
+        return underlying.removeAll(c);
+    }
+
+    @Override
+    public boolean removeIf(Predicate<? super E> filter) {
+        // will throw UnsupportedOperationException
+        return underlying.removeIf(filter);
+    }
+
+    @Override
+    public void clear() {
+        // will throw UnsupportedOperationException
+        underlying.clear();

Review Comment:
   It looks like I had a mix of invoking `underlying.method()` vs. 
`underlying().method()` in `PCollections.ImmutableMap`.  Can you make 
everything consistently use `underlying().method()` both there and here?  
`PCollections.ImmutableSet` seems fine; it's just the map class and here that 
are now using an inconsistent style.  Let's make them all use the same style.  
Apologies for my oversight there, which seems to have crept in here.



##########
server-common/src/main/java/org/apache/kafka/server/immutable/pcollections/PCollectionsImmutableNavigableSet.java:
##########
@@ -0,0 +1,273 @@
+/*
+ * 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.server.immutable.pcollections;
+
+import org.apache.kafka.server.immutable.ImmutableNavigableSet;
+import org.pcollections.TreePSet;
+
+import java.util.Collection;
+import java.util.Comparator;
+import java.util.Iterator;
+import java.util.Objects;
+import java.util.Spliterator;
+import java.util.function.Consumer;
+import java.util.function.Predicate;
+import java.util.stream.Stream;
+
+@SuppressWarnings("deprecation")
+public class PCollectionsImmutableNavigableSet<E> implements 
ImmutableNavigableSet<E> {
+    private final TreePSet<E> underlying;
+
+    /**
+     * @return a wrapped tree-based persistent navigable set that is empty
+     * @param <E> the element type
+     */
+    public static <E extends Comparable<? super E>> 
PCollectionsImmutableNavigableSet<E> empty() {
+        return new PCollectionsImmutableNavigableSet<>(TreePSet.<E>empty());
+    }
+
+    /**
+     * @param e the element
+     * @return a wrapped hash-based persistent set that is empty

Review Comment:
   Looks like copy/paste error -- not hash-based.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: jira-unsubscr...@kafka.apache.org

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org

Reply via email to