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 f2d2284730..729ab6eaa8 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
@@ -53,6 +53,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..ca98847328 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
+ protected 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..0057b79e4b 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,47 @@ LogMessageKeys.DIR_NAME, getName(),
}
}
+ /**
+ * Validate that the given value can be used with this directory.
+ * @param value a potential value
+ * @throws RecordCoreArgumentException if the value is not valid
+ */
+ protected 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 both String
+ * (logical names) and Long (directory layer values) even though its key type is LONG.
+ *
+ * @param value the value to validate
+ * @return {@code true} if the value is valid for this directory
+ */
+ protected 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 ff228ef1be..642fb51d90 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
@@ -410,4 +410,13 @@ default RecordCursor exportAllData(@Nonnull FDBRecordContext
@Nonnull ScanProperties scanProperties) {
throw new UnsupportedOperationException("exportAllData is not supported");
}
+
+ /**
+ * 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..4a95a5db33
--- /dev/null
+++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathSerializer.java
@@ -0,0 +1,208 @@
+/*
+ * 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 can be
+ * useful to both:
+ *
+ *
Reduce the size of the serialized data, particularly when you have a lot of these.
+ *
Allowing 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;
+
+ public KeySpacePathSerializer(@Nonnull final KeySpacePath root) {
+ this.root = root.flatten();
+ }
+
+ @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();
+ }
+
+ @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();
+ final KeySpaceDirectory.KeyType keyType = keySpacePath.getDirectory().getKeyType();
+
+ // Validate null handling: NULL type must have null value, all other types must not have null value
+ if (keyType == KeySpaceDirectory.KeyType.NULL) {
+ if (value != null) {
+ throw new RecordCoreArgumentException("NULL key type must have null value")
+ .addLogInfo(LogMessageKeys.DIR_NAME, keySpacePath.getDirectoryName(),
+ LogMessageKeys.ACTUAL, value);
+ }
+ } else {
+ if (value == null) {
+ throw new RecordCoreArgumentException("Non-NULL key type cannot have null value")
+ .addLogInfo(LogMessageKeys.DIR_NAME, keySpacePath.getDirectoryName(),
+ LogMessageKeys.EXPECTED_TYPE, keyType);
+ }
+ }
+
+ 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..2f64e90452
--- /dev/null
+++ b/fdb-record-layer-core/src/main/proto/keyspace.proto
@@ -0,0 +1,52 @@
+/*
+ * 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 { // TODO find out why we use fixed64 and not just int64
+ // 2 64-bit fields is two tags, the same as 1 bytes field with a length of 16 would be.
+ // 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..06b525c4e1 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;
@@ -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