-
Notifications
You must be signed in to change notification settings - Fork 112
Add new KeySpacePath.exportAllData #3566
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
base: main
Are you sure you want to change the base?
Conversation
This will be returned by export
Most of the time this will be used with remainders, so make most of the tests cover that
import javax.annotation.Nonnull; | ||
import java.util.UUID; | ||
|
||
/** |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
In case you're reviewing this top-to-bottom, this class was extracted from KeySpaceDirectoryTest, but setupSampleData
was added.
@Nullable byte[] continuation, | ||
@Nonnull ScanProperties scanProperties) { | ||
return new LazyCursor<>(toTupleAsync(context) | ||
.thenApply(tuple -> KeyValueCursor.Builder.withSubspace(new Subspace(tuple)) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Just to try and head off some potential problems: in #3397, there's a modification being made to the KeyValueCursor
continuation to wrap it in a protobuf. This would allow us to avoid returning an empty ByteString
as the continuation in case there is a single key range, which may be something that a KeySpacePath
could run into. (It would require that the first key in the keyspace path is set, I think.)
There's sort of an open question as to how we migrate all of the uses over from the old format to the new format. It would be nice if we could set the serialization mode on this one to the new format so that we don't have to worry about migrating it later, I think. Though that would introduce a dependency on #3397 being merged first.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
That's a good point.
I changed exportAllData
on the interface to be EXPERIMENTAL
, and added a note that we expect the continuation change without preserving backwards compatibility when that merges. If the rest of the pieces are in place such that this is useful before #3397 gets merged, we can potentially update the javadoc, and make this be backwards compatible.
this.rawKeyValue = rawKeyValue; | ||
|
||
// Convert the raw key to a Tuple and resolve it starting from the provided path | ||
Tuple keyTuple = Tuple.fromBytes(rawKeyValue.getKey()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Do we need to be concerned with keys that are not Tuple
parseable? I believe all of the keys that we currently generate are Tuple
parseable, though we could adjust that in the future
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
KeySpacePathImpl
always produces tuples, and AFAIK every key we have ever used has been in tuples. I feel like committing to that at this point, is worthwhile. If we find a strong reason to put non-tuple data in a key, we'll have to cross that bridge when we get to it.
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(); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should this validate that pathTuple
is a prefix of keyTuple
? Strictly speaking, I don't think the check is necessary, but it could be a sensible thing to validate
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, I think that's a good double check.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Note: I took advantage of TupleHelpers.isPrefix
, which apparently didn't have any unit tests, but I'm adding them in a followup PR. https://github.com/FoundationDB/fdb-record-layer/pull/3578/files#diff-c7f3ee5c693bba884071e40ab98f13c12a86350a819f71f1c6a60eb7b750623c
I'm not sure if it's worth bringing into this PR.
Tuple keyTuple = Tuple.fromBytes(rawKeyValue.getKey()); | ||
|
||
// First resolve the provided path to get its resolved form | ||
this.resolvedPath = path.toResolvedPathAsync(context).thenCompose(resolvedPath -> { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It almost seems like the method provided should be a method on the KeySpacePath
. I was sort of surprised that we didn't already have it. I can see how we'd have to be clear in the name that we're starting with the full key rather than just the suffix. I suppose we could structure this here to go the other way: we resolve the KeySpacePath
first, use it to get a raw byte prefix, and then resolve the Tuple
suffix using findChildForKey
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, I was also kind of surprised. I was a bit fixated on getting this working, that I didn't pause to think that I should add it there. I'll move most of this there.
// Verify accessor methods | ||
KeyValue retrievedKeyValue = dataInPath.getRawKeyValue(); | ||
assertNotNull(retrievedKeyValue); | ||
assertEquals(originalKeyValue.getKey(), retrievedKeyValue.getKey()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I believe that for arrays, the assertion here ends up asserting that the two arrays are pointer equal. Which I guess makes sense given that the original KeyValue
is just being wrapped. That may be what you inteded to assert here, but I just wanted to make sure that that's what was going on here.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
True, updated, but I'm probably going to make more substantial changes to the api here, namely hiding the key entirely.
KeySpacePath companyPath = root.path("company"); | ||
DataInKeySpacePath dataInPath = new DataInKeySpacePath(companyPath, keyValue, context); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It might be nice to have an assertion somewhere that states that you can choose any path along the way, and all of those will return the same DataInKeySpacePath
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, that was the intent of the parameter. Fixed.
/** | ||
* Class representing a {@link KeyValue} pair within in {@link KeySpacePath}. | ||
*/ | ||
public class DataInKeySpacePath { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should this implement equals
and hashCode
?
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, probably, but I'm not entirely sure how to do that with the CompletableFuture
for the resolved path.
It can either fall through and say it's equal if the original path, and raw value or equal, or, alternately, I can change it so that the constructor takes the resolved path.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I started doing this path, an discovered that ResolvedKeySpacePath.equals
is broken, and I've started working on a fix for that. Its a fair amount of new tests, and a couple other equals/hashcode implementations, so I'm going to pull that out into a separate PR, and come back to this once it is fixed.
Issue: #3594
* @param context the context in which to export | ||
* @return a list of the raw {@code KeyValue}s being exported | ||
*/ | ||
private static List<KeyValue> exportAllData(final KeySpacePath pathToExport, final FDBRecordContext context) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Any reason this returns KeyValue
s instead of DataInKeySpacePath
s?
try (FDBRecordContext context = database.openContext()) { | ||
KeySpacePath continuationPath = root.path("continuation"); | ||
|
||
final ScanProperties scanProperties = ScanProperties.FORWARD_SCAN.with(props -> props.setReturnedRowLimit(limit)); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Should we also test reverse scans like this?
|
||
@ParameterizedTest | ||
@ValueSource(ints = {1, 2, 3, 30}) | ||
void exportAllDataWithContinuation(int limit) { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Another case that is relevant to continuations: scanning a single record key space path. Something like:
- Resolving a path to a tuple then
- Setting that specific key to some value and
- Validating that we get that value back and that the continuation that comes back is sensible (not null and ideally not empty)
I'm not actually sure we'd get that key back (that is, whether the scan is over a strict subspace or not). And if it does return a continuation, I actually think it would return an empty continuation for the first element (because of #3206, see #3397), but it would be nice to know.
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I went through a bunch of the comments, and responded. I've made a bunch of the changes locally, but when I got to implementing DataInKeySpacePath.equals
/hashCode
, I discovered that ResolvedKeySpacePath
does not implement those correctly. I'm going to pause, the work on this PR, to resolve that. I'm submitting my commits in the meantime, for visibility.
@Nullable byte[] continuation, | ||
@Nonnull ScanProperties scanProperties) { | ||
return new LazyCursor<>(toTupleAsync(context) | ||
.thenApply(tuple -> KeyValueCursor.Builder.withSubspace(new Subspace(tuple)) |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
That's a good point.
I changed exportAllData
on the interface to be EXPERIMENTAL
, and added a note that we expect the continuation change without preserving backwards compatibility when that merges. If the rest of the pieces are in place such that this is useful before #3397 gets merged, we can potentially update the javadoc, and make this be backwards compatible.
/** | ||
* Class representing a {@link KeyValue} pair within in {@link KeySpacePath}. | ||
*/ | ||
public class DataInKeySpacePath { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, probably, but I'm not entirely sure how to do that with the CompletableFuture
for the resolved path.
It can either fall through and say it's equal if the original path, and raw value or equal, or, alternately, I can change it so that the constructor takes the resolved path.
this.rawKeyValue = rawKeyValue; | ||
|
||
// Convert the raw key to a Tuple and resolve it starting from the provided path | ||
Tuple keyTuple = Tuple.fromBytes(rawKeyValue.getKey()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
KeySpacePathImpl
always produces tuples, and AFAIK every key we have ever used has been in tuples. I feel like committing to that at this point, is worthwhile. If we find a strong reason to put non-tuple data in a key, we'll have to cross that bridge when we get to it.
*/ | ||
@API(API.Status.UNSTABLE) | ||
@Nonnull | ||
RecordCursor<DataInKeySpacePath> exportAllData(@Nonnull FDBRecordContext context, |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I realized that this is an interface, and while I don't think we intended anyone to extend it, I should probably put a default implementation in.
Tuple keyTuple = Tuple.fromBytes(rawKeyValue.getKey()); | ||
|
||
// First resolve the provided path to get its resolved form | ||
this.resolvedPath = path.toResolvedPathAsync(context).thenCompose(resolvedPath -> { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, I was also kind of surprised. I was a bit fixated on getting this working, that I didn't pause to think that I should add it there. I'll move most of this there.
// Verify accessor methods | ||
KeyValue retrievedKeyValue = dataInPath.getRawKeyValue(); | ||
assertNotNull(retrievedKeyValue); | ||
assertEquals(originalKeyValue.getKey(), retrievedKeyValue.getKey()); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
True, updated, but I'm probably going to make more substantial changes to the api here, namely hiding the key entirely.
assertEquals("blob_id", resolved.getDirectoryName()); | ||
byte[] resolvedBytes = (byte[]) resolved.getResolvedValue(); | ||
assertArrayEquals(blobId, resolvedBytes); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, but I updated assertNameAndValue
to check for byte[]
and use assertArrayEquals
appropriately.
@ValueSource(ints = {0, 1, 2, 3, 4, 5}) | ||
void resolution() { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
It is not, but it was supposed to be used to resolve from a path deeper than the root. Updated the test to actually use it.
KeySpacePath companyPath = root.path("company"); | ||
DataInKeySpacePath dataInPath = new DataInKeySpacePath(companyPath, keyValue, context); |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
Yeah, that was the intent of the parameter. Fixed.
/** | ||
* Class representing a {@link KeyValue} pair within in {@link KeySpacePath}. | ||
*/ | ||
public class DataInKeySpacePath { |
There was a problem hiding this comment.
Choose a reason for hiding this comment
The reason will be displayed to describe this comment to others. Learn more.
I started doing this path, an discovered that ResolvedKeySpacePath.equals
is broken, and I've started working on a fix for that. Its a fair amount of new tests, and a couple other equals/hashcode implementations, so I'm going to pull that out into a separate PR, and come back to this once it is fixed.
Issue: #3594
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