From 8f4922e6831c60b540e89f9b4728a0025bc19904 Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Wed, 27 Aug 2025 15:50:16 -0400 Subject: [PATCH 01/32] Add tests for a KeySpacePath export method to export all the data within the path --- .../foundationdb/keyspace/KeySpacePath.java | 26 + .../keyspace/KeySpacePathImpl.java | 9 + .../keyspace/KeySpacePathWrapper.java | 7 + .../keyspace/KeySpacePathDataExportTest.java | 580 ++++++++++++++++++ 4 files changed, 622 insertions(+) create mode 100644 fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java index d52861ec69..db142ca6eb 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java @@ -566,4 +566,30 @@ default List listSubdirectory(@Nonnull FDBRecordContext co */ @API(API.Status.UNSTABLE) String toString(@Nonnull Tuple tuple); + + /** + * Export all data stored under this KeySpacePath and return it in a RecordCursor. + * This method scans all keys that have this path as a prefix and returns the key-value pairs. + * + * @param context the transaction context in which to perform the data export + * @return a RecordCursor that iterates over all KeyValue pairs under this path + */ + @API(API.Status.UNSTABLE) + @Nonnull + default RecordCursor exportAllData(@Nonnull FDBRecordContext context) { + return exportAllData(context, ScanProperties.FORWARD_SCAN); + } + + /** + * Export all data stored under this KeySpacePath and return it in a RecordCursor. + * This method scans all keys that have this path as a prefix and returns the key-value pairs. + * + * @param context the transaction context in which to perform the data export + * @param scanProperties properties controlling how the scan should be performed + * @return a RecordCursor that iterates over all KeyValue pairs under this path + */ + @API(API.Status.UNSTABLE) + @Nonnull + RecordCursor exportAllData(@Nonnull FDBRecordContext context, + @Nonnull ScanProperties scanProperties); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java index 7102da8936..67e869188c 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java @@ -337,6 +337,15 @@ public String toString() { return toString(null); } + @Nonnull + @Override + public RecordCursor exportAllData(@Nonnull FDBRecordContext context, + @Nonnull ScanProperties scanProperties) { + // TODO: Implement the actual data export functionality + // This is a stub method that will be implemented later + throw new UnsupportedOperationException("exportAllData is not yet implemented"); + } + /** * Returns this path properly wrapped in whatever implementation the directory the path is contained in dictates. */ diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathWrapper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathWrapper.java index 769649b8da..a1602110b1 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathWrapper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathWrapper.java @@ -226,4 +226,11 @@ public String toString() { public String toString(@Nonnull Tuple t) { return inner.toString(t); } + + @Nonnull + @Override + public RecordCursor exportAllData(@Nonnull FDBRecordContext context, + @Nonnull ScanProperties scanProperties) { + return inner.exportAllData(context, scanProperties); + } } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java new file mode 100644 index 0000000000..86c166efe7 --- /dev/null +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java @@ -0,0 +1,580 @@ +/* + * KeySpacePathDataExportTest.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.keyspace; + +import com.apple.foundationdb.KeyValue; +import com.apple.foundationdb.Transaction; +import com.apple.foundationdb.record.RecordCursor; +import com.apple.foundationdb.record.RecordCursorResult; +import com.apple.foundationdb.record.ScanProperties; +import com.apple.foundationdb.record.provider.foundationdb.FDBDatabase; +import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; +import com.apple.foundationdb.record.provider.foundationdb.keyspace.KeySpaceDirectory.KeyType; +import com.apple.foundationdb.record.test.FDBDatabaseExtension; +import com.apple.foundationdb.tuple.Tuple; +import com.apple.test.Tags; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import javax.annotation.Nonnull; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Random; +import java.util.UUID; + +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.assertTrue; + +/** + * Tests for the new KeySpacePath data export feature that fetches all data stored under a KeySpacePath + * and returns it in a RecordCursor<KeyValue>. + */ +@Tag(Tags.RequiresFDB) +public class KeySpacePathDataExportTest { + @RegisterExtension + final FDBDatabaseExtension dbExtension = new FDBDatabaseExtension(); + + private final Random random = new Random(); + + @Test + public void testExportAllDataFromSimplePath() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("root", KeyType.STRING, "test-root") + .addSubdirectory(new KeySpaceDirectory("level1", KeyType.LONG))); + + final FDBDatabase database = dbExtension.getDatabase(); + + // Store test data + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + KeySpacePath basePath = root.path("root"); + + // Add data at different levels + for (int i = 0; i < 5; i++) { + Tuple key = basePath.add("level1", (long) i).toTuple(context); + tr.set(key.pack(), Tuple.from("value" + i).pack()); + + // Add some sub-data under each key + for (int j = 0; j < 3; j++) { + Tuple subKey = key.add("sub" + j); + tr.set(subKey.pack(), Tuple.from("subvalue" + i + "_" + j).pack()); + } + } + context.commit(); + } + + // Export all data from the root path + try (FDBRecordContext context = database.openContext()) { + KeySpacePath rootPath = root.path("root"); + RecordCursor cursor = rootPath.exportAllData(context); + + List allData = cursor.asList().join(); + + // Should have 5 main entries + 15 sub-entries = 20 total + assertEquals(20, allData.size()); + + // Verify the data is sorted by key + for (int i = 1; i < allData.size(); i++) { + assertTrue(Tuple.fromBytes(allData.get(i - 1).getKey()).compareTo( + Tuple.fromBytes(allData.get(i).getKey())) < 0); + } + } + } + + @Test + public void testExportAllDataFromSpecificSubPath() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("app", KeyType.STRING, "myapp") + .addSubdirectory(new KeySpaceDirectory("user", KeyType.LONG)) + .addSubdirectory(new KeySpaceDirectory("data", KeyType.NULL))); + + final FDBDatabase database = dbExtension.getDatabase(); + + // Store test data for multiple users + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + + for (long userId = 1; userId <= 3; userId++) { + KeySpacePath userPath = root.path("app").add("user", userId); + KeySpacePath dataPath = userPath.add("data"); + + // Add data for each user + for (int i = 0; i < 4; i++) { + Tuple key = dataPath.toTuple(context).add("record" + i); + tr.set(key.pack(), Tuple.from("user" + userId + "_data" + i).pack()); + } + } + context.commit(); + } + + // Export data only for user 2 + try (FDBRecordContext context = database.openContext()) { + KeySpacePath user2Path = root.path("app").add("user", 2L); + RecordCursor cursor = user2Path.exportAllData(context); + + List userData = cursor.asList().join(); + + // Should have 4 records for user 2 + assertEquals(4, userData.size()); + + // Verify all data belongs to user 2 + for (KeyValue kv : userData) { + String value = Tuple.fromBytes(kv.getValue()).getString(0); + assertTrue(value.startsWith("user2_")); + } + } + } + + @Test + public void testExportAllDataWithDirectoryLayer() { + KeySpace root = new KeySpace( + new DirectoryLayerDirectory("env", "production") + .addSubdirectory(new KeySpaceDirectory("tenant", KeyType.LONG)) + .addSubdirectory(new DirectoryLayerDirectory("service"))); + + final FDBDatabase database = dbExtension.getDatabase(); + + // Store test data + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + + KeySpacePath basePath = root.path("env").add("tenant", 100L); + + // Add data for different services + String[] services = {"auth", "storage", "compute"}; + for (String service : services) { + KeySpacePath servicePath = basePath.add("service", service); + Tuple serviceKey = servicePath.toTuple(context); + + for (int i = 0; i < 2; i++) { + tr.set(serviceKey.add("config" + i).pack(), + Tuple.from(service + "_config_" + i).pack()); + } + } + context.commit(); + } + + // Export all data from tenant path + try (FDBRecordContext context = database.openContext()) { + KeySpacePath tenantPath = root.path("env").add("tenant", 100L); + RecordCursor cursor = tenantPath.exportAllData(context); + + List allData = cursor.asList().join(); + + // Should have 6 records (3 services * 2 configs each) + assertEquals(6, allData.size()); + + // Verify we have data for all three services + List serviceNames = new ArrayList<>(); + for (KeyValue kv : allData) { + String value = Tuple.fromBytes(kv.getValue()).getString(0); + String serviceName = value.split("_")[0]; + if (!serviceNames.contains(serviceName)) { + serviceNames.add(serviceName); + } + } + assertEquals(3, serviceNames.size()); + assertTrue(serviceNames.containsAll(Arrays.asList("auth", "storage", "compute"))); + } + } + + @Test + public void testExportAllDataWithDifferentKeyTypes() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("mixed", KeyType.STRING, "mixed-types") + .addSubdirectory(new KeySpaceDirectory("strings", KeyType.STRING)) + .addSubdirectory(new KeySpaceDirectory("longs", KeyType.LONG)) + .addSubdirectory(new KeySpaceDirectory("bytes", KeyType.BYTES)) + .addSubdirectory(new KeySpaceDirectory("uuids", KeyType.UUID)) + .addSubdirectory(new KeySpaceDirectory("booleans", KeyType.BOOLEAN))); + + final FDBDatabase database = dbExtension.getDatabase(); + + // Store test data with different key types + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + KeySpacePath basePath = root.path("mixed"); + + // String keys + for (int i = 0; i < 3; i++) { + Tuple key = basePath.add("strings", "str" + i).toTuple(context); + tr.set(key.pack(), Tuple.from("string_value_" + i).pack()); + } + + // Long keys + for (long i = 10; i < 13; i++) { + Tuple key = basePath.add("longs", i).toTuple(context); + tr.set(key.pack(), Tuple.from("long_value_" + i).pack()); + } + + // Bytes keys + for (int i = 0; i < 2; i++) { + byte[] byteKey = new byte[] { (byte) i, (byte) (i + 1) }; + Tuple key = basePath.add("bytes", byteKey).toTuple(context); + tr.set(key.pack(), Tuple.from("bytes_value_" + i).pack()); + } + + // UUID keys + for (int i = 0; i < 2; i++) { + UUID uuid = new UUID(i, i); + Tuple key = basePath.add("uuids", uuid).toTuple(context); + tr.set(key.pack(), Tuple.from("uuid_value_" + i).pack()); + } + + // Boolean keys + for (boolean b : Arrays.asList(true, false)) { + Tuple key = basePath.add("booleans", b).toTuple(context); + tr.set(key.pack(), Tuple.from("boolean_value_" + b).pack()); + } + + context.commit(); + } + + // Export all data and verify different key types + try (FDBRecordContext context = database.openContext()) { + KeySpacePath mixedPath = root.path("mixed"); + RecordCursor cursor = mixedPath.exportAllData(context); + + List allData = cursor.asList().join(); + + // Should have 12 records total (3+3+2+2+2) + assertEquals(12, allData.size()); + + // Verify we have different value types + List valueTypes = new ArrayList<>(); + for (KeyValue kv : allData) { + String value = Tuple.fromBytes(kv.getValue()).getString(0); + String valueType = value.split("_")[0]; + if (!valueTypes.contains(valueType)) { + valueTypes.add(valueType); + } + } + assertEquals(5, valueTypes.size()); + assertTrue(valueTypes.containsAll(Arrays.asList("string", "long", "bytes", "uuid", "boolean"))); + } + } + + @Test + public void testExportAllDataWithConstantValues() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("app", KeyType.STRING, "testapp") + .addSubdirectory(new KeySpaceDirectory("version", KeyType.LONG, 1L)) + .addSubdirectory(new KeySpaceDirectory("data", KeyType.STRING, "records"))); + + final FDBDatabase database = dbExtension.getDatabase(); + + // Store test data using constant values + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + + KeySpacePath dataPath = root.path("app").add("version").add("data"); + Tuple baseKey = dataPath.toTuple(context); + + // Add multiple records under the constant path + for (int i = 0; i < 4; i++) { + tr.set(baseKey.add("record" + i).pack(), + Tuple.from("constant_path_data_" + i).pack()); + } + context.commit(); + } + + // Export data from path with constant values + try (FDBRecordContext context = database.openContext()) { + KeySpacePath appPath = root.path("app"); + RecordCursor cursor = appPath.exportAllData(context); + + List allData = cursor.asList().join(); + + // Should have 4 records + assertEquals(4, allData.size()); + + // Verify all data has expected prefix + for (KeyValue kv : allData) { + String value = Tuple.fromBytes(kv.getValue()).getString(0); + assertTrue(value.startsWith("constant_path_data_")); + } + } + } + + @Test + public void testExportAllDataEmpty() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("empty", KeyType.STRING, "empty-space") + .addSubdirectory(new KeySpaceDirectory("level1", KeyType.LONG))); + + final FDBDatabase database = dbExtension.getDatabase(); + + // Don't store any data + try (FDBRecordContext context = database.openContext()) { + KeySpacePath emptyPath = root.path("empty"); + RecordCursor cursor = emptyPath.exportAllData(context); + + List allData = cursor.asList().join(); + + // Should be empty + assertEquals(0, allData.size()); + } + } + + @Test + public void testExportAllDataWithScanProperties() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("limited", KeyType.STRING, "limited-scan") + .addSubdirectory(new KeySpaceDirectory("item", KeyType.LONG))); + + final FDBDatabase database = dbExtension.getDatabase(); + + // Store many records + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + KeySpacePath basePath = root.path("limited"); + + for (int i = 0; i < 20; i++) { + Tuple key = basePath.add("item", (long) i).toTuple(context); + tr.set(key.pack(), Tuple.from("item_data_" + i).pack()); + } + context.commit(); + } + + // Export with limited scan properties + try (FDBRecordContext context = database.openContext()) { + KeySpacePath limitedPath = root.path("limited"); + ScanProperties scanProps = ScanProperties.FORWARD_SCAN.with(props -> + props.setReturnedRowLimit(5)); + + RecordCursor cursor = limitedPath.exportAllData(context, scanProps); + + List limitedData = cursor.asList().join(); + + // Should have only 5 records due to limit + assertEquals(5, limitedData.size()); + + // Should be the first 5 items + for (int i = 0; i < 5; i++) { + String value = Tuple.fromBytes(limitedData.get(i).getValue()).getString(0); + assertEquals("item_data_" + i, value); + } + } + } + + @ParameterizedTest + @ValueSource(booleans = {true, false}) + public void testExportAllDataReverse(boolean reverse) { + KeySpace root = new KeySpace( + new KeySpaceDirectory("ordered", KeyType.STRING, "ordered-data") + .addSubdirectory(new KeySpaceDirectory("sequence", KeyType.LONG))); + + final FDBDatabase database = dbExtension.getDatabase(); + + // Store ordered data + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + KeySpacePath basePath = root.path("ordered"); + + for (int i = 0; i < 5; i++) { + Tuple key = basePath.add("sequence", (long) i).toTuple(context); + tr.set(key.pack(), Tuple.from("seq_" + i).pack()); + } + context.commit(); + } + + // Export with forward or reverse scan + try (FDBRecordContext context = database.openContext()) { + KeySpacePath orderedPath = root.path("ordered"); + ScanProperties scanProps = new ScanProperties(null, reverse); + + RecordCursor cursor = orderedPath.exportAllData(context, scanProps); + + List allData = cursor.asList().join(); + + assertEquals(5, allData.size()); + + // Verify order based on scan direction + for (int i = 0; i < 5; i++) { + String value = Tuple.fromBytes(allData.get(i).getValue()).getString(0); + int expectedIndex = reverse ? (4 - i) : i; + assertEquals("seq_" + expectedIndex, value); + } + } + } + + @Test + public void testExportAllDataWithDeepNestedStructure() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("org", KeyType.STRING, "company") + .addSubdirectory(new KeySpaceDirectory("dept", KeyType.STRING)) + .addSubdirectory(new KeySpaceDirectory("team", KeyType.LONG)) + .addSubdirectory(new KeySpaceDirectory("member", KeyType.UUID)) + .addSubdirectory(new KeySpaceDirectory("data", KeyType.NULL))); + + final FDBDatabase database = dbExtension.getDatabase(); + + // Create deep nested structure + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + + String[] departments = {"engineering", "sales"}; + for (String dept : departments) { + for (long team = 1; team <= 2; team++) { + for (int member = 0; member < 2; member++) { + UUID memberId = new UUID(dept.hashCode(), team * 100 + member); + KeySpacePath memberPath = root.path("org") + .add("dept", dept) + .add("team", team) + .add("member", memberId) + .add("data"); + + Tuple key = memberPath.toTuple(context); + tr.set(key.add("profile").pack(), + Tuple.from(dept + "_team" + team + "_member" + member).pack()); + tr.set(key.add("settings").pack(), + Tuple.from("settings_" + member).pack()); + } + } + } + context.commit(); + } + + // Export all data from organization root + try (FDBRecordContext context = database.openContext()) { + KeySpacePath orgPath = root.path("org"); + RecordCursor cursor = orgPath.exportAllData(context); + + List allData = cursor.asList().join(); + + // Should have 16 records (2 depts * 2 teams * 2 members * 2 records each) + assertEquals(16, allData.size()); + } + + // Export data from specific department + try (FDBRecordContext context = database.openContext()) { + KeySpacePath engPath = root.path("org").add("dept", "engineering"); + RecordCursor cursor = engPath.exportAllData(context); + + List allData = cursor.asList().join(); + + // Should have 8 records (1 dept * 2 teams * 2 members * 2 records each) + assertEquals(8, allData.size()); + + // Verify all belong to engineering + for (KeyValue kv : allData) { + String value = Tuple.fromBytes(kv.getValue()).getString(0); + if (value.startsWith("engineering_")) { + assertTrue(value.contains("engineering_")); + } + } + } + } + + @Test + public void testExportAllDataWithBinaryData() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("binary", KeyType.STRING, "binary-test") + .addSubdirectory(new KeySpaceDirectory("blob", KeyType.BYTES))); + + final FDBDatabase database = dbExtension.getDatabase(); + + // Store binary data + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + KeySpacePath basePath = root.path("binary"); + + // Store different types of binary data + byte[][] binaryKeys = { + {0x00, 0x01, 0x02}, + {(byte) 0xFF, (byte) 0xFE, (byte) 0xFD}, + {0x7F, 0x00, (byte) 0x80} + }; + + for (int i = 0; i < binaryKeys.length; i++) { + Tuple key = basePath.add("blob", binaryKeys[i]).toTuple(context); + byte[] value = ("binary_data_" + i).getBytes(); + tr.set(key.pack(), value); + } + context.commit(); + } + + // Export binary data + try (FDBRecordContext context = database.openContext()) { + KeySpacePath binaryPath = root.path("binary"); + RecordCursor cursor = binaryPath.exportAllData(context); + + List allData = cursor.asList().join(); + + assertEquals(3, allData.size()); + + // Verify binary data integrity + for (int i = 0; i < allData.size(); i++) { + KeyValue kv = allData.get(i); + String valueStr = new String(kv.getValue()); + assertTrue(valueStr.startsWith("binary_data_")); + } + } + } + + @Test + public void testExportAllDataCursorBehavior() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("cursor", KeyType.STRING, "cursor-test") + .addSubdirectory(new KeySpaceDirectory("item", KeyType.LONG))); + + final FDBDatabase database = dbExtension.getDatabase(); + + // Store test data + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + KeySpacePath basePath = root.path("cursor"); + + for (int i = 0; i < 10; i++) { + Tuple key = basePath.add("item", (long) i).toTuple(context); + tr.set(key.pack(), Tuple.from("cursor_item_" + i).pack()); + } + context.commit(); + } + + // Test cursor behavior + try (FDBRecordContext context = database.openContext()) { + KeySpacePath cursorPath = root.path("cursor"); + RecordCursor cursor = cursorPath.exportAllData(context); + + // Test that cursor can be iterated + List collected = new ArrayList<>(); + RecordCursorResult result; + + while ((result = cursor.getNext()).hasNext()) { + collected.add(result.get()); + } + + assertEquals(10, collected.size()); + assertFalse(result.hasNext()); + + // Verify the reason for stopping + assertEquals(RecordCursor.NoNextReason.SOURCE_EXHAUSTED, result.getNoNextReason()); + } + } +} \ No newline at end of file From f79e912d19057f2cc5bab5894ae71b10e74d1777 Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Wed, 27 Aug 2025 16:03:21 -0400 Subject: [PATCH 02/32] Add continuation tests --- .../foundationdb/keyspace/KeySpacePath.java | 22 +- .../keyspace/KeySpacePathImpl.java | 5 +- .../keyspace/KeySpacePathWrapper.java | 5 +- .../keyspace/KeySpacePathDataExportTest.java | 354 ++++++++++++++++++ 4 files changed, 380 insertions(+), 6 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java index db142ca6eb..6634a7ab9f 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java @@ -577,7 +577,7 @@ default List listSubdirectory(@Nonnull FDBRecordContext co @API(API.Status.UNSTABLE) @Nonnull default RecordCursor exportAllData(@Nonnull FDBRecordContext context) { - return exportAllData(context, ScanProperties.FORWARD_SCAN); + return exportAllData(context, null, ScanProperties.FORWARD_SCAN); } /** @@ -590,6 +590,24 @@ default RecordCursor exportAllData(@Nonnull FDB */ @API(API.Status.UNSTABLE) @Nonnull - RecordCursor exportAllData(@Nonnull FDBRecordContext context, + default RecordCursor exportAllData(@Nonnull FDBRecordContext context, + @Nonnull ScanProperties scanProperties) { + return exportAllData(context, null, scanProperties); + } + + /** + * Export all data stored under this KeySpacePath and return it in a RecordCursor. + * This method scans all keys that have this path as a prefix and returns the key-value pairs. + * Supports continuation to resume scanning from a previous position. + * + * @param context the transaction context in which to perform the data export + * @param continuation optional continuation from a previous export operation, or null to start from the beginning + * @param scanProperties properties controlling how the scan should be performed + * @return a RecordCursor that iterates over all KeyValue pairs under this path + */ + @API(API.Status.UNSTABLE) + @Nonnull + RecordCursor exportAllData(@Nonnull FDBRecordContext context, + @Nullable byte[] continuation, @Nonnull ScanProperties scanProperties); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java index 67e869188c..0cd3e2d28d 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java @@ -339,9 +339,10 @@ public String toString() { @Nonnull @Override - public RecordCursor exportAllData(@Nonnull FDBRecordContext context, + public RecordCursor exportAllData(@Nonnull FDBRecordContext context, + @Nullable byte[] continuation, @Nonnull ScanProperties scanProperties) { - // TODO: Implement the actual data export functionality + // TODO: Implement the actual data export functionality with continuation support // This is a stub method that will be implemented later throw new UnsupportedOperationException("exportAllData is not yet implemented"); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathWrapper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathWrapper.java index a1602110b1..cd39f2c5f0 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathWrapper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathWrapper.java @@ -229,8 +229,9 @@ public String toString(@Nonnull Tuple t) { @Nonnull @Override - public RecordCursor exportAllData(@Nonnull FDBRecordContext context, + public RecordCursor exportAllData(@Nonnull FDBRecordContext context, + @Nullable byte[] continuation, @Nonnull ScanProperties scanProperties) { - return inner.exportAllData(context, scanProperties); + return inner.exportAllData(context, continuation, scanProperties); } } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java index 86c166efe7..e46be34ce7 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java @@ -22,6 +22,7 @@ import com.apple.foundationdb.KeyValue; import com.apple.foundationdb.Transaction; +import com.apple.foundationdb.record.ExecuteProperties; import com.apple.foundationdb.record.RecordCursor; import com.apple.foundationdb.record.RecordCursorResult; import com.apple.foundationdb.record.ScanProperties; @@ -577,4 +578,357 @@ public void testExportAllDataCursorBehavior() { assertEquals(RecordCursor.NoNextReason.SOURCE_EXHAUSTED, result.getNoNextReason()); } } + + @Test + public void testExportAllDataWithContinuation() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("continuation", KeyType.STRING, "continuation-test") + .addSubdirectory(new KeySpaceDirectory("item", KeyType.LONG))); + + final FDBDatabase database = dbExtension.getDatabase(); + + // Store test data + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + KeySpacePath basePath = root.path("continuation"); + + for (int i = 0; i < 20; i++) { + Tuple key = basePath.add("item", (long) i).toTuple(context); + tr.set(key.pack(), Tuple.from("continuation_item_" + i).pack()); + } + context.commit(); + } + + // Export with continuation support + try (FDBRecordContext context = database.openContext()) { + KeySpacePath continuationPath = root.path("continuation"); + + // First export with limit to get continuation + ScanProperties limitedScan = ScanProperties.FORWARD_SCAN.with(props -> + props.setReturnedRowLimit(5)); + + RecordCursor cursor = continuationPath.exportAllData(context, null, limitedScan); + List firstBatch = cursor.asList().join(); + + assertEquals(5, firstBatch.size()); + + // Verify first batch contains items 0-4 + for (int i = 0; i < 5; i++) { + String value = Tuple.fromBytes(firstBatch.get(i).getValue()).getString(0); + assertEquals("continuation_item_" + i, value); + } + + // Get continuation from the cursor result + RecordCursorResult lastResult = cursor.getNext(); + assertFalse(lastResult.hasNext()); + assertEquals(RecordCursor.NoNextReason.RETURN_LIMIT_REACHED, lastResult.getNoNextReason()); + + byte[] continuation = lastResult.getContinuation().toBytes(); + assertNotNull(continuation); + + // Use continuation to get next batch + RecordCursor continuedCursor = continuationPath.exportAllData(context, continuation, limitedScan); + List secondBatch = continuedCursor.asList().join(); + + assertEquals(5, secondBatch.size()); + + // Verify second batch contains items 5-9 + for (int i = 0; i < 5; i++) { + String value = Tuple.fromBytes(secondBatch.get(i).getValue()).getString(0); + assertEquals("continuation_item_" + (i + 5), value); + } + } + } + + @Test + public void testExportAllDataContinuationChaining() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("chain", KeyType.STRING, "chain-test") + .addSubdirectory(new KeySpaceDirectory("batch", KeyType.LONG))); + + final FDBDatabase database = dbExtension.getDatabase(); + + // Store test data + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + KeySpacePath basePath = root.path("chain"); + + for (int i = 0; i < 30; i++) { + Tuple key = basePath.add("batch", (long) i).toTuple(context); + tr.set(key.pack(), Tuple.from("batch_item_" + i).pack()); + } + context.commit(); + } + + // Chain multiple continuations + try (FDBRecordContext context = database.openContext()) { + KeySpacePath chainPath = root.path("chain"); + ScanProperties batchScan = ScanProperties.FORWARD_SCAN.with(props -> + props.setReturnedRowLimit(7)); + + List allCollected = new ArrayList<>(); + byte[] continuation = null; + int batchCount = 0; + + do { + RecordCursor cursor = chainPath.exportAllData(context, continuation, batchScan); + List batch = cursor.asList().join(); + + if (batch.isEmpty()) { + break; + } + + allCollected.addAll(batch); + batchCount++; + + // Get continuation for next batch + RecordCursorResult lastResult = cursor.getNext(); + if (lastResult.hasNext() || lastResult.getNoNextReason() == RecordCursor.NoNextReason.RETURN_LIMIT_REACHED) { + continuation = lastResult.getContinuation().toBytes(); + } else { + continuation = null; + } + + // Safety check to avoid infinite loop + assertTrue(batchCount <= 10, "Too many batches, possible infinite loop"); + + } while (continuation != null); + + // Should have collected all 30 items across multiple batches + assertEquals(30, allCollected.size()); + assertEquals(5, batchCount); // 30 items / 7 per batch = 5 batches (last partial) + + // Verify all items are present and in order + for (int i = 0; i < 30; i++) { + String value = Tuple.fromBytes(allCollected.get(i).getValue()).getString(0); + assertEquals("batch_item_" + i, value); + } + } + } + + @Test + public void testExportAllDataContinuationWithDifferentScanProperties() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("scan", KeyType.STRING, "scan-props-test") + .addSubdirectory(new KeySpaceDirectory("record", KeyType.LONG))); + + final FDBDatabase database = dbExtension.getDatabase(); + + // Store test data + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + KeySpacePath basePath = root.path("scan"); + + for (int i = 0; i < 15; i++) { + Tuple key = basePath.add("record", (long) i).toTuple(context); + tr.set(key.pack(), Tuple.from("scan_record_" + i).pack()); + } + context.commit(); + } + + // Test continuation with reverse scan + try (FDBRecordContext context = database.openContext()) { + KeySpacePath scanPath = root.path("scan"); + ScanProperties reverseScan = new ScanProperties(ExecuteProperties.newBuilder() + .setReturnedRowLimit(5) + .build(), true); // limit 5, reverse + + // First batch in reverse order + RecordCursor cursor = scanPath.exportAllData(context, null, reverseScan); + List firstBatch = cursor.asList().join(); + + assertEquals(5, firstBatch.size()); + + // Verify reverse order (should be items 14, 13, 12, 11, 10) + for (int i = 0; i < 5; i++) { + String value = Tuple.fromBytes(firstBatch.get(i).getValue()).getString(0); + assertEquals("scan_record_" + (14 - i), value); + } + + // Get continuation and continue reverse scan + RecordCursorResult lastResult = cursor.getNext(); + byte[] continuation = lastResult.getContinuation().toBytes(); + + RecordCursor continuedCursor = scanPath.exportAllData(context, continuation, reverseScan); + List secondBatch = continuedCursor.asList().join(); + + assertEquals(5, secondBatch.size()); + + // Verify second batch in reverse order (should be items 9, 8, 7, 6, 5) + for (int i = 0; i < 5; i++) { + String value = Tuple.fromBytes(secondBatch.get(i).getValue()).getString(0); + assertEquals("scan_record_" + (9 - i), value); + } + } + } + + @Test + public void testExportAllDataContinuationWithNestedPaths() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("nested", KeyType.STRING, "nested-continuation") + .addSubdirectory(new KeySpaceDirectory("category", KeyType.STRING)) + .addSubdirectory(new KeySpaceDirectory("item", KeyType.LONG)) + .addSubdirectory(new KeySpaceDirectory("data", KeyType.NULL))); + + final FDBDatabase database = dbExtension.getDatabase(); + + // Store nested test data + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + + String[] categories = {"A", "B", "C"}; + for (String category : categories) { + for (int item = 0; item < 5; item++) { + KeySpacePath dataPath = root.path("nested") + .add("category", category) + .add("item", (long) item) + .add("data"); + + Tuple key = dataPath.toTuple(context); + tr.set(key.pack(), Tuple.from(category + "_item_" + item).pack()); + } + } + context.commit(); + } + + // Export with continuation from nested path + try (FDBRecordContext context = database.openContext()) { + KeySpacePath nestedPath = root.path("nested").add("category", "B"); + ScanProperties limitedScan = ScanProperties.FORWARD_SCAN.with(props -> + props.setReturnedRowLimit(3)); + + // First batch from category B + RecordCursor cursor = nestedPath.exportAllData(context, null, limitedScan); + List firstBatch = cursor.asList().join(); + + assertEquals(3, firstBatch.size()); + + // Verify all are from category B + for (KeyValue kv : firstBatch) { + String value = Tuple.fromBytes(kv.getValue()).getString(0); + assertTrue(value.startsWith("B_item_")); + } + + // Get continuation and get remaining items from category B + RecordCursorResult lastResult = cursor.getNext(); + byte[] continuation = lastResult.getContinuation().toBytes(); + + RecordCursor continuedCursor = nestedPath.exportAllData(context, continuation, limitedScan); + List secondBatch = continuedCursor.asList().join(); + + assertEquals(2, secondBatch.size()); // Only 2 remaining items in category B + + // Verify remaining items are from category B + for (KeyValue kv : secondBatch) { + String value = Tuple.fromBytes(kv.getValue()).getString(0); + assertTrue(value.startsWith("B_item_")); + } + } + } + + @Test + public void testExportAllDataEmptyContinuation() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("empty_cont", KeyType.STRING, "empty-continuation") + .addSubdirectory(new KeySpaceDirectory("item", KeyType.LONG))); + + final FDBDatabase database = dbExtension.getDatabase(); + + // Store minimal test data + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + KeySpacePath basePath = root.path("empty_cont"); + + for (int i = 0; i < 3; i++) { + Tuple key = basePath.add("item", (long) i).toTuple(context); + tr.set(key.pack(), Tuple.from("empty_cont_item_" + i).pack()); + } + context.commit(); + } + + // Test behavior when using continuation on empty results + try (FDBRecordContext context = database.openContext()) { + KeySpacePath emptyContPath = root.path("empty_cont"); + ScanProperties largeLimitScan = ScanProperties.FORWARD_SCAN.with(props -> + props.setReturnedRowLimit(10)); // Larger than available data + + // First export gets all data (no continuation needed) + RecordCursor cursor = emptyContPath.exportAllData(context, null, largeLimitScan); + List allData = cursor.asList().join(); + + assertEquals(3, allData.size()); + + // Get final result + RecordCursorResult finalResult = cursor.getNext(); + assertFalse(finalResult.hasNext()); + assertEquals(RecordCursor.NoNextReason.SOURCE_EXHAUSTED, finalResult.getNoNextReason()); + + // Try to use continuation (should return empty) + byte[] continuation = finalResult.getContinuation().toBytes(); + RecordCursor continuedCursor = emptyContPath.exportAllData(context, continuation, largeLimitScan); + List continuedData = continuedCursor.asList().join(); + + assertEquals(0, continuedData.size()); // Should be empty + } + } + + @Test + public void testExportAllDataContinuationConsistency() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("consistency", KeyType.STRING, "consistency-test") + .addSubdirectory(new KeySpaceDirectory("sequence", KeyType.LONG))); + + final FDBDatabase database = dbExtension.getDatabase(); + + // Store test data + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + KeySpacePath basePath = root.path("consistency"); + + for (int i = 0; i < 12; i++) { + Tuple key = basePath.add("sequence", (long) i).toTuple(context); + tr.set(key.pack(), Tuple.from("consistency_seq_" + i).pack()); + } + context.commit(); + } + + // Test that continuation produces consistent, non-overlapping results + try (FDBRecordContext context = database.openContext()) { + KeySpacePath consistencyPath = root.path("consistency"); + ScanProperties batchScan = ScanProperties.FORWARD_SCAN.with(props -> + props.setReturnedRowLimit(4)); + + // Collect all data using continuations + List collectedValues = new ArrayList<>(); + byte[] continuation = null; + + for (int batch = 0; batch < 3; batch++) { // Expect 3 batches of 4 items each + RecordCursor cursor = consistencyPath.exportAllData(context, continuation, batchScan); + List batchData = cursor.asList().join(); + + assertEquals(4, batchData.size(), "Batch " + batch + " should have 4 items"); + + for (KeyValue kv : batchData) { + String value = Tuple.fromBytes(kv.getValue()).getString(0); + assertFalse(collectedValues.contains(value), "Duplicate value detected: " + value); + collectedValues.add(value); + } + + // Get continuation for next batch + RecordCursorResult lastResult = cursor.getNext(); + if (batch < 2) { // Not the last batch + assertEquals(RecordCursor.NoNextReason.RETURN_LIMIT_REACHED, lastResult.getNoNextReason()); + continuation = lastResult.getContinuation().toBytes(); + } else { // Last batch + assertEquals(RecordCursor.NoNextReason.SOURCE_EXHAUSTED, lastResult.getNoNextReason()); + } + } + + // Verify we got all 12 items in correct order + assertEquals(12, collectedValues.size()); + for (int i = 0; i < 12; i++) { + assertEquals("consistency_seq_" + i, collectedValues.get(i)); + } + } + } } \ No newline at end of file From 80cbad29a30370a9e6a3f134cafe6cd896442ed7 Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Wed, 3 Sep 2025 10:08:43 -0400 Subject: [PATCH 03/32] Fix checkstyle --- .../keyspace/KeySpacePathDataExportTest.java | 37 ++++++++----------- 1 file changed, 16 insertions(+), 21 deletions(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java index e46be34ce7..69e66e6602 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java @@ -38,14 +38,11 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; -import javax.annotation.Nonnull; import java.util.ArrayList; import java.util.Arrays; import java.util.List; -import java.util.Random; import java.util.UUID; -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; @@ -53,15 +50,13 @@ /** * Tests for the new KeySpacePath data export feature that fetches all data stored under a KeySpacePath - * and returns it in a RecordCursor<KeyValue>. + * and returns it in a {@code RecordCursor}. */ @Tag(Tags.RequiresFDB) public class KeySpacePathDataExportTest { @RegisterExtension final FDBDatabaseExtension dbExtension = new FDBDatabaseExtension(); - private final Random random = new Random(); - @Test public void testExportAllDataFromSimplePath() { KeySpace root = new KeySpace( @@ -173,8 +168,8 @@ public void testExportAllDataWithDirectoryLayer() { Tuple serviceKey = servicePath.toTuple(context); for (int i = 0; i < 2; i++) { - tr.set(serviceKey.add("config" + i).pack(), - Tuple.from(service + "_config_" + i).pack()); + tr.set(serviceKey.add("config" + i).pack(), + Tuple.from(service + "_config_" + i).pack()); } } context.commit(); @@ -298,8 +293,8 @@ public void testExportAllDataWithConstantValues() { // Add multiple records under the constant path for (int i = 0; i < 4; i++) { - tr.set(baseKey.add("record" + i).pack(), - Tuple.from("constant_path_data_" + i).pack()); + tr.set(baseKey.add("record" + i).pack(), + Tuple.from("constant_path_data_" + i).pack()); } context.commit(); } @@ -366,7 +361,7 @@ public void testExportAllDataWithScanProperties() { try (FDBRecordContext context = database.openContext()) { KeySpacePath limitedPath = root.path("limited"); ScanProperties scanProps = ScanProperties.FORWARD_SCAN.with(props -> - props.setReturnedRowLimit(5)); + props.setReturnedRowLimit(5)); RecordCursor cursor = limitedPath.exportAllData(context, scanProps); @@ -451,10 +446,10 @@ public void testExportAllDataWithDeepNestedStructure() { .add("data"); Tuple key = memberPath.toTuple(context); - tr.set(key.add("profile").pack(), - Tuple.from(dept + "_team" + team + "_member" + member).pack()); - tr.set(key.add("settings").pack(), - Tuple.from("settings_" + member).pack()); + tr.set(key.add("profile").pack(), + Tuple.from(dept + "_team" + team + "_member" + member).pack()); + tr.set(key.add("settings").pack(), + Tuple.from("settings_" + member).pack()); } } } @@ -605,7 +600,7 @@ public void testExportAllDataWithContinuation() { // First export with limit to get continuation ScanProperties limitedScan = ScanProperties.FORWARD_SCAN.with(props -> - props.setReturnedRowLimit(5)); + props.setReturnedRowLimit(5)); RecordCursor cursor = continuationPath.exportAllData(context, null, limitedScan); List firstBatch = cursor.asList().join(); @@ -664,7 +659,7 @@ public void testExportAllDataContinuationChaining() { try (FDBRecordContext context = database.openContext()) { KeySpacePath chainPath = root.path("chain"); ScanProperties batchScan = ScanProperties.FORWARD_SCAN.with(props -> - props.setReturnedRowLimit(7)); + props.setReturnedRowLimit(7)); List allCollected = new ArrayList<>(); byte[] continuation = null; @@ -795,7 +790,7 @@ public void testExportAllDataContinuationWithNestedPaths() { try (FDBRecordContext context = database.openContext()) { KeySpacePath nestedPath = root.path("nested").add("category", "B"); ScanProperties limitedScan = ScanProperties.FORWARD_SCAN.with(props -> - props.setReturnedRowLimit(3)); + props.setReturnedRowLimit(3)); // First batch from category B RecordCursor cursor = nestedPath.exportAllData(context, null, limitedScan); @@ -850,7 +845,7 @@ public void testExportAllDataEmptyContinuation() { try (FDBRecordContext context = database.openContext()) { KeySpacePath emptyContPath = root.path("empty_cont"); ScanProperties largeLimitScan = ScanProperties.FORWARD_SCAN.with(props -> - props.setReturnedRowLimit(10)); // Larger than available data + props.setReturnedRowLimit(10)); // Larger than available data // First export gets all data (no continuation needed) RecordCursor cursor = emptyContPath.exportAllData(context, null, largeLimitScan); @@ -896,7 +891,7 @@ public void testExportAllDataContinuationConsistency() { try (FDBRecordContext context = database.openContext()) { KeySpacePath consistencyPath = root.path("consistency"); ScanProperties batchScan = ScanProperties.FORWARD_SCAN.with(props -> - props.setReturnedRowLimit(4)); + props.setReturnedRowLimit(4)); // Collect all data using continuations List collectedValues = new ArrayList<>(); @@ -931,4 +926,4 @@ public void testExportAllDataContinuationConsistency() { } } } -} \ No newline at end of file +} From 1959c7381af16ae0b1ecdc7f729e6ae03559c0bc Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Wed, 3 Sep 2025 10:51:16 -0400 Subject: [PATCH 04/32] Reduce duplication in KeySpacePathDataExportTest and remove exportAllData overrides --- .../foundationdb/keyspace/KeySpacePath.java | 28 - .../keyspace/KeySpacePathDataExportTest.java | 650 +++--------------- 2 files changed, 111 insertions(+), 567 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java index 6634a7ab9f..4c06081765 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java @@ -567,34 +567,6 @@ default List listSubdirectory(@Nonnull FDBRecordContext co @API(API.Status.UNSTABLE) String toString(@Nonnull Tuple tuple); - /** - * Export all data stored under this KeySpacePath and return it in a RecordCursor. - * This method scans all keys that have this path as a prefix and returns the key-value pairs. - * - * @param context the transaction context in which to perform the data export - * @return a RecordCursor that iterates over all KeyValue pairs under this path - */ - @API(API.Status.UNSTABLE) - @Nonnull - default RecordCursor exportAllData(@Nonnull FDBRecordContext context) { - return exportAllData(context, null, ScanProperties.FORWARD_SCAN); - } - - /** - * Export all data stored under this KeySpacePath and return it in a RecordCursor. - * This method scans all keys that have this path as a prefix and returns the key-value pairs. - * - * @param context the transaction context in which to perform the data export - * @param scanProperties properties controlling how the scan should be performed - * @return a RecordCursor that iterates over all KeyValue pairs under this path - */ - @API(API.Status.UNSTABLE) - @Nonnull - default RecordCursor exportAllData(@Nonnull FDBRecordContext context, - @Nonnull ScanProperties scanProperties) { - return exportAllData(context, null, scanProperties); - } - /** * Export all data stored under this KeySpacePath and return it in a RecordCursor. * This method scans all keys that have this path as a prefix and returns the key-value pairs. diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java index 69e66e6602..ddae929345 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java @@ -22,9 +22,10 @@ import com.apple.foundationdb.KeyValue; import com.apple.foundationdb.Transaction; -import com.apple.foundationdb.record.ExecuteProperties; import com.apple.foundationdb.record.RecordCursor; +import com.apple.foundationdb.record.RecordCursorContinuation; import com.apple.foundationdb.record.RecordCursorResult; +import com.apple.foundationdb.record.RecordCursorStartContinuation; import com.apple.foundationdb.record.ScanProperties; import com.apple.foundationdb.record.provider.foundationdb.FDBDatabase; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; @@ -40,12 +41,16 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; import java.util.List; +import java.util.Set; import java.util.UUID; +import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; +import java.util.stream.IntStream; 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.assertTrue; /** @@ -53,14 +58,14 @@ * and returns it in a {@code RecordCursor}. */ @Tag(Tags.RequiresFDB) -public class KeySpacePathDataExportTest { +class KeySpacePathDataExportTest { @RegisterExtension final FDBDatabaseExtension dbExtension = new FDBDatabaseExtension(); @Test - public void testExportAllDataFromSimplePath() { + void testExportAllDataFromSimplePath() { KeySpace root = new KeySpace( - new KeySpaceDirectory("root", KeyType.STRING, "test-root") + new KeySpaceDirectory("root", KeyType.STRING, UUID.randomUUID().toString()) .addSubdirectory(new KeySpaceDirectory("level1", KeyType.LONG))); final FDBDatabase database = dbExtension.getDatabase(); @@ -74,7 +79,7 @@ public void testExportAllDataFromSimplePath() { for (int i = 0; i < 5; i++) { Tuple key = basePath.add("level1", (long) i).toTuple(context); tr.set(key.pack(), Tuple.from("value" + i).pack()); - + // Add some sub-data under each key for (int j = 0; j < 3; j++) { Tuple subKey = key.add("sub" + j); @@ -87,10 +92,8 @@ public void testExportAllDataFromSimplePath() { // Export all data from the root path try (FDBRecordContext context = database.openContext()) { KeySpacePath rootPath = root.path("root"); - RecordCursor cursor = rootPath.exportAllData(context); - - List allData = cursor.asList().join(); - + final List allData = exportAllData(rootPath, context); + // Should have 5 main entries + 15 sub-entries = 20 total assertEquals(20, allData.size()); @@ -103,9 +106,9 @@ public void testExportAllDataFromSimplePath() { } @Test - public void testExportAllDataFromSpecificSubPath() { + void testExportAllDataFromSpecificSubPath() { KeySpace root = new KeySpace( - new KeySpaceDirectory("app", KeyType.STRING, "myapp") + new KeySpaceDirectory("app", KeyType.STRING, UUID.randomUUID().toString()) .addSubdirectory(new KeySpaceDirectory("user", KeyType.LONG)) .addSubdirectory(new KeySpaceDirectory("data", KeyType.NULL))); @@ -131,10 +134,8 @@ public void testExportAllDataFromSpecificSubPath() { // Export data only for user 2 try (FDBRecordContext context = database.openContext()) { KeySpacePath user2Path = root.path("app").add("user", 2L); - RecordCursor cursor = user2Path.exportAllData(context); - - List userData = cursor.asList().join(); - + final List userData = exportAllData(user2Path, context); + // Should have 4 records for user 2 assertEquals(4, userData.size()); @@ -147,9 +148,9 @@ public void testExportAllDataFromSpecificSubPath() { } @Test - public void testExportAllDataWithDirectoryLayer() { + void testExportAllDataWithDirectoryLayer() { KeySpace root = new KeySpace( - new DirectoryLayerDirectory("env", "production") + new DirectoryLayerDirectory("env", UUID.randomUUID().toString()) .addSubdirectory(new KeySpaceDirectory("tenant", KeyType.LONG)) .addSubdirectory(new DirectoryLayerDirectory("service"))); @@ -178,21 +179,17 @@ public void testExportAllDataWithDirectoryLayer() { // Export all data from tenant path try (FDBRecordContext context = database.openContext()) { KeySpacePath tenantPath = root.path("env").add("tenant", 100L); - RecordCursor cursor = tenantPath.exportAllData(context); - - List allData = cursor.asList().join(); - + final List allData = exportAllData(tenantPath, context); + // Should have 6 records (3 services * 2 configs each) assertEquals(6, allData.size()); // Verify we have data for all three services - List serviceNames = new ArrayList<>(); + Set serviceNames = new HashSet<>(); for (KeyValue kv : allData) { String value = Tuple.fromBytes(kv.getValue()).getString(0); String serviceName = value.split("_")[0]; - if (!serviceNames.contains(serviceName)) { - serviceNames.add(serviceName); - } + serviceNames.add(serviceName); } assertEquals(3, serviceNames.size()); assertTrue(serviceNames.containsAll(Arrays.asList("auth", "storage", "compute"))); @@ -200,9 +197,9 @@ public void testExportAllDataWithDirectoryLayer() { } @Test - public void testExportAllDataWithDifferentKeyTypes() { + void testExportAllDataWithDifferentKeyTypes() { KeySpace root = new KeySpace( - new KeySpaceDirectory("mixed", KeyType.STRING, "mixed-types") + new KeySpaceDirectory("mixed", KeyType.STRING, UUID.randomUUID().toString()) .addSubdirectory(new KeySpaceDirectory("strings", KeyType.STRING)) .addSubdirectory(new KeySpaceDirectory("longs", KeyType.LONG)) .addSubdirectory(new KeySpaceDirectory("bytes", KeyType.BYTES)) @@ -213,40 +210,22 @@ public void testExportAllDataWithDifferentKeyTypes() { // Store test data with different key types try (FDBRecordContext context = database.openContext()) { - Transaction tr = context.ensureActive(); KeySpacePath basePath = root.path("mixed"); - // String keys - for (int i = 0; i < 3; i++) { - Tuple key = basePath.add("strings", "str" + i).toTuple(context); - tr.set(key.pack(), Tuple.from("string_value_" + i).pack()); - } + // String keys (str0, str1, str2 -> string_value_0, string_value_1, string_value_2) + setData(List.of("str0", "str1", "str2"), context, basePath, "strings", "string_value_"); - // Long keys - for (long i = 10; i < 13; i++) { - Tuple key = basePath.add("longs", i).toTuple(context); - tr.set(key.pack(), Tuple.from("long_value_" + i).pack()); - } + // Long keys (10, 11, 12 -> long_value_10, long_value_11, long_value_12) + setData(List.of(10L, 11L, 12L), context, basePath, "longs", "long_value_"); - // Bytes keys - for (int i = 0; i < 2; i++) { - byte[] byteKey = new byte[] { (byte) i, (byte) (i + 1) }; - Tuple key = basePath.add("bytes", byteKey).toTuple(context); - tr.set(key.pack(), Tuple.from("bytes_value_" + i).pack()); - } + // Bytes keys (arrays -> bytes_value_[0, 1], bytes_value_[1, 2]) + setData(List.of(new byte[]{0, 1}, new byte[]{1, 2}), context, basePath, "bytes", "bytes_value_"); - // UUID keys - for (int i = 0; i < 2; i++) { - UUID uuid = new UUID(i, i); - Tuple key = basePath.add("uuids", uuid).toTuple(context); - tr.set(key.pack(), Tuple.from("uuid_value_" + i).pack()); - } + // UUID keys (UUIDs -> uuid_value_UUID) + setData(List.of(new UUID(0, 0), new UUID(1, 1)), context, basePath, "uuids", "uuid_value_"); - // Boolean keys - for (boolean b : Arrays.asList(true, false)) { - Tuple key = basePath.add("booleans", b).toTuple(context); - tr.set(key.pack(), Tuple.from("boolean_value_" + b).pack()); - } + // Boolean keys (true, false -> boolean_value_true, boolean_value_false) + setData(List.of(true, false), context, basePath, "booleans", "boolean_value_"); context.commit(); } @@ -254,10 +233,8 @@ public void testExportAllDataWithDifferentKeyTypes() { // Export all data and verify different key types try (FDBRecordContext context = database.openContext()) { KeySpacePath mixedPath = root.path("mixed"); - RecordCursor cursor = mixedPath.exportAllData(context); - - List allData = cursor.asList().join(); - + final List allData = exportAllData(mixedPath, context); + // Should have 12 records total (3+3+2+2+2) assertEquals(12, allData.size()); @@ -276,9 +253,9 @@ public void testExportAllDataWithDifferentKeyTypes() { } @Test - public void testExportAllDataWithConstantValues() { + void testExportAllDataWithConstantValues() { KeySpace root = new KeySpace( - new KeySpaceDirectory("app", KeyType.STRING, "testapp") + new KeySpaceDirectory("app", KeyType.STRING, UUID.randomUUID().toString()) .addSubdirectory(new KeySpaceDirectory("version", KeyType.LONG, 1L)) .addSubdirectory(new KeySpaceDirectory("data", KeyType.STRING, "records"))); @@ -302,10 +279,8 @@ public void testExportAllDataWithConstantValues() { // Export data from path with constant values try (FDBRecordContext context = database.openContext()) { KeySpacePath appPath = root.path("app"); - RecordCursor cursor = appPath.exportAllData(context); - - List allData = cursor.asList().join(); - + final List allData = exportAllData(appPath, context); + // Should have 4 records assertEquals(4, allData.size()); @@ -318,9 +293,9 @@ public void testExportAllDataWithConstantValues() { } @Test - public void testExportAllDataEmpty() { + void testExportAllDataEmpty() { KeySpace root = new KeySpace( - new KeySpaceDirectory("empty", KeyType.STRING, "empty-space") + new KeySpaceDirectory("empty", KeyType.STRING, UUID.randomUUID().toString()) .addSubdirectory(new KeySpaceDirectory("level1", KeyType.LONG))); final FDBDatabase database = dbExtension.getDatabase(); @@ -328,101 +303,17 @@ public void testExportAllDataEmpty() { // Don't store any data try (FDBRecordContext context = database.openContext()) { KeySpacePath emptyPath = root.path("empty"); - RecordCursor cursor = emptyPath.exportAllData(context); - - List allData = cursor.asList().join(); - + final List allData = exportAllData(emptyPath, context); + // Should be empty assertEquals(0, allData.size()); } } @Test - public void testExportAllDataWithScanProperties() { - KeySpace root = new KeySpace( - new KeySpaceDirectory("limited", KeyType.STRING, "limited-scan") - .addSubdirectory(new KeySpaceDirectory("item", KeyType.LONG))); - - final FDBDatabase database = dbExtension.getDatabase(); - - // Store many records - try (FDBRecordContext context = database.openContext()) { - Transaction tr = context.ensureActive(); - KeySpacePath basePath = root.path("limited"); - - for (int i = 0; i < 20; i++) { - Tuple key = basePath.add("item", (long) i).toTuple(context); - tr.set(key.pack(), Tuple.from("item_data_" + i).pack()); - } - context.commit(); - } - - // Export with limited scan properties - try (FDBRecordContext context = database.openContext()) { - KeySpacePath limitedPath = root.path("limited"); - ScanProperties scanProps = ScanProperties.FORWARD_SCAN.with(props -> - props.setReturnedRowLimit(5)); - - RecordCursor cursor = limitedPath.exportAllData(context, scanProps); - - List limitedData = cursor.asList().join(); - - // Should have only 5 records due to limit - assertEquals(5, limitedData.size()); - - // Should be the first 5 items - for (int i = 0; i < 5; i++) { - String value = Tuple.fromBytes(limitedData.get(i).getValue()).getString(0); - assertEquals("item_data_" + i, value); - } - } - } - - @ParameterizedTest - @ValueSource(booleans = {true, false}) - public void testExportAllDataReverse(boolean reverse) { - KeySpace root = new KeySpace( - new KeySpaceDirectory("ordered", KeyType.STRING, "ordered-data") - .addSubdirectory(new KeySpaceDirectory("sequence", KeyType.LONG))); - - final FDBDatabase database = dbExtension.getDatabase(); - - // Store ordered data - try (FDBRecordContext context = database.openContext()) { - Transaction tr = context.ensureActive(); - KeySpacePath basePath = root.path("ordered"); - - for (int i = 0; i < 5; i++) { - Tuple key = basePath.add("sequence", (long) i).toTuple(context); - tr.set(key.pack(), Tuple.from("seq_" + i).pack()); - } - context.commit(); - } - - // Export with forward or reverse scan - try (FDBRecordContext context = database.openContext()) { - KeySpacePath orderedPath = root.path("ordered"); - ScanProperties scanProps = new ScanProperties(null, reverse); - - RecordCursor cursor = orderedPath.exportAllData(context, scanProps); - - List allData = cursor.asList().join(); - - assertEquals(5, allData.size()); - - // Verify order based on scan direction - for (int i = 0; i < 5; i++) { - String value = Tuple.fromBytes(allData.get(i).getValue()).getString(0); - int expectedIndex = reverse ? (4 - i) : i; - assertEquals("seq_" + expectedIndex, value); - } - } - } - - @Test - public void testExportAllDataWithDeepNestedStructure() { + void testExportAllDataWithDeepNestedStructure() { KeySpace root = new KeySpace( - new KeySpaceDirectory("org", KeyType.STRING, "company") + new KeySpaceDirectory("org", KeyType.STRING, UUID.randomUUID().toString()) .addSubdirectory(new KeySpaceDirectory("dept", KeyType.STRING)) .addSubdirectory(new KeySpaceDirectory("team", KeyType.LONG)) .addSubdirectory(new KeySpaceDirectory("member", KeyType.UUID)) @@ -459,10 +350,8 @@ public void testExportAllDataWithDeepNestedStructure() { // Export all data from organization root try (FDBRecordContext context = database.openContext()) { KeySpacePath orgPath = root.path("org"); - RecordCursor cursor = orgPath.exportAllData(context); - - List allData = cursor.asList().join(); - + final List allData = exportAllData(orgPath, context); + // Should have 16 records (2 depts * 2 teams * 2 members * 2 records each) assertEquals(16, allData.size()); } @@ -470,10 +359,8 @@ public void testExportAllDataWithDeepNestedStructure() { // Export data from specific department try (FDBRecordContext context = database.openContext()) { KeySpacePath engPath = root.path("org").add("dept", "engineering"); - RecordCursor cursor = engPath.exportAllData(context); - - List allData = cursor.asList().join(); - + final List allData = exportAllData(engPath, context); + // Should have 8 records (1 dept * 2 teams * 2 members * 2 records each) assertEquals(8, allData.size()); @@ -488,9 +375,9 @@ public void testExportAllDataWithDeepNestedStructure() { } @Test - public void testExportAllDataWithBinaryData() { + void testExportAllDataWithBinaryData() { KeySpace root = new KeySpace( - new KeySpaceDirectory("binary", KeyType.STRING, "binary-test") + new KeySpaceDirectory("binary", KeyType.STRING, UUID.randomUUID().toString()) .addSubdirectory(new KeySpaceDirectory("blob", KeyType.BYTES))); final FDBDatabase database = dbExtension.getDatabase(); @@ -518,10 +405,8 @@ public void testExportAllDataWithBinaryData() { // Export binary data try (FDBRecordContext context = database.openContext()) { KeySpacePath binaryPath = root.path("binary"); - RecordCursor cursor = binaryPath.exportAllData(context); - - List allData = cursor.asList().join(); - + final List allData = exportAllData(binaryPath, context); + assertEquals(3, allData.size()); // Verify binary data integrity @@ -533,397 +418,84 @@ public void testExportAllDataWithBinaryData() { } } - @Test - public void testExportAllDataCursorBehavior() { - KeySpace root = new KeySpace( - new KeySpaceDirectory("cursor", KeyType.STRING, "cursor-test") - .addSubdirectory(new KeySpaceDirectory("item", KeyType.LONG))); - - final FDBDatabase database = dbExtension.getDatabase(); - - // Store test data - try (FDBRecordContext context = database.openContext()) { - Transaction tr = context.ensureActive(); - KeySpacePath basePath = root.path("cursor"); - - for (int i = 0; i < 10; i++) { - Tuple key = basePath.add("item", (long) i).toTuple(context); - tr.set(key.pack(), Tuple.from("cursor_item_" + i).pack()); - } - context.commit(); - } - - // Test cursor behavior - try (FDBRecordContext context = database.openContext()) { - KeySpacePath cursorPath = root.path("cursor"); - RecordCursor cursor = cursorPath.exportAllData(context); - - // Test that cursor can be iterated - List collected = new ArrayList<>(); - RecordCursorResult result; - - while ((result = cursor.getNext()).hasNext()) { - collected.add(result.get()); - } - - assertEquals(10, collected.size()); - assertFalse(result.hasNext()); - - // Verify the reason for stopping - assertEquals(RecordCursor.NoNextReason.SOURCE_EXHAUSTED, result.getNoNextReason()); - } - } - - @Test - public void testExportAllDataWithContinuation() { + @ParameterizedTest + @ValueSource(ints = {1, 2, 3, 30}) + void testExportAllDataWithContinuation(int limit) { KeySpace root = new KeySpace( - new KeySpaceDirectory("continuation", KeyType.STRING, "continuation-test") + new KeySpaceDirectory("continuation", KeyType.STRING, UUID.randomUUID().toString()) .addSubdirectory(new KeySpaceDirectory("item", KeyType.LONG))); final FDBDatabase database = dbExtension.getDatabase(); // Store test data + final List> expected = new ArrayList<>(); + expected.add(new ArrayList<>()); try (FDBRecordContext context = database.openContext()) { Transaction tr = context.ensureActive(); KeySpacePath basePath = root.path("continuation"); - - for (int i = 0; i < 20; i++) { - Tuple key = basePath.add("item", (long) i).toTuple(context); - tr.set(key.pack(), Tuple.from("continuation_item_" + i).pack()); - } + + IntStream.range(0, 20).forEach(i -> { + Tuple key = basePath.add("item", (long)i).toTuple(context); + final Tuple value = Tuple.from("continuation_item_" + i); + tr.set(key.pack(), value.pack()); + if (expected.get(expected.size() - 1).size() == limit) { + expected.add(new ArrayList<>()); + } + expected.get(expected.size() - 1).add(value); + }); context.commit(); } // Export with continuation support try (FDBRecordContext context = database.openContext()) { KeySpacePath continuationPath = root.path("continuation"); - - // First export with limit to get continuation - ScanProperties limitedScan = ScanProperties.FORWARD_SCAN.with(props -> - props.setReturnedRowLimit(5)); - - RecordCursor cursor = continuationPath.exportAllData(context, null, limitedScan); - List firstBatch = cursor.asList().join(); - - assertEquals(5, firstBatch.size()); - - // Verify first batch contains items 0-4 - for (int i = 0; i < 5; i++) { - String value = Tuple.fromBytes(firstBatch.get(i).getValue()).getString(0); - assertEquals("continuation_item_" + i, value); - } - - // Get continuation from the cursor result - RecordCursorResult lastResult = cursor.getNext(); - assertFalse(lastResult.hasNext()); - assertEquals(RecordCursor.NoNextReason.RETURN_LIMIT_REACHED, lastResult.getNoNextReason()); - - byte[] continuation = lastResult.getContinuation().toBytes(); - assertNotNull(continuation); - - // Use continuation to get next batch - RecordCursor continuedCursor = continuationPath.exportAllData(context, continuation, limitedScan); - List secondBatch = continuedCursor.asList().join(); - - assertEquals(5, secondBatch.size()); - - // Verify second batch contains items 5-9 - for (int i = 0; i < 5; i++) { - String value = Tuple.fromBytes(secondBatch.get(i).getValue()).getString(0); - assertEquals("continuation_item_" + (i + 5), value); - } - } - } - @Test - public void testExportAllDataContinuationChaining() { - KeySpace root = new KeySpace( - new KeySpaceDirectory("chain", KeyType.STRING, "chain-test") - .addSubdirectory(new KeySpaceDirectory("batch", KeyType.LONG))); - - final FDBDatabase database = dbExtension.getDatabase(); - - // Store test data - try (FDBRecordContext context = database.openContext()) { - Transaction tr = context.ensureActive(); - KeySpacePath basePath = root.path("chain"); - - for (int i = 0; i < 30; i++) { - Tuple key = basePath.add("batch", (long) i).toTuple(context); - tr.set(key.pack(), Tuple.from("batch_item_" + i).pack()); - } - context.commit(); - } - - // Chain multiple continuations - try (FDBRecordContext context = database.openContext()) { - KeySpacePath chainPath = root.path("chain"); - ScanProperties batchScan = ScanProperties.FORWARD_SCAN.with(props -> - props.setReturnedRowLimit(7)); - - List allCollected = new ArrayList<>(); - byte[] continuation = null; - int batchCount = 0; - - do { - RecordCursor cursor = chainPath.exportAllData(context, continuation, batchScan); - List batch = cursor.asList().join(); - - if (batch.isEmpty()) { - break; - } - - allCollected.addAll(batch); - batchCount++; - - // Get continuation for next batch - RecordCursorResult lastResult = cursor.getNext(); - if (lastResult.hasNext() || lastResult.getNoNextReason() == RecordCursor.NoNextReason.RETURN_LIMIT_REACHED) { - continuation = lastResult.getContinuation().toBytes(); - } else { - continuation = null; - } - - // Safety check to avoid infinite loop - assertTrue(batchCount <= 10, "Too many batches, possible infinite loop"); - - } while (continuation != null); - - // Should have collected all 30 items across multiple batches - assertEquals(30, allCollected.size()); - assertEquals(5, batchCount); // 30 items / 7 per batch = 5 batches (last partial) - - // Verify all items are present and in order - for (int i = 0; i < 30; i++) { - String value = Tuple.fromBytes(allCollected.get(i).getValue()).getString(0); - assertEquals("batch_item_" + i, value); + final ScanProperties scanProperties = ScanProperties.FORWARD_SCAN.with(props -> props.setReturnedRowLimit(limit)); + List> actual = new ArrayList<>(); + RecordCursorContinuation continuation = RecordCursorStartContinuation.START; + while (!continuation.isEnd()) { + final RecordCursor cursor = continuationPath.exportAllData(context, continuation.toBytes(), + scanProperties); + final AtomicReference> lastResult = new AtomicReference<>(); + final List batch = cursor.asList(lastResult).join().stream() + .map(keyValue -> Tuple.fromBytes(keyValue.getKey())).collect(Collectors.toList()); + actual.add(batch); + continuation = lastResult.get().getContinuation(); } + assertEquals(expected, actual); } } - @Test - public void testExportAllDataContinuationWithDifferentScanProperties() { - KeySpace root = new KeySpace( - new KeySpaceDirectory("scan", KeyType.STRING, "scan-props-test") - .addSubdirectory(new KeySpaceDirectory("record", KeyType.LONG))); - - final FDBDatabase database = dbExtension.getDatabase(); - - // Store test data - try (FDBRecordContext context = database.openContext()) { - Transaction tr = context.ensureActive(); - KeySpacePath basePath = root.path("scan"); - - for (int i = 0; i < 15; i++) { - Tuple key = basePath.add("record", (long) i).toTuple(context); - tr.set(key.pack(), Tuple.from("scan_record_" + i).pack()); - } - context.commit(); - } - - // Test continuation with reverse scan - try (FDBRecordContext context = database.openContext()) { - KeySpacePath scanPath = root.path("scan"); - ScanProperties reverseScan = new ScanProperties(ExecuteProperties.newBuilder() - .setReturnedRowLimit(5) - .build(), true); // limit 5, reverse - - // First batch in reverse order - RecordCursor cursor = scanPath.exportAllData(context, null, reverseScan); - List firstBatch = cursor.asList().join(); - - assertEquals(5, firstBatch.size()); - - // Verify reverse order (should be items 14, 13, 12, 11, 10) - for (int i = 0; i < 5; i++) { - String value = Tuple.fromBytes(firstBatch.get(i).getValue()).getString(0); - assertEquals("scan_record_" + (14 - i), value); - } - - // Get continuation and continue reverse scan - RecordCursorResult lastResult = cursor.getNext(); - byte[] continuation = lastResult.getContinuation().toBytes(); - - RecordCursor continuedCursor = scanPath.exportAllData(context, continuation, reverseScan); - List secondBatch = continuedCursor.asList().join(); - - assertEquals(5, secondBatch.size()); - - // Verify second batch in reverse order (should be items 9, 8, 7, 6, 5) - for (int i = 0; i < 5; i++) { - String value = Tuple.fromBytes(secondBatch.get(i).getValue()).getString(0); - assertEquals("scan_record_" + (9 - i), value); - } - } - } - - @Test - public void testExportAllDataContinuationWithNestedPaths() { - KeySpace root = new KeySpace( - new KeySpaceDirectory("nested", KeyType.STRING, "nested-continuation") - .addSubdirectory(new KeySpaceDirectory("category", KeyType.STRING)) - .addSubdirectory(new KeySpaceDirectory("item", KeyType.LONG)) - .addSubdirectory(new KeySpaceDirectory("data", KeyType.NULL))); - - final FDBDatabase database = dbExtension.getDatabase(); - - // Store nested test data - try (FDBRecordContext context = database.openContext()) { - Transaction tr = context.ensureActive(); - - String[] categories = {"A", "B", "C"}; - for (String category : categories) { - for (int item = 0; item < 5; item++) { - KeySpacePath dataPath = root.path("nested") - .add("category", category) - .add("item", (long) item) - .add("data"); - - Tuple key = dataPath.toTuple(context); - tr.set(key.pack(), Tuple.from(category + "_item_" + item).pack()); - } - } - context.commit(); - } - - // Export with continuation from nested path - try (FDBRecordContext context = database.openContext()) { - KeySpacePath nestedPath = root.path("nested").add("category", "B"); - ScanProperties limitedScan = ScanProperties.FORWARD_SCAN.with(props -> - props.setReturnedRowLimit(3)); - - // First batch from category B - RecordCursor cursor = nestedPath.exportAllData(context, null, limitedScan); - List firstBatch = cursor.asList().join(); - - assertEquals(3, firstBatch.size()); - - // Verify all are from category B - for (KeyValue kv : firstBatch) { - String value = Tuple.fromBytes(kv.getValue()).getString(0); - assertTrue(value.startsWith("B_item_")); - } - - // Get continuation and get remaining items from category B - RecordCursorResult lastResult = cursor.getNext(); - byte[] continuation = lastResult.getContinuation().toBytes(); - - RecordCursor continuedCursor = nestedPath.exportAllData(context, continuation, limitedScan); - List secondBatch = continuedCursor.asList().join(); - - assertEquals(2, secondBatch.size()); // Only 2 remaining items in category B - - // Verify remaining items are from category B - for (KeyValue kv : secondBatch) { - String value = Tuple.fromBytes(kv.getValue()).getString(0); - assertTrue(value.startsWith("B_item_")); - } + private void setData(List keys, FDBRecordContext context, KeySpacePath basePath, + String subdirectory, String valuePrefix) { + Transaction tr = context.ensureActive(); + for (int i = 0; i < keys.size(); i++) { + Tuple tuple = basePath.add(subdirectory, keys.get(i)).toTuple(context); + tr.set(tuple.pack(), Tuple.from(valuePrefix + i).pack()); } } - @Test - public void testExportAllDataEmptyContinuation() { - KeySpace root = new KeySpace( - new KeySpaceDirectory("empty_cont", KeyType.STRING, "empty-continuation") - .addSubdirectory(new KeySpaceDirectory("item", KeyType.LONG))); - - final FDBDatabase database = dbExtension.getDatabase(); - - // Store minimal test data - try (FDBRecordContext context = database.openContext()) { - Transaction tr = context.ensureActive(); - KeySpacePath basePath = root.path("empty_cont"); - - for (int i = 0; i < 3; i++) { - Tuple key = basePath.add("item", (long) i).toTuple(context); - tr.set(key.pack(), Tuple.from("empty_cont_item_" + i).pack()); - } - context.commit(); - } - - // Test behavior when using continuation on empty results - try (FDBRecordContext context = database.openContext()) { - KeySpacePath emptyContPath = root.path("empty_cont"); - ScanProperties largeLimitScan = ScanProperties.FORWARD_SCAN.with(props -> - props.setReturnedRowLimit(10)); // Larger than available data - - // First export gets all data (no continuation needed) - RecordCursor cursor = emptyContPath.exportAllData(context, null, largeLimitScan); - List allData = cursor.asList().join(); - - assertEquals(3, allData.size()); - - // Get final result - RecordCursorResult finalResult = cursor.getNext(); - assertFalse(finalResult.hasNext()); - assertEquals(RecordCursor.NoNextReason.SOURCE_EXHAUSTED, finalResult.getNoNextReason()); - - // Try to use continuation (should return empty) - byte[] continuation = finalResult.getContinuation().toBytes(); - RecordCursor continuedCursor = emptyContPath.exportAllData(context, continuation, largeLimitScan); - List continuedData = continuedCursor.asList().join(); - - assertEquals(0, continuedData.size()); // Should be empty - } + private static List exportAllData(final KeySpacePath rootPath, final FDBRecordContext context) { + final List asSingleExport = rootPath.exportAllData(context, null, ScanProperties.FORWARD_SCAN).asList().join(); + + final List reversed = rootPath.exportAllData(context, null, ScanProperties.REVERSE_SCAN).asList().join(); + Collections.reverse(reversed); + assertEquals(asSingleExport, reversed); + + final ScanProperties scanProperties = ScanProperties.FORWARD_SCAN.with(props -> props.setReturnedRowLimit(1)); + List asContinuations = new ArrayList<>(); + RecordCursorContinuation continuation = RecordCursorStartContinuation.START; + while (!continuation.isEnd()) { + final RecordCursor cursor = rootPath.exportAllData(context, continuation.toBytes(), + scanProperties); + final AtomicReference> lastResult = new AtomicReference<>(); + final List batch = cursor.asList(lastResult).join(); + asContinuations.addAll(batch); + continuation = lastResult.get().getContinuation(); + assertEquals(1, batch.size()); + } + assertEquals(asSingleExport, asContinuations); + return asSingleExport; } - @Test - public void testExportAllDataContinuationConsistency() { - KeySpace root = new KeySpace( - new KeySpaceDirectory("consistency", KeyType.STRING, "consistency-test") - .addSubdirectory(new KeySpaceDirectory("sequence", KeyType.LONG))); - - final FDBDatabase database = dbExtension.getDatabase(); - - // Store test data - try (FDBRecordContext context = database.openContext()) { - Transaction tr = context.ensureActive(); - KeySpacePath basePath = root.path("consistency"); - - for (int i = 0; i < 12; i++) { - Tuple key = basePath.add("sequence", (long) i).toTuple(context); - tr.set(key.pack(), Tuple.from("consistency_seq_" + i).pack()); - } - context.commit(); - } - - // Test that continuation produces consistent, non-overlapping results - try (FDBRecordContext context = database.openContext()) { - KeySpacePath consistencyPath = root.path("consistency"); - ScanProperties batchScan = ScanProperties.FORWARD_SCAN.with(props -> - props.setReturnedRowLimit(4)); - - // Collect all data using continuations - List collectedValues = new ArrayList<>(); - byte[] continuation = null; - - for (int batch = 0; batch < 3; batch++) { // Expect 3 batches of 4 items each - RecordCursor cursor = consistencyPath.exportAllData(context, continuation, batchScan); - List batchData = cursor.asList().join(); - - assertEquals(4, batchData.size(), "Batch " + batch + " should have 4 items"); - - for (KeyValue kv : batchData) { - String value = Tuple.fromBytes(kv.getValue()).getString(0); - assertFalse(collectedValues.contains(value), "Duplicate value detected: " + value); - collectedValues.add(value); - } - - // Get continuation for next batch - RecordCursorResult lastResult = cursor.getNext(); - if (batch < 2) { // Not the last batch - assertEquals(RecordCursor.NoNextReason.RETURN_LIMIT_REACHED, lastResult.getNoNextReason()); - continuation = lastResult.getContinuation().toBytes(); - } else { // Last batch - assertEquals(RecordCursor.NoNextReason.SOURCE_EXHAUSTED, lastResult.getNoNextReason()); - } - } - - // Verify we got all 12 items in correct order - assertEquals(12, collectedValues.size()); - for (int i = 0; i < 12; i++) { - assertEquals("consistency_seq_" + i, collectedValues.get(i)); - } - } - } } From 9f7c8f24a1aeca1f990c9f7b6a80816708b90d6b Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Wed, 3 Sep 2025 11:56:45 -0400 Subject: [PATCH 05/32] Implement KeySpacePath.exportAllData, and update the tests to make it pass --- .../keyspace/KeySpacePathImpl.java | 12 +++-- .../keyspace/KeySpacePathDataExportTest.java | 44 +++++++++++-------- 2 files changed, 35 insertions(+), 21 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java index 0cd3e2d28d..29a403e2d7 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java @@ -27,6 +27,8 @@ import com.apple.foundationdb.record.ScanProperties; import com.apple.foundationdb.record.ValueRange; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; +import com.apple.foundationdb.record.provider.foundationdb.KeyValueCursor; +import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.ByteArrayUtil; import com.apple.foundationdb.tuple.Tuple; import com.google.common.collect.Lists; @@ -342,9 +344,13 @@ public String toString() { public RecordCursor exportAllData(@Nonnull FDBRecordContext context, @Nullable byte[] continuation, @Nonnull ScanProperties scanProperties) { - // TODO: Implement the actual data export functionality with continuation support - // This is a stub method that will be implemented later - throw new UnsupportedOperationException("exportAllData is not yet implemented"); + final Tuple tuple = toTuple(context); + + return KeyValueCursor.Builder.withSubspace(new Subspace(tuple)) + .setContext(context) + .setContinuation(continuation) + .setScanProperties(scanProperties) + .build(); } /** diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java index ddae929345..ac04000d7f 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java @@ -50,6 +50,7 @@ import java.util.stream.Collectors; import java.util.stream.IntStream; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -109,8 +110,8 @@ void testExportAllDataFromSimplePath() { void testExportAllDataFromSpecificSubPath() { KeySpace root = new KeySpace( new KeySpaceDirectory("app", KeyType.STRING, UUID.randomUUID().toString()) - .addSubdirectory(new KeySpaceDirectory("user", KeyType.LONG)) - .addSubdirectory(new KeySpaceDirectory("data", KeyType.NULL))); + .addSubdirectory(new KeySpaceDirectory("user", KeyType.LONG) + .addSubdirectory(new KeySpaceDirectory("data", KeyType.NULL)))); final FDBDatabase database = dbExtension.getDatabase(); @@ -151,8 +152,8 @@ void testExportAllDataFromSpecificSubPath() { void testExportAllDataWithDirectoryLayer() { KeySpace root = new KeySpace( new DirectoryLayerDirectory("env", UUID.randomUUID().toString()) - .addSubdirectory(new KeySpaceDirectory("tenant", KeyType.LONG)) - .addSubdirectory(new DirectoryLayerDirectory("service"))); + .addSubdirectory(new KeySpaceDirectory("tenant", KeyType.LONG) + .addSubdirectory(new DirectoryLayerDirectory("service")))); final FDBDatabase database = dbExtension.getDatabase(); @@ -256,8 +257,8 @@ void testExportAllDataWithDifferentKeyTypes() { void testExportAllDataWithConstantValues() { KeySpace root = new KeySpace( new KeySpaceDirectory("app", KeyType.STRING, UUID.randomUUID().toString()) - .addSubdirectory(new KeySpaceDirectory("version", KeyType.LONG, 1L)) - .addSubdirectory(new KeySpaceDirectory("data", KeyType.STRING, "records"))); + .addSubdirectory(new KeySpaceDirectory("version", KeyType.LONG, 1L) + .addSubdirectory(new KeySpaceDirectory("data", KeyType.STRING, "records")))); final FDBDatabase database = dbExtension.getDatabase(); @@ -314,10 +315,10 @@ void testExportAllDataEmpty() { void testExportAllDataWithDeepNestedStructure() { KeySpace root = new KeySpace( new KeySpaceDirectory("org", KeyType.STRING, UUID.randomUUID().toString()) - .addSubdirectory(new KeySpaceDirectory("dept", KeyType.STRING)) - .addSubdirectory(new KeySpaceDirectory("team", KeyType.LONG)) - .addSubdirectory(new KeySpaceDirectory("member", KeyType.UUID)) - .addSubdirectory(new KeySpaceDirectory("data", KeyType.NULL))); + .addSubdirectory(new KeySpaceDirectory("dept", KeyType.STRING) + .addSubdirectory(new KeySpaceDirectory("team", KeyType.LONG) + .addSubdirectory(new KeySpaceDirectory("member", KeyType.UUID) + .addSubdirectory(new KeySpaceDirectory("data", KeyType.NULL)))))); final FDBDatabase database = dbExtension.getDatabase(); @@ -428,8 +429,8 @@ void testExportAllDataWithContinuation(int limit) { final FDBDatabase database = dbExtension.getDatabase(); // Store test data - final List> expected = new ArrayList<>(); - expected.add(new ArrayList<>()); + final List> expectedBatches = new ArrayList<>(); + expectedBatches.add(new ArrayList<>()); try (FDBRecordContext context = database.openContext()) { Transaction tr = context.ensureActive(); KeySpacePath basePath = root.path("continuation"); @@ -438,13 +439,16 @@ void testExportAllDataWithContinuation(int limit) { Tuple key = basePath.add("item", (long)i).toTuple(context); final Tuple value = Tuple.from("continuation_item_" + i); tr.set(key.pack(), value.pack()); - if (expected.get(expected.size() - 1).size() == limit) { - expected.add(new ArrayList<>()); + if (expectedBatches.get(expectedBatches.size() - 1).size() == limit) { + expectedBatches.add(new ArrayList<>()); } - expected.get(expected.size() - 1).add(value); + expectedBatches.get(expectedBatches.size() - 1).add(value); }); context.commit(); } + if (20 % limit == 0) { + expectedBatches.add(List.of()); + } // Export with continuation support try (FDBRecordContext context = database.openContext()) { @@ -458,11 +462,11 @@ void testExportAllDataWithContinuation(int limit) { scanProperties); final AtomicReference> lastResult = new AtomicReference<>(); final List batch = cursor.asList(lastResult).join().stream() - .map(keyValue -> Tuple.fromBytes(keyValue.getKey())).collect(Collectors.toList()); + .map(keyValue -> Tuple.fromBytes(keyValue.getValue())).collect(Collectors.toList()); actual.add(batch); continuation = lastResult.get().getContinuation(); } - assertEquals(expected, actual); + assertEquals(expectedBatches, actual); } } @@ -492,7 +496,11 @@ private static List exportAllData(final KeySpacePath rootPath, final F final List batch = cursor.asList(lastResult).join(); asContinuations.addAll(batch); continuation = lastResult.get().getContinuation(); - assertEquals(1, batch.size()); + if (lastResult.get().hasNext()) { + assertEquals(1, batch.size()); + } else { + assertThat(batch.size()).isLessThanOrEqualTo(1); + } } assertEquals(asSingleExport, asContinuations); return asSingleExport; From 5cadd882499b34402874b47eb683e660216e4c36 Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Wed, 3 Sep 2025 12:06:13 -0400 Subject: [PATCH 06/32] Remove usage of asyncToSync in exportAllData --- .../keyspace/KeySpacePathImpl.java | 23 +++++++++++-------- 1 file changed, 13 insertions(+), 10 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java index 29a403e2d7..2f771eef05 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java @@ -20,18 +20,21 @@ package com.apple.foundationdb.record.provider.foundationdb.keyspace; +import com.apple.foundationdb.KeyValue; import com.apple.foundationdb.Range; import com.apple.foundationdb.async.AsyncUtil; import com.apple.foundationdb.record.RecordCoreArgumentException; import com.apple.foundationdb.record.RecordCursor; import com.apple.foundationdb.record.ScanProperties; import com.apple.foundationdb.record.ValueRange; +import com.apple.foundationdb.record.cursors.LazyCursor; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; import com.apple.foundationdb.record.provider.foundationdb.KeyValueCursor; import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.ByteArrayUtil; import com.apple.foundationdb.tuple.Tuple; import com.google.common.collect.Lists; + import javax.annotation.Nonnull; import javax.annotation.Nullable; import java.util.ArrayList; @@ -341,16 +344,16 @@ public String toString() { @Nonnull @Override - public RecordCursor exportAllData(@Nonnull FDBRecordContext context, - @Nullable byte[] continuation, - @Nonnull ScanProperties scanProperties) { - final Tuple tuple = toTuple(context); - - return KeyValueCursor.Builder.withSubspace(new Subspace(tuple)) - .setContext(context) - .setContinuation(continuation) - .setScanProperties(scanProperties) - .build(); + public RecordCursor exportAllData(@Nonnull FDBRecordContext context, + @Nullable byte[] continuation, + @Nonnull ScanProperties scanProperties) { + return new LazyCursor<>(toTupleAsync(context) + .thenApply(tuple -> KeyValueCursor.Builder.withSubspace(new Subspace(tuple)) + .setContext(context) + .setContinuation(continuation) + .setScanProperties(scanProperties) + .build()), + context.getExecutor()); } /** From 6b975d755939878ffbfb3d8c6bf997258e612902 Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Wed, 3 Sep 2025 14:25:46 -0400 Subject: [PATCH 07/32] Add initial implementation of a class for resolved path & value This will be returned by export --- .../keyspace/DataInKeySpacePath.java | 68 +++ .../keyspace/DataInKeySpacePathTest.java | 510 ++++++++++++++++++ 2 files changed, 578 insertions(+) create mode 100644 fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java create mode 100644 fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java new file mode 100644 index 0000000000..0ef0bc8ae6 --- /dev/null +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java @@ -0,0 +1,68 @@ +/* + * DataInKeySpacePath.java + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2015-2025 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.keyspace; + +import com.apple.foundationdb.KeyValue; +import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; +import com.apple.foundationdb.tuple.Tuple; + +import java.util.concurrent.CompletableFuture; + +/** + * Class representing a {@link KeyValue} pair within in {@link KeySpacePath}. + */ +public class DataInKeySpacePath { + + final CompletableFuture resolvedPath; + final KeyValue rawKeyValue; + + public DataInKeySpacePath(KeySpacePath path, KeyValue rawKeyValue, FDBRecordContext context) { + this.rawKeyValue = rawKeyValue; + + // Convert the raw key to a Tuple and resolve it starting from the provided path + Tuple keyTuple = Tuple.fromBytes(rawKeyValue.getKey()); + + // First resolve the provided path to get its resolved form + this.resolvedPath = path.toResolvedPathAsync(context).thenCompose(resolvedPath -> { + // Now use the resolved path to find the child for the key + // We need to figure out how much of the key corresponds to the resolved path + Tuple pathTuple = resolvedPath.toTuple(); + int pathLength = pathTuple.size(); + + // The remaining part of the key should be resolved from the resolved path's directory + if (keyTuple.size() > pathLength) { + // There's more in the key than just the path, so resolve the rest + return resolvedPath.getDirectory().findChildForKey(context, resolvedPath, keyTuple, keyTuple.size(), pathLength); + } else { + // The key exactly matches the path + return CompletableFuture.completedFuture(resolvedPath); + } + }); + } + + public CompletableFuture getResolvedPath() { + return resolvedPath; + } + + public KeyValue getRawKeyValue() { + return rawKeyValue; + } +} diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java new file mode 100644 index 0000000000..ad76adf512 --- /dev/null +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java @@ -0,0 +1,510 @@ +/* + * DataInKeySpacePathTest.java + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2015-2025 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.keyspace; + +import com.apple.foundationdb.KeyValue; +import com.apple.foundationdb.Transaction; +import com.apple.foundationdb.record.provider.foundationdb.FDBDatabase; +import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; +import com.apple.foundationdb.record.provider.foundationdb.keyspace.KeySpaceDirectory.KeyType; +import com.apple.foundationdb.record.test.FDBDatabaseExtension; +import com.apple.foundationdb.tuple.Tuple; +import com.apple.test.Tags; +import org.junit.jupiter.api.Tag; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import java.util.UUID; +import java.util.concurrent.CompletableFuture; + +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.assertTrue; + +/** + * Tests for {@link DataInKeySpacePath}. + */ +@Tag(Tags.RequiresFDB) +class DataInKeySpacePathTest { + @RegisterExtension + final FDBDatabaseExtension dbExtension = new FDBDatabaseExtension(); + + @Test + void testSimpleTwoLevelPath() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("app", KeyType.STRING, UUID.randomUUID().toString()) + .addSubdirectory(new KeySpaceDirectory("user", KeyType.LONG))); + + final FDBDatabase database = dbExtension.getDatabase(); + + // Store test data and create DataInKeySpacePath + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + + KeySpacePath userPath = root.path("app").add("user", 123L); + Tuple keyTuple = userPath.toTuple(context); + byte[] keyBytes = keyTuple.pack(); + byte[] valueBytes = Tuple.from("test_data").pack(); + + tr.set(keyBytes, valueBytes); + KeyValue keyValue = new KeyValue(keyBytes, valueBytes); + + // Create DataInKeySpacePath from the app-level path + KeySpacePath appPath = root.path("app"); + DataInKeySpacePath dataInPath = new DataInKeySpacePath(appPath, keyValue, context); + + // Verify the resolved path + CompletableFuture resolvedFuture = dataInPath.getResolvedPath(); + assertNotNull(resolvedFuture); + + ResolvedKeySpacePath resolved = resolvedFuture.join(); + assertNotNull(resolved); + + // Verify the resolved path has the correct structure + assertEquals("user", resolved.getDirectoryName()); + assertEquals(123L, resolved.getResolvedValue()); + + // Verify parent path + ResolvedKeySpacePath parent = resolved.getParent(); + assertNotNull(parent); + assertEquals("app", parent.getDirectoryName()); + + // Verify the resolved path can recreate the original key + Tuple resolvedTuple = resolved.toTuple(); + assertEquals(keyTuple, resolvedTuple); + + context.commit(); + } + } + + @Test + void testThreeLevelPathWithStringValues() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("service", KeyType.STRING, UUID.randomUUID().toString()) + .addSubdirectory(new KeySpaceDirectory("region", KeyType.STRING) + .addSubdirectory(new KeySpaceDirectory("instance", KeyType.STRING)))); + + final FDBDatabase database = dbExtension.getDatabase(); + + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + + KeySpacePath instancePath = root.path("service") + .add("region", "us-west-2") + .add("instance", "i-1234567890"); + + Tuple keyTuple = instancePath.toTuple(context); + byte[] keyBytes = keyTuple.pack(); + byte[] valueBytes = Tuple.from("instance_data").pack(); + + tr.set(keyBytes, valueBytes); + KeyValue keyValue = new KeyValue(keyBytes, valueBytes); + + // Create DataInKeySpacePath from the service-level path + KeySpacePath servicePath = root.path("service"); + DataInKeySpacePath dataInPath = new DataInKeySpacePath(servicePath, keyValue, context); + + ResolvedKeySpacePath resolved = dataInPath.getResolvedPath().join(); + + // Verify the deepest level + assertEquals("instance", resolved.getDirectoryName()); + assertEquals("i-1234567890", resolved.getResolvedValue()); + + // Verify middle level + ResolvedKeySpacePath regionLevel = resolved.getParent(); + assertNotNull(regionLevel); + assertEquals("region", regionLevel.getDirectoryName()); + assertEquals("us-west-2", regionLevel.getResolvedValue()); + + // Verify top level + ResolvedKeySpacePath serviceLevel = regionLevel.getParent(); + assertNotNull(serviceLevel); + assertEquals("service", serviceLevel.getDirectoryName()); + + // Verify the resolved path can recreate the original key + assertEquals(keyTuple, resolved.toTuple()); + + context.commit(); + } + } + + @Test + void testDeepFiveLevelPathWithMixedTypes() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("company", KeyType.STRING, UUID.randomUUID().toString()) + .addSubdirectory(new KeySpaceDirectory("department", KeyType.STRING) + .addSubdirectory(new KeySpaceDirectory("team_id", KeyType.LONG) + .addSubdirectory(new KeySpaceDirectory("employee_uuid", KeyType.UUID) + .addSubdirectory(new KeySpaceDirectory("active", KeyType.BOOLEAN)))))); + + final FDBDatabase database = dbExtension.getDatabase(); + + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + + UUID employeeId = UUID.randomUUID(); + KeySpacePath employeePath = root.path("company") + .add("department", "engineering") + .add("team_id", 42L) + .add("employee_uuid", employeeId) + .add("active", true); + + Tuple keyTuple = employeePath.toTuple(context); + byte[] keyBytes = keyTuple.pack(); + byte[] valueBytes = Tuple.from("employee_record").pack(); + + tr.set(keyBytes, valueBytes); + KeyValue keyValue = new KeyValue(keyBytes, valueBytes); + + // Create DataInKeySpacePath from the company-level path + KeySpacePath companyPath = root.path("company"); + DataInKeySpacePath dataInPath = new DataInKeySpacePath(companyPath, keyValue, context); + + ResolvedKeySpacePath resolved = dataInPath.getResolvedPath().join(); + + // Verify the deepest level (active) + assertEquals("active", resolved.getDirectoryName()); + assertEquals(true, resolved.getResolvedValue()); + + // Verify employee_uuid level + ResolvedKeySpacePath uuidLevel = resolved.getParent(); + assertNotNull(uuidLevel); + assertEquals("employee_uuid", uuidLevel.getDirectoryName()); + assertEquals(employeeId, uuidLevel.getResolvedValue()); + + // Verify team_id level + ResolvedKeySpacePath teamLevel = uuidLevel.getParent(); + assertNotNull(teamLevel); + assertEquals("team_id", teamLevel.getDirectoryName()); + assertEquals(42L, teamLevel.getResolvedValue()); + + // Verify department level + ResolvedKeySpacePath deptLevel = teamLevel.getParent(); + assertNotNull(deptLevel); + assertEquals("department", deptLevel.getDirectoryName()); + assertEquals("engineering", deptLevel.getResolvedValue()); + + // Verify company level + ResolvedKeySpacePath companyLevel = deptLevel.getParent(); + assertNotNull(companyLevel); + assertEquals("company", companyLevel.getDirectoryName()); + + // Verify the resolved path can recreate the original key + assertEquals(keyTuple, resolved.toTuple()); + + context.commit(); + } + } + + @Test + void testPathWithConstantValues() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("application", KeyType.STRING, UUID.randomUUID().toString()) + .addSubdirectory(new KeySpaceDirectory("version", KeyType.LONG, 1L) + .addSubdirectory(new KeySpaceDirectory("environment", KeyType.STRING, "production") + .addSubdirectory(new KeySpaceDirectory("data", KeyType.STRING))))); + + final FDBDatabase database = dbExtension.getDatabase(); + + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + + KeySpacePath dataPath = root.path("application") + .add("version") // Uses constant value 1L + .add("environment") // Uses constant value "production" + .add("data", "user_records"); + + Tuple keyTuple = dataPath.toTuple(context); + byte[] keyBytes = keyTuple.pack(); + byte[] valueBytes = Tuple.from("constant_test_data").pack(); + + tr.set(keyBytes, valueBytes); + KeyValue keyValue = new KeyValue(keyBytes, valueBytes); + + // Create DataInKeySpacePath from the application-level path + KeySpacePath appPath = root.path("application"); + DataInKeySpacePath dataInPath = new DataInKeySpacePath(appPath, keyValue, context); + + ResolvedKeySpacePath resolved = dataInPath.getResolvedPath().join(); + + // Verify the deepest level + assertEquals("data", resolved.getDirectoryName()); + assertEquals("user_records", resolved.getResolvedValue()); + + // Verify environment level (constant value) + ResolvedKeySpacePath envLevel = resolved.getParent(); + assertNotNull(envLevel); + assertEquals("environment", envLevel.getDirectoryName()); + assertEquals("production", envLevel.getResolvedValue()); + + // Verify version level (constant value) + ResolvedKeySpacePath versionLevel = envLevel.getParent(); + assertNotNull(versionLevel); + assertEquals("version", versionLevel.getDirectoryName()); + assertEquals(1L, versionLevel.getResolvedValue()); + + // Verify application level + ResolvedKeySpacePath applicationLevel = versionLevel.getParent(); + assertNotNull(applicationLevel); + assertEquals("application", applicationLevel.getDirectoryName()); + + // Verify the resolved path can recreate the original key + assertEquals(keyTuple, resolved.toTuple()); + + context.commit(); + } + } + + @Test + void testPathWithDirectoryLayer() { + KeySpace root = new KeySpace( + new DirectoryLayerDirectory("tenant", UUID.randomUUID().toString()) + .addSubdirectory(new KeySpaceDirectory("user_id", KeyType.LONG) + .addSubdirectory(new DirectoryLayerDirectory("service")))); + + final FDBDatabase database = dbExtension.getDatabase(); + + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + + KeySpacePath servicePath = root.path("tenant") + .add("user_id", 999L) + .add("service", "analytics"); + + Tuple keyTuple = servicePath.toTuple(context); + byte[] keyBytes = keyTuple.pack(); + byte[] valueBytes = Tuple.from("directory_layer_data").pack(); + + tr.set(keyBytes, valueBytes); + KeyValue keyValue = new KeyValue(keyBytes, valueBytes); + + // Create DataInKeySpacePath from the tenant-level path + KeySpacePath tenantPath = root.path("tenant"); + DataInKeySpacePath dataInPath = new DataInKeySpacePath(tenantPath, keyValue, context); + + ResolvedKeySpacePath resolved = dataInPath.getResolvedPath().join(); + + // Verify the deepest level (service - DirectoryLayer) + assertEquals("service", resolved.getDirectoryName()); + assertEquals("analytics", resolved.getLogicalValue()); + + // Verify user_id level + ResolvedKeySpacePath userLevel = resolved.getParent(); + assertNotNull(userLevel); + assertEquals("user_id", userLevel.getDirectoryName()); + assertEquals(999L, userLevel.getResolvedValue()); + + // Verify tenant level (DirectoryLayer) + ResolvedKeySpacePath tenantLevel = userLevel.getParent(); + assertNotNull(tenantLevel); + assertEquals("tenant", tenantLevel.getDirectoryName()); + + // Note: DirectoryLayer values are resolved asynchronously, so we verify the structure is correct + assertNotNull(tenantLevel.getResolvedValue()); + + context.commit(); + } + } + + @Test + void testPathWithBinaryData() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("binary_store", KeyType.STRING, UUID.randomUUID().toString()) + .addSubdirectory(new KeySpaceDirectory("blob_id", KeyType.BYTES))); + + final FDBDatabase database = dbExtension.getDatabase(); + + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + + byte[] blobId = {0x01, 0x02, 0x03, (byte) 0xFF, (byte) 0xFE}; + KeySpacePath blobPath = root.path("binary_store").add("blob_id", blobId); + + Tuple keyTuple = blobPath.toTuple(context); + byte[] keyBytes = keyTuple.pack(); + byte[] valueBytes = "binary_test_data".getBytes(); + + tr.set(keyBytes, valueBytes); + KeyValue keyValue = new KeyValue(keyBytes, valueBytes); + + // Create DataInKeySpacePath from the binary_store-level path + KeySpacePath storePath = root.path("binary_store"); + DataInKeySpacePath dataInPath = new DataInKeySpacePath(storePath, keyValue, context); + + ResolvedKeySpacePath resolved = dataInPath.getResolvedPath().join(); + + // Verify the deepest level + assertEquals("blob_id", resolved.getDirectoryName()); + byte[] resolvedBytes = (byte[]) resolved.getResolvedValue(); + assertNotNull(resolvedBytes); + assertEquals(5, resolvedBytes.length); + assertEquals(0x01, resolvedBytes[0]); + assertEquals((byte) 0xFF, resolvedBytes[3]); + + // Verify parent level + ResolvedKeySpacePath storeLevel = resolved.getParent(); + assertNotNull(storeLevel); + assertEquals("binary_store", storeLevel.getDirectoryName()); + + // Verify the resolved path can recreate the original key + assertEquals(keyTuple, resolved.toTuple()); + + context.commit(); + } + } + + @ParameterizedTest + @ValueSource(ints = {1, 2, 3, 4, 5}) + void testVariableDepthPaths(int depth) { + // Build a KeySpace with the specified depth + KeySpaceDirectory rootDir = new KeySpaceDirectory("root", KeyType.STRING, UUID.randomUUID().toString()); + KeySpace root = new KeySpace(rootDir); + KeySpaceDirectory dir = rootDir; + for (int i = 1; i < depth; i++) { + final KeySpaceDirectory next = new KeySpaceDirectory("level" + i, KeyType.LONG); + dir.addSubdirectory(next); + dir = next; + } + + final FDBDatabase database = dbExtension.getDatabase(); + + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + + // Build a path with the specified depth + KeySpacePath currentPath = root.path("root"); + for (int i = 1; i < depth; i++) { + currentPath = currentPath.add("level" + i, i * 100L); + } + + Tuple keyTuple = currentPath.toTuple(context); + byte[] keyBytes = keyTuple.pack(); + byte[] valueBytes = Tuple.from("depth_test_" + depth).pack(); + + tr.set(keyBytes, valueBytes); + KeyValue keyValue = new KeyValue(keyBytes, valueBytes); + + // Create DataInKeySpacePath from the root-level path + KeySpacePath rootPath = root.path("root"); + root.resolveFromKey(context, Tuple.fromBytes(keyValue.getKey())); + DataInKeySpacePath dataInPath = new DataInKeySpacePath(rootPath, keyValue, context); + + ResolvedKeySpacePath resolved = dataInPath.getResolvedPath().join(); + + // Verify the depth by traversing up the path + int actualDepth = 1; // Start at 1 for the root + ResolvedKeySpacePath current = resolved; + while (current.getParent() != null) { + actualDepth++; + current = current.getParent(); + } + assertEquals(depth, actualDepth); + + // Verify the deepest level has the expected name and value + if (depth > 1) { + assertEquals("level" + (depth - 1), resolved.getDirectoryName()); + assertEquals((depth - 1) * 100L, resolved.getResolvedValue()); + } else { + assertEquals("root", resolved.getDirectoryName()); + } + + // Verify the resolved path can recreate the original key + assertEquals(keyTuple, resolved.toTuple()); + + context.commit(); + } + } + + @Test + void testKeyValueAccessors() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("test", KeyType.STRING, UUID.randomUUID().toString())); + + final FDBDatabase database = dbExtension.getDatabase(); + + try (FDBRecordContext context = database.openContext()) { + KeySpacePath testPath = root.path("test"); + Tuple keyTuple = testPath.toTuple(context); + byte[] keyBytes = keyTuple.pack(); + byte[] valueBytes = Tuple.from("accessor_test").pack(); + + KeyValue originalKeyValue = new KeyValue(keyBytes, valueBytes); + DataInKeySpacePath dataInPath = new DataInKeySpacePath(testPath, originalKeyValue, context); + + // Verify accessor methods + KeyValue retrievedKeyValue = dataInPath.getRawKeyValue(); + assertNotNull(retrievedKeyValue); + assertEquals(originalKeyValue.getKey(), retrievedKeyValue.getKey()); + assertEquals(originalKeyValue.getValue(), retrievedKeyValue.getValue()); + + // Verify resolved path future is not null + CompletableFuture resolvedFuture = dataInPath.getResolvedPath(); + assertNotNull(resolvedFuture); + assertTrue(resolvedFuture.isDone() || !resolvedFuture.isCancelled()); + } + } + + @Test + void testNullKeyTypeDirectory() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("base", KeyType.STRING, UUID.randomUUID().toString()) + .addSubdirectory(new KeySpaceDirectory("null_dir", KeyType.NULL))); + + final FDBDatabase database = dbExtension.getDatabase(); + + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + + KeySpacePath nullPath = root.path("base").add("null_dir"); + + Tuple keyTuple = nullPath.toTuple(context); + byte[] keyBytes = keyTuple.pack(); + byte[] valueBytes = Tuple.from("null_type_test").pack(); + + tr.set(keyBytes, valueBytes); + KeyValue keyValue = new KeyValue(keyBytes, valueBytes); + + // Create DataInKeySpacePath from the base-level path + KeySpacePath basePath = root.path("base"); + DataInKeySpacePath dataInPath = new DataInKeySpacePath(basePath, keyValue, context); + + ResolvedKeySpacePath resolved = dataInPath.getResolvedPath().join(); + + // Verify the deepest level (NULL type) + assertEquals("null_dir", resolved.getDirectoryName()); + // NULL type directories have null as their resolved value + assertNull(resolved.getResolvedValue()); + + // Verify parent level + ResolvedKeySpacePath baseLevel = resolved.getParent(); + assertNotNull(baseLevel); + assertEquals("base", baseLevel.getDirectoryName()); + + // Verify the resolved path can recreate the original key + assertEquals(keyTuple, resolved.toTuple()); + + context.commit(); + } + } +} From ed9702b99fc91ab5f183f9002b887350e0748b5c Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Wed, 3 Sep 2025 14:54:03 -0400 Subject: [PATCH 08/32] Validate remainder in more places Most of the time this will be used with remainders, so make most of the tests cover that --- .../keyspace/DataInKeySpacePathTest.java | 75 ++++++++++++++----- 1 file changed, 58 insertions(+), 17 deletions(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java index ad76adf512..86860908d5 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java @@ -26,7 +26,9 @@ import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; import com.apple.foundationdb.record.provider.foundationdb.keyspace.KeySpaceDirectory.KeyType; import com.apple.foundationdb.record.test.FDBDatabaseExtension; +import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.Tuple; +import com.apple.foundationdb.tuple.TupleHelpers; import com.apple.test.Tags; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; @@ -63,8 +65,10 @@ void testSimpleTwoLevelPath() { Transaction tr = context.ensureActive(); KeySpacePath userPath = root.path("app").add("user", 123L); - Tuple keyTuple = userPath.toTuple(context); - byte[] keyBytes = keyTuple.pack(); + final Subspace pathSubspace = userPath.toSubspace(context); + + // Add additional tuple elements after the KeySpacePath (this is how data is actually stored) + byte[] keyBytes = pathSubspace.pack(Tuple.from("record_id", 456L, "metadata")); byte[] valueBytes = Tuple.from("test_data").pack(); tr.set(keyBytes, valueBytes); @@ -90,9 +94,17 @@ void testSimpleTwoLevelPath() { assertNotNull(parent); assertEquals("app", parent.getDirectoryName()); - // Verify the resolved path can recreate the original key + // Verify the resolved path recreates the KeySpacePath portion (not the full key) Tuple resolvedTuple = resolved.toTuple(); - assertEquals(keyTuple, resolvedTuple); + assertEquals(TupleHelpers.subTuple(Tuple.fromBytes(keyBytes), 0, 2), resolvedTuple); + + // Verify that the remainder contains the additional tuple elements + Tuple remainder = resolved.getRemainder(); + assertNotNull(remainder); + assertEquals(3, remainder.size()); + assertEquals("record_id", remainder.getString(0)); + assertEquals(456L, remainder.getLong(1)); + assertEquals("metadata", remainder.getString(2)); context.commit(); } @@ -114,8 +126,9 @@ void testThreeLevelPathWithStringValues() { .add("region", "us-west-2") .add("instance", "i-1234567890"); - Tuple keyTuple = instancePath.toTuple(context); - byte[] keyBytes = keyTuple.pack(); + // Add additional tuple elements after the KeySpacePath + byte[] keyBytes = instancePath.toSubspace(context).pack( + Tuple.from("process_id", "web-server", "port", 8080L)); byte[] valueBytes = Tuple.from("instance_data").pack(); tr.set(keyBytes, valueBytes); @@ -142,8 +155,17 @@ void testThreeLevelPathWithStringValues() { assertNotNull(serviceLevel); assertEquals("service", serviceLevel.getDirectoryName()); - // Verify the resolved path can recreate the original key - assertEquals(keyTuple, resolved.toTuple()); + // Verify the resolved path recreates the KeySpacePath portion + assertEquals(TupleHelpers.subTuple(Tuple.fromBytes(keyBytes), 0, 3), resolved.toTuple()); + + // Verify that the remainder contains the additional tuple elements + Tuple remainder = resolved.getRemainder(); + assertNotNull(remainder); + assertEquals(4, remainder.size()); + assertEquals("process_id", remainder.getString(0)); + assertEquals("web-server", remainder.getString(1)); + assertEquals("port", remainder.getString(2)); + assertEquals(8080L, remainder.getLong(3)); context.commit(); } @@ -170,8 +192,9 @@ void testDeepFiveLevelPathWithMixedTypes() { .add("employee_uuid", employeeId) .add("active", true); - Tuple keyTuple = employeePath.toTuple(context); - byte[] keyBytes = keyTuple.pack(); + // Add additional tuple elements after the KeySpacePath + byte[] keyBytes = employeePath.toSubspace(context).pack( + Tuple.from("salary", 75000L, "start_date", "2023-01-15")); byte[] valueBytes = Tuple.from("employee_record").pack(); tr.set(keyBytes, valueBytes); @@ -210,8 +233,17 @@ void testDeepFiveLevelPathWithMixedTypes() { assertNotNull(companyLevel); assertEquals("company", companyLevel.getDirectoryName()); - // Verify the resolved path can recreate the original key - assertEquals(keyTuple, resolved.toTuple()); + // Verify the resolved path recreates the KeySpacePath portion + assertEquals(TupleHelpers.subTuple(Tuple.fromBytes(keyBytes), 0, 5), resolved.toTuple()); + + // Verify that the remainder contains the additional tuple elements + Tuple remainder = resolved.getRemainder(); + assertNotNull(remainder); + assertEquals(4, remainder.size()); + assertEquals("salary", remainder.getString(0)); + assertEquals(75000L, remainder.getLong(1)); + assertEquals("start_date", remainder.getString(2)); + assertEquals("2023-01-15", remainder.getString(3)); context.commit(); } @@ -234,9 +266,9 @@ void testPathWithConstantValues() { .add("version") // Uses constant value 1L .add("environment") // Uses constant value "production" .add("data", "user_records"); - - Tuple keyTuple = dataPath.toTuple(context); - byte[] keyBytes = keyTuple.pack(); + // Add additional tuple elements after the KeySpacePath + byte[] keyBytes = dataPath.toSubspace(context).pack( + Tuple.from("config_id", 1001L, "version", "v2.1")); byte[] valueBytes = Tuple.from("constant_test_data").pack(); tr.set(keyBytes, valueBytes); @@ -269,8 +301,17 @@ void testPathWithConstantValues() { assertNotNull(applicationLevel); assertEquals("application", applicationLevel.getDirectoryName()); - // Verify the resolved path can recreate the original key - assertEquals(keyTuple, resolved.toTuple()); + // Verify the resolved path recreates the KeySpacePath portion + assertEquals(TupleHelpers.subTuple(Tuple.fromBytes(keyBytes), 0, 4), resolved.toTuple()); + + // Verify that the remainder contains the additional tuple elements + Tuple remainder = resolved.getRemainder(); + assertNotNull(remainder); + assertEquals(4, remainder.size()); + assertEquals("config_id", remainder.getString(0)); + assertEquals(1001L, remainder.getLong(1)); + assertEquals("version", remainder.getString(2)); + assertEquals("v2.1", remainder.getString(3)); context.commit(); } From 0820b9731ac78a3470078d2506a0376ee1836b0f Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Wed, 3 Sep 2025 14:57:36 -0400 Subject: [PATCH 09/32] Cleanup some of the assertions --- .../foundationdb/keyspace/DataInKeySpacePathTest.java | 8 +++----- 1 file changed, 3 insertions(+), 5 deletions(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java index 86860908d5..34a337004d 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java @@ -39,6 +39,7 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; +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; @@ -354,7 +355,7 @@ void testPathWithDirectoryLayer() { ResolvedKeySpacePath userLevel = resolved.getParent(); assertNotNull(userLevel); assertEquals("user_id", userLevel.getDirectoryName()); - assertEquals(999L, userLevel.getResolvedValue()); + assertEquals(999L, userLevel.getLogicalValue()); // Verify tenant level (DirectoryLayer) ResolvedKeySpacePath tenantLevel = userLevel.getParent(); @@ -398,10 +399,7 @@ void testPathWithBinaryData() { // Verify the deepest level assertEquals("blob_id", resolved.getDirectoryName()); byte[] resolvedBytes = (byte[]) resolved.getResolvedValue(); - assertNotNull(resolvedBytes); - assertEquals(5, resolvedBytes.length); - assertEquals(0x01, resolvedBytes[0]); - assertEquals((byte) 0xFF, resolvedBytes[3]); + assertArrayEquals(blobId, resolvedBytes); // Verify parent level ResolvedKeySpacePath storeLevel = resolved.getParent(); From 0750e00cda42b5e5f365fbf4c8c697a6ed259253 Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Wed, 3 Sep 2025 15:24:27 -0400 Subject: [PATCH 10/32] Change exportAllData to return DataInKeySpacePath instead of raw data --- .../foundationdb/keyspace/KeySpacePath.java | 6 ++-- .../keyspace/KeySpacePathImpl.java | 10 +++---- .../keyspace/KeySpacePathWrapper.java | 6 ++-- .../keyspace/KeySpacePathDataExportTest.java | 29 ++++++++++--------- 4 files changed, 27 insertions(+), 24 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java index 4c06081765..501079240d 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java @@ -579,7 +579,7 @@ default List listSubdirectory(@Nonnull FDBRecordContext co */ @API(API.Status.UNSTABLE) @Nonnull - RecordCursor exportAllData(@Nonnull FDBRecordContext context, - @Nullable byte[] continuation, - @Nonnull ScanProperties scanProperties); + RecordCursor exportAllData(@Nonnull FDBRecordContext context, + @Nullable byte[] continuation, + @Nonnull ScanProperties scanProperties); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java index 2f771eef05..fc8fa45ab5 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java @@ -20,7 +20,6 @@ package com.apple.foundationdb.record.provider.foundationdb.keyspace; -import com.apple.foundationdb.KeyValue; import com.apple.foundationdb.Range; import com.apple.foundationdb.async.AsyncUtil; import com.apple.foundationdb.record.RecordCoreArgumentException; @@ -344,16 +343,17 @@ public String toString() { @Nonnull @Override - public RecordCursor exportAllData(@Nonnull FDBRecordContext context, - @Nullable byte[] continuation, - @Nonnull ScanProperties scanProperties) { + public RecordCursor exportAllData(@Nonnull FDBRecordContext context, + @Nullable byte[] continuation, + @Nonnull ScanProperties scanProperties) { return new LazyCursor<>(toTupleAsync(context) .thenApply(tuple -> KeyValueCursor.Builder.withSubspace(new Subspace(tuple)) .setContext(context) .setContinuation(continuation) .setScanProperties(scanProperties) .build()), - context.getExecutor()); + context.getExecutor()) + .map(keyValue -> new DataInKeySpacePath(this, keyValue, context)); } /** diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathWrapper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathWrapper.java index cd39f2c5f0..54d794f2a7 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathWrapper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathWrapper.java @@ -229,9 +229,9 @@ public String toString(@Nonnull Tuple t) { @Nonnull @Override - public RecordCursor exportAllData(@Nonnull FDBRecordContext context, - @Nullable byte[] continuation, - @Nonnull ScanProperties scanProperties) { + public RecordCursor exportAllData(@Nonnull FDBRecordContext context, + @Nullable byte[] continuation, + @Nonnull ScanProperties scanProperties) { return inner.exportAllData(context, continuation, scanProperties); } } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java index ac04000d7f..a428f37b6c 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java @@ -47,7 +47,6 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; -import java.util.stream.Collectors; import java.util.stream.IntStream; import static org.assertj.core.api.Assertions.assertThat; @@ -458,13 +457,15 @@ void testExportAllDataWithContinuation(int limit) { List> actual = new ArrayList<>(); RecordCursorContinuation continuation = RecordCursorStartContinuation.START; while (!continuation.isEnd()) { - final RecordCursor cursor = continuationPath.exportAllData(context, continuation.toBytes(), + final RecordCursor cursor = continuationPath.exportAllData(context, continuation.toBytes(), scanProperties); - final AtomicReference> lastResult = new AtomicReference<>(); - final List batch = cursor.asList(lastResult).join().stream() - .map(keyValue -> Tuple.fromBytes(keyValue.getValue())).collect(Collectors.toList()); + final AtomicReference> tupleResult = new AtomicReference<>(); + final List batch = cursor.map(dataInPath -> { + KeyValue kv = dataInPath.getRawKeyValue(); + return Tuple.fromBytes(kv.getValue()); + }).asList(tupleResult).join(); actual.add(batch); - continuation = lastResult.get().getContinuation(); + continuation = tupleResult.get().getContinuation(); } assertEquals(expectedBatches, actual); } @@ -480,9 +481,11 @@ private void setData(List keys, FDBRecordContext context, KeySpacePath b } private static List exportAllData(final KeySpacePath rootPath, final FDBRecordContext context) { - final List asSingleExport = rootPath.exportAllData(context, null, ScanProperties.FORWARD_SCAN).asList().join(); + final List asSingleExport = rootPath.exportAllData(context, null, ScanProperties.FORWARD_SCAN) + .map(DataInKeySpacePath::getRawKeyValue).asList().join(); - final List reversed = rootPath.exportAllData(context, null, ScanProperties.REVERSE_SCAN).asList().join(); + final List reversed = rootPath.exportAllData(context, null, ScanProperties.REVERSE_SCAN) + .map(DataInKeySpacePath::getRawKeyValue).asList().join(); Collections.reverse(reversed); assertEquals(asSingleExport, reversed); @@ -490,13 +493,13 @@ private static List exportAllData(final KeySpacePath rootPath, final F List asContinuations = new ArrayList<>(); RecordCursorContinuation continuation = RecordCursorStartContinuation.START; while (!continuation.isEnd()) { - final RecordCursor cursor = rootPath.exportAllData(context, continuation.toBytes(), + final RecordCursor cursor = rootPath.exportAllData(context, continuation.toBytes(), scanProperties); - final AtomicReference> lastResult = new AtomicReference<>(); - final List batch = cursor.asList(lastResult).join(); + final AtomicReference> keyValueResult = new AtomicReference<>(); + final List batch = cursor.map(DataInKeySpacePath::getRawKeyValue).asList(keyValueResult).join(); asContinuations.addAll(batch); - continuation = lastResult.get().getContinuation(); - if (lastResult.get().hasNext()) { + continuation = keyValueResult.get().getContinuation(); + if (keyValueResult.get().hasNext()) { assertEquals(1, batch.size()); } else { assertThat(batch.size()).isLessThanOrEqualTo(1); From a6572f13fbc1bdb08535fe149b55e92531433590 Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Thu, 4 Sep 2025 10:10:48 -0400 Subject: [PATCH 11/32] Move EnvironmentKeySpace out into its own file, move wrappers into it --- .../keyspace/EnvironmentKeySpace.java | 159 ++++++++++++++++++ .../keyspace/KeySpaceDirectoryTest.java | 158 ++--------------- 2 files changed, 171 insertions(+), 146 deletions(-) create mode 100644 fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/EnvironmentKeySpace.java diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/EnvironmentKeySpace.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/EnvironmentKeySpace.java new file mode 100644 index 0000000000..e64044058d --- /dev/null +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/EnvironmentKeySpace.java @@ -0,0 +1,159 @@ +/* + * EnvironmentKeySpace.java + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2015-2025 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.keyspace; + +import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; +import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStore; +import com.apple.foundationdb.tuple.Tuple; + +/** + * This provides an example of a way in which you can define a KeySpace in a relatively clean and type-safe + * manner. It defines a keyspace that looks like: + *
+ *    [environment]           - A string the identifies the logical environment (like prod, test, qa, etc.).
+ *      |                       This string is converted by the directory layer as a small integer value.
+ *      +- userid             - An integer ID for each user in the system
+ *         |
+ *         +- [application]   - Tne name of an application the user runs (again, converted by the directory
+ *            |                 layer into a small integer value)
+ *            +- data=1       - Constant value of "1", which is the location of a {@link FDBRecordStore}
+ *            |                 in which application data is to be stored
+ *            +- metadata=2   - Constant value of "2", which is the Location of another FDBRecordStore
+ *                              in which application metadata or configuration information can live.
+ * 
+ * The main point of this class is to demonstrate how you can use the KeySpacePath wrapping facility to provide + * implementations of the path elements that are meaningful to your application environment and type safe. + */ +class EnvironmentKeySpace { + private final KeySpace root; + private final String rootName; + + static final String USER_KEY = "userid"; + static final String APPLICATION_KEY = "application"; + static final String DATA_KEY = "data"; + static final long DATA_VALUE = 1L; + static final String METADATA_KEY = "metadata"; + static final long METADATA_VALUE = 2L; + + /** + * The EnvironmentKeySpace scopes all of the data it stores underneath of a rootName, + * for example, you could define an instance for prod, test, qa, etc. + * + * @param rootName The root name underwhich all data is stored. + */ + public EnvironmentKeySpace(String rootName) { + this.rootName = rootName; + root = new KeySpace( + new DirectoryLayerDirectory(rootName, rootName, EnvironmentRoot::new) + .addSubdirectory(new KeySpaceDirectory(USER_KEY, KeySpaceDirectory.KeyType.LONG, UserPath::new) + .addSubdirectory(new DirectoryLayerDirectory(APPLICATION_KEY, ApplicationPath::new) + .addSubdirectory(new KeySpaceDirectory(DATA_KEY, KeySpaceDirectory.KeyType.LONG, DATA_VALUE, DataPath::new)) + .addSubdirectory(new KeySpaceDirectory(METADATA_KEY, KeySpaceDirectory.KeyType.LONG, METADATA_VALUE, MetadataPath::new))))); + } + + public String getRootName() { + return rootName; + } + + /** + * Returns an implementation of a KeySpacePath that represents the start of the environment. + */ + public EnvironmentRoot root() { + return (EnvironmentRoot)root.path(rootName); + } + + /** + * Given a tuple that represents an FDB key that came from this KeySpace, returns the leaf-most path + * element in which the tuple resides. + */ + public ResolvedKeySpacePath fromKey(FDBRecordContext context, Tuple tuple) { + return root.resolveFromKey(context, tuple); + } + + /** + * A KeySpacePath that represents the logical root of the environment. + */ + static class EnvironmentRoot extends KeySpacePathWrapper { + public EnvironmentRoot(KeySpacePath path) { + super(path); + } + + public KeySpacePath parent() { + return null; + } + + public UserPath userid(long userid) { + return (UserPath) inner.add(USER_KEY, userid); + } + } + + static class UserPath extends KeySpacePathWrapper { + public UserPath(KeySpacePath path) { + super(path); + } + + public ApplicationPath application(String applicationName) { + return (ApplicationPath) inner.add(APPLICATION_KEY, applicationName); + } + + public EnvironmentRoot parent() { + return (EnvironmentRoot) inner.getParent(); + } + } + + static class ApplicationPath extends KeySpacePathWrapper { + public ApplicationPath(KeySpacePath path) { + super(path); + } + + public DataPath dataStore() { + return (DataPath) inner.add(DATA_KEY); + } + + public MetadataPath metadataStore() { + return (MetadataPath) inner.add(METADATA_KEY); + } + + public UserPath parent() { + return (UserPath) inner.getParent(); + } + } + + static class DataPath extends KeySpacePathWrapper { + public DataPath(KeySpacePath path) { + super(path); + } + + public ApplicationPath parent() { + return (ApplicationPath) inner.getParent(); + } + } + + static class MetadataPath extends KeySpacePathWrapper { + public MetadataPath(KeySpacePath path) { + super(path); + } + + public ApplicationPath parent() { + return (ApplicationPath) inner.getParent(); + } + } +} diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpaceDirectoryTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpaceDirectoryTest.java index 0263ca9bf4..5c084cb3fa 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpaceDirectoryTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpaceDirectoryTest.java @@ -31,7 +31,6 @@ import com.apple.foundationdb.record.logging.KeyValueLogMessage; import com.apple.foundationdb.record.provider.foundationdb.FDBDatabase; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; -import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStore; import com.apple.foundationdb.record.provider.foundationdb.FDBStoreTimer; import com.apple.foundationdb.record.provider.foundationdb.keyspace.KeySpaceDirectory.KeyType; import com.apple.foundationdb.record.provider.foundationdb.layers.interning.ScopedInterningLayer; @@ -1336,7 +1335,7 @@ public void testPathWrapperExample() throws Exception { final Tuple dataStoreTuple; final Tuple metadataStoreTuple; try (FDBRecordContext context = database.openContext()) { - ApplicationPath application = keySpace.root().userid(123).application("myApplication"); + EnvironmentKeySpace.ApplicationPath application = keySpace.root().userid(123).application("myApplication"); dataStoreTuple = application.dataStore().toTuple(context); metadataStoreTuple = application.metadataStore().toTuple(context); context.commit(); @@ -1350,9 +1349,9 @@ public void testPathWrapperExample() throws Exception { assertEquals(Tuple.from(entries.get(0), 123L, entries.get(1), EnvironmentKeySpace.METADATA_VALUE), metadataStoreTuple); ResolvedKeySpacePath path = keySpace.fromKey(context, dataStoreTuple); - assertThat(path.toPath(), instanceOf(DataPath.class)); + assertThat(path.toPath(), instanceOf(EnvironmentKeySpace.DataPath.class)); - DataPath mainStorePath = (DataPath) path.toPath(); + EnvironmentKeySpace.DataPath mainStorePath = (EnvironmentKeySpace.DataPath) path.toPath(); assertEquals(EnvironmentKeySpace.DATA_VALUE, mainStorePath.getValue()); assertEquals(EnvironmentKeySpace.DATA_VALUE, mainStorePath.resolveAsync(context).get().getResolvedValue()); assertEquals(entries.get(1), mainStorePath.parent().resolveAsync(context).get().getResolvedValue()); @@ -1362,14 +1361,14 @@ public void testPathWrapperExample() throws Exception { assertEquals("production", mainStorePath.parent().parent().parent().getValue()); assertNull(mainStorePath.parent().parent().parent().parent()); - assertThat(keySpace.fromKey(context, TupleHelpers.subTuple(dataStoreTuple, 0, 1)).toPath(), instanceOf(EnvironmentRoot.class)); - assertThat(keySpace.fromKey(context, TupleHelpers.subTuple(dataStoreTuple, 0, 2)).toPath(), instanceOf(UserPath.class)); - assertThat(keySpace.fromKey(context, TupleHelpers.subTuple(dataStoreTuple, 0, 3)).toPath(), instanceOf(ApplicationPath.class)); + assertThat(keySpace.fromKey(context, TupleHelpers.subTuple(dataStoreTuple, 0, 1)).toPath(), instanceOf(EnvironmentKeySpace.EnvironmentRoot.class)); + assertThat(keySpace.fromKey(context, TupleHelpers.subTuple(dataStoreTuple, 0, 2)).toPath(), instanceOf(EnvironmentKeySpace.UserPath.class)); + assertThat(keySpace.fromKey(context, TupleHelpers.subTuple(dataStoreTuple, 0, 3)).toPath(), instanceOf(EnvironmentKeySpace.ApplicationPath.class)); path = keySpace.fromKey(context, metadataStoreTuple); - assertThat(path.toPath(), instanceOf(MetadataPath.class)); + assertThat(path.toPath(), instanceOf(EnvironmentKeySpace.MetadataPath.class)); - MetadataPath metadataPath = (MetadataPath) path.toPath(); + EnvironmentKeySpace.MetadataPath metadataPath = (EnvironmentKeySpace.MetadataPath) path.toPath(); assertEquals(EnvironmentKeySpace.METADATA_VALUE, metadataPath.getValue()); assertEquals(EnvironmentKeySpace.METADATA_VALUE, metadataPath.resolveAsync(context).get().getResolvedValue()); assertEquals(entries.get(1), metadataPath.parent().resolveAsync(context).get().getResolvedValue()); @@ -1379,14 +1378,14 @@ public void testPathWrapperExample() throws Exception { assertEquals("production", metadataPath.parent().parent().parent().getValue()); assertNull(metadataPath.parent().parent().parent().parent()); - assertThat(keySpace.fromKey(context, TupleHelpers.subTuple(dataStoreTuple, 0, 1)).toPath(), instanceOf(EnvironmentRoot.class)); - assertThat(keySpace.fromKey(context, TupleHelpers.subTuple(dataStoreTuple, 0, 2)).toPath(), instanceOf(UserPath.class)); - assertThat(keySpace.fromKey(context, TupleHelpers.subTuple(dataStoreTuple, 0, 3)).toPath(), instanceOf(ApplicationPath.class)); + assertThat(keySpace.fromKey(context, TupleHelpers.subTuple(dataStoreTuple, 0, 1)).toPath(), instanceOf(EnvironmentKeySpace.EnvironmentRoot.class)); + assertThat(keySpace.fromKey(context, TupleHelpers.subTuple(dataStoreTuple, 0, 2)).toPath(), instanceOf(EnvironmentKeySpace.UserPath.class)); + assertThat(keySpace.fromKey(context, TupleHelpers.subTuple(dataStoreTuple, 0, 3)).toPath(), instanceOf(EnvironmentKeySpace.ApplicationPath.class)); // Create a fake main store "record" key to demonstrate that we can get the key as the remainder Tuple recordTuple = dataStoreTuple.add(1L).add("someStr").add(0L); // 1=record space, record id, 0=unsplit record path = keySpace.fromKey(context, recordTuple); - assertThat(path.toPath(), instanceOf(DataPath.class)); + assertThat(path.toPath(), instanceOf(EnvironmentKeySpace.DataPath.class)); assertEquals(Tuple.from(1L, "someStr", 0L), path.getRemainder()); assertEquals(dataStoreTuple, path.toTuple()); } @@ -1538,137 +1537,4 @@ protected CompletableFuture toTupleValueAsyncImpl(@Nonnull FDBRecordC } } - /** - * This provides an example of a way in which you can define a KeySpace in a relatively clean and type-safe - * manner. It defines a keyspace that looks like: - *
-     *    [environment]           - A string the identifies the logical environment (like prod, test, qa, etc.).
-     *      |                       This string is converted by the directory layer as a small integer value.
-     *      +- userid             - An integer ID for each user in the system
-     *         |
-     *         +- [application]   - Tne name of an application the user runs (again, converted by the directory
-     *            |                 layer into a small integer value)
-     *            +- data=1       - Constant value of "1", which is the location of a {@link FDBRecordStore}
-     *            |                 in which application data is to be stored
-     *            +- metadata=2   - Constant value of "2", which is the Location of another FDBRecordStore
-     *                              in which application metadata or configuration information can live.
-     * 
- * The main point of this class is to demonstrate how you can use the KeySpacePath wrapping facility to provide - * implementations of the path elements that are meaningful to your application environment and type safe. - */ - private static class EnvironmentKeySpace { - private final KeySpace root; - private final String rootName; - - public static String USER_KEY = "userid"; - public static String APPLICATION_KEY = "application"; - public static String DATA_KEY = "data"; - public static long DATA_VALUE = 1L; - public static String METADATA_KEY = "metadata"; - public static long METADATA_VALUE = 2L; - - /** - * The EnvironmentKeySpace scopes all of the data it stores underneath of a rootName, - * for example, you could define an instance for prod, test, qa, etc. - * - * @param rootName The root name underwhich all data is stored. - */ - public EnvironmentKeySpace(String rootName) { - this.rootName = rootName; - root = new KeySpace( - new DirectoryLayerDirectory(rootName, rootName, EnvironmentRoot::new) - .addSubdirectory(new KeySpaceDirectory(USER_KEY, KeyType.LONG, UserPath::new) - .addSubdirectory(new DirectoryLayerDirectory(APPLICATION_KEY, ApplicationPath::new) - .addSubdirectory(new KeySpaceDirectory(DATA_KEY, KeyType.LONG, DATA_VALUE, DataPath::new)) - .addSubdirectory(new KeySpaceDirectory(METADATA_KEY, KeyType.LONG, METADATA_VALUE, MetadataPath::new))))); - } - - public String getRootName() { - return rootName; - } - - /** - * Returns an implementation of a KeySpacePath that represents the start of the environment. - */ - public EnvironmentRoot root() { - return (EnvironmentRoot) root.path(rootName); - } - - /** - * Given a tuple that represents an FDB key that came from this KeySpace, returns the leaf-most path - * element in which the tuple resides. - */ - public ResolvedKeySpacePath fromKey(FDBRecordContext context, Tuple tuple) { - return root.resolveFromKey(context, tuple); - } - } - - /** - * A KeySpacePath that represents the logical root of the environment. - */ - private static class EnvironmentRoot extends KeySpacePathWrapper { - public EnvironmentRoot(KeySpacePath path) { - super(path); - } - - public KeySpacePath parent() { - return null; - } - - public UserPath userid(long userid) { - return (UserPath) inner.add(EnvironmentKeySpace.USER_KEY, userid); - } - } - - private static class UserPath extends KeySpacePathWrapper { - public UserPath(KeySpacePath path) { - super(path); - } - - public ApplicationPath application(String applicationName) { - return (ApplicationPath) inner.add(EnvironmentKeySpace.APPLICATION_KEY, applicationName); - } - - public EnvironmentRoot parent() { - return (EnvironmentRoot) inner.getParent(); - } - } - - private static class ApplicationPath extends KeySpacePathWrapper { - public ApplicationPath(KeySpacePath path) { - super(path); - } - - public DataPath dataStore() { - return (DataPath) inner.add(EnvironmentKeySpace.DATA_KEY); - } - - public MetadataPath metadataStore() { - return (MetadataPath) inner.add(EnvironmentKeySpace.METADATA_KEY); - } - - public UserPath parent() { - return (UserPath) inner.getParent(); - } - } - - private static class DataPath extends KeySpacePathWrapper { - public DataPath(KeySpacePath path) { - super(path); - } - - public ApplicationPath parent() { - return (ApplicationPath) inner.getParent(); - } - } - - private static class MetadataPath extends KeySpacePathWrapper { - public MetadataPath(KeySpacePath path) { - super(path); - } - - public ApplicationPath parent() { - return (ApplicationPath) inner.getParent(); - } - } } From 439ea52ce304fba53bf888c52b8c128eb7fb16da Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Thu, 4 Sep 2025 10:56:23 -0400 Subject: [PATCH 12/32] Add tests of KeySPacePathWrapper.exportAllData --- .../keyspace/KeySpacePathDataExportTest.java | 118 ++++++++++++++++++ 1 file changed, 118 insertions(+) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java index a428f37b6c..7876b1ecdf 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java @@ -39,6 +39,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; +import javax.annotation.Nonnull; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -509,4 +510,121 @@ private static List exportAllData(final KeySpacePath rootPath, final F return asSingleExport; } + @Test + void testExportAllDataThroughKeySpacePathWrapper() { + final FDBDatabase database = dbExtension.getDatabase(); + final EnvironmentKeySpace keySpace = setupEnvironmentKeySpaceData(database); + + // Test export at different levels through wrapper methods + try (FDBRecordContext context = database.openContext()) { + // Export from root level (should get all data) + EnvironmentKeySpace.EnvironmentRoot root = keySpace.root(); + List allData = exportAllData(root, context); + assertEquals(5, allData.size(), "Root level should export all data"); + + // Export from specific user level (should get data for user 100 only) + EnvironmentKeySpace.UserPath user100Path = keySpace.root().userid(100L); + verifyExtractedData(exportAllData(user100Path, context), + 4, "User 100 should have 4 records", + "user100", "All user 100 data should contain 'user100'"); + + // Export from specific application level (app1 for user 100) + EnvironmentKeySpace.ApplicationPath app1User100 = user100Path.application("app1"); + verifyExtractedData(exportAllData(app1User100, context), + 3, "App1 for user 100 should have 3 records (2 data + 1 metadata)", + "user100_app1", "All app1 user100 data should contain 'user100_app1'"); + + // Export from specific data store level + EnvironmentKeySpace.DataPath dataStore = app1User100.dataStore(); + List dataStoreData = exportAllData(dataStore, context); + verifyExtractedData(dataStoreData, + 2, "Data store should have exactly 2 records", + "user100_app1_data", "Data should be from user100 app1 data store"); + + // Export from metadata store level + EnvironmentKeySpace.MetadataPath metadataStore = app1User100.metadataStore(); + verifyExtractedData(exportAllData(metadataStore, context), + 1, "Metadata store should have exactly 1 record", + "user100_app1_meta1", "Metadata value should match"); + + // Verify empty export for user with no data + EnvironmentKeySpace.UserPath user300Path = keySpace.root().userid(300L); + assertEquals(0, exportAllData(user300Path, context).size(), "User 300 should have no data"); + } + } + + @Test + void testExportAllDataThroughKeySpacePathWrapperResolvedPaths() { + final FDBDatabase database = dbExtension.getDatabase(); + final EnvironmentKeySpace keySpace = setupEnvironmentKeySpaceData(database); + + // Test export at different levels through wrapper methods + try (FDBRecordContext context = database.openContext()) { + // Test 4: Export from specific data store level + EnvironmentKeySpace.DataPath dataStore = keySpace.root().userid(100L).application("app1").dataStore(); + final List dataStoreData = dataStore.exportAllData(context, null, ScanProperties.FORWARD_SCAN) + .mapPipelined(DataInKeySpacePath::getResolvedPath, 1).asList().join(); + assertEquals(2, dataStoreData.size()); + // Verify data store records have correct remainder + final ArrayList remainders = new ArrayList<>(); + for (ResolvedKeySpacePath kv : dataStoreData) { + + // Path tuple should be the same + Tuple dataStoreTuple = dataStore.toTuple(context); + assertEquals(dataStoreTuple, kv.toTuple()); + + // Remainder should be the same + remainders.add(kv.getRemainder()); + } + assertEquals(List.of( + Tuple.from("record1"), + Tuple.from("record2", 0), + Tuple.from("record2", 1) + ), remainders); + + } + } + + @Nonnull + private static EnvironmentKeySpace setupEnvironmentKeySpaceData(@Nonnull final FDBDatabase database) { + EnvironmentKeySpace keySpace = new EnvironmentKeySpace("test_env"); + + // Store test data at different levels of the hierarchy + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + + // Create paths for different users and applications + EnvironmentKeySpace.ApplicationPath app1User1 = keySpace.root().userid(100L).application("app1"); + EnvironmentKeySpace.ApplicationPath app2User1 = keySpace.root().userid(100L).application("app2"); + EnvironmentKeySpace.ApplicationPath app1User2 = keySpace.root().userid(200L).application("app1"); + + EnvironmentKeySpace.DataPath dataUser1App1 = app1User1.dataStore(); + EnvironmentKeySpace.MetadataPath metaUser1App1 = app1User1.metadataStore(); + EnvironmentKeySpace.DataPath dataUser1App2 = app2User1.dataStore(); + EnvironmentKeySpace.DataPath dataUser2App1 = app1User2.dataStore(); + + // Store data records with additional tuple elements after the KeySpacePath + tr.set(dataUser1App1.toTuple(context).add("record1").pack(), Tuple.from("user100_app1_data1").pack()); + tr.set(dataUser1App1.toTuple(context).add("record2").add(0).pack(), Tuple.from("user100_app1_data2_0").pack()); + tr.set(dataUser1App1.toTuple(context).add("record2").add(1).pack(), Tuple.from("user100_app1_data2_1").pack()); + tr.set(metaUser1App1.toTuple(context).add("config1").pack(), Tuple.from("user100_app1_meta1").pack()); + tr.set(dataUser1App2.toTuple(context).add("record3").pack(), Tuple.from("user100_app2_data3").pack()); + tr.set(dataUser2App1.toTuple(context).add("record4").pack(), Tuple.from("user200_app1_data4").pack()); + + context.commit(); + } + return keySpace; + } + + private static void verifyExtractedData(final List app1User100Data, + int expectedCount, String expectedCountMessage, + String expectedValueContents, String contentMessage) { + assertEquals(expectedCount, app1User100Data.size(), expectedCountMessage); + + for (KeyValue kv : app1User100Data) { + String value = Tuple.fromBytes(kv.getValue()).getString(0); + assertTrue(value.contains(expectedValueContents), contentMessage); + } + } + } From d9b92abc8d38b3acb8eb11541f513b739cd5a9cf Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Thu, 4 Sep 2025 11:35:20 -0400 Subject: [PATCH 13/32] Cleanup style of test --- .../keyspace/KeySpacePathDataExportTest.java | 17 ++++++----------- 1 file changed, 6 insertions(+), 11 deletions(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java index 7876b1ecdf..cba5201a4f 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java @@ -48,6 +48,7 @@ import java.util.Set; import java.util.UUID; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; import java.util.stream.IntStream; import static org.assertj.core.api.Assertions.assertThat; @@ -240,14 +241,9 @@ void testExportAllDataWithDifferentKeyTypes() { assertEquals(12, allData.size()); // Verify we have different value types - List valueTypes = new ArrayList<>(); - for (KeyValue kv : allData) { - String value = Tuple.fromBytes(kv.getValue()).getString(0); - String valueType = value.split("_")[0]; - if (!valueTypes.contains(valueType)) { - valueTypes.add(valueType); - } - } + Set valueTypes = allData.stream() + .map(kv -> Tuple.fromBytes(kv.getValue()).getString(0).split("_")[0]) + .collect(Collectors.toSet()); assertEquals(5, valueTypes.size()); assertTrue(valueTypes.containsAll(Arrays.asList("string", "long", "bytes", "uuid", "boolean"))); } @@ -353,7 +349,7 @@ void testExportAllDataWithDeepNestedStructure() { KeySpacePath orgPath = root.path("org"); final List allData = exportAllData(orgPath, context); - // Should have 16 records (2 depts * 2 teams * 2 members * 2 records each) + // Should have 16 records (2 departments * 2 teams * 2 members * 2 records each) assertEquals(16, allData.size()); } @@ -411,8 +407,7 @@ void testExportAllDataWithBinaryData() { assertEquals(3, allData.size()); // Verify binary data integrity - for (int i = 0; i < allData.size(); i++) { - KeyValue kv = allData.get(i); + for (KeyValue kv : allData) { String valueStr = new String(kv.getValue()); assertTrue(valueStr.startsWith("binary_data_")); } From cc2420edbbd965dd51379f1ca765058ead380a92 Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Thu, 4 Sep 2025 12:23:13 -0400 Subject: [PATCH 14/32] Fix bug when exporting at the leaf node --- .../keyspace/DataInKeySpacePath.java | 10 +- .../keyspace/DataInKeySpacePathTest.java | 56 +++++-- .../keyspace/EnvironmentKeySpace.java | 36 +++++ .../keyspace/KeySpacePathDataExportTest.java | 137 ++++++++---------- 4 files changed, 149 insertions(+), 90 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java index 0ef0bc8ae6..3db3280a65 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java @@ -23,6 +23,7 @@ import com.apple.foundationdb.KeyValue; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; import com.apple.foundationdb.tuple.Tuple; +import com.apple.foundationdb.tuple.TupleHelpers; import java.util.concurrent.CompletableFuture; @@ -50,7 +51,14 @@ public DataInKeySpacePath(KeySpacePath path, KeyValue rawKeyValue, FDBRecordCont // The remaining part of the key should be resolved from the resolved path's directory if (keyTuple.size() > pathLength) { // There's more in the key than just the path, so resolve the rest - return resolvedPath.getDirectory().findChildForKey(context, resolvedPath, keyTuple, keyTuple.size(), pathLength); + if (resolvedPath.getDirectory().getSubdirectories().isEmpty()) { + return CompletableFuture.completedFuture( + new ResolvedKeySpacePath(resolvedPath.getParent(), resolvedPath.toPath(), + resolvedPath.getResolvedPathValue(), + TupleHelpers.subTuple(keyTuple, pathTuple.size(), keyTuple.size()))); + } else { + return resolvedPath.getDirectory().findChildForKey(context, resolvedPath, keyTuple, keyTuple.size(), pathLength); + } } else { // The key exactly matches the path return CompletableFuture.completedFuture(resolvedPath); diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java index 34a337004d..733bf38798 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java @@ -36,6 +36,7 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; +import java.util.List; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -53,19 +54,23 @@ class DataInKeySpacePathTest { @RegisterExtension final FDBDatabaseExtension dbExtension = new FDBDatabaseExtension(); - @Test - void testSimpleTwoLevelPath() { + @ParameterizedTest + @ValueSource(ints = {0, 1, 2}) + void testSimpleTwoLevelPath(int depth) { KeySpace root = new KeySpace( new KeySpaceDirectory("app", KeyType.STRING, UUID.randomUUID().toString()) - .addSubdirectory(new KeySpaceDirectory("user", KeyType.LONG))); + .addSubdirectory(new KeySpaceDirectory("locality", KeyType.STRING, "Foo") + .addSubdirectory(new KeySpaceDirectory("user", KeyType.LONG)))); final FDBDatabase database = dbExtension.getDatabase(); // Store test data and create DataInKeySpacePath try (FDBRecordContext context = database.openContext()) { Transaction tr = context.ensureActive(); - - KeySpacePath userPath = root.path("app").add("user", 123L); + + KeySpacePath appPath = root.path("app"); + KeySpacePath localityPath = appPath.add("locality"); + KeySpacePath userPath = localityPath.add("user", 123L); final Subspace pathSubspace = userPath.toSubspace(context); // Add additional tuple elements after the KeySpacePath (this is how data is actually stored) @@ -74,10 +79,10 @@ void testSimpleTwoLevelPath() { tr.set(keyBytes, valueBytes); KeyValue keyValue = new KeyValue(keyBytes, valueBytes); - - // Create DataInKeySpacePath from the app-level path - KeySpacePath appPath = root.path("app"); - DataInKeySpacePath dataInPath = new DataInKeySpacePath(appPath, keyValue, context); + + final List queryPaths = List.of(appPath, localityPath, userPath); + DataInKeySpacePath dataInPath = new DataInKeySpacePath( + queryPaths.get(depth), keyValue, context); // Verify the resolved path CompletableFuture resolvedFuture = dataInPath.getResolvedPath(); @@ -93,11 +98,14 @@ void testSimpleTwoLevelPath() { // Verify parent path ResolvedKeySpacePath parent = resolved.getParent(); assertNotNull(parent); - assertEquals("app", parent.getDirectoryName()); - + assertEquals("locality", parent.getDirectoryName()); + ResolvedKeySpacePath grandParent = parent.getParent(); + assertNotNull(grandParent); + assertEquals("app", grandParent.getDirectoryName()); + // Verify the resolved path recreates the KeySpacePath portion (not the full key) Tuple resolvedTuple = resolved.toTuple(); - assertEquals(TupleHelpers.subTuple(Tuple.fromBytes(keyBytes), 0, 2), resolvedTuple); + assertEquals(TupleHelpers.subTuple(Tuple.fromBytes(keyBytes), 0, 3), resolvedTuple); // Verify that the remainder contains the additional tuple elements Tuple remainder = resolved.getRemainder(); @@ -546,4 +554,28 @@ void testNullKeyTypeDirectory() { context.commit(); } } + + @Test + void testWithWrapper() { + final FDBDatabase database = dbExtension.getDatabase(); + final EnvironmentKeySpace keySpace = EnvironmentKeySpace.setupSampleData(database); + + // Test export at different levels through wrapper methods + try (FDBRecordContext context = database.openContext()) { + // Test 4: Export from specific data store level + EnvironmentKeySpace.DataPath dataStore = keySpace.root().userid(100L).application("app1").dataStore(); + + final byte[] key = dataStore.toTuple(context).add("record2").add(0).pack(); + final byte[] value = Tuple.from("data").pack(); + final DataInKeySpacePath dataInKeySpacePath = new DataInKeySpacePath(dataStore, new KeyValue(key, value), context); + + final ResolvedKeySpacePath resolvedPath = dataInKeySpacePath.getResolvedPath().join(); + assertEquals(dataStore.toResolvedPath(context), withoutRemainder(resolvedPath)); + assertEquals(Tuple.from("record2", 0), resolvedPath.getRemainder()); + } + } + + private ResolvedKeySpacePath withoutRemainder(final ResolvedKeySpacePath path) { + return new ResolvedKeySpacePath(path.getParent(), path.toPath(), path.getResolvedPathValue(), null); + } } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/EnvironmentKeySpace.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/EnvironmentKeySpace.java index e64044058d..c1751395cf 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/EnvironmentKeySpace.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/EnvironmentKeySpace.java @@ -20,10 +20,15 @@ package com.apple.foundationdb.record.provider.foundationdb.keyspace; +import com.apple.foundationdb.Transaction; +import com.apple.foundationdb.record.provider.foundationdb.FDBDatabase; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStore; import com.apple.foundationdb.tuple.Tuple; +import javax.annotation.Nonnull; +import java.util.UUID; + /** * This provides an example of a way in which you can define a KeySpace in a relatively clean and type-safe * manner. It defines a keyspace that looks like: @@ -69,6 +74,37 @@ public EnvironmentKeySpace(String rootName) { .addSubdirectory(new KeySpaceDirectory(METADATA_KEY, KeySpaceDirectory.KeyType.LONG, METADATA_VALUE, MetadataPath::new))))); } + @Nonnull + static EnvironmentKeySpace setupSampleData(@Nonnull final FDBDatabase database) { + EnvironmentKeySpace keySpace = new EnvironmentKeySpace(UUID.randomUUID().toString()); + + // Store test data at different levels of the hierarchy + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + + // Create paths for different users and applications + ApplicationPath app1User1 = keySpace.root().userid(100L).application("app1"); + ApplicationPath app2User1 = keySpace.root().userid(100L).application("app2"); + ApplicationPath app1User2 = keySpace.root().userid(200L).application("app1"); + + DataPath dataUser1App1 = app1User1.dataStore(); + MetadataPath metaUser1App1 = app1User1.metadataStore(); + DataPath dataUser1App2 = app2User1.dataStore(); + DataPath dataUser2App1 = app1User2.dataStore(); + + // Store data records with additional tuple elements after the KeySpacePath + tr.set(dataUser1App1.toTuple(context).add("record1").pack(), Tuple.from("user100_app1_data1").pack()); + tr.set(dataUser1App1.toTuple(context).add("record2").add(0).pack(), Tuple.from("user100_app1_data2_0").pack()); + tr.set(dataUser1App1.toTuple(context).add("record2").add(1).pack(), Tuple.from("user100_app1_data2_1").pack()); + tr.set(metaUser1App1.toTuple(context).add("config1").pack(), Tuple.from("user100_app1_meta1").pack()); + tr.set(dataUser1App2.toTuple(context).add("record3").pack(), Tuple.from("user100_app2_data3").pack()); + tr.set(dataUser2App1.toTuple(context).add("record4").pack(), Tuple.from("user200_app1_data4").pack()); + + context.commit(); + } + return keySpace; + } + public String getRootName() { return rootName; } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java index cba5201a4f..abfd6cdcab 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java @@ -33,13 +33,13 @@ import com.apple.foundationdb.record.test.FDBDatabaseExtension; import com.apple.foundationdb.tuple.Tuple; import com.apple.test.Tags; +import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; -import javax.annotation.Nonnull; import java.util.ArrayList; import java.util.Arrays; import java.util.Collections; @@ -467,73 +467,35 @@ void testExportAllDataWithContinuation(int limit) { } } - private void setData(List keys, FDBRecordContext context, KeySpacePath basePath, - String subdirectory, String valuePrefix) { - Transaction tr = context.ensureActive(); - for (int i = 0; i < keys.size(); i++) { - Tuple tuple = basePath.add(subdirectory, keys.get(i)).toTuple(context); - tr.set(tuple.pack(), Tuple.from(valuePrefix + i).pack()); - } - } - - private static List exportAllData(final KeySpacePath rootPath, final FDBRecordContext context) { - final List asSingleExport = rootPath.exportAllData(context, null, ScanProperties.FORWARD_SCAN) - .map(DataInKeySpacePath::getRawKeyValue).asList().join(); - - final List reversed = rootPath.exportAllData(context, null, ScanProperties.REVERSE_SCAN) - .map(DataInKeySpacePath::getRawKeyValue).asList().join(); - Collections.reverse(reversed); - assertEquals(asSingleExport, reversed); - - final ScanProperties scanProperties = ScanProperties.FORWARD_SCAN.with(props -> props.setReturnedRowLimit(1)); - List asContinuations = new ArrayList<>(); - RecordCursorContinuation continuation = RecordCursorStartContinuation.START; - while (!continuation.isEnd()) { - final RecordCursor cursor = rootPath.exportAllData(context, continuation.toBytes(), - scanProperties); - final AtomicReference> keyValueResult = new AtomicReference<>(); - final List batch = cursor.map(DataInKeySpacePath::getRawKeyValue).asList(keyValueResult).join(); - asContinuations.addAll(batch); - continuation = keyValueResult.get().getContinuation(); - if (keyValueResult.get().hasNext()) { - assertEquals(1, batch.size()); - } else { - assertThat(batch.size()).isLessThanOrEqualTo(1); - } - } - assertEquals(asSingleExport, asContinuations); - return asSingleExport; - } - @Test void testExportAllDataThroughKeySpacePathWrapper() { final FDBDatabase database = dbExtension.getDatabase(); - final EnvironmentKeySpace keySpace = setupEnvironmentKeySpaceData(database); + final EnvironmentKeySpace keySpace = EnvironmentKeySpace.setupSampleData(database); // Test export at different levels through wrapper methods try (FDBRecordContext context = database.openContext()) { // Export from root level (should get all data) EnvironmentKeySpace.EnvironmentRoot root = keySpace.root(); List allData = exportAllData(root, context); - assertEquals(5, allData.size(), "Root level should export all data"); + assertEquals(6, allData.size(), "Root level should export all data"); // Export from specific user level (should get data for user 100 only) EnvironmentKeySpace.UserPath user100Path = keySpace.root().userid(100L); verifyExtractedData(exportAllData(user100Path, context), - 4, "User 100 should have 4 records", + 5, "User 100 should have 4 records", "user100", "All user 100 data should contain 'user100'"); // Export from specific application level (app1 for user 100) EnvironmentKeySpace.ApplicationPath app1User100 = user100Path.application("app1"); verifyExtractedData(exportAllData(app1User100, context), - 3, "App1 for user 100 should have 3 records (2 data + 1 metadata)", + 4, "App1 for user 100 should have 4 records (3 data + 1 metadata)", "user100_app1", "All app1 user100 data should contain 'user100_app1'"); // Export from specific data store level EnvironmentKeySpace.DataPath dataStore = app1User100.dataStore(); List dataStoreData = exportAllData(dataStore, context); verifyExtractedData(dataStoreData, - 2, "Data store should have exactly 2 records", + 3, "Data store should have exactly 3 records", "user100_app1_data", "Data should be from user100 app1 data store"); // Export from metadata store level @@ -551,7 +513,7 @@ void testExportAllDataThroughKeySpacePathWrapper() { @Test void testExportAllDataThroughKeySpacePathWrapperResolvedPaths() { final FDBDatabase database = dbExtension.getDatabase(); - final EnvironmentKeySpace keySpace = setupEnvironmentKeySpaceData(database); + final EnvironmentKeySpace keySpace = EnvironmentKeySpace.setupSampleData(database); // Test export at different levels through wrapper methods try (FDBRecordContext context = database.openContext()) { @@ -559,56 +521,77 @@ void testExportAllDataThroughKeySpacePathWrapperResolvedPaths() { EnvironmentKeySpace.DataPath dataStore = keySpace.root().userid(100L).application("app1").dataStore(); final List dataStoreData = dataStore.exportAllData(context, null, ScanProperties.FORWARD_SCAN) .mapPipelined(DataInKeySpacePath::getResolvedPath, 1).asList().join(); - assertEquals(2, dataStoreData.size()); // Verify data store records have correct remainder final ArrayList remainders = new ArrayList<>(); for (ResolvedKeySpacePath kv : dataStoreData) { - // Path tuple should be the same Tuple dataStoreTuple = dataStore.toTuple(context); assertEquals(dataStoreTuple, kv.toTuple()); - - // Remainder should be the same remainders.add(kv.getRemainder()); } assertEquals(List.of( Tuple.from("record1"), Tuple.from("record2", 0), Tuple.from("record2", 1) - ), remainders); + ), remainders, "remainders should be the same"); } } - @Nonnull - private static EnvironmentKeySpace setupEnvironmentKeySpaceData(@Nonnull final FDBDatabase database) { - EnvironmentKeySpace keySpace = new EnvironmentKeySpace("test_env"); + private void setData(List keys, FDBRecordContext context, KeySpacePath basePath, + String subdirectory, String valuePrefix) { + Transaction tr = context.ensureActive(); + for (int i = 0; i < keys.size(); i++) { + Tuple tuple = basePath.add(subdirectory, keys.get(i)).toTuple(context); + tr.set(tuple.pack(), Tuple.from(valuePrefix + i).pack()); + } + } - // Store test data at different levels of the hierarchy - try (FDBRecordContext context = database.openContext()) { - Transaction tr = context.ensureActive(); - - // Create paths for different users and applications - EnvironmentKeySpace.ApplicationPath app1User1 = keySpace.root().userid(100L).application("app1"); - EnvironmentKeySpace.ApplicationPath app2User1 = keySpace.root().userid(100L).application("app2"); - EnvironmentKeySpace.ApplicationPath app1User2 = keySpace.root().userid(200L).application("app1"); - - EnvironmentKeySpace.DataPath dataUser1App1 = app1User1.dataStore(); - EnvironmentKeySpace.MetadataPath metaUser1App1 = app1User1.metadataStore(); - EnvironmentKeySpace.DataPath dataUser1App2 = app2User1.dataStore(); - EnvironmentKeySpace.DataPath dataUser2App1 = app1User2.dataStore(); - - // Store data records with additional tuple elements after the KeySpacePath - tr.set(dataUser1App1.toTuple(context).add("record1").pack(), Tuple.from("user100_app1_data1").pack()); - tr.set(dataUser1App1.toTuple(context).add("record2").add(0).pack(), Tuple.from("user100_app1_data2_0").pack()); - tr.set(dataUser1App1.toTuple(context).add("record2").add(1).pack(), Tuple.from("user100_app1_data2_1").pack()); - tr.set(metaUser1App1.toTuple(context).add("config1").pack(), Tuple.from("user100_app1_meta1").pack()); - tr.set(dataUser1App2.toTuple(context).add("record3").pack(), Tuple.from("user100_app2_data3").pack()); - tr.set(dataUser2App1.toTuple(context).add("record4").pack(), Tuple.from("user200_app1_data4").pack()); - - context.commit(); + private static List exportAllData(final KeySpacePath rootPath, final FDBRecordContext context) { + final List asSingleExport = rootPath.exportAllData(context, null, ScanProperties.FORWARD_SCAN) + .map(DataInKeySpacePath::getRawKeyValue).asList().join(); + + final List resolvedPaths = rootPath.exportAllData(context, null, ScanProperties.FORWARD_SCAN) + .mapPipelined(DataInKeySpacePath::getResolvedPath, 1).asList().join(); + final ResolvedKeySpacePath rootResolvedPath = rootPath.toResolvedPath(context); + for (ResolvedKeySpacePath resolvedPath : resolvedPaths) { + assertStartsWith(rootResolvedPath, resolvedPath); } - return keySpace; + + final List reversed = rootPath.exportAllData(context, null, ScanProperties.REVERSE_SCAN) + .map(DataInKeySpacePath::getRawKeyValue).asList().join(); + Collections.reverse(reversed); + assertEquals(asSingleExport, reversed); + + final ScanProperties scanProperties = ScanProperties.FORWARD_SCAN.with(props -> props.setReturnedRowLimit(1)); + List asContinuations = new ArrayList<>(); + RecordCursorContinuation continuation = RecordCursorStartContinuation.START; + while (!continuation.isEnd()) { + final RecordCursor cursor = rootPath.exportAllData(context, continuation.toBytes(), + scanProperties); + final AtomicReference> keyValueResult = new AtomicReference<>(); + final List batch = cursor.map(DataInKeySpacePath::getRawKeyValue).asList(keyValueResult).join(); + asContinuations.addAll(batch); + continuation = keyValueResult.get().getContinuation(); + if (keyValueResult.get().hasNext()) { + assertEquals(1, batch.size()); + } else { + assertThat(batch.size()).isLessThanOrEqualTo(1); + } + } + + assertEquals(asSingleExport, asContinuations); + return asSingleExport; + } + + private static void assertStartsWith(final ResolvedKeySpacePath rootResolvedPath, ResolvedKeySpacePath resolvedPath) { + do { + if (resolvedPath.equals(rootResolvedPath)) { + return; + } + resolvedPath = resolvedPath.getParent(); + } while (resolvedPath != null); + Assertions.fail("Expected <" + resolvedPath + "> to start with <" + rootResolvedPath + "> but it didn't"); } private static void verifyExtractedData(final List app1User100Data, From e2da4ea568d738226a32dcef2043363aa5ab53eb Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Thu, 4 Sep 2025 16:37:36 -0400 Subject: [PATCH 15/32] Reduce repetitive tests, and repetitive code for assertions --- .../keyspace/DataInKeySpacePathTest.java | 339 ++++++------------ 1 file changed, 100 insertions(+), 239 deletions(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java index 733bf38798..48658b456c 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java @@ -26,7 +26,6 @@ import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; import com.apple.foundationdb.record.provider.foundationdb.keyspace.KeySpaceDirectory.KeyType; import com.apple.foundationdb.record.test.FDBDatabaseExtension; -import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.Tuple; import com.apple.foundationdb.tuple.TupleHelpers; import com.apple.test.Tags; @@ -36,7 +35,6 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; -import java.util.List; import java.util.UUID; import java.util.concurrent.CompletableFuture; @@ -55,213 +53,70 @@ class DataInKeySpacePathTest { final FDBDatabaseExtension dbExtension = new FDBDatabaseExtension(); @ParameterizedTest - @ValueSource(ints = {0, 1, 2}) - void testSimpleTwoLevelPath(int depth) { + @ValueSource(ints = {0, 1, 2, 3, 4, 5}) + void testResolution() { + // Include some extra children to make sure resolution doesn't get confused + final String companyUuid = UUID.randomUUID().toString(); KeySpace root = new KeySpace( - new KeySpaceDirectory("app", KeyType.STRING, UUID.randomUUID().toString()) - .addSubdirectory(new KeySpaceDirectory("locality", KeyType.STRING, "Foo") - .addSubdirectory(new KeySpaceDirectory("user", KeyType.LONG)))); - - final FDBDatabase database = dbExtension.getDatabase(); - - // Store test data and create DataInKeySpacePath - try (FDBRecordContext context = database.openContext()) { - Transaction tr = context.ensureActive(); - - KeySpacePath appPath = root.path("app"); - KeySpacePath localityPath = appPath.add("locality"); - KeySpacePath userPath = localityPath.add("user", 123L); - final Subspace pathSubspace = userPath.toSubspace(context); - - // Add additional tuple elements after the KeySpacePath (this is how data is actually stored) - byte[] keyBytes = pathSubspace.pack(Tuple.from("record_id", 456L, "metadata")); - byte[] valueBytes = Tuple.from("test_data").pack(); - - tr.set(keyBytes, valueBytes); - KeyValue keyValue = new KeyValue(keyBytes, valueBytes); - - final List queryPaths = List.of(appPath, localityPath, userPath); - DataInKeySpacePath dataInPath = new DataInKeySpacePath( - queryPaths.get(depth), keyValue, context); - - // Verify the resolved path - CompletableFuture resolvedFuture = dataInPath.getResolvedPath(); - assertNotNull(resolvedFuture); - - ResolvedKeySpacePath resolved = resolvedFuture.join(); - assertNotNull(resolved); - - // Verify the resolved path has the correct structure - assertEquals("user", resolved.getDirectoryName()); - assertEquals(123L, resolved.getResolvedValue()); - - // Verify parent path - ResolvedKeySpacePath parent = resolved.getParent(); - assertNotNull(parent); - assertEquals("locality", parent.getDirectoryName()); - ResolvedKeySpacePath grandParent = parent.getParent(); - assertNotNull(grandParent); - assertEquals("app", grandParent.getDirectoryName()); - - // Verify the resolved path recreates the KeySpacePath portion (not the full key) - Tuple resolvedTuple = resolved.toTuple(); - assertEquals(TupleHelpers.subTuple(Tuple.fromBytes(keyBytes), 0, 3), resolvedTuple); - - // Verify that the remainder contains the additional tuple elements - Tuple remainder = resolved.getRemainder(); - assertNotNull(remainder); - assertEquals(3, remainder.size()); - assertEquals("record_id", remainder.getString(0)); - assertEquals(456L, remainder.getLong(1)); - assertEquals("metadata", remainder.getString(2)); - - context.commit(); - } - } - - @Test - void testThreeLevelPathWithStringValues() { - KeySpace root = new KeySpace( - new KeySpaceDirectory("service", KeyType.STRING, UUID.randomUUID().toString()) - .addSubdirectory(new KeySpaceDirectory("region", KeyType.STRING) - .addSubdirectory(new KeySpaceDirectory("instance", KeyType.STRING)))); - - final FDBDatabase database = dbExtension.getDatabase(); - - try (FDBRecordContext context = database.openContext()) { - Transaction tr = context.ensureActive(); - - KeySpacePath instancePath = root.path("service") - .add("region", "us-west-2") - .add("instance", "i-1234567890"); - - // Add additional tuple elements after the KeySpacePath - byte[] keyBytes = instancePath.toSubspace(context).pack( - Tuple.from("process_id", "web-server", "port", 8080L)); - byte[] valueBytes = Tuple.from("instance_data").pack(); - - tr.set(keyBytes, valueBytes); - KeyValue keyValue = new KeyValue(keyBytes, valueBytes); - - // Create DataInKeySpacePath from the service-level path - KeySpacePath servicePath = root.path("service"); - DataInKeySpacePath dataInPath = new DataInKeySpacePath(servicePath, keyValue, context); - - ResolvedKeySpacePath resolved = dataInPath.getResolvedPath().join(); - - // Verify the deepest level - assertEquals("instance", resolved.getDirectoryName()); - assertEquals("i-1234567890", resolved.getResolvedValue()); - - // Verify middle level - ResolvedKeySpacePath regionLevel = resolved.getParent(); - assertNotNull(regionLevel); - assertEquals("region", regionLevel.getDirectoryName()); - assertEquals("us-west-2", regionLevel.getResolvedValue()); - - // Verify top level - ResolvedKeySpacePath serviceLevel = regionLevel.getParent(); - assertNotNull(serviceLevel); - assertEquals("service", serviceLevel.getDirectoryName()); - - // Verify the resolved path recreates the KeySpacePath portion - assertEquals(TupleHelpers.subTuple(Tuple.fromBytes(keyBytes), 0, 3), resolved.toTuple()); - - // Verify that the remainder contains the additional tuple elements - Tuple remainder = resolved.getRemainder(); - assertNotNull(remainder); - assertEquals(4, remainder.size()); - assertEquals("process_id", remainder.getString(0)); - assertEquals("web-server", remainder.getString(1)); - assertEquals("port", remainder.getString(2)); - assertEquals(8080L, remainder.getLong(3)); - - context.commit(); - } - } - - @Test - void testDeepFiveLevelPathWithMixedTypes() { - KeySpace root = new KeySpace( - new KeySpaceDirectory("company", KeyType.STRING, UUID.randomUUID().toString()) + new KeySpaceDirectory("company", KeyType.STRING, companyUuid) .addSubdirectory(new KeySpaceDirectory("department", KeyType.STRING) - .addSubdirectory(new KeySpaceDirectory("team_id", KeyType.LONG) - .addSubdirectory(new KeySpaceDirectory("employee_uuid", KeyType.UUID) - .addSubdirectory(new KeySpaceDirectory("active", KeyType.BOOLEAN)))))); + .addSubdirectory(new KeySpaceDirectory("team_id", KeyType.LONG) + .addSubdirectory(new KeySpaceDirectory("employee_uuid", KeyType.UUID) + .addSubdirectory(new KeySpaceDirectory("active", KeyType.BOOLEAN) + .addSubdirectory(new KeySpaceDirectory("data", KeyType.NULL, null)) + .addSubdirectory(new KeySpaceDirectory("metaData", KeyType.LONG, 0)))) + .addSubdirectory(new KeySpaceDirectory("buildings", KeyType.STRING))))); final FDBDatabase database = dbExtension.getDatabase(); try (FDBRecordContext context = database.openContext()) { Transaction tr = context.ensureActive(); - + UUID employeeId = UUID.randomUUID(); KeySpacePath employeePath = root.path("company") .add("department", "engineering") .add("team_id", 42L) .add("employee_uuid", employeeId) - .add("active", true); - + .add("active", true) + .add("data"); + // Add additional tuple elements after the KeySpacePath - byte[] keyBytes = employeePath.toSubspace(context).pack( - Tuple.from("salary", 75000L, "start_date", "2023-01-15")); + final Tuple remainderTuple = Tuple.from("salary", 75000L, "start_date", "2023-01-15"); + byte[] keyBytes = employeePath.toSubspace(context).pack(remainderTuple); byte[] valueBytes = Tuple.from("employee_record").pack(); - + tr.set(keyBytes, valueBytes); KeyValue keyValue = new KeyValue(keyBytes, valueBytes); - + // Create DataInKeySpacePath from the company-level path KeySpacePath companyPath = root.path("company"); DataInKeySpacePath dataInPath = new DataInKeySpacePath(companyPath, keyValue, context); - + ResolvedKeySpacePath resolved = dataInPath.getResolvedPath().join(); - - // Verify the deepest level (active) - assertEquals("active", resolved.getDirectoryName()); - assertEquals(true, resolved.getResolvedValue()); - - // Verify employee_uuid level - ResolvedKeySpacePath uuidLevel = resolved.getParent(); - assertNotNull(uuidLevel); - assertEquals("employee_uuid", uuidLevel.getDirectoryName()); - assertEquals(employeeId, uuidLevel.getResolvedValue()); - - // Verify team_id level - ResolvedKeySpacePath teamLevel = uuidLevel.getParent(); - assertNotNull(teamLevel); - assertEquals("team_id", teamLevel.getDirectoryName()); - assertEquals(42L, teamLevel.getResolvedValue()); - - // Verify department level - ResolvedKeySpacePath deptLevel = teamLevel.getParent(); - assertNotNull(deptLevel); - assertEquals("department", deptLevel.getDirectoryName()); - assertEquals("engineering", deptLevel.getResolvedValue()); - - // Verify company level - ResolvedKeySpacePath companyLevel = deptLevel.getParent(); - assertNotNull(companyLevel); - assertEquals("company", companyLevel.getDirectoryName()); - + + // Verify the path + ResolvedKeySpacePath activeLevel = assertNameAndValue(resolved, "data", null); + ResolvedKeySpacePath uuidLevel = assertNameAndValue(activeLevel, "active", true); + ResolvedKeySpacePath teamLevel = assertNameAndValue(uuidLevel, "employee_uuid", employeeId); + ResolvedKeySpacePath deptLevel = assertNameAndValue(teamLevel, "team_id", 42L); + ResolvedKeySpacePath companyLevel = assertNameAndValue(deptLevel, "department", "engineering"); + assertNull(assertNameAndValue(companyLevel, "company", companyUuid)); + // Verify the resolved path recreates the KeySpacePath portion - assertEquals(TupleHelpers.subTuple(Tuple.fromBytes(keyBytes), 0, 5), resolved.toTuple()); - + assertEquals(TupleHelpers.subTuple(Tuple.fromBytes(keyBytes), 0, 6), resolved.toTuple()); + // Verify that the remainder contains the additional tuple elements - Tuple remainder = resolved.getRemainder(); - assertNotNull(remainder); - assertEquals(4, remainder.size()); - assertEquals("salary", remainder.getString(0)); - assertEquals(75000L, remainder.getLong(1)); - assertEquals("start_date", remainder.getString(2)); - assertEquals("2023-01-15", remainder.getString(3)); - + assertEquals(remainderTuple, resolved.getRemainder()); + context.commit(); } } @Test void testPathWithConstantValues() { + final String appUuid = UUID.randomUUID().toString(); KeySpace root = new KeySpace( - new KeySpaceDirectory("application", KeyType.STRING, UUID.randomUUID().toString()) + new KeySpaceDirectory("application", KeyType.STRING, appUuid) .addSubdirectory(new KeySpaceDirectory("version", KeyType.LONG, 1L) .addSubdirectory(new KeySpaceDirectory("environment", KeyType.STRING, "production") .addSubdirectory(new KeySpaceDirectory("data", KeyType.STRING))))); @@ -289,26 +144,11 @@ void testPathWithConstantValues() { ResolvedKeySpacePath resolved = dataInPath.getResolvedPath().join(); - // Verify the deepest level - assertEquals("data", resolved.getDirectoryName()); - assertEquals("user_records", resolved.getResolvedValue()); - - // Verify environment level (constant value) - ResolvedKeySpacePath envLevel = resolved.getParent(); - assertNotNull(envLevel); - assertEquals("environment", envLevel.getDirectoryName()); - assertEquals("production", envLevel.getResolvedValue()); - - // Verify version level (constant value) - ResolvedKeySpacePath versionLevel = envLevel.getParent(); - assertNotNull(versionLevel); - assertEquals("version", versionLevel.getDirectoryName()); - assertEquals(1L, versionLevel.getResolvedValue()); - - // Verify application level - ResolvedKeySpacePath applicationLevel = versionLevel.getParent(); - assertNotNull(applicationLevel); - assertEquals("application", applicationLevel.getDirectoryName()); + // Verify the path using assertNameAndValue + ResolvedKeySpacePath envLevel = assertNameAndValue(resolved, "data", "user_records"); + ResolvedKeySpacePath versionLevel = assertNameAndValue(envLevel, "environment", "production"); + ResolvedKeySpacePath applicationLevel = assertNameAndValue(versionLevel, "version", 1L); + assertNull(assertNameAndValue(applicationLevel, "application", appUuid)); // Verify the resolved path recreates the KeySpacePath portion assertEquals(TupleHelpers.subTuple(Tuple.fromBytes(keyBytes), 0, 4), resolved.toTuple()); @@ -328,8 +168,9 @@ void testPathWithConstantValues() { @Test void testPathWithDirectoryLayer() { + final String tenantUuid = UUID.randomUUID().toString(); KeySpace root = new KeySpace( - new DirectoryLayerDirectory("tenant", UUID.randomUUID().toString()) + new DirectoryLayerDirectory("tenant", tenantUuid) .addSubdirectory(new KeySpaceDirectory("user_id", KeyType.LONG) .addSubdirectory(new DirectoryLayerDirectory("service")))); @@ -353,34 +194,23 @@ void testPathWithDirectoryLayer() { KeySpacePath tenantPath = root.path("tenant"); DataInKeySpacePath dataInPath = new DataInKeySpacePath(tenantPath, keyValue, context); - ResolvedKeySpacePath resolved = dataInPath.getResolvedPath().join(); - - // Verify the deepest level (service - DirectoryLayer) - assertEquals("service", resolved.getDirectoryName()); - assertEquals("analytics", resolved.getLogicalValue()); - - // Verify user_id level - ResolvedKeySpacePath userLevel = resolved.getParent(); - assertNotNull(userLevel); - assertEquals("user_id", userLevel.getDirectoryName()); - assertEquals(999L, userLevel.getLogicalValue()); - - // Verify tenant level (DirectoryLayer) - ResolvedKeySpacePath tenantLevel = userLevel.getParent(); - assertNotNull(tenantLevel); - assertEquals("tenant", tenantLevel.getDirectoryName()); - - // Note: DirectoryLayer values are resolved asynchronously, so we verify the structure is correct - assertNotNull(tenantLevel.getResolvedValue()); + ResolvedKeySpacePath serviceLevel = dataInPath.getResolvedPath().join(); + final ResolvedKeySpacePath userLevel = assertNameAndDirectoryScopedValue( + serviceLevel, "service", "analytics", servicePath, context); + ResolvedKeySpacePath tenantLevel = assertNameAndValue(userLevel, "user_id", 999L); + + assertNull(assertNameAndDirectoryScopedValue(tenantLevel, "tenant", tenantUuid, tenantPath, context)); + context.commit(); } } @Test void testPathWithBinaryData() { + final String storeUuid = UUID.randomUUID().toString(); KeySpace root = new KeySpace( - new KeySpaceDirectory("binary_store", KeyType.STRING, UUID.randomUUID().toString()) + new KeySpaceDirectory("binary_store", KeyType.STRING, storeUuid) .addSubdirectory(new KeySpaceDirectory("blob_id", KeyType.BYTES))); final FDBDatabase database = dbExtension.getDatabase(); @@ -404,15 +234,13 @@ void testPathWithBinaryData() { ResolvedKeySpacePath resolved = dataInPath.getResolvedPath().join(); - // Verify the deepest level + // Verify the path using assertNameAndValue assertEquals("blob_id", resolved.getDirectoryName()); byte[] resolvedBytes = (byte[]) resolved.getResolvedValue(); assertArrayEquals(blobId, resolvedBytes); - // Verify parent level - ResolvedKeySpacePath storeLevel = resolved.getParent(); - assertNotNull(storeLevel); - assertEquals("binary_store", storeLevel.getDirectoryName()); + ResolvedKeySpacePath storeLevel = assertNameAndValue(resolved.getParent(), "binary_store", storeUuid); + assertNull(storeLevel); // Verify the resolved path can recreate the original key assertEquals(keyTuple, resolved.toTuple()); @@ -425,7 +253,8 @@ void testPathWithBinaryData() { @ValueSource(ints = {1, 2, 3, 4, 5}) void testVariableDepthPaths(int depth) { // Build a KeySpace with the specified depth - KeySpaceDirectory rootDir = new KeySpaceDirectory("root", KeyType.STRING, UUID.randomUUID().toString()); + final String rootUuid = UUID.randomUUID().toString(); + KeySpaceDirectory rootDir = new KeySpaceDirectory("root", KeyType.STRING, rootUuid); KeySpace root = new KeySpace(rootDir); KeySpaceDirectory dir = rootDir; for (int i = 1; i < depth; i++) { @@ -459,9 +288,17 @@ void testVariableDepthPaths(int depth) { ResolvedKeySpacePath resolved = dataInPath.getResolvedPath().join(); + // Verify the path using assertNameAndValue by traversing up the hierarchy + ResolvedKeySpacePath current = resolved; + for (int i = depth - 1; i >= 1; i--) { + current = assertNameAndValue(current, "level" + i, i * 100L); + } + // Finally verify the root level + assertNull(assertNameAndValue(current, "root", rootUuid)); + // Verify the depth by traversing up the path int actualDepth = 1; // Start at 1 for the root - ResolvedKeySpacePath current = resolved; + current = resolved; while (current.getParent() != null) { actualDepth++; current = current.getParent(); @@ -514,8 +351,9 @@ void testKeyValueAccessors() { @Test void testNullKeyTypeDirectory() { + final String baseUuid = UUID.randomUUID().toString(); KeySpace root = new KeySpace( - new KeySpaceDirectory("base", KeyType.STRING, UUID.randomUUID().toString()) + new KeySpaceDirectory("base", KeyType.STRING, baseUuid) .addSubdirectory(new KeySpaceDirectory("null_dir", KeyType.NULL))); final FDBDatabase database = dbExtension.getDatabase(); @@ -538,15 +376,9 @@ void testNullKeyTypeDirectory() { ResolvedKeySpacePath resolved = dataInPath.getResolvedPath().join(); - // Verify the deepest level (NULL type) - assertEquals("null_dir", resolved.getDirectoryName()); - // NULL type directories have null as their resolved value - assertNull(resolved.getResolvedValue()); - - // Verify parent level - ResolvedKeySpacePath baseLevel = resolved.getParent(); - assertNotNull(baseLevel); - assertEquals("base", baseLevel.getDirectoryName()); + // Verify the path using assertNameAndValue + ResolvedKeySpacePath baseLevel = assertNameAndValue(resolved, "null_dir", null); + assertNull(assertNameAndValue(baseLevel, "base", baseUuid)); // Verify the resolved path can recreate the original key assertEquals(keyTuple, resolved.toTuple()); @@ -563,7 +395,8 @@ void testWithWrapper() { // Test export at different levels through wrapper methods try (FDBRecordContext context = database.openContext()) { // Test 4: Export from specific data store level - EnvironmentKeySpace.DataPath dataStore = keySpace.root().userid(100L).application("app1").dataStore(); + final EnvironmentKeySpace.ApplicationPath appPath = keySpace.root().userid(100L).application("app1"); + EnvironmentKeySpace.DataPath dataStore = appPath.dataStore(); final byte[] key = dataStore.toTuple(context).add("record2").add(0).pack(); final byte[] value = Tuple.from("data").pack(); @@ -572,9 +405,37 @@ void testWithWrapper() { final ResolvedKeySpacePath resolvedPath = dataInKeySpacePath.getResolvedPath().join(); assertEquals(dataStore.toResolvedPath(context), withoutRemainder(resolvedPath)); assertEquals(Tuple.from("record2", 0), resolvedPath.getRemainder()); + + // Verify the path using assertNameAndValue + // Note: We expect the path to be: [environment] -> userid -> application -> data + ResolvedKeySpacePath appLevel; + appLevel = assertNameAndValue(resolvedPath, "data", EnvironmentKeySpace.DATA_VALUE); + ResolvedKeySpacePath userLevel = assertNameAndDirectoryScopedValue(appLevel, "application", "app1", + appPath, context); + ResolvedKeySpacePath envLevel = assertNameAndValue(userLevel, "userid", 100L); + assertNull(assertNameAndDirectoryScopedValue(envLevel, keySpace.root().getDirectoryName(), + keySpace.root().getValue(), keySpace.root(), context)); } } + private static ResolvedKeySpacePath assertNameAndDirectoryScopedValue(ResolvedKeySpacePath resolved, + String name, Object logicalValue, + KeySpacePath path, FDBRecordContext context) { + assertNotNull(resolved); + assertEquals(name, resolved.getDirectoryName()); + assertEquals(path.toResolvedPath(context).getResolvedValue(), resolved.getResolvedValue()); + assertEquals(logicalValue, resolved.getLogicalValue()); + return resolved.getParent(); + } + + private static ResolvedKeySpacePath assertNameAndValue(ResolvedKeySpacePath resolved, String name, Object value) { + assertNotNull(resolved); + assertEquals(name, resolved.getDirectoryName()); + assertEquals(value, resolved.getResolvedValue()); + assertEquals(value, resolved.getLogicalValue()); + return resolved.getParent(); + } + private ResolvedKeySpacePath withoutRemainder(final ResolvedKeySpacePath path) { return new ResolvedKeySpacePath(path.getParent(), path.toPath(), path.getResolvedPathValue(), null); } From 5a24630ff58abd6dd45c459195b98f9e57ff53e4 Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Thu, 4 Sep 2025 16:41:57 -0400 Subject: [PATCH 16/32] Dleete some more redundant tests --- .../keyspace/DataInKeySpacePathTest.java | 113 +----------------- 1 file changed, 2 insertions(+), 111 deletions(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java index 48658b456c..acb1689ac9 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java @@ -118,8 +118,8 @@ void testPathWithConstantValues() { KeySpace root = new KeySpace( new KeySpaceDirectory("application", KeyType.STRING, appUuid) .addSubdirectory(new KeySpaceDirectory("version", KeyType.LONG, 1L) - .addSubdirectory(new KeySpaceDirectory("environment", KeyType.STRING, "production") - .addSubdirectory(new KeySpaceDirectory("data", KeyType.STRING))))); + .addSubdirectory(new KeySpaceDirectory("environment", KeyType.STRING, "production") + .addSubdirectory(new KeySpaceDirectory("data", KeyType.STRING))))); final FDBDatabase database = dbExtension.getDatabase(); @@ -249,77 +249,6 @@ void testPathWithBinaryData() { } } - @ParameterizedTest - @ValueSource(ints = {1, 2, 3, 4, 5}) - void testVariableDepthPaths(int depth) { - // Build a KeySpace with the specified depth - final String rootUuid = UUID.randomUUID().toString(); - KeySpaceDirectory rootDir = new KeySpaceDirectory("root", KeyType.STRING, rootUuid); - KeySpace root = new KeySpace(rootDir); - KeySpaceDirectory dir = rootDir; - for (int i = 1; i < depth; i++) { - final KeySpaceDirectory next = new KeySpaceDirectory("level" + i, KeyType.LONG); - dir.addSubdirectory(next); - dir = next; - } - - final FDBDatabase database = dbExtension.getDatabase(); - - try (FDBRecordContext context = database.openContext()) { - Transaction tr = context.ensureActive(); - - // Build a path with the specified depth - KeySpacePath currentPath = root.path("root"); - for (int i = 1; i < depth; i++) { - currentPath = currentPath.add("level" + i, i * 100L); - } - - Tuple keyTuple = currentPath.toTuple(context); - byte[] keyBytes = keyTuple.pack(); - byte[] valueBytes = Tuple.from("depth_test_" + depth).pack(); - - tr.set(keyBytes, valueBytes); - KeyValue keyValue = new KeyValue(keyBytes, valueBytes); - - // Create DataInKeySpacePath from the root-level path - KeySpacePath rootPath = root.path("root"); - root.resolveFromKey(context, Tuple.fromBytes(keyValue.getKey())); - DataInKeySpacePath dataInPath = new DataInKeySpacePath(rootPath, keyValue, context); - - ResolvedKeySpacePath resolved = dataInPath.getResolvedPath().join(); - - // Verify the path using assertNameAndValue by traversing up the hierarchy - ResolvedKeySpacePath current = resolved; - for (int i = depth - 1; i >= 1; i--) { - current = assertNameAndValue(current, "level" + i, i * 100L); - } - // Finally verify the root level - assertNull(assertNameAndValue(current, "root", rootUuid)); - - // Verify the depth by traversing up the path - int actualDepth = 1; // Start at 1 for the root - current = resolved; - while (current.getParent() != null) { - actualDepth++; - current = current.getParent(); - } - assertEquals(depth, actualDepth); - - // Verify the deepest level has the expected name and value - if (depth > 1) { - assertEquals("level" + (depth - 1), resolved.getDirectoryName()); - assertEquals((depth - 1) * 100L, resolved.getResolvedValue()); - } else { - assertEquals("root", resolved.getDirectoryName()); - } - - // Verify the resolved path can recreate the original key - assertEquals(keyTuple, resolved.toTuple()); - - context.commit(); - } - } - @Test void testKeyValueAccessors() { KeySpace root = new KeySpace( @@ -349,44 +278,6 @@ void testKeyValueAccessors() { } } - @Test - void testNullKeyTypeDirectory() { - final String baseUuid = UUID.randomUUID().toString(); - KeySpace root = new KeySpace( - new KeySpaceDirectory("base", KeyType.STRING, baseUuid) - .addSubdirectory(new KeySpaceDirectory("null_dir", KeyType.NULL))); - - final FDBDatabase database = dbExtension.getDatabase(); - - try (FDBRecordContext context = database.openContext()) { - Transaction tr = context.ensureActive(); - - KeySpacePath nullPath = root.path("base").add("null_dir"); - - Tuple keyTuple = nullPath.toTuple(context); - byte[] keyBytes = keyTuple.pack(); - byte[] valueBytes = Tuple.from("null_type_test").pack(); - - tr.set(keyBytes, valueBytes); - KeyValue keyValue = new KeyValue(keyBytes, valueBytes); - - // Create DataInKeySpacePath from the base-level path - KeySpacePath basePath = root.path("base"); - DataInKeySpacePath dataInPath = new DataInKeySpacePath(basePath, keyValue, context); - - ResolvedKeySpacePath resolved = dataInPath.getResolvedPath().join(); - - // Verify the path using assertNameAndValue - ResolvedKeySpacePath baseLevel = assertNameAndValue(resolved, "null_dir", null); - assertNull(assertNameAndValue(baseLevel, "base", baseUuid)); - - // Verify the resolved path can recreate the original key - assertEquals(keyTuple, resolved.toTuple()); - - context.commit(); - } - } - @Test void testWithWrapper() { final FDBDatabase database = dbExtension.getDatabase(); From f1e436a6c475e1358beb6da2bf923662f83287d6 Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Thu, 4 Sep 2025 17:03:31 -0400 Subject: [PATCH 17/32] Some minor test cleanup --- .../keyspace/DataInKeySpacePathTest.java | 12 ++--- .../keyspace/KeySpacePathDataExportTest.java | 45 ++++++++++++------- 2 files changed, 34 insertions(+), 23 deletions(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java index acb1689ac9..1390e3a755 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java @@ -54,7 +54,7 @@ class DataInKeySpacePathTest { @ParameterizedTest @ValueSource(ints = {0, 1, 2, 3, 4, 5}) - void testResolution() { + void resolution() { // Include some extra children to make sure resolution doesn't get confused final String companyUuid = UUID.randomUUID().toString(); KeySpace root = new KeySpace( @@ -113,7 +113,7 @@ void testResolution() { } @Test - void testPathWithConstantValues() { + void pathWithConstantValues() { final String appUuid = UUID.randomUUID().toString(); KeySpace root = new KeySpace( new KeySpaceDirectory("application", KeyType.STRING, appUuid) @@ -167,7 +167,7 @@ void testPathWithConstantValues() { } @Test - void testPathWithDirectoryLayer() { + void pathWithDirectoryLayer() { final String tenantUuid = UUID.randomUUID().toString(); KeySpace root = new KeySpace( new DirectoryLayerDirectory("tenant", tenantUuid) @@ -207,7 +207,7 @@ void testPathWithDirectoryLayer() { } @Test - void testPathWithBinaryData() { + void pathWithBinaryData() { final String storeUuid = UUID.randomUUID().toString(); KeySpace root = new KeySpace( new KeySpaceDirectory("binary_store", KeyType.STRING, storeUuid) @@ -250,7 +250,7 @@ void testPathWithBinaryData() { } @Test - void testKeyValueAccessors() { + void keyValueAccessors() { KeySpace root = new KeySpace( new KeySpaceDirectory("test", KeyType.STRING, UUID.randomUUID().toString())); @@ -279,7 +279,7 @@ void testKeyValueAccessors() { } @Test - void testWithWrapper() { + void withWrapper() { final FDBDatabase database = dbExtension.getDatabase(); final EnvironmentKeySpace keySpace = EnvironmentKeySpace.setupSampleData(database); diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java index abfd6cdcab..d835a8d2ef 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java @@ -65,7 +65,7 @@ class KeySpacePathDataExportTest { final FDBDatabaseExtension dbExtension = new FDBDatabaseExtension(); @Test - void testExportAllDataFromSimplePath() { + void exportAllDataFromSimplePath() { KeySpace root = new KeySpace( new KeySpaceDirectory("root", KeyType.STRING, UUID.randomUUID().toString()) .addSubdirectory(new KeySpaceDirectory("level1", KeyType.LONG))); @@ -108,7 +108,7 @@ void testExportAllDataFromSimplePath() { } @Test - void testExportAllDataFromSpecificSubPath() { + void exportAllDataFromSpecificSubPath() { KeySpace root = new KeySpace( new KeySpaceDirectory("app", KeyType.STRING, UUID.randomUUID().toString()) .addSubdirectory(new KeySpaceDirectory("user", KeyType.LONG) @@ -150,7 +150,7 @@ void testExportAllDataFromSpecificSubPath() { } @Test - void testExportAllDataWithDirectoryLayer() { + void exportAllDataWithDirectoryLayer() { KeySpace root = new KeySpace( new DirectoryLayerDirectory("env", UUID.randomUUID().toString()) .addSubdirectory(new KeySpaceDirectory("tenant", KeyType.LONG) @@ -199,7 +199,7 @@ void testExportAllDataWithDirectoryLayer() { } @Test - void testExportAllDataWithDifferentKeyTypes() { + void exportAllDataWithDifferentKeyTypes() { KeySpace root = new KeySpace( new KeySpaceDirectory("mixed", KeyType.STRING, UUID.randomUUID().toString()) .addSubdirectory(new KeySpaceDirectory("strings", KeyType.STRING)) @@ -250,7 +250,7 @@ void testExportAllDataWithDifferentKeyTypes() { } @Test - void testExportAllDataWithConstantValues() { + void exportAllDataWithConstantValues() { KeySpace root = new KeySpace( new KeySpaceDirectory("app", KeyType.STRING, UUID.randomUUID().toString()) .addSubdirectory(new KeySpaceDirectory("version", KeyType.LONG, 1L) @@ -290,7 +290,7 @@ void testExportAllDataWithConstantValues() { } @Test - void testExportAllDataEmpty() { + void exportAllDataEmpty() { KeySpace root = new KeySpace( new KeySpaceDirectory("empty", KeyType.STRING, UUID.randomUUID().toString()) .addSubdirectory(new KeySpaceDirectory("level1", KeyType.LONG))); @@ -308,7 +308,7 @@ void testExportAllDataEmpty() { } @Test - void testExportAllDataWithDeepNestedStructure() { + void exportAllDataWithDeepNestedStructure() { KeySpace root = new KeySpace( new KeySpaceDirectory("org", KeyType.STRING, UUID.randomUUID().toString()) .addSubdirectory(new KeySpaceDirectory("dept", KeyType.STRING) @@ -372,7 +372,7 @@ void testExportAllDataWithDeepNestedStructure() { } @Test - void testExportAllDataWithBinaryData() { + void exportAllDataWithBinaryData() { KeySpace root = new KeySpace( new KeySpaceDirectory("binary", KeyType.STRING, UUID.randomUUID().toString()) .addSubdirectory(new KeySpaceDirectory("blob", KeyType.BYTES))); @@ -416,7 +416,7 @@ void testExportAllDataWithBinaryData() { @ParameterizedTest @ValueSource(ints = {1, 2, 3, 30}) - void testExportAllDataWithContinuation(int limit) { + void exportAllDataWithContinuation(int limit) { KeySpace root = new KeySpace( new KeySpaceDirectory("continuation", KeyType.STRING, UUID.randomUUID().toString()) .addSubdirectory(new KeySpaceDirectory("item", KeyType.LONG))); @@ -468,7 +468,7 @@ void testExportAllDataWithContinuation(int limit) { } @Test - void testExportAllDataThroughKeySpacePathWrapper() { + void exportAllDataThroughKeySpacePathWrapper() { final FDBDatabase database = dbExtension.getDatabase(); final EnvironmentKeySpace keySpace = EnvironmentKeySpace.setupSampleData(database); @@ -511,7 +511,7 @@ void testExportAllDataThroughKeySpacePathWrapper() { } @Test - void testExportAllDataThroughKeySpacePathWrapperResolvedPaths() { + void exportAllDataThroughKeySpacePathWrapperResolvedPaths() { final FDBDatabase database = dbExtension.getDatabase(); final EnvironmentKeySpace keySpace = EnvironmentKeySpace.setupSampleData(database); @@ -547,27 +547,38 @@ private void setData(List keys, FDBRecordContext context, KeySpacePath b } } - private static List exportAllData(final KeySpacePath rootPath, final FDBRecordContext context) { - final List asSingleExport = rootPath.exportAllData(context, null, ScanProperties.FORWARD_SCAN) + /** + * Export all the data, and make some assertions that can always be done. + * This combines a lot of assertions, but most of the underlying behavior should be well covered by the objects + * that {@link KeySpacePath#exportAllData} is built on. + * @param pathToExport the path being exported + * @param context the context in which to export + * @return a list of the raw {@code KeyValue}s being exported + */ + private static List exportAllData(final KeySpacePath pathToExport, final FDBRecordContext context) { + final List asSingleExport = pathToExport.exportAllData(context, null, ScanProperties.FORWARD_SCAN) .map(DataInKeySpacePath::getRawKeyValue).asList().join(); - final List resolvedPaths = rootPath.exportAllData(context, null, ScanProperties.FORWARD_SCAN) + // assert that the resolved paths contain the right prefix + final List resolvedPaths = pathToExport.exportAllData(context, null, ScanProperties.FORWARD_SCAN) .mapPipelined(DataInKeySpacePath::getResolvedPath, 1).asList().join(); - final ResolvedKeySpacePath rootResolvedPath = rootPath.toResolvedPath(context); + final ResolvedKeySpacePath rootResolvedPath = pathToExport.toResolvedPath(context); for (ResolvedKeySpacePath resolvedPath : resolvedPaths) { assertStartsWith(rootResolvedPath, resolvedPath); } - final List reversed = rootPath.exportAllData(context, null, ScanProperties.REVERSE_SCAN) + // assert that the reverse scan is the same as the forward scan, but in reverse + final List reversed = pathToExport.exportAllData(context, null, ScanProperties.REVERSE_SCAN) .map(DataInKeySpacePath::getRawKeyValue).asList().join(); Collections.reverse(reversed); assertEquals(asSingleExport, reversed); + // Assert continuations work correctly final ScanProperties scanProperties = ScanProperties.FORWARD_SCAN.with(props -> props.setReturnedRowLimit(1)); List asContinuations = new ArrayList<>(); RecordCursorContinuation continuation = RecordCursorStartContinuation.START; while (!continuation.isEnd()) { - final RecordCursor cursor = rootPath.exportAllData(context, continuation.toBytes(), + final RecordCursor cursor = pathToExport.exportAllData(context, continuation.toBytes(), scanProperties); final AtomicReference> keyValueResult = new AtomicReference<>(); final List batch = cursor.map(DataInKeySpacePath::getRawKeyValue).asList(keyValueResult).join(); From 5e310ea360c4d25cbee6ba490c072574322b2306 Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Fri, 17 Oct 2025 17:21:56 -0400 Subject: [PATCH 18/32] Fix test that doesn't account for remainder I left a TODO for myself to add unit tests of withRemainder... --- .../foundationdb/keyspace/ResolvedKeySpacePath.java | 6 ++++++ .../foundationdb/keyspace/KeySpacePathDataExportTest.java | 7 ++++--- 2 files changed, 10 insertions(+), 3 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/ResolvedKeySpacePath.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/ResolvedKeySpacePath.java index 0afd60db10..2142bd382f 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/ResolvedKeySpacePath.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/ResolvedKeySpacePath.java @@ -271,4 +271,10 @@ public static void appendValue(StringBuilder sb, Object value) { sb.append(value); } } + + // TODO test this specifically + public ResolvedKeySpacePath withRemainder(final Tuple newRemainder) { + // this could probably copy the cachedTuple & cachedSubspace + return new ResolvedKeySpacePath(parent, inner, value, newRemainder); + } } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java index d835a8d2ef..567b9c7116 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java @@ -596,12 +596,13 @@ private static List exportAllData(final KeySpacePath pathToExport, fin } private static void assertStartsWith(final ResolvedKeySpacePath rootResolvedPath, ResolvedKeySpacePath resolvedPath) { + ResolvedKeySpacePath searchPath = resolvedPath.withRemainder(null); do { - if (resolvedPath.equals(rootResolvedPath)) { + if (searchPath.equals(rootResolvedPath)) { return; } - resolvedPath = resolvedPath.getParent(); - } while (resolvedPath != null); + searchPath = searchPath.getParent(); + } while (searchPath != null); Assertions.fail("Expected <" + resolvedPath + "> to start with <" + rootResolvedPath + "> but it didn't"); } From 2a5da680a4580e88e3648bcbf4b28c8cae54c09e Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Mon, 20 Oct 2025 11:11:10 -0400 Subject: [PATCH 19/32] Add unit tests of withRemainder --- .../keyspace/ResolvedKeySpacePath.java | 9 ++- .../keyspace/ResolvedKeySpacePathTest.java | 76 ++++++++++++++++++- 2 files changed, 82 insertions(+), 3 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/ResolvedKeySpacePath.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/ResolvedKeySpacePath.java index 2142bd382f..728f5162a7 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/ResolvedKeySpacePath.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/ResolvedKeySpacePath.java @@ -272,8 +272,13 @@ public static void appendValue(StringBuilder sb, Object value) { } } - // TODO test this specifically - public ResolvedKeySpacePath withRemainder(final Tuple newRemainder) { + /** + * Returns a new {@code ResolvedKeySpacePath} that is the same, except with the provided {@link #getRemainder()}. + * @param newRemainder a new remainder. This can be {@code null} to remove the remainder entirely. + * @return a new {@code ResolvedKeySpacePath} that is the same as this, except with a different {@link #getRemainder()}. + */ + @Nonnull + public ResolvedKeySpacePath withRemainder(@Nullable final Tuple newRemainder) { // this could probably copy the cachedTuple & cachedSubspace return new ResolvedKeySpacePath(parent, inner, value, newRemainder); } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/ResolvedKeySpacePathTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/ResolvedKeySpacePathTest.java index be82a58eb6..a86f3d4bd5 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/ResolvedKeySpacePathTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/ResolvedKeySpacePathTest.java @@ -224,6 +224,80 @@ void testRemainderComparedInEquals(boolean constantDirectory) { .orElseThrow(() -> new AssertionError("Paths with different remainders should sometimes have different hash codes")); } + /** Test withRemainder returns a new path with the updated remainder. */ + @ParameterizedTest + @BooleanSource("constantDirectory") + void testWithRemainder(boolean constantDirectory) { + KeySpacePath innerPath = createKeySpacePath(createRootParent(), KeyType.STRING, "resolved", constantDirectory); + PathValue value = new PathValue("resolved", null); + Tuple originalRemainder = Tuple.from("remainder1"); + + ResolvedKeySpacePath original = new ResolvedKeySpacePath(null, innerPath, value, originalRemainder); + + // Test changing remainder + Tuple newRemainder = Tuple.from("remainder2"); + ResolvedKeySpacePath modified = original.withRemainder(newRemainder); + + assertEquals(newRemainder, modified.getRemainder()); + assertEquals(originalRemainder, original.getRemainder(), "Original should be unchanged"); + assertNotEquals(original, modified, "Paths with different remainders should not be equal"); + + // Verify other fields are preserved + assertEquals(original.toPath(), modified.toPath()); + assertEquals(original.getResolvedValue(), modified.getResolvedValue()); + assertEquals(original.getResolvedPathValue(), modified.getResolvedPathValue()); + + // Verify modified is equal to a freshly created path with the new remainder + ResolvedKeySpacePath freshlyCreated = new ResolvedKeySpacePath(null, innerPath, value, newRemainder); + assertEquals(freshlyCreated, modified); + assertEquals(modified, freshlyCreated); + assertEquals(freshlyCreated.hashCode(), modified.hashCode()); + } + + /** Test withRemainder can set remainder to null. */ + @ParameterizedTest + @BooleanSource("constantDirectory") + void testWithRemainderSetToNull(boolean constantDirectory) { + KeySpacePath innerPath = createKeySpacePath(createRootParent(), KeyType.STRING, "resolved", constantDirectory); + PathValue value = new PathValue("resolved", null); + Tuple originalRemainder = Tuple.from("remainder1"); + + ResolvedKeySpacePath original = new ResolvedKeySpacePath(null, innerPath, value, originalRemainder); + ResolvedKeySpacePath modified = original.withRemainder(null); + + assertNull(modified.getRemainder()); + assertEquals(originalRemainder, original.getRemainder(), "Original should be unchanged"); + assertNotEquals(original, modified); + + // Verify modified is equal to a freshly created path with null remainder + ResolvedKeySpacePath freshlyCreated = new ResolvedKeySpacePath(null, innerPath, value, null); + assertEquals(freshlyCreated, modified); + assertEquals(modified, freshlyCreated); + assertEquals(freshlyCreated.hashCode(), modified.hashCode()); + } + + /** Test withRemainder on a path that already has null remainder. */ + @ParameterizedTest + @BooleanSource("constantDirectory") + void testWithRemainderFromNull(boolean constantDirectory) { + KeySpacePath innerPath = createKeySpacePath(createRootParent(), KeyType.STRING, "resolved", constantDirectory); + PathValue value = new PathValue("resolved", null); + + ResolvedKeySpacePath original = new ResolvedKeySpacePath(null, innerPath, value, null); + Tuple newRemainder = Tuple.from("newRemainder"); + ResolvedKeySpacePath modified = original.withRemainder(newRemainder); + + assertEquals(newRemainder, modified.getRemainder()); + assertNull(original.getRemainder(), "Original should be unchanged"); + assertNotEquals(original, modified); + + // Verify modified is equal to a freshly created path with the new remainder + ResolvedKeySpacePath freshlyCreated = new ResolvedKeySpacePath(null, innerPath, value, newRemainder); + assertEquals(freshlyCreated, modified); + assertEquals(modified, freshlyCreated); + assertEquals(freshlyCreated.hashCode(), modified.hashCode()); + } + @Nonnull private KeySpacePath createKeySpacePath(@Nonnull ResolvedKeySpacePath parent, @Nonnull KeyType keyType, @Nullable Object value, boolean constantDirectory) { @@ -235,7 +309,7 @@ private KeySpacePath createKeySpacePath(@Nonnull ResolvedKeySpacePath parent, @N childDir = new KeySpaceDirectory("test", keyType); } parent.getDirectory().addSubdirectory(childDir); - + if (constantDirectory) { return parent.toPath().add("test"); } else { From 9207a3e676876f22b14974e1ac122b118cc2249f Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Mon, 20 Oct 2025 12:19:08 -0400 Subject: [PATCH 20/32] Test reverse continuations --- .../keyspace/KeySpacePathDataExportTest.java | 31 +++++++++++++++---- 1 file changed, 25 insertions(+), 6 deletions(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java index 567b9c7116..935179cafe 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java @@ -32,13 +32,15 @@ import com.apple.foundationdb.record.provider.foundationdb.keyspace.KeySpaceDirectory.KeyType; import com.apple.foundationdb.record.test.FDBDatabaseExtension; import com.apple.foundationdb.tuple.Tuple; +import com.apple.test.ParameterizedTestUtils; import com.apple.test.Tags; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.params.ParameterizedTest; -import org.junit.jupiter.params.provider.ValueSource; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import java.util.ArrayList; import java.util.Arrays; @@ -50,6 +52,7 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import java.util.stream.IntStream; +import java.util.stream.Stream; import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -414,9 +417,16 @@ void exportAllDataWithBinaryData() { } } + static Stream exportAllDataWithContinuation() { + return ParameterizedTestUtils.cartesianProduct( + ParameterizedTestUtils.booleans("forward"), + Stream.of(1, 2, 3, 30) + ); + } + @ParameterizedTest - @ValueSource(ints = {1, 2, 3, 30}) - void exportAllDataWithContinuation(int limit) { + @MethodSource + void exportAllDataWithContinuation(boolean forward, int limit) { KeySpace root = new KeySpace( new KeySpaceDirectory("continuation", KeyType.STRING, UUID.randomUUID().toString()) .addSubdirectory(new KeySpaceDirectory("item", KeyType.LONG))); @@ -430,7 +440,13 @@ void exportAllDataWithContinuation(int limit) { Transaction tr = context.ensureActive(); KeySpacePath basePath = root.path("continuation"); - IntStream.range(0, 20).forEach(i -> { + IntStream sourceStream; + if (forward) { + sourceStream = IntStream.range(0, 20); + } else { + sourceStream = IntStream.iterate(19, i -> i >= 0, i -> i - 1); + } + sourceStream.forEach(i -> { Tuple key = basePath.add("item", (long)i).toTuple(context); final Tuple value = Tuple.from("continuation_item_" + i); tr.set(key.pack(), value.pack()); @@ -441,15 +457,18 @@ void exportAllDataWithContinuation(int limit) { }); context.commit(); } + if (20 % limit == 0) { expectedBatches.add(List.of()); } - + // Export with continuation support try (FDBRecordContext context = database.openContext()) { KeySpacePath continuationPath = root.path("continuation"); - final ScanProperties scanProperties = ScanProperties.FORWARD_SCAN.with(props -> props.setReturnedRowLimit(limit)); + final ScanProperties directionalProperties = forward ? ScanProperties.FORWARD_SCAN : ScanProperties.REVERSE_SCAN; + + final ScanProperties scanProperties = directionalProperties.with(props -> props.setReturnedRowLimit(limit)); List> actual = new ArrayList<>(); RecordCursorContinuation continuation = RecordCursorStartContinuation.START; while (!continuation.isEnd()) { From c43d878a67d44e263f45528924770f26e90719a8 Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Mon, 20 Oct 2025 16:19:45 -0400 Subject: [PATCH 21/32] Remove DataInKeySpacePath.getRawKeyValue I updated the tests to not use KeyValue directly, at which point, having DataInKeySpacePath expose the value and not the raw KeyValue seemed to make sense --- .../keyspace/DataInKeySpacePath.java | 15 ++- .../keyspace/DataInKeySpacePathTest.java | 28 ++-- .../keyspace/KeySpacePathDataExportTest.java | 127 ++++++++++-------- 3 files changed, 98 insertions(+), 72 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java index 3db3280a65..a6cb3ed3b0 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java @@ -25,6 +25,7 @@ import com.apple.foundationdb.tuple.Tuple; import com.apple.foundationdb.tuple.TupleHelpers; +import javax.annotation.Nonnull; import java.util.concurrent.CompletableFuture; /** @@ -32,12 +33,12 @@ */ public class DataInKeySpacePath { - final CompletableFuture resolvedPath; - final KeyValue rawKeyValue; + @Nonnull + private final CompletableFuture resolvedPath; + @Nonnull + private final byte[] value; public DataInKeySpacePath(KeySpacePath path, KeyValue rawKeyValue, FDBRecordContext context) { - this.rawKeyValue = rawKeyValue; - // Convert the raw key to a Tuple and resolve it starting from the provided path Tuple keyTuple = Tuple.fromBytes(rawKeyValue.getKey()); @@ -64,13 +65,15 @@ public DataInKeySpacePath(KeySpacePath path, KeyValue rawKeyValue, FDBRecordCont return CompletableFuture.completedFuture(resolvedPath); } }); + + this.value = rawKeyValue.getValue(); } public CompletableFuture getResolvedPath() { return resolvedPath; } - public KeyValue getRawKeyValue() { - return rawKeyValue; + public byte[] getValue() { + return this.value; } } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java index 1390e3a755..36f6a2b4c6 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java @@ -28,6 +28,7 @@ import com.apple.foundationdb.record.test.FDBDatabaseExtension; import com.apple.foundationdb.tuple.Tuple; import com.apple.foundationdb.tuple.TupleHelpers; +import com.apple.test.BooleanSource; import com.apple.test.Tags; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; @@ -37,6 +38,7 @@ import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.ExecutionException; import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -249,8 +251,9 @@ void pathWithBinaryData() { } } - @Test - void keyValueAccessors() { + @ParameterizedTest + @BooleanSource("withRemainder") + void keyValueAccessors(boolean withRemainder) throws ExecutionException, InterruptedException { KeySpace root = new KeySpace( new KeySpaceDirectory("test", KeyType.STRING, UUID.randomUUID().toString())); @@ -258,23 +261,26 @@ void keyValueAccessors() { try (FDBRecordContext context = database.openContext()) { KeySpacePath testPath = root.path("test"); - Tuple keyTuple = testPath.toTuple(context); - byte[] keyBytes = keyTuple.pack(); + Tuple pathTuple = testPath.toTuple(context); + byte[] keyBytes = withRemainder ? pathTuple.add("Remainder").pack() : pathTuple.pack(); byte[] valueBytes = Tuple.from("accessor_test").pack(); KeyValue originalKeyValue = new KeyValue(keyBytes, valueBytes); DataInKeySpacePath dataInPath = new DataInKeySpacePath(testPath, originalKeyValue, context); - - // Verify accessor methods - KeyValue retrievedKeyValue = dataInPath.getRawKeyValue(); - assertNotNull(retrievedKeyValue); - assertEquals(originalKeyValue.getKey(), retrievedKeyValue.getKey()); - assertEquals(originalKeyValue.getValue(), retrievedKeyValue.getValue()); - + // Verify resolved path future is not null CompletableFuture resolvedFuture = dataInPath.getResolvedPath(); assertNotNull(resolvedFuture); assertTrue(resolvedFuture.isDone() || !resolvedFuture.isCancelled()); + + final ResolvedKeySpacePath resolvedPath = resolvedFuture.get(); + assertEquals(pathTuple, resolvedPath.toTuple()); + assertEquals(originalKeyValue.getValue(), dataInPath.getValue()); + if (withRemainder) { + assertEquals(Tuple.from("Remainder"), resolvedPath.getRemainder()); + } else { + assertNull(resolvedPath.getRemainder()); + } } } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java index 935179cafe..75b2cb3681 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java @@ -20,7 +20,6 @@ package com.apple.foundationdb.record.provider.foundationdb.keyspace; -import com.apple.foundationdb.KeyValue; import com.apple.foundationdb.Transaction; import com.apple.foundationdb.record.RecordCursor; import com.apple.foundationdb.record.RecordCursorContinuation; @@ -49,6 +48,7 @@ import java.util.List; import java.util.Set; import java.util.UUID; +import java.util.concurrent.ExecutionException; import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -60,7 +60,7 @@ /** * Tests for the new KeySpacePath data export feature that fetches all data stored under a KeySpacePath - * and returns it in a {@code RecordCursor}. + * and returns it in a {@code RecordCursor}. */ @Tag(Tags.RequiresFDB) class KeySpacePathDataExportTest { @@ -68,7 +68,7 @@ class KeySpacePathDataExportTest { final FDBDatabaseExtension dbExtension = new FDBDatabaseExtension(); @Test - void exportAllDataFromSimplePath() { + void exportAllDataFromSimplePath() throws ExecutionException, InterruptedException { KeySpace root = new KeySpace( new KeySpaceDirectory("root", KeyType.STRING, UUID.randomUUID().toString()) .addSubdirectory(new KeySpaceDirectory("level1", KeyType.LONG))); @@ -97,19 +97,28 @@ void exportAllDataFromSimplePath() { // Export all data from the root path try (FDBRecordContext context = database.openContext()) { KeySpacePath rootPath = root.path("root"); - final List allData = exportAllData(rootPath, context); + final List allData = exportAllData(rootPath, context); // Should have 5 main entries + 15 sub-entries = 20 total assertEquals(20, allData.size()); - + // Verify the data is sorted by key for (int i = 1; i < allData.size(); i++) { - assertTrue(Tuple.fromBytes(allData.get(i - 1).getKey()).compareTo( - Tuple.fromBytes(allData.get(i).getKey())) < 0); + assertTrue(getKey(allData.get(i - 1)).compareTo(getKey(allData.get(i))) < 0); } } } + // `toTuple` does not include the remainder, I'm not sure if that is intentional, or an oversight. + private Tuple getKey(final DataInKeySpacePath dataInKeySpacePath) throws ExecutionException, InterruptedException { + final ResolvedKeySpacePath resolvedKeySpacePath = dataInKeySpacePath.getResolvedPath().get(); + if (resolvedKeySpacePath.getRemainder() != null) { + return resolvedKeySpacePath.toTuple().addAll(resolvedKeySpacePath.getRemainder()); + } else { + return resolvedKeySpacePath.toTuple(); + } + } + @Test void exportAllDataFromSpecificSubPath() { KeySpace root = new KeySpace( @@ -139,14 +148,14 @@ void exportAllDataFromSpecificSubPath() { // Export data only for user 2 try (FDBRecordContext context = database.openContext()) { KeySpacePath user2Path = root.path("app").add("user", 2L); - final List userData = exportAllData(user2Path, context); + final List userData = exportAllData(user2Path, context); // Should have 4 records for user 2 assertEquals(4, userData.size()); - + // Verify all data belongs to user 2 - for (KeyValue kv : userData) { - String value = Tuple.fromBytes(kv.getValue()).getString(0); + for (DataInKeySpacePath data : userData) { + String value = Tuple.fromBytes(data.getValue()).getString(0); assertTrue(value.startsWith("user2_")); } } @@ -184,15 +193,15 @@ void exportAllDataWithDirectoryLayer() { // Export all data from tenant path try (FDBRecordContext context = database.openContext()) { KeySpacePath tenantPath = root.path("env").add("tenant", 100L); - final List allData = exportAllData(tenantPath, context); + final List allData = exportAllData(tenantPath, context); // Should have 6 records (3 services * 2 configs each) assertEquals(6, allData.size()); - + // Verify we have data for all three services Set serviceNames = new HashSet<>(); - for (KeyValue kv : allData) { - String value = Tuple.fromBytes(kv.getValue()).getString(0); + for (DataInKeySpacePath data : allData) { + String value = Tuple.fromBytes(data.getValue()).getString(0); String serviceName = value.split("_")[0]; serviceNames.add(serviceName); } @@ -238,14 +247,14 @@ void exportAllDataWithDifferentKeyTypes() { // Export all data and verify different key types try (FDBRecordContext context = database.openContext()) { KeySpacePath mixedPath = root.path("mixed"); - final List allData = exportAllData(mixedPath, context); + final List allData = exportAllData(mixedPath, context); // Should have 12 records total (3+3+2+2+2) assertEquals(12, allData.size()); - + // Verify we have different value types Set valueTypes = allData.stream() - .map(kv -> Tuple.fromBytes(kv.getValue()).getString(0).split("_")[0]) + .map(data -> Tuple.fromBytes(data.getValue()).getString(0).split("_")[0]) .collect(Collectors.toSet()); assertEquals(5, valueTypes.size()); assertTrue(valueTypes.containsAll(Arrays.asList("string", "long", "bytes", "uuid", "boolean"))); @@ -279,14 +288,14 @@ void exportAllDataWithConstantValues() { // Export data from path with constant values try (FDBRecordContext context = database.openContext()) { KeySpacePath appPath = root.path("app"); - final List allData = exportAllData(appPath, context); + final List allData = exportAllData(appPath, context); // Should have 4 records assertEquals(4, allData.size()); - + // Verify all data has expected prefix - for (KeyValue kv : allData) { - String value = Tuple.fromBytes(kv.getValue()).getString(0); + for (DataInKeySpacePath data : allData) { + String value = Tuple.fromBytes(data.getValue()).getString(0); assertTrue(value.startsWith("constant_path_data_")); } } @@ -303,7 +312,7 @@ void exportAllDataEmpty() { // Don't store any data try (FDBRecordContext context = database.openContext()) { KeySpacePath emptyPath = root.path("empty"); - final List allData = exportAllData(emptyPath, context); + final List allData = exportAllData(emptyPath, context); // Should be empty assertEquals(0, allData.size()); @@ -350,23 +359,23 @@ void exportAllDataWithDeepNestedStructure() { // Export all data from organization root try (FDBRecordContext context = database.openContext()) { KeySpacePath orgPath = root.path("org"); - final List allData = exportAllData(orgPath, context); + final List allData = exportAllData(orgPath, context); // Should have 16 records (2 departments * 2 teams * 2 members * 2 records each) assertEquals(16, allData.size()); } - + // Export data from specific department try (FDBRecordContext context = database.openContext()) { KeySpacePath engPath = root.path("org").add("dept", "engineering"); - final List allData = exportAllData(engPath, context); + final List allData = exportAllData(engPath, context); // Should have 8 records (1 dept * 2 teams * 2 members * 2 records each) assertEquals(8, allData.size()); - + // Verify all belong to engineering - for (KeyValue kv : allData) { - String value = Tuple.fromBytes(kv.getValue()).getString(0); + for (DataInKeySpacePath data : allData) { + String value = Tuple.fromBytes(data.getValue()).getString(0); if (value.startsWith("engineering_")) { assertTrue(value.contains("engineering_")); } @@ -405,13 +414,13 @@ void exportAllDataWithBinaryData() { // Export binary data try (FDBRecordContext context = database.openContext()) { KeySpacePath binaryPath = root.path("binary"); - final List allData = exportAllData(binaryPath, context); + final List allData = exportAllData(binaryPath, context); assertEquals(3, allData.size()); - + // Verify binary data integrity - for (KeyValue kv : allData) { - String valueStr = new String(kv.getValue()); + for (DataInKeySpacePath data : allData) { + String valueStr = new String(data.getValue()); assertTrue(valueStr.startsWith("binary_data_")); } } @@ -476,8 +485,7 @@ void exportAllDataWithContinuation(boolean forward, int limit) { scanProperties); final AtomicReference> tupleResult = new AtomicReference<>(); final List batch = cursor.map(dataInPath -> { - KeyValue kv = dataInPath.getRawKeyValue(); - return Tuple.fromBytes(kv.getValue()); + return Tuple.fromBytes(dataInPath.getValue()); }).asList(tupleResult).join(); actual.add(batch); continuation = tupleResult.get().getContinuation(); @@ -495,9 +503,9 @@ void exportAllDataThroughKeySpacePathWrapper() { try (FDBRecordContext context = database.openContext()) { // Export from root level (should get all data) EnvironmentKeySpace.EnvironmentRoot root = keySpace.root(); - List allData = exportAllData(root, context); + List allData = exportAllData(root, context); assertEquals(6, allData.size(), "Root level should export all data"); - + // Export from specific user level (should get data for user 100 only) EnvironmentKeySpace.UserPath user100Path = keySpace.root().userid(100L); verifyExtractedData(exportAllData(user100Path, context), @@ -512,11 +520,11 @@ void exportAllDataThroughKeySpacePathWrapper() { // Export from specific data store level EnvironmentKeySpace.DataPath dataStore = app1User100.dataStore(); - List dataStoreData = exportAllData(dataStore, context); + List dataStoreData = exportAllData(dataStore, context); verifyExtractedData(dataStoreData, 3, "Data store should have exactly 3 records", "user100_app1_data", "Data should be from user100 app1 data store"); - + // Export from metadata store level EnvironmentKeySpace.MetadataPath metadataStore = app1User100.metadataStore(); verifyExtractedData(exportAllData(metadataStore, context), @@ -572,11 +580,11 @@ private void setData(List keys, FDBRecordContext context, KeySpacePath b * that {@link KeySpacePath#exportAllData} is built on. * @param pathToExport the path being exported * @param context the context in which to export - * @return a list of the raw {@code KeyValue}s being exported + * @return a list of {@code DataInKeySpacePath}s being exported */ - private static List exportAllData(final KeySpacePath pathToExport, final FDBRecordContext context) { - final List asSingleExport = pathToExport.exportAllData(context, null, ScanProperties.FORWARD_SCAN) - .map(DataInKeySpacePath::getRawKeyValue).asList().join(); + private static List exportAllData(final KeySpacePath pathToExport, final FDBRecordContext context) { + final List asSingleExport = pathToExport.exportAllData(context, null, ScanProperties.FORWARD_SCAN) + .asList().join(); // assert that the resolved paths contain the right prefix final List resolvedPaths = pathToExport.exportAllData(context, null, ScanProperties.FORWARD_SCAN) @@ -587,33 +595,42 @@ private static List exportAllData(final KeySpacePath pathToExport, fin } // assert that the reverse scan is the same as the forward scan, but in reverse - final List reversed = pathToExport.exportAllData(context, null, ScanProperties.REVERSE_SCAN) - .map(DataInKeySpacePath::getRawKeyValue).asList().join(); + final List reversed = pathToExport.exportAllData(context, null, ScanProperties.REVERSE_SCAN) + .asList().join(); Collections.reverse(reversed); - assertEquals(asSingleExport, reversed); + assertDataInKeySpacePathEquals(asSingleExport, reversed); // Assert continuations work correctly final ScanProperties scanProperties = ScanProperties.FORWARD_SCAN.with(props -> props.setReturnedRowLimit(1)); - List asContinuations = new ArrayList<>(); + List asContinuations = new ArrayList<>(); RecordCursorContinuation continuation = RecordCursorStartContinuation.START; while (!continuation.isEnd()) { final RecordCursor cursor = pathToExport.exportAllData(context, continuation.toBytes(), scanProperties); - final AtomicReference> keyValueResult = new AtomicReference<>(); - final List batch = cursor.map(DataInKeySpacePath::getRawKeyValue).asList(keyValueResult).join(); + final AtomicReference> dataInPathResult = new AtomicReference<>(); + final List batch = cursor.asList(dataInPathResult).join(); asContinuations.addAll(batch); - continuation = keyValueResult.get().getContinuation(); - if (keyValueResult.get().hasNext()) { + continuation = dataInPathResult.get().getContinuation(); + if (dataInPathResult.get().hasNext()) { assertEquals(1, batch.size()); } else { assertThat(batch.size()).isLessThanOrEqualTo(1); } } - assertEquals(asSingleExport, asContinuations); + assertDataInKeySpacePathEquals(asSingleExport, asContinuations); return asSingleExport; } + private static void assertDataInKeySpacePathEquals(final List expectedList, + final List actualList) { + assertThat(actualList).zipSatisfy(expectedList, + (actual, other) -> { + assertThat(actual.getResolvedPath().join()).isEqualTo(other.getResolvedPath().join()); + assertThat(actual.getValue()).isEqualTo(other.getValue()); + }); + } + private static void assertStartsWith(final ResolvedKeySpacePath rootResolvedPath, ResolvedKeySpacePath resolvedPath) { ResolvedKeySpacePath searchPath = resolvedPath.withRemainder(null); do { @@ -625,13 +642,13 @@ private static void assertStartsWith(final ResolvedKeySpacePath rootResolvedPath Assertions.fail("Expected <" + resolvedPath + "> to start with <" + rootResolvedPath + "> but it didn't"); } - private static void verifyExtractedData(final List app1User100Data, + private static void verifyExtractedData(final List app1User100Data, int expectedCount, String expectedCountMessage, String expectedValueContents, String contentMessage) { assertEquals(expectedCount, app1User100Data.size(), expectedCountMessage); - for (KeyValue kv : app1User100Data) { - String value = Tuple.fromBytes(kv.getValue()).getString(0); + for (DataInKeySpacePath data : app1User100Data) { + String value = Tuple.fromBytes(data.getValue()).getString(0); assertTrue(value.contains(expectedValueContents), contentMessage); } } From 56c3f0e38e3c08e8b1e7b793eafa748565aaf1d7 Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Mon, 20 Oct 2025 16:38:16 -0400 Subject: [PATCH 22/32] Test continuation behavior while exporting a single key --- .../keyspace/KeySpacePathDataExportTest.java | 52 +++++++++++++++++-- 1 file changed, 48 insertions(+), 4 deletions(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java index 75b2cb3681..87ccc0bc6d 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java @@ -445,9 +445,9 @@ void exportAllDataWithContinuation(boolean forward, int limit) { // Store test data final List> expectedBatches = new ArrayList<>(); expectedBatches.add(new ArrayList<>()); + final KeySpacePath pathToExport = root.path("continuation"); try (FDBRecordContext context = database.openContext()) { Transaction tr = context.ensureActive(); - KeySpacePath basePath = root.path("continuation"); IntStream sourceStream; if (forward) { @@ -456,7 +456,7 @@ void exportAllDataWithContinuation(boolean forward, int limit) { sourceStream = IntStream.iterate(19, i -> i >= 0, i -> i - 1); } sourceStream.forEach(i -> { - Tuple key = basePath.add("item", (long)i).toTuple(context); + Tuple key = pathToExport.add("item", (long)i).toTuple(context); final Tuple value = Tuple.from("continuation_item_" + i); tr.set(key.pack(), value.pack()); if (expectedBatches.get(expectedBatches.size() - 1).size() == limit) { @@ -472,8 +472,52 @@ void exportAllDataWithContinuation(boolean forward, int limit) { } // Export with continuation support + exportWithContinuations(pathToExport, forward, limit, database, expectedBatches); + } + + static Stream exportSingleKeyWithContinuation() { + return ParameterizedTestUtils.cartesianProduct( + ParameterizedTestUtils.booleans("forward"), + ParameterizedTestUtils.booleans("withRemainder"), + Stream.of(1, 3) + ); + } + + @ParameterizedTest + @MethodSource + void exportSingleKeyWithContinuation(boolean forward, boolean withRemainder, int limit) { + KeySpace root = new KeySpace( + new KeySpaceDirectory("continuation", KeyType.STRING, UUID.randomUUID().toString()) + .addSubdirectory(new KeySpaceDirectory("item", KeyType.LONG))); + + final FDBDatabase database = dbExtension.getDatabase(); + + // Store test data + final List> expectedBatches; + final KeySpacePath pathToExport = root.path("continuation"); + try (FDBRecordContext context = database.openContext()) { + Transaction tr = context.ensureActive(); + byte[] key; + if (withRemainder) { + key = pathToExport.toSubspace(context).pack(Tuple.from("continuation")); + } else { + key = pathToExport.toSubspace(context).pack(); + } + final Tuple value = Tuple.from("My Value"); + tr.set(key, value.pack()); + expectedBatches = limit == 1 ? List.of(List.of(value), List.of()) : List.of(List.of(value)); + context.commit(); + } + + // Export with continuation support + exportWithContinuations(pathToExport, forward, limit, database, expectedBatches); + } + + private static void exportWithContinuations(final KeySpacePath pathToExport, + final boolean forward, final int limit, + final FDBDatabase database, + final List> expectedBatches) { try (FDBRecordContext context = database.openContext()) { - KeySpacePath continuationPath = root.path("continuation"); final ScanProperties directionalProperties = forward ? ScanProperties.FORWARD_SCAN : ScanProperties.REVERSE_SCAN; @@ -481,7 +525,7 @@ void exportAllDataWithContinuation(boolean forward, int limit) { List> actual = new ArrayList<>(); RecordCursorContinuation continuation = RecordCursorStartContinuation.START; while (!continuation.isEnd()) { - final RecordCursor cursor = continuationPath.exportAllData(context, continuation.toBytes(), + final RecordCursor cursor = pathToExport.exportAllData(context, continuation.toBytes(), scanProperties); final AtomicReference> tupleResult = new AtomicReference<>(); final List batch = cursor.map(dataInPath -> { From 60ad1d1ed956fdf73a67f61b911bc06ebcf95ba6 Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Tue, 21 Oct 2025 10:47:16 -0400 Subject: [PATCH 23/32] Move code for resolving a key to KeySpacePath and add tests --- .../keyspace/DataInKeySpacePath.java | 30 +--- .../foundationdb/keyspace/KeySpacePath.java | 10 ++ .../keyspace/KeySpacePathImpl.java | 29 ++++ .../keyspace/KeySpacePathWrapper.java | 5 + .../keyspace/KeySpacePathTest.java | 150 ++++++++++++++++++ 5 files changed, 195 insertions(+), 29 deletions(-) create mode 100644 fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathTest.java diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java index a6cb3ed3b0..8c95533cbe 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java @@ -22,8 +22,6 @@ import com.apple.foundationdb.KeyValue; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; -import com.apple.foundationdb.tuple.Tuple; -import com.apple.foundationdb.tuple.TupleHelpers; import javax.annotation.Nonnull; import java.util.concurrent.CompletableFuture; @@ -39,33 +37,7 @@ public class DataInKeySpacePath { private final byte[] value; public DataInKeySpacePath(KeySpacePath path, KeyValue rawKeyValue, FDBRecordContext context) { - // Convert the raw key to a Tuple and resolve it starting from the provided path - Tuple keyTuple = Tuple.fromBytes(rawKeyValue.getKey()); - - // First resolve the provided path to get its resolved form - this.resolvedPath = path.toResolvedPathAsync(context).thenCompose(resolvedPath -> { - // Now use the resolved path to find the child for the key - // We need to figure out how much of the key corresponds to the resolved path - Tuple pathTuple = resolvedPath.toTuple(); - int pathLength = pathTuple.size(); - - // The remaining part of the key should be resolved from the resolved path's directory - if (keyTuple.size() > pathLength) { - // There's more in the key than just the path, so resolve the rest - if (resolvedPath.getDirectory().getSubdirectories().isEmpty()) { - return CompletableFuture.completedFuture( - new ResolvedKeySpacePath(resolvedPath.getParent(), resolvedPath.toPath(), - resolvedPath.getResolvedPathValue(), - TupleHelpers.subTuple(keyTuple, pathTuple.size(), keyTuple.size()))); - } else { - return resolvedPath.getDirectory().findChildForKey(context, resolvedPath, keyTuple, keyTuple.size(), pathLength); - } - } else { - // The key exactly matches the path - return CompletableFuture.completedFuture(resolvedPath); - } - }); - + this.resolvedPath = path.toResolvedPathAsync(context, rawKeyValue.getKey()); this.value = rawKeyValue.getValue(); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java index 501079240d..cb13c9729f 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java @@ -198,6 +198,16 @@ default Tuple toTuple(@Nonnull FDBRecordContext context) { @Nonnull CompletableFuture toResolvedPathAsync(@Nonnull FDBRecordContext context); + /** + * Resolves the given key within this path. + * @param context the transaction to lookup any necessary directory layer entries. + * @param key a raw key from the database + * @return the {@link ResolvedKeySpacePath} corresponding to that key, with a potential remainder. + */ + @API(API.Status.EXPERIMENTAL) + @Nonnull + CompletableFuture toResolvedPathAsync(@Nonnull FDBRecordContext context, byte[] key); + /** * Resolves the path into a {@link ResolvedKeySpacePath}, a form the retains all of the information about * the path itself along with the value to which each path entry is resolved. diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java index d95c07b34b..74161a2cd5 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java @@ -32,6 +32,7 @@ import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.ByteArrayUtil; import com.apple.foundationdb.tuple.Tuple; +import com.apple.foundationdb.tuple.TupleHelpers; import com.google.common.collect.Lists; import javax.annotation.Nonnull; @@ -246,6 +247,34 @@ public CompletableFuture toResolvedPathAsync(@Nonnull FDBR }); } + @Nonnull + @Override + public CompletableFuture toResolvedPathAsync(@Nonnull final FDBRecordContext context, final byte[] key) { + final Tuple keyTuple = Tuple.fromBytes(key); + return toResolvedPathAsync(context).thenCompose(resolvedPath -> { + // Now use the resolved path to find the child for the key + // We need to figure out how much of the key corresponds to the resolved path + Tuple pathTuple = resolvedPath.toTuple(); + int pathLength = pathTuple.size(); + + // The remaining part of the key should be resolved from the resolved path's directory + if (keyTuple.size() > pathLength) { + // There's more in the key than just the path, so resolve the rest + if (resolvedPath.getDirectory().getSubdirectories().isEmpty()) { + return CompletableFuture.completedFuture( + new ResolvedKeySpacePath(resolvedPath.getParent(), resolvedPath.toPath(), + resolvedPath.getResolvedPathValue(), + TupleHelpers.subTuple(keyTuple, pathTuple.size(), keyTuple.size()))); + } else { + return resolvedPath.getDirectory().findChildForKey(context, resolvedPath, keyTuple, keyTuple.size(), pathLength); + } + } else { + // The key exactly matches the path + return CompletableFuture.completedFuture(resolvedPath); + } + }); + } + @Nonnull @Override public CompletableFuture hasDataAsync(@Nonnull FDBRecordContext context) { diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathWrapper.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathWrapper.java index 54d794f2a7..490d98a8f9 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathWrapper.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathWrapper.java @@ -206,6 +206,11 @@ public CompletableFuture toResolvedPathAsync(@Nonnull FDBR return inner.toResolvedPathAsync(context); } + @Nonnull + @Override + public CompletableFuture toResolvedPathAsync(@Nonnull final FDBRecordContext context, final byte[] key) { + return inner.toResolvedPathAsync(context, key); + } @Override public boolean equals(Object obj) { diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathTest.java new file mode 100644 index 0000000000..91637a8a3c --- /dev/null +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathTest.java @@ -0,0 +1,150 @@ +/* + * KeySpacePathTest.java + * + * This source file is part of the FoundationDB open source project + * + * Copyright 2015-2025 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.keyspace; + +import com.apple.foundationdb.record.provider.foundationdb.FDBDatabase; +import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; +import com.apple.foundationdb.record.test.FDBDatabaseExtension; +import com.apple.foundationdb.tuple.Tuple; +import com.apple.test.BooleanSource; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; +import org.junit.jupiter.params.ParameterizedTest; + +import java.util.UUID; + +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; + +/** + * Tests for {@link KeySpacePath}. + * See also {@link KeySpacePathDataExportTest} and {@link ResolvedKeySpacePathTest}. + */ +public class KeySpacePathTest { + + @RegisterExtension + final FDBDatabaseExtension dbExtension = new FDBDatabaseExtension(); + + private static final String ROOT_UUID = UUID.randomUUID().toString(); + + /** + * Creates a KeySpace for testing with a consistent structure. + * The structure is: root -> branch -> leaf + * @param useDirectoryLayer if true, leaf uses DirectoryLayerDirectory; otherwise uses KeySpaceDirectory + * @param useConstantValue if true, adds a constant value to the leaf directory + * @return a KeySpace with the specified configuration + */ + private KeySpace createKeySpace(boolean useDirectoryLayer, boolean useConstantValue) { + KeySpaceDirectory rootDir = new KeySpaceDirectory("test_root", KeySpaceDirectory.KeyType.STRING, ROOT_UUID); + KeySpaceDirectory branchDir = new KeySpaceDirectory("branch", KeySpaceDirectory.KeyType.STRING, "branch_value"); + + KeySpaceDirectory leafDir; + if (useDirectoryLayer) { + if (useConstantValue) { + leafDir = new DirectoryLayerDirectory("leaf", "leaf_constant"); + } else { + leafDir = new DirectoryLayerDirectory("leaf"); + } + } else { + if (useConstantValue) { + leafDir = new KeySpaceDirectory("leaf", KeySpaceDirectory.KeyType.STRING, "leaf_constant"); + } else { + leafDir = new KeySpaceDirectory("leaf", KeySpaceDirectory.KeyType.STRING); + } + } + + branchDir.addSubdirectory(leafDir); + rootDir.addSubdirectory(branchDir); + return new KeySpace(rootDir); + } + + @ParameterizedTest + @BooleanSource({"withRemainder", "useDirectoryLayer", "useConstantValue"}) + void testToResolvedPathAsync(boolean withRemainder, boolean useDirectoryLayer, boolean useConstantValue) { + final FDBDatabase database = dbExtension.getDatabase(); + final KeySpace keySpace = createKeySpace(useDirectoryLayer, useConstantValue); + + try (FDBRecordContext context = database.openContext()) { + KeySpacePath rootPath = keySpace.path("test_root"); + KeySpacePath branchPath = rootPath.add("branch"); + + // Build the full path - add leaf with or without value based on constant + KeySpacePath fullPath; + if (useConstantValue) { + fullPath = branchPath.add("leaf"); + } else { + fullPath = branchPath.add("leaf", "leaf_value"); + } + + // Create a key with or without remainder + byte[] keyBytes; + Tuple expectedRemainder; + if (withRemainder) { + expectedRemainder = Tuple.from("extra", "data"); + keyBytes = fullPath.toSubspace(context).pack(expectedRemainder); + } else { + expectedRemainder = null; + keyBytes = fullPath.toSubspace(context).pack(); + } + + // Test toResolvedPathAsync with the key + ResolvedKeySpacePath resolved = branchPath.toResolvedPathAsync(context, keyBytes).join(); + + assertEquals(fullPath.toResolvedPath(context), resolved.withRemainder(null)); + assertEquals(expectedRemainder, resolved.getRemainder()); + } + } + + @Test + void testToResolvedPathAsyncWithWrapper() { + final FDBDatabase database = dbExtension.getDatabase(); + final EnvironmentKeySpace keySpace = EnvironmentKeySpace.setupSampleData(database); + + try (FDBRecordContext context = database.openContext()) { + // Use the wrapper paths which extend KeySpacePathWrapper + EnvironmentKeySpace.ApplicationPath appPath = keySpace.root().userid(100L).application("app1"); + EnvironmentKeySpace.DataPath dataPath = appPath.dataStore(); + + // Create a key with remainder + Tuple remainderTuple = Tuple.from("record_id", 42L, "version", 1); + byte[] keyBytes = dataPath.toSubspace(context).pack(remainderTuple); + + // Test toResolvedPathAsync on the wrapper - should resolve from appPath through dataPath + ResolvedKeySpacePath resolved = appPath.toResolvedPathAsync(context, keyBytes).join(); + + // Verify the resolved path + assertEquals(EnvironmentKeySpace.DATA_KEY, resolved.getDirectoryName()); + assertEquals(EnvironmentKeySpace.DATA_VALUE, resolved.getResolvedValue()); + assertEquals(remainderTuple, resolved.getRemainder()); + + // Verify parent structure + ResolvedKeySpacePath appLevel = resolved.getParent(); + assertNotNull(appLevel); + assertEquals(EnvironmentKeySpace.APPLICATION_KEY, appLevel.getDirectoryName()); + assertEquals("app1", appLevel.getLogicalValue()); + + ResolvedKeySpacePath userLevel = appLevel.getParent(); + assertNotNull(userLevel); + assertEquals(EnvironmentKeySpace.USER_KEY, userLevel.getDirectoryName()); + assertEquals(100L, userLevel.getResolvedValue()); + } + } +} From fc517e9ac4ea3f72cb38a9a90c1fbee906de22bb Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Tue, 21 Oct 2025 11:00:04 -0400 Subject: [PATCH 24/32] Add path checks & negative tests --- .../keyspace/KeySpacePathImpl.java | 17 +++++++ .../keyspace/KeySpacePathTest.java | 44 +++++++++++++++++++ 2 files changed, 61 insertions(+) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java index 74161a2cd5..760fb13b99 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java @@ -27,6 +27,7 @@ import com.apple.foundationdb.record.ScanProperties; import com.apple.foundationdb.record.ValueRange; import com.apple.foundationdb.record.cursors.LazyCursor; +import com.apple.foundationdb.record.logging.LogMessageKeys; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; import com.apple.foundationdb.record.provider.foundationdb.KeyValueCursor; import com.apple.foundationdb.subspace.Subspace; @@ -257,6 +258,22 @@ public CompletableFuture toResolvedPathAsync(@Nonnull fina Tuple pathTuple = resolvedPath.toTuple(); int pathLength = pathTuple.size(); + // Validate that the key starts with the path + if (keyTuple.size() < pathLength) { + throw new RecordCoreArgumentException("Key is not under this path") + .addLogInfo(LogMessageKeys.EXPECTED, pathTuple, + LogMessageKeys.ACTUAL, keyTuple); + } + + // Verify that the key's prefix matches the path + for (int i = 0; i < pathLength; i++) { + if (!Objects.equals(keyTuple.get(i), pathTuple.get(i))) { + throw new RecordCoreArgumentException("Key is not under this path") + .addLogInfo(LogMessageKeys.EXPECTED, pathTuple, + LogMessageKeys.ACTUAL, keyTuple); + } + } + // The remaining part of the key should be resolved from the resolved path's directory if (keyTuple.size() > pathLength) { // There's more in the key than just the path, so resolve the rest diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathTest.java index 91637a8a3c..e0d349ffa1 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathTest.java @@ -20,6 +20,7 @@ package com.apple.foundationdb.record.provider.foundationdb.keyspace; +import com.apple.foundationdb.record.RecordCoreArgumentException; import com.apple.foundationdb.record.provider.foundationdb.FDBDatabase; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; import com.apple.foundationdb.record.test.FDBDatabaseExtension; @@ -30,9 +31,11 @@ import org.junit.jupiter.params.ParameterizedTest; import java.util.UUID; +import java.util.concurrent.ExecutionException; import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertThrows; /** * Tests for {@link KeySpacePath}. @@ -147,4 +150,45 @@ void testToResolvedPathAsyncWithWrapper() { assertEquals(100L, userLevel.getResolvedValue()); } } + + @Test + void testToResolvedPathAsyncWithKeyNotSubPath() { + final FDBDatabase database = dbExtension.getDatabase(); + final KeySpace keySpace = createKeySpace(false, false); + + try (FDBRecordContext context = database.openContext()) { + KeySpacePath rootPath = keySpace.path("test_root"); + KeySpacePath branchPath = rootPath.add("branch"); + KeySpacePath leafPath = branchPath.add("leaf", "leaf_value"); + + // Create a key that is shorter than branchPath - it stops at root + byte[] shorterKeyBytes = rootPath.toSubspace(context).pack(); + + // Attempting to resolve a key that is not under branchPath should error + ExecutionException ex = assertThrows(ExecutionException.class, () -> { + branchPath.toResolvedPathAsync(context, shorterKeyBytes).get(); + }); + assertEquals(RecordCoreArgumentException.class, ex.getCause().getClass()); + } + } + + @Test + void testToResolvedPathAsyncWithInvalidTuple() { + final FDBDatabase database = dbExtension.getDatabase(); + final KeySpace keySpace = createKeySpace(false, false); + + try (FDBRecordContext context = database.openContext()) { + KeySpacePath rootPath = keySpace.path("test_root"); + KeySpacePath branchPath = rootPath.add("branch"); + + // Create a byte array that is not a valid tuple + byte[] invalidBytes = new byte[]{(byte) 0xFF, (byte) 0xFF, (byte) 0xFF}; + + // Attempting to resolve invalid tuple bytes should error + // The exception is thrown synchronously from Tuple.fromBytes, not wrapped in ExecutionException + assertThrows(IllegalArgumentException.class, () -> { + branchPath.toResolvedPathAsync(context, invalidBytes); + }); + } + } } From 4f5737de2aaa944e3a1fd6a543eff79e3bf2237e Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Tue, 21 Oct 2025 11:02:13 -0400 Subject: [PATCH 25/32] Use TupleHelpers.isPrefix --- .../com/apple/foundationdb/tuple/TupleHelpers.java | 14 +++++++------- .../foundationdb/keyspace/KeySpacePathImpl.java | 12 +----------- 2 files changed, 8 insertions(+), 18 deletions(-) diff --git a/fdb-extensions/src/main/java/com/apple/foundationdb/tuple/TupleHelpers.java b/fdb-extensions/src/main/java/com/apple/foundationdb/tuple/TupleHelpers.java index 547231a462..934b675366 100644 --- a/fdb-extensions/src/main/java/com/apple/foundationdb/tuple/TupleHelpers.java +++ b/fdb-extensions/src/main/java/com/apple/foundationdb/tuple/TupleHelpers.java @@ -141,17 +141,17 @@ public static int packedSizeAsTupleItem(Object item) { /** * Get whether one tuple is a prefix of another. - * @param t1 the potential prefix - * @param t2 the whole tuple - * @return {@code true} if {@code t1} is a prefix of {@code t2} + * @param potentialPrefix the potential prefix + * @param wholeTuple the whole tuple + * @return {@code true} if {@code potentialPrefix} is a prefix of {@code wholeTuple} */ - public static boolean isPrefix(@Nonnull Tuple t1, @Nonnull Tuple t2) { - final int len = t1.size(); - if (t2.size() < len) { + public static boolean isPrefix(@Nonnull Tuple potentialPrefix, @Nonnull Tuple wholeTuple) { + final int len = potentialPrefix.size(); + if (wholeTuple.size() < len) { return false; } for (int i = 0; i < len; i++) { - int rc = TupleUtil.compareItems(t1.get(i), t2.get(i)); + int rc = TupleUtil.compareItems(potentialPrefix.get(i), wholeTuple.get(i)); if (rc != 0) { return false; } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java index 760fb13b99..52628fef1a 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java @@ -258,22 +258,12 @@ public CompletableFuture toResolvedPathAsync(@Nonnull fina Tuple pathTuple = resolvedPath.toTuple(); int pathLength = pathTuple.size(); - // Validate that the key starts with the path - if (keyTuple.size() < pathLength) { + if (!TupleHelpers.isPrefix(pathTuple, keyTuple)) { throw new RecordCoreArgumentException("Key is not under this path") .addLogInfo(LogMessageKeys.EXPECTED, pathTuple, LogMessageKeys.ACTUAL, keyTuple); } - // Verify that the key's prefix matches the path - for (int i = 0; i < pathLength; i++) { - if (!Objects.equals(keyTuple.get(i), pathTuple.get(i))) { - throw new RecordCoreArgumentException("Key is not under this path") - .addLogInfo(LogMessageKeys.EXPECTED, pathTuple, - LogMessageKeys.ACTUAL, keyTuple); - } - } - // The remaining part of the key should be resolved from the resolved path's directory if (keyTuple.size() > pathLength) { // There's more in the key than just the path, so resolve the rest From 714a42a605875e6e94302b2bdb7f5ab7e72ff8c9 Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Tue, 21 Oct 2025 11:04:03 -0400 Subject: [PATCH 26/32] Default implementations for backwards compatibility --- .../provider/foundationdb/keyspace/KeySpacePath.java | 12 ++++++++---- 1 file changed, 8 insertions(+), 4 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java index cb13c9729f..5759ccf4c5 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java @@ -206,7 +206,9 @@ default Tuple toTuple(@Nonnull FDBRecordContext context) { */ @API(API.Status.EXPERIMENTAL) @Nonnull - CompletableFuture toResolvedPathAsync(@Nonnull FDBRecordContext context, byte[] key); + default CompletableFuture toResolvedPathAsync(@Nonnull FDBRecordContext context, byte[] key) { + throw new UnsupportedOperationException("toResolvedPath is not supported"); + } /** * Resolves the path into a {@link ResolvedKeySpacePath}, a form the retains all of the information about @@ -587,9 +589,11 @@ default List listSubdirectory(@Nonnull FDBRecordContext co * @param scanProperties properties controlling how the scan should be performed * @return a RecordCursor that iterates over all KeyValue pairs under this path */ - @API(API.Status.UNSTABLE) + @API(API.Status.EXPERIMENTAL) @Nonnull - RecordCursor exportAllData(@Nonnull FDBRecordContext context, + default RecordCursor exportAllData(@Nonnull FDBRecordContext context, @Nullable byte[] continuation, - @Nonnull ScanProperties scanProperties); + @Nonnull ScanProperties scanProperties) { + throw new UnsupportedOperationException("exportAllData is not supported"); + } } From 118cad731eb7104b6114bb6c04720732ee88d9dc Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Tue, 21 Oct 2025 11:12:43 -0400 Subject: [PATCH 27/32] Resolve from different depths --- .../keyspace/DataInKeySpacePathTest.java | 19 ++++++++++++++++--- 1 file changed, 16 insertions(+), 3 deletions(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java index 36f6a2b4c6..0d01cc67a1 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java @@ -36,9 +36,11 @@ import org.junit.jupiter.params.ParameterizedTest; import org.junit.jupiter.params.provider.ValueSource; +import java.util.List; import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.function.Function; import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; @@ -56,7 +58,7 @@ class DataInKeySpacePathTest { @ParameterizedTest @ValueSource(ints = {0, 1, 2, 3, 4, 5}) - void resolution() { + void resolution(int depth) { // Include some extra children to make sure resolution doesn't get confused final String companyUuid = UUID.randomUUID().toString(); KeySpace root = new KeySpace( @@ -75,6 +77,7 @@ void resolution() { Transaction tr = context.ensureActive(); UUID employeeId = UUID.randomUUID(); + KeySpacePath employeePath = root.path("company") .add("department", "engineering") .add("team_id", 42L) @@ -91,8 +94,18 @@ void resolution() { KeyValue keyValue = new KeyValue(keyBytes, valueBytes); // Create DataInKeySpacePath from the company-level path - KeySpacePath companyPath = root.path("company"); - DataInKeySpacePath dataInPath = new DataInKeySpacePath(companyPath, keyValue, context); + List> pathNavigation = List.of( + path -> path.add("department", "engineering"), + path -> path.add("team_id", 42L), + path -> path.add("employee_uuid", employeeId), + path -> path.add("active", true), + path -> path.add("data") + ); + KeySpacePath toResolve = root.path("company"); + for (final Function function : pathNavigation.subList(0, depth)) { + toResolve = function.apply(toResolve); + } + DataInKeySpacePath dataInPath = new DataInKeySpacePath(toResolve, keyValue, context); ResolvedKeySpacePath resolved = dataInPath.getResolvedPath().join(); From a36a55ee1939e85ed159291d13c1eefda2bf4eb4 Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Tue, 21 Oct 2025 11:14:06 -0400 Subject: [PATCH 28/32] Use assertArrayEquals for byte[] --- .../provider/foundationdb/keyspace/DataInKeySpacePathTest.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java index 0d01cc67a1..6a89e30c34 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java @@ -288,7 +288,7 @@ void keyValueAccessors(boolean withRemainder) throws ExecutionException, Interru final ResolvedKeySpacePath resolvedPath = resolvedFuture.get(); assertEquals(pathTuple, resolvedPath.toTuple()); - assertEquals(originalKeyValue.getValue(), dataInPath.getValue()); + assertArrayEquals(originalKeyValue.getValue(), dataInPath.getValue()); if (withRemainder) { assertEquals(Tuple.from("Remainder"), resolvedPath.getRemainder()); } else { From 1ac438c2a1d15117c7b2f2b0cbdbdbc3557aa319 Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Tue, 21 Oct 2025 11:54:26 -0400 Subject: [PATCH 29/32] Test new default methods This isn't the greatest test, but I think the proper test would be to explicitly list the non-default methods and then use reflection and error if any new methods are added and don't have a default implementation. --- .../foundationdb/keyspace/KeySpacePath.java | 4 ++-- .../keyspace/KeySpacePathTest.java | 22 +++++++++++++++++++ 2 files changed, 24 insertions(+), 2 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java index 5759ccf4c5..7967f3935a 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java @@ -592,8 +592,8 @@ default List listSubdirectory(@Nonnull FDBRecordContext co @API(API.Status.EXPERIMENTAL) @Nonnull default RecordCursor exportAllData(@Nonnull FDBRecordContext context, - @Nullable byte[] continuation, - @Nonnull ScanProperties scanProperties) { + @Nullable byte[] continuation, + @Nonnull ScanProperties scanProperties) { throw new UnsupportedOperationException("exportAllData is not supported"); } } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathTest.java index e0d349ffa1..4d109ce58f 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathTest.java @@ -21,6 +21,7 @@ package com.apple.foundationdb.record.provider.foundationdb.keyspace; import com.apple.foundationdb.record.RecordCoreArgumentException; +import com.apple.foundationdb.record.ScanProperties; import com.apple.foundationdb.record.provider.foundationdb.FDBDatabase; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; import com.apple.foundationdb.record.test.FDBDatabaseExtension; @@ -29,6 +30,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.api.extension.RegisterExtension; import org.junit.jupiter.params.ParameterizedTest; +import org.mockito.Mockito; import java.util.UUID; import java.util.concurrent.ExecutionException; @@ -191,4 +193,24 @@ void testToResolvedPathAsyncWithInvalidTuple() { }); } } + + /** + * Test of methods with default implementations to ensure backwards compatibility, + * in case someone is implementing {@link KeySpacePath}. + **/ + @Test + void testDefaultMethods() { + final KeySpacePath mock = Mockito.mock(KeySpacePath.class); + final FDBDatabase database = dbExtension.getDatabase(); + + try (FDBRecordContext context = database.openContext()) { + // thenCallReadMethod throws an error if there is not a default implementation + Mockito.when(mock.toResolvedPathAsync(Mockito.any(), Mockito.any())).thenCallRealMethod(); + assertThrows(UnsupportedOperationException.class, + () -> mock.toResolvedPathAsync(context, Tuple.from("foo").pack())); + Mockito.when(mock.exportAllData(Mockito.any(), Mockito.any(), Mockito.any())).thenCallRealMethod(); + assertThrows(UnsupportedOperationException.class, + () -> mock.exportAllData(context, null, ScanProperties.FORWARD_SCAN)); + } + } } From 5595a9318f9e6e1cd267ef7c1d212e4a39f6e4d6 Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Tue, 21 Oct 2025 11:57:25 -0400 Subject: [PATCH 30/32] Some cleanup --- .../foundationdb/keyspace/KeySpacePathTest.java | 12 ++++-------- 1 file changed, 4 insertions(+), 8 deletions(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathTest.java index 4d109ce58f..ce6952d697 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathTest.java @@ -43,7 +43,7 @@ * Tests for {@link KeySpacePath}. * See also {@link KeySpacePathDataExportTest} and {@link ResolvedKeySpacePathTest}. */ -public class KeySpacePathTest { +class KeySpacePathTest { @RegisterExtension final FDBDatabaseExtension dbExtension = new FDBDatabaseExtension(); @@ -161,15 +161,13 @@ void testToResolvedPathAsyncWithKeyNotSubPath() { try (FDBRecordContext context = database.openContext()) { KeySpacePath rootPath = keySpace.path("test_root"); KeySpacePath branchPath = rootPath.add("branch"); - KeySpacePath leafPath = branchPath.add("leaf", "leaf_value"); // Create a key that is shorter than branchPath - it stops at root byte[] shorterKeyBytes = rootPath.toSubspace(context).pack(); // Attempting to resolve a key that is not under branchPath should error - ExecutionException ex = assertThrows(ExecutionException.class, () -> { - branchPath.toResolvedPathAsync(context, shorterKeyBytes).get(); - }); + ExecutionException ex = assertThrows(ExecutionException.class, + () -> branchPath.toResolvedPathAsync(context, shorterKeyBytes).get()); assertEquals(RecordCoreArgumentException.class, ex.getCause().getClass()); } } @@ -188,9 +186,7 @@ void testToResolvedPathAsyncWithInvalidTuple() { // Attempting to resolve invalid tuple bytes should error // The exception is thrown synchronously from Tuple.fromBytes, not wrapped in ExecutionException - assertThrows(IllegalArgumentException.class, () -> { - branchPath.toResolvedPathAsync(context, invalidBytes); - }); + assertThrows(IllegalArgumentException.class, () -> branchPath.toResolvedPathAsync(context, invalidBytes)); } } From 1a58550612dc7bf281517617673852aec06c8ef1 Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Thu, 23 Oct 2025 12:10:13 -0400 Subject: [PATCH 31/32] Respond to PR #3566 comments from @ohadzeliger Predominately: - DataInKeySpace now errors if it gets a null value - Improved javadoc linking KeySpace.resolveFromKeyAsync and KeySpacePath.toResolvedPathAsync - Fix error message in default method - Reduce visibility of ResolvedKeySPacePath.withRemainder - new negative tests in DataInKeySpacePathTest - Better coverage of exporting with mixed-type sub-directories --- .../keyspace/DataInKeySpacePath.java | 7 +++ .../foundationdb/keyspace/KeySpace.java | 9 ++-- .../foundationdb/keyspace/KeySpacePath.java | 13 +++-- .../keyspace/ResolvedKeySpacePath.java | 4 +- .../keyspace/DataInKeySpacePathTest.java | 48 ++++++++++++++++++ .../keyspace/KeySpacePathDataExportTest.java | 50 +++++++++---------- 6 files changed, 97 insertions(+), 34 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java index 8c95533cbe..2e6dde4684 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java @@ -21,7 +21,10 @@ package com.apple.foundationdb.record.provider.foundationdb.keyspace; import com.apple.foundationdb.KeyValue; +import com.apple.foundationdb.record.RecordCoreArgumentException; +import com.apple.foundationdb.record.logging.LogMessageKeys; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; +import com.apple.foundationdb.tuple.ByteArrayUtil2; import javax.annotation.Nonnull; import java.util.concurrent.CompletableFuture; @@ -39,6 +42,10 @@ public class DataInKeySpacePath { public DataInKeySpacePath(KeySpacePath path, KeyValue rawKeyValue, FDBRecordContext context) { this.resolvedPath = path.toResolvedPathAsync(context, rawKeyValue.getKey()); this.value = rawKeyValue.getValue(); + if (this.value == null) { + throw new RecordCoreArgumentException("Value cannot be null") + .addLogInfo(LogMessageKeys.KEY, ByteArrayUtil2.loggable(rawKeyValue.getKey())); + } } public CompletableFuture getResolvedPath() { diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpace.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpace.java index eaa7bbd1bd..d7d67936f9 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpace.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpace.java @@ -227,9 +227,12 @@ public KeySpacePath pathFromKey(@Nonnull FDBRecordContext context, @Nonnull Tupl /** * Given a tuple from an FDB key, attempts to determine what path through this directory the tuple * represents, returning a ResolvedKeySpacePath representing the leaf-most directory in the path. - * If entries remained in the tuple beyond the leaf directory, then {@link KeySpacePath#getRemainder()} - * can be used to fetch the remaining portion. - * + *

+ * If entries remained in the tuple beyond the leaf directory, then {@link KeySpacePath#getRemainder()} can be + * used to fetch the remaining portion. + * See also {@link KeySpacePath#toResolvedPathAsync(FDBRecordContext, byte[])} if you need to resolve and you + * know that it is part of a given path. + *

* @param context context used, if needed, for any database operations * @param key the tuple to be decoded * @return a path entry representing the leaf directory entry that corresponds to a value in the diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java index 7967f3935a..4567381c34 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java @@ -199,15 +199,22 @@ default Tuple toTuple(@Nonnull FDBRecordContext context) { CompletableFuture toResolvedPathAsync(@Nonnull FDBRecordContext context); /** - * Resolves the given key within this path. - * @param context the transaction to lookup any necessary directory layer entries. + * Given a tuple from an FDB key, attempts to determine what sub-path through this directory the tuple + * represents, returning a ResolvedKeySpacePath representing the leaf-most directory in the path. + *

+ * If entries remained in the tuple beyond the leaf directory, then {@link KeySpacePath#getRemainder()} + * can be used to fetch the remaining portion. + * See also {@link KeySpace#resolveFromKeyAsync(FDBRecordContext, Tuple)} if you need to resolve from the root. + *

+ * @param context context used, if needed, for any database operations * @param key a raw key from the database * @return the {@link ResolvedKeySpacePath} corresponding to that key, with a potential remainder. + * @throws com.apple.foundationdb.record.RecordCoreArgumentException if the key provided is not part of this path */ @API(API.Status.EXPERIMENTAL) @Nonnull default CompletableFuture toResolvedPathAsync(@Nonnull FDBRecordContext context, byte[] key) { - throw new UnsupportedOperationException("toResolvedPath is not supported"); + throw new UnsupportedOperationException("toResolvedPathAsync is not supported"); } /** diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/ResolvedKeySpacePath.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/ResolvedKeySpacePath.java index 728f5162a7..ddb72d041b 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/ResolvedKeySpacePath.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/ResolvedKeySpacePath.java @@ -24,6 +24,7 @@ import com.apple.foundationdb.subspace.Subspace; import com.apple.foundationdb.tuple.ByteArrayUtil2; import com.apple.foundationdb.tuple.Tuple; +import com.google.common.annotations.VisibleForTesting; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -278,7 +279,8 @@ public static void appendValue(StringBuilder sb, Object value) { * @return a new {@code ResolvedKeySpacePath} that is the same as this, except with a different {@link #getRemainder()}. */ @Nonnull - public ResolvedKeySpacePath withRemainder(@Nullable final Tuple newRemainder) { + @VisibleForTesting + ResolvedKeySpacePath withRemainder(@Nullable final Tuple newRemainder) { // this could probably copy the cachedTuple & cachedSubspace return new ResolvedKeySpacePath(parent, inner, value, newRemainder); } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java index 6a89e30c34..64eccab827 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePathTest.java @@ -22,6 +22,7 @@ import com.apple.foundationdb.KeyValue; import com.apple.foundationdb.Transaction; +import com.apple.foundationdb.record.RecordCoreArgumentException; import com.apple.foundationdb.record.provider.foundationdb.FDBDatabase; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; import com.apple.foundationdb.record.provider.foundationdb.keyspace.KeySpaceDirectory.KeyType; @@ -39,13 +40,16 @@ import java.util.List; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.CompletionException; import java.util.concurrent.ExecutionException; import java.util.function.Function; import static org.junit.jupiter.api.Assertions.assertArrayEquals; import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertInstanceOf; 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; /** @@ -328,6 +332,50 @@ void withWrapper() { } } + @Test + void badPath() { + final String rootUuid = UUID.randomUUID().toString(); + KeySpace root = new KeySpace( + new KeySpaceDirectory("test", KeyType.STRING, rootUuid)); + + final FDBDatabase database = dbExtension.getDatabase(); + + try (FDBRecordContext context = database.openContext()) { + KeySpacePath testPath = root.path("test"); + byte[] keyBytes = Tuple.from("banana", rootUuid).pack(); + byte[] valueBytes = Tuple.from("accessor_test").pack(); + + KeyValue originalKeyValue = new KeyValue(keyBytes, valueBytes); + DataInKeySpacePath dataInPath = new DataInKeySpacePath(testPath, originalKeyValue, context); + final CompletionException completionException = assertThrows(CompletionException.class, + () -> dataInPath.getResolvedPath().join()); + assertInstanceOf(RecordCoreArgumentException.class, completionException.getCause()); + } + } + + /** + * Test if there is a null value. FDB shouldn't ever return this, and if you got it somehow, you wouldn't be + * able to insert it back into a database because {@link com.apple.foundationdb.FDBTransaction#set(byte[], byte[])} + * does not support a {@code null} key or value. + */ + @Test + void nullValue() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("test", KeyType.STRING, UUID.randomUUID().toString())); + + final FDBDatabase database = dbExtension.getDatabase(); + + try (FDBRecordContext context = database.openContext()) { + KeySpacePath testPath = root.path("test"); + byte[] keyBytes = testPath.toTuple(context).pack(); + byte[] valueBytes = null; + + KeyValue originalKeyValue = new KeyValue(keyBytes, valueBytes); + assertThrows(RecordCoreArgumentException.class, + () -> new DataInKeySpacePath(testPath, originalKeyValue, context)); + } + } + private static ResolvedKeySpacePath assertNameAndDirectoryScopedValue(ResolvedKeySpacePath resolved, String name, Object logicalValue, KeySpacePath path, FDBRecordContext context) { diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java index 87ccc0bc6d..52cf9fe423 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathDataExportTest.java @@ -46,6 +46,7 @@ import java.util.Collections; import java.util.HashSet; import java.util.List; +import java.util.Map; import java.util.Set; import java.util.UUID; import java.util.concurrent.ExecutionException; @@ -212,35 +213,30 @@ void exportAllDataWithDirectoryLayer() { @Test void exportAllDataWithDifferentKeyTypes() { - KeySpace root = new KeySpace( - new KeySpaceDirectory("mixed", KeyType.STRING, UUID.randomUUID().toString()) - .addSubdirectory(new KeySpaceDirectory("strings", KeyType.STRING)) - .addSubdirectory(new KeySpaceDirectory("longs", KeyType.LONG)) - .addSubdirectory(new KeySpaceDirectory("bytes", KeyType.BYTES)) - .addSubdirectory(new KeySpaceDirectory("uuids", KeyType.UUID)) - .addSubdirectory(new KeySpaceDirectory("booleans", KeyType.BOOLEAN))); + final KeySpaceDirectory rootDirectory = new KeySpaceDirectory("mixed", KeyType.STRING, UUID.randomUUID().toString()); + Map> dataByType = Map.of( + KeyType.STRING, List.of("str0", "str1", "str2"), + KeyType.LONG, List.of(10L, 11L, 12L), + KeyType.BYTES, List.of(new byte[]{0, 1}, new byte[]{1, 2}), + KeyType.UUID, List.of(new UUID(0, 0), new UUID(1, 1)), + KeyType.BOOLEAN, List.of(true, false), + KeyType.NULL, Collections.singletonList(null), + KeyType.DOUBLE, List.of(3.1415, -2.718281, 13.23E8), + KeyType.FLOAT, List.of(1.4142135f, -5.8f, 130.23f) + ); + dataByType.keySet().forEach(keyType -> + rootDirectory.addSubdirectory(new KeySpaceDirectory(keyType.name(), keyType))); + KeySpace root = new KeySpace(rootDirectory); + assertEquals(Set.of(KeyType.values()), dataByType.keySet()); final FDBDatabase database = dbExtension.getDatabase(); // Store test data with different key types try (FDBRecordContext context = database.openContext()) { KeySpacePath basePath = root.path("mixed"); - - // String keys (str0, str1, str2 -> string_value_0, string_value_1, string_value_2) - setData(List.of("str0", "str1", "str2"), context, basePath, "strings", "string_value_"); - - // Long keys (10, 11, 12 -> long_value_10, long_value_11, long_value_12) - setData(List.of(10L, 11L, 12L), context, basePath, "longs", "long_value_"); - - // Bytes keys (arrays -> bytes_value_[0, 1], bytes_value_[1, 2]) - setData(List.of(new byte[]{0, 1}, new byte[]{1, 2}), context, basePath, "bytes", "bytes_value_"); - - // UUID keys (UUIDs -> uuid_value_UUID) - setData(List.of(new UUID(0, 0), new UUID(1, 1)), context, basePath, "uuids", "uuid_value_"); - - // Boolean keys (true, false -> boolean_value_true, boolean_value_false) - setData(List.of(true, false), context, basePath, "booleans", "boolean_value_"); - + dataByType.forEach((keyType, data) -> + setData(data, context, basePath, keyType.name(), keyType + "_value_") + ); context.commit(); } @@ -249,15 +245,15 @@ void exportAllDataWithDifferentKeyTypes() { KeySpacePath mixedPath = root.path("mixed"); final List allData = exportAllData(mixedPath, context); - // Should have 12 records total (3+3+2+2+2) - assertEquals(12, allData.size()); + assertEquals(dataByType.values().stream().mapToLong(List::size).sum(), + allData.size()); // Verify we have different value types Set valueTypes = allData.stream() .map(data -> Tuple.fromBytes(data.getValue()).getString(0).split("_")[0]) .collect(Collectors.toSet()); - assertEquals(5, valueTypes.size()); - assertTrue(valueTypes.containsAll(Arrays.asList("string", "long", "bytes", "uuid", "boolean"))); + assertEquals((Arrays.stream(KeyType.values()).map(Enum::name).collect(Collectors.toSet())), + valueTypes); } } From f7f2da2aef44c8b0dbe7b836abf238d9ec1b36b5 Mon Sep 17 00:00:00 2001 From: Scott Dugas Date: Fri, 24 Oct 2025 15:09:33 -0400 Subject: [PATCH 32/32] Mark DataInKeySpacePath as EXPERIMENTAL --- .../provider/foundationdb/keyspace/DataInKeySpacePath.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java index 2e6dde4684..a51d52fcd6 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DataInKeySpacePath.java @@ -21,6 +21,7 @@ package com.apple.foundationdb.record.provider.foundationdb.keyspace; import com.apple.foundationdb.KeyValue; +import com.apple.foundationdb.annotation.API; import com.apple.foundationdb.record.RecordCoreArgumentException; import com.apple.foundationdb.record.logging.LogMessageKeys; import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; @@ -32,6 +33,7 @@ /** * Class representing a {@link KeyValue} pair within in {@link KeySpacePath}. */ +@API(API.Status.EXPERIMENTAL) public class DataInKeySpacePath { @Nonnull