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 afbd34599a..25f52f2d6b 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 @@ -56,6 +56,7 @@ public DataInKeySpacePath(@Nonnull final KeySpacePath path, @Nullable final Tupl this.value = value; } + @Nonnull public byte[] getValue() { return this.value; } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DirectoryLayerDirectory.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DirectoryLayerDirectory.java index 73d2c32a98..944fd1b549 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DirectoryLayerDirectory.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/DirectoryLayerDirectory.java @@ -30,6 +30,7 @@ import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.util.Objects; import java.util.Optional; import java.util.concurrent.CompletableFuture; import java.util.function.Function; @@ -148,6 +149,18 @@ public DirectoryLayerDirectory(@Nonnull String name, @Nullable Object value, this.createHooks = createHooks; } + @Override + public boolean isValueValid(@Nullable Object value) { + // DirectoryLayerDirectory accepts both String (logical names) and Long (directory layer values), + // but we're making this method stricter, and I hope that using Long is only for a handful of tests, + // despite comments saying that the resolved value should be allowed. + if (value instanceof String) { + // If this directory has a constant value, check that the provided value matches it + return Objects.equals(getValue(), KeySpaceDirectory.ANY_VALUE) || Objects.equals(getValue(), value); + } + return false; + } + @Override protected void validateConstant(@Nullable Object value) { if (!(value instanceof String)) { diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpaceDirectory.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpaceDirectory.java index f36b073276..5b45f69aff 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpaceDirectory.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpaceDirectory.java @@ -176,6 +176,55 @@ LogMessageKeys.DIR_NAME, getName(), } } + /** + * Validate that the given value can be used with this directory. + *

+ * Ideally this would be called as part of {@link KeySpacePath#add(String, Object)} to ensure the provided value + * is valid, but that code has existed for a long time, so it's possible clients are adding without respecting + * the type. You should call this before calling add to make sure you don't have the same mistakes. At some point + * this will be embedded in {@code add} once there's some confidence that it won't break anyone's environments. + *

+ * @param value a potential value + * @throws RecordCoreArgumentException if the value is not valid + */ + @API(API.Status.EXPERIMENTAL) + public void validateValue(@Nullable Object value) { + // Validate that the value is valid for this directory + if (!isValueValid(value)) { + throw new RecordCoreArgumentException("Value does not match directory requirements") + .addLogInfo(LogMessageKeys.DIR_NAME, name, + LogMessageKeys.EXPECTED_TYPE, getKeyType(), + LogMessageKeys.ACTUAL, value, + "actual_type", value == null ? "null" : value.getClass().getName(), + "expected_value", getValue()); + } + } + + /** + * Checks if the provided value is valid for this directory. This method can be overridden by subclasses + * to provide custom validation logic. For example, {@link DirectoryLayerDirectory} accepts String + * values (logical names) even though its key type is LONG. + * + * @param value the value to validate + * @return {@code true} if the value is valid for this directory + */ + @API(API.Status.EXPERIMENTAL) + public boolean isValueValid(@Nullable Object value) { + // Check if value matches the key type + if (!keyType.isMatch(value)) { + return false; + } + // If this directory has a constant value, check that the provided value matches it + if (this.value != ANY_VALUE) { + if (this.value instanceof byte[] && value instanceof byte[]) { + return Arrays.equals((byte[]) this.value, (byte[]) value); + } else { + return Objects.equals(this.value, value); + } + } + return true; + } + /** * Given a position in a tuple, checks to see if this directory is compatible with the value at the * position, returning either a path indicating that it was compatible or nothing if it was not compatible. 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 d513c87177..64a1770fbf 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 @@ -432,4 +432,13 @@ default RecordCursor exportAllData(@Nonnull FDBRecordContext @Nonnull CompletableFuture importData(@Nonnull FDBRecordContext context, @Nonnull Iterable dataToImport); + + /** + * Two {@link KeySpacePath}s are equal if they have equal values, the same directory (reference equality) and their + * parents are the same. + * @param obj another {@link KeySpacePath} + * @return {@code true} if this path equals {@code obj} + */ + @Override + boolean equals(Object obj); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathSerializer.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathSerializer.java new file mode 100644 index 0000000000..3c1d19f47a --- /dev/null +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathSerializer.java @@ -0,0 +1,219 @@ +/* + * KeySpacePathSerializer.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.annotation.API; +import com.apple.foundationdb.record.RecordCoreArgumentException; +import com.apple.foundationdb.record.logging.LogMessageKeys; +import com.apple.foundationdb.tuple.Tuple; +import com.google.protobuf.ByteString; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.List; +import java.util.UUID; + +/** + * Class for serializing/deserializing between {@link DataInKeySpacePath} and {@link KeySpaceProto.DataInKeySpacePath}. + *

+ * This will serialize relative to a root path, such that the serialized form is relative to that path. + *

    + *
  • This reduces the size of the serialized data, which is particularly important when you have a lot of these.
  • + *
  • This allows the serialized form to be used as an intermediate if you have two identical sub-hierarchies + * in your {@link KeySpace}.
  • + *
+ *

+ * + */ +@API(API.Status.EXPERIMENTAL) +public class KeySpacePathSerializer { + + @Nonnull + private final List root; + + /** + * Constructor a new serializer for serializing relative to a given root path. + * @param root a path which is a parent path of all the data you want to serialize + */ + public KeySpacePathSerializer(@Nonnull final KeySpacePath root) { + this.root = root.flatten(); + } + + /** + * Serialize the given data relative to the root. + * @param data a piece of data that is contained within the root + * @return the data serialized relative to the root + * @throws RecordCoreArgumentException if the given data is not contained within the root + */ + @Nonnull + public ByteString serialize(@Nonnull DataInKeySpacePath data) { + final List dataPath = data.getPath().flatten(); + // two paths are only equal if their parents are equal, so we don't have to validate the whole prefix here + if (dataPath.size() < root.size() || + !dataPath.get(root.size() - 1).equals(root.get(root.size() - 1))) { + throw new RecordCoreArgumentException("Data is not contained within root path"); + } + KeySpaceProto.DataInKeySpacePath.Builder builder = KeySpaceProto.DataInKeySpacePath.newBuilder(); + for (int i = root.size(); i < dataPath.size(); i++) { + final KeySpacePath keySpacePath = dataPath.get(i); + builder.addPath(serialize(keySpacePath)); + } + if (data.getRemainder() != null) { + builder.setRemainder(ByteString.copyFrom(data.getRemainder().pack())); + } + builder.setValue(ByteString.copyFrom(data.getValue())); + return builder.build().toByteString(); + } + + /** + * Deserialize data relative to the root. + *

+ * Note: The given data does not need to have come from the same path, but all sub-paths must be valid. + *

+ * @param bytes a serialized form of {@link DataInKeySpacePath} as provided by {@link #serialize(DataInKeySpacePath)} + * @return the deserialized data + * @throws RecordCoreArgumentException if the bytes cannot be parsed, or one of the path entries is not valid + * @throws NoSuchDirectoryException if it refers to a directory that doesn't exist within the root + */ + @Nonnull + public DataInKeySpacePath deserialize(@Nonnull ByteString bytes) { + try { + KeySpaceProto.DataInKeySpacePath proto = KeySpaceProto.DataInKeySpacePath.parseFrom(bytes); + return deserialize(proto); + } catch (com.google.protobuf.InvalidProtocolBufferException e) { + throw new RecordCoreArgumentException("Failed to parse serialized DataInKeySpacePath", e); + } + } + + @Nonnull + private DataInKeySpacePath deserialize(@Nonnull KeySpaceProto.DataInKeySpacePath proto) { + // Start with the root path + KeySpacePath path = root.get(root.size() - 1); + + // Add each path entry from the proto + for (KeySpaceProto.KeySpacePathEntry entry : proto.getPathList()) { + Object value = deserializeValue(entry); + path.getDirectory().getSubdirectory(entry.getName()).validateValue(value); + path = path.add(entry.getName(), value); + } + + // Extract remainder if present + Tuple remainder = null; + if (proto.hasRemainder()) { + remainder = Tuple.fromBytes(proto.getRemainder().toByteArray()); + } + + // Extract value + if (!proto.hasValue()) { + throw new RecordCoreArgumentException("Serialized data must have a value"); + } + byte[] value = proto.getValue().toByteArray(); + + return new DataInKeySpacePath(path, remainder, value); + } + + @Nullable + private static Object deserializeValue(@Nonnull KeySpaceProto.KeySpacePathEntry entry) { + // Check which value field is set and return the appropriate value + if (entry.hasNullValue()) { + return null; + } else if (entry.hasBytesValue()) { + return entry.getBytesValue().toByteArray(); + } else if (entry.hasStringValue()) { + return entry.getStringValue(); + } else if (entry.hasLongValue()) { + return entry.getLongValue(); + } else if (entry.hasFloatValue()) { + return entry.getFloatValue(); + } else if (entry.hasDoubleValue()) { + return entry.getDoubleValue(); + } else if (entry.hasBooleanValue()) { + return entry.getBooleanValue(); + } else if (entry.hasUuid()) { + KeySpaceProto.KeySpacePathEntry.UUID uuidProto = entry.getUuid(); + return new UUID(uuidProto.getMostSignificantBits(), uuidProto.getLeastSignificantBits()); + } else { + throw new RecordCoreArgumentException("KeySpacePathEntry has no value set") + .addLogInfo(LogMessageKeys.DIR_NAME, entry.getName()); + } + } + + @Nonnull + private static KeySpaceProto.KeySpacePathEntry serialize(@Nonnull final KeySpacePath keySpacePath) { + final Object value = keySpacePath.getValue(); + // Use typeOf to get the actual runtime type of the value, rather than the directory's declared keyType. + // This is important for DirectoryLayerDirectory, which has keyType LONG but typically stores String values. + // If we every support something that takes a value that is not supported via KeyType, we'll need to remove this + // dependency, but right now it is convenient to reuse that enum. + final KeySpaceDirectory.KeyType keyType = value == null + ? KeySpaceDirectory.KeyType.NULL + : KeySpaceDirectory.KeyType.typeOf(value); + + KeySpaceProto.KeySpacePathEntry.Builder builder = KeySpaceProto.KeySpacePathEntry.newBuilder() + .setName(keySpacePath.getDirectoryName()); + try { + switch (keyType) { + case NULL: + builder.setNullValue(true); + break; + case BYTES: + builder.setBytesValue(ByteString.copyFrom((byte[])value)); + break; + case STRING: + builder.setStringValue((String)value); + break; + case LONG: + if (value instanceof Integer) { + builder.setLongValue(((Integer)value).longValue()); + } else { + builder.setLongValue((Long)value); + } + break; + case FLOAT: + builder.setFloatValue((Float)value); + break; + case DOUBLE: + builder.setDoubleValue((Double)value); + break; + case BOOLEAN: + builder.setBooleanValue((Boolean)value); + break; + case UUID: + final UUID uuid = (UUID)value; + builder.getUuidBuilder() + .setLeastSignificantBits(uuid.getLeastSignificantBits()) + .setMostSignificantBits(uuid.getMostSignificantBits()); + break; + default: + throw new IllegalStateException("Unexpected value: " + keyType); + } + } catch (ClassCastException e) { + throw new RecordCoreArgumentException("KeySpacePath has incorrect value type", e) + .addLogInfo( + LogMessageKeys.DIR_NAME, keySpacePath.getDirectoryName(), + LogMessageKeys.EXPECTED_TYPE, keyType, + LogMessageKeys.ACTUAL, value); + + } + return builder.build(); + } + +} diff --git a/fdb-record-layer-core/src/main/proto/keyspace.proto b/fdb-record-layer-core/src/main/proto/keyspace.proto new file mode 100644 index 0000000000..e4c933a340 --- /dev/null +++ b/fdb-record-layer-core/src/main/proto/keyspace.proto @@ -0,0 +1,54 @@ +/* + * keyspace.proto + * + * 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. + */ + +syntax = "proto2"; + +package com.apple.foundationdb.record.provider.foundationdb.keyspace; +option java_outer_classname = "KeySpaceProto"; + +message DataInKeySpacePath { + repeated KeySpacePathEntry path = 1; + optional bytes remainder = 2; + optional bytes value = 3; +} + +// Entry representing logical values for a KeySpacePath entry. +message KeySpacePathEntry { + optional string name = 1; + + // specific boolean to indicate this is supposed to be a null + optional bool nullValue = 2; + optional bytes bytesValue = 3; + optional string stringValue = 4; + optional int64 longValue = 5; + optional float floatValue = 6; + optional double doubleValue = 7; + optional bool booleanValue = 8; + optional UUID uuid = 9; + + message UUID { + // 2 64-bit fields is two tags, the same as 1 bytes field with a length of 16 would be. + // Using int64 would use more space for a lot of numbers since the uuid is random, it could use up to 10 bytes + // instead of the fixed 8 bytes (plus the tag). + // fixed64 would be closer to how these are really used, but would fail the unsigned validator. + optional sfixed64 most_significant_bits = 1; + optional sfixed64 least_significant_bits = 2; + } +} 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 6c9d2d07d6..b6b5b518d2 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 @@ -40,13 +40,17 @@ import com.apple.foundationdb.tuple.Tuple; import com.apple.foundationdb.tuple.TupleHelpers; import com.apple.test.BooleanSource; +import com.apple.test.ParameterizedTestUtils; import com.apple.test.Tags; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; +import org.junit.jupiter.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.Arguments; +import org.junit.jupiter.params.provider.MethodSource; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -67,6 +71,7 @@ import java.util.function.Supplier; import java.util.stream.Collectors; import java.util.stream.IntStream; +import java.util.stream.Stream; import static com.apple.foundationdb.record.TestHelpers.assertThrows; import static com.apple.foundationdb.record.TestHelpers.eventually; @@ -81,6 +86,7 @@ import static org.junit.jupiter.api.Assertions.assertEquals; import static org.junit.jupiter.api.Assertions.assertFalse; import static org.junit.jupiter.api.Assertions.assertNotEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; import static org.junit.jupiter.api.Assertions.assertNotSame; import static org.junit.jupiter.api.Assertions.assertNull; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -430,7 +436,7 @@ public void testAllTypesAnyValues() throws Exception { } } - public KeyTypeValue pickDifferentType(KeyType keyType) { + private KeyTypeValue pickDifferentType(KeyType keyType) { while (true) { KeyTypeValue kv = valueOfEveryType.get(random.nextInt(valueOfEveryType.size())); if (kv.keyType != keyType) { @@ -1558,6 +1564,162 @@ private List resolveBatch(FDBRecordContext context, String... names) { return AsyncUtil.getAll(futures).join(); } + private static final Map VALUES = Map.of( + KeySpaceDirectory.KeyType.NULL, new KeyPathValues(() -> null, + List.of(), List.of("not_null", 42, true)), + KeySpaceDirectory.KeyType.STRING, new KeyPathValues(() -> "foo", + List.of("bar", ""), List.of(3, "foo".getBytes(), true, 3.14)), + KeySpaceDirectory.KeyType.LONG, new KeyPathValues(() -> 42L, + List.of(100L, 0L, -5L, 123, ((long)Integer.MAX_VALUE) + 3), List.of("not_long", 3.14, true, new byte[] {1})), + KeySpaceDirectory.KeyType.FLOAT, new KeyPathValues(() -> 3.14f, + List.of(0.0f, -2.5f, 1.5f), List.of("not_float", 42, true, new byte[] {1}, Double.MAX_VALUE)), + KeySpaceDirectory.KeyType.DOUBLE, new KeyPathValues(() -> 2.71828, + List.of(0.0, -1.5, 3.14159), List.of("not_double", 42, true, new byte[] {1}, 1.5f)), + KeySpaceDirectory.KeyType.BOOLEAN, new KeyPathValues(() -> true, + List.of(false), + List.of("true", 1, 0, new byte[] {1})), + KeySpaceDirectory.KeyType.BYTES, new KeyPathValues(() -> new byte[] {1, 2, 3}, + List.of(new byte[] {4, 5}, new byte[] {(byte)0xFF}, new byte[0]), + List.of("not_bytes", 42, true, 3.14)), + KeySpaceDirectory.KeyType.UUID, new KeyPathValues(() -> UUID.fromString("12345678-1234-1234-1234-123456789abc"), + List.of(UUID.fromString("00000000-0000-0000-0000-000000000000")), + List.of("not_uuid", 42, true, new byte[] {1})) + ); + + @Test + void testAllKeyTypesAreCovered() { + // Ensure that all KeyTypes have test data defined + for (KeySpaceDirectory.KeyType keyType : KeySpaceDirectory.KeyType.values()) { + assertNotNull(VALUES.get(keyType), "KeyType " + keyType + " is not covered in VALUES map"); + } + } + + static Stream testValidateConstant() { + return VALUES.entrySet().stream() + .flatMap(entry -> Stream.concat( + Stream.concat(entry.getValue().otherValidValues.stream(), entry.getValue().invalidValues.stream()) + .map(valueToAdd -> Arguments.of(entry.getKey(), entry.getValue().value.get(), valueToAdd, false)), + Stream.of(Arguments.of(entry.getKey(), entry.getValue().value.get(), entry.getValue().value.get(), true)))); + } + + @ParameterizedTest + @MethodSource + void testValidateConstant(KeySpaceDirectory.KeyType keyType, Object constantValue, Object valueToAdd, boolean isValid) { + final KeySpaceDirectory directory = new KeySpaceDirectory("test_dir", keyType, constantValue); + if (isValid) { + // Should succeed - value matches constant + directory.validateValue(valueToAdd); + } else { + // Should fail - value doesn't match constant or is invalid type + Assertions.assertThrows(RecordCoreArgumentException.class, () -> directory.validateValue(valueToAdd)); + } + } + + static Stream testValidationValidValues() { + return VALUES.entrySet().stream().flatMap(entry -> + Stream.concat( + Stream.of(entry.getValue().value.get()), + entry.getValue().otherValidValues.stream()) + .map(value -> Arguments.of(entry.getKey(), value))); + } + + @ParameterizedTest + @MethodSource + void testValidationValidValues(KeySpaceDirectory.KeyType keyType, Object value) { + // should succeed + new KeySpaceDirectory("test_dir", keyType).validateValue(value); + } + + static Stream testValidationInvalidValues() { + return VALUES.entrySet().stream().flatMap(entry -> + entry.getValue().invalidValues.stream() + .map(value -> Arguments.of(entry.getKey(), value))); + } + + @ParameterizedTest + @MethodSource + void testValidationInvalidValues(KeySpaceDirectory.KeyType keyType, Object value) { + final KeySpaceDirectory directory = new KeySpaceDirectory("test_dir", keyType); + + Assertions.assertThrows(RecordCoreArgumentException.class, () -> directory.validateValue(value), + "value doesn't match the key type"); + } + + static Stream testValidationNullToNonNullType() { + return Stream.of(KeySpaceDirectory.KeyType.values()) + .filter(type -> type != KeySpaceDirectory.KeyType.NULL); + } + + @ParameterizedTest + @MethodSource + void testValidationNullToNonNullType(KeySpaceDirectory.KeyType keyType) { + final KeySpaceDirectory directory = new KeySpaceDirectory("test_dir", keyType); + + Assertions.assertThrows(RecordCoreArgumentException.class, () -> directory.validateValue(null), + "null not allowed for non-NULL types"); + } + + static Stream testDirectoryLayerDirectoryValidateValueValidStrings() { + return Stream.of( + // ANY_VALUE directory - accepts any string + Arguments.of(false, "foo", true), + Arguments.of(false, "bar", true), + Arguments.of(false, "", true), + Arguments.of(false, "any_string_value", true), + + // Constant directory - only accepts matching constant + Arguments.of(true, "production", true), + Arguments.of(true, "staging", false), + Arguments.of(true, "", false), + Arguments.of(true, "other", false) + ); + } + + @ParameterizedTest + @MethodSource + void testDirectoryLayerDirectoryValidateValueValidStrings(boolean isConstant, String value, boolean shouldSucceed) { + DirectoryLayerDirectory directory = isConstant + ? new DirectoryLayerDirectory("test_dir", "production") + : new DirectoryLayerDirectory("test_dir"); + + if (shouldSucceed) { + directory.validateValue(value); + } else { + Assertions.assertThrows(RecordCoreArgumentException.class, () -> directory.validateValue(value), + "value doesn't match constant"); + } + } + + static Stream testDirectoryLayerDirectoryValidateValueInvalidTypes() { + return ParameterizedTestUtils.cartesianProduct( + ParameterizedTestUtils.booleans("isConstant"), + Stream.of(42L, 123, 3.14f, 2.718, true, new byte[]{1, 2, 3}, UUID.randomUUID(), null) + ); + } + + @ParameterizedTest + @MethodSource + void testDirectoryLayerDirectoryValidateValueInvalidTypes(boolean isConstant, Object value) { + DirectoryLayerDirectory directory = isConstant + ? new DirectoryLayerDirectory("test_dir", "production") + : new DirectoryLayerDirectory("test_dir"); + + Assertions.assertThrows(RecordCoreArgumentException.class, () -> directory.validateValue(value), + "DirectoryLayerDirectory only accepts Strings"); + } + + static final class KeyPathValues { + private final Supplier value; + private final List otherValidValues; + private final List invalidValues; + + KeyPathValues(final Supplier value, final List otherValidValues, final List invalidValues) { + this.value = value; + this.otherValidValues = otherValidValues; + this.invalidValues = invalidValues; + } + } + /** Used to validate wrapping of path names. */ public static class PathA extends KeySpacePathWrapper { public PathA(KeySpacePath parent) { @@ -1593,6 +1755,13 @@ public ConstantResolvingKeySpaceDirectory(String name, KeyType keyType, Object c protected CompletableFuture toTupleValueAsyncImpl(@Nonnull FDBRecordContext context, Object value) { return CompletableFuture.completedFuture(new PathValue(resolver.apply(value))); } + + @Override + public boolean isValueValid(@Nullable Object value) { + // ConstantResolvingKeySpaceDirectory accepts any value and transforms it via the resolver + // The resolved value must match the expected key type + return true; + } } } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathSerializerTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathSerializerTest.java new file mode 100644 index 0000000000..0ad513f020 --- /dev/null +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathSerializerTest.java @@ -0,0 +1,503 @@ +/* + * KeySpacePathSerializerTest.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.RecordCoreArgumentException; +import com.apple.foundationdb.record.provider.foundationdb.keyspace.KeySpaceDirectory.KeyType; +import com.apple.foundationdb.tuple.Tuple; +import com.apple.test.ParameterizedTestUtils; +import com.google.protobuf.ByteString; +import org.junit.jupiter.api.Named; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.Arguments; +import org.junit.jupiter.params.provider.MethodSource; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashMap; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +import static org.junit.jupiter.api.Assertions.assertArrayEquals; +import static org.junit.jupiter.api.Assertions.assertEquals; +import static org.junit.jupiter.api.Assertions.assertNotNull; +import static org.junit.jupiter.api.Assertions.assertNull; +import static org.junit.jupiter.api.Assertions.assertThrows; + +/** + * Tests for {@link KeySpacePathSerializer}. + */ +class KeySpacePathSerializerTest { + + private static final UUID TEST_UUID = UUID.randomUUID(); + private static final Map KEY_TYPE_TEST_VALUES; + + static { + // Using HashMap to allow null values (Map.of() doesn't allow nulls) + Map testValues = new HashMap<>(); + testValues.put(KeyType.NULL, null); + testValues.put(KeyType.STRING, "test_string"); + testValues.put(KeyType.LONG, 12345L); + testValues.put(KeyType.FLOAT, 3.14f); + testValues.put(KeyType.DOUBLE, 2.71828); + testValues.put(KeyType.BOOLEAN, true); + testValues.put(KeyType.BYTES, new byte[]{1, 2, 3, (byte) 0xFF}); + testValues.put(KeyType.UUID, TEST_UUID); + KEY_TYPE_TEST_VALUES = Collections.unmodifiableMap(testValues); + } + + @Test + void testKeyTypeTestValuesIncludesAllKeyTypes() { + // Verify that KEY_TYPE_TEST_VALUES contains all KeyType enum values + var allKeyTypes = Arrays.stream(KeyType.values()).collect(Collectors.toSet()); + var coveredKeyTypes = KEY_TYPE_TEST_VALUES.keySet(); + assertEquals(allKeyTypes, coveredKeyTypes); + } + + @Test + void testSerializeAndDeserializeSimplePath() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("app", KeyType.STRING, "myapp") + .addSubdirectory(new KeySpaceDirectory("tenant", KeyType.STRING))); + + KeySpacePath rootPath = root.path("app"); + KeySpacePath fullPath = rootPath.add("tenant", "tenant1"); + byte[] value = new byte[]{1, 2, 3, 4}; + + DataInKeySpacePath data = new DataInKeySpacePath(fullPath, null, value); + DataInKeySpacePath deserialized = serializeAndDeserialize(rootPath, data); + + assertEquals(fullPath.getDirectoryName(), deserialized.getPath().getDirectoryName()); + assertEquals("tenant1", deserialized.getPath().getValue()); + assertNull(deserialized.getRemainder()); + assertArrayEquals(value, deserialized.getValue()); + } + + @Test + void testSerializeAndDeserializeWithRemainder() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("app", KeyType.STRING, "myapp") + .addSubdirectory(new KeySpaceDirectory("records", KeyType.STRING))); + + KeySpacePath rootPath = root.path("app"); + KeySpacePath fullPath = rootPath.add("records", "store1"); + Tuple remainder = Tuple.from("key1", "key2"); + byte[] value = new byte[]{10, 20, 30}; + + DataInKeySpacePath data = new DataInKeySpacePath(fullPath, remainder, value); + DataInKeySpacePath deserialized = serializeAndDeserialize(rootPath, data); + + assertEquals("store1", deserialized.getPath().getValue()); + assertNotNull(deserialized.getRemainder()); + assertEquals(remainder, deserialized.getRemainder()); + assertArrayEquals(value, deserialized.getValue()); + } + + @Test + void testSerializeAndDeserializeMultiLevelPath() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("root", KeyType.STRING, "r") + .addSubdirectory(new KeySpaceDirectory("level1", KeyType.STRING) + .addSubdirectory(new KeySpaceDirectory("level2", KeyType.LONG) + .addSubdirectory(new KeySpaceDirectory("level3", KeyType.STRING))))); + + KeySpacePath rootPath = root.path("root"); + KeySpacePath fullPath = rootPath + .add("level1", "l1value") + .add("level2", 42L) + .add("level3", "l3value"); + byte[] value = new byte[]{100}; + + DataInKeySpacePath data = new DataInKeySpacePath(fullPath, null, value); + DataInKeySpacePath deserialized = serializeAndDeserialize(rootPath, data); + + assertEquals("l3value", deserialized.getPath().getValue()); + assertArrayEquals(value, deserialized.getValue()); + } + + static Stream testSerializeDeserializeAllKeyTypes() { + return ParameterizedTestUtils.cartesianProduct( + KEY_TYPE_TEST_VALUES.entrySet().stream(), + ParameterizedTestUtils.booleans("isConstant")); + } + + @ParameterizedTest + @MethodSource + void testSerializeDeserializeAllKeyTypes(Map.Entry typeAndValue, boolean isConstant) { + final KeyType keyType = typeAndValue.getKey(); + final Object value = typeAndValue.getValue(); + KeySpace root = new KeySpace( + new KeySpaceDirectory("root", KeyType.STRING, "root") + .addSubdirectory(isConstant ? + new KeySpaceDirectory("typed", keyType, value) : + new KeySpaceDirectory("typed", keyType))); + + KeySpacePath rootPath = root.path("root"); + KeySpacePath fullPath = rootPath.add("typed", value); + byte[] dataValue = new byte[]{1}; + + DataInKeySpacePath data = new DataInKeySpacePath(fullPath, null, dataValue); + final DataInKeySpacePath deserialized = serializeAndDeserialize(rootPath, data); + + if (value instanceof byte[]) { + assertArrayEquals((byte[]) value, (byte[]) deserialized.getPath().getValue()); + } else { + assertEquals(value, deserialized.getPath().getValue()); + } + assertArrayEquals(dataValue, deserialized.getValue()); + } + + @Test + void testSerializeWithIntegerForLongKeyType() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("root", KeyType.STRING, "root") + .addSubdirectory(new KeySpaceDirectory("long_dir", KeyType.LONG))); + + KeySpacePath rootPath = root.path("root"); + // Pass an Integer for a LONG key type + KeySpacePath fullPath = rootPath.add("long_dir", 42); + byte[] value = new byte[]{1}; + + DataInKeySpacePath data = new DataInKeySpacePath(fullPath, null, value); + final DataInKeySpacePath deserialized = serializeAndDeserialize(rootPath, data); + + // Should deserialize as Long + assertEquals(42L, deserialized.getPath().getValue()); + } + + @Test + void testSerializeRootOnly() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("root", KeyType.STRING, "rootval")); + + KeySpacePath rootPath = root.path("root"); + byte[] value = new byte[]{5, 6, 7}; + + DataInKeySpacePath data = new DataInKeySpacePath(rootPath, null, value); + final DataInKeySpacePath deserialized = serializeAndDeserialize(rootPath, data); + + assertEquals("rootval", deserialized.getPath().getValue()); + assertArrayEquals(value, deserialized.getValue()); + } + + @Test + void testSerializePathNotContainedInRoot() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("app1", KeyType.STRING, "app1") + .addSubdirectory(new KeySpaceDirectory("tenant", KeyType.STRING))); + + KeySpace otherRoot = new KeySpace( + new KeySpaceDirectory("app2", KeyType.STRING, "app2") + .addSubdirectory(new KeySpaceDirectory("tenant", KeyType.STRING))); + + KeySpacePath rootPath = root.path("app1"); + KeySpacePath otherPath = otherRoot.path("app2").add("tenant", "t1"); + byte[] value = new byte[]{1}; + + DataInKeySpacePath data = new DataInKeySpacePath(otherPath, null, value); + KeySpacePathSerializer serializer = new KeySpacePathSerializer(rootPath); + assertThrows(RecordCoreArgumentException.class, () -> serializer.serialize(data)); + } + + @Test + void testSerializeWithEmptyValue() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("root", KeyType.STRING, "root")); + + KeySpacePath rootPath = root.path("root"); + byte[] value = new byte[]{}; + + DataInKeySpacePath data = new DataInKeySpacePath(rootPath, null, value); + final DataInKeySpacePath deserialized = serializeAndDeserialize(rootPath, data); + + assertArrayEquals(value, deserialized.getValue()); + assertEquals(0, deserialized.getValue().length); + } + + @Test + void testDeserializeInvalidProto() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("root", KeyType.STRING, "root")); + + KeySpacePath rootPath = root.path("root"); + KeySpacePathSerializer serializer = new KeySpacePathSerializer(rootPath); + + // Create invalid ByteString + ByteString invalid = ByteString.copyFrom(new byte[]{1, 2, 3, 4, 5}); + + assertThrows(RecordCoreArgumentException.class, () -> serializer.deserialize(invalid)); + } + + @Test + void testRoundTripWithComplexRemainder() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("db", KeyType.STRING, "database")); + + KeySpacePath rootPath = root.path("db"); + Tuple remainder = Tuple.from("string", 123L, 3.14, true, new byte[]{1, 2}); + byte[] value = new byte[]{9, 8, 7}; + + DataInKeySpacePath data = new DataInKeySpacePath(rootPath, remainder, value); + final DataInKeySpacePath deserialized = serializeAndDeserialize(rootPath, data); + + assertEquals(remainder, deserialized.getRemainder()); + assertArrayEquals(value, deserialized.getValue()); + } + + @Test + void testSerializeNullKeyType() { + KeySpace root = new KeySpace( + new KeySpaceDirectory("root", KeyType.STRING, "root") + .addSubdirectory(new KeySpaceDirectory("null_dir", KeyType.NULL))); + + KeySpacePath rootPath = root.path("root"); + KeySpacePath fullPath = rootPath.add("null_dir"); + byte[] value = new byte[]{1}; + + DataInKeySpacePath data = new DataInKeySpacePath(fullPath, null, value); + final DataInKeySpacePath deserialized = serializeAndDeserialize(rootPath, data); + + assertNull(deserialized.getPath().getValue()); + } + + static Stream testSerializeDeserializeDifferentRoot() { + return Stream.of( + Arguments.of(Named.of("Same", new KeySpaceDirectory("tenant", KeyType.STRING) + .addSubdirectory(new KeySpaceDirectory("record", KeyType.LONG))), + Named.of("successful", null)), + Arguments.of(Named.of("Extra subdirectories in destination", + new KeySpaceDirectory("tenant", KeyType.STRING) + .addSubdirectory(new KeySpaceDirectory("users", KeyType.STRING)) + .addSubdirectory(new KeySpaceDirectory("groups", KeyType.BYTES)) + .addSubdirectory(new KeySpaceDirectory("record", KeyType.LONG)) + .addSubdirectory(new KeySpaceDirectory("settings", KeyType.BOOLEAN))), + Named.of("successful", null)), + Arguments.of(Named.of("Different constant", + new KeySpaceDirectory("tenant", KeyType.STRING) + .addSubdirectory(new KeySpaceDirectory("record", KeyType.LONG, 104L))), + RecordCoreArgumentException.class), + Arguments.of(Named.of("Different type", + new KeySpaceDirectory("tenant", KeyType.STRING) + .addSubdirectory(new KeySpaceDirectory("record", KeyType.STRING))), + RecordCoreArgumentException.class), + Arguments.of(Named.of("Null type", + new KeySpaceDirectory("tenant", KeyType.STRING) + .addSubdirectory(new KeySpaceDirectory("record", KeyType.NULL))), + RecordCoreArgumentException.class), + Arguments.of(Named.of("Different name", + new KeySpaceDirectory("tenant", KeyType.STRING) + .addSubdirectory(new KeySpaceDirectory("compact disc", KeyType.STRING))), + NoSuchDirectoryException.class), + Arguments.of(Named.of("Missing subdirectory", + new KeySpaceDirectory("tenant", KeyType.STRING)), + NoSuchDirectoryException.class)); + } + + @ParameterizedTest + @MethodSource + void testSerializeDeserializeDifferentRoot(KeySpaceDirectory destDirectory, + @Nullable Class errorType) { + KeySpace keySpace = new KeySpace( + new KeySpaceDirectory("source_app", KeyType.STRING, "app1") + .addSubdirectory(new KeySpaceDirectory("tenant", KeyType.STRING) + .addSubdirectory(new KeySpaceDirectory("record", KeyType.LONG))), + new KeySpaceDirectory("dest_app", KeyType.STRING, "app2") + .addSubdirectory(destDirectory)); + + // Create data in source hierarchy + KeySpacePath sourcePath = keySpace.path("source_app") + .add("tenant", "tenant1") + .add("record", 42L); + byte[] value = new byte[]{10, 20, 30}; + DataInKeySpacePath sourceData = new DataInKeySpacePath(sourcePath, null, value); + + // Serialize from source + KeySpacePathSerializer sourceSerializer = new KeySpacePathSerializer(keySpace.path("source_app")); + ByteString serialized1 = sourceSerializer.serialize(sourceData); + + // Deserialize to destination + KeySpacePathSerializer destSerializer = new KeySpacePathSerializer(keySpace.path("dest_app")); + + if (errorType == null) { + DataInKeySpacePath deserializedData = destSerializer.deserialize(serialized1); + + assertEquals("dest_app", deserializedData.getPath().getParent().getParent().getDirectoryName()); + assertEquals("tenant1", deserializedData.getPath().getParent().getValue()); + assertEquals(42L, deserializedData.getPath().getValue()); + assertArrayEquals(new byte[] {10, 20, 30}, deserializedData.getValue()); + } else { + assertThrows(errorType, () -> destSerializer.deserialize(serialized1)); + } + } + + @Test + void testDeserializeSameStructureDifferentRootValue() { + // Create a KeySpace with a single root having multiple children with identical structures + KeySpace keySpace = new KeySpace( + new KeySpaceDirectory("environments", KeyType.NULL) + .addSubdirectory(new KeySpaceDirectory("production", KeyType.STRING, "prod") + .addSubdirectory(new KeySpaceDirectory("database", KeyType.STRING) + .addSubdirectory(new KeySpaceDirectory("table", KeyType.STRING)))) + .addSubdirectory(new KeySpaceDirectory("staging", KeyType.STRING, "stage") + .addSubdirectory(new KeySpaceDirectory("database", KeyType.STRING) + .addSubdirectory(new KeySpaceDirectory("table", KeyType.STRING))))); + + // Create data in production + KeySpacePath sourcePath = keySpace.path("environments").add("production") + .add("database", "db1") + .add("table", "users"); + Tuple remainder = Tuple.from("primary_key", 12345L); + byte[] value = new byte[]{(byte) 0xFF, 0x00, 0x11}; + DataInKeySpacePath sourceData = new DataInKeySpacePath(sourcePath, remainder, value); + + // Serialize from production + KeySpacePathSerializer sourceSerializer = new KeySpacePathSerializer(keySpace.path("environments").add("production")); + ByteString serialized = sourceSerializer.serialize(sourceData); + + // Deserialize to staging + KeySpacePathSerializer destSerializer = new KeySpacePathSerializer(keySpace.path("environments").add("staging")); + DataInKeySpacePath deserializedData = destSerializer.deserialize(serialized); + + // Verify the root value changed but path structure and data preserved + assertEquals("staging", deserializedData.getPath().getParent().getParent().getDirectoryName()); + assertEquals("stage", deserializedData.getPath().getParent().getParent().getValue()); + + // The logical path values should be preserved + assertEquals("db1", deserializedData.getPath().getParent().getValue()); + assertEquals("users", deserializedData.getPath().getValue()); + assertEquals(remainder, deserializedData.getRemainder()); + assertArrayEquals(value, deserializedData.getValue()); + } + + @Test + void testSerializeDirectoryLayerDirectoryWithStringValue() { + // DirectoryLayerDirectory has KeyType.LONG but typically accepts String values + // The serializer uses KeyType.typeOf to determine the value is a String and serialize it as STRING + KeySpace root = new KeySpace( + new KeySpaceDirectory("app", KeyType.STRING, "myapp") + .addSubdirectory(new DirectoryLayerDirectory("tenant") + .addSubdirectory(new KeySpaceDirectory("data", KeyType.STRING)))); + + KeySpacePath rootPath = root.path("app"); + // Use String value (the typical usage with DirectoryLayerDirectory) + KeySpacePath fullPath = rootPath.add("tenant", "my_tenant").add("data", "mydata"); + byte[] value = new byte[]{1, 2, 3}; + + DataInKeySpacePath data = new DataInKeySpacePath(fullPath, null, value); + DataInKeySpacePath deserialized = serializeAndDeserialize(rootPath, data); + + // Verify the String value was preserved + assertEquals("my_tenant", deserialized.getPath().getParent().getValue()); + assertEquals("mydata", deserialized.getPath().getValue()); + assertArrayEquals(value, deserialized.getValue()); + } + + @Test + void testSerializeDirectoryLayerDirectoryWithConstant() { + // Test DirectoryLayerDirectory with a constant String value + KeySpace root = new KeySpace( + new KeySpaceDirectory("app", KeyType.STRING, "myapp") + .addSubdirectory(new DirectoryLayerDirectory("tenant", "my_tenant") + .addSubdirectory(new KeySpaceDirectory("data", KeyType.STRING)))); + + KeySpacePath rootPath = root.path("app"); + KeySpacePath fullPath = rootPath.add("tenant", "my_tenant").add("data", "mydata"); + byte[] value = new byte[]{1, 2, 3}; + + DataInKeySpacePath data = new DataInKeySpacePath(fullPath, null, value); + DataInKeySpacePath deserialized = serializeAndDeserialize(rootPath, data); + + // Verify the constant String value was preserved + assertEquals("my_tenant", deserialized.getPath().getParent().getValue()); + assertEquals("mydata", deserialized.getPath().getValue()); + assertArrayEquals(value, deserialized.getValue()); + } + + @Test + void testSerializeDirectoryLayerDirectoryMultiLevel() { + // Test multiple DirectoryLayerDirectory nodes in a path with String values + KeySpace root = new KeySpace( + new KeySpaceDirectory("root", KeyType.STRING, "r") + .addSubdirectory(new DirectoryLayerDirectory("app") + .addSubdirectory(new DirectoryLayerDirectory("tenant") + .addSubdirectory(new KeySpaceDirectory("record", KeyType.LONG))))); + + KeySpacePath rootPath = root.path("root"); + // Use String values for DirectoryLayerDirectory nodes + KeySpacePath fullPath = rootPath + .add("app", "my_app") + .add("tenant", "my_tenant") + .add("record", 12345L); + byte[] value = new byte[]{10, 20}; + + DataInKeySpacePath data = new DataInKeySpacePath(fullPath, null, value); + DataInKeySpacePath deserialized = serializeAndDeserialize(rootPath, data); + + assertEquals("my_app", deserialized.getPath().getParent().getParent().getValue()); + assertEquals("my_tenant", deserialized.getPath().getParent().getValue()); + assertEquals(12345L, deserialized.getPath().getValue()); + assertArrayEquals(value, deserialized.getValue()); + } + + @Test + void testDeserializeDirectoryLayerDirectoryToDifferentRoot() { + // Test that DirectoryLayerDirectory paths can be serialized from one root and deserialized to another + KeySpace keySpace = new KeySpace( + new KeySpaceDirectory("source_app", KeyType.STRING, "app1") + .addSubdirectory(new DirectoryLayerDirectory("tenant") + .addSubdirectory(new KeySpaceDirectory("record", KeyType.LONG))), + new KeySpaceDirectory("dest_app", KeyType.STRING, "app2") + .addSubdirectory(new DirectoryLayerDirectory("tenant") + .addSubdirectory(new KeySpaceDirectory("record", KeyType.LONG)))); + + // Create data in source hierarchy with String value for DirectoryLayerDirectory + KeySpacePath sourcePath = keySpace.path("source_app") + .add("tenant", "tenant1") + .add("record", 123L); + byte[] value = new byte[]{5, 6, 7}; + DataInKeySpacePath sourceData = new DataInKeySpacePath(sourcePath, null, value); + + // Serialize from source + KeySpacePathSerializer sourceSerializer = new KeySpacePathSerializer(keySpace.path("source_app")); + ByteString serialized = sourceSerializer.serialize(sourceData); + + // Deserialize to destination + KeySpacePathSerializer destSerializer = new KeySpacePathSerializer(keySpace.path("dest_app")); + DataInKeySpacePath deserializedData = destSerializer.deserialize(serialized); + + // Verify structure is preserved + assertEquals("dest_app", deserializedData.getPath().getParent().getParent().getDirectoryName()); + assertEquals("tenant1", deserializedData.getPath().getParent().getValue()); + assertEquals(123L, deserializedData.getPath().getValue()); + assertArrayEquals(value, deserializedData.getValue()); + } + + @Nonnull + private static DataInKeySpacePath serializeAndDeserialize(final KeySpacePath rootPath, final DataInKeySpacePath data) { + final KeySpacePathSerializer serializer = new KeySpacePathSerializer(rootPath); + return serializer.deserialize(serializer.serialize(data)); + } +}