From f3ded4a293ff770a5c9a2c9b044e8e127c549a5e Mon Sep 17 00:00:00 2001 From: ohad Date: Fri, 13 Feb 2026 16:29:37 -0500 Subject: [PATCH 01/29] Initial commit of splitting queue entries --- .../provider/foundationdb/FDBRecordStore.java | 3 +- .../provider/foundationdb/SplitHelper.java | 56 +++++++++++----- .../provider/foundationdb/SplitKeyHelper.java | 35 ++++++++++ .../VersioningSplitKeyHelper.java | 64 +++++++++++++++++++ .../foundationdb/SplitHelperTest.java | 6 +- .../lucene/directory/PendingWriteQueue.java | 25 ++++---- 6 files changed, 157 insertions(+), 32 deletions(-) create mode 100644 fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyHelper.java create mode 100644 fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyHelper.java diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java index c49df95f6f..5affd81d5d 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java @@ -664,7 +664,8 @@ private FDBStoredRecord serializeAndSaveRecord(@Nonnull R final FDBRecordVersion splitVersion = useOldVersionFormat() ? null : version; final SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); preloadCache.invalidate(primaryKey); // clear out cache of older value if present - SplitHelper.saveWithSplit(context, recordsSubspace(), recordBuilder.getPrimaryKey(), serialized, splitVersion, metaData.isSplitLongRecords(), omitUnsplitRecordSuffix, true, oldSizeInfo, sizeInfo); + SplitHelper.saveWithSplit(context, recordsSubspace(), recordBuilder.getPrimaryKey(), serialized, splitVersion, metaData.isSplitLongRecords(), omitUnsplitRecordSuffix, + SplitHelper.DefaultSplitKeyHelper.INSTANCE, true, oldSizeInfo, sizeInfo); countKeysAndValues(FDBStoreTimer.Counts.SAVE_RECORD_KEY, FDBStoreTimer.Counts.SAVE_RECORD_KEY_BYTES, FDBStoreTimer.Counts.SAVE_RECORD_VALUE_BYTES, sizeInfo); recordBuilder.setSize(sizeInfo); diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java index 19e7151c30..c66bc87f4c 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java @@ -104,7 +104,7 @@ private SplitHelper() { */ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnull final Subspace subspace, @Nonnull final Tuple key, @Nonnull final byte[] serialized, @Nullable final FDBRecordVersion version) { - saveWithSplit(context, subspace, key, serialized, version, true, false, false, null, null); + saveWithSplit(context, subspace, key, serialized, version, true, false, DefaultSplitKeyHelper.INSTANCE, false, null, null); } /** @@ -124,6 +124,7 @@ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnu public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnull final Subspace subspace, @Nonnull final Tuple key, @Nonnull final byte[] serialized, @Nullable final FDBRecordVersion version, final boolean splitLongRecords, final boolean omitUnsplitSuffix, + final SplitKeyHelper splitKeyHelper, final boolean clearBasedOnPreviousSizeInfo, @Nullable final FDBStoredSizes previousSizeInfo, @Nullable SizeInfo sizeInfo) { if (omitUnsplitSuffix && version != null) { @@ -132,7 +133,6 @@ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnu .addLogInfo(LogMessageKeys.SUBSPACE, ByteArrayUtil2.loggable(subspace.pack())) .addLogInfo(LogMessageKeys.VERSION, version); } - final Transaction tr = context.ensureActive(); if (serialized.length > SplitHelper.SPLIT_RECORD_SIZE) { if (!splitLongRecords) { throw new RecordCoreException("Record is too long to be stored in a single value; consider split_long_records") @@ -140,9 +140,9 @@ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnu .addLogInfo(LogMessageKeys.SUBSPACE, ByteArrayUtil2.loggable(subspace.pack())) .addLogInfo(LogMessageKeys.VALUE_SIZE, serialized.length); } - writeSplitRecord(context, subspace, key, serialized, clearBasedOnPreviousSizeInfo, previousSizeInfo, sizeInfo); + writeSplitRecord(context, subspace, key, serialized, splitKeyHelper, clearBasedOnPreviousSizeInfo, previousSizeInfo, sizeInfo); } else { - if (splitLongRecords || previousSizeInfo == null || previousSizeInfo.isVersionedInline()) { + if (splitKeyHelper.clearBeforeWrite() && (splitLongRecords || previousSizeInfo == null || previousSizeInfo.isVersionedInline())) { clearPreviousSplitRecord(context, subspace, key, clearBasedOnPreviousSizeInfo, previousSizeInfo); } final Tuple recordKey; @@ -151,24 +151,27 @@ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnu } else { recordKey = key; } - final byte[] keyBytes = subspace.pack(recordKey); - tr.set(keyBytes, serialized); + final byte[] keyBytes = splitKeyHelper.packSplitKey(subspace, recordKey); + splitKeyHelper.writeSplit(context, keyBytes, serialized); if (sizeInfo != null) { sizeInfo.set(keyBytes, serialized); sizeInfo.setSplit(false); } } - writeVersion(context, subspace, key, version, sizeInfo); + // TODO + writeVersion(context, subspace, key, version, sizeInfo, splitKeyHelper); } @SuppressWarnings("PMD.CloseResource") private static void writeSplitRecord(@Nonnull final FDBRecordContext context, @Nonnull final Subspace subspace, @Nonnull final Tuple key, @Nonnull final byte[] serialized, + final SplitKeyHelper splitKeyHelper, final boolean clearBasedOnPreviousSizeInfo, @Nullable final FDBStoredSizes previousSizeInfo, @Nullable SizeInfo sizeInfo) { - final Transaction tr = context.ensureActive(); final Subspace keySplitSubspace = subspace.subspace(key); - clearPreviousSplitRecord(context, subspace, key, clearBasedOnPreviousSizeInfo, previousSizeInfo); + if (splitKeyHelper.clearBeforeWrite()) { + clearPreviousSplitRecord(context, subspace, key, clearBasedOnPreviousSizeInfo, previousSizeInfo); + } long index = SplitHelper.START_SPLIT_RECORD; int offset = 0; while (offset < serialized.length) { @@ -176,9 +179,9 @@ private static void writeSplitRecord(@Nonnull final FDBRecordContext context, @N if (nextOffset > serialized.length) { nextOffset = serialized.length; } - final byte[] keyBytes = keySplitSubspace.pack(index); + final byte[] keyBytes = splitKeyHelper.packSplitKey(keySplitSubspace, Tuple.from(index)); final byte[] valueBytes = Arrays.copyOfRange(serialized, offset, nextOffset); - tr.set(keyBytes, valueBytes); + splitKeyHelper.writeSplit(context, keyBytes, valueBytes); if (sizeInfo != null) { if (offset == 0) { sizeInfo.set(keyBytes, valueBytes); @@ -194,19 +197,20 @@ private static void writeSplitRecord(@Nonnull final FDBRecordContext context, @N @SuppressWarnings("PMD.CloseResource") private static void writeVersion(@Nonnull final FDBRecordContext context, @Nonnull final Subspace subspace, @Nonnull final Tuple key, - @Nullable final FDBRecordVersion version, @Nullable final SizeInfo sizeInfo) { + // TODO: Do we need? + @Nullable final FDBRecordVersion version, @Nullable final SizeInfo sizeInfo, final SplitKeyHelper splitKeyHelper) { if (version == null) { if (sizeInfo != null) { sizeInfo.setVersionedInline(false); } return; } - final Transaction tr = context.ensureActive(); - final byte[] keyBytes = subspace.pack(key.add(RECORD_VERSION)); + final byte[] keyBytes = splitKeyHelper.packSplitKey(subspace, key.add(RECORD_VERSION)); final byte[] valueBytes = packVersion(version); if (version.isComplete()) { - tr.set(keyBytes, valueBytes); + splitKeyHelper.writeSplit(context, keyBytes, valueBytes); } else { + // TODO context.addVersionMutation(MutationType.SET_VERSIONSTAMPED_VALUE, keyBytes, valueBytes); context.addToLocalVersionCache(keyBytes, version.getLocalVersion()); } @@ -342,6 +346,7 @@ private static void clearPreviousSplitRecord(@Nonnull final FDBRecordContext con tr.clear(keySplitSubspace.range()); // Clears both unsplit and previous longer split. } final byte[] versionKey = keySplitSubspace.pack(RECORD_VERSION); + // todo context.getLocalVersion(versionKey).ifPresent(localVersion -> context.removeVersionMutation(versionKey)); } @@ -1173,6 +1178,27 @@ private void logKey(@Nonnull String staticMessage, boolean done) { } } + public static final class DefaultSplitKeyHelper implements SplitKeyHelper { + public static final DefaultSplitKeyHelper INSTANCE = new DefaultSplitKeyHelper(); + + @Override + public boolean clearBeforeWrite() { + return true; + } + + @Override + public byte[] packSplitKey(final Subspace subspace, final Tuple key) { + return subspace.pack(key); + } + + @Override + @SuppressWarnings("PMD.CloseResource") + public void writeSplit(final FDBRecordContext context, final byte[] keyBytes, final byte[] valueBytes) { + final Transaction tr = context.ensureActive(); + tr.set(keyBytes, valueBytes); + } + } + /** * Exception thrown when only part of a split record is found. */ diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyHelper.java new file mode 100644 index 0000000000..ce3b418406 --- /dev/null +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyHelper.java @@ -0,0 +1,35 @@ +/* + * SplitKeyHelper.java + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2015-2026 Apple Inc. and the FoundationDB project authors + * + * Licensed 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 com.apple.foundationdb.record.provider.foundationdb; + +import com.apple.foundationdb.subspace.Subspace; +import com.apple.foundationdb.tuple.Tuple; + +/** + * An interface extracting the generation and persistence of keys used in the {@link SplitHelper}. + */ +public interface SplitKeyHelper { + boolean clearBeforeWrite(); + + byte[] packSplitKey(Subspace subspace, Tuple key); + + void writeSplit(FDBRecordContext context, byte[] keyBytes, byte[] valueBytes); +} diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyHelper.java new file mode 100644 index 0000000000..edfb69313d --- /dev/null +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyHelper.java @@ -0,0 +1,64 @@ +/* + * VersioningSplitKeyHelper.java + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2015-2026 Apple Inc. and the FoundationDB project authors + * + * Licensed 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 com.apple.foundationdb.record.provider.foundationdb; + +import com.apple.foundationdb.MutationType; +import com.apple.foundationdb.record.RecordCoreInternalException; +import com.apple.foundationdb.subspace.Subspace; +import com.apple.foundationdb.tuple.ByteArrayUtil; +import com.apple.foundationdb.tuple.Tuple; +import com.apple.foundationdb.tuple.Versionstamp; + +import java.util.Arrays; + +public class VersioningSplitKeyHelper implements SplitKeyHelper { + private Versionstamp versionstamp; + + public VersioningSplitKeyHelper(final Versionstamp versionstamp) { + this.versionstamp = versionstamp; + } + + // Since the key has a version, no conflicts are expected, so no need to clean + @Override + public boolean clearBeforeWrite() { + return false; + } + + @Override + public byte[] packSplitKey(final Subspace subspace, final Tuple key) { + // This uses the same version (local and global for all the splits + Tuple keyTuple = Tuple.from(versionstamp).addAll(key); + return subspace.packWithVersionstamp(keyTuple); + } + + @Override + public void writeSplit(final FDBRecordContext context, final byte[] keyBytes, final byte[] valueBytes) { + final byte[] current = context.addVersionMutation( + MutationType.SET_VERSIONSTAMPED_KEY, + keyBytes, + valueBytes); + + if (current != null) { + // This should never happen + throw new RecordCoreInternalException("Key with version overwritten"); + } + } +} diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java index 7d15cd5a04..2a9db3441a 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java @@ -220,7 +220,8 @@ private SplitHelper.SizeInfo saveUnsuccessfully(@Nonnull F @Nonnull Class errClazz, @Nonnull String errMessage) { final SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); E e = assertThrows(errClazz, - () -> SplitHelper.saveWithSplit(context, subspace, key, serialized, version, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, previousSizeInfo != null, previousSizeInfo, sizeInfo)); + () -> SplitHelper.saveWithSplit(context, subspace, key, serialized, version, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, + SplitHelper.DefaultSplitKeyHelper.INSTANCE, previousSizeInfo != null, previousSizeInfo, sizeInfo)); assertThat(e.getMessage(), containsString(errMessage)); assertEquals(0, sizeInfo.getKeyCount()); @@ -244,7 +245,8 @@ private SplitHelper.SizeInfo saveSuccessfully(@Nonnull FDBRecordContext context, @Nonnull SplitHelperTestConfig testConfig, @Nullable FDBStoredSizes previousSizeInfo) { final SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); - SplitHelper.saveWithSplit(context, subspace, key, serialized, version, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, previousSizeInfo != null, previousSizeInfo, sizeInfo); + SplitHelper.saveWithSplit(context, subspace, key, serialized, version, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, + SplitHelper.DefaultSplitKeyHelper.INSTANCE, previousSizeInfo != null, previousSizeInfo, sizeInfo); int dataKeyCount = (serialized.length - 1) / SplitHelper.SPLIT_RECORD_SIZE + 1; boolean isSplit = dataKeyCount > 1; int keyCount = dataKeyCount; diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java index 282cd6a4bd..aa938161b1 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java @@ -40,8 +40,12 @@ import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordVersion; import com.apple.foundationdb.record.provider.foundationdb.KeyValueCursor; +import com.apple.foundationdb.record.provider.foundationdb.SplitHelper; +import com.apple.foundationdb.record.provider.foundationdb.SplitKeyHelper; +import com.apple.foundationdb.record.provider.foundationdb.VersioningSplitKeyHelper; import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.Tuple; +import com.apple.foundationdb.tuple.TupleHelpers; import com.apple.foundationdb.tuple.Versionstamp; import com.google.protobuf.ByteString; import org.apache.lucene.index.IndexWriter; @@ -182,7 +186,9 @@ public void clearEntry(@Nonnull FDBRecordContext context, @Nonnull QueueEntry en throw new RecordCoreArgumentException("Queue item should have complete version stamp"); } - context.ensureActive().clear(queueSubspace.pack(entry.versionstamp)); + // The only element of the key is the completed version stamp + final Tuple key = Tuple.from(entry.getVersionstamp()); + SplitHelper.deleteSplit(context, queueSubspace, key, true, false, false, null); // Record metrics context.increment(LuceneEvents.Counts.LUCENE_PENDING_QUEUE_CLEAR); @@ -295,20 +301,11 @@ private void enqueueOperationInternal( // Build key with incomplete versionStamp with a new local version FDBRecordVersion recordVersion = FDBRecordVersion.incomplete(context.claimLocalVersion()); - Tuple keyTuple = Tuple.from(recordVersion.toVersionstamp()); - byte[] queueKey = queueSubspace.packWithVersionstamp(keyTuple); + // Use the version in the key helper for all splits of the same entry + SplitKeyHelper keyHelper = new VersioningSplitKeyHelper(recordVersion.toVersionstamp()); byte[] value = builder.build().toByteArray(); - - // Use addVersionMutation to let FDB assign the versionStamp - final byte[] current = context.addVersionMutation( - MutationType.SET_VERSIONSTAMPED_KEY, - queueKey, - value); - - if (current != null) { - // This should never happen - throw new RecordCoreInternalException("Pending queue item overwritten"); - } + // save with splits + SplitHelper.saveWithSplit(context, queueSubspace, TupleHelpers.EMPTY, value, null, true, false, keyHelper, false, null, null); // Record metrics context.increment(LuceneEvents.Counts.LUCENE_PENDING_QUEUE_WRITE); From d7e130d52d213bb095f948bdc8c9f35d98c6318d Mon Sep 17 00:00:00 2001 From: ohad Date: Mon, 16 Feb 2026 14:30:46 -0500 Subject: [PATCH 02/29] Style --- .../record/provider/foundationdb/VersioningSplitKeyHelper.java | 3 --- 1 file changed, 3 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyHelper.java index edfb69313d..6e0a2b526f 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyHelper.java @@ -23,12 +23,9 @@ import com.apple.foundationdb.MutationType; import com.apple.foundationdb.record.RecordCoreInternalException; import com.apple.foundationdb.subspace.Subspace; -import com.apple.foundationdb.tuple.ByteArrayUtil; import com.apple.foundationdb.tuple.Tuple; import com.apple.foundationdb.tuple.Versionstamp; -import java.util.Arrays; - public class VersioningSplitKeyHelper implements SplitKeyHelper { private Versionstamp versionstamp; From a3ef2ef1e4c42a87372940b27bd5a64aa10218ef Mon Sep 17 00:00:00 2001 From: ohad Date: Wed, 18 Feb 2026 18:49:20 -0500 Subject: [PATCH 03/29] Implement serializer and allowed version/clear options --- .../DefaultSplitKeyValueHelper.java | 63 +++++++++++++ .../provider/foundationdb/FDBRecordStore.java | 2 +- .../provider/foundationdb/SplitHelper.java | 40 ++------ .../provider/foundationdb/SplitKeyHelper.java | 35 ------- .../foundationdb/SplitKeyValueHelper.java | 60 ++++++++++++ ...ava => VersioningSplitKeyValueHelper.java} | 37 +++++++- .../record/lucene/directory/FDBDirectory.java | 2 +- .../lucene/directory/PendingWriteQueue.java | 20 ++-- .../directory/PendingWritesQueueHelper.java | 5 +- .../directory/PendingWriteQueueTest.java | 92 +++++++++++++++++-- 10 files changed, 265 insertions(+), 91 deletions(-) create mode 100644 fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/DefaultSplitKeyValueHelper.java delete mode 100644 fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyHelper.java create mode 100644 fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyValueHelper.java rename fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/{VersioningSplitKeyHelper.java => VersioningSplitKeyValueHelper.java} (52%) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/DefaultSplitKeyValueHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/DefaultSplitKeyValueHelper.java new file mode 100644 index 0000000000..afd627f492 --- /dev/null +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/DefaultSplitKeyValueHelper.java @@ -0,0 +1,63 @@ +/* + * DefaultSplitKeyValueHelper.java + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2015-2026 Apple Inc. and the FoundationDB project authors + * + * Licensed 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 com.apple.foundationdb.record.provider.foundationdb; + +import com.apple.foundationdb.Transaction; +import com.apple.foundationdb.subspace.Subspace; +import com.apple.foundationdb.tuple.Tuple; + +/** + * The default implementation of the {@link SplitKeyValueHelper}. + * This implementation is using the subspace serialization and transaction set methods. + */ +public class DefaultSplitKeyValueHelper implements SplitKeyValueHelper { + public static final DefaultSplitKeyValueHelper INSTANCE = new DefaultSplitKeyValueHelper(); + + /** + * Since this is setting keys directly, there is a chance that existing keys will interfere with the new keys. + * @return true to ensure all prior keys are cleared from the subspace + */ + @Override + public boolean shouldClearBeforeWrite() { + return true; + } + + /** + * Value can mutate version stamp with this helper. + * @return true if the SplitHelper should mutate a version in the value + */ + @Override + public boolean supportsVersionInValue() { + return true; + } + + @Override + public byte[] packSplitKey(final Subspace subspace, final Tuple key) { + return subspace.pack(key); + } + + @Override + @SuppressWarnings("PMD.CloseResource") + public void writeSplit(final FDBRecordContext context, final byte[] keyBytes, final byte[] valueBytes) { + final Transaction tr = context.ensureActive(); + tr.set(keyBytes, valueBytes); + } +} diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java index 5affd81d5d..44e0f9a0bb 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java @@ -665,7 +665,7 @@ private FDBStoredRecord serializeAndSaveRecord(@Nonnull R final SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); preloadCache.invalidate(primaryKey); // clear out cache of older value if present SplitHelper.saveWithSplit(context, recordsSubspace(), recordBuilder.getPrimaryKey(), serialized, splitVersion, metaData.isSplitLongRecords(), omitUnsplitRecordSuffix, - SplitHelper.DefaultSplitKeyHelper.INSTANCE, true, oldSizeInfo, sizeInfo); + DefaultSplitKeyValueHelper.INSTANCE, true, oldSizeInfo, sizeInfo); countKeysAndValues(FDBStoreTimer.Counts.SAVE_RECORD_KEY, FDBStoreTimer.Counts.SAVE_RECORD_KEY_BYTES, FDBStoreTimer.Counts.SAVE_RECORD_VALUE_BYTES, sizeInfo); recordBuilder.setSize(sizeInfo); diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java index c66bc87f4c..ebfc51c54c 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java @@ -33,6 +33,7 @@ import com.apple.foundationdb.record.FDBRecordStoreProperties; import com.apple.foundationdb.record.RecordCoreArgumentException; import com.apple.foundationdb.record.RecordCoreException; +import com.apple.foundationdb.record.RecordCoreInternalException; import com.apple.foundationdb.record.RecordCoreStorageException; import com.apple.foundationdb.record.RecordCursor; import com.apple.foundationdb.record.RecordCursorContinuation; @@ -104,7 +105,7 @@ private SplitHelper() { */ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnull final Subspace subspace, @Nonnull final Tuple key, @Nonnull final byte[] serialized, @Nullable final FDBRecordVersion version) { - saveWithSplit(context, subspace, key, serialized, version, true, false, DefaultSplitKeyHelper.INSTANCE, false, null, null); + saveWithSplit(context, subspace, key, serialized, version, true, false, DefaultSplitKeyValueHelper.INSTANCE, false, null, null); } /** @@ -124,7 +125,7 @@ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnu public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnull final Subspace subspace, @Nonnull final Tuple key, @Nonnull final byte[] serialized, @Nullable final FDBRecordVersion version, final boolean splitLongRecords, final boolean omitUnsplitSuffix, - final SplitKeyHelper splitKeyHelper, + final SplitKeyValueHelper splitKeyHelper, final boolean clearBasedOnPreviousSizeInfo, @Nullable final FDBStoredSizes previousSizeInfo, @Nullable SizeInfo sizeInfo) { if (omitUnsplitSuffix && version != null) { @@ -142,7 +143,7 @@ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnu } writeSplitRecord(context, subspace, key, serialized, splitKeyHelper, clearBasedOnPreviousSizeInfo, previousSizeInfo, sizeInfo); } else { - if (splitKeyHelper.clearBeforeWrite() && (splitLongRecords || previousSizeInfo == null || previousSizeInfo.isVersionedInline())) { + if (splitKeyHelper.shouldClearBeforeWrite() && (splitLongRecords || previousSizeInfo == null || previousSizeInfo.isVersionedInline())) { clearPreviousSplitRecord(context, subspace, key, clearBasedOnPreviousSizeInfo, previousSizeInfo); } final Tuple recordKey; @@ -158,18 +159,17 @@ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnu sizeInfo.setSplit(false); } } - // TODO writeVersion(context, subspace, key, version, sizeInfo, splitKeyHelper); } @SuppressWarnings("PMD.CloseResource") private static void writeSplitRecord(@Nonnull final FDBRecordContext context, @Nonnull final Subspace subspace, @Nonnull final Tuple key, @Nonnull final byte[] serialized, - final SplitKeyHelper splitKeyHelper, + final SplitKeyValueHelper splitKeyHelper, final boolean clearBasedOnPreviousSizeInfo, @Nullable final FDBStoredSizes previousSizeInfo, @Nullable SizeInfo sizeInfo) { final Subspace keySplitSubspace = subspace.subspace(key); - if (splitKeyHelper.clearBeforeWrite()) { + if (splitKeyHelper.shouldClearBeforeWrite()) { clearPreviousSplitRecord(context, subspace, key, clearBasedOnPreviousSizeInfo, previousSizeInfo); } long index = SplitHelper.START_SPLIT_RECORD; @@ -197,20 +197,21 @@ private static void writeSplitRecord(@Nonnull final FDBRecordContext context, @N @SuppressWarnings("PMD.CloseResource") private static void writeVersion(@Nonnull final FDBRecordContext context, @Nonnull final Subspace subspace, @Nonnull final Tuple key, - // TODO: Do we need? - @Nullable final FDBRecordVersion version, @Nullable final SizeInfo sizeInfo, final SplitKeyHelper splitKeyHelper) { + @Nullable final FDBRecordVersion version, @Nullable final SizeInfo sizeInfo, final SplitKeyValueHelper splitKeyHelper) { if (version == null) { if (sizeInfo != null) { sizeInfo.setVersionedInline(false); } return; } + if (!splitKeyHelper.supportsVersionInValue()) { + throw new RecordCoreInternalException("Split version is not supported for this helper"); + } final byte[] keyBytes = splitKeyHelper.packSplitKey(subspace, key.add(RECORD_VERSION)); final byte[] valueBytes = packVersion(version); if (version.isComplete()) { splitKeyHelper.writeSplit(context, keyBytes, valueBytes); } else { - // TODO context.addVersionMutation(MutationType.SET_VERSIONSTAMPED_VALUE, keyBytes, valueBytes); context.addToLocalVersionCache(keyBytes, version.getLocalVersion()); } @@ -1178,27 +1179,6 @@ private void logKey(@Nonnull String staticMessage, boolean done) { } } - public static final class DefaultSplitKeyHelper implements SplitKeyHelper { - public static final DefaultSplitKeyHelper INSTANCE = new DefaultSplitKeyHelper(); - - @Override - public boolean clearBeforeWrite() { - return true; - } - - @Override - public byte[] packSplitKey(final Subspace subspace, final Tuple key) { - return subspace.pack(key); - } - - @Override - @SuppressWarnings("PMD.CloseResource") - public void writeSplit(final FDBRecordContext context, final byte[] keyBytes, final byte[] valueBytes) { - final Transaction tr = context.ensureActive(); - tr.set(keyBytes, valueBytes); - } - } - /** * Exception thrown when only part of a split record is found. */ diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyHelper.java deleted file mode 100644 index ce3b418406..0000000000 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyHelper.java +++ /dev/null @@ -1,35 +0,0 @@ -/* - * SplitKeyHelper.java - * - * This source file is part of the FoundationDB open source project - * - * Copyright 2015-2026 Apple Inc. and the FoundationDB project authors - * - * Licensed 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 com.apple.foundationdb.record.provider.foundationdb; - -import com.apple.foundationdb.subspace.Subspace; -import com.apple.foundationdb.tuple.Tuple; - -/** - * An interface extracting the generation and persistence of keys used in the {@link SplitHelper}. - */ -public interface SplitKeyHelper { - boolean clearBeforeWrite(); - - byte[] packSplitKey(Subspace subspace, Tuple key); - - void writeSplit(FDBRecordContext context, byte[] keyBytes, byte[] valueBytes); -} diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyValueHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyValueHelper.java new file mode 100644 index 0000000000..d967a30c80 --- /dev/null +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyValueHelper.java @@ -0,0 +1,60 @@ +/* + * SplitKeyHelper.java + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2015-2026 Apple Inc. and the FoundationDB project authors + * + * Licensed 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 com.apple.foundationdb.record.provider.foundationdb; + +import com.apple.foundationdb.subspace.Subspace; +import com.apple.foundationdb.tuple.Tuple; + +/** + * An interface extracting the generation and persistence of keys used in the {@link SplitHelper}. + */ +public interface SplitKeyValueHelper { + /** + * Whether the {@link SplitHelper} should clear older entries before writing new ones. + * In the cases where old splits may exist and overlap with the new entries, the SplitHelper should clear the + * subspace before writing any new splits. THis method is used to determine whether this is necessary. + * @return TRUE if the SplitHelper should clear the subspace before writing new splits. + */ + boolean shouldClearBeforeWrite(); + + /** + * Whether the helper allows version mutation in the values. + * There are cases where the value cannot have version mutation, for example, when the needs to have one. + * @return true if version mutations are allowed for the value of the k/v pair, false if not + */ + boolean supportsVersionInValue(); + + /** + * Serialize a key to a format that can be saved to FDB. + * @param subspace the subspace to use for the key + * @param key the key Tuple to use for the rest of the key + * @return the serialized form of the key + */ + byte[] packSplitKey(Subspace subspace, Tuple key); + + /** + * Write a key/value pair to FDB. + * @param context the transaction to use for writing + * @param keyBytes the key to use + * @param valueBytes the value to use + */ + void writeSplit(FDBRecordContext context, byte[] keyBytes, byte[] valueBytes); +} diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java similarity index 52% rename from fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyHelper.java rename to fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java index 6e0a2b526f..47c4d2be3e 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java @@ -26,22 +26,51 @@ import com.apple.foundationdb.tuple.Tuple; import com.apple.foundationdb.tuple.Versionstamp; -public class VersioningSplitKeyHelper implements SplitKeyHelper { +/** + * A {@link SplitKeyValueHelper} that is used when the Key contains a {@link Versionstamp}. + *

This implementation should be used when the key contains a version stamp, as it will ensure that the proper FDB APIs + * encode and decode the key correctly.

+ *

This class is stateful (has a single {@link Versionstamp}) that is going to be used for all splits of a single K/V, + * so that all contain the same fixed part and can be correlated after the commit.

+ *

The resulting FDB key looks like:

+ *
+ *     [versionstamp, original-key, split-suffix]
+ * 
+ *

which means that the entries are sorted by their insertion order (versionstamp order), then grouped by their + * split suffixes.

+ */ +public class VersioningSplitKeyValueHelper implements SplitKeyValueHelper { private Versionstamp versionstamp; - public VersioningSplitKeyHelper(final Versionstamp versionstamp) { + public VersioningSplitKeyValueHelper(final Versionstamp versionstamp) { this.versionstamp = versionstamp; } - // Since the key has a version, no conflicts are expected, so no need to clean + /** + * No need to clear subspace. + * Since the key has a unique component (version), no conflicts are expected, so no need to clean before saving new splits. + * Furthermore, since the key contains a version stamp, we don't know the actual key contents ahead of committing + * the transaction, and so no clean can be done. + * @return false, as new keys should not interfere with old ones. + */ + @Override + public boolean shouldClearBeforeWrite() { + return false; + } + + /** + * Since the key has versions, prevent the values from having them. + * @return false, since only keys or values are allowed to mutate in FDB, and this mutates the keys + */ @Override - public boolean clearBeforeWrite() { + public boolean supportsVersionInValue() { return false; } @Override public byte[] packSplitKey(final Subspace subspace, final Tuple key) { // This uses the same version (local and global for all the splits + // Use versionstamp first to ensure proper sorting and since split suffix should be at the end Tuple keyTuple = Tuple.from(versionstamp).addAll(key); return subspace.packWithVersionstamp(keyTuple); } diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/FDBDirectory.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/FDBDirectory.java index 6ae0c51171..9db655e701 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/FDBDirectory.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/FDBDirectory.java @@ -1053,7 +1053,7 @@ public void clearOngoingMergeIndicatorButFailIfNonEmpty() { } public PendingWriteQueue createPendingWritesQueue() { - return new PendingWriteQueue(pendingWritesQueueSubspace, maxPendingWritesToReplay); + return new PendingWriteQueue(pendingWritesQueueSubspace, maxPendingWritesToReplay, serializer); } public int getBlockCacheMaximumSize() { diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java index aa938161b1..5a0c74343a 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java @@ -20,7 +20,6 @@ package com.apple.foundationdb.record.lucene.directory; -import com.apple.foundationdb.MutationType; import com.apple.foundationdb.annotation.API; import com.apple.foundationdb.record.ExecuteProperties; import com.apple.foundationdb.record.RecordCoreArgumentException; @@ -41,8 +40,8 @@ import com.apple.foundationdb.record.provider.foundationdb.FDBRecordVersion; import com.apple.foundationdb.record.provider.foundationdb.KeyValueCursor; import com.apple.foundationdb.record.provider.foundationdb.SplitHelper; -import com.apple.foundationdb.record.provider.foundationdb.SplitKeyHelper; -import com.apple.foundationdb.record.provider.foundationdb.VersioningSplitKeyHelper; +import com.apple.foundationdb.record.provider.foundationdb.SplitKeyValueHelper; +import com.apple.foundationdb.record.provider.foundationdb.VersioningSplitKeyValueHelper; import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.Tuple; import com.apple.foundationdb.tuple.TupleHelpers; @@ -93,6 +92,7 @@ public class PendingWriteQueue { * Default: 0 (unlimited) */ private int maxEntriesToReplay; + private LuceneSerializer serializer; private final Subspace queueSubspace; @@ -102,13 +102,14 @@ public class PendingWriteQueue { * @param queueSubspace the subspace for this partition's queue, should include the partition ID and grouping key, * as necessary */ - public PendingWriteQueue(@Nonnull Subspace queueSubspace) { - this(queueSubspace, MAX_PENDING_ENTRIES_TO_REPLAY); + public PendingWriteQueue(@Nonnull Subspace queueSubspace, final LuceneSerializer serializer) { + this(queueSubspace, MAX_PENDING_ENTRIES_TO_REPLAY, serializer); } - public PendingWriteQueue(@Nonnull Subspace queueSubspace, int maxEntriesToReplay) { + public PendingWriteQueue(@Nonnull Subspace queueSubspace, int maxEntriesToReplay, LuceneSerializer serializer) { this.queueSubspace = queueSubspace; this.maxEntriesToReplay = maxEntriesToReplay; + this.serializer = serializer; } /** @@ -167,7 +168,7 @@ public RecordCursor getQueueCursor( .setScanProperties(scanProperties) .setContinuation(continuation) .build(); - return cursor.map(kv -> PendingWritesQueueHelper.toQueueEntry(queueSubspace, kv)); + return cursor.map(kv -> PendingWritesQueueHelper.toQueueEntry(queueSubspace, serializer, kv)); } /** @@ -302,8 +303,8 @@ private void enqueueOperationInternal( // Build key with incomplete versionStamp with a new local version FDBRecordVersion recordVersion = FDBRecordVersion.incomplete(context.claimLocalVersion()); // Use the version in the key helper for all splits of the same entry - SplitKeyHelper keyHelper = new VersioningSplitKeyHelper(recordVersion.toVersionstamp()); - byte[] value = builder.build().toByteArray(); + SplitKeyValueHelper keyHelper = new VersioningSplitKeyValueHelper(recordVersion.toVersionstamp()); + byte[] value = serializer.encode(builder.build().toByteArray()); // save with splits SplitHelper.saveWithSplit(context, queueSubspace, TupleHelpers.EMPTY, value, null, true, false, keyHelper, false, null, null); @@ -314,6 +315,7 @@ private void enqueueOperationInternal( LOGGER.debug(getLogMessage("Enqueued operation") .addKeyAndValue(LuceneLogMessageKeys.OPERATION_TYPE, operationType) .addKeyAndValue(LogMessageKeys.SUBSPACE, queueSubspace) + .addKeyAndValue(LogMessageKeys.VALUE_SIZE, value.length) .toString()); } } diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWritesQueueHelper.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWritesQueueHelper.java index 5d7c705b60..00c3cfbfb2 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWritesQueueHelper.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWritesQueueHelper.java @@ -40,11 +40,12 @@ @API(API.Status.INTERNAL) public final class PendingWritesQueueHelper { - public static PendingWriteQueue.QueueEntry toQueueEntry(final Subspace queueSubspace, final KeyValue kv) { + public static PendingWriteQueue.QueueEntry toQueueEntry(final Subspace queueSubspace, LuceneSerializer serializer, final KeyValue kv) { try { Tuple keyTuple = queueSubspace.unpack(kv.getKey()); final Versionstamp versionstamp = keyTuple.getVersionstamp(0); - LucenePendingWriteQueueProto.PendingWriteItem item = LucenePendingWriteQueueProto.PendingWriteItem.parseFrom(kv.getValue()); + final byte[] value = serializer.decode(kv.getValue()); + LucenePendingWriteQueueProto.PendingWriteItem item = LucenePendingWriteQueueProto.PendingWriteItem.parseFrom(value); return new PendingWriteQueue.QueueEntry(versionstamp, item); } catch (InvalidProtocolBufferException e) { throw new RecordCoreInternalException("Failed to parse queue item", e); diff --git a/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueueTest.java b/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueueTest.java index d7110a06e2..1cef8a59c2 100644 --- a/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueueTest.java +++ b/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueueTest.java @@ -24,6 +24,7 @@ import com.apple.foundationdb.record.IndexEntry; import com.apple.foundationdb.record.IsolationLevel; import com.apple.foundationdb.record.RecordCoreArgumentException; +import com.apple.foundationdb.record.RecordCoreInternalException; import com.apple.foundationdb.record.RecordCursor; import com.apple.foundationdb.record.RecordCursorContinuation; import com.apple.foundationdb.record.RecordCursorResult; @@ -56,6 +57,7 @@ import com.google.common.collect.Streams; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Assumptions; +import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Disabled; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; @@ -99,6 +101,13 @@ */ @Tag(Tags.RequiresFDB) public class PendingWriteQueueTest extends FDBRecordStoreTestBase { + LuceneSerializer serializer; + + @BeforeEach + void setup() { + serializer = new LuceneSerializer(true, false, null, true); + } + @ParameterizedTest @EnumSource void testEnqueueAndIterate(LucenePendingWriteQueueProto.PendingWriteItem.OperationType operationType) { @@ -106,7 +115,7 @@ void testEnqueueAndIterate(LucenePendingWriteQueueProto.PendingWriteItem.Operati Assumptions.assumeFalse(operationType.equals(LucenePendingWriteQueueProto.PendingWriteItem.OperationType.OPERATION_TYPE_UNSPECIFIED)); List docs = createTestDocuments(); - PendingWriteQueue queue = new PendingWriteQueue(new Subspace(Tuple.from(UUID.randomUUID().toString()))); + PendingWriteQueue queue = new PendingWriteQueue(new Subspace(Tuple.from(UUID.randomUUID().toString())), serializer); try (FDBRecordContext context = openContext()) { docs.forEach(doc -> { @@ -132,7 +141,7 @@ void testEnqueueAndIterate(LucenePendingWriteQueueProto.PendingWriteItem.Operati void testEnqueueMultipleTransactions() { List docs = createTestDocuments(); List moreDocs = createTestDocuments(); - PendingWriteQueue queue = new PendingWriteQueue(new Subspace(Tuple.from(UUID.randomUUID().toString()))); + PendingWriteQueue queue = new PendingWriteQueue(new Subspace(Tuple.from(UUID.randomUUID().toString())), serializer); try (FDBRecordContext context = openContext()) { docs.forEach(doc -> { @@ -154,7 +163,7 @@ void testEnqueueMultipleTransactions() { @Test void testEnqueueAndDelete() { List docs = createTestDocuments(); - PendingWriteQueue queue = new PendingWriteQueue(new Subspace(Tuple.from(UUID.randomUUID().toString()))); + PendingWriteQueue queue = new PendingWriteQueue(new Subspace(Tuple.from(UUID.randomUUID().toString())), serializer); try (FDBRecordContext context = openContext()) { docs.forEach(doc -> { @@ -185,7 +194,7 @@ void testEnqueueAndDelete() { @Test void testDeleteAll() { List docs = createTestDocuments(); - PendingWriteQueue queue = new PendingWriteQueue(new Subspace(Tuple.from(UUID.randomUUID().toString()))); + PendingWriteQueue queue = new PendingWriteQueue(new Subspace(Tuple.from(UUID.randomUUID().toString())), serializer); try (FDBRecordContext context = openContext()) { docs.forEach(doc -> { @@ -211,13 +220,13 @@ void testDeleteAll() { @Test void testIterateEmptyQueue() { - PendingWriteQueue queue = new PendingWriteQueue(new Subspace(Tuple.from(UUID.randomUUID().toString()))); + PendingWriteQueue queue = new PendingWriteQueue(new Subspace(Tuple.from(UUID.randomUUID().toString())), serializer); assertQueueEntries(queue, Collections.emptyList(), LucenePendingWriteQueueProto.PendingWriteItem.OperationType.INSERT); } @Test void testWrongValueType() { - PendingWriteQueue queue = new PendingWriteQueue(new Subspace(Tuple.from(UUID.randomUUID().toString()))); + PendingWriteQueue queue = new PendingWriteQueue(new Subspace(Tuple.from(UUID.randomUUID().toString())), serializer); final LuceneDocumentFromRecord.DocumentField fieldWithWrongType = createField("f", 5, LuceneIndexExpressions.DocumentFieldType.STRING, true, true); @@ -229,7 +238,7 @@ void testWrongValueType() { @Test void testUnsupportedFieldConfigType() { - PendingWriteQueue queue = new PendingWriteQueue(new Subspace(Tuple.from(UUID.randomUUID().toString()))); + PendingWriteQueue queue = new PendingWriteQueue(new Subspace(Tuple.from(UUID.randomUUID().toString())), serializer); final LuceneDocumentFromRecord.DocumentField fieldWithWrongConfig = createField("f", 5, LuceneIndexExpressions.DocumentFieldType.INT, true, true, Map.of("Double", 5.42D)); @@ -243,7 +252,7 @@ void testUnsupportedFieldConfigType() { void testIterateWithContinuations() { List docs = createTestDocuments(); List moreDocs = createTestDocuments(); - PendingWriteQueue queue = new PendingWriteQueue(new Subspace(Tuple.from(UUID.randomUUID().toString()))); + PendingWriteQueue queue = new PendingWriteQueue(new Subspace(Tuple.from(UUID.randomUUID().toString())), serializer); try (FDBRecordContext context = openContext()) { docs.forEach(doc -> { @@ -302,7 +311,7 @@ void testIterateWithContinuations() { @Test void testIsQueueEmpty() { List docs = createTestDocuments(); - PendingWriteQueue queue = new PendingWriteQueue(new Subspace(Tuple.from(UUID.randomUUID().toString()))); + PendingWriteQueue queue = new PendingWriteQueue(new Subspace(Tuple.from(UUID.randomUUID().toString())), serializer); try (FDBRecordContext context = openContext()) { assertTrue(queue.isQueueEmpty(context).join()); @@ -322,6 +331,53 @@ void testIsQueueEmpty() { }); } + @Test + void testFailToSerialize() { + List docs = createTestDocuments(); + LuceneSerializer failingSerializer = new FailingLuceneSerializer(); + + PendingWriteQueue queue = new PendingWriteQueue(new Subspace(Tuple.from(UUID.randomUUID().toString())), failingSerializer); + + try (FDBRecordContext context = openContext()) { + final TestDocument doc = docs.get(0); + Assertions.assertThatThrownBy(() -> queue.enqueueInsert(context, doc.getPrimaryKey(), doc.getFields())) + .isInstanceOf(RecordCoreInternalException.class) + .hasMessageContaining("Failing to encode"); + + // Commit here should do nothing as the queue should still be empty + commit(context); + } + + try (FDBRecordContext context = openContext()) { + assertTrue(queue.isQueueEmpty(context).join(), "Expected isQueueEmpty to return true"); + commit(context); + } + } + + @Test + void testFailToDeserialize() { + List docs = createTestDocuments(); + LuceneSerializer failingSerializer = new FailingLuceneSerializer(); + + final Subspace queueSubspace = new Subspace(Tuple.from(UUID.randomUUID().toString())); + PendingWriteQueue queue = new PendingWriteQueue(queueSubspace, serializer); + PendingWriteQueue failingQueue = new PendingWriteQueue(queueSubspace, failingSerializer); + + try (FDBRecordContext context = openContext()) { + final TestDocument doc = docs.get(0); + // save a single doc uasing the good queue + queue.enqueueInsert(context, doc.getPrimaryKey(), doc.getFields()); + commit(context); + } + + try (FDBRecordContext context = openContext()) { + RecordCursor queueCursor = failingQueue.getQueueCursor(context, ScanProperties.FORWARD_SCAN, null); + Assertions.assertThatThrownBy(() -> queueCursor.asList().get()) + .hasCauseInstanceOf(RecordCoreInternalException.class) + .hasMessageContaining("Failing to decode"); + } + } + @Test void testPendingQueueSimple() { // Test simple non-partitioned pending queue life cycle @@ -1266,4 +1322,22 @@ public List getFields() { return fields; } } + + private static class FailingLuceneSerializer extends LuceneSerializer { + public FailingLuceneSerializer() { + super(true, false, null, true); + } + + @Nullable + @Override + public byte[] encode(@Nullable final byte[] data) { + throw new RecordCoreInternalException("Failing to encode"); + } + + @Nullable + @Override + public byte[] decode(@Nullable final byte[] data) { + throw new RecordCoreInternalException("Failing to decode"); + } + } } From 6bdd1cf915e91864e9805f2bdfd1405f79950f63 Mon Sep 17 00:00:00 2001 From: ohad Date: Thu, 19 Feb 2026 14:58:57 -0500 Subject: [PATCH 04/29] Rebase from main --- .../record/provider/foundationdb/SplitHelperTest.java | 4 ++-- .../record/lucene/directory/PendingWriteQueue.java | 1 + 2 files changed, 3 insertions(+), 2 deletions(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java index 2a9db3441a..2a6b976304 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java @@ -221,7 +221,7 @@ private SplitHelper.SizeInfo saveUnsuccessfully(@Nonnull F final SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); E e = assertThrows(errClazz, () -> SplitHelper.saveWithSplit(context, subspace, key, serialized, version, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, - SplitHelper.DefaultSplitKeyHelper.INSTANCE, previousSizeInfo != null, previousSizeInfo, sizeInfo)); + DefaultSplitKeyValueHelper.INSTANCE, previousSizeInfo != null, previousSizeInfo, sizeInfo)); assertThat(e.getMessage(), containsString(errMessage)); assertEquals(0, sizeInfo.getKeyCount()); @@ -246,7 +246,7 @@ private SplitHelper.SizeInfo saveSuccessfully(@Nonnull FDBRecordContext context, @Nullable FDBStoredSizes previousSizeInfo) { final SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); SplitHelper.saveWithSplit(context, subspace, key, serialized, version, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, - SplitHelper.DefaultSplitKeyHelper.INSTANCE, previousSizeInfo != null, previousSizeInfo, sizeInfo); + DefaultSplitKeyValueHelper.INSTANCE, previousSizeInfo != null, previousSizeInfo, sizeInfo); int dataKeyCount = (serialized.length - 1) / SplitHelper.SPLIT_RECORD_SIZE + 1; boolean isSplit = dataKeyCount > 1; int keyCount = dataKeyCount; diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java index a72899f224..23f8c0fb79 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java @@ -20,6 +20,7 @@ package com.apple.foundationdb.record.lucene.directory; +import com.apple.foundationdb.MutationType; import com.apple.foundationdb.annotation.API; import com.apple.foundationdb.record.ExecuteProperties; import com.apple.foundationdb.record.RecordCoreArgumentException; From 783c3b104094e312b92eb7b44d8f45b79a01c666 Mon Sep 17 00:00:00 2001 From: ohad Date: Fri, 20 Feb 2026 10:39:34 -0500 Subject: [PATCH 05/29] Complete unsplitting --- .../provider/foundationdb/FDBRawRecord.java | 2 +- .../lucene/directory/PendingWriteQueue.java | 17 +++++++--- .../directory/PendingWritesQueueHelper.java | 14 ++++----- .../directory/PendingWriteQueueTest.java | 31 +++++++++---------- 4 files changed, 36 insertions(+), 28 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRawRecord.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRawRecord.java index 552aa4660f..238a9457e0 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRawRecord.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRawRecord.java @@ -33,7 +33,7 @@ * any splits have been removed), and its version. It also includes sizing information describing * the record's on-disk footprint. */ -class FDBRawRecord implements FDBStoredSizes { +public class FDBRawRecord implements FDBStoredSizes { @Nonnull private final Tuple primaryKey; @Nonnull private final byte[] rawRecord; @Nullable private final FDBRecordVersion version; diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java index 23f8c0fb79..18bffadab3 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java @@ -38,6 +38,7 @@ import com.apple.foundationdb.record.lucene.LuceneLogMessageKeys; import com.apple.foundationdb.record.lucene.LucenePendingWriteQueueProto; import com.apple.foundationdb.record.metadata.Index; +import com.apple.foundationdb.record.provider.foundationdb.FDBRawRecord; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordVersion; import com.apple.foundationdb.record.provider.foundationdb.KeyValueCursor; @@ -183,13 +184,21 @@ public RecordCursor getQueueCursor( @Nonnull FDBRecordContext context, @Nonnull ScanProperties scanProperties, @Nullable byte[] continuation) { - - final KeyValueCursor cursor = KeyValueCursor.Builder.newBuilder(queueSubspace) + KeyValueCursor inner = KeyValueCursor.Builder.newBuilder(queueSubspace) .setContext(context) - .setScanProperties(scanProperties) + .setScanProperties(scanProperties + .with(ExecuteProperties::clearRowAndTimeLimits) + .with(ExecuteProperties::clearSkipAndLimit) + .with(ExecuteProperties::clearState)) .setContinuation(continuation) .build(); - return cursor.map(kv -> PendingWritesQueueHelper.toQueueEntry(queueSubspace, serializer, kv)); + RecordCursor unsplitter = new SplitHelper.KeyValueUnsplitter( + context, queueSubspace, inner, + false, null, scanProperties) + .limitRowsTo(scanProperties.getExecuteProperties().getReturnedRowLimit()); + + return unsplitter.map(rawRecord -> + PendingWritesQueueHelper.toQueueEntry(serializer, rawRecord.getPrimaryKey(), rawRecord.getRawRecord())); } /** diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWritesQueueHelper.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWritesQueueHelper.java index 00c3cfbfb2..f57b9bf0bb 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWritesQueueHelper.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWritesQueueHelper.java @@ -20,14 +20,12 @@ package com.apple.foundationdb.record.lucene.directory; -import com.apple.foundationdb.KeyValue; import com.apple.foundationdb.annotation.API; import com.apple.foundationdb.record.RecordCoreArgumentException; -import com.apple.foundationdb.record.RecordCoreInternalException; +import com.apple.foundationdb.record.RecordCoreStorageException; import com.apple.foundationdb.record.lucene.LuceneDocumentFromRecord; import com.apple.foundationdb.record.lucene.LuceneIndexExpressions; import com.apple.foundationdb.record.lucene.LucenePendingWriteQueueProto; -import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.Tuple; import com.apple.foundationdb.tuple.Versionstamp; import com.google.protobuf.InvalidProtocolBufferException; @@ -40,15 +38,17 @@ @API(API.Status.INTERNAL) public final class PendingWritesQueueHelper { - public static PendingWriteQueue.QueueEntry toQueueEntry(final Subspace queueSubspace, LuceneSerializer serializer, final KeyValue kv) { + /** + * Convert a raw record back to a queue entry. + */ + public static PendingWriteQueue.QueueEntry toQueueEntry(LuceneSerializer serializer, Tuple keyTuple, byte[] valueBytes) { try { - Tuple keyTuple = queueSubspace.unpack(kv.getKey()); final Versionstamp versionstamp = keyTuple.getVersionstamp(0); - final byte[] value = serializer.decode(kv.getValue()); + final byte[] value = serializer.decode(valueBytes); LucenePendingWriteQueueProto.PendingWriteItem item = LucenePendingWriteQueueProto.PendingWriteItem.parseFrom(value); return new PendingWriteQueue.QueueEntry(versionstamp, item); } catch (InvalidProtocolBufferException e) { - throw new RecordCoreInternalException("Failed to parse queue item", e); + throw new RecordCoreStorageException("Failed to parse queue item", e); } } diff --git a/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueueTest.java b/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueueTest.java index c66d7ff305..e5d1d32948 100644 --- a/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueueTest.java +++ b/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueueTest.java @@ -32,17 +32,16 @@ import com.apple.foundationdb.record.lucene.LuceneEvents; import com.apple.foundationdb.record.lucene.LuceneIndexExpressions; import com.apple.foundationdb.record.lucene.LucenePendingWriteQueueProto; -import com.apple.foundationdb.record.provider.foundationdb.FDBExceptions; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStoreTestBase; import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.Tuple; +import com.apple.test.BooleanSource; import com.apple.test.Tags; import com.google.common.collect.Streams; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.BeforeEach; -import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -75,6 +74,7 @@ class PendingWriteQueueTest extends FDBRecordStoreTestBase { void setup() { serializer = new LuceneSerializer(true, false, null, true); } + @ParameterizedTest @EnumSource void testEnqueueAndIterate(LucenePendingWriteQueueProto.PendingWriteItem.OperationType operationType) { @@ -355,22 +355,29 @@ void testFailToDeserialize() { } } - @Test - void testQueueItemTooLarge() throws Exception { + @ParameterizedTest + @BooleanSource("useCompression") + void testLargeQueueItem(boolean useCompression) throws Exception { + // Test that we can store large queue items with and without compression StringBuilder builder = new StringBuilder(); for (int i = 0 ; i < 100_000 ; i++) { builder.append("Hello "); } String hugeString = builder.toString(); TestDocument docWithHugeString = new TestDocument(primaryKey("Huge"), - List.of(createField("f2", hugeString.toString(), LuceneIndexExpressions.DocumentFieldType.STRING, false, false))); + List.of(createField("f2", hugeString, LuceneIndexExpressions.DocumentFieldType.STRING, false, false))); - LuceneSerializer passThroughSerializer = new PassThroughLuceneSerializer(); + LuceneSerializer serializerToUse; + if (useCompression) { + serializerToUse = serializer; + } else { + serializerToUse = new PassThroughLuceneSerializer(); + } PendingWriteQueue queue; try (FDBRecordContext context = openContext()) { - queue = getQueue(context); - // save a single doc using the good queue (should succeed since the serializer compresses the data) + queue = getQueue(context, serializerToUse); + // save a single doc using the (should succeed since we split the records even for uncompressed) queue.enqueueInsert(context, docWithHugeString.getPrimaryKey(), docWithHugeString.getFields()); commit(context); } @@ -381,14 +388,6 @@ void testQueueItemTooLarge() throws Exception { assertEquals(1, list.size()); assertEquals(hugeString, list.get(0).getDocumentFields().get(0).getStringValue()); } - - try (FDBRecordContext context = openContext()) { - PendingWriteQueue failingQueue = getQueue(context, passThroughSerializer); - // save a single doc using the bad serializer (should fail as the entry will be too large) - failingQueue.enqueueInsert(context, docWithHugeString.getPrimaryKey(), docWithHugeString.getFields()); - Assertions.assertThatThrownBy(() -> commit(context)) - .isInstanceOf(FDBExceptions.FDBStoreValueSizeException.class); - } } private PendingWriteQueue getQueue(FDBRecordContext context) { From 8656bdb7efaa190ee6cae928ec66a38f5427788c Mon Sep 17 00:00:00 2001 From: ohad Date: Mon, 23 Feb 2026 15:06:22 -0500 Subject: [PATCH 06/29] Small fixes. --- .../foundationdb/record/provider/foundationdb/SplitHelper.java | 3 ++- .../record/provider/foundationdb/SplitKeyValueHelper.java | 2 +- .../provider/foundationdb/VersioningSplitKeyValueHelper.java | 2 +- 3 files changed, 4 insertions(+), 3 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java index ebfc51c54c..47473e78ab 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java @@ -144,6 +144,8 @@ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnu writeSplitRecord(context, subspace, key, serialized, splitKeyHelper, clearBasedOnPreviousSizeInfo, previousSizeInfo, sizeInfo); } else { if (splitKeyHelper.shouldClearBeforeWrite() && (splitLongRecords || previousSizeInfo == null || previousSizeInfo.isVersionedInline())) { + // Note that the clearPreviousSplitRecords also removes previous entries from cache, and in the case + // of !shouldClearBeforeWrite this is OK since the new cached version will be replaced by this one clearPreviousSplitRecord(context, subspace, key, clearBasedOnPreviousSizeInfo, previousSizeInfo); } final Tuple recordKey; @@ -347,7 +349,6 @@ private static void clearPreviousSplitRecord(@Nonnull final FDBRecordContext con tr.clear(keySplitSubspace.range()); // Clears both unsplit and previous longer split. } final byte[] versionKey = keySplitSubspace.pack(RECORD_VERSION); - // todo context.getLocalVersion(versionKey).ifPresent(localVersion -> context.removeVersionMutation(versionKey)); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyValueHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyValueHelper.java index d967a30c80..76a7a7228a 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyValueHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyValueHelper.java @@ -1,5 +1,5 @@ /* - * SplitKeyHelper.java + * SplitKeyValueHelper.java * * This source file is part of the FoundationDB open source project * diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java index 47c4d2be3e..6e2bac9017 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java @@ -1,5 +1,5 @@ /* - * VersioningSplitKeyHelper.java + * VersioningSplitKeyValueHelper.java * * This source file is part of the FoundationDB open source project * From fc24b5b62e25cc94f8d889c558a012eabab8f3ce Mon Sep 17 00:00:00 2001 From: ohad Date: Tue, 24 Feb 2026 14:01:46 -0500 Subject: [PATCH 07/29] Add more tests for large document. --- .../directory/PendingWriteQueueTest.java | 89 +++++++++++++++---- 1 file changed, 72 insertions(+), 17 deletions(-) diff --git a/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueueTest.java b/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueueTest.java index e5d1d32948..9807c82f77 100644 --- a/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueueTest.java +++ b/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueueTest.java @@ -36,6 +36,7 @@ import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStoreTestBase; import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.Tuple; +import com.apple.foundationdb.tuple.Versionstamp; import com.apple.test.BooleanSource; import com.apple.test.Tags; import com.google.common.collect.Streams; @@ -240,16 +241,16 @@ void testIterateWithContinuations() { commit(context); } - // There are 8 documents, reading with limit=3 will create 2 continuations + // There are 10 documents, reading with limit=4 will create 2 continuations List allResults = new ArrayList<>(); RecordCursorContinuation continuation; ScanProperties scanProperties = new ScanProperties(ExecuteProperties.newBuilder() - .setReturnedRowLimit(3) + .setReturnedRowLimit(4) .setIsolationLevel(IsolationLevel.SERIALIZABLE) .build()); - // First iteration - 3 elements + // First iteration - 4 elements try (FDBRecordContext context = openContext()) { final RecordCursor cursor = queue.getQueueCursor(context, scanProperties, null); final RecordCursorResult lastResult = cursor.forEachResult(result -> { @@ -257,9 +258,9 @@ void testIterateWithContinuations() { }).join(); continuation = lastResult.getContinuation(); } - assertEquals(3, allResults.size()); + assertEquals(4, allResults.size()); - // Second iteration - 3 elements + // Second iteration - 4 elements try (FDBRecordContext context = openContext()) { final RecordCursor cursor = queue.getQueueCursor(context, scanProperties, continuation.toBytes()); final RecordCursorResult lastResult = cursor.forEachResult(result -> { @@ -267,9 +268,9 @@ void testIterateWithContinuations() { }).join(); continuation = lastResult.getContinuation(); } - assertEquals(6, allResults.size()); + assertEquals(8, allResults.size()); - // Third iteration - 2 elements + // Third iteration - 2 element try (FDBRecordContext context = openContext()) { final RecordCursor cursor = queue.getQueueCursor(context, scanProperties, continuation.toBytes()); final RecordCursorResult lastResult = cursor.forEachResult(result -> { @@ -277,7 +278,7 @@ void testIterateWithContinuations() { }).join(); continuation = lastResult.getContinuation(); } - assertEquals(8, allResults.size()); + assertEquals(10, allResults.size()); // Ensure all documents show up in the results List allDocs = Streams.concat(docs.stream(), moreDocs.stream()).collect(Collectors.toList()); @@ -359,13 +360,7 @@ void testFailToDeserialize() { @BooleanSource("useCompression") void testLargeQueueItem(boolean useCompression) throws Exception { // Test that we can store large queue items with and without compression - StringBuilder builder = new StringBuilder(); - for (int i = 0 ; i < 100_000 ; i++) { - builder.append("Hello "); - } - String hugeString = builder.toString(); - TestDocument docWithHugeString = new TestDocument(primaryKey("Huge"), - List.of(createField("f2", hugeString, LuceneIndexExpressions.DocumentFieldType.STRING, false, false))); + TestDocument docWithHugeString = createHugeDocument(); LuceneSerializer serializerToUse; if (useCompression) { @@ -386,10 +381,68 @@ void testLargeQueueItem(boolean useCompression) throws Exception { RecordCursor queueCursor = queue.getQueueCursor(context, ScanProperties.FORWARD_SCAN, null); List list = queueCursor.asList().get(); assertEquals(1, list.size()); - assertEquals(hugeString, list.get(0).getDocumentFields().get(0).getStringValue()); + assertEquals(docWithHugeString.getFields().get(0).getValue(), list.get(0).getDocumentFields().get(0).getStringValue()); } } + @Test + void testLargeQueueItemDelete() { + // A split entry (>100KB) must be fully removed when clearEntry is called + TestDocument docWithHugeString = createHugeDocument(); + TestDocument normalDoc = new TestDocument(primaryKey("Normal"), + List.of(createField("f", "small", LuceneIndexExpressions.DocumentFieldType.STRING, false, false))); + + PendingWriteQueue queue; + try (FDBRecordContext context = openContext()) { + queue = getQueue(context, new PassThroughLuceneSerializer()); + queue.enqueueInsert(context, docWithHugeString.getPrimaryKey(), docWithHugeString.getFields()); + queue.enqueueInsert(context, normalDoc.getPrimaryKey(), normalDoc.getFields()); + commit(context); + } + + List entries; + try (FDBRecordContext context = openContext()) { + entries = queue.getQueueCursor(context, ScanProperties.FORWARD_SCAN, null).asList().join(); + } + assertEquals(2, entries.size()); + + // Delete the large entry + try (FDBRecordContext context = openContext()) { + queue.clearEntry(context, entries.get(0)); + commit(context); + } + + // Only the normal doc should remain + try (FDBRecordContext context = openContext()) { + List remaining = queue.getQueueCursor(context, ScanProperties.FORWARD_SCAN, null).asList().join(); + assertEquals(1, remaining.size()); + assertEquals("small", remaining.get(0).getDocumentFields().get(0).getStringValue()); + } + } + + @Test + void testClearEntryWithIncompleteVersionstamp() { + try (FDBRecordContext context = openContext()) { + PendingWriteQueue queue = getQueue(context); + // Manufacture an entry with an incomplete versionstamp + Versionstamp incomplete = Versionstamp.incomplete(0); + PendingWriteQueue.QueueEntry entryWithIncompleteStamp = new PendingWriteQueue.QueueEntry( + incomplete, + LucenePendingWriteQueueProto.PendingWriteItem.getDefaultInstance()); + Assertions.assertThatThrownBy(() -> queue.clearEntry(context, entryWithIncompleteStamp)) + .isInstanceOf(RecordCoreArgumentException.class) + .hasMessageContaining("complete"); + } + } + + @Nonnull + private TestDocument createHugeDocument() { + String hugeString = "Hello ".repeat(100_000); + TestDocument docWithHugeString = new TestDocument(primaryKey("Huge"), + List.of(createField("f", hugeString, LuceneIndexExpressions.DocumentFieldType.STRING, false, false))); + return docWithHugeString; + } + private PendingWriteQueue getQueue(FDBRecordContext context) { return getQueue(context, serializer); } @@ -506,7 +559,9 @@ private List createTestDocuments() { createField("long field", 6L, LuceneIndexExpressions.DocumentFieldType.LONG, true, false), createField("double field", 3.14D, LuceneIndexExpressions.DocumentFieldType.DOUBLE, true, true))); - return List.of(docWithNoFields, docWithOneFields, docWithMultipleFields, docWithAllFieldTypes); + TestDocument hugeDoc = createHugeDocument(); + + return List.of(docWithNoFields, docWithOneFields, docWithMultipleFields, hugeDoc, docWithAllFieldTypes); } @Nonnull From c2a3cc60c4d92e8d38b2b9a21b0637433ead7a1d Mon Sep 17 00:00:00 2001 From: ohad Date: Tue, 24 Feb 2026 14:16:40 -0500 Subject: [PATCH 08/29] Add useVersionInKey dimension to SplitHelperTestConfig Adds a useVersionInKey boolean to SplitHelperTestConfig, a keyHelper(int localVersion) factory method, and extends allValidConfigs() to produce versioning variants of all existing configs (excluding isDryRun=true, which is incompatible with versionstamps). This lays the groundwork for multi-transaction test methods that exercise VersioningSplitKeyValueHelper alongside the existing DefaultSplitKeyValueHelper tests. Co-Authored-By: Claude Sonnet 4.6 --- .../foundationdb/SplitHelperTest.java | 40 ++++++++++++++----- 1 file changed, 29 insertions(+), 11 deletions(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java index 2a6b976304..e97ba1d11c 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java @@ -37,6 +37,7 @@ import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.ByteArrayUtil; import com.apple.foundationdb.tuple.Tuple; +import com.apple.foundationdb.tuple.Versionstamp; import com.apple.test.BooleanSource; import com.apple.test.Tags; import com.google.common.collect.Lists; @@ -131,13 +132,23 @@ static class SplitHelperTestConfig { private final boolean unrollRecordDeletes; private final boolean loadViaGets; private final boolean isDryRun; + private final boolean useVersionInKey; - public SplitHelperTestConfig(boolean splitLongRecords, boolean omitUnsplitSuffix, boolean unrollRecordDeletes, boolean loadViaGets, boolean isDryRun) { + public SplitHelperTestConfig(boolean splitLongRecords, boolean omitUnsplitSuffix, boolean unrollRecordDeletes, + boolean loadViaGets, boolean isDryRun, boolean useVersionInKey) { this.splitLongRecords = splitLongRecords; this.omitUnsplitSuffix = omitUnsplitSuffix; this.unrollRecordDeletes = unrollRecordDeletes; this.loadViaGets = loadViaGets; this.isDryRun = isDryRun; + this.useVersionInKey = useVersionInKey; + } + + public SplitKeyValueHelper keyHelper(int localVersion) { + if (useVersionInKey) { + return new VersioningSplitKeyValueHelper(Versionstamp.incomplete(localVersion)); + } + return DefaultSplitKeyValueHelper.INSTANCE; } @Nonnull @@ -159,6 +170,7 @@ public String toString() { ", unrollRecordDeletes=" + unrollRecordDeletes + ", loadViaGets=" + loadViaGets + ", isDryRun=" + isDryRun + + ", useVersionInKey=" + useVersionInKey + '}'; } @@ -171,28 +183,34 @@ public boolean equals(final Object o) { return false; } final SplitHelperTestConfig that = (SplitHelperTestConfig)o; - return splitLongRecords == that.splitLongRecords && omitUnsplitSuffix == that.omitUnsplitSuffix && unrollRecordDeletes == that.unrollRecordDeletes && loadViaGets == that.loadViaGets && isDryRun == that.isDryRun; + return splitLongRecords == that.splitLongRecords && omitUnsplitSuffix == that.omitUnsplitSuffix && + unrollRecordDeletes == that.unrollRecordDeletes && loadViaGets == that.loadViaGets && + isDryRun == that.isDryRun && useVersionInKey == that.useVersionInKey; } @Override public int hashCode() { - return Objects.hash(splitLongRecords, omitUnsplitSuffix, unrollRecordDeletes, loadViaGets); + return Objects.hash(splitLongRecords, omitUnsplitSuffix, unrollRecordDeletes, loadViaGets, useVersionInKey); } public static Stream allValidConfigs() { // Note that splitLongRecords="true" && omitUnsplitSuffix="true" is not valid - return Stream.of(false, true).flatMap(splitLongRecords -> - (splitLongRecords ? Stream.of(false) : Stream.of(false, true)).flatMap(omitUnsplitSuffix -> - Stream.of(false, true).flatMap(unrollRecordDeletes -> - Stream.of(false, true).flatMap(loadViaGets -> - Stream.of(false, true).map(isDryRun -> - new SplitHelperTestConfig(splitLongRecords, omitUnsplitSuffix, unrollRecordDeletes, loadViaGets, isDryRun)))))); + // Note that useVersionInKey="true" && isDryRun="true" is not valid (versionstamp never completes without commit) + return Stream.of(false, true).flatMap(useVersionInKey -> + Stream.of(false, true).flatMap(splitLongRecords -> + (splitLongRecords ? Stream.of(false) : Stream.of(false, true)).flatMap(omitUnsplitSuffix -> + Stream.of(false, true).flatMap(unrollRecordDeletes -> + Stream.of(false, true).flatMap(loadViaGets -> + Stream.of(false, true) + .filter(isDryRun -> !useVersionInKey || !isDryRun) + .map(isDryRun -> + new SplitHelperTestConfig(splitLongRecords, omitUnsplitSuffix, unrollRecordDeletes, loadViaGets, isDryRun, useVersionInKey))))))); } public static SplitHelperTestConfig getDefault() { return new SplitHelperTestConfig(true, false, FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES.getDefaultValue(), - FDBRecordStoreProperties.LOAD_RECORDS_VIA_GETS.getDefaultValue(), false); + FDBRecordStoreProperties.LOAD_RECORDS_VIA_GETS.getDefaultValue(), false, false); } } @@ -825,7 +843,7 @@ private FDBRawRecord scanSingleRecord(@Nonnull FDBRecordContext context, boolean @ParameterizedTest(name = "scan[reverse = {0}]") @BooleanSource public void scanSingleRecords(boolean reverse) { - loadSingleRecords(new SplitHelperTestConfig(true, false, FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES.getDefaultValue(), false, false), + loadSingleRecords(new SplitHelperTestConfig(true, false, FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES.getDefaultValue(), false, false, false), (context, key, expectedSizes, expectedContents, version) -> scanSingleRecord(context, reverse, key, expectedSizes, expectedContents, version)); } From 8c260e075a3826d1f98a3987ce143053660293dd Mon Sep 17 00:00:00 2001 From: ohad Date: Tue, 24 Feb 2026 17:13:45 -0500 Subject: [PATCH 09/29] Add VersioningSplitKeyValueHelper coverage to SplitHelperTest MIME-Version: 1.0 Content-Type: text/plain; charset=UTF-8 Content-Transfer-Encoding: 8bit Fix a bug in SplitHelper.writeSplitRecord where split keys were packed using subspace.subspace(key) + Tuple.from(index), producing an inconsistent layout compared to the unsplit path for VersioningSplitKeyValueHelper. Now both call packSplitKey(subspace, key.add(index)), which VersioningSplitKeyValueHelper handles as [subspace][vs][key][index] — consistent with [subspace][vs][key][0] for unsplit records. Add SplitHelperTestConfig.useVersionInKey and keyHelper(localVersion) to drive tests through VersioningSplitKeyValueHelper. Extend allValidConfigs() to cover both helpers. Add Assumptions.assumeFalse guards where tests are incompatible with versioning keys. Introduce saveOnly + verifySuccessfullySaved helpers split from saveSuccessfully, and new saveWithSplitMultipleTransactions test that writes in tx1 and verifies in tx2, covering both default and versioning key helpers. Add parameterized defaultHelperSplitKeyEquivalence test asserting that the old call site subspace.subspace(key).pack(index) equals the new subspace.pack(key.add(index)) for DefaultSplitKeyValueHelper across all relevant key/index combinations. Co-Authored-By: Claude Sonnet 4.6 --- .../provider/foundationdb/SplitHelper.java | 3 +- .../VersioningSplitKeyValueHelper.java | 6 +- .../foundationdb/SplitHelperTest.java | 167 +++++++++++++++--- 3 files changed, 151 insertions(+), 25 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java index 47473e78ab..b66e228d52 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java @@ -170,7 +170,6 @@ private static void writeSplitRecord(@Nonnull final FDBRecordContext context, @N final SplitKeyValueHelper splitKeyHelper, final boolean clearBasedOnPreviousSizeInfo, @Nullable final FDBStoredSizes previousSizeInfo, @Nullable SizeInfo sizeInfo) { - final Subspace keySplitSubspace = subspace.subspace(key); if (splitKeyHelper.shouldClearBeforeWrite()) { clearPreviousSplitRecord(context, subspace, key, clearBasedOnPreviousSizeInfo, previousSizeInfo); } @@ -181,7 +180,7 @@ private static void writeSplitRecord(@Nonnull final FDBRecordContext context, @N if (nextOffset > serialized.length) { nextOffset = serialized.length; } - final byte[] keyBytes = splitKeyHelper.packSplitKey(keySplitSubspace, Tuple.from(index)); + final byte[] keyBytes = splitKeyHelper.packSplitKey(subspace, key.add(index)); final byte[] valueBytes = Arrays.copyOfRange(serialized, offset, nextOffset); splitKeyHelper.writeSplit(context, keyBytes, valueBytes); if (sizeInfo != null) { diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java index 6e2bac9017..0c3bd112d9 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java @@ -36,8 +36,8 @@ *
  *     [versionstamp, original-key, split-suffix]
  * 
- *

which means that the entries are sorted by their insertion order (versionstamp order), then grouped by their - * split suffixes.

+ *

which means that the entries are sorted by their insertion order (versionstamp order), then by the + * original key, with split fragments last.

*/ public class VersioningSplitKeyValueHelper implements SplitKeyValueHelper { private Versionstamp versionstamp; @@ -69,7 +69,7 @@ public boolean supportsVersionInValue() { @Override public byte[] packSplitKey(final Subspace subspace, final Tuple key) { - // This uses the same version (local and global for all the splits + // This uses the same version (local and global) for all the splits // Use versionstamp first to ensure proper sorting and since split suffix should be at the end Tuple keyTuple = Tuple.from(versionstamp).addAll(key); return subspace.packWithVersionstamp(keyTuple); diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java index e97ba1d11c..2bc8ad28f7 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java @@ -258,38 +258,55 @@ private SplitHelper.SizeInfo saveUnsuccessfully(@Nonnull F return sizeInfo; } - private SplitHelper.SizeInfo saveSuccessfully(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, - @Nullable FDBRecordVersion version, - @Nonnull SplitHelperTestConfig testConfig, - @Nullable FDBStoredSizes previousSizeInfo) { + private SplitHelper.SizeInfo saveOnly(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, + @Nullable FDBRecordVersion version, + @Nonnull SplitHelperTestConfig testConfig, + @Nullable FDBStoredSizes previousSizeInfo, + int localVersion) { final SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); SplitHelper.saveWithSplit(context, subspace, key, serialized, version, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, - DefaultSplitKeyValueHelper.INSTANCE, previousSizeInfo != null, previousSizeInfo, sizeInfo); + testConfig.keyHelper(localVersion), previousSizeInfo != null, previousSizeInfo, sizeInfo); int dataKeyCount = (serialized.length - 1) / SplitHelper.SPLIT_RECORD_SIZE + 1; boolean isSplit = dataKeyCount > 1; int keyCount = dataKeyCount; if (version != null) { keyCount += 1; } - int keySize = (subspace.pack().length + key.pack().length) * keyCount; assertEquals(isSplit, sizeInfo.isSplit()); assertEquals(keyCount, sizeInfo.getKeyCount()); - if (testConfig.hasSplitPoints()) { - // Add in the the counters the split points. - if (!isSplit) { - keySize += 1; // As 0 requires 1 byte when Tuple packed - } else { - keySize += dataKeyCount * 2; // As each split point is two bytes when tuple packed - } - } - if (version != null) { - keySize += 2; - } int valueSize = serialized.length + (version != null ? 1 + FDBRecordVersion.VERSION_LENGTH : 0); - assertEquals(keySize, sizeInfo.getKeySize()); assertEquals(valueSize, sizeInfo.getValueSize()); assertEquals(version != null, sizeInfo.isVersionedInline()); + if (!testConfig.useVersionInKey) { + // Key size can only be asserted when the key is fully known at write time (not the case for versioning keys, + // since the versionstamp in the key is not yet resolved until after commit) + int keySize = (subspace.pack().length + key.pack().length) * keyCount; + if (testConfig.hasSplitPoints()) { + // Add in the counters the split points. + if (!isSplit) { + keySize += 1; // As 0 requires 1 byte when Tuple packed + } else { + keySize += dataKeyCount * 2; // As each split point is two bytes when tuple packed + } + } + if (version != null) { + keySize += 2; + } + assertEquals(keySize, sizeInfo.getKeySize()); + } + return sizeInfo; + } + private void verifySuccessfullySaved(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, + @Nullable FDBRecordVersion version, + @Nonnull SplitHelperTestConfig testConfig) { + // Similar to the calculation in saveOnly + int dataKeyCount = (serialized.length - 1) / SplitHelper.SPLIT_RECORD_SIZE + 1; + boolean isSplit = dataKeyCount > 1; + int keyCount = dataKeyCount; + if (version != null) { + keyCount += 1; + } final Subspace keySubspace = subspace.subspace(key); RecordCursorIterator kvCursor = KeyValueCursor.Builder.withSubspace(keySubspace) .setContext(context) @@ -349,7 +366,17 @@ private SplitHelper.SizeInfo saveSuccessfully(@Nonnull FDBRecordContext context, } else { assertNull(versionBytes); } + assertNotNull(valueBytes); + assertArrayEquals(serialized, valueBytes); + } + private SplitHelper.SizeInfo saveSuccessfully(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, + @Nullable FDBRecordVersion version, + @Nonnull SplitHelperTestConfig testConfig, + @Nullable FDBStoredSizes previousSizeInfo) { + // localVersion=0: single-transaction callers always use useVersionInKey=false, so the value doesn't matter + SplitHelper.SizeInfo sizeInfo = saveOnly(context, key, serialized, version, testConfig, previousSizeInfo, 0); + verifySuccessfullySaved(context, key, serialized, version, testConfig); return sizeInfo; } @@ -427,10 +454,43 @@ private SplitHelper.SizeInfo saveWithSplit(@Nonnull FDBRecordContext context, @N return saveWithSplit(context, key, serialized, null, testConfig); } + public static Stream splitKeyEquivalenceCases() { + return Stream.of( + Arguments.of(Tuple.from(1066L), SplitHelper.START_SPLIT_RECORD), + Arguments.of(Tuple.from(1066L), SplitHelper.START_SPLIT_RECORD + 1), + Arguments.of(Tuple.from(1066L), SplitHelper.START_SPLIT_RECORD + 2), + Arguments.of(Tuple.from(1066L), SplitHelper.UNSPLIT_RECORD), + Arguments.of(Tuple.from(1066L), SplitHelper.RECORD_VERSION), + Arguments.of(Tuple.from(1066L, "extra"), SplitHelper.START_SPLIT_RECORD), + Arguments.of(Tuple.from(), SplitHelper.START_SPLIT_RECORD), + Arguments.of(Tuple.from(), SplitHelper.UNSPLIT_RECORD) + ); + } + + /** + * Verify that the refactored split call site in {@link SplitHelper} — which now passes + * {@code subspace} + {@code key.add(index)} to {@code packSplitKey} instead of the old + * {@code subspace.subspace(key)} + {@code Tuple.from(index)} — produces identical byte keys + * for {@link DefaultSplitKeyValueHelper}. + */ + @MethodSource("splitKeyEquivalenceCases") + @ParameterizedTest(name = "defaultHelperSplitKeyEquivalence[key={0}, index={1}]") + void defaultHelperSplitKeyEquivalence(Tuple key, long index) { + Subspace subspace = new Subspace(Tuple.from("test")); + + // Old call site in SplitHelper: subspace.subspace(key).pack(index) + byte[] oldKey = subspace.subspace(key).pack(index); + // New call site: packSplitKey(subspace, key.add(index)) = subspace.pack(key.add(index)) + byte[] newKey = DefaultSplitKeyValueHelper.INSTANCE.packSplitKey(subspace, key.add(index)); + + assertArrayEquals(oldKey, newKey); + } + @MethodSource("testConfigs") @ParameterizedTest(name = "saveWithSplit[{0}]") public void saveWithSplit(@Nonnull SplitHelperTestConfig testConfig) { this.testConfig = testConfig; + Assumptions.assumeFalse(testConfig.useVersionInKey); try (FDBRecordContext context = openContext()) { // No version FDBStoredSizes sizes1 = saveWithSplit(context, Tuple.from(1066L), SHORT_STRING, testConfig); @@ -448,13 +508,78 @@ public void saveWithSplit(@Nonnull SplitHelperTestConfig testConfig) { } } + @MethodSource("testConfigs") + @ParameterizedTest(name = "saveWithSplitMultipleTransactions[{0}]") + public void saveWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig testConfig) { + this.testConfig = testConfig; + + final Tuple key1 = Tuple.from(1066L); + final Tuple key2 = Tuple.from(1415L); + final Tuple key3 = Tuple.from(1776L); + final int localVersion1; + final int localVersion2; + final int localVersion3; + final byte[] globalVs; + try (FDBRecordContext context = openContext()) { + localVersion1 = context.claimLocalVersion(); + saveWithSplitForMultipleTransactions(context, key1, SHORT_STRING, null, testConfig, null, localVersion1); + localVersion2 = context.claimLocalVersion(); + saveWithSplitForMultipleTransactions(context, key2, LONG_STRING, null, testConfig, null, localVersion2); + localVersion3 = context.claimLocalVersion(); + saveWithSplitForMultipleTransactions(context, key3, VERY_LONG_STRING, null, testConfig, null, localVersion3); + commit(context); + globalVs = context.getVersionStamp(); + } + + final Tuple verifyKey1; + final Tuple verifyKey2; + final Tuple verifyKey3; + if (testConfig.useVersionInKey) { + verifyKey1 = Tuple.from(Versionstamp.complete(globalVs, localVersion1)).addAll(key1); + verifyKey2 = Tuple.from(Versionstamp.complete(globalVs, localVersion2)).addAll(key2); + verifyKey3 = Tuple.from(Versionstamp.complete(globalVs, localVersion3)).addAll(key3); + } else { + verifyKey1 = key1; + verifyKey2 = key2; + verifyKey3 = key3; + } + if (!testConfig.isDryRun) { + try (FDBRecordContext context = openContext()) { + verifySuccessfullySaved(context, verifyKey1, SHORT_STRING, null, testConfig); + if (testConfig.splitLongRecords) { + verifySuccessfullySaved(context, verifyKey2, LONG_STRING, null, testConfig); + verifySuccessfullySaved(context, verifyKey3, VERY_LONG_STRING, null, testConfig); + } + } + } + } + + private SplitHelper.SizeInfo saveWithSplitForMultipleTransactions(@Nonnull FDBRecordContext context, @Nonnull Tuple key, + byte[] serialized, + @Nullable FDBRecordVersion version, + @Nonnull SplitHelperTestConfig testConfig, + @Nullable FDBStoredSizes previousSizeInfo, + int localVersion) { + if (testConfig.omitUnsplitSuffix && version != null) { + return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, + RecordCoreArgumentException.class, "Cannot include version"); + } else if (!testConfig.splitLongRecords && serialized.length > SplitHelper.SPLIT_RECORD_SIZE) { + return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, + RecordCoreException.class, "Record is too long"); + } else if (testConfig.isDryRun) { + return dryRunSetSizeInfo(context, key, serialized, version, testConfig, previousSizeInfo); + } else { + return saveOnly(context, key, serialized, version, testConfig, previousSizeInfo, localVersion); + } + } + @MethodSource("testConfigs") @ParameterizedTest(name = "saveWithSplitAndIncompleteVersions[{0}]") public void saveWithSplitAndIncompleteVersions(SplitHelperTestConfig testConfig) { this.testConfig = testConfig; + Assumptions.assumeFalse(testConfig.useVersionInKey); final byte[] versionstamp; try (FDBRecordContext context = openContext()) { - // With incomplete version saveWithSplit(context, Tuple.from(962L), SHORT_STRING, FDBRecordVersion.incomplete(context.claimLocalVersion()), testConfig); saveWithSplit(context, Tuple.from(967L), LONG_STRING, FDBRecordVersion.incomplete(context.claimLocalVersion()), testConfig); saveWithSplit(context, Tuple.from(996L), VERY_LONG_STRING, FDBRecordVersion.incomplete(context.claimLocalVersion()), testConfig); @@ -495,8 +620,8 @@ public void saveWithSplitAndIncompleteVersions(SplitHelperTestConfig testConfig) @ParameterizedTest(name = "saveWithSplitAndCompleteVersion[{0}]") public void saveWithSplitAndCompleteVersions(SplitHelperTestConfig testConfig) { this.testConfig = testConfig; + Assumptions.assumeFalse(testConfig.useVersionInKey); try (FDBRecordContext context = openContext()) { - // With complete version byte[] globalVersion = "karlgrosse".getBytes(StandardCharsets.US_ASCII); saveWithSplit(context, Tuple.from(800L), SHORT_STRING, FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()), testConfig); saveWithSplit(context, Tuple.from(813L), LONG_STRING, FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()), testConfig); @@ -614,6 +739,7 @@ static Stream deleteWithSplitAndVersion() { @MethodSource("testConfigs") public void deleteWithSplitAndVersion(SplitHelperTestConfig testConfig) { this.testConfig = testConfig; + Assumptions.assumeFalse(testConfig.useVersionInKey); Assumptions.assumeFalse(testConfig.omitUnsplitSuffix); final byte[] globalVersion = "chrysan_th".getBytes(StandardCharsets.US_ASCII); try (FDBRecordContext context = openContext()) { @@ -787,6 +913,7 @@ private FDBRawRecord loadWithSplit(@Nonnull FDBRecordContext context, @Nonnull T @ParameterizedTest(name = "loadWithSplit[{0}]") public void loadWithSplit(SplitHelperTestConfig testConfig) { this.testConfig = testConfig; + Assumptions.assumeFalse(testConfig.useVersionInKey); loadSingleRecords(testConfig, (context, key, expectedSizes, expectedContents, version) -> loadWithSplit(context, key, testConfig, expectedSizes, expectedContents, version)); From 493a40f4d30e8145e63f5705e14cdd9bc7487133 Mon Sep 17 00:00:00 2001 From: ohad Date: Wed, 25 Feb 2026 14:56:26 -0500 Subject: [PATCH 10/29] Refactor writeDummyRecord/writeDummyKV and add deleteWithSplitMultipleTransactions - Add useVersionInKey and localVersion parameters to writeDummyRecord, threading both through to writeDummyKV - writeDummyKV now handles versionstamp prepending and FDB write dispatch (SET_VERSIONSTAMPED_KEY vs plain set) based on useVersionInKey - Remove writeDummyVersionedRecord (now redundant with writeDummyRecord) - Add deleteWithSplitMultipleTransactions: write in tx1, delete in tx2, verify empty in tx3, covering both default and versioning key helpers Co-Authored-By: Claude Sonnet 4.6 --- .../foundationdb/SplitHelperTest.java | 132 +++++++++++++++--- 1 file changed, 115 insertions(+), 17 deletions(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java index 2bc8ad28f7..45825472c9 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java @@ -21,8 +21,8 @@ package com.apple.foundationdb.record.provider.foundationdb; import com.apple.foundationdb.KeyValue; +import com.apple.foundationdb.MutationType; import com.apple.foundationdb.ReadTransaction; -import com.apple.foundationdb.Transaction; import com.apple.foundationdb.record.ExecuteProperties; import com.apple.foundationdb.record.FDBRecordStoreProperties; import com.apple.foundationdb.record.RecordCoreArgumentException; @@ -646,47 +646,61 @@ public void saveWithSplitAndCompleteVersions(SplitHelperTestConfig testConfig) { } @Nonnull - private FDBStoredSizes writeDummyRecord(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @Nullable FDBRecordVersion version, int splits, boolean omitUnsplitSuffix) { - final Transaction tr = context.ensureActive(); + private FDBStoredSizes writeDummyRecord(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @Nullable FDBRecordVersion version, int splits, boolean omitUnsplitSuffix, boolean useVersionInKey, int localVersion) { SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); if (version != null) { assertThat(omitUnsplitSuffix, is(false)); sizeInfo.setVersionedInline(true); - byte[] keyBytes = subspace.pack(key.add(SplitHelper.RECORD_VERSION)); + Tuple keyTuple = key.add(SplitHelper.RECORD_VERSION); byte[] valueBytes = SplitHelper.packVersion(version); - tr.set(keyBytes, valueBytes); - sizeInfo.add(keyBytes, valueBytes); + // Note that this will not mutate the version in the value + writeDummyKV(context, keyTuple, valueBytes, sizeInfo, useVersionInKey, localVersion); } if (splits == 1) { if (omitUnsplitSuffix) { - byte[] keyBytes = subspace.pack(key); - sizeInfo.add(keyBytes, SHORT_STRING); - tr.set(keyBytes, SHORT_STRING); + Tuple keyTuple = key; + byte[] valueBytes = SHORT_STRING; + writeDummyKV(context, keyTuple, valueBytes, sizeInfo, useVersionInKey, localVersion); } else { - byte[] keyBytes = subspace.pack(key.add(SplitHelper.UNSPLIT_RECORD)); - sizeInfo.add(keyBytes, SHORT_STRING); - tr.set(keyBytes, SHORT_STRING); + Tuple keyTuple = key.add(SplitHelper.UNSPLIT_RECORD); + byte[] valueBytes = SHORT_STRING; + writeDummyKV(context, keyTuple, valueBytes, sizeInfo, useVersionInKey, localVersion); } sizeInfo.setSplit(false); } else { for (int i = 0; i < splits; i++) { - byte[] keyBytes = subspace.pack(key.add(SplitHelper.START_SPLIT_RECORD + i)); - sizeInfo.add(keyBytes, SHORT_STRING); - tr.set(keyBytes, SHORT_STRING); + Tuple keyTuple = key.add(SplitHelper.START_SPLIT_RECORD + i); + byte[] valueBytes = SHORT_STRING; + writeDummyKV(context, keyTuple, valueBytes, sizeInfo, useVersionInKey, localVersion); } sizeInfo.setSplit(true); } return sizeInfo; } + private void writeDummyKV(@Nonnull FDBRecordContext context, @Nonnull Tuple keyTuple, + byte[] valueBytes, @Nonnull SplitHelper.SizeInfo sizeInfo, boolean useVersionInKey, int localVersion) { + byte[] keyBytes; + // Mimic the work done in both SplitKeyValueHelper + if (useVersionInKey) { + Tuple versionedKeyTuple = Tuple.from(Versionstamp.incomplete(localVersion)).addAll(keyTuple); + keyBytes = subspace.packWithVersionstamp(versionedKeyTuple); + context.addVersionMutation(MutationType.SET_VERSIONSTAMPED_KEY, keyBytes, valueBytes); + } else { + keyBytes = subspace.pack(keyTuple); + context.ensureActive().set(keyBytes, valueBytes); + } + sizeInfo.add(keyBytes, valueBytes); + } + @Nonnull private FDBStoredSizes writeDummyRecord(@Nonnull FDBRecordContext context, @Nonnull Tuple key, int splits, boolean omitUnsplitSuffix) { - return writeDummyRecord(context, key, null, splits, omitUnsplitSuffix); + return writeDummyRecord(context, key, null, splits, omitUnsplitSuffix, false, 0); } @Nonnull private FDBStoredSizes writeDummyRecord(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @Nonnull FDBRecordVersion version, int splits) { - return writeDummyRecord(context, key, version, splits, false); + return writeDummyRecord(context, key, version, splits, false, false, 0); } private void deleteSplit(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @@ -729,6 +743,90 @@ public void deleteWithSplit(SplitHelperTestConfig testConfig) { } } + @MethodSource("testConfigs") + @ParameterizedTest(name = "deleteWithSplitMultipleTransactions[{0}]") + public void deleteWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig testConfig) { + this.testConfig = testConfig; + + final Tuple key1 = Tuple.from(-660L); + final Tuple key2 = Tuple.from(-581L); + final Tuple key3 = Tuple.from(-549L); + final Tuple key4 = Tuple.from(-510L); + + // tx1: write records + final int localVersion1; + final int localVersion2; + final int localVersion3; + final int localVersion4; + final byte[] globalVs; + try (FDBRecordContext context = openContext()) { + if (testConfig.useVersionInKey) { + localVersion1 = context.claimLocalVersion(); + writeDummyRecord(context, key1, null, 1, testConfig.omitUnsplitSuffix, true, localVersion1); + localVersion2 = context.claimLocalVersion(); + writeDummyRecord(context, key2, null, 1, testConfig.omitUnsplitSuffix, true, localVersion2); + if (testConfig.splitLongRecords) { + localVersion3 = context.claimLocalVersion(); + writeDummyRecord(context, key3, null, 5, testConfig.omitUnsplitSuffix, true, localVersion3); + localVersion4 = context.claimLocalVersion(); + writeDummyRecord(context, key4, null, 5, testConfig.omitUnsplitSuffix, true, localVersion4); + } else { + localVersion3 = -1; + localVersion4 = -1; + } + } else { + writeDummyRecord(context, key1, 1, testConfig.omitUnsplitSuffix); + writeDummyRecord(context, key2, 1, testConfig.omitUnsplitSuffix); + localVersion1 = localVersion2 = localVersion3 = localVersion4 = -1; + if (testConfig.splitLongRecords) { + writeDummyRecord(context, key3, 5, testConfig.omitUnsplitSuffix); + writeDummyRecord(context, key4, 5, testConfig.omitUnsplitSuffix); + } + } + commit(context); + globalVs = context.getVersionStamp(); + } + + // Resolve the delete keys: for versioning helper the stored key is [vs][originalKey] + final Tuple deleteKey1; + final Tuple deleteKey2; + final Tuple deleteKey3; + final Tuple deleteKey4; + if (testConfig.useVersionInKey) { + deleteKey1 = Tuple.from(Versionstamp.complete(globalVs, localVersion1)).addAll(key1); + deleteKey2 = Tuple.from(Versionstamp.complete(globalVs, localVersion2)).addAll(key2); + deleteKey3 = testConfig.splitLongRecords ? Tuple.from(Versionstamp.complete(globalVs, localVersion3)).addAll(key3) : null; + deleteKey4 = testConfig.splitLongRecords ? Tuple.from(Versionstamp.complete(globalVs, localVersion4)).addAll(key4) : null; + } else { + deleteKey1 = key1; + deleteKey2 = key2; + deleteKey3 = key3; + deleteKey4 = key4; + } + + // tx2: delete records + try (FDBRecordContext context = openContext()) { + SplitHelper.deleteSplit(context, subspace, deleteKey1, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, false, null); + SplitHelper.deleteSplit(context, subspace, deleteKey2, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, false, null); + if (testConfig.splitLongRecords) { + SplitHelper.deleteSplit(context, subspace, deleteKey3, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, false, null); + SplitHelper.deleteSplit(context, subspace, deleteKey4, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, false, null); + } + commit(context); + } + + // tx3: verify subspace is empty + try (FDBRecordContext context = openContext()) { + int count = KeyValueCursor.Builder.withSubspace(subspace) + .setContext(context) + .setScanProperties(ScanProperties.FORWARD_SCAN) + .build() + .getCount() + .join(); + assertEquals(0, count); + } + } + static Stream deleteWithSplitAndVersion() { return Stream.of(false, true).flatMap(splitLongRecords -> Stream.of(false, true).map(unrollSingleRecordDeletes -> From 57f960db1beed34117982ca266311081fddfb668 Mon Sep 17 00:00:00 2001 From: ohad Date: Wed, 25 Feb 2026 17:58:54 -0500 Subject: [PATCH 11/29] Add scanMultipleRecordsWithVersioningKey and writeDummyRecordsMultipleTransactions - Add writeDummyRecordsMultipleTransactions(boolean useVersionInKey): writes 50 records in tx1 (with FDBRecordVersion when useVersionInKey=false, with versionstamp keys when true), then scans back in tx2 to build expected FDBRawRecord list with accurate committed sizes - Add scanMultipleRecordsWithVersioningKey: parallel to scanMultipleRecords, uses writeDummyRecordsMultipleTransactions(true) for setup Co-Authored-By: Claude Sonnet 4.6 --- .../foundationdb/SplitHelperTest.java | 83 ++++++++++++++++++- 1 file changed, 82 insertions(+), 1 deletion(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java index 45825472c9..a5ff575f24 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java @@ -1084,7 +1084,7 @@ private List writeDummyRecords() { FDBRecordVersion version = (i % 2 == 0) ? FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()) : null; byte[] rawBytes = (i % 4 < 2) ? SHORT_STRING : MEDIUM_STRING; Tuple key = Tuple.from(currKey); - FDBStoredSizes sizes = writeDummyRecord(context, key, version, (i % 4 < 2) ? 1 : MEDIUM_COPIES, false); + FDBStoredSizes sizes = writeDummyRecord(context, key, version, (i % 4 < 2) ? 1 : MEDIUM_COPIES, false, false, 0); rawRecords.add(new FDBRawRecord(key, rawBytes, version, sizes)); long temp = currKey + nextKey; @@ -1098,6 +1098,60 @@ private List writeDummyRecords() { return rawRecords; } + private List writeDummyRecordsMultipleTransactions(boolean useVersionInKey) { + final byte[] globalVersion = "_cushions_".getBytes(StandardCharsets.US_ASCII); + // Generate primary keys using a generalization of the Fibonacci formula: https://oeis.org/A247698 + long currKey = 2308L; + long nextKey = 4261L; + + final Tuple[] keys = new Tuple[50]; + final byte[][] rawBytesArr = new byte[50][]; + final FDBRecordVersion[] versions = new FDBRecordVersion[50]; + final int[] localVersions = new int[50]; + + final byte[] globalVs; + try (FDBRecordContext context = openContext()) { + for (int i = 0; i < 50; i++) { + keys[i] = Tuple.from(currKey); + rawBytesArr[i] = (i % 4 < 2) ? SHORT_STRING : MEDIUM_STRING; + versions[i] = (!useVersionInKey && i % 2 == 0) ? FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()) : null; + localVersions[i] = useVersionInKey ? context.claimLocalVersion() : 0; + writeDummyRecord(context, keys[i], versions[i], (i % 4 < 2) ? 1 : MEDIUM_COPIES, false, useVersionInKey, localVersions[i]); + + long temp = currKey + nextKey; + currKey = nextKey; + nextKey = temp; + } + commit(context); + globalVs = context.getVersionStamp(); + } + + // Scan back in a second transaction to build expected FDBRawRecord list with accurate committed sizes + final List rawRecords = new ArrayList<>(); + try (FDBRecordContext context = openContext()) { + KeyValueCursor kvCursor = KeyValueCursor.Builder.withSubspace(subspace) + .setContext(context) + .setRange(TupleRange.ALL) + .setScanProperties(ScanProperties.FORWARD_SCAN) + .build(); + List scannedRecords = new SplitHelper.KeyValueUnsplitter(context, subspace, kvCursor, false, null, ScanProperties.FORWARD_SCAN) + .asList().join(); + for (int i = 0; i < 50; i++) { + Tuple expectedKey = useVersionInKey + ? Tuple.from(Versionstamp.complete(globalVs, localVersions[i])).addAll(keys[i]) + : keys[i]; + FDBRawRecord scanned = scannedRecords.stream() + .filter(r -> r.getPrimaryKey().equals(expectedKey)) + .findFirst() + .orElseThrow(() -> new AssertionError("Missing record for key " + expectedKey)); + rawRecords.add(new FDBRawRecord(expectedKey, rawBytesArr[i], versions[i], scanned)); + } + commit(context); + } + + return rawRecords; + } + @ParameterizedTest(name = "scanMultipleRecords[reverse = {0}]") @BooleanSource public void scanMultipleRecords(boolean reverse) { @@ -1125,6 +1179,33 @@ public void scanMultipleRecords(boolean reverse) { } } + @ParameterizedTest(name = "scanMultipleRecordsMultipleTransactions[reverse = {0}]") + @BooleanSource + void scanMultipleRecordsMultipleTransactions(boolean reverse) { + final ScanProperties scanProperties = reverse ? ScanProperties.REVERSE_SCAN : ScanProperties.FORWARD_SCAN; + List rawRecords = writeDummyRecordsMultipleTransactions(true); + + try (FDBRecordContext context = openContext()) { + KeyValueCursor kvCursor = KeyValueCursor.Builder.withSubspace(subspace) + .setContext(context) + .setRange(TupleRange.ALL) + .setScanProperties(scanProperties) + .build(); + List readRecords = new SplitHelper.KeyValueUnsplitter(context, subspace, kvCursor, false, null, scanProperties) + .asList().join(); + if (reverse) { + readRecords = Lists.reverse(readRecords); + } + assertEquals(rawRecords.size(), readRecords.size()); + for (int i = 0; i < rawRecords.size(); i++) { + assertEquals(rawRecords.get(i), readRecords.get(i)); + } + assertEquals(rawRecords, readRecords); + + commit(context); + } + } + @MethodSource("limitsAndReverseArgs") @ParameterizedTest(name = "scanContinuations [returnLimit = {0}, readLimit = {1}, reverse = {2}]") public void scanContinuations(final int returnLimit, final int readLimit, final boolean reverse) { From 84598da6a3d5512cb041fa06f3f948578ff82213 Mon Sep 17 00:00:00 2001 From: ohad Date: Wed, 25 Feb 2026 18:12:49 -0500 Subject: [PATCH 12/29] Add scanContinuationsMultipleTransactions Parallel to scanContinuations, uses writeDummyRecordsMultipleTransactions(true) for setup so records are written with versionstamp keys and read back in a separate transaction with accurate committed sizes. Co-Authored-By: Claude Sonnet 4.6 --- .../foundationdb/SplitHelperTest.java | 76 +++++++++++++++++++ 1 file changed, 76 insertions(+) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java index a5ff575f24..72f624a096 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java @@ -1281,4 +1281,80 @@ public void scanContinuations(final int returnLimit, final int readLimit, final commit(context); } } + + @MethodSource("limitsAndReverseArgs") + @ParameterizedTest(name = "scanContinuationsMultipleTransactions [returnLimit = {0}, readLimit = {1}, reverse = {2}]") + void scanContinuationsMultipleTransactions(final int returnLimit, final int readLimit, final boolean reverse) { + List rawRecords = writeDummyRecordsMultipleTransactions(true); + if (reverse) { + rawRecords = Lists.reverse(rawRecords); + } + final Iterator expectedRecordIterator = rawRecords.iterator(); + + try (FDBRecordContext context = openContext()) { + byte[] continuation = null; + + do { + final ExecuteProperties executeProperties = ExecuteProperties.newBuilder() + .setReturnedRowLimit(returnLimit) + .setScannedRecordsLimit(readLimit) + .build(); + ScanProperties scanProperties = new ScanProperties(executeProperties, reverse); + RecordCursor kvCursor = KeyValueCursor.Builder.withSubspace(subspace) + .setContext(context) + .setRange(TupleRange.ALL) + .setScanProperties(scanProperties.with(ExecuteProperties::clearRowAndTimeLimits).with(ExecuteProperties::clearState)) + .setContinuation(continuation) + .build(); + RecordCursorIterator recordCursor = new SplitHelper.KeyValueUnsplitter(context, subspace, kvCursor, false, null, scanProperties.with(ExecuteProperties::clearReturnedRowLimit)) + .limitRowsTo(returnLimit) + .asIterator(); + + int retrieved = 0; + int rowsScanned = 0; + while (recordCursor.hasNext()) { + assertThat(retrieved, lessThan(returnLimit)); + assertThat(rowsScanned, lessThanOrEqualTo(readLimit)); + + FDBRawRecord nextRecord = recordCursor.next(); + assertNotNull(nextRecord); + assertThat(expectedRecordIterator.hasNext(), is(true)); + FDBRawRecord expectedRecord = expectedRecordIterator.next(); + assertEquals(expectedRecord, nextRecord); + + rowsScanned += nextRecord.getKeyCount(); + retrieved += 1; + } + + if (retrieved > 0) { + continuation = recordCursor.getContinuation(); + if (retrieved >= returnLimit) { + assertEquals(RecordCursor.NoNextReason.RETURN_LIMIT_REACHED, recordCursor.getNoNextReason()); + assertNotNull(continuation); + } else if (rowsScanned > readLimit) { + assertEquals(RecordCursor.NoNextReason.SCAN_LIMIT_REACHED, recordCursor.getNoNextReason()); + assertNotNull(continuation); + } else if (rowsScanned < readLimit) { + assertEquals(RecordCursor.NoNextReason.SOURCE_EXHAUSTED, recordCursor.getNoNextReason()); + } else { + // If we read exactly as many records as is allowed by the read record limit, then + // this probably means that we hit SCAN_LIMIT_REACHED, but it's also possible to + // hit SOURCE_EXHAUSTED if we hit the record read limit at exactly the same time + // as we needed to do another speculative read to determine if a split record + // continues or not. + assertEquals(readLimit, rowsScanned); + assertThat(recordCursor.getNoNextReason(), is(oneOf(RecordCursor.NoNextReason.SCAN_LIMIT_REACHED, RecordCursor.NoNextReason.SOURCE_EXHAUSTED))); + if (!recordCursor.getNoNextReason().isSourceExhausted()) { + assertNotNull(recordCursor.getContinuation()); + } + } + } else { + assertNull(recordCursor.getContinuation()); + continuation = null; + } + } while (continuation != null); + + commit(context); + } + } } From 7071e4083d49c7afbc175d027a5ee8bf61e63f8d Mon Sep 17 00:00:00 2001 From: ohad Date: Wed, 25 Feb 2026 19:02:06 -0500 Subject: [PATCH 13/29] cleanup comments --- .../provider/foundationdb/DefaultSplitKeyValueHelper.java | 2 +- .../record/provider/foundationdb/FDBRawRecord.java | 2 ++ .../record/provider/foundationdb/SplitHelper.java | 6 ++++-- .../record/provider/foundationdb/SplitKeyValueHelper.java | 4 ++-- .../foundationdb/VersioningSplitKeyValueHelper.java | 1 + 5 files changed, 10 insertions(+), 5 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/DefaultSplitKeyValueHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/DefaultSplitKeyValueHelper.java index afd627f492..308977823b 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/DefaultSplitKeyValueHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/DefaultSplitKeyValueHelper.java @@ -42,7 +42,7 @@ public boolean shouldClearBeforeWrite() { /** * Value can mutate version stamp with this helper. - * @return true if the SplitHelper should mutate a version in the value + * @return true since the SplitHelper should mutate a version in the value */ @Override public boolean supportsVersionInValue() { diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRawRecord.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRawRecord.java index 238a9457e0..a393d84bf0 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRawRecord.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRawRecord.java @@ -20,6 +20,7 @@ package com.apple.foundationdb.record.provider.foundationdb; +import com.apple.foundationdb.annotation.API; import com.apple.foundationdb.tuple.Tuple; import javax.annotation.Nonnull; @@ -33,6 +34,7 @@ * any splits have been removed), and its version. It also includes sizing information describing * the record's on-disk footprint. */ +@API(API.Status.INTERNAL) public class FDBRawRecord implements FDBStoredSizes { @Nonnull private final Tuple primaryKey; @Nonnull private final byte[] rawRecord; diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java index b66e228d52..2150af7d0d 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java @@ -144,8 +144,9 @@ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnu writeSplitRecord(context, subspace, key, serialized, splitKeyHelper, clearBasedOnPreviousSizeInfo, previousSizeInfo, sizeInfo); } else { if (splitKeyHelper.shouldClearBeforeWrite() && (splitLongRecords || previousSizeInfo == null || previousSizeInfo.isVersionedInline())) { - // Note that the clearPreviousSplitRecords also removes previous entries from cache, and in the case - // of !shouldClearBeforeWrite this is OK since the new cached version will be replaced by this one + // Note that the clearPreviousSplitRecords also removes version splits from the context cache + // We can skip this for the shouldClearBeforeWrite==false case since we don't have a version split either + // TODO: Do we need to clear the cache for the in-transaction keyWithValues with the same local version? clearPreviousSplitRecord(context, subspace, key, clearBasedOnPreviousSizeInfo, previousSizeInfo); } final Tuple recordKey; @@ -206,6 +207,7 @@ private static void writeVersion(@Nonnull final FDBRecordContext context, @Nonnu return; } if (!splitKeyHelper.supportsVersionInValue()) { + // Cannot write version in the k/v value since it is not supported by this helper (e.g. version is needed in the key) throw new RecordCoreInternalException("Split version is not supported for this helper"); } final byte[] keyBytes = splitKeyHelper.packSplitKey(subspace, key.add(RECORD_VERSION)); diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyValueHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyValueHelper.java index 76a7a7228a..f1500112d3 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyValueHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyValueHelper.java @@ -30,14 +30,14 @@ public interface SplitKeyValueHelper { /** * Whether the {@link SplitHelper} should clear older entries before writing new ones. * In the cases where old splits may exist and overlap with the new entries, the SplitHelper should clear the - * subspace before writing any new splits. THis method is used to determine whether this is necessary. + * subspace before writing any new splits. This method is used to determine whether this is necessary. * @return TRUE if the SplitHelper should clear the subspace before writing new splits. */ boolean shouldClearBeforeWrite(); /** * Whether the helper allows version mutation in the values. - * There are cases where the value cannot have version mutation, for example, when the needs to have one. + * There are cases where the value cannot have version mutation, for example, when the key needs to have one. * @return true if version mutations are allowed for the value of the k/v pair, false if not */ boolean supportsVersionInValue(); diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java index 0c3bd112d9..505ab205c7 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java @@ -51,6 +51,7 @@ public VersioningSplitKeyValueHelper(final Versionstamp versionstamp) { * Since the key has a unique component (version), no conflicts are expected, so no need to clean before saving new splits. * Furthermore, since the key contains a version stamp, we don't know the actual key contents ahead of committing * the transaction, and so no clean can be done. + * TODO: We may need to clear the local transaction version mutation cache * @return false, as new keys should not interfere with old ones. */ @Override From 0f89ece8107d9d4a047cda728538c1c1d374b468 Mon Sep 17 00:00:00 2001 From: ohad Date: Wed, 25 Feb 2026 21:01:28 -0500 Subject: [PATCH 14/29] Add tests for the case where we overwrite same record with version in key, added comments --- .../provider/foundationdb/SplitHelper.java | 5 +-- .../VersioningSplitKeyValueHelper.java | 7 +++- .../foundationdb/SplitHelperTest.java | 34 +++++++++++++++++++ 3 files changed, 43 insertions(+), 3 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java index 2150af7d0d..dac464709a 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java @@ -145,8 +145,7 @@ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnu } else { if (splitKeyHelper.shouldClearBeforeWrite() && (splitLongRecords || previousSizeInfo == null || previousSizeInfo.isVersionedInline())) { // Note that the clearPreviousSplitRecords also removes version splits from the context cache - // We can skip this for the shouldClearBeforeWrite==false case since we don't have a version split either - // TODO: Do we need to clear the cache for the in-transaction keyWithValues with the same local version? + // This is not currently supported for the case where we have versions in the keys since we can't trace the old values down clearPreviousSplitRecord(context, subspace, key, clearBasedOnPreviousSizeInfo, previousSizeInfo); } final Tuple recordKey; @@ -172,6 +171,8 @@ private static void writeSplitRecord(@Nonnull final FDBRecordContext context, @N final boolean clearBasedOnPreviousSizeInfo, @Nullable final FDBStoredSizes previousSizeInfo, @Nullable SizeInfo sizeInfo) { if (splitKeyHelper.shouldClearBeforeWrite()) { + // Note that the clearPreviousSplitRecords also removes version splits from the context cache + // This is not currently supported for the case where we have versions in the keys since we can't trace the old values down clearPreviousSplitRecord(context, subspace, key, clearBasedOnPreviousSizeInfo, previousSizeInfo); } long index = SplitHelper.START_SPLIT_RECORD; diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java index 505ab205c7..9996fb2e86 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java @@ -38,6 +38,12 @@ * *

which means that the entries are sorted by their insertion order (versionstamp order), then by the * original key, with split fragments last.

+ * + * NOTE: Since this class uses versionstamps in the FDB Key, it does not know the actual key used until after the commit. + * As a result, it cannot override previous values saved within the same transaction. Split records saved using this feature + * are not stored in the RYW cache and attempts to write over them (using the same local version and PK) before the commit + * may result in corrupt data. + * Do not override records with another write using teh same local version and PK. */ public class VersioningSplitKeyValueHelper implements SplitKeyValueHelper { private Versionstamp versionstamp; @@ -51,7 +57,6 @@ public VersioningSplitKeyValueHelper(final Versionstamp versionstamp) { * Since the key has a unique component (version), no conflicts are expected, so no need to clean before saving new splits. * Furthermore, since the key contains a version stamp, we don't know the actual key contents ahead of committing * the transaction, and so no clean can be done. - * TODO: We may need to clear the local transaction version mutation cache * @return false, as new keys should not interfere with old ones. */ @Override diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java index 72f624a096..db4080eb76 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java @@ -27,6 +27,7 @@ import com.apple.foundationdb.record.FDBRecordStoreProperties; import com.apple.foundationdb.record.RecordCoreArgumentException; import com.apple.foundationdb.record.RecordCoreException; +import com.apple.foundationdb.record.RecordCoreInternalException; import com.apple.foundationdb.record.RecordCursor; import com.apple.foundationdb.record.RecordCursorIterator; import com.apple.foundationdb.record.RecordCursorResult; @@ -44,6 +45,7 @@ import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; @@ -74,6 +76,7 @@ import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Tests for checking the validity of the "split helper" utility class that handles breaking @@ -1357,4 +1360,35 @@ void scanContinuationsMultipleTransactions(final int returnLimit, final int read commit(context); } } + + /** + * When two saveWithSplit calls use the same incomplete versionstamp (same localVersion, same key) within + * one transaction, we may get a failure or data corruption. The localVersionCache (map by key) may contain + * previous + * values from an identical key (same versionstamp/localversion/PK) but different splits and may not collide + * directly + * with the previous values. This test shows the case where there is a collision since the split numbers are the + * same. + */ + @Test + void saveWithSplitVersionInKeyOverwriteInTransaction() { + final Tuple key = Tuple.from(1066L); + final int localVersion; + try (FDBRecordContext context = openContext()) { + localVersion = context.claimLocalVersion(); + // First write: VERY_LONG_STRING requires multiple splits + final VersioningSplitKeyValueHelper splitKeyHelper = new VersioningSplitKeyValueHelper(Versionstamp.incomplete(localVersion)); + SplitHelper.saveWithSplit(context, subspace, key, VERY_LONG_STRING, null, + true, false, + splitKeyHelper, + false, null, null); + + // Second write: LONG_STRING — same localVersion, same key, shorter value (fewer splits) + final RecordCoreInternalException ex = assertThrows(RecordCoreInternalException.class, () -> SplitHelper.saveWithSplit(context, subspace, key, LONG_STRING, null, + true, false, + splitKeyHelper, + false, null, null)); + assertTrue(ex.getMessage().contains("Key with version overwritten")); + } + } } From e7dd2342d3c6fe0d28205edc91e3c726dfa081cd Mon Sep 17 00:00:00 2001 From: ohad Date: Sat, 28 Feb 2026 11:49:58 -0500 Subject: [PATCH 15/29] Create separate test with multiple transaction, revert split helper test to original, some tests converted. --- .../SplitHelperMultipleTransactionsTest.java | 1354 +++++++++++++++++ .../foundationdb/SplitHelperTest.java | 551 +------ 2 files changed, 1428 insertions(+), 477 deletions(-) create mode 100644 fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java new file mode 100644 index 0000000000..736c07cf75 --- /dev/null +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java @@ -0,0 +1,1354 @@ +/* + * SplitHelperTest.java + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2015-2018 Apple Inc. and the FoundationDB project authors + * + * Licensed 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 com.apple.foundationdb.record.provider.foundationdb; + +import com.apple.foundationdb.KeyValue; +import com.apple.foundationdb.MutationType; +import com.apple.foundationdb.ReadTransaction; +import com.apple.foundationdb.record.ExecuteProperties; +import com.apple.foundationdb.record.FDBRecordStoreProperties; +import com.apple.foundationdb.record.RecordCoreArgumentException; +import com.apple.foundationdb.record.RecordCoreException; +import com.apple.foundationdb.record.RecordCoreInternalException; +import com.apple.foundationdb.record.RecordCursor; +import com.apple.foundationdb.record.RecordCursorIterator; +import com.apple.foundationdb.record.RecordCursorResult; +import com.apple.foundationdb.record.ScanProperties; +import com.apple.foundationdb.record.TupleRange; +import com.apple.foundationdb.record.provider.foundationdb.properties.RecordLayerPropertyStorage; +import com.apple.foundationdb.subspace.Subspace; +import com.apple.foundationdb.tuple.ByteArrayUtil; +import com.apple.foundationdb.tuple.Tuple; +import com.apple.foundationdb.tuple.Versionstamp; +import com.apple.test.BooleanSource; +import com.apple.test.Tags; +import com.google.common.collect.Lists; +import org.junit.jupiter.api.Assumptions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.nio.ByteBuffer; +import java.nio.charset.StandardCharsets; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; +import java.util.Objects; +import java.util.concurrent.ExecutionException; +import java.util.stream.LongStream; +import java.util.stream.Stream; + +import static org.hamcrest.MatcherAssert.assertThat; +import static org.hamcrest.Matchers.anyOf; +import static org.hamcrest.Matchers.containsString; +import static org.hamcrest.Matchers.is; +import static org.hamcrest.Matchers.lessThan; +import static org.hamcrest.Matchers.lessThanOrEqualTo; +import static org.hamcrest.Matchers.oneOf; +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; + +/** + * Tests for checking the validity of the "split helper" utility class that handles breaking + * records across key-value pairs and putting them back together again. + */ +@Tag(Tags.RequiresFDB) +public class SplitHelperMultipleTransactionsTest extends FDBRecordStoreTestBase { + + // From the traditional nursery rhyme + private static final byte[] HUMPTY_DUMPTY = + ("Humpty Dumpty sat on a wall,\n" + + "Humpty Dumpty had a great fall\n" + + "All the king's horses and all the king's men\n" + + "Couldn't put Humpty Dumpty together again.\n").getBytes(StandardCharsets.UTF_8); + + private static final int MEDIUM_COPIES = 5; + private static final int MEDIUM_LEGNTH = HUMPTY_DUMPTY.length * MEDIUM_COPIES; + private static final int LONG_LENGTH = HUMPTY_DUMPTY.length * 1_000; // requires 1 split + private static final int VERY_LONG_LENGTH = HUMPTY_DUMPTY.length * 2_000; // requires 2 splits + private static final byte[] SHORT_STRING = HUMPTY_DUMPTY; + private static final byte[] MEDIUM_STRING; + private static final byte[] LONG_STRING; + private static final byte[] VERY_LONG_STRING; + + private Subspace subspace; + private SplitHelperTestConfig testConfig = SplitHelperTestConfig.getDefault(); + + static { + ByteBuffer mediumBuffer = ByteBuffer.allocate(MEDIUM_LEGNTH); + for (int i = 0; i < MEDIUM_COPIES; i++) { + mediumBuffer.put(HUMPTY_DUMPTY); + } + MEDIUM_STRING = mediumBuffer.array(); + ByteBuffer longBuffer = ByteBuffer.allocate(LONG_LENGTH); + while (longBuffer.position() < LONG_LENGTH - HUMPTY_DUMPTY.length) { + longBuffer.put(HUMPTY_DUMPTY); + } + LONG_STRING = longBuffer.array(); + ByteBuffer veryLongBuffer = ByteBuffer.allocate(VERY_LONG_LENGTH); + while (veryLongBuffer.position() < VERY_LONG_LENGTH - HUMPTY_DUMPTY.length) { + veryLongBuffer.put(HUMPTY_DUMPTY); + } + VERY_LONG_STRING = veryLongBuffer.array(); + } + + @BeforeEach + public void setSubspace() { + try (FDBRecordContext context = openContext()) { + subspace = path.toSubspace(context); + } + } + + static class SplitHelperTestConfig { + private final boolean splitLongRecords; + private final boolean omitUnsplitSuffix; + private final boolean unrollRecordDeletes; + private final boolean loadViaGets; + private final boolean isDryRun; + private final boolean useVersionInKey; + + public SplitHelperTestConfig(boolean splitLongRecords, boolean omitUnsplitSuffix, boolean unrollRecordDeletes, + boolean loadViaGets, boolean isDryRun, boolean useVersionInKey) { + this.splitLongRecords = splitLongRecords; + this.omitUnsplitSuffix = omitUnsplitSuffix; + this.unrollRecordDeletes = unrollRecordDeletes; + this.loadViaGets = loadViaGets; + this.isDryRun = isDryRun; + this.useVersionInKey = useVersionInKey; + } + + public SplitKeyValueHelper keyHelper(int localVersion) { + if (useVersionInKey) { + return new VersioningSplitKeyValueHelper(Versionstamp.incomplete(localVersion)); + } else { + return DefaultSplitKeyValueHelper.INSTANCE; + } + } + + @Nonnull + public RecordLayerPropertyStorage.Builder setProps(@Nonnull RecordLayerPropertyStorage.Builder props) { + return props + .addProp(FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES, unrollRecordDeletes) + .addProp(FDBRecordStoreProperties.LOAD_RECORDS_VIA_GETS, loadViaGets); + } + + public boolean hasSplitPoints() { + return splitLongRecords || !omitUnsplitSuffix; + } + + @Override + public String toString() { + return "SplitHelperTestConfig{" + + "splitLongRecords=" + splitLongRecords + + ", omitUnsplitSuffix=" + omitUnsplitSuffix + + ", unrollRecordDeletes=" + unrollRecordDeletes + + ", loadViaGets=" + loadViaGets + + ", isDryRun=" + isDryRun + + ", useVersionInKey=" + useVersionInKey + + '}'; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final SplitHelperTestConfig that = (SplitHelperTestConfig)o; + return splitLongRecords == that.splitLongRecords && omitUnsplitSuffix == that.omitUnsplitSuffix && + unrollRecordDeletes == that.unrollRecordDeletes && loadViaGets == that.loadViaGets && + isDryRun == that.isDryRun && useVersionInKey == that.useVersionInKey; + } + + @Override + public int hashCode() { + return Objects.hash(splitLongRecords, omitUnsplitSuffix, unrollRecordDeletes, loadViaGets, useVersionInKey); + } + + public static Stream allValidConfigs() { + // Note that splitLongRecords="true" && omitUnsplitSuffix="true" is not valid + // Note that useVersionInKey="true" && isDryRun="true" is not valid (versionstamp never completes without commit) + return Stream.of(false, true).flatMap(useVersionInKey -> + Stream.of(false, true).flatMap(splitLongRecords -> + (splitLongRecords ? Stream.of(false) : Stream.of(false, true)).flatMap(omitUnsplitSuffix -> + Stream.of(false, true).flatMap(unrollRecordDeletes -> + Stream.of(false, true).flatMap(loadViaGets -> + Stream.of(false, true) + .filter(isDryRun -> !useVersionInKey || !isDryRun) + .map(isDryRun -> + new SplitHelperTestConfig(splitLongRecords, omitUnsplitSuffix, unrollRecordDeletes, loadViaGets, isDryRun, useVersionInKey))))))); + } + + public static SplitHelperTestConfig getDefault() { + return new SplitHelperTestConfig(true, false, + FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES.getDefaultValue(), + FDBRecordStoreProperties.LOAD_RECORDS_VIA_GETS.getDefaultValue(), false, false); + } + } + + public static Stream testConfigs() { + return SplitHelperTestConfig.allValidConfigs().map(Arguments::of); + } + + @Nonnull + public static Stream limitsAndReverseArgs() { + List limits = Arrays.asList(1, 2, 7, Integer.MAX_VALUE); + return limits.stream() + .flatMap(returnLimit -> limits.stream() + .flatMap(readLimit -> Stream.of(Arguments.of(returnLimit, readLimit, false), Arguments.of(returnLimit, readLimit, true)))); + } + + @Override + protected RecordLayerPropertyStorage.Builder addDefaultProps(final RecordLayerPropertyStorage.Builder props) { + return testConfig.setProps(super.addDefaultProps(props)); + } + + private SplitHelper.SizeInfo saveUnsuccessfully(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, + @Nullable FDBRecordVersion version, + @Nonnull SplitHelperTestConfig testConfig, + @Nullable FDBStoredSizes previousSizeInfo, + @Nonnull Class errClazz, @Nonnull String errMessage) { + final SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); + E e = assertThrows(errClazz, + () -> SplitHelper.saveWithSplit(context, subspace, key, serialized, version, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, + testConfig.keyHelper(0), previousSizeInfo != null, previousSizeInfo, sizeInfo)); + assertThat(e.getMessage(), containsString(errMessage)); + + assertEquals(0, sizeInfo.getKeyCount()); + assertEquals(0, sizeInfo.getKeySize()); + assertEquals(0, sizeInfo.getValueSize()); + assertThat(sizeInfo.isVersionedInline(), is(false)); + + assertEquals(0, previousSizeInfo == null ? 0 : previousSizeInfo.getKeyCount()); + + return sizeInfo; + } + + private SplitHelper.SizeInfo saveOnly(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, + @Nullable FDBRecordVersion version, + @Nonnull SplitHelperTestConfig testConfig, + @Nullable FDBStoredSizes previousSizeInfo, + int localVersion) { + final SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); + SplitHelper.saveWithSplit(context, subspace, key, serialized, version, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, + testConfig.keyHelper(localVersion), previousSizeInfo != null, previousSizeInfo, sizeInfo); + int dataKeyCount = (serialized.length - 1) / SplitHelper.SPLIT_RECORD_SIZE + 1; + boolean isSplit = dataKeyCount > 1; + int keyCount = dataKeyCount; + if (version != null) { + keyCount += 1; + } + assertEquals(isSplit, sizeInfo.isSplit()); + assertEquals(keyCount, sizeInfo.getKeyCount()); + int valueSize = serialized.length + (version != null ? 1 + FDBRecordVersion.VERSION_LENGTH : 0); + assertEquals(valueSize, sizeInfo.getValueSize()); + assertEquals(version != null, sizeInfo.isVersionedInline()); + if (!testConfig.useVersionInKey) { + // Key size can only be asserted when the key is fully known at write time (not the case for versioning keys, + // since the versionstamp in the key is not yet resolved until after commit) + int keySize = (subspace.pack().length + key.pack().length) * keyCount; + if (testConfig.hasSplitPoints()) { + // Add in the counters the split points. + if (!isSplit) { + keySize += 1; // As 0 requires 1 byte when Tuple packed + } else { + keySize += dataKeyCount * 2; // As each split point is two bytes when tuple packed + } + } + if (version != null) { + keySize += 2; + } + assertEquals(keySize, sizeInfo.getKeySize()); + } + return sizeInfo; + } + + private Tuple toVerifyKey(Tuple key, byte[] versionStamp, int localVersion, boolean versionInKey) { + if (versionInKey) { + return Tuple.from(Versionstamp.complete(versionStamp, localVersion)).addAll(key); + } else { + return key; + } + } + + private void verifySuccessfullySaved(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, + @Nullable FDBRecordVersion version, + @Nonnull SplitHelperTestConfig testConfig) { + // Similar to the calculation in saveOnly + int dataKeyCount = (serialized.length - 1) / SplitHelper.SPLIT_RECORD_SIZE + 1; + boolean isSplit = dataKeyCount > 1; + int keyCount = dataKeyCount; + if (version != null) { + keyCount += 1; + } + final Subspace keySubspace = subspace.subspace(key); + RecordCursorIterator kvCursor = KeyValueCursor.Builder.withSubspace(keySubspace) + .setContext(context) + .setScanProperties(ScanProperties.FORWARD_SCAN) + .build() + .asIterator(); + List indexes = new ArrayList<>(keyCount); + byte[] versionBytes = null; + byte[] valueBytes = null; + while (kvCursor.hasNext()) { + KeyValue kv = kvCursor.next(); + Tuple suffix = keySubspace.unpack(kv.getKey()); + if (testConfig.omitUnsplitSuffix) { + assertThat(suffix.isEmpty(), is(true)); + valueBytes = kv.getValue(); + } else { + Long index = suffix.getLong(0); + indexes.add(index); + if (index == SplitHelper.RECORD_VERSION) { + versionBytes = kv.getValue(); + } else { + if (valueBytes == null) { + valueBytes = kv.getValue(); + } else { + valueBytes = ByteArrayUtil.join(valueBytes, kv.getValue()); + } + } + } + } + List expectedIndexes; + if (testConfig.omitUnsplitSuffix) { + expectedIndexes = Collections.emptyList(); + } else { + expectedIndexes = new ArrayList<>(keyCount); + if (version != null && version.isComplete()) { + expectedIndexes.add(SplitHelper.RECORD_VERSION); + } + if (!isSplit) { + expectedIndexes.add(SplitHelper.UNSPLIT_RECORD); + } else { + LongStream.range(SplitHelper.START_SPLIT_RECORD, SplitHelper.START_SPLIT_RECORD + dataKeyCount) + .forEach(expectedIndexes::add); + } + } + assertEquals(expectedIndexes, indexes); + + assertNotNull(valueBytes); + assertArrayEquals(serialized, valueBytes); + + if (version != null) { + if (!version.isComplete()) { + assertNull(versionBytes); + } else { + assertNotNull(versionBytes); + assertEquals(version, FDBRecordVersion.fromVersionstamp(Tuple.fromBytes(versionBytes).getVersionstamp(0))); + } + } else { + assertNull(versionBytes); + } + } + +// private SplitHelper.SizeInfo saveSuccessfully(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, +// @Nullable FDBRecordVersion version, +// @Nonnull SplitHelperTestConfig testConfig, +// @Nullable FDBStoredSizes previousSizeInfo) { +// // localVersion=0: single-transaction callers always use useVersionInKey=false, so the value doesn't matter +// SplitHelper.SizeInfo sizeInfo = saveOnly(context, key, serialized, version, testConfig, previousSizeInfo, 0); +// verifySuccessfullySaved(context, key, serialized, version, testConfig); +// return sizeInfo; +// } + +// private SplitHelper.SizeInfo dryRunSetSizeInfo(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, +// @Nullable FDBRecordVersion version, +// @Nonnull SplitHelperTestConfig testConfig, +// @Nullable FDBStoredSizes previousSizeInfo) { +// final SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); +// SplitHelper.dryRunSaveWithSplitOnlySetSizeInfo(subspace, key, serialized, version, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, sizeInfo); +// +// int dataKeyCount = (serialized.length - 1) / SplitHelper.SPLIT_RECORD_SIZE + 1; +// boolean isSplit = dataKeyCount > 1; +// int keyCount = dataKeyCount; +// if (version != null) { +// keyCount += 1; +// } +// int keySize = (subspace.pack().length + key.pack().length) * keyCount; +// assertEquals(isSplit, sizeInfo.isSplit()); +// assertEquals(keyCount, sizeInfo.getKeyCount()); +// if (testConfig.hasSplitPoints()) { +// // Add in the the counters the split points. +// if (!isSplit) { +// keySize += 1; // As 0 requires 1 byte when Tuple packed +// } else { +// keySize += dataKeyCount * 2; // As each split point is two bytes when tuple packed +// } +// } +// if (version != null) { +// keySize += 2; +// } +// int valueSize = serialized.length + (version != null ? 1 + FDBRecordVersion.VERSION_LENGTH : 0); +// assertEquals(keySize, sizeInfo.getKeySize()); +// assertEquals(valueSize, sizeInfo.getValueSize()); +// assertEquals(version != null, sizeInfo.isVersionedInline()); +// // assert nothing is written +// int count = KeyValueCursor.Builder.withSubspace(subspace.subspace(key)) +// .setContext(context) +// .setScanProperties(ScanProperties.FORWARD_SCAN) +// .build() +// .getCount() +// .join(); +// assertEquals(0, previousSizeInfo == null ? count : previousSizeInfo.getKeyCount() + count); +// sizeInfo.reset(); +// return sizeInfo; +// } + +// private SplitHelper.SizeInfo saveWithSplit(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, +// @Nullable FDBRecordVersion version, +// @Nonnull SplitHelperTestConfig testConfig, +// @Nullable FDBStoredSizes previousSizeInfo) { +// if (testConfig.omitUnsplitSuffix && version != null) { +// return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, +// RecordCoreArgumentException.class, "Cannot include version"); +// } else if (!testConfig.splitLongRecords && serialized.length > SplitHelper.SPLIT_RECORD_SIZE) { +// return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, +// RecordCoreException.class, "Record is too long"); +// } else if (testConfig.isDryRun) { +// return dryRunSetSizeInfo(context, key, serialized, version, testConfig, previousSizeInfo); +// } else { +// return saveSuccessfully(context, key, serialized, version, testConfig, previousSizeInfo); +// } +// } + +// private SplitHelper.SizeInfo saveWithSplit(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, +// @Nullable FDBRecordVersion version, @Nonnull SplitHelperTestConfig testConfig) { +// return saveWithSplit(context, key, serialized, version, testConfig, null); +// } +// +// private SplitHelper.SizeInfo saveWithSplit(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, @Nonnull SplitHelperTestConfig testConfig, +// @Nullable FDBStoredSizes previousSizeInfo) { +// return saveWithSplit(context, key, serialized, null, testConfig, previousSizeInfo); +// } +// +// private SplitHelper.SizeInfo saveWithSplit(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, @Nonnull SplitHelperTestConfig testConfig) { +// return saveWithSplit(context, key, serialized, null, testConfig); +// } + +// @MethodSource("testConfigs") +// @ParameterizedTest(name = "saveWithSplit[{0}]") +// public void saveWithSplit(@Nonnull SplitHelperTestConfig testConfig) { +// this.testConfig = testConfig; +// // For versionInKey we need multiple transactions - one to save the key and one to read the timestamp from it +// Assumptions.assumeFalse(testConfig.useVersionInKey); +// try (FDBRecordContext context = openContext()) { +// // No version +// FDBStoredSizes sizes1 = saveWithSplit(context, Tuple.from(1066L), SHORT_STRING, testConfig); +// FDBStoredSizes sizes2 = saveWithSplit(context, Tuple.from(1415L), LONG_STRING, testConfig); +// FDBStoredSizes sizes3 = saveWithSplit(context, Tuple.from(1776L), VERY_LONG_STRING, testConfig); +// +// // Save over some things using the previous split points +// if (testConfig.splitLongRecords) { +// saveWithSplit(context, Tuple.from(1066L), VERY_LONG_STRING, testConfig, sizes1); +// saveWithSplit(context, Tuple.from(1776), LONG_STRING, testConfig, sizes3); +// } +// saveWithSplit(context, Tuple.from(1415L), SHORT_STRING, testConfig, sizes2); +// +// commit(context); +// } +// } + + @MethodSource("testConfigs") + @ParameterizedTest(name = "saveWithSplitMultipleTransactions[{0}]") + public void saveWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig testConfig) { + // dry run does not support transactions + Assumptions.assumeFalse(testConfig.isDryRun); + this.testConfig = testConfig; + + final Tuple key1 = Tuple.from(1066L); + final Tuple key2 = Tuple.from(1415L); + final Tuple key3 = Tuple.from(1776L); + final int localVersion1; + final int localVersion2; + final int localVersion3; + final SplitHelper.SizeInfo sizes1; + final SplitHelper.SizeInfo sizes2; + final SplitHelper.SizeInfo sizes3; + byte[] globalVersionstamp; + try (FDBRecordContext context = openContext()) { + // save with no version and no previousSizeInfo + localVersion1 = context.claimLocalVersion(); + sizes1 = saveWithSplitForMultipleTransactions(context, key1, SHORT_STRING, null, testConfig, null, localVersion1); + localVersion2 = context.claimLocalVersion(); + sizes2 = saveWithSplitForMultipleTransactions(context, key2, LONG_STRING, null, testConfig, null, localVersion2); + localVersion3 = context.claimLocalVersion(); + sizes3 = saveWithSplitForMultipleTransactions(context, key3, VERY_LONG_STRING, null, testConfig, null, localVersion3); + + commit(context); + globalVersionstamp = context.getVersionStamp(); + } + + final Tuple verifyKey1 = toVerifyKey(key1, globalVersionstamp, localVersion1, testConfig.useVersionInKey); + final Tuple verifyKey2 = toVerifyKey(key2, globalVersionstamp, localVersion2, testConfig.useVersionInKey); + final Tuple verifyKey3 = toVerifyKey(key3, globalVersionstamp, localVersion3, testConfig.useVersionInKey); + try (FDBRecordContext context = openContext()) { + verifySuccessfullySaved(context, verifyKey1, SHORT_STRING, null, testConfig); + if (testConfig.splitLongRecords) { + verifySuccessfullySaved(context, verifyKey2, LONG_STRING, null, testConfig); + verifySuccessfullySaved(context, verifyKey3, VERY_LONG_STRING, null, testConfig); + } + } + + int localVersion4 = 0; + int localVersion5 = 0; + int localVersion6 = 0; + try (FDBRecordContext context = openContext()) { + // Save over some things using the previous split points + if (testConfig.splitLongRecords) { + localVersion4 = context.claimLocalVersion(); + saveWithSplitForMultipleTransactions(context, key1, VERY_LONG_STRING, null, testConfig, sizes1, localVersion4); + localVersion5 = context.claimLocalVersion(); + saveWithSplitForMultipleTransactions(context, key3, LONG_STRING, null, testConfig, sizes3, localVersion5); + } + localVersion6 = context.claimLocalVersion(); + saveWithSplitForMultipleTransactions(context, key2, SHORT_STRING, null, testConfig, sizes2, localVersion6); + commit(context); + globalVersionstamp = context.getVersionStamp(); + } + Tuple verifyKey4 = toVerifyKey(key1, globalVersionstamp, localVersion4, testConfig.useVersionInKey); + Tuple verifyKey5 = toVerifyKey(key3, globalVersionstamp, localVersion5, testConfig.useVersionInKey); + Tuple verifyKey6 = toVerifyKey(key2, globalVersionstamp, localVersion6, testConfig.useVersionInKey); + try (FDBRecordContext context = openContext()) { + if (testConfig.splitLongRecords) { + verifySuccessfullySaved(context, verifyKey4, VERY_LONG_STRING, null, testConfig); + verifySuccessfullySaved(context, verifyKey5, LONG_STRING, null, testConfig); + } + verifySuccessfullySaved(context, verifyKey6, SHORT_STRING, null, testConfig); + } + } + + private SplitHelper.SizeInfo saveWithSplitForMultipleTransactions(@Nonnull FDBRecordContext context, @Nonnull Tuple key, + byte[] serialized, + @Nullable FDBRecordVersion version, + @Nonnull SplitHelperTestConfig testConfig, + @Nullable FDBStoredSizes previousSizeInfo, + int localVersion) { + if (testConfig.omitUnsplitSuffix && version != null) { + return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, + RecordCoreArgumentException.class, "Cannot include version"); + } else if (!testConfig.splitLongRecords && serialized.length > SplitHelper.SPLIT_RECORD_SIZE) { + return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, + RecordCoreException.class, "Record is too long"); + } else { + return saveOnly(context, key, serialized, version, testConfig, previousSizeInfo, localVersion); + } + } + // TODO +// @MethodSource("testConfigs") +// @ParameterizedTest(name = "saveWithSplitAndCompleteVersion[{0}]") +// public void saveWithSplitAndCompleteVersions(SplitHelperTestConfig testConfig) { +// this.testConfig = testConfig; +// Assumptions.assumeFalse(testConfig.useVersionInKey); +// try (FDBRecordContext context = openContext()) { +// byte[] globalVersion = "karlgrosse".getBytes(StandardCharsets.US_ASCII); +// saveWithSplit(context, Tuple.from(800L), SHORT_STRING, FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()), testConfig); +// saveWithSplit(context, Tuple.from(813L), LONG_STRING, FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()), testConfig); +// saveWithSplit(context, Tuple.from(823L), VERY_LONG_STRING, FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()), testConfig); +// +// // Save over the records *without* using the previous size info +// saveWithSplit(context, Tuple.from(800L), SHORT_STRING, testConfig); +// saveWithSplit(context, Tuple.from(813L), LONG_STRING, testConfig); +// saveWithSplit(context, Tuple.from(823L), VERY_LONG_STRING, testConfig); +// +// FDBStoredSizes sizes4 = saveWithSplit(context, Tuple.from(800L), SHORT_STRING, FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()), testConfig); +// FDBStoredSizes sizes5 = saveWithSplit(context, Tuple.from(813L), LONG_STRING, FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()), testConfig); +// FDBStoredSizes sizes6 = saveWithSplit(context, Tuple.from(823L), VERY_LONG_STRING, FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()), testConfig); +// +// // Save over the records *with* using the previous size info +// saveWithSplit(context, Tuple.from(800L), SHORT_STRING, testConfig, sizes4); +// saveWithSplit(context, Tuple.from(813L), LONG_STRING, testConfig, sizes5); +// saveWithSplit(context, Tuple.from(823L), VERY_LONG_STRING, testConfig, sizes6); +// +// commit(context); +// } +// } + + @Nonnull + private FDBStoredSizes writeDummyRecord(@Nonnull FDBRecordContext context, @Nonnull Tuple key, + @Nullable FDBRecordVersion version, int splits, + boolean omitUnsplitSuffix, boolean useVersionInKey, int localVersion) { + SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); + if (version != null) { + assertThat(omitUnsplitSuffix, is(false)); + assertThat(useVersionInKey, is(false)); + sizeInfo.setVersionedInline(true); + Tuple keyTuple = key.add(SplitHelper.RECORD_VERSION); + byte[] valueBytes = SplitHelper.packVersion(version); + // Note that this will not mutate the version in the value + writeDummyKV(context, keyTuple, valueBytes, sizeInfo, useVersionInKey, localVersion); + } + if (splits == 1) { + if (omitUnsplitSuffix) { + Tuple keyTuple = key; + byte[] valueBytes = SHORT_STRING; + writeDummyKV(context, keyTuple, valueBytes, sizeInfo, useVersionInKey, localVersion); + } else { + Tuple keyTuple = key.add(SplitHelper.UNSPLIT_RECORD); + byte[] valueBytes = SHORT_STRING; + writeDummyKV(context, keyTuple, valueBytes, sizeInfo, useVersionInKey, localVersion); + } + sizeInfo.setSplit(false); + } else { + for (int i = 0; i < splits; i++) { + Tuple keyTuple = key.add(SplitHelper.START_SPLIT_RECORD + i); + byte[] valueBytes = SHORT_STRING; + writeDummyKV(context, keyTuple, valueBytes, sizeInfo, useVersionInKey, localVersion); + } + sizeInfo.setSplit(true); + } + return sizeInfo; + } + + @Nonnull + private FDBStoredSizes writeDummyRecord(@Nonnull FDBRecordContext context, @Nonnull Tuple key, int splits, boolean omitUnsplitSuffix) { + return writeDummyRecord(context, key, null, splits, omitUnsplitSuffix, false, 0); + } + + @Nonnull + private FDBStoredSizes writeDummyRecord(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @Nonnull FDBRecordVersion version, int splits) { + return writeDummyRecord(context, key, version, splits, false, false, 0); + } + + private void writeDummyKV(@Nonnull FDBRecordContext context, @Nonnull Tuple keyTuple, + byte[] valueBytes, @Nullable SplitHelper.SizeInfo sizeInfo, boolean useVersionInKey, int localVersion) { + byte[] keyBytes; + // Mimic the work done in both SplitKeyValueHelper + if (useVersionInKey) { + Tuple versionedKeyTuple = Tuple.from(Versionstamp.incomplete(localVersion)).addAll(keyTuple); + keyBytes = subspace.packWithVersionstamp(versionedKeyTuple); + context.addVersionMutation(MutationType.SET_VERSIONSTAMPED_KEY, keyBytes, valueBytes); + } else { + keyBytes = subspace.pack(keyTuple); + context.ensureActive().set(keyBytes, valueBytes); + } + if (sizeInfo != null) { + sizeInfo.add(keyBytes, valueBytes); + } + } + + private void deleteSplit(@Nonnull FDBRecordContext context, @Nonnull Tuple key, + @Nonnull SplitHelperTestConfig testConfig, + @Nullable FDBStoredSizes sizeInfo) { + SplitHelper.deleteSplit(context, subspace, key, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, sizeInfo != null, sizeInfo); + int count = KeyValueCursor.Builder.withSubspace(subspace.subspace(key)) + .setContext(context) + .setScanProperties(ScanProperties.FORWARD_SCAN) + .build() + .getCount() + .join(); + assertEquals(0, count); + } + + @MethodSource("testConfigs") + @ParameterizedTest(name = "deleteWithSplitMultipleTransactions[{0}]") + public void deleteWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig testConfig) { + this.testConfig = testConfig; + + final Tuple key1 = Tuple.from(-660L); + final Tuple key2 = Tuple.from(-581L); + final Tuple key3 = Tuple.from(-549L); + final Tuple key4 = Tuple.from(-510L); + + // tx1: write records + int localVersion1; + int localVersion2; + int localVersion3 = 0; + int localVersion4 = 0; + final FDBStoredSizes sizes1; + final FDBStoredSizes sizes2; + FDBStoredSizes sizes3 = null; + FDBStoredSizes sizes4 = null; + byte[] globalVersionStamp; + try (FDBRecordContext context = openContext()) { + localVersion1 = context.claimLocalVersion(); + sizes1 = writeDummyRecord(context, key1, null, 1, testConfig.omitUnsplitSuffix, testConfig.useVersionInKey, localVersion1); + localVersion2 = context.claimLocalVersion(); + sizes2 = writeDummyRecord(context, key2, null, 1, testConfig.omitUnsplitSuffix, testConfig.useVersionInKey, localVersion2); + if (testConfig.splitLongRecords) { + localVersion3 = context.claimLocalVersion(); + sizes3 = writeDummyRecord(context, key3, null, 5, testConfig.omitUnsplitSuffix, testConfig.useVersionInKey, localVersion3); + localVersion4 = context.claimLocalVersion(); + sizes4 = writeDummyRecord(context, key4, null, 5, testConfig.omitUnsplitSuffix, testConfig.useVersionInKey, localVersion4); + } + commit(context); + globalVersionStamp = context.getVersionStamp(); + } + + final Tuple deleteKey1 = toVerifyKey(key1, globalVersionStamp, localVersion1, testConfig.useVersionInKey); + final Tuple deleteKey2 = toVerifyKey(key2, globalVersionStamp, localVersion2, testConfig.useVersionInKey); + final Tuple deleteKey3 = toVerifyKey(key3, globalVersionStamp, localVersion3, testConfig.useVersionInKey); + final Tuple deleteKey4 = toVerifyKey(key4, globalVersionStamp, localVersion4, testConfig.useVersionInKey); + + // tx2: delete records + try (FDBRecordContext context = openContext()) { + deleteSplit(context, deleteKey1, testConfig, sizes1); + deleteSplit(context, deleteKey2, testConfig, sizes2); + if (testConfig.splitLongRecords) { + deleteSplit(context, deleteKey3, testConfig, sizes3); + deleteSplit(context, deleteKey4, testConfig, sizes4); + } + commit(context); + } + } +// +// static Stream deleteWithSplitAndVersion() { +// return Stream.of(false, true).flatMap(splitLongRecords -> +// Stream.of(false, true).map(unrollSingleRecordDeletes -> +// Arguments.of(splitLongRecords, unrollSingleRecordDeletes))); +// } + + @FunctionalInterface + private interface LoadRecordFunction { + FDBRawRecord load(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @Nullable FDBStoredSizes sizes, @Nullable byte[] expectedContents, @Nullable FDBRecordVersion version); + + default FDBRawRecord load(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @Nullable FDBStoredSizes sizes, @Nullable byte[] expectedContents) { + return load(context, key, sizes, expectedContents, null); + } + } + + private void loadSingleRecordsMultipleVersions(SplitHelperTestConfig testConfig, @Nonnull LoadRecordFunction loadRecordFunction) { + final Tuple key1 = Tuple.from(1042L); + final Tuple key2 = Tuple.from(1066L); + final Tuple key3 = Tuple.from(1087L); + final Tuple key4 = Tuple.from(1100L); + final Tuple key5 = Tuple.from(1135L); + final Tuple key6 = Tuple.from(1189L); + final Tuple key7 = Tuple.from(1199L); + final Tuple key8 = Tuple.from(1216L); + final Tuple key9 = Tuple.from(1272L); + int localVersion2; + int localVersion3 = 0; + int localVersion4 = 0; + int localVersion5 = 0; + int localVersion6 = 0; + int localVersion7 = 0; + int localVersion8 = 0; + int localVersion9 = 0; + FDBRecordVersion version3 = null; + FDBRecordVersion version4 = null; + FDBRecordVersion version9 = null; + FDBStoredSizes sizes2; + FDBStoredSizes sizes3 = null; + FDBStoredSizes sizes5 = null; + final byte[] valueGlobalVersion = "-hastings-".getBytes(StandardCharsets.US_ASCII); + byte[] keyGlobalVersion; + try (FDBRecordContext context = openContext()) { + // One unsplit record + localVersion2 = context.claimLocalVersion(); + sizes2 = writeDummyRecord(context, key2, null, 1, testConfig.omitUnsplitSuffix, testConfig.useVersionInKey, localVersion2); + assertThat(sizes2.isSplit(), is(false)); + + if ((!testConfig.omitUnsplitSuffix) && (!testConfig.useVersionInKey)) { + // One record with version + localVersion3 = context.claimLocalVersion(); + version3 = FDBRecordVersion.complete(valueGlobalVersion, localVersion3); + sizes3 = writeDummyRecord(context, key3, version3, 1); + assertThat(sizes3.isVersionedInline(), is(true)); + + // One version but missing record + localVersion4 = context.claimLocalVersion(); + version4 = FDBRecordVersion.complete(valueGlobalVersion, localVersion4); + writeDummyRecord(context, key4, version4, 1); + context.ensureActive().clear(subspace.pack(key4.add(SplitHelper.UNSPLIT_RECORD))); + } + + if (testConfig.splitLongRecords) { + // One split record + localVersion5 = context.claimLocalVersion(); + sizes5 = writeDummyRecord(context, key5, null, MEDIUM_COPIES, false, testConfig.useVersionInKey, localVersion5); + assertEquals(MEDIUM_COPIES, sizes5.getKeyCount()); + + // One split record but then delete the last split point (no way to distinguish this from just inserting one fewer split) + writeDummyRecord(context, key5, MEDIUM_COPIES + 1, false); + + // One split record then delete the first split point + localVersion6 = context.claimLocalVersion(); + writeDummyRecord(context, key6, null, MEDIUM_COPIES, false, testConfig.useVersionInKey, localVersion6); + + // One split record then delete the middle split point + localVersion7 = context.claimLocalVersion(); + writeDummyRecord(context, key7, null, MEDIUM_COPIES, false, testConfig.useVersionInKey, localVersion7); + + // One split record then add an extra key in the middle + localVersion8 = context.claimLocalVersion(); + writeDummyRecord(context, key8, null, MEDIUM_COPIES, false, testConfig.useVersionInKey, localVersion8); + writeDummyKV(context, key8.add(SplitHelper.START_SPLIT_RECORD + 2).add( 0L), HUMPTY_DUMPTY, null, testConfig.useVersionInKey, localVersion8); + + // One split record with version then delete the first split point + if (!testConfig.useVersionInKey) { + localVersion9 = context.claimLocalVersion(); + version9 = FDBRecordVersion.complete(valueGlobalVersion, context.claimLocalVersion()); + writeDummyRecord(context, key9, version9, MEDIUM_COPIES); + } + } + + commit(context); + keyGlobalVersion = context.getVersionStamp(); + } + + // transaction 2 - delete any items needing deleting + try (FDBRecordContext context = openContext()) { + // One split record but then delete the last split point (no way to distinguish this from just inserting one fewer split) + if (testConfig.splitLongRecords) { + final Tuple deleteKey5 = toVerifyKey(key5.add(SplitHelper.START_SPLIT_RECORD + MEDIUM_COPIES), keyGlobalVersion, localVersion5, testConfig.useVersionInKey); + context.ensureActive().clear(subspace.pack(deleteKey5)); + // One split record then delete the first split point + final Tuple deleteKey6 = toVerifyKey(key6.add(SplitHelper.START_SPLIT_RECORD), keyGlobalVersion, localVersion6, testConfig.useVersionInKey); + context.ensureActive().clear(subspace.pack(deleteKey6)); + // One split record then delete the middle split point + final Tuple deleteKey7 = toVerifyKey(key7.add(SplitHelper.START_SPLIT_RECORD + 2), keyGlobalVersion, localVersion7, testConfig.useVersionInKey); + context.ensureActive().clear(subspace.pack(deleteKey7)); + if (!testConfig.useVersionInKey) { + // One split record with version then delete the first split point + final Tuple DeleteKey9 = toVerifyKey(key9.add(SplitHelper.START_SPLIT_RECORD), keyGlobalVersion, localVersion9, testConfig.useVersionInKey); + context.ensureActive().clear(subspace.pack(DeleteKey9)); + } + } + commit(context); + } + + final Tuple completeKey1 = toVerifyKey(key1, keyGlobalVersion, 0, testConfig.useVersionInKey); + final Tuple completeKey2 = toVerifyKey(key2, keyGlobalVersion, localVersion2, testConfig.useVersionInKey); + final Tuple completeKey3 = toVerifyKey(key3, keyGlobalVersion, localVersion3, testConfig.useVersionInKey); + final Tuple completeKey4 = toVerifyKey(key4, keyGlobalVersion, localVersion4, testConfig.useVersionInKey); + final Tuple completeKey5 = toVerifyKey(key5, keyGlobalVersion, localVersion5, testConfig.useVersionInKey); + final Tuple completeKey6 = toVerifyKey(key6, keyGlobalVersion, localVersion6, testConfig.useVersionInKey); + final Tuple completeKey7 = toVerifyKey(key7, keyGlobalVersion, localVersion7, testConfig.useVersionInKey); + final Tuple completeKey8 = toVerifyKey(key8, keyGlobalVersion, localVersion8, testConfig.useVersionInKey); + final Tuple completeKey9 = toVerifyKey(key9, keyGlobalVersion, localVersion9, testConfig.useVersionInKey); + + // transaction 3 - verify + try (FDBRecordContext context = openContext()) { + // No record + loadRecordFunction.load(context, completeKey1, null, null); + // One unsplit record + loadRecordFunction.load(context, completeKey2, sizes2, HUMPTY_DUMPTY); + if ((!testConfig.omitUnsplitSuffix) && (!testConfig.useVersionInKey)) { + // One record with version + loadRecordFunction.load(context, completeKey3, sizes3, HUMPTY_DUMPTY, version3); + // One version but missing record + final FDBRecordVersion v4 = version4; + assertThrows(SplitHelper.FoundSplitWithoutStartException.class, + () -> loadRecordFunction.load(context, completeKey4, null, null, v4)); + } + if (testConfig.splitLongRecords) { + // One split record + // One split record but then delete the last split point (no way to distinguish this from just inserting one fewer split) + loadRecordFunction.load(context, completeKey5, sizes5, MEDIUM_STRING); + // One split record then delete the first split point + if (testConfig.loadViaGets) { + loadRecordFunction.load(context, completeKey6, null, null); + } else { + assertThrows(SplitHelper.FoundSplitWithoutStartException.class, + () -> loadRecordFunction.load(context, completeKey6, null, null)); + } + // One split record then delete the middle split point + RecordCoreException err7 = assertThrows(RecordCoreException.class, + () -> loadRecordFunction.load(context, completeKey7, null, null)); + assertThat(err7.getMessage(), containsString("Split record segments out of order")); + // One split record then add an extra key in the middle + RecordCoreException err8 = assertThrows(RecordCoreException.class, + () -> loadRecordFunction.load(context, completeKey8, null, null)); + assertThat(err8.getMessage(), anyOf( + containsString("Expected only a single key extension"), + containsString("Split record segments out of order") + )); + // One split record with version then delete the first split point + if (!testConfig.useVersionInKey) { + final FDBRecordVersion v9 = version9; + assertThrows(SplitHelper.FoundSplitWithoutStartException.class, + () -> loadRecordFunction.load(context, completeKey9, null, null, v9)); + } + } + } + } + + @Nullable + private FDBRawRecord loadWithSplit(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @Nonnull SplitHelperTestConfig testConfig, + @Nullable FDBStoredSizes expectedSizes, @Nullable byte[] expectedContents, @Nullable FDBRecordVersion expectedVersion) { + final ReadTransaction tr = context.ensureActive(); + SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); + FDBRawRecord rawRecord; + try { + rawRecord = SplitHelper.loadWithSplit(tr, context, subspace, key, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, sizeInfo).get(); + } catch (InterruptedException | ExecutionException e) { + throw FDBExceptions.wrapException(e); + } + + if (expectedSizes == null || expectedContents == null) { + assertNull(rawRecord); + } else { + assertNotNull(rawRecord); + assertArrayEquals(expectedContents, rawRecord.getRawRecord()); + int valueSize = expectedContents.length; + if (expectedVersion != null) { + valueSize += 1 + FDBRecordVersion.VERSION_LENGTH; + } + assertEquals(valueSize, rawRecord.getValueSize()); + if (!testConfig.splitLongRecords) { + assertThat(rawRecord.isSplit(), is(false)); + } + if (testConfig.omitUnsplitSuffix) { + assertThat(rawRecord.isVersionedInline(), is(false)); + } + boolean isSplit = rawRecord.getKeyCount() - (expectedVersion != null ? 1 : 0) != 1; + assertEquals(isSplit, rawRecord.isSplit()); + assertEquals(key, rawRecord.getPrimaryKey()); + if (expectedVersion != null) { + assertThat(rawRecord.isVersionedInline(), is(true)); + assertEquals(expectedVersion, rawRecord.getVersion()); + } else { + assertThat(rawRecord.isVersionedInline(), is(false)); + assertNull(rawRecord.getVersion()); + } + + // Verify that the expected sizes are the same as the ones retrieved + assertEquals(expectedSizes.getKeyCount(), rawRecord.getKeyCount()); + assertEquals(expectedSizes.getValueSize(), rawRecord.getValueSize()); + assertEquals(expectedSizes.isSplit(), rawRecord.isSplit()); + assertEquals(expectedSizes.isVersionedInline(), rawRecord.isVersionedInline()); + + // Verify using sizeInfo and using the raw record get the same size information + assertEquals(rawRecord.getKeyCount(), sizeInfo.getKeyCount()); + assertEquals(rawRecord.getValueSize(), sizeInfo.getValueSize()); + assertEquals(rawRecord.isSplit(), sizeInfo.isSplit()); + assertEquals(rawRecord.isVersionedInline(), sizeInfo.isVersionedInline()); + + // Do not attempt to compare key sizes if the keys contain incomplete version stamps + if (!testConfig.useVersionInKey) { + assertEquals(expectedSizes.getKeySize(), rawRecord.getKeySize()); + assertEquals(rawRecord.getKeySize(), sizeInfo.getKeySize()); + } + } + + return rawRecord; + } + + @Nullable + private FDBRawRecord loadWithSplit(@Nonnull FDBRecordContext context, @Nonnull Tuple key, SplitHelperTestConfig testConfig, + @Nullable FDBStoredSizes expectedSizes, @Nullable byte[] expectedContents) { + return loadWithSplit(context, key, testConfig, expectedSizes, expectedContents, null); + } + + + + @Test + void loadWithSplitIsolated() { + loadWithSplitMultipleVersions(new SplitHelperTestConfig(false, false, false, false, false, false)); + } + + + + @MethodSource("testConfigs") + @ParameterizedTest(name = "loadWithSplitMultipleVersions[{0}]") + public void loadWithSplitMultipleVersions(SplitHelperTestConfig testConfig) { + this.testConfig = testConfig; + loadSingleRecordsMultipleVersions(testConfig, + (context, key, expectedSizes, expectedContents, version) -> loadWithSplit(context, key, testConfig, expectedSizes, expectedContents, version)); + + if (testConfig.splitLongRecords) { + try (FDBRecordContext context = openContext()) { + // Unsplit record followed by some unsplit stuff + // This particular error is caught by the single key unsplitter but not the mulit-key one + writeDummyRecord(context, Tuple.from(1307L), 1, false); + writeDummyRecord(context, Tuple.from(1307L), MEDIUM_COPIES, false); + RecordCoreException err = assertThrows(RecordCoreException.class, + () -> loadWithSplit(context, Tuple.from(1307L), testConfig, null, null)); + assertThat(err.getMessage(), containsString("Unsplit value followed by split")); + + commit(context); + } + } + } + + private FDBRawRecord scanSingleRecord(@Nonnull FDBRecordContext context, boolean reverse, @Nonnull Tuple key, @Nullable FDBStoredSizes expectedSizes, @Nullable byte[] expectedContents, @Nullable FDBRecordVersion version) { + final ScanProperties scanProperties = reverse ? ScanProperties.REVERSE_SCAN : ScanProperties.FORWARD_SCAN; + KeyValueCursor kvCursor = KeyValueCursor.Builder.withSubspace(subspace) + .setContext(context) + .setRange(TupleRange.allOf(key)) + .setScanProperties(scanProperties) + .build(); + SplitHelper.KeyValueUnsplitter kvUnsplitter = new SplitHelper.KeyValueUnsplitter(context, subspace, kvCursor, false, null, scanProperties); + + RecordCursorResult result = kvUnsplitter.getNext(); + if (expectedSizes == null || expectedContents == null) { + assertThat(result.hasNext(), is(false)); + assertEquals(RecordCursor.NoNextReason.SOURCE_EXHAUSTED, result.getNoNextReason()); + return null; + } else { + assertThat(result.hasNext(), is(true)); + final FDBRawRecord rawRecord = result.get(); + result = kvUnsplitter.getNext(); + assertThat(result.hasNext(), is(false)); + assertEquals(RecordCursor.NoNextReason.SOURCE_EXHAUSTED, result.getNoNextReason()); + + assertNotNull(rawRecord); + assertEquals(key, rawRecord.getPrimaryKey()); + assertArrayEquals(expectedContents, rawRecord.getRawRecord()); + assertEquals(expectedSizes.getKeyCount(), rawRecord.getKeyCount()); + assertEquals(expectedSizes.getKeySize(), rawRecord.getKeySize()); + assertEquals(expectedSizes.getValueSize(), rawRecord.getValueSize()); + boolean isSplit = rawRecord.getKeyCount() - (rawRecord.isVersionedInline() ? 1 : 0) != 1; + assertEquals(rawRecord.getKeyCount() - (rawRecord.isVersionedInline() ? 1 : 0) != 1, expectedSizes.isSplit()); + assertEquals(version != null, expectedSizes.isVersionedInline()); + + return rawRecord; + } + } + + @ParameterizedTest(name = "scan[reverse = {0}]") + @BooleanSource + public void scanSingleRecords(boolean reverse) { + loadSingleRecordsMultipleVersions(new SplitHelperTestConfig(true, false, FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES.getDefaultValue(), false, false, false), + (context, key, expectedSizes, expectedContents, version) -> scanSingleRecord(context, reverse, key, expectedSizes, expectedContents, version)); + } + + private List writeDummyRecords() { + final byte[] globalVersion = "_cushions_".getBytes(StandardCharsets.US_ASCII); + final List rawRecords = new ArrayList<>(); + // Generate primary keys using a generalization of the Fibonacci formula: https://oeis.org/A247698 + long currKey = 2308L; + long nextKey = 4261L; + + try (FDBRecordContext context = openContext()) { + for (int i = 0; i < 50; i++) { + FDBRecordVersion version = (i % 2 == 0) ? FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()) : null; + byte[] rawBytes = (i % 4 < 2) ? SHORT_STRING : MEDIUM_STRING; + Tuple key = Tuple.from(currKey); + FDBStoredSizes sizes = writeDummyRecord(context, key, version, (i % 4 < 2) ? 1 : MEDIUM_COPIES, false, false, 0); + rawRecords.add(new FDBRawRecord(key, rawBytes, version, sizes)); + + long temp = currKey + nextKey; + currKey = nextKey; + nextKey = temp; + } + + commit(context); + } + + return rawRecords; + } + + private List writeDummyRecordsMultipleTransactions(boolean useVersionInKey) { + final byte[] globalVersion = "_cushions_".getBytes(StandardCharsets.US_ASCII); + // Generate primary keys using a generalization of the Fibonacci formula: https://oeis.org/A247698 + long currKey = 2308L; + long nextKey = 4261L; + + final Tuple[] keys = new Tuple[50]; + final byte[][] rawBytesArr = new byte[50][]; + final FDBRecordVersion[] versions = new FDBRecordVersion[50]; + final int[] localVersions = new int[50]; + + final byte[] globalVs; + try (FDBRecordContext context = openContext()) { + for (int i = 0; i < 50; i++) { + keys[i] = Tuple.from(currKey); + rawBytesArr[i] = (i % 4 < 2) ? SHORT_STRING : MEDIUM_STRING; + versions[i] = (!useVersionInKey && i % 2 == 0) ? FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()) : null; + localVersions[i] = useVersionInKey ? context.claimLocalVersion() : 0; + writeDummyRecord(context, keys[i], versions[i], (i % 4 < 2) ? 1 : MEDIUM_COPIES, false, useVersionInKey, localVersions[i]); + + long temp = currKey + nextKey; + currKey = nextKey; + nextKey = temp; + } + commit(context); + globalVs = context.getVersionStamp(); + } + + // Scan back in a second transaction to build expected FDBRawRecord list with accurate committed sizes + final List rawRecords = new ArrayList<>(); + try (FDBRecordContext context = openContext()) { + KeyValueCursor kvCursor = KeyValueCursor.Builder.withSubspace(subspace) + .setContext(context) + .setRange(TupleRange.ALL) + .setScanProperties(ScanProperties.FORWARD_SCAN) + .build(); + List scannedRecords = new SplitHelper.KeyValueUnsplitter(context, subspace, kvCursor, false, null, ScanProperties.FORWARD_SCAN) + .asList().join(); + for (int i = 0; i < 50; i++) { + Tuple expectedKey = useVersionInKey + ? Tuple.from(Versionstamp.complete(globalVs, localVersions[i])).addAll(keys[i]) + : keys[i]; + FDBRawRecord scanned = scannedRecords.stream() + .filter(r -> r.getPrimaryKey().equals(expectedKey)) + .findFirst() + .orElseThrow(() -> new AssertionError("Missing record for key " + expectedKey)); + rawRecords.add(new FDBRawRecord(expectedKey, rawBytesArr[i], versions[i], scanned)); + } + commit(context); + } + + return rawRecords; + } + + @ParameterizedTest(name = "scanMultipleRecords[reverse = {0}]") + @BooleanSource + public void scanMultipleRecords(boolean reverse) { + final ScanProperties scanProperties = reverse ? ScanProperties.REVERSE_SCAN : ScanProperties.FORWARD_SCAN; + List rawRecords = writeDummyRecords(); + + try (FDBRecordContext context = openContext()) { + KeyValueCursor kvCursor = KeyValueCursor.Builder.withSubspace(subspace) + .setContext(context) + .setRange(TupleRange.ALL) + .setScanProperties(scanProperties) + .build(); + List readRecords = new SplitHelper.KeyValueUnsplitter(context, subspace, kvCursor, false, null, scanProperties) + .asList().join(); + if (reverse) { + readRecords = Lists.reverse(readRecords); + } + assertEquals(rawRecords.size(), readRecords.size()); + for (int i = 0; i < rawRecords.size(); i++) { + assertEquals(rawRecords.get(i), readRecords.get(i)); + } + assertEquals(rawRecords, readRecords); + + commit(context); + } + } + + @ParameterizedTest(name = "scanMultipleRecordsMultipleTransactions[reverse = {0}]") + @BooleanSource + void scanMultipleRecordsMultipleTransactions(boolean reverse) { + final ScanProperties scanProperties = reverse ? ScanProperties.REVERSE_SCAN : ScanProperties.FORWARD_SCAN; + List rawRecords = writeDummyRecordsMultipleTransactions(true); + + try (FDBRecordContext context = openContext()) { + KeyValueCursor kvCursor = KeyValueCursor.Builder.withSubspace(subspace) + .setContext(context) + .setRange(TupleRange.ALL) + .setScanProperties(scanProperties) + .build(); + List readRecords = new SplitHelper.KeyValueUnsplitter(context, subspace, kvCursor, false, null, scanProperties) + .asList().join(); + if (reverse) { + readRecords = Lists.reverse(readRecords); + } + assertEquals(rawRecords.size(), readRecords.size()); + for (int i = 0; i < rawRecords.size(); i++) { + assertEquals(rawRecords.get(i), readRecords.get(i)); + } + assertEquals(rawRecords, readRecords); + + commit(context); + } + } + + @MethodSource("limitsAndReverseArgs") + @ParameterizedTest(name = "scanContinuations [returnLimit = {0}, readLimit = {1}, reverse = {2}]") + public void scanContinuations(final int returnLimit, final int readLimit, final boolean reverse) { + List rawRecords = writeDummyRecords(); + if (reverse) { + rawRecords = Lists.reverse(rawRecords); + } + final Iterator expectedRecordIterator = rawRecords.iterator(); + + try (FDBRecordContext context = openContext()) { + byte[] continuation = null; + + do { + final ExecuteProperties executeProperties = ExecuteProperties.newBuilder() + .setReturnedRowLimit(returnLimit) + .setScannedRecordsLimit(readLimit) + .build(); + ScanProperties scanProperties = new ScanProperties(executeProperties, reverse); + RecordCursor kvCursor = KeyValueCursor.Builder.withSubspace(subspace) + .setContext(context) + .setRange(TupleRange.ALL) + .setScanProperties(scanProperties.with(ExecuteProperties::clearRowAndTimeLimits).with(ExecuteProperties::clearState)) + .setContinuation(continuation) + .build(); + RecordCursorIterator recordCursor = new SplitHelper.KeyValueUnsplitter(context, subspace, kvCursor, false, null, scanProperties.with(ExecuteProperties::clearReturnedRowLimit)) + .limitRowsTo(returnLimit) + .asIterator(); + + int retrieved = 0; + int rowsScanned = 0; + while (recordCursor.hasNext()) { + assertThat(retrieved, lessThan(returnLimit)); + assertThat(rowsScanned, lessThanOrEqualTo(readLimit)); + + FDBRawRecord nextRecord = recordCursor.next(); + assertNotNull(nextRecord); + assertThat(expectedRecordIterator.hasNext(), is(true)); + FDBRawRecord expectedRecord = expectedRecordIterator.next(); + assertEquals(expectedRecord, nextRecord); + + rowsScanned += nextRecord.getKeyCount(); + retrieved += 1; + } + + if (retrieved > 0) { + continuation = recordCursor.getContinuation(); + if (retrieved >= returnLimit) { + assertEquals(RecordCursor.NoNextReason.RETURN_LIMIT_REACHED, recordCursor.getNoNextReason()); + assertNotNull(continuation); + } else if (rowsScanned > readLimit) { + assertEquals(RecordCursor.NoNextReason.SCAN_LIMIT_REACHED, recordCursor.getNoNextReason()); + assertNotNull(continuation); + } else if (rowsScanned < readLimit) { + assertEquals(RecordCursor.NoNextReason.SOURCE_EXHAUSTED, recordCursor.getNoNextReason()); + } else { + // If we read exactly as many records as is allowed by the read record limit, then + // this probably means that we hit SCAN_LIMIT_REACHED, but it's also possible to + // hit SOURCE_EXHAUSTED if we hit the record read limit at exactly the same time + // as we needed to do another speculative read to determine if a split record + // continues or not. + assertEquals(readLimit, rowsScanned); + assertThat(recordCursor.getNoNextReason(), is(oneOf(RecordCursor.NoNextReason.SCAN_LIMIT_REACHED, RecordCursor.NoNextReason.SOURCE_EXHAUSTED))); + if (!recordCursor.getNoNextReason().isSourceExhausted()) { + assertNotNull(recordCursor.getContinuation()); + } + } + } else { + assertNull(recordCursor.getContinuation()); + continuation = null; + } + } while (continuation != null); + + commit(context); + } + } + + @MethodSource("limitsAndReverseArgs") + @ParameterizedTest(name = "scanContinuationsMultipleTransactions [returnLimit = {0}, readLimit = {1}, reverse = {2}]") + void scanContinuationsMultipleTransactions(final int returnLimit, final int readLimit, final boolean reverse) { + List rawRecords = writeDummyRecordsMultipleTransactions(true); + if (reverse) { + rawRecords = Lists.reverse(rawRecords); + } + final Iterator expectedRecordIterator = rawRecords.iterator(); + + try (FDBRecordContext context = openContext()) { + byte[] continuation = null; + + do { + final ExecuteProperties executeProperties = ExecuteProperties.newBuilder() + .setReturnedRowLimit(returnLimit) + .setScannedRecordsLimit(readLimit) + .build(); + ScanProperties scanProperties = new ScanProperties(executeProperties, reverse); + RecordCursor kvCursor = KeyValueCursor.Builder.withSubspace(subspace) + .setContext(context) + .setRange(TupleRange.ALL) + .setScanProperties(scanProperties.with(ExecuteProperties::clearRowAndTimeLimits).with(ExecuteProperties::clearState)) + .setContinuation(continuation) + .build(); + RecordCursorIterator recordCursor = new SplitHelper.KeyValueUnsplitter(context, subspace, kvCursor, false, null, scanProperties.with(ExecuteProperties::clearReturnedRowLimit)) + .limitRowsTo(returnLimit) + .asIterator(); + + int retrieved = 0; + int rowsScanned = 0; + while (recordCursor.hasNext()) { + assertThat(retrieved, lessThan(returnLimit)); + assertThat(rowsScanned, lessThanOrEqualTo(readLimit)); + + FDBRawRecord nextRecord = recordCursor.next(); + assertNotNull(nextRecord); + assertThat(expectedRecordIterator.hasNext(), is(true)); + FDBRawRecord expectedRecord = expectedRecordIterator.next(); + assertEquals(expectedRecord, nextRecord); + + rowsScanned += nextRecord.getKeyCount(); + retrieved += 1; + } + + if (retrieved > 0) { + continuation = recordCursor.getContinuation(); + if (retrieved >= returnLimit) { + assertEquals(RecordCursor.NoNextReason.RETURN_LIMIT_REACHED, recordCursor.getNoNextReason()); + assertNotNull(continuation); + } else if (rowsScanned > readLimit) { + assertEquals(RecordCursor.NoNextReason.SCAN_LIMIT_REACHED, recordCursor.getNoNextReason()); + assertNotNull(continuation); + } else if (rowsScanned < readLimit) { + assertEquals(RecordCursor.NoNextReason.SOURCE_EXHAUSTED, recordCursor.getNoNextReason()); + } else { + // If we read exactly as many records as is allowed by the read record limit, then + // this probably means that we hit SCAN_LIMIT_REACHED, but it's also possible to + // hit SOURCE_EXHAUSTED if we hit the record read limit at exactly the same time + // as we needed to do another speculative read to determine if a split record + // continues or not. + assertEquals(readLimit, rowsScanned); + assertThat(recordCursor.getNoNextReason(), is(oneOf(RecordCursor.NoNextReason.SCAN_LIMIT_REACHED, RecordCursor.NoNextReason.SOURCE_EXHAUSTED))); + if (!recordCursor.getNoNextReason().isSourceExhausted()) { + assertNotNull(recordCursor.getContinuation()); + } + } + } else { + assertNull(recordCursor.getContinuation()); + continuation = null; + } + } while (continuation != null); + + commit(context); + } + } + + /** + * When two saveWithSplit calls use the same incomplete versionstamp (same localVersion, same key) within + * one transaction, we may get a failure or data corruption. The localVersionCache (map by key) may contain + * previous + * values from an identical key (same versionstamp/localversion/PK) but different splits and may not collide + * directly + * with the previous values. This test shows the case where there is a collision since the split numbers are the + * same. + */ + @Test + void saveWithSplitVersionInKeyOverwriteInTransaction() { + final Tuple key = Tuple.from(1066L); + final int localVersion; + try (FDBRecordContext context = openContext()) { + localVersion = context.claimLocalVersion(); + // First write: VERY_LONG_STRING requires multiple splits + final VersioningSplitKeyValueHelper splitKeyHelper = new VersioningSplitKeyValueHelper(Versionstamp.incomplete(localVersion)); + SplitHelper.saveWithSplit(context, subspace, key, VERY_LONG_STRING, null, + true, false, + splitKeyHelper, + false, null, null); + + // Second write: LONG_STRING — same localVersion, same key, shorter value (fewer splits) + final RecordCoreInternalException ex = assertThrows(RecordCoreInternalException.class, () -> SplitHelper.saveWithSplit(context, subspace, key, LONG_STRING, null, + true, false, + splitKeyHelper, + false, null, null)); + assertTrue(ex.getMessage().contains("Key with version overwritten")); + } + } +} diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java index db4080eb76..afc68eda4a 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java @@ -21,13 +21,12 @@ package com.apple.foundationdb.record.provider.foundationdb; import com.apple.foundationdb.KeyValue; -import com.apple.foundationdb.MutationType; import com.apple.foundationdb.ReadTransaction; +import com.apple.foundationdb.Transaction; import com.apple.foundationdb.record.ExecuteProperties; import com.apple.foundationdb.record.FDBRecordStoreProperties; import com.apple.foundationdb.record.RecordCoreArgumentException; import com.apple.foundationdb.record.RecordCoreException; -import com.apple.foundationdb.record.RecordCoreInternalException; import com.apple.foundationdb.record.RecordCursor; import com.apple.foundationdb.record.RecordCursorIterator; import com.apple.foundationdb.record.RecordCursorResult; @@ -38,14 +37,12 @@ import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.ByteArrayUtil; import com.apple.foundationdb.tuple.Tuple; -import com.apple.foundationdb.tuple.Versionstamp; import com.apple.test.BooleanSource; import com.apple.test.Tags; import com.google.common.collect.Lists; import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; -import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.MethodSource; @@ -76,7 +73,6 @@ import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; -import static org.junit.jupiter.api.Assertions.assertTrue; /** * Tests for checking the validity of the "split helper" utility class that handles breaking @@ -135,23 +131,13 @@ static class SplitHelperTestConfig { private final boolean unrollRecordDeletes; private final boolean loadViaGets; private final boolean isDryRun; - private final boolean useVersionInKey; - public SplitHelperTestConfig(boolean splitLongRecords, boolean omitUnsplitSuffix, boolean unrollRecordDeletes, - boolean loadViaGets, boolean isDryRun, boolean useVersionInKey) { + public SplitHelperTestConfig(boolean splitLongRecords, boolean omitUnsplitSuffix, boolean unrollRecordDeletes, boolean loadViaGets, boolean isDryRun) { this.splitLongRecords = splitLongRecords; this.omitUnsplitSuffix = omitUnsplitSuffix; this.unrollRecordDeletes = unrollRecordDeletes; this.loadViaGets = loadViaGets; this.isDryRun = isDryRun; - this.useVersionInKey = useVersionInKey; - } - - public SplitKeyValueHelper keyHelper(int localVersion) { - if (useVersionInKey) { - return new VersioningSplitKeyValueHelper(Versionstamp.incomplete(localVersion)); - } - return DefaultSplitKeyValueHelper.INSTANCE; } @Nonnull @@ -173,7 +159,6 @@ public String toString() { ", unrollRecordDeletes=" + unrollRecordDeletes + ", loadViaGets=" + loadViaGets + ", isDryRun=" + isDryRun + - ", useVersionInKey=" + useVersionInKey + '}'; } @@ -186,34 +171,28 @@ public boolean equals(final Object o) { return false; } final SplitHelperTestConfig that = (SplitHelperTestConfig)o; - return splitLongRecords == that.splitLongRecords && omitUnsplitSuffix == that.omitUnsplitSuffix && - unrollRecordDeletes == that.unrollRecordDeletes && loadViaGets == that.loadViaGets && - isDryRun == that.isDryRun && useVersionInKey == that.useVersionInKey; + return splitLongRecords == that.splitLongRecords && omitUnsplitSuffix == that.omitUnsplitSuffix && unrollRecordDeletes == that.unrollRecordDeletes && loadViaGets == that.loadViaGets && isDryRun == that.isDryRun; } @Override public int hashCode() { - return Objects.hash(splitLongRecords, omitUnsplitSuffix, unrollRecordDeletes, loadViaGets, useVersionInKey); + return Objects.hash(splitLongRecords, omitUnsplitSuffix, unrollRecordDeletes, loadViaGets); } public static Stream allValidConfigs() { // Note that splitLongRecords="true" && omitUnsplitSuffix="true" is not valid - // Note that useVersionInKey="true" && isDryRun="true" is not valid (versionstamp never completes without commit) - return Stream.of(false, true).flatMap(useVersionInKey -> - Stream.of(false, true).flatMap(splitLongRecords -> - (splitLongRecords ? Stream.of(false) : Stream.of(false, true)).flatMap(omitUnsplitSuffix -> - Stream.of(false, true).flatMap(unrollRecordDeletes -> - Stream.of(false, true).flatMap(loadViaGets -> - Stream.of(false, true) - .filter(isDryRun -> !useVersionInKey || !isDryRun) - .map(isDryRun -> - new SplitHelperTestConfig(splitLongRecords, omitUnsplitSuffix, unrollRecordDeletes, loadViaGets, isDryRun, useVersionInKey))))))); + return Stream.of(false, true).flatMap(splitLongRecords -> + (splitLongRecords ? Stream.of(false) : Stream.of(false, true)).flatMap(omitUnsplitSuffix -> + Stream.of(false, true).flatMap(unrollRecordDeletes -> + Stream.of(false, true).flatMap(loadViaGets -> + Stream.of(false, true).map(isDryRun -> + new SplitHelperTestConfig(splitLongRecords, omitUnsplitSuffix, unrollRecordDeletes, loadViaGets, isDryRun)))))); } public static SplitHelperTestConfig getDefault() { return new SplitHelperTestConfig(true, false, FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES.getDefaultValue(), - FDBRecordStoreProperties.LOAD_RECORDS_VIA_GETS.getDefaultValue(), false, false); + FDBRecordStoreProperties.LOAD_RECORDS_VIA_GETS.getDefaultValue(), false); } } @@ -261,55 +240,38 @@ private SplitHelper.SizeInfo saveUnsuccessfully(@Nonnull F return sizeInfo; } - private SplitHelper.SizeInfo saveOnly(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, - @Nullable FDBRecordVersion version, - @Nonnull SplitHelperTestConfig testConfig, - @Nullable FDBStoredSizes previousSizeInfo, - int localVersion) { + private SplitHelper.SizeInfo saveSuccessfully(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, + @Nullable FDBRecordVersion version, + @Nonnull SplitHelperTestConfig testConfig, + @Nullable FDBStoredSizes previousSizeInfo) { final SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); SplitHelper.saveWithSplit(context, subspace, key, serialized, version, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, - testConfig.keyHelper(localVersion), previousSizeInfo != null, previousSizeInfo, sizeInfo); + DefaultSplitKeyValueHelper.INSTANCE, previousSizeInfo != null, previousSizeInfo, sizeInfo); int dataKeyCount = (serialized.length - 1) / SplitHelper.SPLIT_RECORD_SIZE + 1; boolean isSplit = dataKeyCount > 1; int keyCount = dataKeyCount; if (version != null) { keyCount += 1; } + int keySize = (subspace.pack().length + key.pack().length) * keyCount; assertEquals(isSplit, sizeInfo.isSplit()); assertEquals(keyCount, sizeInfo.getKeyCount()); - int valueSize = serialized.length + (version != null ? 1 + FDBRecordVersion.VERSION_LENGTH : 0); - assertEquals(valueSize, sizeInfo.getValueSize()); - assertEquals(version != null, sizeInfo.isVersionedInline()); - if (!testConfig.useVersionInKey) { - // Key size can only be asserted when the key is fully known at write time (not the case for versioning keys, - // since the versionstamp in the key is not yet resolved until after commit) - int keySize = (subspace.pack().length + key.pack().length) * keyCount; - if (testConfig.hasSplitPoints()) { - // Add in the counters the split points. - if (!isSplit) { - keySize += 1; // As 0 requires 1 byte when Tuple packed - } else { - keySize += dataKeyCount * 2; // As each split point is two bytes when tuple packed - } - } - if (version != null) { - keySize += 2; + if (testConfig.hasSplitPoints()) { + // Add in the the counters the split points. + if (!isSplit) { + keySize += 1; // As 0 requires 1 byte when Tuple packed + } else { + keySize += dataKeyCount * 2; // As each split point is two bytes when tuple packed } - assertEquals(keySize, sizeInfo.getKeySize()); } - return sizeInfo; - } - - private void verifySuccessfullySaved(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, - @Nullable FDBRecordVersion version, - @Nonnull SplitHelperTestConfig testConfig) { - // Similar to the calculation in saveOnly - int dataKeyCount = (serialized.length - 1) / SplitHelper.SPLIT_RECORD_SIZE + 1; - boolean isSplit = dataKeyCount > 1; - int keyCount = dataKeyCount; if (version != null) { - keyCount += 1; + keySize += 2; } + int valueSize = serialized.length + (version != null ? 1 + FDBRecordVersion.VERSION_LENGTH : 0); + assertEquals(keySize, sizeInfo.getKeySize()); + assertEquals(valueSize, sizeInfo.getValueSize()); + assertEquals(version != null, sizeInfo.isVersionedInline()); + final Subspace keySubspace = subspace.subspace(key); RecordCursorIterator kvCursor = KeyValueCursor.Builder.withSubspace(keySubspace) .setContext(context) @@ -369,17 +331,7 @@ private void verifySuccessfullySaved(@Nonnull FDBRecordContext context, @Nonnull } else { assertNull(versionBytes); } - assertNotNull(valueBytes); - assertArrayEquals(serialized, valueBytes); - } - private SplitHelper.SizeInfo saveSuccessfully(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, - @Nullable FDBRecordVersion version, - @Nonnull SplitHelperTestConfig testConfig, - @Nullable FDBStoredSizes previousSizeInfo) { - // localVersion=0: single-transaction callers always use useVersionInKey=false, so the value doesn't matter - SplitHelper.SizeInfo sizeInfo = saveOnly(context, key, serialized, version, testConfig, previousSizeInfo, 0); - verifySuccessfullySaved(context, key, serialized, version, testConfig); return sizeInfo; } @@ -457,43 +409,10 @@ private SplitHelper.SizeInfo saveWithSplit(@Nonnull FDBRecordContext context, @N return saveWithSplit(context, key, serialized, null, testConfig); } - public static Stream splitKeyEquivalenceCases() { - return Stream.of( - Arguments.of(Tuple.from(1066L), SplitHelper.START_SPLIT_RECORD), - Arguments.of(Tuple.from(1066L), SplitHelper.START_SPLIT_RECORD + 1), - Arguments.of(Tuple.from(1066L), SplitHelper.START_SPLIT_RECORD + 2), - Arguments.of(Tuple.from(1066L), SplitHelper.UNSPLIT_RECORD), - Arguments.of(Tuple.from(1066L), SplitHelper.RECORD_VERSION), - Arguments.of(Tuple.from(1066L, "extra"), SplitHelper.START_SPLIT_RECORD), - Arguments.of(Tuple.from(), SplitHelper.START_SPLIT_RECORD), - Arguments.of(Tuple.from(), SplitHelper.UNSPLIT_RECORD) - ); - } - - /** - * Verify that the refactored split call site in {@link SplitHelper} — which now passes - * {@code subspace} + {@code key.add(index)} to {@code packSplitKey} instead of the old - * {@code subspace.subspace(key)} + {@code Tuple.from(index)} — produces identical byte keys - * for {@link DefaultSplitKeyValueHelper}. - */ - @MethodSource("splitKeyEquivalenceCases") - @ParameterizedTest(name = "defaultHelperSplitKeyEquivalence[key={0}, index={1}]") - void defaultHelperSplitKeyEquivalence(Tuple key, long index) { - Subspace subspace = new Subspace(Tuple.from("test")); - - // Old call site in SplitHelper: subspace.subspace(key).pack(index) - byte[] oldKey = subspace.subspace(key).pack(index); - // New call site: packSplitKey(subspace, key.add(index)) = subspace.pack(key.add(index)) - byte[] newKey = DefaultSplitKeyValueHelper.INSTANCE.packSplitKey(subspace, key.add(index)); - - assertArrayEquals(oldKey, newKey); - } - @MethodSource("testConfigs") @ParameterizedTest(name = "saveWithSplit[{0}]") public void saveWithSplit(@Nonnull SplitHelperTestConfig testConfig) { this.testConfig = testConfig; - Assumptions.assumeFalse(testConfig.useVersionInKey); try (FDBRecordContext context = openContext()) { // No version FDBStoredSizes sizes1 = saveWithSplit(context, Tuple.from(1066L), SHORT_STRING, testConfig); @@ -511,76 +430,10 @@ public void saveWithSplit(@Nonnull SplitHelperTestConfig testConfig) { } } - @MethodSource("testConfigs") - @ParameterizedTest(name = "saveWithSplitMultipleTransactions[{0}]") - public void saveWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig testConfig) { - this.testConfig = testConfig; - - final Tuple key1 = Tuple.from(1066L); - final Tuple key2 = Tuple.from(1415L); - final Tuple key3 = Tuple.from(1776L); - final int localVersion1; - final int localVersion2; - final int localVersion3; - final byte[] globalVs; - try (FDBRecordContext context = openContext()) { - localVersion1 = context.claimLocalVersion(); - saveWithSplitForMultipleTransactions(context, key1, SHORT_STRING, null, testConfig, null, localVersion1); - localVersion2 = context.claimLocalVersion(); - saveWithSplitForMultipleTransactions(context, key2, LONG_STRING, null, testConfig, null, localVersion2); - localVersion3 = context.claimLocalVersion(); - saveWithSplitForMultipleTransactions(context, key3, VERY_LONG_STRING, null, testConfig, null, localVersion3); - commit(context); - globalVs = context.getVersionStamp(); - } - - final Tuple verifyKey1; - final Tuple verifyKey2; - final Tuple verifyKey3; - if (testConfig.useVersionInKey) { - verifyKey1 = Tuple.from(Versionstamp.complete(globalVs, localVersion1)).addAll(key1); - verifyKey2 = Tuple.from(Versionstamp.complete(globalVs, localVersion2)).addAll(key2); - verifyKey3 = Tuple.from(Versionstamp.complete(globalVs, localVersion3)).addAll(key3); - } else { - verifyKey1 = key1; - verifyKey2 = key2; - verifyKey3 = key3; - } - if (!testConfig.isDryRun) { - try (FDBRecordContext context = openContext()) { - verifySuccessfullySaved(context, verifyKey1, SHORT_STRING, null, testConfig); - if (testConfig.splitLongRecords) { - verifySuccessfullySaved(context, verifyKey2, LONG_STRING, null, testConfig); - verifySuccessfullySaved(context, verifyKey3, VERY_LONG_STRING, null, testConfig); - } - } - } - } - - private SplitHelper.SizeInfo saveWithSplitForMultipleTransactions(@Nonnull FDBRecordContext context, @Nonnull Tuple key, - byte[] serialized, - @Nullable FDBRecordVersion version, - @Nonnull SplitHelperTestConfig testConfig, - @Nullable FDBStoredSizes previousSizeInfo, - int localVersion) { - if (testConfig.omitUnsplitSuffix && version != null) { - return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, - RecordCoreArgumentException.class, "Cannot include version"); - } else if (!testConfig.splitLongRecords && serialized.length > SplitHelper.SPLIT_RECORD_SIZE) { - return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, - RecordCoreException.class, "Record is too long"); - } else if (testConfig.isDryRun) { - return dryRunSetSizeInfo(context, key, serialized, version, testConfig, previousSizeInfo); - } else { - return saveOnly(context, key, serialized, version, testConfig, previousSizeInfo, localVersion); - } - } - @MethodSource("testConfigs") @ParameterizedTest(name = "saveWithSplitAndIncompleteVersions[{0}]") public void saveWithSplitAndIncompleteVersions(SplitHelperTestConfig testConfig) { this.testConfig = testConfig; - Assumptions.assumeFalse(testConfig.useVersionInKey); final byte[] versionstamp; try (FDBRecordContext context = openContext()) { saveWithSplit(context, Tuple.from(962L), SHORT_STRING, FDBRecordVersion.incomplete(context.claimLocalVersion()), testConfig); @@ -623,8 +476,8 @@ public void saveWithSplitAndIncompleteVersions(SplitHelperTestConfig testConfig) @ParameterizedTest(name = "saveWithSplitAndCompleteVersion[{0}]") public void saveWithSplitAndCompleteVersions(SplitHelperTestConfig testConfig) { this.testConfig = testConfig; - Assumptions.assumeFalse(testConfig.useVersionInKey); try (FDBRecordContext context = openContext()) { + // With complete version byte[] globalVersion = "karlgrosse".getBytes(StandardCharsets.US_ASCII); saveWithSplit(context, Tuple.from(800L), SHORT_STRING, FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()), testConfig); saveWithSplit(context, Tuple.from(813L), LONG_STRING, FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()), testConfig); @@ -649,61 +502,47 @@ public void saveWithSplitAndCompleteVersions(SplitHelperTestConfig testConfig) { } @Nonnull - private FDBStoredSizes writeDummyRecord(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @Nullable FDBRecordVersion version, int splits, boolean omitUnsplitSuffix, boolean useVersionInKey, int localVersion) { + private FDBStoredSizes writeDummyRecord(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @Nullable FDBRecordVersion version, int splits, boolean omitUnsplitSuffix) { + final Transaction tr = context.ensureActive(); SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); if (version != null) { assertThat(omitUnsplitSuffix, is(false)); sizeInfo.setVersionedInline(true); - Tuple keyTuple = key.add(SplitHelper.RECORD_VERSION); + byte[] keyBytes = subspace.pack(key.add(SplitHelper.RECORD_VERSION)); byte[] valueBytes = SplitHelper.packVersion(version); - // Note that this will not mutate the version in the value - writeDummyKV(context, keyTuple, valueBytes, sizeInfo, useVersionInKey, localVersion); + tr.set(keyBytes, valueBytes); + sizeInfo.add(keyBytes, valueBytes); } if (splits == 1) { if (omitUnsplitSuffix) { - Tuple keyTuple = key; - byte[] valueBytes = SHORT_STRING; - writeDummyKV(context, keyTuple, valueBytes, sizeInfo, useVersionInKey, localVersion); + byte[] keyBytes = subspace.pack(key); + sizeInfo.add(keyBytes, SHORT_STRING); + tr.set(keyBytes, SHORT_STRING); } else { - Tuple keyTuple = key.add(SplitHelper.UNSPLIT_RECORD); - byte[] valueBytes = SHORT_STRING; - writeDummyKV(context, keyTuple, valueBytes, sizeInfo, useVersionInKey, localVersion); + byte[] keyBytes = subspace.pack(key.add(SplitHelper.UNSPLIT_RECORD)); + sizeInfo.add(keyBytes, SHORT_STRING); + tr.set(keyBytes, SHORT_STRING); } sizeInfo.setSplit(false); } else { for (int i = 0; i < splits; i++) { - Tuple keyTuple = key.add(SplitHelper.START_SPLIT_RECORD + i); - byte[] valueBytes = SHORT_STRING; - writeDummyKV(context, keyTuple, valueBytes, sizeInfo, useVersionInKey, localVersion); + byte[] keyBytes = subspace.pack(key.add(SplitHelper.START_SPLIT_RECORD + i)); + sizeInfo.add(keyBytes, SHORT_STRING); + tr.set(keyBytes, SHORT_STRING); } sizeInfo.setSplit(true); } return sizeInfo; } - private void writeDummyKV(@Nonnull FDBRecordContext context, @Nonnull Tuple keyTuple, - byte[] valueBytes, @Nonnull SplitHelper.SizeInfo sizeInfo, boolean useVersionInKey, int localVersion) { - byte[] keyBytes; - // Mimic the work done in both SplitKeyValueHelper - if (useVersionInKey) { - Tuple versionedKeyTuple = Tuple.from(Versionstamp.incomplete(localVersion)).addAll(keyTuple); - keyBytes = subspace.packWithVersionstamp(versionedKeyTuple); - context.addVersionMutation(MutationType.SET_VERSIONSTAMPED_KEY, keyBytes, valueBytes); - } else { - keyBytes = subspace.pack(keyTuple); - context.ensureActive().set(keyBytes, valueBytes); - } - sizeInfo.add(keyBytes, valueBytes); - } - @Nonnull private FDBStoredSizes writeDummyRecord(@Nonnull FDBRecordContext context, @Nonnull Tuple key, int splits, boolean omitUnsplitSuffix) { - return writeDummyRecord(context, key, null, splits, omitUnsplitSuffix, false, 0); + return writeDummyRecord(context, key, null, splits, omitUnsplitSuffix); } @Nonnull private FDBStoredSizes writeDummyRecord(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @Nonnull FDBRecordVersion version, int splits) { - return writeDummyRecord(context, key, version, splits, false, false, 0); + return writeDummyRecord(context, key, version, splits, false); } private void deleteSplit(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @@ -746,90 +585,6 @@ public void deleteWithSplit(SplitHelperTestConfig testConfig) { } } - @MethodSource("testConfigs") - @ParameterizedTest(name = "deleteWithSplitMultipleTransactions[{0}]") - public void deleteWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig testConfig) { - this.testConfig = testConfig; - - final Tuple key1 = Tuple.from(-660L); - final Tuple key2 = Tuple.from(-581L); - final Tuple key3 = Tuple.from(-549L); - final Tuple key4 = Tuple.from(-510L); - - // tx1: write records - final int localVersion1; - final int localVersion2; - final int localVersion3; - final int localVersion4; - final byte[] globalVs; - try (FDBRecordContext context = openContext()) { - if (testConfig.useVersionInKey) { - localVersion1 = context.claimLocalVersion(); - writeDummyRecord(context, key1, null, 1, testConfig.omitUnsplitSuffix, true, localVersion1); - localVersion2 = context.claimLocalVersion(); - writeDummyRecord(context, key2, null, 1, testConfig.omitUnsplitSuffix, true, localVersion2); - if (testConfig.splitLongRecords) { - localVersion3 = context.claimLocalVersion(); - writeDummyRecord(context, key3, null, 5, testConfig.omitUnsplitSuffix, true, localVersion3); - localVersion4 = context.claimLocalVersion(); - writeDummyRecord(context, key4, null, 5, testConfig.omitUnsplitSuffix, true, localVersion4); - } else { - localVersion3 = -1; - localVersion4 = -1; - } - } else { - writeDummyRecord(context, key1, 1, testConfig.omitUnsplitSuffix); - writeDummyRecord(context, key2, 1, testConfig.omitUnsplitSuffix); - localVersion1 = localVersion2 = localVersion3 = localVersion4 = -1; - if (testConfig.splitLongRecords) { - writeDummyRecord(context, key3, 5, testConfig.omitUnsplitSuffix); - writeDummyRecord(context, key4, 5, testConfig.omitUnsplitSuffix); - } - } - commit(context); - globalVs = context.getVersionStamp(); - } - - // Resolve the delete keys: for versioning helper the stored key is [vs][originalKey] - final Tuple deleteKey1; - final Tuple deleteKey2; - final Tuple deleteKey3; - final Tuple deleteKey4; - if (testConfig.useVersionInKey) { - deleteKey1 = Tuple.from(Versionstamp.complete(globalVs, localVersion1)).addAll(key1); - deleteKey2 = Tuple.from(Versionstamp.complete(globalVs, localVersion2)).addAll(key2); - deleteKey3 = testConfig.splitLongRecords ? Tuple.from(Versionstamp.complete(globalVs, localVersion3)).addAll(key3) : null; - deleteKey4 = testConfig.splitLongRecords ? Tuple.from(Versionstamp.complete(globalVs, localVersion4)).addAll(key4) : null; - } else { - deleteKey1 = key1; - deleteKey2 = key2; - deleteKey3 = key3; - deleteKey4 = key4; - } - - // tx2: delete records - try (FDBRecordContext context = openContext()) { - SplitHelper.deleteSplit(context, subspace, deleteKey1, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, false, null); - SplitHelper.deleteSplit(context, subspace, deleteKey2, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, false, null); - if (testConfig.splitLongRecords) { - SplitHelper.deleteSplit(context, subspace, deleteKey3, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, false, null); - SplitHelper.deleteSplit(context, subspace, deleteKey4, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, false, null); - } - commit(context); - } - - // tx3: verify subspace is empty - try (FDBRecordContext context = openContext()) { - int count = KeyValueCursor.Builder.withSubspace(subspace) - .setContext(context) - .setScanProperties(ScanProperties.FORWARD_SCAN) - .build() - .getCount() - .join(); - assertEquals(0, count); - } - } - static Stream deleteWithSplitAndVersion() { return Stream.of(false, true).flatMap(splitLongRecords -> Stream.of(false, true).map(unrollSingleRecordDeletes -> @@ -840,7 +595,6 @@ static Stream deleteWithSplitAndVersion() { @MethodSource("testConfigs") public void deleteWithSplitAndVersion(SplitHelperTestConfig testConfig) { this.testConfig = testConfig; - Assumptions.assumeFalse(testConfig.useVersionInKey); Assumptions.assumeFalse(testConfig.omitUnsplitSuffix); final byte[] globalVersion = "chrysan_th".getBytes(StandardCharsets.US_ASCII); try (FDBRecordContext context = openContext()) { @@ -1014,7 +768,6 @@ private FDBRawRecord loadWithSplit(@Nonnull FDBRecordContext context, @Nonnull T @ParameterizedTest(name = "loadWithSplit[{0}]") public void loadWithSplit(SplitHelperTestConfig testConfig) { this.testConfig = testConfig; - Assumptions.assumeFalse(testConfig.useVersionInKey); loadSingleRecords(testConfig, (context, key, expectedSizes, expectedContents, version) -> loadWithSplit(context, key, testConfig, expectedSizes, expectedContents, version)); @@ -1071,7 +824,7 @@ private FDBRawRecord scanSingleRecord(@Nonnull FDBRecordContext context, boolean @ParameterizedTest(name = "scan[reverse = {0}]") @BooleanSource public void scanSingleRecords(boolean reverse) { - loadSingleRecords(new SplitHelperTestConfig(true, false, FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES.getDefaultValue(), false, false, false), + loadSingleRecords(new SplitHelperTestConfig(true, false, FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES.getDefaultValue(), false, false), (context, key, expectedSizes, expectedContents, version) -> scanSingleRecord(context, reverse, key, expectedSizes, expectedContents, version)); } @@ -1087,7 +840,7 @@ private List writeDummyRecords() { FDBRecordVersion version = (i % 2 == 0) ? FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()) : null; byte[] rawBytes = (i % 4 < 2) ? SHORT_STRING : MEDIUM_STRING; Tuple key = Tuple.from(currKey); - FDBStoredSizes sizes = writeDummyRecord(context, key, version, (i % 4 < 2) ? 1 : MEDIUM_COPIES, false, false, 0); + FDBStoredSizes sizes = writeDummyRecord(context, key, version, (i % 4 < 2) ? 1 : MEDIUM_COPIES, false); rawRecords.add(new FDBRawRecord(key, rawBytes, version, sizes)); long temp = currKey + nextKey; @@ -1101,60 +854,6 @@ private List writeDummyRecords() { return rawRecords; } - private List writeDummyRecordsMultipleTransactions(boolean useVersionInKey) { - final byte[] globalVersion = "_cushions_".getBytes(StandardCharsets.US_ASCII); - // Generate primary keys using a generalization of the Fibonacci formula: https://oeis.org/A247698 - long currKey = 2308L; - long nextKey = 4261L; - - final Tuple[] keys = new Tuple[50]; - final byte[][] rawBytesArr = new byte[50][]; - final FDBRecordVersion[] versions = new FDBRecordVersion[50]; - final int[] localVersions = new int[50]; - - final byte[] globalVs; - try (FDBRecordContext context = openContext()) { - for (int i = 0; i < 50; i++) { - keys[i] = Tuple.from(currKey); - rawBytesArr[i] = (i % 4 < 2) ? SHORT_STRING : MEDIUM_STRING; - versions[i] = (!useVersionInKey && i % 2 == 0) ? FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()) : null; - localVersions[i] = useVersionInKey ? context.claimLocalVersion() : 0; - writeDummyRecord(context, keys[i], versions[i], (i % 4 < 2) ? 1 : MEDIUM_COPIES, false, useVersionInKey, localVersions[i]); - - long temp = currKey + nextKey; - currKey = nextKey; - nextKey = temp; - } - commit(context); - globalVs = context.getVersionStamp(); - } - - // Scan back in a second transaction to build expected FDBRawRecord list with accurate committed sizes - final List rawRecords = new ArrayList<>(); - try (FDBRecordContext context = openContext()) { - KeyValueCursor kvCursor = KeyValueCursor.Builder.withSubspace(subspace) - .setContext(context) - .setRange(TupleRange.ALL) - .setScanProperties(ScanProperties.FORWARD_SCAN) - .build(); - List scannedRecords = new SplitHelper.KeyValueUnsplitter(context, subspace, kvCursor, false, null, ScanProperties.FORWARD_SCAN) - .asList().join(); - for (int i = 0; i < 50; i++) { - Tuple expectedKey = useVersionInKey - ? Tuple.from(Versionstamp.complete(globalVs, localVersions[i])).addAll(keys[i]) - : keys[i]; - FDBRawRecord scanned = scannedRecords.stream() - .filter(r -> r.getPrimaryKey().equals(expectedKey)) - .findFirst() - .orElseThrow(() -> new AssertionError("Missing record for key " + expectedKey)); - rawRecords.add(new FDBRawRecord(expectedKey, rawBytesArr[i], versions[i], scanned)); - } - commit(context); - } - - return rawRecords; - } - @ParameterizedTest(name = "scanMultipleRecords[reverse = {0}]") @BooleanSource public void scanMultipleRecords(boolean reverse) { @@ -1182,33 +881,6 @@ public void scanMultipleRecords(boolean reverse) { } } - @ParameterizedTest(name = "scanMultipleRecordsMultipleTransactions[reverse = {0}]") - @BooleanSource - void scanMultipleRecordsMultipleTransactions(boolean reverse) { - final ScanProperties scanProperties = reverse ? ScanProperties.REVERSE_SCAN : ScanProperties.FORWARD_SCAN; - List rawRecords = writeDummyRecordsMultipleTransactions(true); - - try (FDBRecordContext context = openContext()) { - KeyValueCursor kvCursor = KeyValueCursor.Builder.withSubspace(subspace) - .setContext(context) - .setRange(TupleRange.ALL) - .setScanProperties(scanProperties) - .build(); - List readRecords = new SplitHelper.KeyValueUnsplitter(context, subspace, kvCursor, false, null, scanProperties) - .asList().join(); - if (reverse) { - readRecords = Lists.reverse(readRecords); - } - assertEquals(rawRecords.size(), readRecords.size()); - for (int i = 0; i < rawRecords.size(); i++) { - assertEquals(rawRecords.get(i), readRecords.get(i)); - } - assertEquals(rawRecords, readRecords); - - commit(context); - } - } - @MethodSource("limitsAndReverseArgs") @ParameterizedTest(name = "scanContinuations [returnLimit = {0}, readLimit = {1}, reverse = {2}]") public void scanContinuations(final int returnLimit, final int readLimit, final boolean reverse) { @@ -1285,110 +957,35 @@ public void scanContinuations(final int returnLimit, final int readLimit, final } } - @MethodSource("limitsAndReverseArgs") - @ParameterizedTest(name = "scanContinuationsMultipleTransactions [returnLimit = {0}, readLimit = {1}, reverse = {2}]") - void scanContinuationsMultipleTransactions(final int returnLimit, final int readLimit, final boolean reverse) { - List rawRecords = writeDummyRecordsMultipleTransactions(true); - if (reverse) { - rawRecords = Lists.reverse(rawRecords); - } - final Iterator expectedRecordIterator = rawRecords.iterator(); - - try (FDBRecordContext context = openContext()) { - byte[] continuation = null; - - do { - final ExecuteProperties executeProperties = ExecuteProperties.newBuilder() - .setReturnedRowLimit(returnLimit) - .setScannedRecordsLimit(readLimit) - .build(); - ScanProperties scanProperties = new ScanProperties(executeProperties, reverse); - RecordCursor kvCursor = KeyValueCursor.Builder.withSubspace(subspace) - .setContext(context) - .setRange(TupleRange.ALL) - .setScanProperties(scanProperties.with(ExecuteProperties::clearRowAndTimeLimits).with(ExecuteProperties::clearState)) - .setContinuation(continuation) - .build(); - RecordCursorIterator recordCursor = new SplitHelper.KeyValueUnsplitter(context, subspace, kvCursor, false, null, scanProperties.with(ExecuteProperties::clearReturnedRowLimit)) - .limitRowsTo(returnLimit) - .asIterator(); - - int retrieved = 0; - int rowsScanned = 0; - while (recordCursor.hasNext()) { - assertThat(retrieved, lessThan(returnLimit)); - assertThat(rowsScanned, lessThanOrEqualTo(readLimit)); - - FDBRawRecord nextRecord = recordCursor.next(); - assertNotNull(nextRecord); - assertThat(expectedRecordIterator.hasNext(), is(true)); - FDBRawRecord expectedRecord = expectedRecordIterator.next(); - assertEquals(expectedRecord, nextRecord); - - rowsScanned += nextRecord.getKeyCount(); - retrieved += 1; - } - - if (retrieved > 0) { - continuation = recordCursor.getContinuation(); - if (retrieved >= returnLimit) { - assertEquals(RecordCursor.NoNextReason.RETURN_LIMIT_REACHED, recordCursor.getNoNextReason()); - assertNotNull(continuation); - } else if (rowsScanned > readLimit) { - assertEquals(RecordCursor.NoNextReason.SCAN_LIMIT_REACHED, recordCursor.getNoNextReason()); - assertNotNull(continuation); - } else if (rowsScanned < readLimit) { - assertEquals(RecordCursor.NoNextReason.SOURCE_EXHAUSTED, recordCursor.getNoNextReason()); - } else { - // If we read exactly as many records as is allowed by the read record limit, then - // this probably means that we hit SCAN_LIMIT_REACHED, but it's also possible to - // hit SOURCE_EXHAUSTED if we hit the record read limit at exactly the same time - // as we needed to do another speculative read to determine if a split record - // continues or not. - assertEquals(readLimit, rowsScanned); - assertThat(recordCursor.getNoNextReason(), is(oneOf(RecordCursor.NoNextReason.SCAN_LIMIT_REACHED, RecordCursor.NoNextReason.SOURCE_EXHAUSTED))); - if (!recordCursor.getNoNextReason().isSourceExhausted()) { - assertNotNull(recordCursor.getContinuation()); - } - } - } else { - assertNull(recordCursor.getContinuation()); - continuation = null; - } - } while (continuation != null); - - commit(context); - } + public static Stream splitKeyEquivalenceCases() { + return Stream.of( + Arguments.of(Tuple.from(1066L), SplitHelper.START_SPLIT_RECORD), + Arguments.of(Tuple.from(1066L), SplitHelper.START_SPLIT_RECORD + 1), + Arguments.of(Tuple.from(1066L), SplitHelper.START_SPLIT_RECORD + 2), + Arguments.of(Tuple.from(1066L), SplitHelper.UNSPLIT_RECORD), + Arguments.of(Tuple.from(1066L), SplitHelper.RECORD_VERSION), + Arguments.of(Tuple.from(1066L, "extra"), SplitHelper.START_SPLIT_RECORD), + Arguments.of(Tuple.from(), SplitHelper.START_SPLIT_RECORD), + Arguments.of(Tuple.from(), SplitHelper.UNSPLIT_RECORD) + ); } /** - * When two saveWithSplit calls use the same incomplete versionstamp (same localVersion, same key) within - * one transaction, we may get a failure or data corruption. The localVersionCache (map by key) may contain - * previous - * values from an identical key (same versionstamp/localversion/PK) but different splits and may not collide - * directly - * with the previous values. This test shows the case where there is a collision since the split numbers are the - * same. + * Verify that the refactored split call site in {@link SplitHelper} — which now passes + * {@code subspace} + {@code key.add(index)} to {@code packSplitKey} instead of the old + * {@code subspace.subspace(key)} + {@code Tuple.from(index)} — produces identical byte keys + * for {@link DefaultSplitKeyValueHelper}. */ - @Test - void saveWithSplitVersionInKeyOverwriteInTransaction() { - final Tuple key = Tuple.from(1066L); - final int localVersion; - try (FDBRecordContext context = openContext()) { - localVersion = context.claimLocalVersion(); - // First write: VERY_LONG_STRING requires multiple splits - final VersioningSplitKeyValueHelper splitKeyHelper = new VersioningSplitKeyValueHelper(Versionstamp.incomplete(localVersion)); - SplitHelper.saveWithSplit(context, subspace, key, VERY_LONG_STRING, null, - true, false, - splitKeyHelper, - false, null, null); - - // Second write: LONG_STRING — same localVersion, same key, shorter value (fewer splits) - final RecordCoreInternalException ex = assertThrows(RecordCoreInternalException.class, () -> SplitHelper.saveWithSplit(context, subspace, key, LONG_STRING, null, - true, false, - splitKeyHelper, - false, null, null)); - assertTrue(ex.getMessage().contains("Key with version overwritten")); - } + @MethodSource("splitKeyEquivalenceCases") + @ParameterizedTest(name = "defaultHelperSplitKeyEquivalence[key={0}, index={1}]") + void defaultHelperSplitKeyEquivalence(Tuple key, long index) { + Subspace subspace = new Subspace(Tuple.from("test")); + + // Old call site in SplitHelper: subspace.subspace(key).pack(index) + byte[] oldKey = subspace.subspace(key).pack(index); + // New call site: packSplitKey(subspace, key.add(index)) = subspace.pack(key.add(index)) + byte[] newKey = DefaultSplitKeyValueHelper.INSTANCE.packSplitKey(subspace, key.add(index)); + + assertArrayEquals(oldKey, newKey); } } From cf54a2d6726050f0101422c3ebc7d219c5395392 Mon Sep 17 00:00:00 2001 From: ohad Date: Sat, 28 Feb 2026 22:26:24 -0500 Subject: [PATCH 16/29] Convert more tests --- .../SplitHelperMultipleTransactionsTest.java | 227 ++++++++++-------- .../foundationdb/SplitHelperTest.java | 1 + 2 files changed, 126 insertions(+), 102 deletions(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java index 736c07cf75..578a7f3dd0 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java @@ -39,6 +39,7 @@ import com.apple.foundationdb.tuple.Tuple; import com.apple.foundationdb.tuple.Versionstamp; import com.apple.test.BooleanSource; +import com.apple.test.ParameterizedTestUtils; import com.apple.test.Tags; import com.google.common.collect.Lists; import org.junit.jupiter.api.Assumptions; @@ -47,7 +48,9 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.MethodSource; +import org.junit.jupiter.params.shadow.com.univocity.parsers.common.ArgumentUtils; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -221,14 +224,6 @@ public static Stream testConfigs() { return SplitHelperTestConfig.allValidConfigs().map(Arguments::of); } - @Nonnull - public static Stream limitsAndReverseArgs() { - List limits = Arrays.asList(1, 2, 7, Integer.MAX_VALUE); - return limits.stream() - .flatMap(returnLimit -> limits.stream() - .flatMap(readLimit -> Stream.of(Arguments.of(returnLimit, readLimit, false), Arguments.of(returnLimit, readLimit, true)))); - } - @Override protected RecordLayerPropertyStorage.Builder addDefaultProps(final RecordLayerPropertyStorage.Builder props) { return testConfig.setProps(super.addDefaultProps(props)); @@ -294,7 +289,7 @@ private SplitHelper.SizeInfo saveOnly(@Nonnull FDBRecordContext context, @Nonnul return sizeInfo; } - private Tuple toVerifyKey(Tuple key, byte[] versionStamp, int localVersion, boolean versionInKey) { + private Tuple toCompleteKey(Tuple key, byte[] versionStamp, int localVersion, boolean versionInKey) { if (versionInKey) { return Tuple.from(Versionstamp.complete(versionStamp, localVersion)).addAll(key); } else { @@ -510,9 +505,9 @@ public void saveWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig tes globalVersionstamp = context.getVersionStamp(); } - final Tuple verifyKey1 = toVerifyKey(key1, globalVersionstamp, localVersion1, testConfig.useVersionInKey); - final Tuple verifyKey2 = toVerifyKey(key2, globalVersionstamp, localVersion2, testConfig.useVersionInKey); - final Tuple verifyKey3 = toVerifyKey(key3, globalVersionstamp, localVersion3, testConfig.useVersionInKey); + final Tuple verifyKey1 = toCompleteKey(key1, globalVersionstamp, localVersion1, testConfig.useVersionInKey); + final Tuple verifyKey2 = toCompleteKey(key2, globalVersionstamp, localVersion2, testConfig.useVersionInKey); + final Tuple verifyKey3 = toCompleteKey(key3, globalVersionstamp, localVersion3, testConfig.useVersionInKey); try (FDBRecordContext context = openContext()) { verifySuccessfullySaved(context, verifyKey1, SHORT_STRING, null, testConfig); if (testConfig.splitLongRecords) { @@ -537,9 +532,9 @@ public void saveWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig tes commit(context); globalVersionstamp = context.getVersionStamp(); } - Tuple verifyKey4 = toVerifyKey(key1, globalVersionstamp, localVersion4, testConfig.useVersionInKey); - Tuple verifyKey5 = toVerifyKey(key3, globalVersionstamp, localVersion5, testConfig.useVersionInKey); - Tuple verifyKey6 = toVerifyKey(key2, globalVersionstamp, localVersion6, testConfig.useVersionInKey); + Tuple verifyKey4 = toCompleteKey(key1, globalVersionstamp, localVersion4, testConfig.useVersionInKey); + Tuple verifyKey5 = toCompleteKey(key3, globalVersionstamp, localVersion5, testConfig.useVersionInKey); + Tuple verifyKey6 = toCompleteKey(key2, globalVersionstamp, localVersion6, testConfig.useVersionInKey); try (FDBRecordContext context = openContext()) { if (testConfig.splitLongRecords) { verifySuccessfullySaved(context, verifyKey4, VERY_LONG_STRING, null, testConfig); @@ -706,10 +701,10 @@ public void deleteWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig t globalVersionStamp = context.getVersionStamp(); } - final Tuple deleteKey1 = toVerifyKey(key1, globalVersionStamp, localVersion1, testConfig.useVersionInKey); - final Tuple deleteKey2 = toVerifyKey(key2, globalVersionStamp, localVersion2, testConfig.useVersionInKey); - final Tuple deleteKey3 = toVerifyKey(key3, globalVersionStamp, localVersion3, testConfig.useVersionInKey); - final Tuple deleteKey4 = toVerifyKey(key4, globalVersionStamp, localVersion4, testConfig.useVersionInKey); + final Tuple deleteKey1 = toCompleteKey(key1, globalVersionStamp, localVersion1, testConfig.useVersionInKey); + final Tuple deleteKey2 = toCompleteKey(key2, globalVersionStamp, localVersion2, testConfig.useVersionInKey); + final Tuple deleteKey3 = toCompleteKey(key3, globalVersionStamp, localVersion3, testConfig.useVersionInKey); + final Tuple deleteKey4 = toCompleteKey(key4, globalVersionStamp, localVersion4, testConfig.useVersionInKey); // tx2: delete records try (FDBRecordContext context = openContext()) { @@ -733,12 +728,13 @@ public void deleteWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig t private interface LoadRecordFunction { FDBRawRecord load(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @Nullable FDBStoredSizes sizes, @Nullable byte[] expectedContents, @Nullable FDBRecordVersion version); + // TODO: Remoe this implementation default FDBRawRecord load(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @Nullable FDBStoredSizes sizes, @Nullable byte[] expectedContents) { return load(context, key, sizes, expectedContents, null); } } - private void loadSingleRecordsMultipleVersions(SplitHelperTestConfig testConfig, @Nonnull LoadRecordFunction loadRecordFunction) { + private void loadSingleRecordsMultipleTransactions(SplitHelperTestConfig testConfig, @Nonnull LoadRecordFunction loadRecordFunction) { final Tuple key1 = Tuple.from(1042L); final Tuple key2 = Tuple.from(1066L); final Tuple key3 = Tuple.from(1087L); @@ -822,32 +818,32 @@ private void loadSingleRecordsMultipleVersions(SplitHelperTestConfig testConfig, try (FDBRecordContext context = openContext()) { // One split record but then delete the last split point (no way to distinguish this from just inserting one fewer split) if (testConfig.splitLongRecords) { - final Tuple deleteKey5 = toVerifyKey(key5.add(SplitHelper.START_SPLIT_RECORD + MEDIUM_COPIES), keyGlobalVersion, localVersion5, testConfig.useVersionInKey); + final Tuple deleteKey5 = toCompleteKey(key5.add(SplitHelper.START_SPLIT_RECORD + MEDIUM_COPIES), keyGlobalVersion, localVersion5, testConfig.useVersionInKey); context.ensureActive().clear(subspace.pack(deleteKey5)); // One split record then delete the first split point - final Tuple deleteKey6 = toVerifyKey(key6.add(SplitHelper.START_SPLIT_RECORD), keyGlobalVersion, localVersion6, testConfig.useVersionInKey); + final Tuple deleteKey6 = toCompleteKey(key6.add(SplitHelper.START_SPLIT_RECORD), keyGlobalVersion, localVersion6, testConfig.useVersionInKey); context.ensureActive().clear(subspace.pack(deleteKey6)); // One split record then delete the middle split point - final Tuple deleteKey7 = toVerifyKey(key7.add(SplitHelper.START_SPLIT_RECORD + 2), keyGlobalVersion, localVersion7, testConfig.useVersionInKey); + final Tuple deleteKey7 = toCompleteKey(key7.add(SplitHelper.START_SPLIT_RECORD + 2), keyGlobalVersion, localVersion7, testConfig.useVersionInKey); context.ensureActive().clear(subspace.pack(deleteKey7)); if (!testConfig.useVersionInKey) { // One split record with version then delete the first split point - final Tuple DeleteKey9 = toVerifyKey(key9.add(SplitHelper.START_SPLIT_RECORD), keyGlobalVersion, localVersion9, testConfig.useVersionInKey); + final Tuple DeleteKey9 = toCompleteKey(key9.add(SplitHelper.START_SPLIT_RECORD), keyGlobalVersion, localVersion9, testConfig.useVersionInKey); context.ensureActive().clear(subspace.pack(DeleteKey9)); } } commit(context); } - final Tuple completeKey1 = toVerifyKey(key1, keyGlobalVersion, 0, testConfig.useVersionInKey); - final Tuple completeKey2 = toVerifyKey(key2, keyGlobalVersion, localVersion2, testConfig.useVersionInKey); - final Tuple completeKey3 = toVerifyKey(key3, keyGlobalVersion, localVersion3, testConfig.useVersionInKey); - final Tuple completeKey4 = toVerifyKey(key4, keyGlobalVersion, localVersion4, testConfig.useVersionInKey); - final Tuple completeKey5 = toVerifyKey(key5, keyGlobalVersion, localVersion5, testConfig.useVersionInKey); - final Tuple completeKey6 = toVerifyKey(key6, keyGlobalVersion, localVersion6, testConfig.useVersionInKey); - final Tuple completeKey7 = toVerifyKey(key7, keyGlobalVersion, localVersion7, testConfig.useVersionInKey); - final Tuple completeKey8 = toVerifyKey(key8, keyGlobalVersion, localVersion8, testConfig.useVersionInKey); - final Tuple completeKey9 = toVerifyKey(key9, keyGlobalVersion, localVersion9, testConfig.useVersionInKey); + final Tuple completeKey1 = toCompleteKey(key1, keyGlobalVersion, 0, testConfig.useVersionInKey); + final Tuple completeKey2 = toCompleteKey(key2, keyGlobalVersion, localVersion2, testConfig.useVersionInKey); + final Tuple completeKey3 = toCompleteKey(key3, keyGlobalVersion, localVersion3, testConfig.useVersionInKey); + final Tuple completeKey4 = toCompleteKey(key4, keyGlobalVersion, localVersion4, testConfig.useVersionInKey); + final Tuple completeKey5 = toCompleteKey(key5, keyGlobalVersion, localVersion5, testConfig.useVersionInKey); + final Tuple completeKey6 = toCompleteKey(key6, keyGlobalVersion, localVersion6, testConfig.useVersionInKey); + final Tuple completeKey7 = toCompleteKey(key7, keyGlobalVersion, localVersion7, testConfig.useVersionInKey); + final Tuple completeKey8 = toCompleteKey(key8, keyGlobalVersion, localVersion8, testConfig.useVersionInKey); + final Tuple completeKey9 = toCompleteKey(key9, keyGlobalVersion, localVersion9, testConfig.useVersionInKey); // transaction 3 - verify try (FDBRecordContext context = openContext()) { @@ -962,38 +958,42 @@ private FDBRawRecord loadWithSplit(@Nonnull FDBRecordContext context, @Nonnull T return loadWithSplit(context, key, testConfig, expectedSizes, expectedContents, null); } - - - @Test - void loadWithSplitIsolated() { - loadWithSplitMultipleVersions(new SplitHelperTestConfig(false, false, false, false, false, false)); - } - - - @MethodSource("testConfigs") - @ParameterizedTest(name = "loadWithSplitMultipleVersions[{0}]") - public void loadWithSplitMultipleVersions(SplitHelperTestConfig testConfig) { + @ParameterizedTest(name = "loadWithSplitMultipleTransactions[{0}]") + public void loadWithSplitMultipleTransactions(SplitHelperTestConfig testConfig) { + Assumptions.assumeFalse(testConfig.isDryRun); this.testConfig = testConfig; - loadSingleRecordsMultipleVersions(testConfig, - (context, key, expectedSizes, expectedContents, version) -> loadWithSplit(context, key, testConfig, expectedSizes, expectedContents, version)); + loadSingleRecordsMultipleTransactions(testConfig, + (context, key, expectedSizes, expectedContents, version) -> + loadWithSplit(context, key, testConfig, expectedSizes, expectedContents, version)); if (testConfig.splitLongRecords) { + final Tuple key = Tuple.from(1307L); + final int localVersion; + final byte[] globalVersion; + // Unsplit record followed by some unsplit stuff + // This particular error is caught by the single key unsplitter but not the mulit-key one try (FDBRecordContext context = openContext()) { - // Unsplit record followed by some unsplit stuff - // This particular error is caught by the single key unsplitter but not the mulit-key one - writeDummyRecord(context, Tuple.from(1307L), 1, false); - writeDummyRecord(context, Tuple.from(1307L), MEDIUM_COPIES, false); - RecordCoreException err = assertThrows(RecordCoreException.class, - () -> loadWithSplit(context, Tuple.from(1307L), testConfig, null, null)); - assertThat(err.getMessage(), containsString("Unsplit value followed by split")); + localVersion = context.claimLocalVersion(); + writeDummyRecord(context, key, null, 1, false, testConfig.useVersionInKey, localVersion); + writeDummyRecord(context, key, null, MEDIUM_COPIES, false, testConfig.useVersionInKey, localVersion); commit(context); + globalVersion = context.getVersionStamp(); + } + Tuple completeKey = toCompleteKey(key, globalVersion, localVersion, testConfig.useVersionInKey); + try (FDBRecordContext context = openContext()) { + RecordCoreException err = assertThrows(RecordCoreException.class, + () -> loadWithSplit(context, completeKey, testConfig, null, null)); + assertThat(err.getMessage(), containsString("Unsplit value followed by split")); } } } - private FDBRawRecord scanSingleRecord(@Nonnull FDBRecordContext context, boolean reverse, @Nonnull Tuple key, @Nullable FDBStoredSizes expectedSizes, @Nullable byte[] expectedContents, @Nullable FDBRecordVersion version) { + private FDBRawRecord scanSingleRecord(@Nonnull FDBRecordContext context, boolean reverse, + @Nonnull Tuple key, @Nullable FDBStoredSizes expectedSizes, + @Nullable byte[] expectedContents, @Nullable FDBRecordVersion version, + boolean useVersionInKey) { final ScanProperties scanProperties = reverse ? ScanProperties.REVERSE_SCAN : ScanProperties.FORWARD_SCAN; KeyValueCursor kvCursor = KeyValueCursor.Builder.withSubspace(subspace) .setContext(context) @@ -1018,21 +1018,23 @@ private FDBRawRecord scanSingleRecord(@Nonnull FDBRecordContext context, boolean assertEquals(key, rawRecord.getPrimaryKey()); assertArrayEquals(expectedContents, rawRecord.getRawRecord()); assertEquals(expectedSizes.getKeyCount(), rawRecord.getKeyCount()); - assertEquals(expectedSizes.getKeySize(), rawRecord.getKeySize()); assertEquals(expectedSizes.getValueSize(), rawRecord.getValueSize()); boolean isSplit = rawRecord.getKeyCount() - (rawRecord.isVersionedInline() ? 1 : 0) != 1; assertEquals(rawRecord.getKeyCount() - (rawRecord.isVersionedInline() ? 1 : 0) != 1, expectedSizes.isSplit()); assertEquals(version != null, expectedSizes.isVersionedInline()); - + if (!useVersionInKey) { + assertEquals(expectedSizes.getKeySize(), rawRecord.getKeySize()); + } return rawRecord; } } - @ParameterizedTest(name = "scan[reverse = {0}]") - @BooleanSource - public void scanSingleRecords(boolean reverse) { - loadSingleRecordsMultipleVersions(new SplitHelperTestConfig(true, false, FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES.getDefaultValue(), false, false, false), - (context, key, expectedSizes, expectedContents, version) -> scanSingleRecord(context, reverse, key, expectedSizes, expectedContents, version)); + @ParameterizedTest(name = "scan[reverse = {0}, useVersionInKey = {1}]") + @CsvSource({"false, false", "false, true", "true, false", "true, true"}) + public void scanSingleRecordsMultipleTransactions(boolean reverse, boolean useVersionInKey) { + loadSingleRecordsMultipleTransactions(new SplitHelperTestConfig(true, false, FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES.getDefaultValue(), false, false, useVersionInKey), + (context, key, expectedSizes, expectedContents, version) -> + scanSingleRecord(context, reverse, key, expectedSizes, expectedContents, version, useVersionInKey)); } private List writeDummyRecords() { @@ -1062,54 +1064,40 @@ private List writeDummyRecords() { } private List writeDummyRecordsMultipleTransactions(boolean useVersionInKey) { - final byte[] globalVersion = "_cushions_".getBytes(StandardCharsets.US_ASCII); + final byte[] valueVersion = "_cushions_".getBytes(StandardCharsets.US_ASCII); // Generate primary keys using a generalization of the Fibonacci formula: https://oeis.org/A247698 long currKey = 2308L; long nextKey = 4261L; - final Tuple[] keys = new Tuple[50]; - final byte[][] rawBytesArr = new byte[50][]; - final FDBRecordVersion[] versions = new FDBRecordVersion[50]; - final int[] localVersions = new int[50]; + final int numRecords = 50; + final Tuple[] keys = new Tuple[numRecords]; + final byte[][] rawBytesArr = new byte[numRecords][]; + final FDBRecordVersion[] versions = new FDBRecordVersion[numRecords]; + final int[] localVersions = new int[numRecords]; + final FDBStoredSizes[] sizes = new FDBStoredSizes[numRecords]; - final byte[] globalVs; + final byte[] globalVersionStamp; try (FDBRecordContext context = openContext()) { - for (int i = 0; i < 50; i++) { + for (int i = 0; i < numRecords; i++) { keys[i] = Tuple.from(currKey); rawBytesArr[i] = (i % 4 < 2) ? SHORT_STRING : MEDIUM_STRING; - versions[i] = (!useVersionInKey && i % 2 == 0) ? FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()) : null; + versions[i] = (!useVersionInKey && (i % 2 == 0)) ? FDBRecordVersion.complete(valueVersion, context.claimLocalVersion()) : null; localVersions[i] = useVersionInKey ? context.claimLocalVersion() : 0; - writeDummyRecord(context, keys[i], versions[i], (i % 4 < 2) ? 1 : MEDIUM_COPIES, false, useVersionInKey, localVersions[i]); + sizes[i] = writeDummyRecord(context, keys[i], versions[i], (i % 4 < 2) ? 1 : MEDIUM_COPIES, false, useVersionInKey, localVersions[i]); long temp = currKey + nextKey; currKey = nextKey; nextKey = temp; } commit(context); - globalVs = context.getVersionStamp(); + globalVersionStamp = context.getVersionStamp(); } - // Scan back in a second transaction to build expected FDBRawRecord list with accurate committed sizes + // Produce the raw records final List rawRecords = new ArrayList<>(); - try (FDBRecordContext context = openContext()) { - KeyValueCursor kvCursor = KeyValueCursor.Builder.withSubspace(subspace) - .setContext(context) - .setRange(TupleRange.ALL) - .setScanProperties(ScanProperties.FORWARD_SCAN) - .build(); - List scannedRecords = new SplitHelper.KeyValueUnsplitter(context, subspace, kvCursor, false, null, ScanProperties.FORWARD_SCAN) - .asList().join(); - for (int i = 0; i < 50; i++) { - Tuple expectedKey = useVersionInKey - ? Tuple.from(Versionstamp.complete(globalVs, localVersions[i])).addAll(keys[i]) - : keys[i]; - FDBRawRecord scanned = scannedRecords.stream() - .filter(r -> r.getPrimaryKey().equals(expectedKey)) - .findFirst() - .orElseThrow(() -> new AssertionError("Missing record for key " + expectedKey)); - rawRecords.add(new FDBRawRecord(expectedKey, rawBytesArr[i], versions[i], scanned)); - } - commit(context); + for (int i = 0; i < numRecords; i++) { + Tuple expectedKey = toCompleteKey(keys[i], globalVersionStamp, localVersions[i], useVersionInKey); + rawRecords.add(new FDBRawRecord(expectedKey, rawBytesArr[i], versions[i], sizes[i])); } return rawRecords; @@ -1142,11 +1130,11 @@ public void scanMultipleRecords(boolean reverse) { } } - @ParameterizedTest(name = "scanMultipleRecordsMultipleTransactions[reverse = {0}]") - @BooleanSource - void scanMultipleRecordsMultipleTransactions(boolean reverse) { + @ParameterizedTest(name = "scanMultipleRecordsMultipleTransactions[reverse = {0}, useVersionInKey = {1}]") + @CsvSource({"false, false", "false, true", "true, false", "true, true"}) + void scanMultipleRecordsMultipleTransactions(boolean reverse, boolean useVersionInKey) { final ScanProperties scanProperties = reverse ? ScanProperties.REVERSE_SCAN : ScanProperties.FORWARD_SCAN; - List rawRecords = writeDummyRecordsMultipleTransactions(true); + List rawRecords = writeDummyRecordsMultipleTransactions(useVersionInKey); try (FDBRecordContext context = openContext()) { KeyValueCursor kvCursor = KeyValueCursor.Builder.withSubspace(subspace) @@ -1161,14 +1149,27 @@ void scanMultipleRecordsMultipleTransactions(boolean reverse) { } assertEquals(rawRecords.size(), readRecords.size()); for (int i = 0; i < rawRecords.size(); i++) { - assertEquals(rawRecords.get(i), readRecords.get(i)); + if (useVersionInKey) { + assertEqualsNoKeySize(rawRecords.get(i), readRecords.get(i)); + } else { + assertEquals(rawRecords.get(i), readRecords.get(i)); + } } - assertEquals(rawRecords, readRecords); commit(context); } } + private void assertEqualsNoKeySize(final FDBRawRecord expected, final FDBRawRecord actual) { + assertEquals(expected.getPrimaryKey(), actual.getPrimaryKey()); + assertArrayEquals(expected.getRawRecord(), actual.getRawRecord()); + assertEquals(expected.getVersion(), actual.getVersion()); + assertEquals(expected.getKeyCount(), actual.getKeyCount()); + assertEquals(expected.getValueSize(), actual.getValueSize()); + assertEquals(expected.isSplit(), actual.isSplit()); + assertEquals(expected.isVersionedInline(), actual.isVersionedInline()); + } + @MethodSource("limitsAndReverseArgs") @ParameterizedTest(name = "scanContinuations [returnLimit = {0}, readLimit = {1}, reverse = {2}]") public void scanContinuations(final int returnLimit, final int readLimit, final boolean reverse) { @@ -1245,10 +1246,28 @@ public void scanContinuations(final int returnLimit, final int readLimit, final } } - @MethodSource("limitsAndReverseArgs") - @ParameterizedTest(name = "scanContinuationsMultipleTransactions [returnLimit = {0}, readLimit = {1}, reverse = {2}]") - void scanContinuationsMultipleTransactions(final int returnLimit, final int readLimit, final boolean reverse) { - List rawRecords = writeDummyRecordsMultipleTransactions(true); + @Nonnull + public static Stream limitsAndReverseArgs() { + List limits = Arrays.asList(1, 2, 7, Integer.MAX_VALUE); + return limits.stream() + .flatMap(returnLimit -> limits.stream() + .flatMap(readLimit -> Stream.of(Arguments.of(returnLimit, readLimit, false), Arguments.of(returnLimit, readLimit, true)))); + } + + @Nonnull + public static Stream limitsReverseVersionArgs() { + List limits = List.of(1, 2, 7, Integer.MAX_VALUE); + return ParameterizedTestUtils.cartesianProduct( + limits.stream(), + limits.stream(), + ParameterizedTestUtils.booleans("reverse"), + ParameterizedTestUtils.booleans("useVersionInKey")); + } + + @MethodSource("limitsReverseVersionArgs") + @ParameterizedTest + void scanContinuationsMultipleTransactions(final int returnLimit, final int readLimit, final boolean reverse, boolean useVersionInKey) { + List rawRecords = writeDummyRecordsMultipleTransactions(useVersionInKey); if (reverse) { rawRecords = Lists.reverse(rawRecords); } @@ -1283,7 +1302,11 @@ void scanContinuationsMultipleTransactions(final int returnLimit, final int read assertNotNull(nextRecord); assertThat(expectedRecordIterator.hasNext(), is(true)); FDBRawRecord expectedRecord = expectedRecordIterator.next(); - assertEquals(expectedRecord, nextRecord); + if (useVersionInKey) { + assertEqualsNoKeySize(expectedRecord, nextRecord); + } else { + assertEquals(expectedRecord, nextRecord); + } rowsScanned += nextRecord.getKeyCount(); retrieved += 1; diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java index afc68eda4a..238ded1cdc 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java @@ -436,6 +436,7 @@ public void saveWithSplitAndIncompleteVersions(SplitHelperTestConfig testConfig) this.testConfig = testConfig; final byte[] versionstamp; try (FDBRecordContext context = openContext()) { + // With incomplete version saveWithSplit(context, Tuple.from(962L), SHORT_STRING, FDBRecordVersion.incomplete(context.claimLocalVersion()), testConfig); saveWithSplit(context, Tuple.from(967L), LONG_STRING, FDBRecordVersion.incomplete(context.claimLocalVersion()), testConfig); saveWithSplit(context, Tuple.from(996L), VERY_LONG_STRING, FDBRecordVersion.incomplete(context.claimLocalVersion()), testConfig); From 05c958eb199be7255fd33a41594ad8e080d38278 Mon Sep 17 00:00:00 2001 From: ohad Date: Sun, 1 Mar 2026 00:13:46 -0500 Subject: [PATCH 17/29] Cleanup --- .../SplitHelperMultipleTransactionsTest.java | 444 +++++------------- 1 file changed, 125 insertions(+), 319 deletions(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java index 578a7f3dd0..e4d4461193 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java @@ -1,9 +1,9 @@ /* - * SplitHelperTest.java + * SplitHelperMultipleTransactionsTest.java * * This source file is part of the FoundationDB open source project * - * Copyright 2015-2018 Apple Inc. and the FoundationDB project authors + * Copyright 2015-2026 Apple Inc. and the FoundationDB project authors * * Licensed under the Apache License, Version 2.0 (the "License"); * you may not use this file except in compliance with the License. @@ -38,7 +38,6 @@ import com.apple.foundationdb.tuple.ByteArrayUtil; import com.apple.foundationdb.tuple.Tuple; import com.apple.foundationdb.tuple.Versionstamp; -import com.apple.test.BooleanSource; import com.apple.test.ParameterizedTestUtils; import com.apple.test.Tags; import com.google.common.collect.Lists; @@ -50,14 +49,12 @@ import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.MethodSource; -import org.junit.jupiter.params.shadow.com.univocity.parsers.common.ArgumentUtils; import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.nio.ByteBuffer; import java.nio.charset.StandardCharsets; import java.util.ArrayList; -import java.util.Arrays; import java.util.Collections; import java.util.Iterator; import java.util.List; @@ -81,8 +78,19 @@ import static org.junit.jupiter.api.Assertions.assertTrue; /** - * Tests for checking the validity of the "split helper" utility class that handles breaking - * records across key-value pairs and putting them back together again. + * A test that uses the same test cases as in {@link SplitHelperTest}, with the added behavior of multiple transactions. + * Tests here ar named similarly to the ones in {@link SplitHelperTest} with the added "MultipleTransactions" + * suffix, so that they can be traced back to their origin. + * For tests that require the "versionInKey" to be TRUE, this is the only way to get them to run, as the + * verification of the key content can be done after a commit. + * The pattern used in this test normally splits each test flow in two: + *
    + *
  • Save record and record global and local version
  • + *
  • Verify the saved content using the recorded version
  • + *
+ * In addition to testing
useKeyInVersion
this way, other configurations were also tested + * to extend coverage over the single-transaction tests. + * - */ @Tag(Tags.RequiresFDB) public class SplitHelperMultipleTransactionsTest extends FDBRecordStoreTestBase { @@ -250,6 +258,10 @@ private SplitHelper.SizeInfo saveUnsuccessfully(@Nonnull F return sizeInfo; } + /** + * This represents the first part of {@link SplitHelperTest#saveSuccessfully(FDBRecordContext, Tuple, byte[], FDBRecordVersion, SplitHelperTest.SplitHelperTestConfig, FDBStoredSizes)}. + * The other part is located in {@link #verifySuccessfullySaved(FDBRecordContext, Tuple, byte[], FDBRecordVersion, SplitHelperTestConfig)}. + */ private SplitHelper.SizeInfo saveOnly(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, @Nullable FDBRecordVersion version, @Nonnull SplitHelperTestConfig testConfig, @@ -289,17 +301,21 @@ private SplitHelper.SizeInfo saveOnly(@Nonnull FDBRecordContext context, @Nonnul return sizeInfo; } - private Tuple toCompleteKey(Tuple key, byte[] versionStamp, int localVersion, boolean versionInKey) { - if (versionInKey) { - return Tuple.from(Versionstamp.complete(versionStamp, localVersion)).addAll(key); - } else { - return key; - } - } - + /** + * This represents the second part of {@link SplitHelperTest#saveSuccessfully(FDBRecordContext, Tuple, byte[], FDBRecordVersion, SplitHelperTest.SplitHelperTestConfig, FDBStoredSizes)}. + */ private void verifySuccessfullySaved(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, @Nullable FDBRecordVersion version, @Nonnull SplitHelperTestConfig testConfig) { + // do nothing if not saveOnly was not actually invoked + if (testConfig.omitUnsplitSuffix && version != null) { + // cannot include version + return; + } else if (!testConfig.splitLongRecords && serialized.length > SplitHelper.SPLIT_RECORD_SIZE) { + // Record is too long + return; + } + // Similar to the calculation in saveOnly int dataKeyCount = (serialized.length - 1) / SplitHelper.SPLIT_RECORD_SIZE + 1; boolean isSplit = dataKeyCount > 1; @@ -368,116 +384,17 @@ private void verifySuccessfullySaved(@Nonnull FDBRecordContext context, @Nonnull } } -// private SplitHelper.SizeInfo saveSuccessfully(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, -// @Nullable FDBRecordVersion version, -// @Nonnull SplitHelperTestConfig testConfig, -// @Nullable FDBStoredSizes previousSizeInfo) { -// // localVersion=0: single-transaction callers always use useVersionInKey=false, so the value doesn't matter -// SplitHelper.SizeInfo sizeInfo = saveOnly(context, key, serialized, version, testConfig, previousSizeInfo, 0); -// verifySuccessfullySaved(context, key, serialized, version, testConfig); -// return sizeInfo; -// } - -// private SplitHelper.SizeInfo dryRunSetSizeInfo(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, -// @Nullable FDBRecordVersion version, -// @Nonnull SplitHelperTestConfig testConfig, -// @Nullable FDBStoredSizes previousSizeInfo) { -// final SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); -// SplitHelper.dryRunSaveWithSplitOnlySetSizeInfo(subspace, key, serialized, version, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, sizeInfo); -// -// int dataKeyCount = (serialized.length - 1) / SplitHelper.SPLIT_RECORD_SIZE + 1; -// boolean isSplit = dataKeyCount > 1; -// int keyCount = dataKeyCount; -// if (version != null) { -// keyCount += 1; -// } -// int keySize = (subspace.pack().length + key.pack().length) * keyCount; -// assertEquals(isSplit, sizeInfo.isSplit()); -// assertEquals(keyCount, sizeInfo.getKeyCount()); -// if (testConfig.hasSplitPoints()) { -// // Add in the the counters the split points. -// if (!isSplit) { -// keySize += 1; // As 0 requires 1 byte when Tuple packed -// } else { -// keySize += dataKeyCount * 2; // As each split point is two bytes when tuple packed -// } -// } -// if (version != null) { -// keySize += 2; -// } -// int valueSize = serialized.length + (version != null ? 1 + FDBRecordVersion.VERSION_LENGTH : 0); -// assertEquals(keySize, sizeInfo.getKeySize()); -// assertEquals(valueSize, sizeInfo.getValueSize()); -// assertEquals(version != null, sizeInfo.isVersionedInline()); -// // assert nothing is written -// int count = KeyValueCursor.Builder.withSubspace(subspace.subspace(key)) -// .setContext(context) -// .setScanProperties(ScanProperties.FORWARD_SCAN) -// .build() -// .getCount() -// .join(); -// assertEquals(0, previousSizeInfo == null ? count : previousSizeInfo.getKeyCount() + count); -// sizeInfo.reset(); -// return sizeInfo; -// } - -// private SplitHelper.SizeInfo saveWithSplit(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, -// @Nullable FDBRecordVersion version, -// @Nonnull SplitHelperTestConfig testConfig, -// @Nullable FDBStoredSizes previousSizeInfo) { -// if (testConfig.omitUnsplitSuffix && version != null) { -// return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, -// RecordCoreArgumentException.class, "Cannot include version"); -// } else if (!testConfig.splitLongRecords && serialized.length > SplitHelper.SPLIT_RECORD_SIZE) { -// return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, -// RecordCoreException.class, "Record is too long"); -// } else if (testConfig.isDryRun) { -// return dryRunSetSizeInfo(context, key, serialized, version, testConfig, previousSizeInfo); -// } else { -// return saveSuccessfully(context, key, serialized, version, testConfig, previousSizeInfo); -// } -// } - -// private SplitHelper.SizeInfo saveWithSplit(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, -// @Nullable FDBRecordVersion version, @Nonnull SplitHelperTestConfig testConfig) { -// return saveWithSplit(context, key, serialized, version, testConfig, null); -// } -// -// private SplitHelper.SizeInfo saveWithSplit(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, @Nonnull SplitHelperTestConfig testConfig, -// @Nullable FDBStoredSizes previousSizeInfo) { -// return saveWithSplit(context, key, serialized, null, testConfig, previousSizeInfo); -// } -// -// private SplitHelper.SizeInfo saveWithSplit(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, @Nonnull SplitHelperTestConfig testConfig) { -// return saveWithSplit(context, key, serialized, null, testConfig); -// } - -// @MethodSource("testConfigs") -// @ParameterizedTest(name = "saveWithSplit[{0}]") -// public void saveWithSplit(@Nonnull SplitHelperTestConfig testConfig) { -// this.testConfig = testConfig; -// // For versionInKey we need multiple transactions - one to save the key and one to read the timestamp from it -// Assumptions.assumeFalse(testConfig.useVersionInKey); -// try (FDBRecordContext context = openContext()) { -// // No version -// FDBStoredSizes sizes1 = saveWithSplit(context, Tuple.from(1066L), SHORT_STRING, testConfig); -// FDBStoredSizes sizes2 = saveWithSplit(context, Tuple.from(1415L), LONG_STRING, testConfig); -// FDBStoredSizes sizes3 = saveWithSplit(context, Tuple.from(1776L), VERY_LONG_STRING, testConfig); -// -// // Save over some things using the previous split points -// if (testConfig.splitLongRecords) { -// saveWithSplit(context, Tuple.from(1066L), VERY_LONG_STRING, testConfig, sizes1); -// saveWithSplit(context, Tuple.from(1776), LONG_STRING, testConfig, sizes3); -// } -// saveWithSplit(context, Tuple.from(1415L), SHORT_STRING, testConfig, sizes2); -// -// commit(context); -// } -// } + private Tuple toCompleteKey(Tuple key, byte[] versionStamp, int localVersion, boolean versionInKey) { + if (versionInKey) { + return Tuple.from(Versionstamp.complete(versionStamp, localVersion)).addAll(key); + } else { + return key; + } + } @MethodSource("testConfigs") @ParameterizedTest(name = "saveWithSplitMultipleTransactions[{0}]") - public void saveWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig testConfig) { + void saveWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig testConfig) { // dry run does not support transactions Assumptions.assumeFalse(testConfig.isDryRun); this.testConfig = testConfig; @@ -510,10 +427,8 @@ public void saveWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig tes final Tuple verifyKey3 = toCompleteKey(key3, globalVersionstamp, localVersion3, testConfig.useVersionInKey); try (FDBRecordContext context = openContext()) { verifySuccessfullySaved(context, verifyKey1, SHORT_STRING, null, testConfig); - if (testConfig.splitLongRecords) { - verifySuccessfullySaved(context, verifyKey2, LONG_STRING, null, testConfig); - verifySuccessfullySaved(context, verifyKey3, VERY_LONG_STRING, null, testConfig); - } + verifySuccessfullySaved(context, verifyKey2, LONG_STRING, null, testConfig); + verifySuccessfullySaved(context, verifyKey3, VERY_LONG_STRING, null, testConfig); } int localVersion4 = 0; @@ -560,35 +475,85 @@ private SplitHelper.SizeInfo saveWithSplitForMultipleTransactions(@Nonnull FDBRe return saveOnly(context, key, serialized, version, testConfig, previousSizeInfo, localVersion); } } - // TODO -// @MethodSource("testConfigs") -// @ParameterizedTest(name = "saveWithSplitAndCompleteVersion[{0}]") -// public void saveWithSplitAndCompleteVersions(SplitHelperTestConfig testConfig) { -// this.testConfig = testConfig; -// Assumptions.assumeFalse(testConfig.useVersionInKey); -// try (FDBRecordContext context = openContext()) { -// byte[] globalVersion = "karlgrosse".getBytes(StandardCharsets.US_ASCII); -// saveWithSplit(context, Tuple.from(800L), SHORT_STRING, FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()), testConfig); -// saveWithSplit(context, Tuple.from(813L), LONG_STRING, FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()), testConfig); -// saveWithSplit(context, Tuple.from(823L), VERY_LONG_STRING, FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()), testConfig); -// -// // Save over the records *without* using the previous size info -// saveWithSplit(context, Tuple.from(800L), SHORT_STRING, testConfig); -// saveWithSplit(context, Tuple.from(813L), LONG_STRING, testConfig); -// saveWithSplit(context, Tuple.from(823L), VERY_LONG_STRING, testConfig); -// -// FDBStoredSizes sizes4 = saveWithSplit(context, Tuple.from(800L), SHORT_STRING, FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()), testConfig); -// FDBStoredSizes sizes5 = saveWithSplit(context, Tuple.from(813L), LONG_STRING, FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()), testConfig); -// FDBStoredSizes sizes6 = saveWithSplit(context, Tuple.from(823L), VERY_LONG_STRING, FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()), testConfig); -// -// // Save over the records *with* using the previous size info -// saveWithSplit(context, Tuple.from(800L), SHORT_STRING, testConfig, sizes4); -// saveWithSplit(context, Tuple.from(813L), LONG_STRING, testConfig, sizes5); -// saveWithSplit(context, Tuple.from(823L), VERY_LONG_STRING, testConfig, sizes6); -// -// commit(context); -// } -// } + + @MethodSource("testConfigs") + @ParameterizedTest(name = "saveWithSplitAndCompleteVersion[{0}]") + public void saveWithSplitAndCompleteVersionsMultipleTransactions(SplitHelperTestConfig testConfig) { + this.testConfig = testConfig; + Assumptions.assumeFalse(testConfig.useVersionInKey); + byte[] globalVersion = "karlgrosse".getBytes(StandardCharsets.US_ASCII); + final Tuple key1 = Tuple.from(800L); + final Tuple key2 = Tuple.from(813L); + final Tuple key3 = Tuple.from(823L); + FDBRecordVersion version1; + FDBRecordVersion version2; + FDBRecordVersion version3; + try (FDBRecordContext context = openContext()) { + version1 = FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()); + saveWithSplitForMultipleTransactions(context, key1, SHORT_STRING, version1, testConfig, null, 0); + version2 = FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()); + saveWithSplitForMultipleTransactions(context, key2, LONG_STRING, version2, testConfig, null, 0); + version3 = FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()); + saveWithSplitForMultipleTransactions(context, key3, VERY_LONG_STRING, version3, testConfig, null, 0); + commit(context); + } + try (FDBRecordContext context = openContext()) { + verifySuccessfullySaved(context, key1, SHORT_STRING, version1, testConfig); + verifySuccessfullySaved(context, key2, LONG_STRING, version2, testConfig); + verifySuccessfullySaved(context, key3, VERY_LONG_STRING, version3, testConfig); + } + + // Save over the records *without* using the previous size info + try (FDBRecordContext context = openContext()) { + saveWithSplitForMultipleTransactions(context, key1, SHORT_STRING, null, testConfig, null, 0); + saveWithSplitForMultipleTransactions(context, key2, LONG_STRING, null, testConfig, null, 0); + saveWithSplitForMultipleTransactions(context, key3, VERY_LONG_STRING, null, testConfig, null, 0); + commit(context); + } + try (FDBRecordContext context = openContext()) { + verifySuccessfullySaved(context, key1, SHORT_STRING, null, testConfig); + verifySuccessfullySaved(context, key2, LONG_STRING, null, testConfig); + verifySuccessfullySaved(context, key3, VERY_LONG_STRING, null, testConfig); + commit(context); + } + + FDBStoredSizes sizes4; + FDBStoredSizes sizes5; + FDBStoredSizes sizes6; + FDBRecordVersion version4; + FDBRecordVersion version5; + FDBRecordVersion version6; + try (FDBRecordContext context = openContext()) { + version4 = FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()); + sizes4 = saveWithSplitForMultipleTransactions(context, key1, SHORT_STRING, version4, testConfig, null, 0); + version5 = FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()); + sizes5 = saveWithSplitForMultipleTransactions(context, key2, LONG_STRING, version5, testConfig, null, 0); + version6 = FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()); + sizes6 = saveWithSplitForMultipleTransactions(context, key3, VERY_LONG_STRING, version6, testConfig, null, 0); + commit(context); + } + try (FDBRecordContext context = openContext()) { + verifySuccessfullySaved(context, key1, SHORT_STRING, version4, testConfig); + verifySuccessfullySaved(context, key2, LONG_STRING, version5, testConfig); + verifySuccessfullySaved(context, key3, VERY_LONG_STRING, version6, testConfig); + commit(context); + } + + // Save over the records *with* using the previous size info + try (FDBRecordContext context = openContext()) { + saveWithSplitForMultipleTransactions(context, key1, SHORT_STRING, null, testConfig, sizes4, 0); + saveWithSplitForMultipleTransactions(context, key2, LONG_STRING, null, testConfig, sizes5, 0); + saveWithSplitForMultipleTransactions(context, key3, VERY_LONG_STRING, null, testConfig, sizes6, 0); + commit(context); + } + + try (FDBRecordContext context = openContext()) { + verifySuccessfullySaved(context, key1, SHORT_STRING, null, testConfig); + verifySuccessfullySaved(context, key2, LONG_STRING, null, testConfig); + verifySuccessfullySaved(context, key3, VERY_LONG_STRING, null, testConfig); + commit(context); + } + } @Nonnull private FDBStoredSizes writeDummyRecord(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @@ -626,16 +591,6 @@ private FDBStoredSizes writeDummyRecord(@Nonnull FDBRecordContext context, @Nonn return sizeInfo; } - @Nonnull - private FDBStoredSizes writeDummyRecord(@Nonnull FDBRecordContext context, @Nonnull Tuple key, int splits, boolean omitUnsplitSuffix) { - return writeDummyRecord(context, key, null, splits, omitUnsplitSuffix, false, 0); - } - - @Nonnull - private FDBStoredSizes writeDummyRecord(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @Nonnull FDBRecordVersion version, int splits) { - return writeDummyRecord(context, key, version, splits, false, false, 0); - } - private void writeDummyKV(@Nonnull FDBRecordContext context, @Nonnull Tuple keyTuple, byte[] valueBytes, @Nullable SplitHelper.SizeInfo sizeInfo, boolean useVersionInKey, int localVersion) { byte[] keyBytes; @@ -717,12 +672,6 @@ public void deleteWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig t commit(context); } } -// -// static Stream deleteWithSplitAndVersion() { -// return Stream.of(false, true).flatMap(splitLongRecords -> -// Stream.of(false, true).map(unrollSingleRecordDeletes -> -// Arguments.of(splitLongRecords, unrollSingleRecordDeletes))); -// } @FunctionalInterface private interface LoadRecordFunction { @@ -770,13 +719,13 @@ private void loadSingleRecordsMultipleTransactions(SplitHelperTestConfig testCon // One record with version localVersion3 = context.claimLocalVersion(); version3 = FDBRecordVersion.complete(valueGlobalVersion, localVersion3); - sizes3 = writeDummyRecord(context, key3, version3, 1); + sizes3 = writeDummyRecord(context, key3, version3, 1, false, false, 0); assertThat(sizes3.isVersionedInline(), is(true)); // One version but missing record localVersion4 = context.claimLocalVersion(); version4 = FDBRecordVersion.complete(valueGlobalVersion, localVersion4); - writeDummyRecord(context, key4, version4, 1); + writeDummyRecord(context, key4, version4, 1, false, false, 0); context.ensureActive().clear(subspace.pack(key4.add(SplitHelper.UNSPLIT_RECORD))); } @@ -787,7 +736,7 @@ private void loadSingleRecordsMultipleTransactions(SplitHelperTestConfig testCon assertEquals(MEDIUM_COPIES, sizes5.getKeyCount()); // One split record but then delete the last split point (no way to distinguish this from just inserting one fewer split) - writeDummyRecord(context, key5, MEDIUM_COPIES + 1, false); + writeDummyRecord(context, key5, null, MEDIUM_COPIES + 1, false, testConfig.useVersionInKey, localVersion5); // One split record then delete the first split point localVersion6 = context.claimLocalVersion(); @@ -806,7 +755,7 @@ private void loadSingleRecordsMultipleTransactions(SplitHelperTestConfig testCon if (!testConfig.useVersionInKey) { localVersion9 = context.claimLocalVersion(); version9 = FDBRecordVersion.complete(valueGlobalVersion, context.claimLocalVersion()); - writeDummyRecord(context, key9, version9, MEDIUM_COPIES); + writeDummyRecord(context, key9, version9, MEDIUM_COPIES, false, false, 0); } } @@ -952,12 +901,6 @@ private FDBRawRecord loadWithSplit(@Nonnull FDBRecordContext context, @Nonnull T return rawRecord; } - @Nullable - private FDBRawRecord loadWithSplit(@Nonnull FDBRecordContext context, @Nonnull Tuple key, SplitHelperTestConfig testConfig, - @Nullable FDBStoredSizes expectedSizes, @Nullable byte[] expectedContents) { - return loadWithSplit(context, key, testConfig, expectedSizes, expectedContents, null); - } - @MethodSource("testConfigs") @ParameterizedTest(name = "loadWithSplitMultipleTransactions[{0}]") public void loadWithSplitMultipleTransactions(SplitHelperTestConfig testConfig) { @@ -984,7 +927,7 @@ public void loadWithSplitMultipleTransactions(SplitHelperTestConfig testConfig) Tuple completeKey = toCompleteKey(key, globalVersion, localVersion, testConfig.useVersionInKey); try (FDBRecordContext context = openContext()) { RecordCoreException err = assertThrows(RecordCoreException.class, - () -> loadWithSplit(context, completeKey, testConfig, null, null)); + () -> loadWithSplit(context, completeKey, testConfig, null, null, null)); assertThat(err.getMessage(), containsString("Unsplit value followed by split")); } } @@ -1037,32 +980,6 @@ public void scanSingleRecordsMultipleTransactions(boolean reverse, boolean useVe scanSingleRecord(context, reverse, key, expectedSizes, expectedContents, version, useVersionInKey)); } - private List writeDummyRecords() { - final byte[] globalVersion = "_cushions_".getBytes(StandardCharsets.US_ASCII); - final List rawRecords = new ArrayList<>(); - // Generate primary keys using a generalization of the Fibonacci formula: https://oeis.org/A247698 - long currKey = 2308L; - long nextKey = 4261L; - - try (FDBRecordContext context = openContext()) { - for (int i = 0; i < 50; i++) { - FDBRecordVersion version = (i % 2 == 0) ? FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()) : null; - byte[] rawBytes = (i % 4 < 2) ? SHORT_STRING : MEDIUM_STRING; - Tuple key = Tuple.from(currKey); - FDBStoredSizes sizes = writeDummyRecord(context, key, version, (i % 4 < 2) ? 1 : MEDIUM_COPIES, false, false, 0); - rawRecords.add(new FDBRawRecord(key, rawBytes, version, sizes)); - - long temp = currKey + nextKey; - currKey = nextKey; - nextKey = temp; - } - - commit(context); - } - - return rawRecords; - } - private List writeDummyRecordsMultipleTransactions(boolean useVersionInKey) { final byte[] valueVersion = "_cushions_".getBytes(StandardCharsets.US_ASCII); // Generate primary keys using a generalization of the Fibonacci formula: https://oeis.org/A247698 @@ -1103,33 +1020,6 @@ private List writeDummyRecordsMultipleTransactions(boolean useVers return rawRecords; } - @ParameterizedTest(name = "scanMultipleRecords[reverse = {0}]") - @BooleanSource - public void scanMultipleRecords(boolean reverse) { - final ScanProperties scanProperties = reverse ? ScanProperties.REVERSE_SCAN : ScanProperties.FORWARD_SCAN; - List rawRecords = writeDummyRecords(); - - try (FDBRecordContext context = openContext()) { - KeyValueCursor kvCursor = KeyValueCursor.Builder.withSubspace(subspace) - .setContext(context) - .setRange(TupleRange.ALL) - .setScanProperties(scanProperties) - .build(); - List readRecords = new SplitHelper.KeyValueUnsplitter(context, subspace, kvCursor, false, null, scanProperties) - .asList().join(); - if (reverse) { - readRecords = Lists.reverse(readRecords); - } - assertEquals(rawRecords.size(), readRecords.size()); - for (int i = 0; i < rawRecords.size(); i++) { - assertEquals(rawRecords.get(i), readRecords.get(i)); - } - assertEquals(rawRecords, readRecords); - - commit(context); - } - } - @ParameterizedTest(name = "scanMultipleRecordsMultipleTransactions[reverse = {0}, useVersionInKey = {1}]") @CsvSource({"false, false", "false, true", "true, false", "true, true"}) void scanMultipleRecordsMultipleTransactions(boolean reverse, boolean useVersionInKey) { @@ -1170,90 +1060,6 @@ private void assertEqualsNoKeySize(final FDBRawRecord expected, final FDBRawReco assertEquals(expected.isVersionedInline(), actual.isVersionedInline()); } - @MethodSource("limitsAndReverseArgs") - @ParameterizedTest(name = "scanContinuations [returnLimit = {0}, readLimit = {1}, reverse = {2}]") - public void scanContinuations(final int returnLimit, final int readLimit, final boolean reverse) { - List rawRecords = writeDummyRecords(); - if (reverse) { - rawRecords = Lists.reverse(rawRecords); - } - final Iterator expectedRecordIterator = rawRecords.iterator(); - - try (FDBRecordContext context = openContext()) { - byte[] continuation = null; - - do { - final ExecuteProperties executeProperties = ExecuteProperties.newBuilder() - .setReturnedRowLimit(returnLimit) - .setScannedRecordsLimit(readLimit) - .build(); - ScanProperties scanProperties = new ScanProperties(executeProperties, reverse); - RecordCursor kvCursor = KeyValueCursor.Builder.withSubspace(subspace) - .setContext(context) - .setRange(TupleRange.ALL) - .setScanProperties(scanProperties.with(ExecuteProperties::clearRowAndTimeLimits).with(ExecuteProperties::clearState)) - .setContinuation(continuation) - .build(); - RecordCursorIterator recordCursor = new SplitHelper.KeyValueUnsplitter(context, subspace, kvCursor, false, null, scanProperties.with(ExecuteProperties::clearReturnedRowLimit)) - .limitRowsTo(returnLimit) - .asIterator(); - - int retrieved = 0; - int rowsScanned = 0; - while (recordCursor.hasNext()) { - assertThat(retrieved, lessThan(returnLimit)); - assertThat(rowsScanned, lessThanOrEqualTo(readLimit)); - - FDBRawRecord nextRecord = recordCursor.next(); - assertNotNull(nextRecord); - assertThat(expectedRecordIterator.hasNext(), is(true)); - FDBRawRecord expectedRecord = expectedRecordIterator.next(); - assertEquals(expectedRecord, nextRecord); - - rowsScanned += nextRecord.getKeyCount(); - retrieved += 1; - } - - if (retrieved > 0) { - continuation = recordCursor.getContinuation(); - if (retrieved >= returnLimit) { - assertEquals(RecordCursor.NoNextReason.RETURN_LIMIT_REACHED, recordCursor.getNoNextReason()); - assertNotNull(continuation); - } else if (rowsScanned > readLimit) { - assertEquals(RecordCursor.NoNextReason.SCAN_LIMIT_REACHED, recordCursor.getNoNextReason()); - assertNotNull(continuation); - } else if (rowsScanned < readLimit) { - assertEquals(RecordCursor.NoNextReason.SOURCE_EXHAUSTED, recordCursor.getNoNextReason()); - } else { - // If we read exactly as many records as is allowed by the read record limit, then - // this probably means that we hit SCAN_LIMIT_REACHED, but it's also possible to - // hit SOURCE_EXHAUSTED if we hit the record read limit at exactly the same time - // as we needed to do another speculative read to determine if a split record - // continues or not. - assertEquals(readLimit, rowsScanned); - assertThat(recordCursor.getNoNextReason(), is(oneOf(RecordCursor.NoNextReason.SCAN_LIMIT_REACHED, RecordCursor.NoNextReason.SOURCE_EXHAUSTED))); - if (!recordCursor.getNoNextReason().isSourceExhausted()) { - assertNotNull(recordCursor.getContinuation()); - } - } - } else { - assertNull(recordCursor.getContinuation()); - continuation = null; - } - } while (continuation != null); - - commit(context); - } - } - - @Nonnull - public static Stream limitsAndReverseArgs() { - List limits = Arrays.asList(1, 2, 7, Integer.MAX_VALUE); - return limits.stream() - .flatMap(returnLimit -> limits.stream() - .flatMap(readLimit -> Stream.of(Arguments.of(returnLimit, readLimit, false), Arguments.of(returnLimit, readLimit, true)))); - } - @Nonnull public static Stream limitsReverseVersionArgs() { List limits = List.of(1, 2, 7, Integer.MAX_VALUE); From fb0085c138af0dd4368b5a84c514175fd052ae6e Mon Sep 17 00:00:00 2001 From: ohad Date: Sun, 1 Mar 2026 23:34:19 -0500 Subject: [PATCH 18/29] More test changes, extract TestConfig and reuse from both test classes. --- .../provider/foundationdb/SplitHelper.java | 8 +- .../SplitHelperMultipleTransactionsTest.java | 255 +++++++----------- .../foundationdb/SplitHelperTest.java | 90 +------ .../foundationdb/SplitHelperTestConfig.java | 123 +++++++++ 4 files changed, 240 insertions(+), 236 deletions(-) create mode 100644 fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTestConfig.java diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java index dac464709a..7457562ad7 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java @@ -134,6 +134,10 @@ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnu .addLogInfo(LogMessageKeys.SUBSPACE, ByteArrayUtil2.loggable(subspace.pack())) .addLogInfo(LogMessageKeys.VERSION, version); } + if ((version != null) && !splitKeyHelper.supportsVersionInValue()) { + // Cannot write version in the k/v value since it is not supported by this helper (e.g. version is needed in the key) + throw new RecordCoreArgumentException("Split version is not supported for this helper"); + } if (serialized.length > SplitHelper.SPLIT_RECORD_SIZE) { if (!splitLongRecords) { throw new RecordCoreException("Record is too long to be stored in a single value; consider split_long_records") @@ -207,10 +211,6 @@ private static void writeVersion(@Nonnull final FDBRecordContext context, @Nonnu } return; } - if (!splitKeyHelper.supportsVersionInValue()) { - // Cannot write version in the k/v value since it is not supported by this helper (e.g. version is needed in the key) - throw new RecordCoreInternalException("Split version is not supported for this helper"); - } final byte[] keyBytes = splitKeyHelper.packSplitKey(subspace, key.add(RECORD_VERSION)); final byte[] valueBytes = packVersion(version); if (version.isComplete()) { diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java index e4d4461193..6d602f7714 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java @@ -41,7 +41,6 @@ import com.apple.test.ParameterizedTestUtils; import com.apple.test.Tags; import com.google.common.collect.Lists; -import org.junit.jupiter.api.Assumptions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; @@ -58,7 +57,6 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.Objects; import java.util.concurrent.ExecutionException; import java.util.stream.LongStream; import java.util.stream.Stream; @@ -88,8 +86,8 @@ *
  • Save record and record global and local version
  • *
  • Verify the saved content using the recorded version
  • * - * In addition to testing
    useKeyInVersion
    this way, other configurations were also tested - * to extend coverage over the single-transaction tests. + * In addition to testing
    useVersionInKey
    this way, other configurations are also tested + * to extend coverage beyond the single-transaction tests. * - */ @Tag(Tags.RequiresFDB) @@ -139,97 +137,8 @@ public void setSubspace() { } } - static class SplitHelperTestConfig { - private final boolean splitLongRecords; - private final boolean omitUnsplitSuffix; - private final boolean unrollRecordDeletes; - private final boolean loadViaGets; - private final boolean isDryRun; - private final boolean useVersionInKey; - - public SplitHelperTestConfig(boolean splitLongRecords, boolean omitUnsplitSuffix, boolean unrollRecordDeletes, - boolean loadViaGets, boolean isDryRun, boolean useVersionInKey) { - this.splitLongRecords = splitLongRecords; - this.omitUnsplitSuffix = omitUnsplitSuffix; - this.unrollRecordDeletes = unrollRecordDeletes; - this.loadViaGets = loadViaGets; - this.isDryRun = isDryRun; - this.useVersionInKey = useVersionInKey; - } - - public SplitKeyValueHelper keyHelper(int localVersion) { - if (useVersionInKey) { - return new VersioningSplitKeyValueHelper(Versionstamp.incomplete(localVersion)); - } else { - return DefaultSplitKeyValueHelper.INSTANCE; - } - } - - @Nonnull - public RecordLayerPropertyStorage.Builder setProps(@Nonnull RecordLayerPropertyStorage.Builder props) { - return props - .addProp(FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES, unrollRecordDeletes) - .addProp(FDBRecordStoreProperties.LOAD_RECORDS_VIA_GETS, loadViaGets); - } - - public boolean hasSplitPoints() { - return splitLongRecords || !omitUnsplitSuffix; - } - - @Override - public String toString() { - return "SplitHelperTestConfig{" + - "splitLongRecords=" + splitLongRecords + - ", omitUnsplitSuffix=" + omitUnsplitSuffix + - ", unrollRecordDeletes=" + unrollRecordDeletes + - ", loadViaGets=" + loadViaGets + - ", isDryRun=" + isDryRun + - ", useVersionInKey=" + useVersionInKey + - '}'; - } - - @Override - public boolean equals(final Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - final SplitHelperTestConfig that = (SplitHelperTestConfig)o; - return splitLongRecords == that.splitLongRecords && omitUnsplitSuffix == that.omitUnsplitSuffix && - unrollRecordDeletes == that.unrollRecordDeletes && loadViaGets == that.loadViaGets && - isDryRun == that.isDryRun && useVersionInKey == that.useVersionInKey; - } - - @Override - public int hashCode() { - return Objects.hash(splitLongRecords, omitUnsplitSuffix, unrollRecordDeletes, loadViaGets, useVersionInKey); - } - - public static Stream allValidConfigs() { - // Note that splitLongRecords="true" && omitUnsplitSuffix="true" is not valid - // Note that useVersionInKey="true" && isDryRun="true" is not valid (versionstamp never completes without commit) - return Stream.of(false, true).flatMap(useVersionInKey -> - Stream.of(false, true).flatMap(splitLongRecords -> - (splitLongRecords ? Stream.of(false) : Stream.of(false, true)).flatMap(omitUnsplitSuffix -> - Stream.of(false, true).flatMap(unrollRecordDeletes -> - Stream.of(false, true).flatMap(loadViaGets -> - Stream.of(false, true) - .filter(isDryRun -> !useVersionInKey || !isDryRun) - .map(isDryRun -> - new SplitHelperTestConfig(splitLongRecords, omitUnsplitSuffix, unrollRecordDeletes, loadViaGets, isDryRun, useVersionInKey))))))); - } - - public static SplitHelperTestConfig getDefault() { - return new SplitHelperTestConfig(true, false, - FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES.getDefaultValue(), - FDBRecordStoreProperties.LOAD_RECORDS_VIA_GETS.getDefaultValue(), false, false); - } - } - - public static Stream testConfigs() { - return SplitHelperTestConfig.allValidConfigs().map(Arguments::of); + public static Stream testConfigsNoDryRun() { + return SplitHelperTestConfig.getConfigsNoDryRun().map(Arguments::of); } @Override @@ -259,7 +168,7 @@ private SplitHelper.SizeInfo saveUnsuccessfully(@Nonnull F } /** - * This represents the first part of {@link SplitHelperTest#saveSuccessfully(FDBRecordContext, Tuple, byte[], FDBRecordVersion, SplitHelperTest.SplitHelperTestConfig, FDBStoredSizes)}. + * This represents the first part of {@link SplitHelperTest#saveSuccessfully}. * The other part is located in {@link #verifySuccessfullySaved(FDBRecordContext, Tuple, byte[], FDBRecordVersion, SplitHelperTestConfig)}. */ private SplitHelper.SizeInfo saveOnly(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, @@ -302,7 +211,7 @@ private SplitHelper.SizeInfo saveOnly(@Nonnull FDBRecordContext context, @Nonnul } /** - * This represents the second part of {@link SplitHelperTest#saveSuccessfully(FDBRecordContext, Tuple, byte[], FDBRecordVersion, SplitHelperTest.SplitHelperTestConfig, FDBStoredSizes)}. + * This represents the second part of {@link SplitHelperTest#saveSuccessfully}. */ private void verifySuccessfullySaved(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, @Nullable FDBRecordVersion version, @@ -311,6 +220,9 @@ private void verifySuccessfullySaved(@Nonnull FDBRecordContext context, @Nonnull if (testConfig.omitUnsplitSuffix && version != null) { // cannot include version return; + } else if (testConfig.useVersionInKey && (version != null)) { + // cannot use version in both key and value + return; } else if (!testConfig.splitLongRecords && serialized.length > SplitHelper.SPLIT_RECORD_SIZE) { // Record is too long return; @@ -392,11 +304,10 @@ private Tuple toCompleteKey(Tuple key, byte[] versionStamp, int localVersion, bo } } - @MethodSource("testConfigs") + @MethodSource("testConfigsNoDryRun") @ParameterizedTest(name = "saveWithSplitMultipleTransactions[{0}]") void saveWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig testConfig) { // dry run does not support transactions - Assumptions.assumeFalse(testConfig.isDryRun); this.testConfig = testConfig; final Tuple key1 = Tuple.from(1066L); @@ -409,6 +320,7 @@ void saveWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig testConfig final SplitHelper.SizeInfo sizes2; final SplitHelper.SizeInfo sizes3; byte[] globalVersionstamp; + // Transaction #1: save some values try (FDBRecordContext context = openContext()) { // save with no version and no previousSizeInfo localVersion1 = context.claimLocalVersion(); @@ -425,6 +337,7 @@ void saveWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig testConfig final Tuple verifyKey1 = toCompleteKey(key1, globalVersionstamp, localVersion1, testConfig.useVersionInKey); final Tuple verifyKey2 = toCompleteKey(key2, globalVersionstamp, localVersion2, testConfig.useVersionInKey); final Tuple verifyKey3 = toCompleteKey(key3, globalVersionstamp, localVersion3, testConfig.useVersionInKey); + // Transaction #2: verify saved values try (FDBRecordContext context = openContext()) { verifySuccessfullySaved(context, verifyKey1, SHORT_STRING, null, testConfig); verifySuccessfullySaved(context, verifyKey2, LONG_STRING, null, testConfig); @@ -434,6 +347,7 @@ void saveWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig testConfig int localVersion4 = 0; int localVersion5 = 0; int localVersion6 = 0; + // Transaction #3: modify saved values try (FDBRecordContext context = openContext()) { // Save over some things using the previous split points if (testConfig.splitLongRecords) { @@ -450,6 +364,7 @@ void saveWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig testConfig Tuple verifyKey4 = toCompleteKey(key1, globalVersionstamp, localVersion4, testConfig.useVersionInKey); Tuple verifyKey5 = toCompleteKey(key3, globalVersionstamp, localVersion5, testConfig.useVersionInKey); Tuple verifyKey6 = toCompleteKey(key2, globalVersionstamp, localVersion6, testConfig.useVersionInKey); + // Transaction #4: verify changes try (FDBRecordContext context = openContext()) { if (testConfig.splitLongRecords) { verifySuccessfullySaved(context, verifyKey4, VERY_LONG_STRING, null, testConfig); @@ -468,6 +383,9 @@ private SplitHelper.SizeInfo saveWithSplitForMultipleTransactions(@Nonnull FDBRe if (testConfig.omitUnsplitSuffix && version != null) { return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, RecordCoreArgumentException.class, "Cannot include version"); + } else if (testConfig.useVersionInKey && version != null) { + return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, + RecordCoreException.class, "Split version is not supported for this helper"); } else if (!testConfig.splitLongRecords && serialized.length > SplitHelper.SPLIT_RECORD_SIZE) { return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, RecordCoreException.class, "Record is too long"); @@ -476,12 +394,18 @@ private SplitHelper.SizeInfo saveWithSplitForMultipleTransactions(@Nonnull FDBRe } } - @MethodSource("testConfigs") + @MethodSource("testConfigsNoDryRun") @ParameterizedTest(name = "saveWithSplitAndCompleteVersion[{0}]") public void saveWithSplitAndCompleteVersionsMultipleTransactions(SplitHelperTestConfig testConfig) { this.testConfig = testConfig; - Assumptions.assumeFalse(testConfig.useVersionInKey); - byte[] globalVersion = "karlgrosse".getBytes(StandardCharsets.US_ASCII); + byte[] globalValueVersion = "karlgrosse".getBytes(StandardCharsets.US_ASCII); + byte[] globalKeyVersion; + int localVersion1; + int localVersion2; + int localVersion3; + Tuple completeKey1; + Tuple completeKey2; + Tuple completeKey3; final Tuple key1 = Tuple.from(800L); final Tuple key2 = Tuple.from(813L); final Tuple key3 = Tuple.from(823L); @@ -489,31 +413,49 @@ public void saveWithSplitAndCompleteVersionsMultipleTransactions(SplitHelperTest FDBRecordVersion version2; FDBRecordVersion version3; try (FDBRecordContext context = openContext()) { - version1 = FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()); - saveWithSplitForMultipleTransactions(context, key1, SHORT_STRING, version1, testConfig, null, 0); - version2 = FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()); - saveWithSplitForMultipleTransactions(context, key2, LONG_STRING, version2, testConfig, null, 0); - version3 = FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()); - saveWithSplitForMultipleTransactions(context, key3, VERY_LONG_STRING, version3, testConfig, null, 0); + version1 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); + localVersion1 = context.claimLocalVersion(); + saveWithSplitForMultipleTransactions(context, key1, SHORT_STRING, version1, testConfig, null, localVersion1); + localVersion2 = context.claimLocalVersion(); + version2 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); + saveWithSplitForMultipleTransactions(context, key2, LONG_STRING, version2, testConfig, null, localVersion2); + localVersion3 = context.claimLocalVersion(); + version3 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); + saveWithSplitForMultipleTransactions(context, key3, VERY_LONG_STRING, version3, testConfig, null, localVersion3); commit(context); + globalKeyVersion = context.getVersionStamp(); } - try (FDBRecordContext context = openContext()) { - verifySuccessfullySaved(context, key1, SHORT_STRING, version1, testConfig); - verifySuccessfullySaved(context, key2, LONG_STRING, version2, testConfig); - verifySuccessfullySaved(context, key3, VERY_LONG_STRING, version3, testConfig); + // in some cases nothing gets written (all saves are "saveUnsuccessfully) so the transaction is read-only + // and there is no global version + if (globalKeyVersion != null) { + completeKey1 = toCompleteKey(key1, globalKeyVersion, localVersion1, testConfig.useVersionInKey); + completeKey2 = toCompleteKey(key2, globalKeyVersion, localVersion2, testConfig.useVersionInKey); + completeKey3 = toCompleteKey(key3, globalKeyVersion, localVersion3, testConfig.useVersionInKey); + try (FDBRecordContext context = openContext()) { + verifySuccessfullySaved(context, completeKey1, SHORT_STRING, version1, testConfig); + verifySuccessfullySaved(context, completeKey2, LONG_STRING, version2, testConfig); + verifySuccessfullySaved(context, completeKey3, VERY_LONG_STRING, version3, testConfig); + } } // Save over the records *without* using the previous size info try (FDBRecordContext context = openContext()) { - saveWithSplitForMultipleTransactions(context, key1, SHORT_STRING, null, testConfig, null, 0); - saveWithSplitForMultipleTransactions(context, key2, LONG_STRING, null, testConfig, null, 0); - saveWithSplitForMultipleTransactions(context, key3, VERY_LONG_STRING, null, testConfig, null, 0); + localVersion1 = context.claimLocalVersion(); + saveWithSplitForMultipleTransactions(context, key1, SHORT_STRING, null, testConfig, null, localVersion1); + localVersion2 = context.claimLocalVersion(); + saveWithSplitForMultipleTransactions(context, key2, LONG_STRING, null, testConfig, null, localVersion2); + localVersion3 = context.claimLocalVersion(); + saveWithSplitForMultipleTransactions(context, key3, VERY_LONG_STRING, null, testConfig, null, localVersion3); commit(context); + globalKeyVersion = context.getVersionStamp(); } + completeKey1 = toCompleteKey(key1, globalKeyVersion, localVersion1, testConfig.useVersionInKey); + completeKey2 = toCompleteKey(key2, globalKeyVersion, localVersion2, testConfig.useVersionInKey); + completeKey3 = toCompleteKey(key3, globalKeyVersion, localVersion3, testConfig.useVersionInKey); try (FDBRecordContext context = openContext()) { - verifySuccessfullySaved(context, key1, SHORT_STRING, null, testConfig); - verifySuccessfullySaved(context, key2, LONG_STRING, null, testConfig); - verifySuccessfullySaved(context, key3, VERY_LONG_STRING, null, testConfig); + verifySuccessfullySaved(context, completeKey1, SHORT_STRING, null, testConfig); + verifySuccessfullySaved(context, completeKey2, LONG_STRING, null, testConfig); + verifySuccessfullySaved(context, completeKey3, VERY_LONG_STRING, null, testConfig); commit(context); } @@ -524,33 +466,50 @@ public void saveWithSplitAndCompleteVersionsMultipleTransactions(SplitHelperTest FDBRecordVersion version5; FDBRecordVersion version6; try (FDBRecordContext context = openContext()) { - version4 = FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()); - sizes4 = saveWithSplitForMultipleTransactions(context, key1, SHORT_STRING, version4, testConfig, null, 0); - version5 = FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()); - sizes5 = saveWithSplitForMultipleTransactions(context, key2, LONG_STRING, version5, testConfig, null, 0); - version6 = FDBRecordVersion.complete(globalVersion, context.claimLocalVersion()); - sizes6 = saveWithSplitForMultipleTransactions(context, key3, VERY_LONG_STRING, version6, testConfig, null, 0); + localVersion1 = context.claimLocalVersion(); + version4 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); + sizes4 = saveWithSplitForMultipleTransactions(context, key1, SHORT_STRING, version4, testConfig, null, localVersion1); + localVersion2 = context.claimLocalVersion(); + version5 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); + sizes5 = saveWithSplitForMultipleTransactions(context, key2, LONG_STRING, version5, testConfig, null, localVersion2); + localVersion3 = context.claimLocalVersion(); + version6 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); + sizes6 = saveWithSplitForMultipleTransactions(context, key3, VERY_LONG_STRING, version6, testConfig, null, localVersion3); commit(context); + globalKeyVersion = context.getVersionStamp(); } - try (FDBRecordContext context = openContext()) { - verifySuccessfullySaved(context, key1, SHORT_STRING, version4, testConfig); - verifySuccessfullySaved(context, key2, LONG_STRING, version5, testConfig); - verifySuccessfullySaved(context, key3, VERY_LONG_STRING, version6, testConfig); - commit(context); + // in some cases nothing gets written (all saves are "saveUnsuccessfully) so the transaction is read-only + // and there is no global version + if (globalKeyVersion != null) { + completeKey1 = toCompleteKey(key1, globalKeyVersion, localVersion1, testConfig.useVersionInKey); + completeKey2 = toCompleteKey(key2, globalKeyVersion, localVersion2, testConfig.useVersionInKey); + completeKey3 = toCompleteKey(key3, globalKeyVersion, localVersion3, testConfig.useVersionInKey); + try (FDBRecordContext context = openContext()) { + verifySuccessfullySaved(context, completeKey1, SHORT_STRING, version4, testConfig); + verifySuccessfullySaved(context, completeKey2, LONG_STRING, version5, testConfig); + verifySuccessfullySaved(context, completeKey3, VERY_LONG_STRING, version6, testConfig); + commit(context); + } } // Save over the records *with* using the previous size info try (FDBRecordContext context = openContext()) { - saveWithSplitForMultipleTransactions(context, key1, SHORT_STRING, null, testConfig, sizes4, 0); - saveWithSplitForMultipleTransactions(context, key2, LONG_STRING, null, testConfig, sizes5, 0); - saveWithSplitForMultipleTransactions(context, key3, VERY_LONG_STRING, null, testConfig, sizes6, 0); + localVersion1 = context.claimLocalVersion(); + saveWithSplitForMultipleTransactions(context, key1, SHORT_STRING, null, testConfig, sizes4, localVersion1); + localVersion2 = context.claimLocalVersion(); + saveWithSplitForMultipleTransactions(context, key2, LONG_STRING, null, testConfig, sizes5, localVersion2); + localVersion3 = context.claimLocalVersion(); + saveWithSplitForMultipleTransactions(context, key3, VERY_LONG_STRING, null, testConfig, sizes6, localVersion3); commit(context); + globalKeyVersion = context.getVersionStamp(); } - + completeKey1 = toCompleteKey(key1, globalKeyVersion, localVersion1, testConfig.useVersionInKey); + completeKey2 = toCompleteKey(key2, globalKeyVersion, localVersion2, testConfig.useVersionInKey); + completeKey3 = toCompleteKey(key3, globalKeyVersion, localVersion3, testConfig.useVersionInKey); try (FDBRecordContext context = openContext()) { - verifySuccessfullySaved(context, key1, SHORT_STRING, null, testConfig); - verifySuccessfullySaved(context, key2, LONG_STRING, null, testConfig); - verifySuccessfullySaved(context, key3, VERY_LONG_STRING, null, testConfig); + verifySuccessfullySaved(context, completeKey1, SHORT_STRING, null, testConfig); + verifySuccessfullySaved(context, completeKey2, LONG_STRING, null, testConfig); + verifySuccessfullySaved(context, completeKey3, VERY_LONG_STRING, null, testConfig); commit(context); } } @@ -621,7 +580,7 @@ private void deleteSplit(@Nonnull FDBRecordContext context, @Nonnull Tuple key, assertEquals(0, count); } - @MethodSource("testConfigs") + @MethodSource("testConfigsNoDryRun") @ParameterizedTest(name = "deleteWithSplitMultipleTransactions[{0}]") public void deleteWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig testConfig) { this.testConfig = testConfig; @@ -676,11 +635,6 @@ public void deleteWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig t @FunctionalInterface private interface LoadRecordFunction { FDBRawRecord load(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @Nullable FDBStoredSizes sizes, @Nullable byte[] expectedContents, @Nullable FDBRecordVersion version); - - // TODO: Remoe this implementation - default FDBRawRecord load(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @Nullable FDBStoredSizes sizes, @Nullable byte[] expectedContents) { - return load(context, key, sizes, expectedContents, null); - } } private void loadSingleRecordsMultipleTransactions(SplitHelperTestConfig testConfig, @Nonnull LoadRecordFunction loadRecordFunction) { @@ -797,9 +751,9 @@ private void loadSingleRecordsMultipleTransactions(SplitHelperTestConfig testCon // transaction 3 - verify try (FDBRecordContext context = openContext()) { // No record - loadRecordFunction.load(context, completeKey1, null, null); + loadRecordFunction.load(context, completeKey1, null, null, null); // One unsplit record - loadRecordFunction.load(context, completeKey2, sizes2, HUMPTY_DUMPTY); + loadRecordFunction.load(context, completeKey2, sizes2, HUMPTY_DUMPTY, null); if ((!testConfig.omitUnsplitSuffix) && (!testConfig.useVersionInKey)) { // One record with version loadRecordFunction.load(context, completeKey3, sizes3, HUMPTY_DUMPTY, version3); @@ -811,21 +765,21 @@ private void loadSingleRecordsMultipleTransactions(SplitHelperTestConfig testCon if (testConfig.splitLongRecords) { // One split record // One split record but then delete the last split point (no way to distinguish this from just inserting one fewer split) - loadRecordFunction.load(context, completeKey5, sizes5, MEDIUM_STRING); + loadRecordFunction.load(context, completeKey5, sizes5, MEDIUM_STRING, null); // One split record then delete the first split point if (testConfig.loadViaGets) { - loadRecordFunction.load(context, completeKey6, null, null); + loadRecordFunction.load(context, completeKey6, null, null, null); } else { assertThrows(SplitHelper.FoundSplitWithoutStartException.class, - () -> loadRecordFunction.load(context, completeKey6, null, null)); + () -> loadRecordFunction.load(context, completeKey6, null, null, null)); } // One split record then delete the middle split point RecordCoreException err7 = assertThrows(RecordCoreException.class, - () -> loadRecordFunction.load(context, completeKey7, null, null)); + () -> loadRecordFunction.load(context, completeKey7, null, null, null)); assertThat(err7.getMessage(), containsString("Split record segments out of order")); // One split record then add an extra key in the middle RecordCoreException err8 = assertThrows(RecordCoreException.class, - () -> loadRecordFunction.load(context, completeKey8, null, null)); + () -> loadRecordFunction.load(context, completeKey8, null, null, null)); assertThat(err8.getMessage(), anyOf( containsString("Expected only a single key extension"), containsString("Split record segments out of order") @@ -901,10 +855,9 @@ private FDBRawRecord loadWithSplit(@Nonnull FDBRecordContext context, @Nonnull T return rawRecord; } - @MethodSource("testConfigs") + @MethodSource("testConfigsNoDryRun") @ParameterizedTest(name = "loadWithSplitMultipleTransactions[{0}]") public void loadWithSplitMultipleTransactions(SplitHelperTestConfig testConfig) { - Assumptions.assumeFalse(testConfig.isDryRun); this.testConfig = testConfig; loadSingleRecordsMultipleTransactions(testConfig, (context, key, expectedSizes, expectedContents, version) -> diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java index 238ded1cdc..f76d13d5f1 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java @@ -56,7 +56,6 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; -import java.util.Objects; import java.util.concurrent.ExecutionException; import java.util.stream.LongStream; import java.util.stream.Stream; @@ -125,79 +124,8 @@ public void setSubspace() { } } - static class SplitHelperTestConfig { - private final boolean splitLongRecords; - private final boolean omitUnsplitSuffix; - private final boolean unrollRecordDeletes; - private final boolean loadViaGets; - private final boolean isDryRun; - - public SplitHelperTestConfig(boolean splitLongRecords, boolean omitUnsplitSuffix, boolean unrollRecordDeletes, boolean loadViaGets, boolean isDryRun) { - this.splitLongRecords = splitLongRecords; - this.omitUnsplitSuffix = omitUnsplitSuffix; - this.unrollRecordDeletes = unrollRecordDeletes; - this.loadViaGets = loadViaGets; - this.isDryRun = isDryRun; - } - - @Nonnull - public RecordLayerPropertyStorage.Builder setProps(@Nonnull RecordLayerPropertyStorage.Builder props) { - return props - .addProp(FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES, unrollRecordDeletes) - .addProp(FDBRecordStoreProperties.LOAD_RECORDS_VIA_GETS, loadViaGets); - } - - public boolean hasSplitPoints() { - return splitLongRecords || !omitUnsplitSuffix; - } - - @Override - public String toString() { - return "SplitHelperTestConfig{" + - "splitLongRecords=" + splitLongRecords + - ", omitUnsplitSuffix=" + omitUnsplitSuffix + - ", unrollRecordDeletes=" + unrollRecordDeletes + - ", loadViaGets=" + loadViaGets + - ", isDryRun=" + isDryRun + - '}'; - } - - @Override - public boolean equals(final Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - final SplitHelperTestConfig that = (SplitHelperTestConfig)o; - return splitLongRecords == that.splitLongRecords && omitUnsplitSuffix == that.omitUnsplitSuffix && unrollRecordDeletes == that.unrollRecordDeletes && loadViaGets == that.loadViaGets && isDryRun == that.isDryRun; - } - - @Override - public int hashCode() { - return Objects.hash(splitLongRecords, omitUnsplitSuffix, unrollRecordDeletes, loadViaGets); - } - - public static Stream allValidConfigs() { - // Note that splitLongRecords="true" && omitUnsplitSuffix="true" is not valid - return Stream.of(false, true).flatMap(splitLongRecords -> - (splitLongRecords ? Stream.of(false) : Stream.of(false, true)).flatMap(omitUnsplitSuffix -> - Stream.of(false, true).flatMap(unrollRecordDeletes -> - Stream.of(false, true).flatMap(loadViaGets -> - Stream.of(false, true).map(isDryRun -> - new SplitHelperTestConfig(splitLongRecords, omitUnsplitSuffix, unrollRecordDeletes, loadViaGets, isDryRun)))))); - } - - public static SplitHelperTestConfig getDefault() { - return new SplitHelperTestConfig(true, false, - FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES.getDefaultValue(), - FDBRecordStoreProperties.LOAD_RECORDS_VIA_GETS.getDefaultValue(), false); - } - } - - public static Stream testConfigs() { - return SplitHelperTestConfig.allValidConfigs().map(Arguments::of); + public static Stream testConfigsNoVersionInKey() { + return SplitHelperTestConfig.getConfigsNoVersionInKey().map(Arguments::of); } @Nonnull @@ -409,7 +337,7 @@ private SplitHelper.SizeInfo saveWithSplit(@Nonnull FDBRecordContext context, @N return saveWithSplit(context, key, serialized, null, testConfig); } - @MethodSource("testConfigs") + @MethodSource("testConfigsNoVersionInKey") @ParameterizedTest(name = "saveWithSplit[{0}]") public void saveWithSplit(@Nonnull SplitHelperTestConfig testConfig) { this.testConfig = testConfig; @@ -430,7 +358,7 @@ public void saveWithSplit(@Nonnull SplitHelperTestConfig testConfig) { } } - @MethodSource("testConfigs") + @MethodSource("testConfigsNoVersionInKey") @ParameterizedTest(name = "saveWithSplitAndIncompleteVersions[{0}]") public void saveWithSplitAndIncompleteVersions(SplitHelperTestConfig testConfig) { this.testConfig = testConfig; @@ -473,7 +401,7 @@ public void saveWithSplitAndIncompleteVersions(SplitHelperTestConfig testConfig) } } - @MethodSource("testConfigs") + @MethodSource("testConfigsNoVersionInKey") @ParameterizedTest(name = "saveWithSplitAndCompleteVersion[{0}]") public void saveWithSplitAndCompleteVersions(SplitHelperTestConfig testConfig) { this.testConfig = testConfig; @@ -559,7 +487,7 @@ private void deleteSplit(@Nonnull FDBRecordContext context, @Nonnull Tuple key, assertEquals(0, count); } - @MethodSource("testConfigs") + @MethodSource("testConfigsNoVersionInKey") @ParameterizedTest(name = "deleteWithSplit[{0}]") public void deleteWithSplit(SplitHelperTestConfig testConfig) { this.testConfig = testConfig; @@ -593,7 +521,7 @@ static Stream deleteWithSplitAndVersion() { } @ParameterizedTest(name = "deleteWithSplitAndVersion[{0}]") - @MethodSource("testConfigs") + @MethodSource("testConfigsNoVersionInKey") public void deleteWithSplitAndVersion(SplitHelperTestConfig testConfig) { this.testConfig = testConfig; Assumptions.assumeFalse(testConfig.omitUnsplitSuffix); @@ -765,7 +693,7 @@ private FDBRawRecord loadWithSplit(@Nonnull FDBRecordContext context, @Nonnull T return loadWithSplit(context, key, testConfig, expectedSizes, expectedContents, null); } - @MethodSource("testConfigs") + @MethodSource("testConfigsNoVersionInKey") @ParameterizedTest(name = "loadWithSplit[{0}]") public void loadWithSplit(SplitHelperTestConfig testConfig) { this.testConfig = testConfig; @@ -825,7 +753,7 @@ private FDBRawRecord scanSingleRecord(@Nonnull FDBRecordContext context, boolean @ParameterizedTest(name = "scan[reverse = {0}]") @BooleanSource public void scanSingleRecords(boolean reverse) { - loadSingleRecords(new SplitHelperTestConfig(true, false, FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES.getDefaultValue(), false, false), + loadSingleRecords(new SplitHelperTestConfig(true, false, FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES.getDefaultValue(), false, false, false), (context, key, expectedSizes, expectedContents, version) -> scanSingleRecord(context, reverse, key, expectedSizes, expectedContents, version)); } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTestConfig.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTestConfig.java new file mode 100644 index 0000000000..46eff6d847 --- /dev/null +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTestConfig.java @@ -0,0 +1,123 @@ +/* + * SplitHelperTestConfig.java + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2015-2026 Apple Inc. and the FoundationDB project authors + * + * Licensed 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 com.apple.foundationdb.record.provider.foundationdb; + +import com.apple.foundationdb.record.FDBRecordStoreProperties; +import com.apple.foundationdb.record.provider.foundationdb.properties.RecordLayerPropertyStorage; +import com.apple.foundationdb.tuple.Versionstamp; + +import javax.annotation.Nonnull; +import java.util.Objects; +import java.util.stream.Stream; + +public class SplitHelperTestConfig { + public final boolean splitLongRecords; + public final boolean omitUnsplitSuffix; + public final boolean unrollRecordDeletes; + public final boolean loadViaGets; + public final boolean isDryRun; + public final boolean useVersionInKey; + + public SplitHelperTestConfig(boolean splitLongRecords, boolean omitUnsplitSuffix, boolean unrollRecordDeletes, + boolean loadViaGets, boolean isDryRun, boolean useVersionInKey) { + this.splitLongRecords = splitLongRecords; + this.omitUnsplitSuffix = omitUnsplitSuffix; + this.unrollRecordDeletes = unrollRecordDeletes; + this.loadViaGets = loadViaGets; + this.isDryRun = isDryRun; + this.useVersionInKey = useVersionInKey; + } + + public SplitKeyValueHelper keyHelper(int localVersion) { + if (useVersionInKey) { + return new VersioningSplitKeyValueHelper(Versionstamp.incomplete(localVersion)); + } else { + return DefaultSplitKeyValueHelper.INSTANCE; + } + } + + @Nonnull + public RecordLayerPropertyStorage.Builder setProps(@Nonnull RecordLayerPropertyStorage.Builder props) { + return props + .addProp(FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES, unrollRecordDeletes) + .addProp(FDBRecordStoreProperties.LOAD_RECORDS_VIA_GETS, loadViaGets); + } + + public boolean hasSplitPoints() { + return splitLongRecords || !omitUnsplitSuffix; + } + + @Override + public String toString() { + return "SplitHelperTestConfig{" + + "splitLongRecords=" + splitLongRecords + + ", omitUnsplitSuffix=" + omitUnsplitSuffix + + ", unrollRecordDeletes=" + unrollRecordDeletes + + ", loadViaGets=" + loadViaGets + + ", isDryRun=" + isDryRun + + ", useVersionInKey=" + useVersionInKey + + '}'; + } + + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + final SplitHelperTestConfig that = (SplitHelperTestConfig)o; + return splitLongRecords == that.splitLongRecords && omitUnsplitSuffix == that.omitUnsplitSuffix && + unrollRecordDeletes == that.unrollRecordDeletes && loadViaGets == that.loadViaGets && + isDryRun == that.isDryRun && useVersionInKey == that.useVersionInKey; + } + + @Override + public int hashCode() { + return Objects.hash(splitLongRecords, omitUnsplitSuffix, unrollRecordDeletes, loadViaGets, useVersionInKey); + } + + public static Stream allValidConfigs() { + // Note that splitLongRecords="true" && omitUnsplitSuffix="true" is not valid + return Stream.of(false, true).flatMap(useVersionInKey -> + Stream.of(false, true).flatMap(splitLongRecords -> + (splitLongRecords ? Stream.of(false) : Stream.of(false, true)).flatMap(omitUnsplitSuffix -> + Stream.of(false, true).flatMap(unrollRecordDeletes -> + Stream.of(false, true).flatMap(loadViaGets -> + Stream.of(false, true).map(isDryRun -> + new SplitHelperTestConfig(splitLongRecords, omitUnsplitSuffix, unrollRecordDeletes, loadViaGets, isDryRun, useVersionInKey))))))); + } + + public static Stream getConfigsNoDryRun() { + return allValidConfigs().filter(config -> !config.isDryRun); + } + + public static Stream getConfigsNoVersionInKey() { + return allValidConfigs().filter(config -> !config.useVersionInKey); + } + + public static SplitHelperTestConfig getDefault() { + return new SplitHelperTestConfig(true, false, + FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES.getDefaultValue(), + FDBRecordStoreProperties.LOAD_RECORDS_VIA_GETS.getDefaultValue(), false, false); + } +} From 8f7038ec94c209d7335fed569cec1f22f2a38481 Mon Sep 17 00:00:00 2001 From: ohad Date: Mon, 2 Mar 2026 14:25:19 -0500 Subject: [PATCH 19/29] typos, hashcode --- .../provider/foundationdb/VersioningSplitKeyValueHelper.java | 2 +- .../record/provider/foundationdb/SplitHelperTestConfig.java | 2 +- 2 files changed, 2 insertions(+), 2 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java index 9996fb2e86..31206c781b 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java @@ -43,7 +43,7 @@ * As a result, it cannot override previous values saved within the same transaction. Split records saved using this feature * are not stored in the RYW cache and attempts to write over them (using the same local version and PK) before the commit * may result in corrupt data. - * Do not override records with another write using teh same local version and PK. + * Do not override records with another write using the same local version and PK. */ public class VersioningSplitKeyValueHelper implements SplitKeyValueHelper { private Versionstamp versionstamp; diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTestConfig.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTestConfig.java index 46eff6d847..470a9a9032 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTestConfig.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTestConfig.java @@ -93,7 +93,7 @@ public boolean equals(final Object o) { @Override public int hashCode() { - return Objects.hash(splitLongRecords, omitUnsplitSuffix, unrollRecordDeletes, loadViaGets, useVersionInKey); + return Objects.hash(splitLongRecords, omitUnsplitSuffix, unrollRecordDeletes, loadViaGets, isDryRun, useVersionInKey); } public static Stream allValidConfigs() { From c1b641c93cb480c377dd9cfecf2333094633f886 Mon Sep 17 00:00:00 2001 From: ohad Date: Tue, 3 Mar 2026 15:44:39 -0500 Subject: [PATCH 20/29] PR comments. --- .../DefaultSplitKeyValueHelper.java | 4 + .../foundationdb/SplitKeyValueHelper.java | 6 +- .../VersioningSplitKeyValueHelper.java | 16 +- .../SplitHelperMultipleTransactionsTest.java | 157 +++++++++--------- 4 files changed, 98 insertions(+), 85 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/DefaultSplitKeyValueHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/DefaultSplitKeyValueHelper.java index 308977823b..7258fbf439 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/DefaultSplitKeyValueHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/DefaultSplitKeyValueHelper.java @@ -21,13 +21,17 @@ package com.apple.foundationdb.record.provider.foundationdb; import com.apple.foundationdb.Transaction; +import com.apple.foundationdb.annotation.API; import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.Tuple; /** * The default implementation of the {@link SplitKeyValueHelper}. * This implementation is using the subspace serialization and transaction set methods. + * This implementation does not support {@link com.apple.foundationdb.tuple.Versionstamp} in the key (as it allows + * one to reside in the value). */ +@API(API.Status.INTERNAL) public class DefaultSplitKeyValueHelper implements SplitKeyValueHelper { public static final DefaultSplitKeyValueHelper INSTANCE = new DefaultSplitKeyValueHelper(); diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyValueHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyValueHelper.java index f1500112d3..99c8957ce5 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyValueHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyValueHelper.java @@ -20,25 +20,27 @@ package com.apple.foundationdb.record.provider.foundationdb; +import com.apple.foundationdb.annotation.API; import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.Tuple; /** * An interface extracting the generation and persistence of keys used in the {@link SplitHelper}. */ +@API(API.Status.INTERNAL) public interface SplitKeyValueHelper { /** * Whether the {@link SplitHelper} should clear older entries before writing new ones. * In the cases where old splits may exist and overlap with the new entries, the SplitHelper should clear the * subspace before writing any new splits. This method is used to determine whether this is necessary. - * @return TRUE if the SplitHelper should clear the subspace before writing new splits. + * @return {@code true} if the SplitHelper should clear the subspace before writing new splits. */ boolean shouldClearBeforeWrite(); /** * Whether the helper allows version mutation in the values. * There are cases where the value cannot have version mutation, for example, when the key needs to have one. - * @return true if version mutations are allowed for the value of the k/v pair, false if not + * @return {@code true} if version mutations are allowed for the value of the k/v pair, false if not */ boolean supportsVersionInValue(); diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java index 31206c781b..0af6e1bc63 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java @@ -21,6 +21,7 @@ package com.apple.foundationdb.record.provider.foundationdb; import com.apple.foundationdb.MutationType; +import com.apple.foundationdb.annotation.API; import com.apple.foundationdb.record.RecordCoreInternalException; import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.Tuple; @@ -28,7 +29,7 @@ /** * A {@link SplitKeyValueHelper} that is used when the Key contains a {@link Versionstamp}. - *

    This implementation should be used when the key contains a version stamp, as it will ensure that the proper FDB APIs + *

    This implementation should be used when the key contains a {@link Versionstamp}, as it will ensure that the proper FDB APIs * encode and decode the key correctly.

    *

    This class is stateful (has a single {@link Versionstamp}) that is going to be used for all splits of a single K/V, * so that all contain the same fixed part and can be correlated after the commit.

    @@ -39,12 +40,13 @@ *

    which means that the entries are sorted by their insertion order (versionstamp order), then by the * original key, with split fragments last.

    * - * NOTE: Since this class uses versionstamps in the FDB Key, it does not know the actual key used until after the commit. + * NOTE: Since this class uses {@link Versionstamp} in the FDB Key, it does not know the actual key used until after the commit. * As a result, it cannot override previous values saved within the same transaction. Split records saved using this feature * are not stored in the RYW cache and attempts to write over them (using the same local version and PK) before the commit * may result in corrupt data. * Do not override records with another write using the same local version and PK. */ +@API(API.Status.INTERNAL) public class VersioningSplitKeyValueHelper implements SplitKeyValueHelper { private Versionstamp versionstamp; @@ -55,9 +57,9 @@ public VersioningSplitKeyValueHelper(final Versionstamp versionstamp) { /** * No need to clear subspace. * Since the key has a unique component (version), no conflicts are expected, so no need to clean before saving new splits. - * Furthermore, since the key contains a version stamp, we don't know the actual key contents ahead of committing + * Furthermore, since the key contains a {@link Versionstamp}, we don't know the actual key contents ahead of committing * the transaction, and so no clean can be done. - * @return false, as new keys should not interfere with old ones. + * @return {@code false}, as new keys should not interfere with old ones. */ @Override public boolean shouldClearBeforeWrite() { @@ -66,7 +68,7 @@ public boolean shouldClearBeforeWrite() { /** * Since the key has versions, prevent the values from having them. - * @return false, since only keys or values are allowed to mutate in FDB, and this mutates the keys + * @return {@code false}, since only keys or values are allowed to mutate in FDB, and this mutates the keys */ @Override public boolean supportsVersionInValue() { @@ -89,7 +91,9 @@ public void writeSplit(final FDBRecordContext context, final byte[] keyBytes, fi valueBytes); if (current != null) { - // This should never happen + // This should never happen. It means that the same key (and suffix) and local version were used for subsequent + // write. It is most likely not an intended flow and this check will protect against that. + // It is an incomplete check since the same record can have different suffixes to the primary key that would not collide. throw new RecordCoreInternalException("Key with version overwritten"); } } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java index 6d602f7714..57fe8b1702 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java @@ -77,10 +77,10 @@ /** * A test that uses the same test cases as in {@link SplitHelperTest}, with the added behavior of multiple transactions. - * Tests here ar named similarly to the ones in {@link SplitHelperTest} with the added "MultipleTransactions" + * Tests here are named similarly to the ones in {@link SplitHelperTest} for tracability. * suffix, so that they can be traced back to their origin. * For tests that require the "versionInKey" to be TRUE, this is the only way to get them to run, as the - * verification of the key content can be done after a commit. + * verification of the key content can only be done after a commit. * The pattern used in this test normally splits each test flow in two: *
      *
    • Save record and record global and local version
    • @@ -216,7 +216,8 @@ private SplitHelper.SizeInfo saveOnly(@Nonnull FDBRecordContext context, @Nonnul private void verifySuccessfullySaved(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, @Nullable FDBRecordVersion version, @Nonnull SplitHelperTestConfig testConfig) { - // do nothing if not saveOnly was not actually invoked + // Match the logic of "saveWithSplit" and do nothing here if it called "saveUnsuccessfully" + // since in these cases no record was saved if (testConfig.omitUnsplitSuffix && version != null) { // cannot include version return; @@ -304,10 +305,29 @@ private Tuple toCompleteKey(Tuple key, byte[] versionStamp, int localVersion, bo } } + private SplitHelper.SizeInfo saveWithSplit(@Nonnull FDBRecordContext context, @Nonnull Tuple key, + byte[] serialized, + @Nullable FDBRecordVersion version, + @Nonnull SplitHelperTestConfig testConfig, + @Nullable FDBStoredSizes previousSizeInfo, + int localVersion) { + if (testConfig.omitUnsplitSuffix && version != null) { + return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, + RecordCoreArgumentException.class, "Cannot include version"); + } else if (testConfig.useVersionInKey && version != null) { + return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, + RecordCoreException.class, "Split version is not supported for this helper"); + } else if (!testConfig.splitLongRecords && serialized.length > SplitHelper.SPLIT_RECORD_SIZE) { + return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, + RecordCoreException.class, "Record is too long"); + } else { + return saveOnly(context, key, serialized, version, testConfig, previousSizeInfo, localVersion); + } + } + @MethodSource("testConfigsNoDryRun") - @ParameterizedTest(name = "saveWithSplitMultipleTransactions[{0}]") - void saveWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig testConfig) { - // dry run does not support transactions + @ParameterizedTest(name = "saveWithSplit[{0}]") + void saveWithSplit(@Nonnull SplitHelperTestConfig testConfig) { this.testConfig = testConfig; final Tuple key1 = Tuple.from(1066L); @@ -324,11 +344,11 @@ void saveWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig testConfig try (FDBRecordContext context = openContext()) { // save with no version and no previousSizeInfo localVersion1 = context.claimLocalVersion(); - sizes1 = saveWithSplitForMultipleTransactions(context, key1, SHORT_STRING, null, testConfig, null, localVersion1); + sizes1 = saveWithSplit(context, key1, SHORT_STRING, null, testConfig, null, localVersion1); localVersion2 = context.claimLocalVersion(); - sizes2 = saveWithSplitForMultipleTransactions(context, key2, LONG_STRING, null, testConfig, null, localVersion2); + sizes2 = saveWithSplit(context, key2, LONG_STRING, null, testConfig, null, localVersion2); localVersion3 = context.claimLocalVersion(); - sizes3 = saveWithSplitForMultipleTransactions(context, key3, VERY_LONG_STRING, null, testConfig, null, localVersion3); + sizes3 = saveWithSplit(context, key3, VERY_LONG_STRING, null, testConfig, null, localVersion3); commit(context); globalVersionstamp = context.getVersionStamp(); @@ -348,55 +368,38 @@ void saveWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig testConfig int localVersion5 = 0; int localVersion6 = 0; // Transaction #3: modify saved values - try (FDBRecordContext context = openContext()) { - // Save over some things using the previous split points - if (testConfig.splitLongRecords) { - localVersion4 = context.claimLocalVersion(); - saveWithSplitForMultipleTransactions(context, key1, VERY_LONG_STRING, null, testConfig, sizes1, localVersion4); - localVersion5 = context.claimLocalVersion(); - saveWithSplitForMultipleTransactions(context, key3, LONG_STRING, null, testConfig, sizes3, localVersion5); + // When there is a version in the key we do not override the previous records, so there is nothing to do here + if (!testConfig.useVersionInKey) { + try (FDBRecordContext context = openContext()) { + // Save over some things using the previous split points + if (testConfig.splitLongRecords) { + localVersion4 = context.claimLocalVersion(); + saveWithSplit(context, key1, VERY_LONG_STRING, null, testConfig, sizes1, localVersion4); + localVersion5 = context.claimLocalVersion(); + saveWithSplit(context, key3, LONG_STRING, null, testConfig, sizes3, localVersion5); + } + localVersion6 = context.claimLocalVersion(); + saveWithSplit(context, key2, SHORT_STRING, null, testConfig, sizes2, localVersion6); + commit(context); + globalVersionstamp = context.getVersionStamp(); } - localVersion6 = context.claimLocalVersion(); - saveWithSplitForMultipleTransactions(context, key2, SHORT_STRING, null, testConfig, sizes2, localVersion6); - commit(context); - globalVersionstamp = context.getVersionStamp(); - } - Tuple verifyKey4 = toCompleteKey(key1, globalVersionstamp, localVersion4, testConfig.useVersionInKey); - Tuple verifyKey5 = toCompleteKey(key3, globalVersionstamp, localVersion5, testConfig.useVersionInKey); - Tuple verifyKey6 = toCompleteKey(key2, globalVersionstamp, localVersion6, testConfig.useVersionInKey); - // Transaction #4: verify changes - try (FDBRecordContext context = openContext()) { - if (testConfig.splitLongRecords) { - verifySuccessfullySaved(context, verifyKey4, VERY_LONG_STRING, null, testConfig); - verifySuccessfullySaved(context, verifyKey5, LONG_STRING, null, testConfig); + Tuple verifyKey4 = toCompleteKey(key1, globalVersionstamp, localVersion4, testConfig.useVersionInKey); + Tuple verifyKey5 = toCompleteKey(key3, globalVersionstamp, localVersion5, testConfig.useVersionInKey); + Tuple verifyKey6 = toCompleteKey(key2, globalVersionstamp, localVersion6, testConfig.useVersionInKey); + // Transaction #4: verify changes + try (FDBRecordContext context = openContext()) { + if (testConfig.splitLongRecords) { + verifySuccessfullySaved(context, verifyKey4, VERY_LONG_STRING, null, testConfig); + verifySuccessfullySaved(context, verifyKey5, LONG_STRING, null, testConfig); + } + verifySuccessfullySaved(context, verifyKey6, SHORT_STRING, null, testConfig); } - verifySuccessfullySaved(context, verifyKey6, SHORT_STRING, null, testConfig); - } - } - - private SplitHelper.SizeInfo saveWithSplitForMultipleTransactions(@Nonnull FDBRecordContext context, @Nonnull Tuple key, - byte[] serialized, - @Nullable FDBRecordVersion version, - @Nonnull SplitHelperTestConfig testConfig, - @Nullable FDBStoredSizes previousSizeInfo, - int localVersion) { - if (testConfig.omitUnsplitSuffix && version != null) { - return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, - RecordCoreArgumentException.class, "Cannot include version"); - } else if (testConfig.useVersionInKey && version != null) { - return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, - RecordCoreException.class, "Split version is not supported for this helper"); - } else if (!testConfig.splitLongRecords && serialized.length > SplitHelper.SPLIT_RECORD_SIZE) { - return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, - RecordCoreException.class, "Record is too long"); - } else { - return saveOnly(context, key, serialized, version, testConfig, previousSizeInfo, localVersion); } } @MethodSource("testConfigsNoDryRun") @ParameterizedTest(name = "saveWithSplitAndCompleteVersion[{0}]") - public void saveWithSplitAndCompleteVersionsMultipleTransactions(SplitHelperTestConfig testConfig) { + public void saveWithSplitAndCompleteVersions(SplitHelperTestConfig testConfig) { this.testConfig = testConfig; byte[] globalValueVersion = "karlgrosse".getBytes(StandardCharsets.US_ASCII); byte[] globalKeyVersion; @@ -415,13 +418,13 @@ public void saveWithSplitAndCompleteVersionsMultipleTransactions(SplitHelperTest try (FDBRecordContext context = openContext()) { version1 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); localVersion1 = context.claimLocalVersion(); - saveWithSplitForMultipleTransactions(context, key1, SHORT_STRING, version1, testConfig, null, localVersion1); + saveWithSplit(context, key1, SHORT_STRING, version1, testConfig, null, localVersion1); localVersion2 = context.claimLocalVersion(); version2 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); - saveWithSplitForMultipleTransactions(context, key2, LONG_STRING, version2, testConfig, null, localVersion2); + saveWithSplit(context, key2, LONG_STRING, version2, testConfig, null, localVersion2); localVersion3 = context.claimLocalVersion(); version3 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); - saveWithSplitForMultipleTransactions(context, key3, VERY_LONG_STRING, version3, testConfig, null, localVersion3); + saveWithSplit(context, key3, VERY_LONG_STRING, version3, testConfig, null, localVersion3); commit(context); globalKeyVersion = context.getVersionStamp(); } @@ -441,11 +444,11 @@ public void saveWithSplitAndCompleteVersionsMultipleTransactions(SplitHelperTest // Save over the records *without* using the previous size info try (FDBRecordContext context = openContext()) { localVersion1 = context.claimLocalVersion(); - saveWithSplitForMultipleTransactions(context, key1, SHORT_STRING, null, testConfig, null, localVersion1); + saveWithSplit(context, key1, SHORT_STRING, null, testConfig, null, localVersion1); localVersion2 = context.claimLocalVersion(); - saveWithSplitForMultipleTransactions(context, key2, LONG_STRING, null, testConfig, null, localVersion2); + saveWithSplit(context, key2, LONG_STRING, null, testConfig, null, localVersion2); localVersion3 = context.claimLocalVersion(); - saveWithSplitForMultipleTransactions(context, key3, VERY_LONG_STRING, null, testConfig, null, localVersion3); + saveWithSplit(context, key3, VERY_LONG_STRING, null, testConfig, null, localVersion3); commit(context); globalKeyVersion = context.getVersionStamp(); } @@ -468,13 +471,13 @@ public void saveWithSplitAndCompleteVersionsMultipleTransactions(SplitHelperTest try (FDBRecordContext context = openContext()) { localVersion1 = context.claimLocalVersion(); version4 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); - sizes4 = saveWithSplitForMultipleTransactions(context, key1, SHORT_STRING, version4, testConfig, null, localVersion1); + sizes4 = saveWithSplit(context, key1, SHORT_STRING, version4, testConfig, null, localVersion1); localVersion2 = context.claimLocalVersion(); version5 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); - sizes5 = saveWithSplitForMultipleTransactions(context, key2, LONG_STRING, version5, testConfig, null, localVersion2); + sizes5 = saveWithSplit(context, key2, LONG_STRING, version5, testConfig, null, localVersion2); localVersion3 = context.claimLocalVersion(); version6 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); - sizes6 = saveWithSplitForMultipleTransactions(context, key3, VERY_LONG_STRING, version6, testConfig, null, localVersion3); + sizes6 = saveWithSplit(context, key3, VERY_LONG_STRING, version6, testConfig, null, localVersion3); commit(context); globalKeyVersion = context.getVersionStamp(); } @@ -495,11 +498,11 @@ public void saveWithSplitAndCompleteVersionsMultipleTransactions(SplitHelperTest // Save over the records *with* using the previous size info try (FDBRecordContext context = openContext()) { localVersion1 = context.claimLocalVersion(); - saveWithSplitForMultipleTransactions(context, key1, SHORT_STRING, null, testConfig, sizes4, localVersion1); + saveWithSplit(context, key1, SHORT_STRING, null, testConfig, sizes4, localVersion1); localVersion2 = context.claimLocalVersion(); - saveWithSplitForMultipleTransactions(context, key2, LONG_STRING, null, testConfig, sizes5, localVersion2); + saveWithSplit(context, key2, LONG_STRING, null, testConfig, sizes5, localVersion2); localVersion3 = context.claimLocalVersion(); - saveWithSplitForMultipleTransactions(context, key3, VERY_LONG_STRING, null, testConfig, sizes6, localVersion3); + saveWithSplit(context, key3, VERY_LONG_STRING, null, testConfig, sizes6, localVersion3); commit(context); globalKeyVersion = context.getVersionStamp(); } @@ -581,8 +584,8 @@ private void deleteSplit(@Nonnull FDBRecordContext context, @Nonnull Tuple key, } @MethodSource("testConfigsNoDryRun") - @ParameterizedTest(name = "deleteWithSplitMultipleTransactions[{0}]") - public void deleteWithSplitMultipleTransactions(@Nonnull SplitHelperTestConfig testConfig) { + @ParameterizedTest(name = "deleteWithSplit[{0}]") + public void deleteWithSplit(@Nonnull SplitHelperTestConfig testConfig) { this.testConfig = testConfig; final Tuple key1 = Tuple.from(-660L); @@ -637,7 +640,7 @@ private interface LoadRecordFunction { FDBRawRecord load(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @Nullable FDBStoredSizes sizes, @Nullable byte[] expectedContents, @Nullable FDBRecordVersion version); } - private void loadSingleRecordsMultipleTransactions(SplitHelperTestConfig testConfig, @Nonnull LoadRecordFunction loadRecordFunction) { + private void loadSingleRecords(SplitHelperTestConfig testConfig, @Nonnull LoadRecordFunction loadRecordFunction) { final Tuple key1 = Tuple.from(1042L); final Tuple key2 = Tuple.from(1066L); final Tuple key3 = Tuple.from(1087L); @@ -856,10 +859,10 @@ private FDBRawRecord loadWithSplit(@Nonnull FDBRecordContext context, @Nonnull T } @MethodSource("testConfigsNoDryRun") - @ParameterizedTest(name = "loadWithSplitMultipleTransactions[{0}]") - public void loadWithSplitMultipleTransactions(SplitHelperTestConfig testConfig) { + @ParameterizedTest(name = "loadWithSplit[{0}]") + public void loadWithSplit(SplitHelperTestConfig testConfig) { this.testConfig = testConfig; - loadSingleRecordsMultipleTransactions(testConfig, + loadSingleRecords(testConfig, (context, key, expectedSizes, expectedContents, version) -> loadWithSplit(context, key, testConfig, expectedSizes, expectedContents, version)); @@ -927,13 +930,13 @@ private FDBRawRecord scanSingleRecord(@Nonnull FDBRecordContext context, boolean @ParameterizedTest(name = "scan[reverse = {0}, useVersionInKey = {1}]") @CsvSource({"false, false", "false, true", "true, false", "true, true"}) - public void scanSingleRecordsMultipleTransactions(boolean reverse, boolean useVersionInKey) { - loadSingleRecordsMultipleTransactions(new SplitHelperTestConfig(true, false, FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES.getDefaultValue(), false, false, useVersionInKey), + public void scanSingleRecords(boolean reverse, boolean useVersionInKey) { + loadSingleRecords(new SplitHelperTestConfig(true, false, FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES.getDefaultValue(), false, false, useVersionInKey), (context, key, expectedSizes, expectedContents, version) -> scanSingleRecord(context, reverse, key, expectedSizes, expectedContents, version, useVersionInKey)); } - private List writeDummyRecordsMultipleTransactions(boolean useVersionInKey) { + private List writeDummyRecords(boolean useVersionInKey) { final byte[] valueVersion = "_cushions_".getBytes(StandardCharsets.US_ASCII); // Generate primary keys using a generalization of the Fibonacci formula: https://oeis.org/A247698 long currKey = 2308L; @@ -973,11 +976,11 @@ private List writeDummyRecordsMultipleTransactions(boolean useVers return rawRecords; } - @ParameterizedTest(name = "scanMultipleRecordsMultipleTransactions[reverse = {0}, useVersionInKey = {1}]") + @ParameterizedTest(name = "scanMultipleRecords[reverse = {0}, useVersionInKey = {1}]") @CsvSource({"false, false", "false, true", "true, false", "true, true"}) - void scanMultipleRecordsMultipleTransactions(boolean reverse, boolean useVersionInKey) { + void scanMultipleRecords(boolean reverse, boolean useVersionInKey) { final ScanProperties scanProperties = reverse ? ScanProperties.REVERSE_SCAN : ScanProperties.FORWARD_SCAN; - List rawRecords = writeDummyRecordsMultipleTransactions(useVersionInKey); + List rawRecords = writeDummyRecords(useVersionInKey); try (FDBRecordContext context = openContext()) { KeyValueCursor kvCursor = KeyValueCursor.Builder.withSubspace(subspace) @@ -1025,8 +1028,8 @@ public static Stream limitsReverseVersionArgs() { @MethodSource("limitsReverseVersionArgs") @ParameterizedTest - void scanContinuationsMultipleTransactions(final int returnLimit, final int readLimit, final boolean reverse, boolean useVersionInKey) { - List rawRecords = writeDummyRecordsMultipleTransactions(useVersionInKey); + void scanContinuations(final int returnLimit, final int readLimit, final boolean reverse, boolean useVersionInKey) { + List rawRecords = writeDummyRecords(useVersionInKey); if (reverse) { rawRecords = Lists.reverse(rawRecords); } From 1a68f22599ccf42f5296dc729812057c821cb251 Mon Sep 17 00:00:00 2001 From: ohad Date: Wed, 4 Mar 2026 16:32:30 -0500 Subject: [PATCH 21/29] PR comments: Add backwards-compatible method to API --- .../provider/foundationdb/FDBRecordStore.java | 3 +-- .../provider/foundationdb/SplitHelper.java | 24 +++++++++++++++++++ .../foundationdb/SplitHelperTest.java | 6 ++--- 3 files changed, 27 insertions(+), 6 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java index 44e0f9a0bb..c49df95f6f 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java @@ -664,8 +664,7 @@ private FDBStoredRecord serializeAndSaveRecord(@Nonnull R final FDBRecordVersion splitVersion = useOldVersionFormat() ? null : version; final SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); preloadCache.invalidate(primaryKey); // clear out cache of older value if present - SplitHelper.saveWithSplit(context, recordsSubspace(), recordBuilder.getPrimaryKey(), serialized, splitVersion, metaData.isSplitLongRecords(), omitUnsplitRecordSuffix, - DefaultSplitKeyValueHelper.INSTANCE, true, oldSizeInfo, sizeInfo); + SplitHelper.saveWithSplit(context, recordsSubspace(), recordBuilder.getPrimaryKey(), serialized, splitVersion, metaData.isSplitLongRecords(), omitUnsplitRecordSuffix, true, oldSizeInfo, sizeInfo); countKeysAndValues(FDBStoreTimer.Counts.SAVE_RECORD_KEY, FDBStoreTimer.Counts.SAVE_RECORD_KEY_BYTES, FDBStoreTimer.Counts.SAVE_RECORD_VALUE_BYTES, sizeInfo); recordBuilder.setSize(sizeInfo); diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java index 7457562ad7..2ce6864684 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java @@ -122,6 +122,30 @@ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnu * @param sizeInfo optional size information to populate */ @SuppressWarnings("PMD.CloseResource") + public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnull final Subspace subspace, + @Nonnull final Tuple key, @Nonnull final byte[] serialized, @Nullable final FDBRecordVersion version, + final boolean splitLongRecords, final boolean omitUnsplitSuffix, + final boolean clearBasedOnPreviousSizeInfo, @Nullable final FDBStoredSizes previousSizeInfo, + @Nullable SizeInfo sizeInfo) { + + saveWithSplit(context, subspace, key, serialized, version, splitLongRecords, omitUnsplitSuffix, DefaultSplitKeyValueHelper.INSTANCE, clearBasedOnPreviousSizeInfo, previousSizeInfo, sizeInfo); + } + + /** + * Save serialized representation using multiple keys if necessary, clearing only as much as needed. + * @param context write transaction + * @param subspace subspace to save in + * @param key key within subspace + * @param serialized serialized representation + * @param version the version to store inline with this record + * @param splitLongRecords true if multiple keys should be used; if false, serialized must fit in a single key + * @param omitUnsplitSuffix if splitLongRecords is false, then this will omit a suffix added to the end of the key if true for backwards-compatibility reasons + * @param splitKeyHelper an instance of {@link SplitKeyValueHelper} to use for the operation + * @param clearBasedOnPreviousSizeInfo if splitLongRecords, whether to use previousSizeInfo to determine how much to clear + * @param previousSizeInfo if clearBasedOnPreviousSizeInfo, the {@link FDBStoredSizes} for any old record, or null if there was no old record + * @param sizeInfo optional size information to populate + */ + @SuppressWarnings("PMD.CloseResource") public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnull final Subspace subspace, @Nonnull final Tuple key, @Nonnull final byte[] serialized, @Nullable final FDBRecordVersion version, final boolean splitLongRecords, final boolean omitUnsplitSuffix, diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java index f76d13d5f1..63f2e1b0f4 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java @@ -148,8 +148,7 @@ private SplitHelper.SizeInfo saveUnsuccessfully(@Nonnull F @Nonnull Class errClazz, @Nonnull String errMessage) { final SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); E e = assertThrows(errClazz, - () -> SplitHelper.saveWithSplit(context, subspace, key, serialized, version, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, - DefaultSplitKeyValueHelper.INSTANCE, previousSizeInfo != null, previousSizeInfo, sizeInfo)); + () -> SplitHelper.saveWithSplit(context, subspace, key, serialized, version, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, previousSizeInfo != null, previousSizeInfo, sizeInfo)); assertThat(e.getMessage(), containsString(errMessage)); assertEquals(0, sizeInfo.getKeyCount()); @@ -173,8 +172,7 @@ private SplitHelper.SizeInfo saveSuccessfully(@Nonnull FDBRecordContext context, @Nonnull SplitHelperTestConfig testConfig, @Nullable FDBStoredSizes previousSizeInfo) { final SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); - SplitHelper.saveWithSplit(context, subspace, key, serialized, version, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, - DefaultSplitKeyValueHelper.INSTANCE, previousSizeInfo != null, previousSizeInfo, sizeInfo); + SplitHelper.saveWithSplit(context, subspace, key, serialized, version, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, previousSizeInfo != null, previousSizeInfo, sizeInfo); int dataKeyCount = (serialized.length - 1) / SplitHelper.SPLIT_RECORD_SIZE + 1; boolean isSplit = dataKeyCount > 1; int keyCount = dataKeyCount; From d7deeed2ac6d0c9b3292280aa89a70bec00f075c Mon Sep 17 00:00:00 2001 From: ohad Date: Wed, 4 Mar 2026 17:36:42 -0500 Subject: [PATCH 22/29] More PR comments --- .../SplitHelperMultipleTransactionsTest.java | 11 +++++++---- .../lucene/directory/PendingWritesQueueHelper.java | 3 ++- .../lucene/directory/PendingWriteQueueTest.java | 14 +++++++++++--- 3 files changed, 20 insertions(+), 8 deletions(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java index 57fe8b1702..978eb4e2df 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java @@ -38,6 +38,7 @@ import com.apple.foundationdb.tuple.ByteArrayUtil; import com.apple.foundationdb.tuple.Tuple; import com.apple.foundationdb.tuple.Versionstamp; +import com.apple.test.BooleanSource; import com.apple.test.ParameterizedTestUtils; import com.apple.test.Tags; import com.google.common.collect.Lists; @@ -415,6 +416,7 @@ public void saveWithSplitAndCompleteVersions(SplitHelperTestConfig testConfig) { FDBRecordVersion version1; FDBRecordVersion version2; FDBRecordVersion version3; + // Save records with complete version in the value try (FDBRecordContext context = openContext()) { version1 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); localVersion1 = context.claimLocalVersion(); @@ -428,7 +430,7 @@ public void saveWithSplitAndCompleteVersions(SplitHelperTestConfig testConfig) { commit(context); globalKeyVersion = context.getVersionStamp(); } - // in some cases nothing gets written (all saves are "saveUnsuccessfully) so the transaction is read-only + // in some cases nothing gets written (all saves are "saveUnsuccessfully") so the transaction is read-only // and there is no global version if (globalKeyVersion != null) { completeKey1 = toCompleteKey(key1, globalKeyVersion, localVersion1, testConfig.useVersionInKey); @@ -441,7 +443,7 @@ public void saveWithSplitAndCompleteVersions(SplitHelperTestConfig testConfig) { } } - // Save over the records *without* using the previous size info + // Save over the records *without* using the previous size info and with no version in the value try (FDBRecordContext context = openContext()) { localVersion1 = context.claimLocalVersion(); saveWithSplit(context, key1, SHORT_STRING, null, testConfig, null, localVersion1); @@ -468,6 +470,7 @@ public void saveWithSplitAndCompleteVersions(SplitHelperTestConfig testConfig) { FDBRecordVersion version4; FDBRecordVersion version5; FDBRecordVersion version6; + // save again with complete version in key and no previous size info try (FDBRecordContext context = openContext()) { localVersion1 = context.claimLocalVersion(); version4 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); @@ -929,7 +932,7 @@ private FDBRawRecord scanSingleRecord(@Nonnull FDBRecordContext context, boolean } @ParameterizedTest(name = "scan[reverse = {0}, useVersionInKey = {1}]") - @CsvSource({"false, false", "false, true", "true, false", "true, true"}) + @BooleanSource({"reverse", "useVersionInKey"}) public void scanSingleRecords(boolean reverse, boolean useVersionInKey) { loadSingleRecords(new SplitHelperTestConfig(true, false, FDBRecordStoreProperties.UNROLL_SINGLE_RECORD_DELETES.getDefaultValue(), false, false, useVersionInKey), (context, key, expectedSizes, expectedContents, version) -> @@ -977,7 +980,7 @@ private List writeDummyRecords(boolean useVersionInKey) { } @ParameterizedTest(name = "scanMultipleRecords[reverse = {0}, useVersionInKey = {1}]") - @CsvSource({"false, false", "false, true", "true, false", "true, true"}) + @BooleanSource({"reverse", "useVersionInKey"}) void scanMultipleRecords(boolean reverse, boolean useVersionInKey) { final ScanProperties scanProperties = reverse ? ScanProperties.REVERSE_SCAN : ScanProperties.FORWARD_SCAN; List rawRecords = writeDummyRecords(useVersionInKey); diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWritesQueueHelper.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWritesQueueHelper.java index f57b9bf0bb..e36b1a328d 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWritesQueueHelper.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWritesQueueHelper.java @@ -48,7 +48,8 @@ public static PendingWriteQueue.QueueEntry toQueueEntry(LuceneSerializer seriali LucenePendingWriteQueueProto.PendingWriteItem item = LucenePendingWriteQueueProto.PendingWriteItem.parseFrom(value); return new PendingWriteQueue.QueueEntry(versionstamp, item); } catch (InvalidProtocolBufferException e) { - throw new RecordCoreStorageException("Failed to parse queue item", e); + throw new RecordCoreStorageException("Failed to parse queue item", e) + .addLogInfo("key", keyTuple); } } diff --git a/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueueTest.java b/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueueTest.java index 9807c82f77..5d4f40b0e9 100644 --- a/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueueTest.java +++ b/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueueTest.java @@ -55,6 +55,8 @@ import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Random; +import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -69,6 +71,8 @@ */ @Tag(Tags.RequiresFDB) class PendingWriteQueueTest extends FDBRecordStoreTestBase { + private static final String CHARACTERS = "abcdefghijklmnopqrstuvwxyz0123456789"; + LuceneSerializer serializer; @BeforeEach @@ -372,7 +376,7 @@ void testLargeQueueItem(boolean useCompression) throws Exception { try (FDBRecordContext context = openContext()) { queue = getQueue(context, serializerToUse); - // save a single doc using the (should succeed since we split the records even for uncompressed) + // save a single doc using the appropriate serializer (should succeed since we split the records even for uncompressed) queue.enqueueInsert(context, docWithHugeString.getPrimaryKey(), docWithHugeString.getFields()); commit(context); } @@ -437,9 +441,13 @@ void testClearEntryWithIncompleteVersionstamp() { @Nonnull private TestDocument createHugeDocument() { - String hugeString = "Hello ".repeat(100_000); + Random random = ThreadLocalRandom.current(); + StringBuilder builder = new StringBuilder(); + for (int i = 0 ; i < 500_000 ; i++) { + builder.append(CHARACTERS.charAt(random.nextInt(CHARACTERS.length()))); + } TestDocument docWithHugeString = new TestDocument(primaryKey("Huge"), - List.of(createField("f", hugeString, LuceneIndexExpressions.DocumentFieldType.STRING, false, false))); + List.of(createField("f", builder.toString(), LuceneIndexExpressions.DocumentFieldType.STRING, false, false))); return docWithHugeString; } From e9da6a3b5b3821fd46d9b144bad5c69524e79d5d Mon Sep 17 00:00:00 2001 From: ohad Date: Wed, 4 Mar 2026 18:05:22 -0500 Subject: [PATCH 23/29] Style --- .../foundationdb/record/provider/foundationdb/SplitHelper.java | 1 - .../foundationdb/SplitHelperMultipleTransactionsTest.java | 1 - 2 files changed, 2 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java index 2ce6864684..bc9a3e738f 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java @@ -33,7 +33,6 @@ import com.apple.foundationdb.record.FDBRecordStoreProperties; import com.apple.foundationdb.record.RecordCoreArgumentException; import com.apple.foundationdb.record.RecordCoreException; -import com.apple.foundationdb.record.RecordCoreInternalException; import com.apple.foundationdb.record.RecordCoreStorageException; import com.apple.foundationdb.record.RecordCursor; import com.apple.foundationdb.record.RecordCursorContinuation; diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java index 978eb4e2df..955d54ae98 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java @@ -47,7 +47,6 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.Arguments; -import org.junit.jupiter.params.provider.CsvSource; import org.junit.jupiter.params.provider.MethodSource; import javax.annotation.Nonnull; From f93aa9f615158910da51765e019bf3a796812138 Mon Sep 17 00:00:00 2001 From: ohad Date: Wed, 4 Mar 2026 22:54:09 -0500 Subject: [PATCH 24/29] Split two test methods that were too long --- .../SplitHelperMultipleTransactionsTest.java | 133 ++++++++++-------- 1 file changed, 74 insertions(+), 59 deletions(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java index 955d54ae98..65dfa15c46 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java @@ -403,44 +403,15 @@ public void saveWithSplitAndCompleteVersions(SplitHelperTestConfig testConfig) { this.testConfig = testConfig; byte[] globalValueVersion = "karlgrosse".getBytes(StandardCharsets.US_ASCII); byte[] globalKeyVersion; - int localVersion1; - int localVersion2; - int localVersion3; - Tuple completeKey1; - Tuple completeKey2; - Tuple completeKey3; final Tuple key1 = Tuple.from(800L); final Tuple key2 = Tuple.from(813L); final Tuple key3 = Tuple.from(823L); - FDBRecordVersion version1; - FDBRecordVersion version2; - FDBRecordVersion version3; - // Save records with complete version in the value - try (FDBRecordContext context = openContext()) { - version1 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); - localVersion1 = context.claimLocalVersion(); - saveWithSplit(context, key1, SHORT_STRING, version1, testConfig, null, localVersion1); - localVersion2 = context.claimLocalVersion(); - version2 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); - saveWithSplit(context, key2, LONG_STRING, version2, testConfig, null, localVersion2); - localVersion3 = context.claimLocalVersion(); - version3 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); - saveWithSplit(context, key3, VERY_LONG_STRING, version3, testConfig, null, localVersion3); - commit(context); - globalKeyVersion = context.getVersionStamp(); - } - // in some cases nothing gets written (all saves are "saveUnsuccessfully") so the transaction is read-only - // and there is no global version - if (globalKeyVersion != null) { - completeKey1 = toCompleteKey(key1, globalKeyVersion, localVersion1, testConfig.useVersionInKey); - completeKey2 = toCompleteKey(key2, globalKeyVersion, localVersion2, testConfig.useVersionInKey); - completeKey3 = toCompleteKey(key3, globalKeyVersion, localVersion3, testConfig.useVersionInKey); - try (FDBRecordContext context = openContext()) { - verifySuccessfullySaved(context, completeKey1, SHORT_STRING, version1, testConfig); - verifySuccessfullySaved(context, completeKey2, LONG_STRING, version2, testConfig); - verifySuccessfullySaved(context, completeKey3, VERY_LONG_STRING, version3, testConfig); - } - } + int localVersion1; + int localVersion2; + int localVersion3; + + // save records with version in the value + saveInitialRecords(testConfig, globalValueVersion, key1, key2, key3); // Save over the records *without* using the previous size info and with no version in the value try (FDBRecordContext context = openContext()) { @@ -453,9 +424,9 @@ public void saveWithSplitAndCompleteVersions(SplitHelperTestConfig testConfig) { commit(context); globalKeyVersion = context.getVersionStamp(); } - completeKey1 = toCompleteKey(key1, globalKeyVersion, localVersion1, testConfig.useVersionInKey); - completeKey2 = toCompleteKey(key2, globalKeyVersion, localVersion2, testConfig.useVersionInKey); - completeKey3 = toCompleteKey(key3, globalKeyVersion, localVersion3, testConfig.useVersionInKey); + Tuple completeKey1 = toCompleteKey(key1, globalKeyVersion, localVersion1, testConfig.useVersionInKey); + Tuple completeKey2 = toCompleteKey(key2, globalKeyVersion, localVersion2, testConfig.useVersionInKey); + Tuple completeKey3 = toCompleteKey(key3, globalKeyVersion, localVersion3, testConfig.useVersionInKey); try (FDBRecordContext context = openContext()) { verifySuccessfullySaved(context, completeKey1, SHORT_STRING, null, testConfig); verifySuccessfullySaved(context, completeKey2, LONG_STRING, null, testConfig); @@ -519,6 +490,46 @@ public void saveWithSplitAndCompleteVersions(SplitHelperTestConfig testConfig) { } } + private void saveInitialRecords(final SplitHelperTestConfig testConfig, final byte[] globalValueVersion, final Tuple key1, final Tuple key2, final Tuple key3) { + byte[] globalKeyVersion; + Tuple completeKey1; + Tuple completeKey2; + Tuple completeKey3; + int localVersion1; + int localVersion2; + int localVersion3; + FDBRecordVersion version1; + FDBRecordVersion version2; + FDBRecordVersion version3; + + // Save records with completed version in the value + try (FDBRecordContext context = openContext()) { + version1 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); + localVersion1 = context.claimLocalVersion(); + saveWithSplit(context, key1, SHORT_STRING, version1, testConfig, null, localVersion1); + localVersion2 = context.claimLocalVersion(); + version2 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); + saveWithSplit(context, key2, LONG_STRING, version2, testConfig, null, localVersion2); + localVersion3 = context.claimLocalVersion(); + version3 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); + saveWithSplit(context, key3, VERY_LONG_STRING, version3, testConfig, null, localVersion3); + commit(context); + globalKeyVersion = context.getVersionStamp(); + } + // in some cases nothing gets written (all saves are "saveUnsuccessfully") so the transaction is read-only + // and there is no global version + if (globalKeyVersion != null) { + completeKey1 = toCompleteKey(key1, globalKeyVersion, localVersion1, testConfig.useVersionInKey); + completeKey2 = toCompleteKey(key2, globalKeyVersion, localVersion2, testConfig.useVersionInKey); + completeKey3 = toCompleteKey(key3, globalKeyVersion, localVersion3, testConfig.useVersionInKey); + try (FDBRecordContext context = openContext()) { + verifySuccessfullySaved(context, completeKey1, SHORT_STRING, version1, testConfig); + verifySuccessfullySaved(context, completeKey2, LONG_STRING, version2, testConfig); + verifySuccessfullySaved(context, completeKey3, VERY_LONG_STRING, version3, testConfig); + } + } + } + @Nonnull private FDBStoredSizes writeDummyRecord(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @Nullable FDBRecordVersion version, int splits, @@ -742,49 +753,53 @@ private void loadSingleRecords(SplitHelperTestConfig testConfig, @Nonnull LoadRe } commit(context); } - - final Tuple completeKey1 = toCompleteKey(key1, keyGlobalVersion, 0, testConfig.useVersionInKey); - final Tuple completeKey2 = toCompleteKey(key2, keyGlobalVersion, localVersion2, testConfig.useVersionInKey); - final Tuple completeKey3 = toCompleteKey(key3, keyGlobalVersion, localVersion3, testConfig.useVersionInKey); - final Tuple completeKey4 = toCompleteKey(key4, keyGlobalVersion, localVersion4, testConfig.useVersionInKey); - final Tuple completeKey5 = toCompleteKey(key5, keyGlobalVersion, localVersion5, testConfig.useVersionInKey); - final Tuple completeKey6 = toCompleteKey(key6, keyGlobalVersion, localVersion6, testConfig.useVersionInKey); - final Tuple completeKey7 = toCompleteKey(key7, keyGlobalVersion, localVersion7, testConfig.useVersionInKey); - final Tuple completeKey8 = toCompleteKey(key8, keyGlobalVersion, localVersion8, testConfig.useVersionInKey); - final Tuple completeKey9 = toCompleteKey(key9, keyGlobalVersion, localVersion9, testConfig.useVersionInKey); - + List completedKeys = List.of( + toCompleteKey(key1, keyGlobalVersion, 0, testConfig.useVersionInKey), + toCompleteKey(key2, keyGlobalVersion, localVersion2, testConfig.useVersionInKey), + toCompleteKey(key3, keyGlobalVersion, localVersion3, testConfig.useVersionInKey), + toCompleteKey(key4, keyGlobalVersion, localVersion4, testConfig.useVersionInKey), + toCompleteKey(key5, keyGlobalVersion, localVersion5, testConfig.useVersionInKey), + toCompleteKey(key6, keyGlobalVersion, localVersion6, testConfig.useVersionInKey), + toCompleteKey(key7, keyGlobalVersion, localVersion7, testConfig.useVersionInKey), + toCompleteKey(key8, keyGlobalVersion, localVersion8, testConfig.useVersionInKey), + toCompleteKey(key9, keyGlobalVersion, localVersion9, testConfig.useVersionInKey)); // transaction 3 - verify + verifyRecords(testConfig, loadRecordFunction, completedKeys, + sizes2, sizes3, sizes5, version3, version4, version9); + } + + private void verifyRecords(final SplitHelperTestConfig testConfig, final @Nonnull LoadRecordFunction loadRecordFunction, final List completeKeys, final FDBStoredSizes sizes2, final FDBStoredSizes sizes3, final FDBStoredSizes sizes5, final FDBRecordVersion version3, final FDBRecordVersion version4, final FDBRecordVersion version9) { try (FDBRecordContext context = openContext()) { // No record - loadRecordFunction.load(context, completeKey1, null, null, null); + loadRecordFunction.load(context, completeKeys.get(0), null, null, null); // One unsplit record - loadRecordFunction.load(context, completeKey2, sizes2, HUMPTY_DUMPTY, null); + loadRecordFunction.load(context, completeKeys.get(1), sizes2, HUMPTY_DUMPTY, null); if ((!testConfig.omitUnsplitSuffix) && (!testConfig.useVersionInKey)) { // One record with version - loadRecordFunction.load(context, completeKey3, sizes3, HUMPTY_DUMPTY, version3); + loadRecordFunction.load(context, completeKeys.get(2), sizes3, HUMPTY_DUMPTY, version3); // One version but missing record final FDBRecordVersion v4 = version4; assertThrows(SplitHelper.FoundSplitWithoutStartException.class, - () -> loadRecordFunction.load(context, completeKey4, null, null, v4)); + () -> loadRecordFunction.load(context, completeKeys.get(3), null, null, v4)); } if (testConfig.splitLongRecords) { // One split record // One split record but then delete the last split point (no way to distinguish this from just inserting one fewer split) - loadRecordFunction.load(context, completeKey5, sizes5, MEDIUM_STRING, null); + loadRecordFunction.load(context, completeKeys.get(4), sizes5, MEDIUM_STRING, null); // One split record then delete the first split point if (testConfig.loadViaGets) { - loadRecordFunction.load(context, completeKey6, null, null, null); + loadRecordFunction.load(context, completeKeys.get(5), null, null, null); } else { assertThrows(SplitHelper.FoundSplitWithoutStartException.class, - () -> loadRecordFunction.load(context, completeKey6, null, null, null)); + () -> loadRecordFunction.load(context, completeKeys.get(5), null, null, null)); } // One split record then delete the middle split point RecordCoreException err7 = assertThrows(RecordCoreException.class, - () -> loadRecordFunction.load(context, completeKey7, null, null, null)); + () -> loadRecordFunction.load(context, completeKeys.get(6), null, null, null)); assertThat(err7.getMessage(), containsString("Split record segments out of order")); // One split record then add an extra key in the middle RecordCoreException err8 = assertThrows(RecordCoreException.class, - () -> loadRecordFunction.load(context, completeKey8, null, null, null)); + () -> loadRecordFunction.load(context, completeKeys.get(7), null, null, null)); assertThat(err8.getMessage(), anyOf( containsString("Expected only a single key extension"), containsString("Split record segments out of order") @@ -793,7 +808,7 @@ private void loadSingleRecords(SplitHelperTestConfig testConfig, @Nonnull LoadRe if (!testConfig.useVersionInKey) { final FDBRecordVersion v9 = version9; assertThrows(SplitHelper.FoundSplitWithoutStartException.class, - () -> loadRecordFunction.load(context, completeKey9, null, null, v9)); + () -> loadRecordFunction.load(context, completeKeys.get(8), null, null, v9)); } } } From 96eccea7a2d2da961763eca3210da11eb6c477d3 Mon Sep 17 00:00:00 2001 From: ohad Date: Sat, 7 Mar 2026 20:18:13 -0500 Subject: [PATCH 25/29] Revert API changes, use tuple.hasIncompleteVersion, adjust tests --- .../provider/foundationdb/SplitHelper.java | 81 ++++----- .../SplitHelperMultipleTransactionsTest.java | 157 ++++++++---------- .../lucene/directory/PendingWriteQueue.java | 2 +- 3 files changed, 114 insertions(+), 126 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java index bc9a3e738f..a190432031 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java @@ -33,6 +33,7 @@ import com.apple.foundationdb.record.FDBRecordStoreProperties; import com.apple.foundationdb.record.RecordCoreArgumentException; import com.apple.foundationdb.record.RecordCoreException; +import com.apple.foundationdb.record.RecordCoreInternalException; import com.apple.foundationdb.record.RecordCoreStorageException; import com.apple.foundationdb.record.RecordCursor; import com.apple.foundationdb.record.RecordCursorContinuation; @@ -104,31 +105,9 @@ private SplitHelper() { */ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnull final Subspace subspace, @Nonnull final Tuple key, @Nonnull final byte[] serialized, @Nullable final FDBRecordVersion version) { - saveWithSplit(context, subspace, key, serialized, version, true, false, DefaultSplitKeyValueHelper.INSTANCE, false, null, null); + saveWithSplit(context, subspace, key, serialized, version, true, false, false, null, null); } - /** - * Save serialized representation using multiple keys if necessary, clearing only as much as needed. - * @param context write transaction - * @param subspace subspace to save in - * @param key key within subspace - * @param serialized serialized representation - * @param version the version to store inline with this record - * @param splitLongRecords true if multiple keys should be used; if false, serialized must fit in a single key - * @param omitUnsplitSuffix if splitLongRecords is false, then this will omit a suffix added to the end of the key if true for backwards-compatibility reasons - * @param clearBasedOnPreviousSizeInfo if splitLongRecords, whether to use previousSizeInfo to determine how much to clear - * @param previousSizeInfo if clearBasedOnPreviousSizeInfo, the {@link FDBStoredSizes} for any old record, or null if there was no old record - * @param sizeInfo optional size information to populate - */ - @SuppressWarnings("PMD.CloseResource") - public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnull final Subspace subspace, - @Nonnull final Tuple key, @Nonnull final byte[] serialized, @Nullable final FDBRecordVersion version, - final boolean splitLongRecords, final boolean omitUnsplitSuffix, - final boolean clearBasedOnPreviousSizeInfo, @Nullable final FDBStoredSizes previousSizeInfo, - @Nullable SizeInfo sizeInfo) { - - saveWithSplit(context, subspace, key, serialized, version, splitLongRecords, omitUnsplitSuffix, DefaultSplitKeyValueHelper.INSTANCE, clearBasedOnPreviousSizeInfo, previousSizeInfo, sizeInfo); - } /** * Save serialized representation using multiple keys if necessary, clearing only as much as needed. @@ -139,7 +118,6 @@ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnu * @param version the version to store inline with this record * @param splitLongRecords true if multiple keys should be used; if false, serialized must fit in a single key * @param omitUnsplitSuffix if splitLongRecords is false, then this will omit a suffix added to the end of the key if true for backwards-compatibility reasons - * @param splitKeyHelper an instance of {@link SplitKeyValueHelper} to use for the operation * @param clearBasedOnPreviousSizeInfo if splitLongRecords, whether to use previousSizeInfo to determine how much to clear * @param previousSizeInfo if clearBasedOnPreviousSizeInfo, the {@link FDBStoredSizes} for any old record, or null if there was no old record * @param sizeInfo optional size information to populate @@ -148,7 +126,6 @@ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnu public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnull final Subspace subspace, @Nonnull final Tuple key, @Nonnull final byte[] serialized, @Nullable final FDBRecordVersion version, final boolean splitLongRecords, final boolean omitUnsplitSuffix, - final SplitKeyValueHelper splitKeyHelper, final boolean clearBasedOnPreviousSizeInfo, @Nullable final FDBStoredSizes previousSizeInfo, @Nullable SizeInfo sizeInfo) { if (omitUnsplitSuffix && version != null) { @@ -157,9 +134,10 @@ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnu .addLogInfo(LogMessageKeys.SUBSPACE, ByteArrayUtil2.loggable(subspace.pack())) .addLogInfo(LogMessageKeys.VERSION, version); } - if ((version != null) && !splitKeyHelper.supportsVersionInValue()) { - // Cannot write version in the k/v value since it is not supported by this helper (e.g. version is needed in the key) - throw new RecordCoreArgumentException("Split version is not supported for this helper"); + boolean hasVersionInKey = key.hasIncompleteVersionstamp(); + if ((version != null) && hasVersionInKey) { + // Cannot have versionStamps in BOTH key and value + throw new RecordCoreArgumentException("Cannot save versionStamp in both key and value"); } if (serialized.length > SplitHelper.SPLIT_RECORD_SIZE) { if (!splitLongRecords) { @@ -168,9 +146,11 @@ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnu .addLogInfo(LogMessageKeys.SUBSPACE, ByteArrayUtil2.loggable(subspace.pack())) .addLogInfo(LogMessageKeys.VALUE_SIZE, serialized.length); } - writeSplitRecord(context, subspace, key, serialized, splitKeyHelper, clearBasedOnPreviousSizeInfo, previousSizeInfo, sizeInfo); + writeSplitRecord(context, subspace, key, serialized, hasVersionInKey, clearBasedOnPreviousSizeInfo, previousSizeInfo, sizeInfo); } else { - if (splitKeyHelper.shouldClearBeforeWrite() && (splitLongRecords || previousSizeInfo == null || previousSizeInfo.isVersionedInline())) { + // an incomplete version in the key means we shouldn't delete previous values for the record (since they all have + // completed versions by now) + if (!hasVersionInKey && (splitLongRecords || previousSizeInfo == null || previousSizeInfo.isVersionedInline())) { // Note that the clearPreviousSplitRecords also removes version splits from the context cache // This is not currently supported for the case where we have versions in the keys since we can't trace the old values down clearPreviousSplitRecord(context, subspace, key, clearBasedOnPreviousSizeInfo, previousSizeInfo); @@ -181,23 +161,43 @@ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnu } else { recordKey = key; } - final byte[] keyBytes = splitKeyHelper.packSplitKey(subspace, recordKey); - splitKeyHelper.writeSplit(context, keyBytes, serialized); + byte[] keyBytes = writeSplitValue(context, subspace, recordKey, serialized, sizeInfo); if (sizeInfo != null) { sizeInfo.set(keyBytes, serialized); sizeInfo.setSplit(false); } } - writeVersion(context, subspace, key, version, sizeInfo, splitKeyHelper); + writeVersion(context, subspace, key, version, sizeInfo); + } + + private static byte[] writeSplitValue(FDBRecordContext context, Subspace subspace, Tuple recordKey, byte[] serialized, @Nullable SizeInfo sizeInfo) { + byte[] keyBytes; + if (recordKey.hasIncompleteVersionstamp()) { + keyBytes = subspace.packWithVersionstamp(recordKey); + byte[] current = context.addVersionMutation( + MutationType.SET_VERSIONSTAMPED_KEY, + keyBytes, + serialized); + if (current != null) { + // This should never happen. It means that the same key (and suffix) and local version were used for subsequent + // write. It is most likely not an intended flow and this check will protect against that. + // It is an incomplete check since the same record can have different suffixes to the primary key that would not collide. + throw new RecordCoreInternalException("Key with version overwritten"); + } + } else { + keyBytes = subspace.pack(recordKey); + context.ensureActive().set(keyBytes, serialized); + } + return keyBytes; } @SuppressWarnings("PMD.CloseResource") private static void writeSplitRecord(@Nonnull final FDBRecordContext context, @Nonnull final Subspace subspace, @Nonnull final Tuple key, @Nonnull final byte[] serialized, - final SplitKeyValueHelper splitKeyHelper, + boolean hasVersionInKey, final boolean clearBasedOnPreviousSizeInfo, @Nullable final FDBStoredSizes previousSizeInfo, @Nullable SizeInfo sizeInfo) { - if (splitKeyHelper.shouldClearBeforeWrite()) { + if (!hasVersionInKey) { // Note that the clearPreviousSplitRecords also removes version splits from the context cache // This is not currently supported for the case where we have versions in the keys since we can't trace the old values down clearPreviousSplitRecord(context, subspace, key, clearBasedOnPreviousSizeInfo, previousSizeInfo); @@ -209,9 +209,8 @@ private static void writeSplitRecord(@Nonnull final FDBRecordContext context, @N if (nextOffset > serialized.length) { nextOffset = serialized.length; } - final byte[] keyBytes = splitKeyHelper.packSplitKey(subspace, key.add(index)); final byte[] valueBytes = Arrays.copyOfRange(serialized, offset, nextOffset); - splitKeyHelper.writeSplit(context, keyBytes, valueBytes); + byte[] keyBytes = writeSplitValue(context, subspace, key.add(index), valueBytes, sizeInfo); if (sizeInfo != null) { if (offset == 0) { sizeInfo.set(keyBytes, valueBytes); @@ -227,17 +226,19 @@ private static void writeSplitRecord(@Nonnull final FDBRecordContext context, @N @SuppressWarnings("PMD.CloseResource") private static void writeVersion(@Nonnull final FDBRecordContext context, @Nonnull final Subspace subspace, @Nonnull final Tuple key, - @Nullable final FDBRecordVersion version, @Nullable final SizeInfo sizeInfo, final SplitKeyValueHelper splitKeyHelper) { + @Nullable final FDBRecordVersion version, @Nullable final SizeInfo sizeInfo) { if (version == null) { if (sizeInfo != null) { sizeInfo.setVersionedInline(false); } return; } - final byte[] keyBytes = splitKeyHelper.packSplitKey(subspace, key.add(RECORD_VERSION)); + // At this point we know the key does not have a version + // TODO: Create a flavor of writeSplit for this case + final byte[] keyBytes = subspace.pack(key.add(RECORD_VERSION)); final byte[] valueBytes = packVersion(version); if (version.isComplete()) { - splitKeyHelper.writeSplit(context, keyBytes, valueBytes); + context.ensureActive().set(keyBytes, valueBytes); } else { context.addVersionMutation(MutationType.SET_VERSIONSTAMPED_VALUE, keyBytes, valueBytes); context.addToLocalVersionCache(keyBytes, version.getLocalVersion()); diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java index 65dfa15c46..9019b5c379 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java @@ -154,7 +154,7 @@ private SplitHelper.SizeInfo saveUnsuccessfully(@Nonnull F final SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); E e = assertThrows(errClazz, () -> SplitHelper.saveWithSplit(context, subspace, key, serialized, version, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, - testConfig.keyHelper(0), previousSizeInfo != null, previousSizeInfo, sizeInfo)); + previousSizeInfo != null, previousSizeInfo, sizeInfo)); assertThat(e.getMessage(), containsString(errMessage)); assertEquals(0, sizeInfo.getKeyCount()); @@ -174,11 +174,10 @@ private SplitHelper.SizeInfo saveUnsuccessfully(@Nonnull F private SplitHelper.SizeInfo saveOnly(@Nonnull FDBRecordContext context, @Nonnull Tuple key, byte[] serialized, @Nullable FDBRecordVersion version, @Nonnull SplitHelperTestConfig testConfig, - @Nullable FDBStoredSizes previousSizeInfo, - int localVersion) { + @Nullable FDBStoredSizes previousSizeInfo) { final SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); SplitHelper.saveWithSplit(context, subspace, key, serialized, version, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, - testConfig.keyHelper(localVersion), previousSizeInfo != null, previousSizeInfo, sizeInfo); + previousSizeInfo != null, previousSizeInfo, sizeInfo); int dataKeyCount = (serialized.length - 1) / SplitHelper.SPLIT_RECORD_SIZE + 1; boolean isSplit = dataKeyCount > 1; int keyCount = dataKeyCount; @@ -297,6 +296,14 @@ private void verifySuccessfullySaved(@Nonnull FDBRecordContext context, @Nonnull } } + private Tuple toIncompleteKey(Tuple key, int localVersion, boolean versionInKey) { + if (versionInKey) { + return Tuple.from(Versionstamp.incomplete(localVersion)).addAll(key); + } else { + return key; + } + } + private Tuple toCompleteKey(Tuple key, byte[] versionStamp, int localVersion, boolean versionInKey) { if (versionInKey) { return Tuple.from(Versionstamp.complete(versionStamp, localVersion)).addAll(key); @@ -309,19 +316,18 @@ private SplitHelper.SizeInfo saveWithSplit(@Nonnull FDBRecordContext context, @N byte[] serialized, @Nullable FDBRecordVersion version, @Nonnull SplitHelperTestConfig testConfig, - @Nullable FDBStoredSizes previousSizeInfo, - int localVersion) { + @Nullable FDBStoredSizes previousSizeInfo) { if (testConfig.omitUnsplitSuffix && version != null) { return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, RecordCoreArgumentException.class, "Cannot include version"); } else if (testConfig.useVersionInKey && version != null) { return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, - RecordCoreException.class, "Split version is not supported for this helper"); + RecordCoreException.class, "Cannot save versionStamp in both key and value"); } else if (!testConfig.splitLongRecords && serialized.length > SplitHelper.SPLIT_RECORD_SIZE) { return saveUnsuccessfully(context, key, serialized, version, testConfig, previousSizeInfo, RecordCoreException.class, "Record is too long"); } else { - return saveOnly(context, key, serialized, version, testConfig, previousSizeInfo, localVersion); + return saveOnly(context, key, serialized, version, testConfig, previousSizeInfo); } } @@ -344,24 +350,21 @@ void saveWithSplit(@Nonnull SplitHelperTestConfig testConfig) { try (FDBRecordContext context = openContext()) { // save with no version and no previousSizeInfo localVersion1 = context.claimLocalVersion(); - sizes1 = saveWithSplit(context, key1, SHORT_STRING, null, testConfig, null, localVersion1); + sizes1 = saveWithSplit(context, toIncompleteKey(key1, localVersion1, testConfig.useVersionInKey), SHORT_STRING, null, testConfig, null); localVersion2 = context.claimLocalVersion(); - sizes2 = saveWithSplit(context, key2, LONG_STRING, null, testConfig, null, localVersion2); + sizes2 = saveWithSplit(context, toIncompleteKey(key2, localVersion2, testConfig.useVersionInKey), LONG_STRING, null, testConfig, null); localVersion3 = context.claimLocalVersion(); - sizes3 = saveWithSplit(context, key3, VERY_LONG_STRING, null, testConfig, null, localVersion3); + sizes3 = saveWithSplit(context, toIncompleteKey(key3, localVersion3, testConfig.useVersionInKey), VERY_LONG_STRING, null, testConfig, null); commit(context); globalVersionstamp = context.getVersionStamp(); } - final Tuple verifyKey1 = toCompleteKey(key1, globalVersionstamp, localVersion1, testConfig.useVersionInKey); - final Tuple verifyKey2 = toCompleteKey(key2, globalVersionstamp, localVersion2, testConfig.useVersionInKey); - final Tuple verifyKey3 = toCompleteKey(key3, globalVersionstamp, localVersion3, testConfig.useVersionInKey); // Transaction #2: verify saved values try (FDBRecordContext context = openContext()) { - verifySuccessfullySaved(context, verifyKey1, SHORT_STRING, null, testConfig); - verifySuccessfullySaved(context, verifyKey2, LONG_STRING, null, testConfig); - verifySuccessfullySaved(context, verifyKey3, VERY_LONG_STRING, null, testConfig); + verifySuccessfullySaved(context, toCompleteKey(key1, globalVersionstamp, localVersion1, testConfig.useVersionInKey), SHORT_STRING, null, testConfig); + verifySuccessfullySaved(context, toCompleteKey(key2, globalVersionstamp, localVersion2, testConfig.useVersionInKey), LONG_STRING, null, testConfig); + verifySuccessfullySaved(context, toCompleteKey(key3, globalVersionstamp, localVersion3, testConfig.useVersionInKey), VERY_LONG_STRING, null, testConfig); } int localVersion4 = 0; @@ -374,25 +377,22 @@ void saveWithSplit(@Nonnull SplitHelperTestConfig testConfig) { // Save over some things using the previous split points if (testConfig.splitLongRecords) { localVersion4 = context.claimLocalVersion(); - saveWithSplit(context, key1, VERY_LONG_STRING, null, testConfig, sizes1, localVersion4); + saveWithSplit(context, toIncompleteKey(key1, localVersion4, testConfig.useVersionInKey), VERY_LONG_STRING, null, testConfig, sizes1); localVersion5 = context.claimLocalVersion(); - saveWithSplit(context, key3, LONG_STRING, null, testConfig, sizes3, localVersion5); + saveWithSplit(context, toIncompleteKey(key3, localVersion5, testConfig.useVersionInKey), LONG_STRING, null, testConfig, sizes3); } localVersion6 = context.claimLocalVersion(); - saveWithSplit(context, key2, SHORT_STRING, null, testConfig, sizes2, localVersion6); + saveWithSplit(context, toIncompleteKey(key2, localVersion6, testConfig.useVersionInKey), SHORT_STRING, null, testConfig, sizes2); commit(context); globalVersionstamp = context.getVersionStamp(); } - Tuple verifyKey4 = toCompleteKey(key1, globalVersionstamp, localVersion4, testConfig.useVersionInKey); - Tuple verifyKey5 = toCompleteKey(key3, globalVersionstamp, localVersion5, testConfig.useVersionInKey); - Tuple verifyKey6 = toCompleteKey(key2, globalVersionstamp, localVersion6, testConfig.useVersionInKey); // Transaction #4: verify changes try (FDBRecordContext context = openContext()) { if (testConfig.splitLongRecords) { - verifySuccessfullySaved(context, verifyKey4, VERY_LONG_STRING, null, testConfig); - verifySuccessfullySaved(context, verifyKey5, LONG_STRING, null, testConfig); + verifySuccessfullySaved(context, toCompleteKey(key1, globalVersionstamp, localVersion4, testConfig.useVersionInKey), VERY_LONG_STRING, null, testConfig); + verifySuccessfullySaved(context, toCompleteKey(key3, globalVersionstamp, localVersion5, testConfig.useVersionInKey), LONG_STRING, null, testConfig); } - verifySuccessfullySaved(context, verifyKey6, SHORT_STRING, null, testConfig); + verifySuccessfullySaved(context, toCompleteKey(key2, globalVersionstamp, localVersion6, testConfig.useVersionInKey), SHORT_STRING, null, testConfig); } } } @@ -411,26 +411,23 @@ public void saveWithSplitAndCompleteVersions(SplitHelperTestConfig testConfig) { int localVersion3; // save records with version in the value - saveInitialRecords(testConfig, globalValueVersion, key1, key2, key3); + saveInitialRecords(testConfig, globalValueVersion, key1, key2, key3, testConfig.useVersionInKey); // Save over the records *without* using the previous size info and with no version in the value try (FDBRecordContext context = openContext()) { localVersion1 = context.claimLocalVersion(); - saveWithSplit(context, key1, SHORT_STRING, null, testConfig, null, localVersion1); + saveWithSplit(context, toIncompleteKey(key1, localVersion1, testConfig.useVersionInKey), SHORT_STRING, null, testConfig, null); localVersion2 = context.claimLocalVersion(); - saveWithSplit(context, key2, LONG_STRING, null, testConfig, null, localVersion2); + saveWithSplit(context, toIncompleteKey(key2, localVersion2, testConfig.useVersionInKey), LONG_STRING, null, testConfig, null); localVersion3 = context.claimLocalVersion(); - saveWithSplit(context, key3, VERY_LONG_STRING, null, testConfig, null, localVersion3); + saveWithSplit(context, toIncompleteKey(key3, localVersion3, testConfig.useVersionInKey), VERY_LONG_STRING, null, testConfig, null); commit(context); globalKeyVersion = context.getVersionStamp(); } - Tuple completeKey1 = toCompleteKey(key1, globalKeyVersion, localVersion1, testConfig.useVersionInKey); - Tuple completeKey2 = toCompleteKey(key2, globalKeyVersion, localVersion2, testConfig.useVersionInKey); - Tuple completeKey3 = toCompleteKey(key3, globalKeyVersion, localVersion3, testConfig.useVersionInKey); try (FDBRecordContext context = openContext()) { - verifySuccessfullySaved(context, completeKey1, SHORT_STRING, null, testConfig); - verifySuccessfullySaved(context, completeKey2, LONG_STRING, null, testConfig); - verifySuccessfullySaved(context, completeKey3, VERY_LONG_STRING, null, testConfig); + verifySuccessfullySaved(context, toCompleteKey(key1, globalKeyVersion, localVersion1, testConfig.useVersionInKey), SHORT_STRING, null, testConfig); + verifySuccessfullySaved(context, toCompleteKey(key2, globalKeyVersion, localVersion2, testConfig.useVersionInKey), LONG_STRING, null, testConfig); + verifySuccessfullySaved(context, toCompleteKey(key3, globalKeyVersion, localVersion3, testConfig.useVersionInKey), VERY_LONG_STRING, null, testConfig); commit(context); } @@ -444,26 +441,23 @@ public void saveWithSplitAndCompleteVersions(SplitHelperTestConfig testConfig) { try (FDBRecordContext context = openContext()) { localVersion1 = context.claimLocalVersion(); version4 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); - sizes4 = saveWithSplit(context, key1, SHORT_STRING, version4, testConfig, null, localVersion1); + sizes4 = saveWithSplit(context, toIncompleteKey(key1, localVersion1, testConfig.useVersionInKey), SHORT_STRING, version4, testConfig, null); localVersion2 = context.claimLocalVersion(); version5 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); - sizes5 = saveWithSplit(context, key2, LONG_STRING, version5, testConfig, null, localVersion2); + sizes5 = saveWithSplit(context, toIncompleteKey(key2, localVersion2, testConfig.useVersionInKey), LONG_STRING, version5, testConfig, null); localVersion3 = context.claimLocalVersion(); version6 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); - sizes6 = saveWithSplit(context, key3, VERY_LONG_STRING, version6, testConfig, null, localVersion3); + sizes6 = saveWithSplit(context, toIncompleteKey(key3, localVersion3, testConfig.useVersionInKey), VERY_LONG_STRING, version6, testConfig, null); commit(context); globalKeyVersion = context.getVersionStamp(); } // in some cases nothing gets written (all saves are "saveUnsuccessfully) so the transaction is read-only // and there is no global version if (globalKeyVersion != null) { - completeKey1 = toCompleteKey(key1, globalKeyVersion, localVersion1, testConfig.useVersionInKey); - completeKey2 = toCompleteKey(key2, globalKeyVersion, localVersion2, testConfig.useVersionInKey); - completeKey3 = toCompleteKey(key3, globalKeyVersion, localVersion3, testConfig.useVersionInKey); try (FDBRecordContext context = openContext()) { - verifySuccessfullySaved(context, completeKey1, SHORT_STRING, version4, testConfig); - verifySuccessfullySaved(context, completeKey2, LONG_STRING, version5, testConfig); - verifySuccessfullySaved(context, completeKey3, VERY_LONG_STRING, version6, testConfig); + verifySuccessfullySaved(context, toCompleteKey(key1, globalKeyVersion, localVersion1, testConfig.useVersionInKey), SHORT_STRING, version4, testConfig); + verifySuccessfullySaved(context, toCompleteKey(key2, globalKeyVersion, localVersion2, testConfig.useVersionInKey), LONG_STRING, version5, testConfig); + verifySuccessfullySaved(context, toCompleteKey(key3, globalKeyVersion, localVersion3, testConfig.useVersionInKey), VERY_LONG_STRING, version6, testConfig); commit(context); } } @@ -471,30 +465,24 @@ public void saveWithSplitAndCompleteVersions(SplitHelperTestConfig testConfig) { // Save over the records *with* using the previous size info try (FDBRecordContext context = openContext()) { localVersion1 = context.claimLocalVersion(); - saveWithSplit(context, key1, SHORT_STRING, null, testConfig, sizes4, localVersion1); + saveWithSplit(context, toIncompleteKey(key1, localVersion1, testConfig.useVersionInKey), SHORT_STRING, null, testConfig, sizes4); localVersion2 = context.claimLocalVersion(); - saveWithSplit(context, key2, LONG_STRING, null, testConfig, sizes5, localVersion2); + saveWithSplit(context, toIncompleteKey(key2, localVersion2, testConfig.useVersionInKey), LONG_STRING, null, testConfig, sizes5); localVersion3 = context.claimLocalVersion(); - saveWithSplit(context, key3, VERY_LONG_STRING, null, testConfig, sizes6, localVersion3); + saveWithSplit(context, toIncompleteKey(key3, localVersion3, testConfig.useVersionInKey), VERY_LONG_STRING, null, testConfig, sizes6); commit(context); globalKeyVersion = context.getVersionStamp(); } - completeKey1 = toCompleteKey(key1, globalKeyVersion, localVersion1, testConfig.useVersionInKey); - completeKey2 = toCompleteKey(key2, globalKeyVersion, localVersion2, testConfig.useVersionInKey); - completeKey3 = toCompleteKey(key3, globalKeyVersion, localVersion3, testConfig.useVersionInKey); try (FDBRecordContext context = openContext()) { - verifySuccessfullySaved(context, completeKey1, SHORT_STRING, null, testConfig); - verifySuccessfullySaved(context, completeKey2, LONG_STRING, null, testConfig); - verifySuccessfullySaved(context, completeKey3, VERY_LONG_STRING, null, testConfig); + verifySuccessfullySaved(context, toCompleteKey(key1, globalKeyVersion, localVersion1, testConfig.useVersionInKey), SHORT_STRING, null, testConfig); + verifySuccessfullySaved(context, toCompleteKey(key2, globalKeyVersion, localVersion2, testConfig.useVersionInKey), LONG_STRING, null, testConfig); + verifySuccessfullySaved(context, toCompleteKey(key3, globalKeyVersion, localVersion3, testConfig.useVersionInKey), VERY_LONG_STRING, null, testConfig); commit(context); } } - private void saveInitialRecords(final SplitHelperTestConfig testConfig, final byte[] globalValueVersion, final Tuple key1, final Tuple key2, final Tuple key3) { + private void saveInitialRecords(final SplitHelperTestConfig testConfig, final byte[] globalValueVersion, final Tuple key1, final Tuple key2, final Tuple key3, final boolean versionInKey) { byte[] globalKeyVersion; - Tuple completeKey1; - Tuple completeKey2; - Tuple completeKey3; int localVersion1; int localVersion2; int localVersion3; @@ -506,26 +494,23 @@ private void saveInitialRecords(final SplitHelperTestConfig testConfig, final by try (FDBRecordContext context = openContext()) { version1 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); localVersion1 = context.claimLocalVersion(); - saveWithSplit(context, key1, SHORT_STRING, version1, testConfig, null, localVersion1); + saveWithSplit(context, toIncompleteKey(key1, localVersion1, versionInKey), SHORT_STRING, version1, testConfig, null); localVersion2 = context.claimLocalVersion(); version2 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); - saveWithSplit(context, key2, LONG_STRING, version2, testConfig, null, localVersion2); + saveWithSplit(context, toIncompleteKey(key2, localVersion2, versionInKey), LONG_STRING, version2, testConfig, null); localVersion3 = context.claimLocalVersion(); version3 = FDBRecordVersion.complete(globalValueVersion, context.claimLocalVersion()); - saveWithSplit(context, key3, VERY_LONG_STRING, version3, testConfig, null, localVersion3); + saveWithSplit(context, toIncompleteKey(key3, localVersion3, versionInKey), VERY_LONG_STRING, version3, testConfig, null); commit(context); globalKeyVersion = context.getVersionStamp(); } // in some cases nothing gets written (all saves are "saveUnsuccessfully") so the transaction is read-only // and there is no global version if (globalKeyVersion != null) { - completeKey1 = toCompleteKey(key1, globalKeyVersion, localVersion1, testConfig.useVersionInKey); - completeKey2 = toCompleteKey(key2, globalKeyVersion, localVersion2, testConfig.useVersionInKey); - completeKey3 = toCompleteKey(key3, globalKeyVersion, localVersion3, testConfig.useVersionInKey); try (FDBRecordContext context = openContext()) { - verifySuccessfullySaved(context, completeKey1, SHORT_STRING, version1, testConfig); - verifySuccessfullySaved(context, completeKey2, LONG_STRING, version2, testConfig); - verifySuccessfullySaved(context, completeKey3, VERY_LONG_STRING, version3, testConfig); + verifySuccessfullySaved(context, toCompleteKey(key1, globalKeyVersion, localVersion1, testConfig.useVersionInKey), SHORT_STRING, version1, testConfig); + verifySuccessfullySaved(context, toCompleteKey(key2, globalKeyVersion, localVersion2, testConfig.useVersionInKey), LONG_STRING, version2, testConfig); + verifySuccessfullySaved(context, toCompleteKey(key3, globalKeyVersion, localVersion3, testConfig.useVersionInKey), VERY_LONG_STRING, version3, testConfig); } } } @@ -568,8 +553,9 @@ private FDBStoredSizes writeDummyRecord(@Nonnull FDBRecordContext context, @Nonn private void writeDummyKV(@Nonnull FDBRecordContext context, @Nonnull Tuple keyTuple, byte[] valueBytes, @Nullable SplitHelper.SizeInfo sizeInfo, boolean useVersionInKey, int localVersion) { + byte[] keyBytes; - // Mimic the work done in both SplitKeyValueHelper + // Mimic the work done in SplitHelper.writeSplitValue if (useVersionInKey) { Tuple versionedKeyTuple = Tuple.from(Versionstamp.incomplete(localVersion)).addAll(keyTuple); keyBytes = subspace.packWithVersionstamp(versionedKeyTuple); @@ -586,13 +572,22 @@ private void writeDummyKV(@Nonnull FDBRecordContext context, @Nonnull Tuple keyT private void deleteSplit(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @Nonnull SplitHelperTestConfig testConfig, @Nullable FDBStoredSizes sizeInfo) { - SplitHelper.deleteSplit(context, subspace, key, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, sizeInfo != null, sizeInfo); int count = KeyValueCursor.Builder.withSubspace(subspace.subspace(key)) .setContext(context) .setScanProperties(ScanProperties.FORWARD_SCAN) .build() .getCount() .join(); + assertTrue(count > 0); + + SplitHelper.deleteSplit(context, subspace, key, testConfig.splitLongRecords, testConfig.omitUnsplitSuffix, sizeInfo != null, sizeInfo); + + count = KeyValueCursor.Builder.withSubspace(subspace.subspace(key)) + .setContext(context) + .setScanProperties(ScanProperties.FORWARD_SCAN) + .build() + .getCount() + .join(); assertEquals(0, count); } @@ -631,18 +626,13 @@ public void deleteWithSplit(@Nonnull SplitHelperTestConfig testConfig) { globalVersionStamp = context.getVersionStamp(); } - final Tuple deleteKey1 = toCompleteKey(key1, globalVersionStamp, localVersion1, testConfig.useVersionInKey); - final Tuple deleteKey2 = toCompleteKey(key2, globalVersionStamp, localVersion2, testConfig.useVersionInKey); - final Tuple deleteKey3 = toCompleteKey(key3, globalVersionStamp, localVersion3, testConfig.useVersionInKey); - final Tuple deleteKey4 = toCompleteKey(key4, globalVersionStamp, localVersion4, testConfig.useVersionInKey); - // tx2: delete records try (FDBRecordContext context = openContext()) { - deleteSplit(context, deleteKey1, testConfig, sizes1); - deleteSplit(context, deleteKey2, testConfig, sizes2); + deleteSplit(context, toCompleteKey(key1, globalVersionStamp, localVersion1, testConfig.useVersionInKey), testConfig, sizes1); + deleteSplit(context, toCompleteKey(key2, globalVersionStamp, localVersion2, testConfig.useVersionInKey), testConfig, sizes2); if (testConfig.splitLongRecords) { - deleteSplit(context, deleteKey3, testConfig, sizes3); - deleteSplit(context, deleteKey4, testConfig, sizes4); + deleteSplit(context, toCompleteKey(key3, globalVersionStamp, localVersion3, testConfig.useVersionInKey), testConfig, sizes3); + deleteSplit(context, toCompleteKey(key4, globalVersionStamp, localVersion4, testConfig.useVersionInKey), testConfig, sizes4); } commit(context); } @@ -1134,21 +1124,18 @@ void scanContinuations(final int returnLimit, final int readLimit, final boolean */ @Test void saveWithSplitVersionInKeyOverwriteInTransaction() { - final Tuple key = Tuple.from(1066L); - final int localVersion; try (FDBRecordContext context = openContext()) { - localVersion = context.claimLocalVersion(); + final int localVersion = context.claimLocalVersion(); // First write: VERY_LONG_STRING requires multiple splits - final VersioningSplitKeyValueHelper splitKeyHelper = new VersioningSplitKeyValueHelper(Versionstamp.incomplete(localVersion)); + final Tuple key = Tuple.from(Versionstamp.incomplete(localVersion)).add(1066); SplitHelper.saveWithSplit(context, subspace, key, VERY_LONG_STRING, null, true, false, - splitKeyHelper, false, null, null); // Second write: LONG_STRING — same localVersion, same key, shorter value (fewer splits) - final RecordCoreInternalException ex = assertThrows(RecordCoreInternalException.class, () -> SplitHelper.saveWithSplit(context, subspace, key, LONG_STRING, null, + final RecordCoreInternalException ex = assertThrows(RecordCoreInternalException.class, () -> + SplitHelper.saveWithSplit(context, subspace, key, LONG_STRING, null, true, false, - splitKeyHelper, false, null, null)); assertTrue(ex.getMessage().contains("Key with version overwritten")); } diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java index 18bffadab3..cd4bfe917e 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java @@ -347,7 +347,7 @@ private void enqueueOperationInternal( SplitKeyValueHelper keyHelper = new VersioningSplitKeyValueHelper(recordVersion.toVersionstamp()); byte[] value = serializer.encode(builder.build().toByteArray()); // save with splits - SplitHelper.saveWithSplit(context, queueSubspace, TupleHelpers.EMPTY, value, null, true, false, keyHelper, false, null, null); + SplitHelper.saveWithSplit(context, queueSubspace, TupleHelpers.EMPTY, value, null, true, false, false, null, null); // Atomically increment the queue size counter mutateQueueSizeCounter(context, 1); From 452420800fe646354613444254fdfcee06d5261d Mon Sep 17 00:00:00 2001 From: ohad Date: Mon, 9 Mar 2026 11:18:21 -0400 Subject: [PATCH 26/29] More refactoring, add asserts for key size in tests --- .../provider/foundationdb/SplitHelper.java | 48 ++++----- .../VersioningSplitKeyValueHelper.java | 100 ------------------ .../SplitHelperMultipleTransactionsTest.java | 20 ++-- .../foundationdb/SplitHelperTestConfig.java | 8 -- .../lucene/directory/PendingWriteQueue.java | 9 +- 5 files changed, 34 insertions(+), 151 deletions(-) delete mode 100644 fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java index a190432031..e7d223d640 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java @@ -108,7 +108,6 @@ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnu saveWithSplit(context, subspace, key, serialized, version, true, false, false, null, null); } - /** * Save serialized representation using multiple keys if necessary, clearing only as much as needed. * @param context write transaction @@ -161,7 +160,7 @@ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnu } else { recordKey = key; } - byte[] keyBytes = writeSplitValue(context, subspace, recordKey, serialized, sizeInfo); + final byte[] keyBytes = writeSplitValue(context, subspace, recordKey, serialized); if (sizeInfo != null) { sizeInfo.set(keyBytes, serialized); sizeInfo.setSplit(false); @@ -170,27 +169,6 @@ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnu writeVersion(context, subspace, key, version, sizeInfo); } - private static byte[] writeSplitValue(FDBRecordContext context, Subspace subspace, Tuple recordKey, byte[] serialized, @Nullable SizeInfo sizeInfo) { - byte[] keyBytes; - if (recordKey.hasIncompleteVersionstamp()) { - keyBytes = subspace.packWithVersionstamp(recordKey); - byte[] current = context.addVersionMutation( - MutationType.SET_VERSIONSTAMPED_KEY, - keyBytes, - serialized); - if (current != null) { - // This should never happen. It means that the same key (and suffix) and local version were used for subsequent - // write. It is most likely not an intended flow and this check will protect against that. - // It is an incomplete check since the same record can have different suffixes to the primary key that would not collide. - throw new RecordCoreInternalException("Key with version overwritten"); - } - } else { - keyBytes = subspace.pack(recordKey); - context.ensureActive().set(keyBytes, serialized); - } - return keyBytes; - } - @SuppressWarnings("PMD.CloseResource") private static void writeSplitRecord(@Nonnull final FDBRecordContext context, @Nonnull final Subspace subspace, @Nonnull final Tuple key, @Nonnull final byte[] serialized, @@ -210,7 +188,7 @@ private static void writeSplitRecord(@Nonnull final FDBRecordContext context, @N nextOffset = serialized.length; } final byte[] valueBytes = Arrays.copyOfRange(serialized, offset, nextOffset); - byte[] keyBytes = writeSplitValue(context, subspace, key.add(index), valueBytes, sizeInfo); + final byte[] keyBytes = writeSplitValue(context, subspace, key.add(index), valueBytes); if (sizeInfo != null) { if (offset == 0) { sizeInfo.set(keyBytes, valueBytes); @@ -224,6 +202,27 @@ private static void writeSplitRecord(@Nonnull final FDBRecordContext context, @N } } + private static byte[] writeSplitValue(FDBRecordContext context, Subspace subspace, Tuple recordKey, byte[] serialized) { + byte[] keyBytes; + if (recordKey.hasIncompleteVersionstamp()) { + keyBytes = subspace.packWithVersionstamp(recordKey); + byte[] current = context.addVersionMutation( + MutationType.SET_VERSIONSTAMPED_KEY, + keyBytes, + serialized); + if (current != null) { + // This should never happen. It means that the same key (and suffix) and local version were used for subsequent + // write. It is most likely not an intended flow and this check will protect against that. + // It is an incomplete check since the same record can have different suffixes to the primary key that would not collide. + throw new RecordCoreInternalException("Key with version overwritten"); + } + } else { + keyBytes = subspace.pack(recordKey); + context.ensureActive().set(keyBytes, serialized); + } + return keyBytes; + } + @SuppressWarnings("PMD.CloseResource") private static void writeVersion(@Nonnull final FDBRecordContext context, @Nonnull final Subspace subspace, @Nonnull final Tuple key, @Nullable final FDBRecordVersion version, @Nullable final SizeInfo sizeInfo) { @@ -234,7 +233,6 @@ private static void writeVersion(@Nonnull final FDBRecordContext context, @Nonnu return; } // At this point we know the key does not have a version - // TODO: Create a flavor of writeSplit for this case final byte[] keyBytes = subspace.pack(key.add(RECORD_VERSION)); final byte[] valueBytes = packVersion(version); if (version.isComplete()) { diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java deleted file mode 100644 index 0af6e1bc63..0000000000 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/VersioningSplitKeyValueHelper.java +++ /dev/null @@ -1,100 +0,0 @@ -/* - * VersioningSplitKeyValueHelper.java - * - * This source file is part of the FoundationDB open source project - * - * Copyright 2015-2026 Apple Inc. and the FoundationDB project authors - * - * Licensed 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 com.apple.foundationdb.record.provider.foundationdb; - -import com.apple.foundationdb.MutationType; -import com.apple.foundationdb.annotation.API; -import com.apple.foundationdb.record.RecordCoreInternalException; -import com.apple.foundationdb.subspace.Subspace; -import com.apple.foundationdb.tuple.Tuple; -import com.apple.foundationdb.tuple.Versionstamp; - -/** - * A {@link SplitKeyValueHelper} that is used when the Key contains a {@link Versionstamp}. - *

      This implementation should be used when the key contains a {@link Versionstamp}, as it will ensure that the proper FDB APIs - * encode and decode the key correctly.

      - *

      This class is stateful (has a single {@link Versionstamp}) that is going to be used for all splits of a single K/V, - * so that all contain the same fixed part and can be correlated after the commit.

      - *

      The resulting FDB key looks like:

      - *
      - *     [versionstamp, original-key, split-suffix]
      - * 
      - *

      which means that the entries are sorted by their insertion order (versionstamp order), then by the - * original key, with split fragments last.

      - * - * NOTE: Since this class uses {@link Versionstamp} in the FDB Key, it does not know the actual key used until after the commit. - * As a result, it cannot override previous values saved within the same transaction. Split records saved using this feature - * are not stored in the RYW cache and attempts to write over them (using the same local version and PK) before the commit - * may result in corrupt data. - * Do not override records with another write using the same local version and PK. - */ -@API(API.Status.INTERNAL) -public class VersioningSplitKeyValueHelper implements SplitKeyValueHelper { - private Versionstamp versionstamp; - - public VersioningSplitKeyValueHelper(final Versionstamp versionstamp) { - this.versionstamp = versionstamp; - } - - /** - * No need to clear subspace. - * Since the key has a unique component (version), no conflicts are expected, so no need to clean before saving new splits. - * Furthermore, since the key contains a {@link Versionstamp}, we don't know the actual key contents ahead of committing - * the transaction, and so no clean can be done. - * @return {@code false}, as new keys should not interfere with old ones. - */ - @Override - public boolean shouldClearBeforeWrite() { - return false; - } - - /** - * Since the key has versions, prevent the values from having them. - * @return {@code false}, since only keys or values are allowed to mutate in FDB, and this mutates the keys - */ - @Override - public boolean supportsVersionInValue() { - return false; - } - - @Override - public byte[] packSplitKey(final Subspace subspace, final Tuple key) { - // This uses the same version (local and global) for all the splits - // Use versionstamp first to ensure proper sorting and since split suffix should be at the end - Tuple keyTuple = Tuple.from(versionstamp).addAll(key); - return subspace.packWithVersionstamp(keyTuple); - } - - @Override - public void writeSplit(final FDBRecordContext context, final byte[] keyBytes, final byte[] valueBytes) { - final byte[] current = context.addVersionMutation( - MutationType.SET_VERSIONSTAMPED_KEY, - keyBytes, - valueBytes); - - if (current != null) { - // This should never happen. It means that the same key (and suffix) and local version were used for subsequent - // write. It is most likely not an intended flow and this check will protect against that. - // It is an incomplete check since the same record can have different suffixes to the primary key that would not collide. - throw new RecordCoreInternalException("Key with version overwritten"); - } - } -} diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java index 9019b5c379..8d07121899 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java @@ -806,7 +806,7 @@ private void verifyRecords(final SplitHelperTestConfig testConfig, final @Nonnul @Nullable private FDBRawRecord loadWithSplit(@Nonnull FDBRecordContext context, @Nonnull Tuple key, @Nonnull SplitHelperTestConfig testConfig, - @Nullable FDBStoredSizes expectedSizes, @Nullable byte[] expectedContents, @Nullable FDBRecordVersion expectedVersion) { + @Nullable FDBStoredSizes expectedSizes, @Nullable byte[] expectedContents, @Nullable FDBRecordVersion expectedVersion, boolean useVersionInKey) { final ReadTransaction tr = context.ensureActive(); SplitHelper.SizeInfo sizeInfo = new SplitHelper.SizeInfo(); FDBRawRecord rawRecord; @@ -854,12 +854,9 @@ private FDBRawRecord loadWithSplit(@Nonnull FDBRecordContext context, @Nonnull T assertEquals(rawRecord.getValueSize(), sizeInfo.getValueSize()); assertEquals(rawRecord.isSplit(), sizeInfo.isSplit()); assertEquals(rawRecord.isVersionedInline(), sizeInfo.isVersionedInline()); - - // Do not attempt to compare key sizes if the keys contain incomplete version stamps - if (!testConfig.useVersionInKey) { - assertEquals(expectedSizes.getKeySize(), rawRecord.getKeySize()); - assertEquals(rawRecord.getKeySize(), sizeInfo.getKeySize()); - } + int keySizeAdjustment = useVersionInKey ? -4 * expectedSizes.getKeyCount() : 0; // 4 bytes diff between complete and incomplete versionStamp + assertEquals(expectedSizes.getKeySize() + keySizeAdjustment, rawRecord.getKeySize()); + assertEquals(rawRecord.getKeySize(), sizeInfo.getKeySize()); } return rawRecord; @@ -871,7 +868,7 @@ public void loadWithSplit(SplitHelperTestConfig testConfig) { this.testConfig = testConfig; loadSingleRecords(testConfig, (context, key, expectedSizes, expectedContents, version) -> - loadWithSplit(context, key, testConfig, expectedSizes, expectedContents, version)); + loadWithSplit(context, key, testConfig, expectedSizes, expectedContents, version, testConfig.useVersionInKey)); if (testConfig.splitLongRecords) { final Tuple key = Tuple.from(1307L); @@ -890,7 +887,7 @@ public void loadWithSplit(SplitHelperTestConfig testConfig) { Tuple completeKey = toCompleteKey(key, globalVersion, localVersion, testConfig.useVersionInKey); try (FDBRecordContext context = openContext()) { RecordCoreException err = assertThrows(RecordCoreException.class, - () -> loadWithSplit(context, completeKey, testConfig, null, null, null)); + () -> loadWithSplit(context, completeKey, testConfig, null, null, null, testConfig.useVersionInKey)); assertThat(err.getMessage(), containsString("Unsplit value followed by split")); } } @@ -928,9 +925,8 @@ private FDBRawRecord scanSingleRecord(@Nonnull FDBRecordContext context, boolean boolean isSplit = rawRecord.getKeyCount() - (rawRecord.isVersionedInline() ? 1 : 0) != 1; assertEquals(rawRecord.getKeyCount() - (rawRecord.isVersionedInline() ? 1 : 0) != 1, expectedSizes.isSplit()); assertEquals(version != null, expectedSizes.isVersionedInline()); - if (!useVersionInKey) { - assertEquals(expectedSizes.getKeySize(), rawRecord.getKeySize()); - } + int keySizeAdjustment = useVersionInKey ? -4 * expectedSizes.getKeyCount() : 0; // 4 bytes diff between complete and incomplete versionStamp + assertEquals(expectedSizes.getKeySize() + keySizeAdjustment, rawRecord.getKeySize()); return rawRecord; } } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTestConfig.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTestConfig.java index 470a9a9032..4305427bf0 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTestConfig.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTestConfig.java @@ -46,14 +46,6 @@ public SplitHelperTestConfig(boolean splitLongRecords, boolean omitUnsplitSuffix this.useVersionInKey = useVersionInKey; } - public SplitKeyValueHelper keyHelper(int localVersion) { - if (useVersionInKey) { - return new VersioningSplitKeyValueHelper(Versionstamp.incomplete(localVersion)); - } else { - return DefaultSplitKeyValueHelper.INSTANCE; - } - } - @Nonnull public RecordLayerPropertyStorage.Builder setProps(@Nonnull RecordLayerPropertyStorage.Builder props) { return props diff --git a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java index cd4bfe917e..6bdb550b0e 100644 --- a/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java +++ b/fdb-record-layer-lucene/src/main/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueue.java @@ -43,11 +43,8 @@ import com.apple.foundationdb.record.provider.foundationdb.FDBRecordVersion; import com.apple.foundationdb.record.provider.foundationdb.KeyValueCursor; import com.apple.foundationdb.record.provider.foundationdb.SplitHelper; -import com.apple.foundationdb.record.provider.foundationdb.SplitKeyValueHelper; -import com.apple.foundationdb.record.provider.foundationdb.VersioningSplitKeyValueHelper; import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.Tuple; -import com.apple.foundationdb.tuple.TupleHelpers; import com.apple.foundationdb.tuple.Versionstamp; import com.google.protobuf.ByteString; import org.apache.lucene.index.IndexWriter; @@ -343,11 +340,11 @@ private void enqueueOperationInternal( // Build key with incomplete versionStamp with a new local version FDBRecordVersion recordVersion = FDBRecordVersion.incomplete(context.claimLocalVersion()); - // Use the version in the key helper for all splits of the same entry - SplitKeyValueHelper keyHelper = new VersioningSplitKeyValueHelper(recordVersion.toVersionstamp()); + // key contains only an incomplete version + Tuple keyTuple = Tuple.from(recordVersion.toVersionstamp()); byte[] value = serializer.encode(builder.build().toByteArray()); // save with splits - SplitHelper.saveWithSplit(context, queueSubspace, TupleHelpers.EMPTY, value, null, true, false, false, null, null); + SplitHelper.saveWithSplit(context, queueSubspace, keyTuple, value, null, true, false, false, null, null); // Atomically increment the queue size counter mutateQueueSizeCounter(context, 1); From db3155685356e139c1f8542e61e85e3e2498b962 Mon Sep 17 00:00:00 2001 From: ohad Date: Mon, 9 Mar 2026 12:22:42 -0400 Subject: [PATCH 27/29] Removed helper and implementation --- .../DefaultSplitKeyValueHelper.java | 67 ------------------- .../foundationdb/SplitKeyValueHelper.java | 62 ----------------- .../foundationdb/SplitHelperTest.java | 5 +- 3 files changed, 2 insertions(+), 132 deletions(-) delete mode 100644 fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/DefaultSplitKeyValueHelper.java delete mode 100644 fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyValueHelper.java diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/DefaultSplitKeyValueHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/DefaultSplitKeyValueHelper.java deleted file mode 100644 index 7258fbf439..0000000000 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/DefaultSplitKeyValueHelper.java +++ /dev/null @@ -1,67 +0,0 @@ -/* - * DefaultSplitKeyValueHelper.java - * - * This source file is part of the FoundationDB open source project - * - * Copyright 2015-2026 Apple Inc. and the FoundationDB project authors - * - * Licensed 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 com.apple.foundationdb.record.provider.foundationdb; - -import com.apple.foundationdb.Transaction; -import com.apple.foundationdb.annotation.API; -import com.apple.foundationdb.subspace.Subspace; -import com.apple.foundationdb.tuple.Tuple; - -/** - * The default implementation of the {@link SplitKeyValueHelper}. - * This implementation is using the subspace serialization and transaction set methods. - * This implementation does not support {@link com.apple.foundationdb.tuple.Versionstamp} in the key (as it allows - * one to reside in the value). - */ -@API(API.Status.INTERNAL) -public class DefaultSplitKeyValueHelper implements SplitKeyValueHelper { - public static final DefaultSplitKeyValueHelper INSTANCE = new DefaultSplitKeyValueHelper(); - - /** - * Since this is setting keys directly, there is a chance that existing keys will interfere with the new keys. - * @return true to ensure all prior keys are cleared from the subspace - */ - @Override - public boolean shouldClearBeforeWrite() { - return true; - } - - /** - * Value can mutate version stamp with this helper. - * @return true since the SplitHelper should mutate a version in the value - */ - @Override - public boolean supportsVersionInValue() { - return true; - } - - @Override - public byte[] packSplitKey(final Subspace subspace, final Tuple key) { - return subspace.pack(key); - } - - @Override - @SuppressWarnings("PMD.CloseResource") - public void writeSplit(final FDBRecordContext context, final byte[] keyBytes, final byte[] valueBytes) { - final Transaction tr = context.ensureActive(); - tr.set(keyBytes, valueBytes); - } -} diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyValueHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyValueHelper.java deleted file mode 100644 index 99c8957ce5..0000000000 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitKeyValueHelper.java +++ /dev/null @@ -1,62 +0,0 @@ -/* - * SplitKeyValueHelper.java - * - * This source file is part of the FoundationDB open source project - * - * Copyright 2015-2026 Apple Inc. and the FoundationDB project authors - * - * Licensed 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 com.apple.foundationdb.record.provider.foundationdb; - -import com.apple.foundationdb.annotation.API; -import com.apple.foundationdb.subspace.Subspace; -import com.apple.foundationdb.tuple.Tuple; - -/** - * An interface extracting the generation and persistence of keys used in the {@link SplitHelper}. - */ -@API(API.Status.INTERNAL) -public interface SplitKeyValueHelper { - /** - * Whether the {@link SplitHelper} should clear older entries before writing new ones. - * In the cases where old splits may exist and overlap with the new entries, the SplitHelper should clear the - * subspace before writing any new splits. This method is used to determine whether this is necessary. - * @return {@code true} if the SplitHelper should clear the subspace before writing new splits. - */ - boolean shouldClearBeforeWrite(); - - /** - * Whether the helper allows version mutation in the values. - * There are cases where the value cannot have version mutation, for example, when the key needs to have one. - * @return {@code true} if version mutations are allowed for the value of the k/v pair, false if not - */ - boolean supportsVersionInValue(); - - /** - * Serialize a key to a format that can be saved to FDB. - * @param subspace the subspace to use for the key - * @param key the key Tuple to use for the rest of the key - * @return the serialized form of the key - */ - byte[] packSplitKey(Subspace subspace, Tuple key); - - /** - * Write a key/value pair to FDB. - * @param context the transaction to use for writing - * @param keyBytes the key to use - * @param valueBytes the value to use - */ - void writeSplit(FDBRecordContext context, byte[] keyBytes, byte[] valueBytes); -} diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java index 63f2e1b0f4..9c2a7b4104 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java @@ -900,8 +900,7 @@ public static Stream splitKeyEquivalenceCases() { /** * Verify that the refactored split call site in {@link SplitHelper} — which now passes * {@code subspace} + {@code key.add(index)} to {@code packSplitKey} instead of the old - * {@code subspace.subspace(key)} + {@code Tuple.from(index)} — produces identical byte keys - * for {@link DefaultSplitKeyValueHelper}. + * {@code subspace.subspace(key)} + {@code Tuple.from(index)} — produces identical byte keys. */ @MethodSource("splitKeyEquivalenceCases") @ParameterizedTest(name = "defaultHelperSplitKeyEquivalence[key={0}, index={1}]") @@ -911,7 +910,7 @@ void defaultHelperSplitKeyEquivalence(Tuple key, long index) { // Old call site in SplitHelper: subspace.subspace(key).pack(index) byte[] oldKey = subspace.subspace(key).pack(index); // New call site: packSplitKey(subspace, key.add(index)) = subspace.pack(key.add(index)) - byte[] newKey = DefaultSplitKeyValueHelper.INSTANCE.packSplitKey(subspace, key.add(index)); + byte[] newKey = subspace.pack(key.add(index)); assertArrayEquals(oldKey, newKey); } From 3e68faa330d7c08320e516f1bd101f97c95c0d66 Mon Sep 17 00:00:00 2001 From: ohad Date: Mon, 9 Mar 2026 17:35:45 -0400 Subject: [PATCH 28/29] More test fixes --- .../SplitHelperMultipleTransactionsTest.java | 53 +++++++++++++++++-- .../foundationdb/SplitHelperTestConfig.java | 1 - 2 files changed, 50 insertions(+), 4 deletions(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java index 8d07121899..e45d80d680 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperMultipleTransactionsTest.java @@ -57,6 +57,7 @@ import java.util.Collections; import java.util.Iterator; import java.util.List; +import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; import java.util.stream.LongStream; import java.util.stream.Stream; @@ -70,6 +71,7 @@ import static org.hamcrest.Matchers.oneOf; import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertThrows; @@ -818,6 +820,7 @@ private FDBRawRecord loadWithSplit(@Nonnull FDBRecordContext context, @Nonnull T if (expectedSizes == null || expectedContents == null) { assertNull(rawRecord); + assertEmpty(sizeInfo); } else { assertNotNull(rawRecord); assertArrayEquals(expectedContents, rawRecord.getRawRecord()); @@ -832,7 +835,8 @@ private FDBRawRecord loadWithSplit(@Nonnull FDBRecordContext context, @Nonnull T if (testConfig.omitUnsplitSuffix) { assertThat(rawRecord.isVersionedInline(), is(false)); } - boolean isSplit = rawRecord.getKeyCount() - (expectedVersion != null ? 1 : 0) != 1; + final int versionKeyCount = expectedVersion != null ? 1 : 0; + boolean isSplit = rawRecord.getKeyCount() - versionKeyCount != 1; assertEquals(isSplit, rawRecord.isSplit()); assertEquals(key, rawRecord.getPrimaryKey()); if (expectedVersion != null) { @@ -862,6 +866,14 @@ private FDBRawRecord loadWithSplit(@Nonnull FDBRecordContext context, @Nonnull T return rawRecord; } + private void assertEmpty(final SplitHelper.SizeInfo sizeInfo) { + assertEquals(0, sizeInfo.getKeyCount()); + assertEquals(0, sizeInfo.getKeySize()); + assertEquals(0, sizeInfo.getValueSize()); + assertFalse(sizeInfo.isSplit()); + assertFalse(sizeInfo.isVersionedInline()); + } + @MethodSource("testConfigsNoDryRun") @ParameterizedTest(name = "loadWithSplit[{0}]") public void loadWithSplit(SplitHelperTestConfig testConfig) { @@ -1112,8 +1124,7 @@ void scanContinuations(final int returnLimit, final int readLimit, final boolean /** * When two saveWithSplit calls use the same incomplete versionstamp (same localVersion, same key) within * one transaction, we may get a failure or data corruption. The localVersionCache (map by key) may contain - * previous - * values from an identical key (same versionstamp/localversion/PK) but different splits and may not collide + * previous values from an identical key (same versionstamp/localversion/PK) but different splits and may not collide * directly * with the previous values. This test shows the case where there is a collision since the split numbers are the * same. @@ -1136,4 +1147,40 @@ void saveWithSplitVersionInKeyOverwriteInTransaction() { assertTrue(ex.getMessage().contains("Key with version overwritten")); } } + + /** + * When two saveWithSplit calls use the same incomplete versionstamp (same localVersion, same key) within + * one transaction, we may get a failure or data corruption. The localVersionCache (map by key) may contain + * previous values from an identical key (same versionstamp/localversion/PK) but different splits and may not + * collide + * directly with the previous values. This test shows the data corruption when two record overlap. + */ + @Test + void saveWithSplitVersionInKeyOverlapSplits() { + final byte[] globalVersion; + final int localVersion; + final Tuple key = Tuple.from(1066); + try (FDBRecordContext context = openContext()) { + localVersion = context.claimLocalVersion(); + // First write: VERY_LONG_STRING requires multiple splits + final Tuple incompleteKey = Tuple.from(Versionstamp.incomplete(localVersion)).add(1066); + SplitHelper.saveWithSplit(context, subspace, incompleteKey, VERY_LONG_STRING, null, + true, false, + false, null, null); + + // Second write: SHORT_STRING — same localVersion, same key, shorter value (one split) + SplitHelper.saveWithSplit(context, subspace, incompleteKey, SHORT_STRING, null, + true, false, + false, null, null); + commit(context); + globalVersion = context.getVersionStamp(); + } + // Record is corrupt, as the '0' and '1..n' suffixes did not conflict, and none was in the RYW cache upon writing. + try (FDBRecordContext context = openContext()) { + final Tuple completeKey = toCompleteKey(key, globalVersion, localVersion, true); + Exception ex = assertThrows(CompletionException.class, () -> + SplitHelper.loadWithSplit(context.ensureActive(), context, subspace, completeKey, true, false, null).join()); + assertTrue(ex.getMessage().contains("Unsplit value followed by split")); + } + } } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTestConfig.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTestConfig.java index 4305427bf0..f4bd996153 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTestConfig.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTestConfig.java @@ -22,7 +22,6 @@ import com.apple.foundationdb.record.FDBRecordStoreProperties; import com.apple.foundationdb.record.provider.foundationdb.properties.RecordLayerPropertyStorage; -import com.apple.foundationdb.tuple.Versionstamp; import javax.annotation.Nonnull; import java.util.Objects; From f196baf646858a891482245a6f62369d83726b71 Mon Sep 17 00:00:00 2001 From: ohad Date: Wed, 11 Mar 2026 11:56:36 -0400 Subject: [PATCH 29/29] PR comments --- .../provider/foundationdb/SplitHelper.java | 6 ++-- .../foundationdb/SplitHelperTest.java | 12 +++---- .../directory/PendingWriteQueueTest.java | 32 ++++++++++++------- 3 files changed, 31 insertions(+), 19 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java index e7d223d640..d4861f7e60 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelper.java @@ -147,11 +147,12 @@ public static void saveWithSplit(@Nonnull final FDBRecordContext context, @Nonnu } writeSplitRecord(context, subspace, key, serialized, hasVersionInKey, clearBasedOnPreviousSizeInfo, previousSizeInfo, sizeInfo); } else { - // an incomplete version in the key means we shouldn't delete previous values for the record (since they all have - // completed versions by now) + // An incomplete version in the key means that we shouldn't delete previous k/v pairs using these keys since, + // in the DB, from previous transactions, they would have been completed the versions already (and so wouldn't match) if (!hasVersionInKey && (splitLongRecords || previousSizeInfo == null || previousSizeInfo.isVersionedInline())) { // Note that the clearPreviousSplitRecords also removes version splits from the context cache // This is not currently supported for the case where we have versions in the keys since we can't trace the old values down + // Will be improved on in a separate PR clearPreviousSplitRecord(context, subspace, key, clearBasedOnPreviousSizeInfo, previousSizeInfo); } final Tuple recordKey; @@ -214,6 +215,7 @@ private static byte[] writeSplitValue(FDBRecordContext context, Subspace subspac // This should never happen. It means that the same key (and suffix) and local version were used for subsequent // write. It is most likely not an intended flow and this check will protect against that. // It is an incomplete check since the same record can have different suffixes to the primary key that would not collide. + // Namely, if one is split, and one is not split they will not overlap. Anything else and the would. throw new RecordCoreInternalException("Key with version overwritten"); } } else { diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java index 9c2a7b4104..0b7b964ded 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/SplitHelperTest.java @@ -884,7 +884,7 @@ public void scanContinuations(final int returnLimit, final int readLimit, final } } - public static Stream splitKeyEquivalenceCases() { + public static Stream previousKeyEquivalence() { return Stream.of( Arguments.of(Tuple.from(1066L), SplitHelper.START_SPLIT_RECORD), Arguments.of(Tuple.from(1066L), SplitHelper.START_SPLIT_RECORD + 1), @@ -902,14 +902,14 @@ public static Stream splitKeyEquivalenceCases() { * {@code subspace} + {@code key.add(index)} to {@code packSplitKey} instead of the old * {@code subspace.subspace(key)} + {@code Tuple.from(index)} — produces identical byte keys. */ - @MethodSource("splitKeyEquivalenceCases") - @ParameterizedTest(name = "defaultHelperSplitKeyEquivalence[key={0}, index={1}]") - void defaultHelperSplitKeyEquivalence(Tuple key, long index) { + @MethodSource("previousKeyEquivalence") + @ParameterizedTest(name = "previousKeyEquivalence[key={0}, index={1}]") + void previousKeyEquivalence(Tuple key, long index) { Subspace subspace = new Subspace(Tuple.from("test")); - // Old call site in SplitHelper: subspace.subspace(key).pack(index) + // Old call: subspace.subspace(key).pack(index) byte[] oldKey = subspace.subspace(key).pack(index); - // New call site: packSplitKey(subspace, key.add(index)) = subspace.pack(key.add(index)) + // New call subspace.pack(key.add(index)) byte[] newKey = subspace.pack(key.add(index)); assertArrayEquals(oldKey, newKey); diff --git a/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueueTest.java b/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueueTest.java index 5d4f40b0e9..cb79ca15d9 100644 --- a/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueueTest.java +++ b/fdb-record-layer-lucene/src/test/java/com/apple/foundationdb/record/lucene/directory/PendingWriteQueueTest.java @@ -37,7 +37,9 @@ import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.Tuple; import com.apple.foundationdb.tuple.Versionstamp; -import com.apple.test.BooleanSource; +import com.apple.test.ParameterizedTestUtils; +import com.apple.test.RandomSeedSource; +import com.apple.test.RandomizedTestUtils; import com.apple.test.Tags; import com.google.common.collect.Streams; import org.assertj.core.api.Assertions; @@ -46,7 +48,9 @@ import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; import org.junit.jupiter.params.provider.EnumSource; +import org.junit.jupiter.params.provider.MethodSource; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -56,7 +60,6 @@ import java.util.List; import java.util.Map; import java.util.Random; -import java.util.concurrent.ThreadLocalRandom; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -360,11 +363,17 @@ void testFailToDeserialize() { } } + static Stream seedAndUseCompression() { + return ParameterizedTestUtils.cartesianProduct( + RandomizedTestUtils.randomSeeds(476373L, 7768884L), + ParameterizedTestUtils.booleans("useCompression")); + } + @ParameterizedTest - @BooleanSource("useCompression") - void testLargeQueueItem(boolean useCompression) throws Exception { + @MethodSource("seedAndUseCompression") + void testLargeQueueItem(long seed, boolean useCompression) throws Exception { // Test that we can store large queue items with and without compression - TestDocument docWithHugeString = createHugeDocument(); + TestDocument docWithHugeString = createHugeDocument(new Random(seed)); LuceneSerializer serializerToUse; if (useCompression) { @@ -389,10 +398,11 @@ void testLargeQueueItem(boolean useCompression) throws Exception { } } - @Test - void testLargeQueueItemDelete() { + @ParameterizedTest + @RandomSeedSource({663377L, 7758893L}) + void testLargeQueueItemDelete(long seed) { // A split entry (>100KB) must be fully removed when clearEntry is called - TestDocument docWithHugeString = createHugeDocument(); + TestDocument docWithHugeString = createHugeDocument(new Random(seed)); TestDocument normalDoc = new TestDocument(primaryKey("Normal"), List.of(createField("f", "small", LuceneIndexExpressions.DocumentFieldType.STRING, false, false))); @@ -440,8 +450,7 @@ void testClearEntryWithIncompleteVersionstamp() { } @Nonnull - private TestDocument createHugeDocument() { - Random random = ThreadLocalRandom.current(); + private TestDocument createHugeDocument(Random random) { StringBuilder builder = new StringBuilder(); for (int i = 0 ; i < 500_000 ; i++) { builder.append(CHARACTERS.charAt(random.nextInt(CHARACTERS.length()))); @@ -567,7 +576,8 @@ private List createTestDocuments() { createField("long field", 6L, LuceneIndexExpressions.DocumentFieldType.LONG, true, false), createField("double field", 3.14D, LuceneIndexExpressions.DocumentFieldType.DOUBLE, true, true))); - TestDocument hugeDoc = createHugeDocument(); + Random random = new Random(); + TestDocument hugeDoc = createHugeDocument(random); return List.of(docWithNoFields, docWithOneFields, docWithMultipleFields, hugeDoc, docWithAllFieldTypes); }