|
| 1 | +/* |
| 2 | + * Licensed to the Apache Software Foundation (ASF) under one |
| 3 | + * or more contributor license agreements. See the NOTICE file |
| 4 | + * distributed with this work for additional information |
| 5 | + * regarding copyright ownership. The ASF licenses this file |
| 6 | + * to you under the Apache License, Version 2.0 (the |
| 7 | + * "License"); you may not use this file except in compliance |
| 8 | + * with the License. You may obtain a copy of the License at |
| 9 | + * |
| 10 | + * http://www.apache.org/licenses/LICENSE-2.0 |
| 11 | + * |
| 12 | + * Unless required by applicable law or agreed to in writing, software |
| 13 | + * distributed under the License is distributed on an "AS IS" BASIS, |
| 14 | + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. |
| 15 | + * See the License for the specific language governing permissions and |
| 16 | + * limitations under the License. |
| 17 | + */ |
| 18 | + |
| 19 | +package org.apache.cassandra.io.sstable; |
| 20 | + |
| 21 | +import java.lang.reflect.Field; |
| 22 | +import java.util.HashSet; |
| 23 | +import java.util.Optional; |
| 24 | +import java.util.Set; |
| 25 | +import java.util.concurrent.BrokenBarrierException; |
| 26 | +import java.util.concurrent.ConcurrentHashMap; |
| 27 | +import java.util.concurrent.CountDownLatch; |
| 28 | +import java.util.concurrent.CyclicBarrier; |
| 29 | +import java.util.concurrent.ExecutorService; |
| 30 | +import java.util.concurrent.Executors; |
| 31 | +import java.util.concurrent.TimeUnit; |
| 32 | +import java.util.concurrent.atomic.AtomicReference; |
| 33 | +import java.util.function.Supplier; |
| 34 | +import java.util.stream.Stream; |
| 35 | + |
| 36 | +import org.junit.After; |
| 37 | +import org.junit.Before; |
| 38 | +import org.junit.Test; |
| 39 | +import org.mockito.Mockito; |
| 40 | + |
| 41 | +import de.huxhorn.sulky.ulid.ULID; |
| 42 | + |
| 43 | +import static org.assertj.core.api.Assertions.assertThat; |
| 44 | + |
| 45 | +/** |
| 46 | + * Tests for ULIDBasedSSTableId.Builder.generator() method focusing on the specific |
| 47 | + * retry logic and state transitions introduced in PR#2175. |
| 48 | + * |
| 49 | + * Note: Basic generator functionality (uniqueness, sorting, concurrent access) is already |
| 50 | + * tested in SSTableIdTest.testULIDBasedIdProperties() and generatorFuzzTest(). |
| 51 | + * |
| 52 | + * These tests specifically focus on: |
| 53 | + * - The retry loop when nextStrictlyMonotonicValue() returns empty Optional |
| 54 | + * - State transitions from null to initialized |
| 55 | + * - concurrency generation that may trigger retry logic |
| 56 | + */ |
| 57 | +public class ULIDBasedSSTableIdGeneratorTest |
| 58 | +{ |
| 59 | + private AtomicReference<ULID.Value> prevRef; |
| 60 | + private Field prevRefField; |
| 61 | + |
| 62 | + @Before |
| 63 | + public void setUp() throws Exception |
| 64 | + { |
| 65 | + prevRefField = ULIDBasedSSTableId.Builder.class.getDeclaredField("prevRef"); |
| 66 | + prevRefField.setAccessible(true); |
| 67 | + prevRef = (AtomicReference<ULID.Value>) prevRefField.get(null); |
| 68 | + prevRef.set(null); |
| 69 | + } |
| 70 | + |
| 71 | + /** |
| 72 | + * Test initial state: when prevRef is null, generator should call ulid.nextValue() |
| 73 | + * and successfully set the prevRef. |
| 74 | + */ |
| 75 | + @Test |
| 76 | + public void testGeneratorInitialState() throws Exception |
| 77 | + { |
| 78 | + Supplier<ULIDBasedSSTableId> generator = ULIDBasedSSTableId.Builder.instance.generator(Stream.empty()); |
| 79 | + |
| 80 | + ULIDBasedSSTableId id1 = generator.get(); |
| 81 | + assertThat(id1).isNotNull(); |
| 82 | + assertThat(prevRef.get()).isNotNull(); |
| 83 | + assertThat(prevRef.get()).isEqualTo(id1.ulid); |
| 84 | + } |
| 85 | + |
| 86 | + /** |
| 87 | + * Test monotonic progression: subsequent calls should generate strictly monotonic ULIDs. |
| 88 | + */ |
| 89 | + @Test |
| 90 | + public void testGeneratorMonotonicProgression() |
| 91 | + { |
| 92 | + Supplier<ULIDBasedSSTableId> generator = ULIDBasedSSTableId.Builder.instance.generator(Stream.empty()); |
| 93 | + |
| 94 | + ULIDBasedSSTableId id1 = generator.get(); |
| 95 | + ULIDBasedSSTableId id2 = generator.get(); |
| 96 | + ULIDBasedSSTableId id3 = generator.get(); |
| 97 | + |
| 98 | + assertThat(id1.compareTo(id2)).isLessThan(0); |
| 99 | + assertThat(id2.compareTo(id3)).isLessThan(0); |
| 100 | + assertThat(id1.compareTo(id3)).isLessThan(0); |
| 101 | + } |
| 102 | + |
| 103 | + @Test |
| 104 | + public void testGeneratorRetryOnEmptyOptional() throws Exception |
| 105 | + { |
| 106 | + // Use a real ULID just to manufacture realistic ULID.Value instances |
| 107 | + ULID realUlid = new ULID(); |
| 108 | + |
| 109 | + // Simulate an already existing previous value so that the generator |
| 110 | + // takes the nextStrictlyMonotonicValue(prevVal) branch. |
| 111 | + ULID.Value prevVal = realUlid.nextValue(); |
| 112 | + prevRef.set(prevVal); |
| 113 | + |
| 114 | + // This is the value we expect to be used after the retry. |
| 115 | + ULID.Value nextVal = realUlid.nextValue(); |
| 116 | + |
| 117 | + // Mock the ULID that will be injected into the Builder |
| 118 | + ULID mockUlid = Mockito.mock(ULID.class); |
| 119 | + |
| 120 | + // First call returns Optional.empty() → used to trigger the retry. |
| 121 | + // Second call returns nextVal → should be the value that generator uses. |
| 122 | + Mockito.when(mockUlid.nextStrictlyMonotonicValue(prevVal)) |
| 123 | + .thenReturn(Optional.empty(), Optional.of(nextVal)); |
| 124 | + |
| 125 | + // Create a Builder that uses our mock ULID instead of the static one |
| 126 | + ULIDBasedSSTableId.Builder builder = new ULIDBasedSSTableId.Builder(mockUlid); |
| 127 | + Supplier<ULIDBasedSSTableId> generator = builder.generator(Stream.empty()); |
| 128 | + |
| 129 | + // With the old loop condition (newVal != null && !CAS), |
| 130 | + // this would have exited after the Optional.empty() with newVal == null, |
| 131 | + // leading to a NPE in ULIDBasedSSTableId(newVal). |
| 132 | + ULIDBasedSSTableId id = generator.get(); |
| 133 | + |
| 134 | + // Now we expect a successful retry and a non-null ULID |
| 135 | + assertThat(id).isNotNull(); |
| 136 | + assertThat(id.ulid).isEqualTo(nextVal); |
| 137 | + assertThat(prevRef.get()).isEqualTo(nextVal); |
| 138 | + |
| 139 | + // Ensure we actually hit the retry path (two calls: empty, then present) |
| 140 | + Mockito.verify(mockUlid, Mockito.times(2)).nextStrictlyMonotonicValue(prevVal); |
| 141 | + } |
| 142 | + |
| 143 | + /** |
| 144 | + * Test the retry loop when nextStrictlyMonotonicValue |
| 145 | + * by generating many IDs rapidly in a tight loop, which may |
| 146 | + * trigger the retry logic when the timestamp hasn't advanced enough for |
| 147 | + * strictly monotonic values. |
| 148 | + */ |
| 149 | + @Test |
| 150 | + public void testGeneratorRetry() throws Exception |
| 151 | + { |
| 152 | + Supplier<ULIDBasedSSTableId> generator = ULIDBasedSSTableId.Builder.instance.generator(Stream.empty()); |
| 153 | + |
| 154 | + // Generate many IDs in rapid succession |
| 155 | + // This increases the likelihood of hitting the case where nextStrictlyMonotonicValue |
| 156 | + // returns empty because the timestamp hasn't advanced |
| 157 | + Set<ULIDBasedSSTableId> ids = new HashSet<>(); |
| 158 | + for (int i = 0; i < 1000; i++) |
| 159 | + { |
| 160 | + ULIDBasedSSTableId id = generator.get(); |
| 161 | + assertThat(id).isNotNull(); |
| 162 | + assertThat(ids.add(id)).isTrue(); // Ensure all IDs are unique |
| 163 | + } |
| 164 | + |
| 165 | + assertThat(ids).hasSize(1000); |
| 166 | + |
| 167 | + // Verify monotonic ordering |
| 168 | + ULIDBasedSSTableId[] sortedIds = ids.toArray(new ULIDBasedSSTableId[0]); |
| 169 | + java.util.Arrays.sort(sortedIds); |
| 170 | + for (int i = 1; i < sortedIds.length; i++) |
| 171 | + assertThat(sortedIds[i - 1].compareTo(sortedIds[i])).isLessThan(0); |
| 172 | + } |
| 173 | + |
| 174 | + /** |
| 175 | + * Test concurrent access: multiple threads generating IDs simultaneously |
| 176 | + * should produce unique, monotonically increasing IDs without duplicates. |
| 177 | + */ |
| 178 | + @Test |
| 179 | + public void testGeneratorConcurrentAccess() throws Exception |
| 180 | + { |
| 181 | + final int NUM_THREADS = 20; |
| 182 | + final int IDS_PER_THREAD = 50; |
| 183 | + |
| 184 | + ExecutorService executor = Executors.newFixedThreadPool(NUM_THREADS); |
| 185 | + CyclicBarrier barrier = new CyclicBarrier(NUM_THREADS); |
| 186 | + Set<ULIDBasedSSTableId> allIds = ConcurrentHashMap.newKeySet(); |
| 187 | + CountDownLatch latch = new CountDownLatch(NUM_THREADS); |
| 188 | + |
| 189 | + Supplier<ULIDBasedSSTableId> generator = ULIDBasedSSTableId.Builder.instance.generator(Stream.empty()); |
| 190 | + |
| 191 | + for (int i = 0; i < NUM_THREADS; i++) |
| 192 | + { |
| 193 | + executor.submit(() -> { |
| 194 | + try |
| 195 | + { |
| 196 | + barrier.await(); // Synchronize start |
| 197 | + for (int j = 0; j < IDS_PER_THREAD; j++) |
| 198 | + { |
| 199 | + ULIDBasedSSTableId id = generator.get(); |
| 200 | + assertThat(id).isNotNull(); |
| 201 | + allIds.add(id); |
| 202 | + } |
| 203 | + } |
| 204 | + catch (InterruptedException | BrokenBarrierException e) |
| 205 | + { |
| 206 | + throw new RuntimeException(e); |
| 207 | + } |
| 208 | + finally |
| 209 | + { |
| 210 | + latch.countDown(); |
| 211 | + } |
| 212 | + }); |
| 213 | + } |
| 214 | + |
| 215 | + assertThat(latch.await(30, TimeUnit.SECONDS)).isTrue(); |
| 216 | + executor.shutdown(); |
| 217 | + |
| 218 | + // Verify all IDs are unique |
| 219 | + assertThat(allIds).hasSize(NUM_THREADS * IDS_PER_THREAD); |
| 220 | + |
| 221 | + // Verify monotonic ordering |
| 222 | + ULIDBasedSSTableId[] sortedIds = allIds.toArray(new ULIDBasedSSTableId[0]); |
| 223 | + java.util.Arrays.sort(sortedIds); |
| 224 | + |
| 225 | + for (int i = 1; i < sortedIds.length; i++) |
| 226 | + assertThat(sortedIds[i - 1].compareTo(sortedIds[i])).isLessThan(0); |
| 227 | + } |
| 228 | + |
| 229 | +} |
0 commit comments