Skip to content

Commit dcb0f30

Browse files
Add RecordCountKey State (#3334)
This adds a mechanism for controlling the state of the _deprecated_ RecordCountKey, that can be set on the RecordMetaData. Note: This is different from `COUNT` indexes. Resolves #3326 As noted in the javadocs: The new state is controlled by `FDBRecordStore.updateRecordCountStateAsync`, and allows: 1. Changing a READABLE RecordCount to WRITE_ONLY 2. Changing a WRITE_ONLY RecordCount back to READABLE 3. Changing any state to DISABLED The default state is READABLE, and it will stay READABLE unless explicitly updated. If the RecordCount is WRITE_ONLY then it will maintain the count, but not allow querying it. If the RecordCount is DISABLED it will clear the data, and stop maintaining the values. In order to support this new state, a new `FormatVersion` has been added: `RECORD_COUNT_STATE` In order to use the new functionality, the stores must be built with at least this format version. --------- Co-authored-by: Alec Grieser <[email protected]>
1 parent 964600c commit dcb0f30

File tree

8 files changed

+604
-61
lines changed

8 files changed

+604
-61
lines changed

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

Lines changed: 100 additions & 20 deletions
Original file line numberDiff line numberDiff line change
@@ -559,13 +559,21 @@ protected <M extends Message> CompletableFuture<FDBStoredRecord<M>> saveTypedRec
559559

560560
@SuppressWarnings("PMD.CloseResource")
561561
private <M extends Message> void addRecordCount(@Nonnull RecordMetaData metaData, @Nonnull FDBStoredRecord<M> rec, @Nonnull byte[] increment) {
562-
if (metaData.getRecordCountKey() == null) {
563-
return;
562+
if (metaData.getRecordCountKey() != null) {
563+
beginRecordStoreStateRead();
564+
try {
565+
RecordMetaDataProto.DataStoreInfo header = recordStoreStateRef.get().getStoreHeader();
566+
// We do not need to check the format version here. In order for it to be DISABLED we would have to be
567+
// on a format version that supports such a state.
568+
if (header.getRecordCountState() != RecordMetaDataProto.DataStoreInfo.RecordCountState.DISABLED) {
569+
Key.Evaluated subkey = metaData.getRecordCountKey().evaluateSingleton(rec);
570+
final byte[] keyBytes = getSubspace().pack(Tuple.from(RECORD_COUNT_KEY).addAll(subkey.toTupleAppropriateList()));
571+
ensureContextActive().mutate(MutationType.ADD, keyBytes, increment);
572+
}
573+
} finally {
574+
endRecordStoreStateRead();
575+
}
564576
}
565-
final Transaction tr = ensureContextActive();
566-
Key.Evaluated subkey = metaData.getRecordCountKey().evaluateSingleton(rec);
567-
final byte[] keyBytes = getSubspace().pack(Tuple.from(RECORD_COUNT_KEY).addAll(subkey.toTupleAppropriateList()));
568-
tr.mutate(MutationType.ADD, keyBytes, increment);
569577
}
570578

571579
@Nullable
@@ -1851,7 +1859,9 @@ private Key.Evaluated deleteRecordsWhereCheckRecordTypes() {
18511859
}
18521860

18531861
final KeyExpression recordCountKey = getRecordMetaData().getRecordCountKey();
1854-
if (recordCountKey != null) {
1862+
if (recordCountKey != null
1863+
// we don't need to call beginRecordStoreStateRead(), that is checked in deleteRecordsWhereAsync
1864+
&& recordStoreStateRef.get().getStoreHeader().getRecordCountState() != RecordMetaDataProto.DataStoreInfo.RecordCountState.DISABLED) {
18551865
final QueryToKeyMatcher.Match match = matcher.matchesSatisfyingQuery(recordCountKey);
18561866
if (match.getType() != QueryToKeyMatcher.MatchType.EQUALITY) {
18571867
throw new Query.InvalidExpressionException("Record count key not matching for deleteRecordsWhere");
@@ -2077,8 +2087,11 @@ private CompletableFuture<Void> run() {
20772087
context.clear(versionRange);
20782088
}
20792089

2090+
20802091
final KeyExpression recordCountKey = getRecordMetaData().getRecordCountKey();
2081-
if (recordCountKey != null) {
2092+
if (recordCountKey != null
2093+
// we don't need to call beginRecordStoreStateRead(), that is checked in deleteRecordsWhereAsync
2094+
&& recordStoreStateRef.get().getStoreHeader().getRecordCountState() != RecordMetaDataProto.DataStoreInfo.RecordCountState.DISABLED) {
20822095
if (prefix.size() == recordCountKey.getColumnSize()) {
20832096
// Delete a single record used for counting
20842097
context.clear(getSubspace().pack(Tuple.from(RECORD_COUNT_KEY).addAll(prefix)));
@@ -2153,17 +2166,39 @@ private CompletableFuture<Long> estimateSize(@Nonnull Range range, long startTim
21532166
@Override
21542167
public CompletableFuture<Long> getSnapshotRecordCount(@Nonnull KeyExpression key, @Nonnull Key.Evaluated value,
21552168
@Nonnull IndexQueryabilityFilter indexQueryabilityFilter) {
2156-
if (getRecordMetaData().getRecordCountKey() != null) {
2157-
if (key.getColumnSize() != value.size()) {
2158-
throw recordCoreException("key and value are not the same size");
2159-
}
2160-
final ReadTransaction tr = context.readTransaction(true);
2161-
final Tuple subkey = Tuple.from(RECORD_COUNT_KEY).addAll(value.toTupleAppropriateList());
2162-
if (getRecordMetaData().getRecordCountKey().equals(key)) {
2163-
return tr.get(getSubspace().pack(subkey)).thenApply(FDBRecordStore::decodeRecordCount);
2164-
} else if (key.isPrefixKey(getRecordMetaData().getRecordCountKey())) {
2165-
AsyncIterable<KeyValue> kvs = tr.getRange(getSubspace().range(Tuple.from(RECORD_COUNT_KEY)));
2166-
return MoreAsyncUtil.reduce(getExecutor(), kvs.iterator(), 0L, (count, kv) -> count + decodeRecordCount(kv.getValue()));
2169+
final RecordMetaData recordMetaData = getRecordMetaData();
2170+
if (recordMetaData.getRecordCountKey() != null) {
2171+
beginRecordStoreStateRead();
2172+
boolean futureCreated = false;
2173+
try {
2174+
RecordMetaDataProto.DataStoreInfo header = recordStoreStateRef.get().getStoreHeader();
2175+
// We can always check the state, even if the formatVersion is older, because older versions will always
2176+
// have the default of READABLE
2177+
if (header.getRecordCountState() == RecordMetaDataProto.DataStoreInfo.RecordCountState.READABLE) {
2178+
if (key.getColumnSize() != value.size()) {
2179+
throw recordCoreException("key and value are not the same size");
2180+
}
2181+
final ReadTransaction tr = context.readTransaction(true);
2182+
final Tuple subkey = Tuple.from(RECORD_COUNT_KEY).addAll(value.toTupleAppropriateList());
2183+
if (recordMetaData.getRecordCountKey().equals(key)) {
2184+
final CompletableFuture<Long> result = tr.get(getSubspace().pack(subkey))
2185+
.thenApply(FDBRecordStore::decodeRecordCount)
2186+
.whenComplete((ignored, error) -> endRecordStoreStateRead());
2187+
futureCreated = true;
2188+
return result;
2189+
} else if (key.isPrefixKey(recordMetaData.getRecordCountKey())) {
2190+
AsyncIterable<KeyValue> kvs = tr.getRange(getSubspace().range(Tuple.from(RECORD_COUNT_KEY)));
2191+
final CompletableFuture<Long> result = MoreAsyncUtil.reduce(getExecutor(), kvs.iterator(), 0L,
2192+
(count, kv) -> count + decodeRecordCount(kv.getValue()))
2193+
.whenComplete((ignored, error) -> endRecordStoreStateRead());
2194+
futureCreated = true;
2195+
return result;
2196+
}
2197+
}
2198+
} finally {
2199+
if (!futureCreated) {
2200+
endRecordStoreStateRead();
2201+
}
21672202
}
21682203
}
21692204
return evaluateAggregateFunction(Collections.emptyList(), IndexFunctionHelper.count(key),
@@ -3237,6 +3272,48 @@ public void clearHeaderUserField(@Nonnull String userField) {
32373272
context.asyncToSync(FDBStoreTimer.Waits.WAIT_EDIT_HEADER_USER_FIELD, clearHeaderUserFieldAsync(userField));
32383273
}
32393274

3275+
/**
3276+
* Update the RecordCount to have a new state.
3277+
* <p>
3278+
* The state can go from {@code READABLE} to {@code WRITE_ONLY}, which is mostly useful to validate that if you
3279+
* drop the {@link RecordMetaData#getRecordCountKey() recordCountKey} from the metadata, it won't have adverse
3280+
* affects.
3281+
* The state can always be set to {@code DISABLED}, at which point there is no way to any other state. This
3282+
* limitation exists, in large part, because there is not a way to rebuild the count across indexes, and since
3283+
* it is long deprecated, investing in that doesn't make sense.
3284+
* </p>
3285+
* @param newState the new state to update it to.
3286+
* @return a future once that completes once the state is updated, and the data is cleared.
3287+
*/
3288+
public CompletableFuture<Void> updateRecordCountStateAsync(@Nonnull RecordMetaDataProto.DataStoreInfo.RecordCountState newState) {
3289+
return updateStoreHeaderAsync(builder -> {
3290+
if (!getFormatVersionEnum().isAtLeast(FormatVersion.RECORD_COUNT_STATE)) {
3291+
throw new RecordCoreException("Store does not support updating record count state")
3292+
.addLogInfo(LogMessageKeys.FORMAT_VERSION, getFormatVersionEnum());
3293+
}
3294+
final RecordMetaDataProto.DataStoreInfo.RecordCountState existing = getRecordStoreState().getStoreHeader().getRecordCountState();
3295+
if (existing == newState) {
3296+
return builder;
3297+
}
3298+
boolean toWriteOnly = existing == RecordMetaDataProto.DataStoreInfo.RecordCountState.READABLE &&
3299+
newState == RecordMetaDataProto.DataStoreInfo.RecordCountState.WRITE_ONLY;
3300+
boolean toReadable = existing == RecordMetaDataProto.DataStoreInfo.RecordCountState.WRITE_ONLY &&
3301+
newState == RecordMetaDataProto.DataStoreInfo.RecordCountState.READABLE;
3302+
if (toWriteOnly || toReadable || newState == RecordMetaDataProto.DataStoreInfo.RecordCountState.DISABLED) {
3303+
builder.setRecordCountState(newState);
3304+
if (newState == RecordMetaDataProto.DataStoreInfo.RecordCountState.DISABLED) {
3305+
// Note: getSubspace().range(tuple) does not include getSubspace().pack(tuple), but this does.
3306+
// Ungrouped recordCountKey will be stored directly at getSubspace().pack(tuple).
3307+
ensureContextActive().clear(Range.startsWith(getSubspace().pack(Tuple.from(RECORD_COUNT_KEY))));
3308+
}
3309+
return builder;
3310+
}
3311+
throw new RecordCoreException("Invalid state transition for RecordCountState")
3312+
.addLogInfo(LogMessageKeys.OLD, existing)
3313+
.addLogInfo(LogMessageKeys.NEW, newState);
3314+
});
3315+
}
3316+
32403317
// Actually (1) writes the index state to the database and (2) updates the cached state with the new state
32413318
@SuppressWarnings("PMD.CloseResource")
32423319
private void updateIndexState(@Nonnull String indexName, byte[] indexKey, @Nonnull IndexState indexState) {
@@ -4841,6 +4918,8 @@ protected boolean checkPossiblyRebuildRecordCounts(@Nonnull RecordMetaData metaD
48414918

48424919
if (rebuildRecordCounts) {
48434920
// We want to clear all record counts.
4921+
// This code will leave data behind if the previous RecordCountKey was not grouped
4922+
// https://github.com/FoundationDB/fdb-record-layer/issues/3335
48444923
if (existingStore) {
48454924
context.clear(getSubspace().range(Tuple.from(RECORD_COUNT_KEY)));
48464925
}
@@ -4865,7 +4944,8 @@ protected boolean checkPossiblyRebuildRecordCounts(@Nonnull RecordMetaData metaD
48654944
@SuppressWarnings("PMD.CloseResource")
48664945
public void addRebuildRecordCountsJob(List<CompletableFuture<Void>> work) {
48674946
final KeyExpression recordCountKey = getRecordMetaData().getRecordCountKey();
4868-
if (recordCountKey == null) {
4947+
if (recordCountKey == null ||
4948+
getRecordStoreState().getStoreHeader().getRecordCountState() == RecordMetaDataProto.DataStoreInfo.RecordCountState.DISABLED) {
48694949
return;
48704950
}
48714951
if (LOGGER.isDebugEnabled()) {

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

Lines changed: 8 additions & 2 deletions
Original file line numberDiff line numberDiff line change
@@ -21,6 +21,7 @@
2121
package com.apple.foundationdb.record.provider.foundationdb;
2222

2323
import com.apple.foundationdb.annotation.API;
24+
import com.apple.foundationdb.record.RecordMetaDataProto;
2425

2526
import javax.annotation.Nonnull;
2627
import java.util.Arrays;
@@ -53,7 +54,7 @@ public enum FormatVersion implements Comparable<FormatVersion> {
5354
*/
5455
INFO_ADDED(1),
5556
/**
56-
* This FormatVersion introduces support for tracking record conuts as defined by:
57+
* This FormatVersion introduces support for tracking record counts as defined by:
5758
* {@link com.apple.foundationdb.record.RecordMetaData#getRecordCountKey()}.
5859
*/
5960
RECORD_COUNT_ADDED(2),
@@ -145,7 +146,12 @@ public enum FormatVersion implements Comparable<FormatVersion> {
145146
/**
146147
* This FormatVersion allows building non-idempotent indexes (e.g. COUNT) from a source index.
147148
*/
148-
CHECK_INDEX_BUILD_TYPE_DURING_UPDATE(10);
149+
CHECK_INDEX_BUILD_TYPE_DURING_UPDATE(10),
150+
/**
151+
* This FormatVersion allows setting a state for the RecordCountKey on an individual store.
152+
* @see RecordMetaDataProto.DataStoreInfo#getRecordCountState()
153+
*/
154+
RECORD_COUNT_STATE(11);
149155

150156
private final int value;
151157

fdb-record-layer-core/src/main/proto/record_metadata.proto

Lines changed: 21 additions & 0 deletions
Original file line numberDiff line numberDiff line change
@@ -95,6 +95,27 @@ message DataStoreInfo {
9595
// When considering using this, one should also consider introducing a new record type to store the data.
9696
// This was introduced with FormatVersion.HEADER_USER_FIELDS.
9797
repeated UserFieldEntry user_field = 8;
98+
99+
// The state of the RecordCount as defined by record_count_key. This is similar to IndexState, but since there is no
100+
// mechanism for building the RecordCount across multiple transactions, the state transitions are slightly different.
101+
enum RecordCountState {
102+
// The RecordCountKey is maintained and usable for FDBRecordStore.getSnapshotRecordCount
103+
READABLE = 1;
104+
// The RecordCount as defined by record_count_key is maintained, but not queryable
105+
// This can be freely updated back to READABLE
106+
WRITE_ONLY = 2;
107+
// The RecordCount as defined by record_count_key is not maintained.
108+
// Once disabled, there is no way to go back to WriteOnly
109+
DISABLED = 3;
110+
}
111+
112+
// Whether the RecordCount as defined by record_count_key is usable by FDBRecordStore.getSnapshotRecordCount and
113+
// whether it is maintained at all.
114+
// This helps as a mechanism to de-risk removing the deprecated RecordCountKey from metadata by disabling it for
115+
// queries first, and then disabling entirely, but only for specific stores. Once validated that the necessary COUNT
116+
// indexes are working sufficiently, the RecordCountKey can be removed from the metadata more safely.
117+
// This was introduced with FormatVersion.RECORD_COUNT_STATE
118+
optional RecordCountState record_count_state = 9;
98119
}
99120

100121
message Index {

0 commit comments

Comments
 (0)