Skip to content

Commit 4542468

Browse files
authored
Add new KeySpacePath.exportAllData (#3566)
This introduces a new KeySpacePath.exportAllData to export all the data stored in the path. This can eventually be used to import into another cluster, or back into the same cluster, after clearing. Other than the path information, the data exported is raw bytes, with no transformation or indicated semantics. Resolves: [3572](#3572)
1 parent 7cca930 commit 4542468

File tree

13 files changed

+1781
-157
lines changed

13 files changed

+1781
-157
lines changed

fdb-extensions/src/main/java/com/apple/foundationdb/tuple/TupleHelpers.java

Lines changed: 7 additions & 7 deletions
Original file line numberDiff line numberDiff line change
@@ -141,17 +141,17 @@ public static int packedSizeAsTupleItem(Object item) {
141141

142142
/**
143143
* Get whether one tuple is a prefix of another.
144-
* @param t1 the potential prefix
145-
* @param t2 the whole tuple
146-
* @return {@code true} if {@code t1} is a prefix of {@code t2}
144+
* @param potentialPrefix the potential prefix
145+
* @param wholeTuple the whole tuple
146+
* @return {@code true} if {@code potentialPrefix} is a prefix of {@code wholeTuple}
147147
*/
148-
public static boolean isPrefix(@Nonnull Tuple t1, @Nonnull Tuple t2) {
149-
final int len = t1.size();
150-
if (t2.size() < len) {
148+
public static boolean isPrefix(@Nonnull Tuple potentialPrefix, @Nonnull Tuple wholeTuple) {
149+
final int len = potentialPrefix.size();
150+
if (wholeTuple.size() < len) {
151151
return false;
152152
}
153153
for (int i = 0; i < len; i++) {
154-
int rc = TupleUtil.compareItems(t1.get(i), t2.get(i));
154+
int rc = TupleUtil.compareItems(potentialPrefix.get(i), wholeTuple.get(i));
155155
if (rc != 0) {
156156
return false;
157157
}
Lines changed: 60 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -0,0 +1,60 @@
1+
/*
2+
* DataInKeySpacePath.java
3+
*
4+
* This source file is part of the FoundationDB open source project
5+
*
6+
* Copyright 2015-2025 Apple Inc. and the FoundationDB project authors
7+
*
8+
* Licensed under the Apache License, Version 2.0 (the "License");
9+
* you may not use this file except in compliance with the License.
10+
* You may obtain a copy of the License at
11+
*
12+
* http://www.apache.org/licenses/LICENSE-2.0
13+
*
14+
* Unless required by applicable law or agreed to in writing, software
15+
* distributed under the License is distributed on an "AS IS" BASIS,
16+
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
17+
* See the License for the specific language governing permissions and
18+
* limitations under the License.
19+
*/
20+
21+
package com.apple.foundationdb.record.provider.foundationdb.keyspace;
22+
23+
import com.apple.foundationdb.KeyValue;
24+
import com.apple.foundationdb.annotation.API;
25+
import com.apple.foundationdb.record.RecordCoreArgumentException;
26+
import com.apple.foundationdb.record.logging.LogMessageKeys;
27+
import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext;
28+
import com.apple.foundationdb.tuple.ByteArrayUtil2;
29+
30+
import javax.annotation.Nonnull;
31+
import java.util.concurrent.CompletableFuture;
32+
33+
/**
34+
* Class representing a {@link KeyValue} pair within in {@link KeySpacePath}.
35+
*/
36+
@API(API.Status.EXPERIMENTAL)
37+
public class DataInKeySpacePath {
38+
39+
@Nonnull
40+
private final CompletableFuture<ResolvedKeySpacePath> resolvedPath;
41+
@Nonnull
42+
private final byte[] value;
43+
44+
public DataInKeySpacePath(KeySpacePath path, KeyValue rawKeyValue, FDBRecordContext context) {
45+
this.resolvedPath = path.toResolvedPathAsync(context, rawKeyValue.getKey());
46+
this.value = rawKeyValue.getValue();
47+
if (this.value == null) {
48+
throw new RecordCoreArgumentException("Value cannot be null")
49+
.addLogInfo(LogMessageKeys.KEY, ByteArrayUtil2.loggable(rawKeyValue.getKey()));
50+
}
51+
}
52+
53+
public CompletableFuture<ResolvedKeySpacePath> getResolvedPath() {
54+
return resolvedPath;
55+
}
56+
57+
public byte[] getValue() {
58+
return this.value;
59+
}
60+
}

fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpace.java

Lines changed: 6 additions & 3 deletions
Original file line numberDiff line numberDiff line change
@@ -227,9 +227,12 @@ public KeySpacePath pathFromKey(@Nonnull FDBRecordContext context, @Nonnull Tupl
227227
/**
228228
* Given a tuple from an FDB key, attempts to determine what path through this directory the tuple
229229
* represents, returning a <code>ResolvedKeySpacePath</code> representing the leaf-most directory in the path.
230-
* If entries remained in the tuple beyond the leaf directory, then {@link KeySpacePath#getRemainder()}
231-
* can be used to fetch the remaining portion.
232-
*
230+
* <p>
231+
* If entries remained in the tuple beyond the leaf directory, then {@link KeySpacePath#getRemainder()} can be
232+
* used to fetch the remaining portion.
233+
* See also {@link KeySpacePath#toResolvedPathAsync(FDBRecordContext, byte[])} if you need to resolve and you
234+
* know that it is part of a given path.
235+
* </p>
233236
* @param context context used, if needed, for any database operations
234237
* @param key the tuple to be decoded
235238
* @return a path entry representing the leaf directory entry that corresponds to a value in the

fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePath.java

Lines changed: 37 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -198,6 +198,25 @@ default Tuple toTuple(@Nonnull FDBRecordContext context) {
198198
@Nonnull
199199
CompletableFuture<ResolvedKeySpacePath> toResolvedPathAsync(@Nonnull FDBRecordContext context);
200200

201+
/**
202+
* Given a tuple from an FDB key, attempts to determine what sub-path through this directory the tuple
203+
* represents, returning a <code>ResolvedKeySpacePath</code> representing the leaf-most directory in the path.
204+
* <p>
205+
* If entries remained in the tuple beyond the leaf directory, then {@link KeySpacePath#getRemainder()}
206+
* can be used to fetch the remaining portion.
207+
* See also {@link KeySpace#resolveFromKeyAsync(FDBRecordContext, Tuple)} if you need to resolve from the root.
208+
* </p>
209+
* @param context context used, if needed, for any database operations
210+
* @param key a raw key from the database
211+
* @return the {@link ResolvedKeySpacePath} corresponding to that key, with a potential remainder.
212+
* @throws com.apple.foundationdb.record.RecordCoreArgumentException if the key provided is not part of this path
213+
*/
214+
@API(API.Status.EXPERIMENTAL)
215+
@Nonnull
216+
default CompletableFuture<ResolvedKeySpacePath> toResolvedPathAsync(@Nonnull FDBRecordContext context, byte[] key) {
217+
throw new UnsupportedOperationException("toResolvedPathAsync is not supported");
218+
}
219+
201220
/**
202221
* Resolves the path into a {@link ResolvedKeySpacePath}, a form the retains all of the information about
203222
* the path itself along with the value to which each path entry is resolved.
@@ -566,4 +585,22 @@ default List<ResolvedKeySpacePath> listSubdirectory(@Nonnull FDBRecordContext co
566585
*/
567586
@API(API.Status.UNSTABLE)
568587
String toString(@Nonnull Tuple tuple);
588+
589+
/**
590+
* Export all data stored under this KeySpacePath and return it in a RecordCursor.
591+
* This method scans all keys that have this path as a prefix and returns the key-value pairs.
592+
* Supports continuation to resume scanning from a previous position.
593+
*
594+
* @param context the transaction context in which to perform the data export
595+
* @param continuation optional continuation from a previous export operation, or null to start from the beginning
596+
* @param scanProperties properties controlling how the scan should be performed
597+
* @return a RecordCursor that iterates over all KeyValue pairs under this path
598+
*/
599+
@API(API.Status.EXPERIMENTAL)
600+
@Nonnull
601+
default RecordCursor<DataInKeySpacePath> exportAllData(@Nonnull FDBRecordContext context,
602+
@Nullable byte[] continuation,
603+
@Nonnull ScanProperties scanProperties) {
604+
throw new UnsupportedOperationException("exportAllData is not supported");
605+
}
569606
}

fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathImpl.java

Lines changed: 55 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -26,10 +26,16 @@
2626
import com.apple.foundationdb.record.RecordCursor;
2727
import com.apple.foundationdb.record.ScanProperties;
2828
import com.apple.foundationdb.record.ValueRange;
29+
import com.apple.foundationdb.record.cursors.LazyCursor;
30+
import com.apple.foundationdb.record.logging.LogMessageKeys;
2931
import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext;
32+
import com.apple.foundationdb.record.provider.foundationdb.KeyValueCursor;
33+
import com.apple.foundationdb.subspace.Subspace;
3034
import com.apple.foundationdb.tuple.ByteArrayUtil;
3135
import com.apple.foundationdb.tuple.Tuple;
36+
import com.apple.foundationdb.tuple.TupleHelpers;
3237
import com.google.common.collect.Lists;
38+
3339
import javax.annotation.Nonnull;
3440
import javax.annotation.Nullable;
3541
import java.util.ArrayList;
@@ -242,6 +248,40 @@ public CompletableFuture<ResolvedKeySpacePath> toResolvedPathAsync(@Nonnull FDBR
242248
});
243249
}
244250

251+
@Nonnull
252+
@Override
253+
public CompletableFuture<ResolvedKeySpacePath> toResolvedPathAsync(@Nonnull final FDBRecordContext context, final byte[] key) {
254+
final Tuple keyTuple = Tuple.fromBytes(key);
255+
return toResolvedPathAsync(context).thenCompose(resolvedPath -> {
256+
// Now use the resolved path to find the child for the key
257+
// We need to figure out how much of the key corresponds to the resolved path
258+
Tuple pathTuple = resolvedPath.toTuple();
259+
int pathLength = pathTuple.size();
260+
261+
if (!TupleHelpers.isPrefix(pathTuple, keyTuple)) {
262+
throw new RecordCoreArgumentException("Key is not under this path")
263+
.addLogInfo(LogMessageKeys.EXPECTED, pathTuple,
264+
LogMessageKeys.ACTUAL, keyTuple);
265+
}
266+
267+
// The remaining part of the key should be resolved from the resolved path's directory
268+
if (keyTuple.size() > pathLength) {
269+
// There's more in the key than just the path, so resolve the rest
270+
if (resolvedPath.getDirectory().getSubdirectories().isEmpty()) {
271+
return CompletableFuture.completedFuture(
272+
new ResolvedKeySpacePath(resolvedPath.getParent(), resolvedPath.toPath(),
273+
resolvedPath.getResolvedPathValue(),
274+
TupleHelpers.subTuple(keyTuple, pathTuple.size(), keyTuple.size())));
275+
} else {
276+
return resolvedPath.getDirectory().findChildForKey(context, resolvedPath, keyTuple, keyTuple.size(), pathLength);
277+
}
278+
} else {
279+
// The key exactly matches the path
280+
return CompletableFuture.completedFuture(resolvedPath);
281+
}
282+
});
283+
}
284+
245285
@Nonnull
246286
@Override
247287
public CompletableFuture<Boolean> hasDataAsync(@Nonnull FDBRecordContext context) {
@@ -331,6 +371,21 @@ public String toString() {
331371
return toString(null);
332372
}
333373

374+
@Nonnull
375+
@Override
376+
public RecordCursor<DataInKeySpacePath> exportAllData(@Nonnull FDBRecordContext context,
377+
@Nullable byte[] continuation,
378+
@Nonnull ScanProperties scanProperties) {
379+
return new LazyCursor<>(toTupleAsync(context)
380+
.thenApply(tuple -> KeyValueCursor.Builder.withSubspace(new Subspace(tuple))
381+
.setContext(context)
382+
.setContinuation(continuation)
383+
.setScanProperties(scanProperties)
384+
.build()),
385+
context.getExecutor())
386+
.map(keyValue -> new DataInKeySpacePath(this, keyValue, context));
387+
}
388+
334389
/**
335390
* Returns this path properly wrapped in whatever implementation the directory the path is contained in dictates.
336391
*/

fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/KeySpacePathWrapper.java

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -206,6 +206,11 @@ public CompletableFuture<ResolvedKeySpacePath> toResolvedPathAsync(@Nonnull FDBR
206206
return inner.toResolvedPathAsync(context);
207207
}
208208

209+
@Nonnull
210+
@Override
211+
public CompletableFuture<ResolvedKeySpacePath> toResolvedPathAsync(@Nonnull final FDBRecordContext context, final byte[] key) {
212+
return inner.toResolvedPathAsync(context, key);
213+
}
209214

210215
@Override
211216
public boolean equals(Object obj) {
@@ -226,4 +231,12 @@ public String toString() {
226231
public String toString(@Nonnull Tuple t) {
227232
return inner.toString(t);
228233
}
234+
235+
@Nonnull
236+
@Override
237+
public RecordCursor<DataInKeySpacePath> exportAllData(@Nonnull FDBRecordContext context,
238+
@Nullable byte[] continuation,
239+
@Nonnull ScanProperties scanProperties) {
240+
return inner.exportAllData(context, continuation, scanProperties);
241+
}
229242
}

fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/keyspace/ResolvedKeySpacePath.java

Lines changed: 13 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -24,6 +24,7 @@
2424
import com.apple.foundationdb.subspace.Subspace;
2525
import com.apple.foundationdb.tuple.ByteArrayUtil2;
2626
import com.apple.foundationdb.tuple.Tuple;
27+
import com.google.common.annotations.VisibleForTesting;
2728

2829
import javax.annotation.Nonnull;
2930
import javax.annotation.Nullable;
@@ -271,4 +272,16 @@ public static void appendValue(StringBuilder sb, Object value) {
271272
sb.append(value);
272273
}
273274
}
275+
276+
/**
277+
* Returns a new {@code ResolvedKeySpacePath} that is the same, except with the provided {@link #getRemainder()}.
278+
* @param newRemainder a new remainder. This can be {@code null} to remove the remainder entirely.
279+
* @return a new {@code ResolvedKeySpacePath} that is the same as this, except with a different {@link #getRemainder()}.
280+
*/
281+
@Nonnull
282+
@VisibleForTesting
283+
ResolvedKeySpacePath withRemainder(@Nullable final Tuple newRemainder) {
284+
// this could probably copy the cachedTuple & cachedSubspace
285+
return new ResolvedKeySpacePath(parent, inner, value, newRemainder);
286+
}
274287
}

0 commit comments

Comments
 (0)