Skip to content
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -58,6 +58,7 @@ public enum LogMessageKeys {
TRANSACTION_ID,
TRANSACTION_NAME,
AGE_SECONDS,
AGE_MILLISECONDS,
CONSTITUENT,
TOTAL_MICROS,
// record splitting/unsplitting
Expand Down Expand Up @@ -162,7 +163,7 @@ public enum LogMessageKeys {
RECORDS_PER_SECOND,
DOCUMENT,
SESSION_ID,
INDEXER_SESSION_ID,
EXISTING_INDEXER_ID,
INDEXER_ID,
INDEX_STATE_PRECONDITION,
INITIAL_INDEX_STATE,
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -4965,7 +4965,9 @@ public void removeFormerIndex(FormerIndex formerIndex) {
}

private void clearReadableIndexBuildData(Index index) {
// Clear index maintenance data that is unneeded once the index becomes readable
IndexingRangeSet.forIndexBuild(this, index).clear();
IndexingHeartbeat.clearAllHeartbeats(this, index);
}

@SuppressWarnings("PMD.CloseResource")
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -413,6 +413,10 @@ public enum Waits implements Wait {
WAIT_INDEX_OPERATION("wait for index operation"),
/** Wait for indexing type stamp operation. */
WAIT_INDEX_TYPESTAMP_OPERATION("wait for indexing type stamp operation"),
/** Wait for clearing indexing heartbeats. */
WAIT_INDEX_CLEAR_HEARTBEATS("Wait for clearing indexing heartbeats"),
/** Wait for reading indexing heartbeats. */
WAIT_INDEX_READ_HEARTBEATS("Wait for reading indexing heartbeats"),
/** Wait for adding an index. */
WAIT_ADD_INDEX("wait for adding an index"),
/** Wait for dropping an index. */
Expand Down

Large diffs are not rendered by default.

Original file line number Diff line number Diff line change
Expand Up @@ -103,7 +103,7 @@ private Index getSourceIndex(RecordMetaData metaData) {
throw new ValidationException("no source index",
LogMessageKeys.INDEX_NAME, common.getIndex().getName(),
LogMessageKeys.SOURCE_INDEX, policy.getSourceIndex(),
LogMessageKeys.INDEXER_ID, common.getUuid());
LogMessageKeys.INDEXER_ID, common.getIndexerId());
}

@Nonnull
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,7 +27,6 @@
import com.apple.foundationdb.record.metadata.Index;
import com.apple.foundationdb.record.metadata.MetaDataException;
import com.apple.foundationdb.record.metadata.RecordType;
import com.apple.foundationdb.record.provider.foundationdb.synchronizedsession.SynchronizedSessionRunner;
import com.apple.foundationdb.record.query.plan.synthetic.SyntheticRecordPlanner;
import com.apple.foundationdb.tuple.Tuple;

Expand All @@ -50,10 +49,9 @@

@API(API.Status.INTERNAL)
public class IndexingCommon {
private final UUID uuid = UUID.randomUUID();
private final UUID indexerId = UUID.randomUUID();

@Nonnull private final FDBDatabaseRunner runner;
@Nullable private SynchronizedSessionRunner synchronizedSessionRunner = null;

@Nonnull private final FDBRecordStore.Builder recordStoreBuilder;
@Nonnull private final AtomicLong totalRecordsScanned;
Expand Down Expand Up @@ -137,8 +135,8 @@ private void fillTargetIndexers(@Nonnull List<Index> targetIndexes, @Nullable Co
}
}

public UUID getUuid() {
return uuid;
public UUID getIndexerId() {
return indexerId;
}

public List<Object> indexLogMessageKeyValues() {
Expand All @@ -158,7 +156,7 @@ public List<Object> indexLogMessageKeyValues(@Nullable String transactionName, @
logIf(true, keyValues,
LogMessageKeys.TARGET_INDEX_NAME, getTargetIndexesNames(),
LogMessageKeys.RECORDS_SCANNED, totalRecordsScanned.get(),
LogMessageKeys.INDEXER_ID, uuid);
LogMessageKeys.INDEXER_ID, indexerId);

if (moreKeyValues != null && !moreKeyValues.isEmpty()) {
keyValues.addAll(moreKeyValues);
Expand All @@ -176,11 +174,6 @@ private void logIf(boolean condition, List<Object> list, @Nonnull Object... a) {

@Nonnull
public FDBDatabaseRunner getRunner() {
return synchronizedSessionRunner == null ? runner : synchronizedSessionRunner;
}

@Nonnull
public FDBDatabaseRunner getNonSynchronizedRunner() {
return runner;
}

Expand Down Expand Up @@ -258,15 +251,6 @@ public FDBRecordStore.Builder getRecordStoreBuilder() {
return recordStoreBuilder;
}

@Nullable
public SynchronizedSessionRunner getSynchronizedSessionRunner() {
return synchronizedSessionRunner;
}

public void setSynchronizedSessionRunner(@Nullable final SynchronizedSessionRunner synchronizedSessionRunner) {
this.synchronizedSessionRunner = synchronizedSessionRunner;
}

@Nonnull
public AtomicLong getTotalRecordsScanned() {
return totalRecordsScanned;
Expand All @@ -287,8 +271,5 @@ public boolean loadConfig() {

public void close() {
runner.close();
if (synchronizedSessionRunner != null) {
synchronizedSessionRunner.close();
}
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,206 @@
/*
* IndexingHeartbeat.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;

import com.apple.foundationdb.KeyValue;
import com.apple.foundationdb.annotation.API;
import com.apple.foundationdb.async.AsyncIterator;
import com.apple.foundationdb.async.AsyncUtil;
import com.apple.foundationdb.record.IndexBuildProto;
import com.apple.foundationdb.record.logging.KeyValueLogMessage;
import com.apple.foundationdb.record.logging.LogMessageKeys;
import com.apple.foundationdb.record.metadata.Index;
import com.apple.foundationdb.synchronizedsession.SynchronizedSessionLockedException;
import com.google.protobuf.InvalidProtocolBufferException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import javax.annotation.Nonnull;
import java.util.HashMap;
import java.util.Map;
import java.util.UUID;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.atomic.AtomicInteger;

/**
* Indexing Shared Heartbeats can be used to define and handle "active" indexing processes.
* Every indexer should update its unique heartbeat during its indexing iteration. If the indexing session is optimized for
* non-mutual (as defined by the indexing type, see {@link IndexBuildProto.IndexBuildIndexingStamp}), detecting an existing
* active heartbeat will help preventing concurrent, conflicting, indexing attempts.
* In addition, the heartbeats can be used by users to query activity status of ongoing indexing sessions.
*/
@API(API.Status.INTERNAL)
public class IndexingHeartbeat {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This seems like a bit of the code could be more generic. Particularly, I see that the indexingMethod in the heartbeat is not read by anything, indicating that is more for logging purposes, and could be replaced by a string.
If you pull out the logic for what the limit should be you could have this completely independent of indexing, and provide a raw subspace to store the information. Doing so would make some testing scenarios easier, and allow for re-use in different scenarios.
It would also allow for testing some of this logic with ThrottledRetryingIterator, and eventually could allow for a LimitedConcurrencyThrottledIterator which would be analogous to SynchronizedSession. That wouldn't be immediately relevant, but would allow for more controlled testing of some of the behaviors as you would be able to have better control over when individual futures complete, and run more interesting code in the transactions (such as fetching the list of existing heartbeats yourself).

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Going in a completely different direction, would it make sense to combine the heartbeats more with the type stamp?

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The indexingMethod is for:
a) Debugging (if we'll ever find a leftover heartbeat).
b) Providing extra information when heartbeats are queried. Hopefully that will enable better decision making in any automated system that monitors indexing processes (decisions such as: adding a mutual indexing thread, converting to mutual, stopping a slow by-source-index session and restart indexing, etc.)

I am not sure that I understand your LimitedConcurrencyThrottledIterator idea.

The type stamp is complementary to the heartbeat. Initially I thought about making a copy of the type stamp to each heartbeat, but it seems easier - for integrity reasons - to keep the type stamp as is.

Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

We discussed offline, and decided to take a balanced approach of changing the method to info, so that this is, on disk, future-proofed for being upgraded to be more generic if we want to use it elsewhere. There would still need to be some substantial changes to the code, but it would need to take a subspace instead of a store and an index.

// [prefix, indexerId] -> [indexing-type, genesis time, heartbeat time]
@Nonnull
private static final Logger logger = LoggerFactory.getLogger(IndexingHeartbeat.class);
public static final String INVALID_HEARTBEAT_INFO = "<< Invalid Heartbeat >>";

final UUID indexerId;
final String info;
final long genesisTimeMilliseconds;
final long leaseLength;
final boolean allowMutual;

public IndexingHeartbeat(final UUID indexerId, String info, long leaseLength, boolean allowMutual) {
this.indexerId = indexerId;
this.info = info;
this.leaseLength = leaseLength;
this.allowMutual = allowMutual;
this.genesisTimeMilliseconds = nowMilliseconds();
}

public void updateHeartbeat(@Nonnull FDBRecordStore store, @Nonnull Index index) {
byte[] key = IndexingSubspaces.indexHeartbeatSubspace(store, index, indexerId).pack();
byte[] value = IndexBuildProto.IndexBuildHeartbeat.newBuilder()
.setInfo(info)
.setGenesisTimeMilliseconds(genesisTimeMilliseconds)
.setHeartbeatTimeMilliseconds(nowMilliseconds())
.build().toByteArray();
store.ensureContextActive().set(key, value);
}

public CompletableFuture<Void> checkAndUpdateHeartbeat(@Nonnull FDBRecordStore store, @Nonnull Index index) {
// complete exceptionally if non-mutual, other exists
if (allowMutual) {
updateHeartbeat(store, index);
return AsyncUtil.DONE;
}

final AsyncIterator<KeyValue> iterator = heartbeatsIterator(store, index);
final long now = nowMilliseconds();
return AsyncUtil.whileTrue(() -> iterator.onHasNext()
.thenApply(hasNext -> {
if (!hasNext) {
return false;
}
final KeyValue kv = iterator.next();
try {
final UUID otherIndexerId = heartbeatKeyToIndexerId(store, index, kv.getKey());
if (!otherIndexerId.equals(this.indexerId)) {
final IndexBuildProto.IndexBuildHeartbeat otherHeartbeat = IndexBuildProto.IndexBuildHeartbeat.parseFrom(kv.getValue());
final long age = now - otherHeartbeat.getHeartbeatTimeMilliseconds();
if (age > 0 && age < leaseLength) {
// For practical reasons, this exception is backward compatible to the Synchronized Lock one
throw new SynchronizedSessionLockedException("Failed to initialize the session because of an existing session in progress")
.addLogInfo(LogMessageKeys.INDEXER_ID, indexerId)
.addLogInfo(LogMessageKeys.EXISTING_INDEXER_ID, otherIndexerId)
.addLogInfo(LogMessageKeys.AGE_MILLISECONDS, age)
.addLogInfo(LogMessageKeys.TIME_LIMIT_MILLIS, leaseLength);

Check warning on line 107 in fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java

View check run for this annotation

fdb.teamscale.io / Teamscale | Findings

fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java#L103-L107

This method is slightly nested [0]. Consider extracting helper methods or reducing the nesting by using early breaks or returns. [0] https://fdb.teamscale.io/findings/details/foundationdb-fdb-record-layer?t=FORK_MR%2F3530%2Fjjezra%2Findexing_heartbeat%3AHEAD&id=2B4C6A20A0074C20402C955A907ADCBC
}
}
} catch (InvalidProtocolBufferException e) {
if (logger.isWarnEnabled()) {
logger.warn(KeyValueLogMessage.of("Bad indexing heartbeat item",
LogMessageKeys.KEY, kv.getKey(),
LogMessageKeys.VALUE, kv.getValue()));
}
}
return true;
}))
.thenApply(ignore -> {
updateHeartbeat(store, index);
return null;
});

Check warning on line 122 in fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java

View check run for this annotation

fdb.teamscale.io / Teamscale | Findings

fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java#L119-L122

Method always returns the same value (null) https://fdb.teamscale.io/findings/details/foundationdb-fdb-record-layer?t=FORK_MR%2F3530%2Fjjezra%2Findexing_heartbeat%3AHEAD&id=659A9600736B3B6D24326DC7135B3F76
}

public void clearHeartbeat(@Nonnull FDBRecordStore store, @Nonnull Index index) {
store.ensureContextActive().clear(IndexingSubspaces.indexHeartbeatSubspace(store, index, indexerId).pack());
}

public static void clearAllHeartbeats(@Nonnull FDBRecordStore store, @Nonnull Index index) {
store.ensureContextActive().clear(IndexingSubspaces.indexHeartbeatSubspace(store, index).range());
}

public static CompletableFuture<Map<UUID, IndexBuildProto.IndexBuildHeartbeat>> getIndexingHeartbeats(FDBRecordStore store, Index index, int maxCount) {
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

You could remove the need for this having a maxCount argument, by returning a RecordCursor<Pair<UUID, IndexBuildProto.IndexBuildHeartbeat>>.
You could allow it to take a continuation, if you wanted, but I doubt that's necessary; if there are too many heartbeats to scan transactionally, you almost certainly have some that need clearing.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Wouldn't that be over engineering? The maxCount is not intended to be used but in the case of something going very wrong. For now, the only place it can be a non-zero is if triggered by an external API call.

final Map<UUID, IndexBuildProto.IndexBuildHeartbeat> ret = new HashMap<>();
final AsyncIterator<KeyValue> iterator = heartbeatsIterator(store, index);
final AtomicInteger iterationCount = new AtomicInteger(0);
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ignoring my comment above about using a RecordCursor (which I think would be better), there is MoreAsyncUtil.limitIterable, although that method probably deserves some tests.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

What advantage does MoreAsyncUtil.limitIterable have over the iterationCount?

return AsyncUtil.whileTrue(() -> iterator.onHasNext()
.thenApply(hasNext -> {
if (!hasNext) {
return false;
}
if (maxCount > 0 && maxCount < iterationCount.incrementAndGet()) {
return false;
}
final KeyValue kv = iterator.next();
final UUID otherIndexerId = heartbeatKeyToIndexerId(store, index, kv.getKey());
try {
final IndexBuildProto.IndexBuildHeartbeat otherHeartbeat = IndexBuildProto.IndexBuildHeartbeat.parseFrom(kv.getValue());
ret.put(otherIndexerId, otherHeartbeat);
} catch (InvalidProtocolBufferException e) {
// Let the caller know about this invalid heartbeat.
ret.put(otherIndexerId, IndexBuildProto.IndexBuildHeartbeat.newBuilder()
.setInfo(INVALID_HEARTBEAT_INFO)
.setGenesisTimeMilliseconds(0)
.setHeartbeatTimeMilliseconds(0)
.build());
}
return true;
}))
.thenApply(ignore -> ret);
}

public static CompletableFuture<Integer> clearIndexingHeartbeats(@Nonnull FDBRecordStore store, @Nonnull Index index, long minAgeMilliseconds, int maxIteration) {
final AsyncIterator<KeyValue> iterator = heartbeatsIterator(store, index);
final AtomicInteger deleteCount = new AtomicInteger(0);
final AtomicInteger iterationCount = new AtomicInteger(0);
final long now = nowMilliseconds();
return AsyncUtil.whileTrue(() -> iterator.onHasNext()
.thenApply(hasNext -> {
if (!hasNext) {
return false;
}
if (maxIteration > 0 && maxIteration < iterationCount.incrementAndGet()) {
return false;
}
final KeyValue kv = iterator.next();
boolean shouldRemove;
try {
final IndexBuildProto.IndexBuildHeartbeat otherHeartbeat = IndexBuildProto.IndexBuildHeartbeat.parseFrom(kv.getValue());
// remove heartbeat if too old
shouldRemove = now >= otherHeartbeat.getHeartbeatTimeMilliseconds() + minAgeMilliseconds;
} catch (InvalidProtocolBufferException e) {
// remove heartbeat if invalid
shouldRemove = true;
Comment on lines +183 to +184
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Does this introduce a risk if we're running in an environment with future versions of the record layer?
I think if proper protobuf evolution is followed, it should be fine, but I'm not sure.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

The function should only be triggered by the user, and only if something went wrong while clearing the heartbeat (session end, or whenever the index because readable). If a protobuf upgrade will make it inconsistent (which is unthinkable while you are still around:) ) it is much more likely that a new code be called to clear old unusable heartbeats from a previous version.

}
if (shouldRemove) {
store.ensureContextActive().clear(kv.getKey());
deleteCount.incrementAndGet();
}
return true;
}))
.thenApply(ignore -> deleteCount.get());
}

private static AsyncIterator<KeyValue> heartbeatsIterator(FDBRecordStore store, Index index) {
return store.getContext().ensureActive().getRange(IndexingSubspaces.indexHeartbeatSubspace(store, index).range()).iterator();
}

private static UUID heartbeatKeyToIndexerId(FDBRecordStore store, Index index, byte[] key) {
return IndexingSubspaces.indexHeartbeatSubspace(store, index).unpack(key).getUUID(0);
}

private static long nowMilliseconds() {
return System.currentTimeMillis();
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -77,7 +77,7 @@ CompletableFuture<Void> mergeIndex(@Nullable SubspaceProvider subspaceProvider)
// Merge operation may take a long time, hence the runner's context must be a read-only. Ensure that it
// isn't a synchronized one, which may attempt a heartbeat write
// Note: this runAsync will retry according to the runner's "maxAttempts" setting
common.getNonSynchronizedRunner().runAsync(context -> openRecordStore(context)
common.getRunner().runAsync(context -> openRecordStore(context)
.thenCompose(store -> {
mergeStartTime.set(System.nanoTime());
final IndexDeferredMaintenanceControl mergeControl = store.getIndexDeferredMaintenanceControl();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -27,6 +27,7 @@
import com.apple.foundationdb.tuple.Tuple;

import javax.annotation.Nonnull;
import java.util.UUID;

/**
* List of subspaces related to the indexing/index-scrubbing processes.
Expand All @@ -40,6 +41,7 @@ public final class IndexingSubspaces {
private static final Object INDEX_SCRUBBED_RECORDS_RANGES_ZERO = 4L;
private static final Object INDEX_SCRUBBED_RECORDS_RANGES = 5L;
private static final Object INDEX_SCRUBBED_INDEX_RANGES = 6L;
private static final Object INDEX_BUILD_HEARTBEAT_PREFIX = 7L;
Copy link
Collaborator

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

FDBRecordStore.checkFirstKeyIsHeader needs to be updated, probably also needed for some of the other ones that already existed.
That method should probably call into a method in this class, although that doesn't need to be done in this PR.


private IndexingSubspaces() {
throw new IllegalStateException("Utility class");
Expand Down Expand Up @@ -83,6 +85,29 @@ public static Subspace indexBuildTypeSubspace(@Nonnull FDBRecordStoreBase<?> sto
return indexBuildSubspace(store, index, INDEX_BUILD_TYPE_VERSION);
}

/**
* Subspace that stores the indexing heartbeat.
* @param store store
* @param index index
* @return subspace
*/
@Nonnull
public static Subspace indexHeartbeatSubspace(@Nonnull FDBRecordStoreBase<?> store, @Nonnull Index index) {
return indexBuildSubspace(store, index, INDEX_BUILD_HEARTBEAT_PREFIX);
}

/**
* Subspace that stores the indexing heartbeat.
* @param store store
* @param index index
* @param indexerId session id
* @return subspace
*/
@Nonnull
public static Subspace indexHeartbeatSubspace(@Nonnull FDBRecordStoreBase<?> store, @Nonnull Index index, @Nonnull UUID indexerId) {
return indexHeartbeatSubspace(store, index).subspace(Tuple.from(indexerId));
}

/**
* Subspace that stores scrubbed records ranges of the zero range-id. This subspace is backward compatible
* to record ranges scrubbed before range-id was introduced.
Expand Down Expand Up @@ -184,5 +209,7 @@ public static void eraseAllIndexingDataButTheLock(@Nonnull FDBRecordContext cont
eraseAllIndexingScrubbingData(context, store, index);
context.clear(Range.startsWith(indexBuildScannedRecordsSubspace(store, index).pack()));
context.clear(Range.startsWith(indexBuildTypeSubspace(store, index).pack()));
// The heartbeats, unlike the sync lock, may be erased here. If needed, an appropriate heartbeat will be set after this clear & within the same transaction.
context.clear(Range.startsWith(indexHeartbeatSubspace(store, index).pack()));
}
}
Loading