Skip to content

Commit b110964

Browse files
CNDB-15558: ULID-based SSTable ID generation can fail with an NPE
ULID-based SSTable ID generation can fail with an NPE when generating a new ID. The root cause is that the underlying ULID generator can generate an empty Optional when the clock is moved backwards to before the previously generated ID or in certain rare overflow conditions when timestamp collides. If it's our first time through the generation loop, we prematurely exit with a null newVal. Top of the error stack: ``` java.lang.NullPointerException at org.apache.cassandra.utils.TimeUUID.approximateFromULID(TimeUUID.java:58) at org.apache.cassandra.io.sstable.ULIDBasedSSTableId.<init>(ULIDBasedSSTableId.java:52) at org.apache.cassandra.io.sstable.ULIDBasedSSTableId$Builder.lambda$generator$0(ULIDBasedSSTableId.java:129) ``` This can cause a flush to fail. Continue looping until newVal gets a value. The loop can spin until the corrected time catches up to the time of the most recently used ULID generation ID. This should be a short duration in a healthy cluster without large time corrections from sync. Tests are added in ULIDBasedSSTableIdGeneratorTest A package-protected constructor is introduced for ULIDBasedSSTableIdGeneratorTest.testGeneratorRetryOnEmptyOptional() Cassandra Applicability: upstream doesn't have ULIDBasedSSTableId (and won't because CASSANDRA-17048).
1 parent f358303 commit b110964

File tree

2 files changed

+235
-3
lines changed

2 files changed

+235
-3
lines changed

src/java/org/apache/cassandra/io/sstable/ULIDBasedSSTableId.java

Lines changed: 10 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -28,6 +28,7 @@
2828
import java.util.stream.Stream;
2929
import javax.annotation.Nonnull;
3030

31+
import com.google.common.annotations.VisibleForTesting;
3132
import com.google.common.base.Preconditions;
3233

3334
import de.huxhorn.sulky.ulid.ULID;
@@ -103,11 +104,17 @@ public static class Builder implements SSTableId.Builder<ULIDBasedSSTableId>
103104
{
104105
private static final Pattern PATTERN = Pattern.compile("[0-9a-z]{26}", Pattern.CASE_INSENSITIVE);
105106

106-
public static final Builder instance = new Builder();
107+
public static final Builder instance = new Builder(new ULID());
107108

108-
private static final ULID ulid = new ULID();
109+
private final ULID ulid;
109110
private static final AtomicReference<ULID.Value> prevRef = new AtomicReference<>();
110111

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

0 commit comments

Comments
 (0)