From 46b391ee059dfbaafb8bd8397ae180f177ce695c Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Mon, 4 Aug 2025 11:27:34 -0400 Subject: [PATCH 01/26] Online Indexer: replace the synchronized runner with a heartbeat Each indexing session, for each index, will create a key-value heartbeat of the format: [prefix, xid] -> [indexing-type, genesis time, heartbeat time] Indexing session that are expected to be exclusive will throw an exception if another, active, session exists. Motivation: 1. Represent the heartbeat in every index during multi target indexing (currently - only the master index has a sync lock) 2. Keep a heartbeat during mutual indexing, which can allow better automatic decision making 3. Decide about exclusiveness according to the indexing method (currently - user input) Resolve #3529 --- .../provider/foundationdb/IndexingBase.java | 144 ++++++------------ .../provider/foundationdb/IndexingCommon.java | 20 +-- .../foundationdb/IndexingHeartbeat.java | 117 ++++++++++++++ .../provider/foundationdb/IndexingMerger.java | 2 +- .../foundationdb/IndexingSubspaces.java | 26 ++++ .../provider/foundationdb/OnlineIndexer.java | 2 +- .../src/main/proto/index_build.proto | 7 + 7 files changed, 199 insertions(+), 119 deletions(-) create mode 100644 fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java index 4371c1d230..e8a900b4d1 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java @@ -45,13 +45,10 @@ import com.apple.foundationdb.record.provider.common.StoreTimer; import com.apple.foundationdb.record.provider.common.StoreTimerSnapshot; import com.apple.foundationdb.record.provider.foundationdb.indexing.IndexingRangeSet; -import com.apple.foundationdb.record.provider.foundationdb.synchronizedsession.SynchronizedSessionRunner; import com.apple.foundationdb.record.query.plan.RecordQueryPlanner; import com.apple.foundationdb.record.query.plan.synthetic.SyntheticRecordFromStoredRecordPlan; import com.apple.foundationdb.record.query.plan.synthetic.SyntheticRecordPlanner; import com.apple.foundationdb.subspace.Subspace; -import com.apple.foundationdb.synchronizedsession.SynchronizedSession; -import com.apple.foundationdb.synchronizedsession.SynchronizedSessionLockedException; import com.apple.foundationdb.tuple.ByteArrayUtil2; import com.apple.foundationdb.tuple.Tuple; import com.google.protobuf.Message; @@ -81,7 +78,6 @@ import java.util.concurrent.atomic.AtomicReference; import java.util.function.BiFunction; import java.util.function.Function; -import java.util.function.Supplier; import java.util.stream.Collectors; /** @@ -106,6 +102,7 @@ public abstract class IndexingBase { private final long startingTimeMillis; private long lastTypeStampCheckMillis; private Map indexingMergerMap = null; + private IndexingHeartbeat heartbeat = null; // this will stay null for index scrubbing IndexingBase(@Nonnull IndexingCommon common, @Nonnull OnlineIndexer.IndexingPolicy policy) { @@ -157,28 +154,13 @@ public CompletableFuture buildIndexAsync(boolean markReadable, boolean use KeyValueLogMessage message = KeyValueLogMessage.build("build index online", LogMessageKeys.SHOULD_MARK_READABLE, markReadable); long startNanos = System.nanoTime(); - final CompletableFuture buildIndexAsyncFuture; FDBDatabaseRunner runner = getRunner(); - Index index = common.getPrimaryIndex(); - if (runner.getTimer() != null) { - lastProgressSnapshot = StoreTimerSnapshot.from(runner.getTimer()); + final FDBStoreTimer timer = runner.getTimer(); + if ( timer != null) { + lastProgressSnapshot = StoreTimerSnapshot.from(timer); } - if (useSyncLock) { - buildIndexAsyncFuture = runner - .runAsync(context -> openRecordStore(context).thenApply(store -> IndexingSubspaces.indexBuildLockSubspace(store, index)), - common.indexLogMessageKeyValues("IndexingBase::indexBuildLockSubspace")) - .thenCompose(lockSubspace -> runner.startSynchronizedSessionAsync(lockSubspace, common.config.getLeaseLengthMillis())) - .thenCompose(synchronizedRunner -> { - message.addKeyAndValue(LogMessageKeys.SESSION_ID, synchronizedRunner.getSessionId()); - return runWithSynchronizedRunnerAndEndSession(synchronizedRunner, - () -> handleStateAndDoBuildIndexAsync(markReadable, message)); - }); - } else { - message.addKeyAndValue(LogMessageKeys.SESSION_ID, "none"); - common.setSynchronizedSessionRunner(null); - buildIndexAsyncFuture = handleStateAndDoBuildIndexAsync(markReadable, message); - } - return buildIndexAsyncFuture.whenComplete((vignore, ex) -> { + message.addKeyAndValue(LogMessageKeys.SESSION_ID, common.getUuid()); + return handleStateAndDoBuildIndexAsync(markReadable, message).whenComplete((vignore, ex) -> { message.addKeysAndValues(indexingLogMessageKeyValues()) // add these here to pick up state accumulated during build .addKeysAndValues(common.indexLogMessageKeyValues()) .addKeyAndValue(LogMessageKeys.TOTAL_MICROS, TimeUnit.NANOSECONDS.toMicros(System.nanoTime() - startNanos)); @@ -193,36 +175,6 @@ public CompletableFuture buildIndexAsync(boolean markReadable, boolean use }); } - @SuppressWarnings("PMD.CloseResource") - private CompletableFuture runWithSynchronizedRunnerAndEndSession( - @Nonnull SynchronizedSessionRunner newSynchronizedRunner, @Nonnull Supplier> runnable) { - final SynchronizedSessionRunner currentSynchronizedRunner1 = common.getSynchronizedSessionRunner(); - if (currentSynchronizedRunner1 == null) { - common.setSynchronizedSessionRunner(newSynchronizedRunner); - return MoreAsyncUtil.composeWhenComplete(runnable.get(), (result, ex) -> { - final SynchronizedSessionRunner currentSynchronizedRunner2 = common.getSynchronizedSessionRunner(); - if (newSynchronizedRunner.equals(currentSynchronizedRunner2)) { - common.setSynchronizedSessionRunner(null); - } else { - if (LOGGER.isWarnEnabled()) { - LOGGER.warn(KeyValueLogMessage.build("synchronizedSessionRunner was modified during the run", - LogMessageKeys.SESSION_ID, newSynchronizedRunner.getSessionId(), - LogMessageKeys.INDEXER_SESSION_ID, currentSynchronizedRunner2 == null ? null : currentSynchronizedRunner2.getSessionId()) - .addKeysAndValues(common.indexLogMessageKeyValues()) - .toString()); - } - } - return newSynchronizedRunner.endSessionAsync(); - }, getRunner().getDatabase()::mapAsyncToSyncException); - } else { - return newSynchronizedRunner.endSessionAsync().thenApply(vignore -> { - throw new RecordCoreException("another synchronized session is running on the indexer", - LogMessageKeys.SESSION_ID, newSynchronizedRunner.getSessionId(), - LogMessageKeys.INDEXER_SESSION_ID, currentSynchronizedRunner1.getSessionId()); - }); - } - } - abstract List indexingLogMessageKeyValues(); @Nonnull @@ -314,7 +266,7 @@ private CompletableFuture markIndexesWriteOnly(boolean continueBuild, FDBR @Nonnull public CompletableFuture markReadableIfBuilt() { AtomicBoolean allReadable = new AtomicBoolean(true); - return common.getNonSynchronizedRunner().runAsync(context -> openRecordStore(context).thenCompose(store -> + return common.getRunner().runAsync(context -> openRecordStore(context).thenCompose(store -> forEachTargetIndex(index -> { if (store.isIndexReadable(index)) { return AsyncUtil.DONE; @@ -335,6 +287,7 @@ public CompletableFuture markReadableIfBuilt() { ).thenApply(ignore -> allReadable.get()), common.indexLogMessageKeyValues("IndexingBase::markReadableIfBuilt")); } + @Nonnull public CompletableFuture markIndexReadable(boolean markReadablePlease) { if (!markReadablePlease) { @@ -360,12 +313,16 @@ public CompletableFuture markIndexReadable(boolean markReadablePlease) private CompletableFuture markIndexReadableSingleTarget(Index index, AtomicBoolean anythingChanged, AtomicReference runtimeExceptionAtomicReference) { // An extension function to reduce markIndexReadable's complexity - return common.getNonSynchronizedRunner().runAsync(context -> + return common.getRunner().runAsync(context -> common.getRecordStoreBuilder().copyBuilder().setContext(context).openAsync() - .thenCompose(store -> - policy.shouldAllowUniquePendingState(store) ? - store.markIndexReadableOrUniquePending(index) : - store.markIndexReadable(index)) + .thenCompose(store -> { + if (heartbeat != null) { + heartbeat.clearHeartbeat(store, index); + } + return policy.shouldAllowUniquePendingState(store) ? + store.markIndexReadableOrUniquePending(index) : + store.markIndexReadable(index); + }) ).handle((changed, ex) -> { if (ex == null) { if (Boolean.TRUE.equals(changed)) { @@ -388,6 +345,7 @@ public void enforceStampOverwrite() { private CompletableFuture setIndexingTypeOrThrow(FDBRecordStore store, boolean continuedBuild) { // continuedBuild is set if this session isn't a continuation of a previous indexing IndexBuildProto.IndexBuildIndexingStamp indexingTypeStamp = getIndexingTypeStamp(store); + heartbeat = new IndexingHeartbeat(common.getUuid(), indexingTypeStamp.getMethod()); return forEachTargetIndex(index -> setIndexingTypeOrThrow(store, continuedBuild, index, indexingTypeStamp)); } @@ -428,21 +386,6 @@ private CompletableFuture setIndexingTypeOrThrow(FDBRecordStore store, boo } // Here: check if type conversion is allowed if (continuedBuild && shouldAllowTypeConversionContinue(newStamp, savedStamp)) { - // Special case: partly built by another indexing method, but may be continued with the current one - if (savedStamp.getMethod().equals(IndexBuildProto.IndexBuildIndexingStamp.Method.MULTI_TARGET_BY_RECORDS)) { - // Here: throw an exception if there is an active multi-target session that includes this index - final String otherPrimaryIndexName = savedStamp.getTargetIndex(0); - if (!otherPrimaryIndexName.equals(common.getPrimaryIndex().getName())) { - // Note: For protection, avoid breaking an active multi-target session. This leads to a certain - // inconsistency for buildIndex that is called with a false `useSyncLock` - sync lock will be - // checked during a method conversion, but not during a simple "same method" continue. - return throwIfSyncedLock(otherPrimaryIndexName, store, newStamp, savedStamp) - .thenCompose(ignore -> { - store.saveIndexingTypeStamp(index, newStamp); - return AsyncUtil.DONE; - }); - } - } store.saveIndexingTypeStamp(index, newStamp); return AsyncUtil.DONE; } @@ -476,23 +419,6 @@ private static IndexBuildProto.IndexBuildIndexingStamp blocklessStampOf(IndexBui .build(); } - CompletableFuture throwIfSyncedLock(String otherIndexName, FDBRecordStore store, IndexBuildProto.IndexBuildIndexingStamp newStamp, IndexBuildProto.IndexBuildIndexingStamp savedStamp) { - final Index otherIndex = store.getRecordMetaData().getIndex(otherIndexName); - final Subspace mainLockSubspace = IndexingSubspaces.indexBuildLockSubspace(store, otherIndex); - return SynchronizedSession.checkActiveSessionExists(store.ensureContextActive(), mainLockSubspace) - .thenApply(hasActiveSession -> { - if (Boolean.TRUE.equals(hasActiveSession)) { - throw new SynchronizedSessionLockedException("Failed to takeover indexing while part of a multi-target with an existing session in progress") - .addLogInfo(LogMessageKeys.SUBSPACE, mainLockSubspace) - .addLogInfo(LogMessageKeys.PRIMARY_INDEX, otherIndexName) - .addLogInfo(LogMessageKeys.EXPECTED, PartlyBuiltException.stampToString(newStamp)) - .addLogInfo(LogMessageKeys.ACTUAL, PartlyBuiltException.stampToString(savedStamp)); - } - return null; - }); - - } - @Nonnull private CompletableFuture throwAsByRecordsUnlessNoRecordWasScanned(boolean noRecordScanned, FDBRecordStore store, @@ -885,21 +811,43 @@ private CompletableFuture hadTransactionReachedLimits(FDBRecordStore st } private CompletableFuture validateTypeStamp(@Nonnull FDBRecordStore store) { + if (shouldValidate()) { + // check other heartbeats (if exclusive) & typestamp + final IndexBuildProto.IndexBuildIndexingStamp expectedTypeStamp = getIndexingTypeStamp(store); + return forEachTargetIndex(index -> CompletableFuture.allOf( + updateHeartbeat(true, store, index), + store.loadIndexingTypeStampAsync(index) + .thenAccept(typeStamp -> validateTypeStamp(typeStamp, expectedTypeStamp, index)) + )); + } else { + // update only + return forEachTargetIndex(index -> updateHeartbeat(false, store, index)); + } + } + + private CompletableFuture updateHeartbeat(boolean validate, FDBRecordStore store, Index index) { + if (heartbeat != null) { + if (validate) { + return heartbeat.checkAndUpdateHeartbeat(store, index); + } + heartbeat.updateHeartbeat(store, index); + } + return AsyncUtil.DONE; + } + + private boolean shouldValidate() { final long minimalInterval = policy.getCheckIndexingMethodFrequencyMilliseconds(); if (minimalInterval < 0 || isScrubber) { - return AsyncUtil.DONE; + return false; } if (minimalInterval > 0) { final long now = System.currentTimeMillis(); if (now < lastTypeStampCheckMillis + minimalInterval) { - return AsyncUtil.DONE; + return false; } lastTypeStampCheckMillis = now; } - final IndexBuildProto.IndexBuildIndexingStamp expectedTypeStamp = getIndexingTypeStamp(store); - return forEachTargetIndex(index -> - store.loadIndexingTypeStampAsync(index) - .thenAccept(typeStamp -> validateTypeStamp(typeStamp, expectedTypeStamp, index))); + return true; } private void validateTypeStamp(final IndexBuildProto.IndexBuildIndexingStamp typeStamp, diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingCommon.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingCommon.java index 9bf8066786..8cc380b2bd 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingCommon.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingCommon.java @@ -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; @@ -50,10 +49,10 @@ @API(API.Status.INTERNAL) public class IndexingCommon { + // TODO? get uuid from caller to allow lock takeover private final UUID uuid = UUID.randomUUID(); @Nonnull private final FDBDatabaseRunner runner; - @Nullable private SynchronizedSessionRunner synchronizedSessionRunner = null; @Nonnull private final FDBRecordStore.Builder recordStoreBuilder; @Nonnull private final AtomicLong totalRecordsScanned; @@ -176,11 +175,6 @@ private void logIf(boolean condition, List list, @Nonnull Object... a) { @Nonnull public FDBDatabaseRunner getRunner() { - return synchronizedSessionRunner == null ? runner : synchronizedSessionRunner; - } - - @Nonnull - public FDBDatabaseRunner getNonSynchronizedRunner() { return runner; } @@ -258,15 +252,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; @@ -287,8 +272,5 @@ public boolean loadConfig() { public void close() { runner.close(); - if (synchronizedSessionRunner != null) { - synchronizedSessionRunner.close(); - } } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java new file mode 100644 index 0000000000..f04a3189db --- /dev/null +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java @@ -0,0 +1,117 @@ +/* + * 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.async.AsyncIterator; +import com.apple.foundationdb.async.AsyncUtil; +import com.apple.foundationdb.record.IndexBuildProto; +import com.apple.foundationdb.record.logging.LogMessageKeys; +import com.apple.foundationdb.record.metadata.Index; +import com.apple.foundationdb.synchronizedsession.SynchronizedSessionLockedException; +import com.apple.foundationdb.tuple.Tuple; +import com.google.protobuf.InvalidProtocolBufferException; + +import javax.annotation.Nonnull; +import java.util.UUID; +import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; + +public class IndexingHeartbeat { + // [prefix, xid] -> [indexing-type, genesis time, heartbeat time] + final UUID sessionId; + final IndexBuildProto.IndexBuildIndexingStamp.Method indexingMethod; + final long genesisTimeMilliseconds; + + public IndexingHeartbeat(final UUID sessionId, IndexBuildProto.IndexBuildIndexingStamp.Method indexingMethod) { + this.sessionId = sessionId; + this.indexingMethod = indexingMethod; + this.genesisTimeMilliseconds = nowMilliseconds(); + } + + public void updateHeartbeat(@Nonnull FDBRecordStore store, @Nonnull Index index) { + byte[] key = IndexingSubspaces.indexheartbeatSubspace(store, index, sessionId).pack(); + byte[] value = IndexBuildProto.IndexingHeartbeat.newBuilder() + .setMethod(indexingMethod) + .setGenesisTimeMilliseconds(genesisTimeMilliseconds) + .setHeartbeatTimeMilliseconds(nowMilliseconds()) + .build().toByteArray(); + store.ensureContextActive().set(key, value); + } + + public CompletableFuture checkAndUpdateHeartbeat(@Nonnull FDBRecordStore store, @Nonnull Index index) { + // complete exceptionally if non-mutual, other exists + switch (indexingMethod) { + case SCRUB_REPAIR: + case MUTUAL_BY_RECORDS: + return AsyncUtil.DONE; + + case BY_RECORDS: + case MULTI_TARGET_BY_RECORDS: + case BY_INDEX: + final AsyncIterator iterator = heartbeatsIterator(store, index); + return AsyncUtil.whileTrue(() -> iterator.onHasNext() + .thenApply(hasNext -> { + if (!hasNext) { + return false; + } + validateNonCompetingHeartbeat(iterator.next()); + return true; + })); + + default: + throw new IndexingBase.ValidationException("invalid indexing method", + LogMessageKeys.INDEXING_METHOD, indexingMethod); + } + } + + private void validateNonCompetingHeartbeat(KeyValue kv) { + final Tuple keyTuple = Tuple.from(kv.getKey()); + final UUID otherSessionId = keyTuple.getUUID(keyTuple.size() - 1); + final long now = nowMilliseconds(); + if (!otherSessionId.equals(this.sessionId)) { + try { + final IndexBuildProto.IndexingHeartbeat otherHearbeat = IndexBuildProto.IndexingHeartbeat.parseFrom(kv.getValue()); + final long age = otherHearbeat.getHeartbeatTimeMilliseconds() - now; + if (age < TimeUnit.SECONDS.toMillis(10)) { + throw new SynchronizedSessionLockedException("Failed to initialize the session because of an existing session in progress"); + // TODO: log details + } + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException(e); + } + } + } + + public void clearHeartbeat(@Nonnull FDBRecordStore store, @Nonnull Index index) { + store.ensureContextActive().clear(IndexingSubspaces.indexheartbeatSubspace(store, index, sessionId).pack()); + } + + + public AsyncIterator heartbeatsIterator(FDBRecordStore store, Index index) { + return store.getContext().ensureActive().snapshot().getRange(IndexingSubspaces.indexheartbeatSubspace(store, index).range()).iterator(); + } + + + private static long nowMilliseconds() { + return System.currentTimeMillis(); + } +} diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingMerger.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingMerger.java index 1269541d43..043c70fe20 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingMerger.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingMerger.java @@ -77,7 +77,7 @@ CompletableFuture 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(); diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingSubspaces.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingSubspaces.java index 10b564ec20..da885668b3 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingSubspaces.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingSubspaces.java @@ -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. @@ -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; private IndexingSubspaces() { throw new IllegalStateException("Utility class"); @@ -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 sessionId session id + * @return subspace + */ + @Nonnull + public static Subspace indexheartbeatSubspace(@Nonnull FDBRecordStoreBase store, @Nonnull Index index, @Nonnull UUID sessionId) { + return indexheartbeatSubspace(store, index).subspace(Tuple.from(sessionId)); + } + /** * 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. @@ -184,5 +209,6 @@ 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())); + context.clear(Range.startsWith(indexheartbeatSubspace(store, index).pack())); } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java index 81c8f561a3..2f2e74dc51 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java @@ -1235,7 +1235,7 @@ public static class Builder { private DesiredAction ifReadable = DesiredAction.CONTINUE; private boolean doAllowUniquePendingState = false; private Set allowedTakeoverSet = null; - private long checkIndexingStampFrequency = 60_000; + private long checkIndexingStampFrequency = 30_000; private boolean useMutualIndexing = false; private List useMutualIndexingBoundaries = null; private boolean allowUnblock = false; diff --git a/fdb-record-layer-core/src/main/proto/index_build.proto b/fdb-record-layer-core/src/main/proto/index_build.proto index eff6d2df1b..d9d1186038 100644 --- a/fdb-record-layer-core/src/main/proto/index_build.proto +++ b/fdb-record-layer-core/src/main/proto/index_build.proto @@ -36,6 +36,7 @@ message IndexBuildIndexingStamp { MUTUAL_BY_RECORDS = 5; // scan records, build multiple target indexes, while allowing multiple indexer processes NONE = 6; // return this stamp in query when null - never to be written in the DB }; + optional Method method = 1; optional bytes source_index_subspace_key = 2; // relevant only with BY_INDEX method optional int32 source_index_last_modified_version = 3; // only with BY_INDEX method @@ -46,4 +47,10 @@ message IndexBuildIndexingStamp { optional string blockID = 7; // optional, a short string that describes the reason for the block. } + message IndexingHeartbeat { + required IndexBuildIndexingStamp.Method method = 1; + required int64 genesisTimeMilliseconds = 2; + required int64 heartbeatTimeMilliseconds = 3; + } + From d1dfc5c6e07024f4e91cff827f3e7cd13f703099 Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Tue, 5 Aug 2025 16:54:04 -0400 Subject: [PATCH 02/26] Set/clear heartbeats at start/end --- .../provider/foundationdb/FDBStoreTimer.java | 4 + .../provider/foundationdb/IndexingBase.java | 77 +++++++++++++------ .../foundationdb/IndexingHeartbeat.java | 11 ++- 3 files changed, 66 insertions(+), 26 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBStoreTimer.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBStoreTimer.java index 07d1af133d..29bf3a5b76 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBStoreTimer.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBStoreTimer.java @@ -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. */ diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java index e8a900b4d1..674c4d9e91 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java @@ -160,19 +160,35 @@ public CompletableFuture buildIndexAsync(boolean markReadable, boolean use lastProgressSnapshot = StoreTimerSnapshot.from(timer); } message.addKeyAndValue(LogMessageKeys.SESSION_ID, common.getUuid()); - return handleStateAndDoBuildIndexAsync(markReadable, message).whenComplete((vignore, ex) -> { - message.addKeysAndValues(indexingLogMessageKeyValues()) // add these here to pick up state accumulated during build - .addKeysAndValues(common.indexLogMessageKeyValues()) - .addKeyAndValue(LogMessageKeys.TOTAL_MICROS, TimeUnit.NANOSECONDS.toMicros(System.nanoTime() - startNanos)); - if (LOGGER.isWarnEnabled() && (ex != null)) { - message.addKeyAndValue(LogMessageKeys.RESULT, "failure"); - message.addKeysAndValues(throttle.logMessageKeyValues()); // this "last attempt" snapshot information can help debugging - LOGGER.warn(message.toString(), ex); - } else if (LOGGER.isInfoEnabled()) { - message.addKeyAndValue(LogMessageKeys.RESULT, "success"); - LOGGER.info(message.toString()); - } - }); + AtomicReference indexingException = new AtomicReference<>(null); + return handleStateAndDoBuildIndexAsync(markReadable, message) + .handle((ret, ex) -> { + if (ex != null) { + indexingException.set(ex); + } + message.addKeysAndValues(indexingLogMessageKeyValues()) // add these here to pick up state accumulated during build + .addKeysAndValues(common.indexLogMessageKeyValues()) + .addKeyAndValue(LogMessageKeys.TOTAL_MICROS, TimeUnit.NANOSECONDS.toMicros(System.nanoTime() - startNanos)); + if (LOGGER.isWarnEnabled() && (ex != null)) { + message.addKeyAndValue(LogMessageKeys.RESULT, "failure"); + message.addKeysAndValues(throttle.logMessageKeyValues()); // this "last attempt" snapshot information can help debugging + LOGGER.warn(message.toString(), ex); + } else if (LOGGER.isInfoEnabled()) { + message.addKeyAndValue(LogMessageKeys.RESULT, "success"); + LOGGER.info(message.toString()); + } + return ret; + }) + .thenCompose(ignore -> clearHeartbeats()) + .handle((ignore, exIgnore) -> { + Throwable ex = indexingException.get(); + if (ex instanceof RuntimeException) { + throw (RuntimeException) ex; + } else if (ex != null) { + throw new RuntimeException(ex); + } + return null; + }); } abstract List indexingLogMessageKeyValues(); @@ -266,7 +282,7 @@ private CompletableFuture markIndexesWriteOnly(boolean continueBuild, FDBR @Nonnull public CompletableFuture markReadableIfBuilt() { AtomicBoolean allReadable = new AtomicBoolean(true); - return common.getRunner().runAsync(context -> openRecordStore(context).thenCompose(store -> + return getRunner().runAsync(context -> openRecordStore(context).thenCompose(store -> forEachTargetIndex(index -> { if (store.isIndexReadable(index)) { return AsyncUtil.DONE; @@ -313,12 +329,9 @@ public CompletableFuture markIndexReadable(boolean markReadablePlease) private CompletableFuture markIndexReadableSingleTarget(Index index, AtomicBoolean anythingChanged, AtomicReference runtimeExceptionAtomicReference) { // An extension function to reduce markIndexReadable's complexity - return common.getRunner().runAsync(context -> + return getRunner().runAsync(context -> common.getRecordStoreBuilder().copyBuilder().setContext(context).openAsync() .thenCompose(store -> { - if (heartbeat != null) { - heartbeat.clearHeartbeat(store, index); - } return policy.shouldAllowUniquePendingState(store) ? store.markIndexReadableOrUniquePending(index) : store.markIndexReadable(index); @@ -352,12 +365,13 @@ private CompletableFuture setIndexingTypeOrThrow(FDBRecordStore store, boo @Nonnull private CompletableFuture setIndexingTypeOrThrow(FDBRecordStore store, boolean continuedBuild, Index index, IndexBuildProto.IndexBuildIndexingStamp newStamp) { + final CompletableFuture checkUpdateHeartbeat = updateHeartbeat(true, store, index); if (forceStampOverwrite && !continuedBuild) { // Fresh session + overwrite = no questions asked store.saveIndexingTypeStamp(index, newStamp); - return AsyncUtil.DONE; + return checkUpdateHeartbeat; } - return store.loadIndexingTypeStampAsync(index) + return checkUpdateHeartbeat.thenCompose(ignore -> store.loadIndexingTypeStampAsync(index) .thenCompose(savedStamp -> { if (savedStamp == null) { if (continuedBuild && newStamp.getMethod() != @@ -399,7 +413,7 @@ private CompletableFuture setIndexingTypeOrThrow(FDBRecordStore store, boo } // fall down to exception throw newPartlyBuiltException(continuedBuild, savedStamp, newStamp, index); - }); + })); } private boolean shouldAllowTypeConversionContinue(IndexBuildProto.IndexBuildIndexingStamp newStamp, IndexBuildProto.IndexBuildIndexingStamp savedStamp) { @@ -521,7 +535,7 @@ protected CompletableFuture doneOrThrottleDelayAndMaybeLogProgress(bool validateTimeLimit(toWait); - CompletableFuture delay = MoreAsyncUtil.delayedFuture(toWait, TimeUnit.MILLISECONDS, common.getRunner().getScheduledExecutor()).thenApply(vignore3 -> true); + CompletableFuture delay = MoreAsyncUtil.delayedFuture(toWait, TimeUnit.MILLISECONDS, getRunner().getScheduledExecutor()).thenApply(vignore3 -> true); if (getRunner().getTimer() != null) { delay = getRunner().getTimer().instrument(FDBStoreTimer.Events.INDEXER_DELAY, delay, getRunner().getExecutor()); } @@ -835,6 +849,25 @@ private CompletableFuture updateHeartbeat(boolean validate, FDBRecordStore return AsyncUtil.DONE; } + private CompletableFuture clearHeartbeats() { + // Here: if the heartbeat was *not* cleared while marking the index readable, it would be cleared in + // these dedicated transaction. Heartbeat clearing is not a blocker but a "best effort" operation. + if (heartbeat == null) { + return AsyncUtil.DONE; + } + return forEachTargetIndex(this::clearHeartbeatSingleTarget); + } + + private CompletableFuture clearHeartbeatSingleTarget(Index index) { + return getRunner().runAsync(context -> + common.getRecordStoreBuilder().copyBuilder().setContext(context).openAsync() + .thenApply(store -> { + heartbeat.clearHeartbeat(store, index); + return null; + })); + } + + private boolean shouldValidate() { final long minimalInterval = policy.getCheckIndexingMethodFrequencyMilliseconds(); if (minimalInterval < 0 || isScrubber) { diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java index f04a3189db..b0ade84d09 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java @@ -84,14 +84,17 @@ public CompletableFuture checkAndUpdateHeartbeat(@Nonnull FDBRecordStore s } private void validateNonCompetingHeartbeat(KeyValue kv) { - final Tuple keyTuple = Tuple.from(kv.getKey()); + final Tuple keyTuple = Tuple.fromBytes(kv.getKey()); + if (keyTuple.size() < 2) { // expecting 8 + return; + } final UUID otherSessionId = keyTuple.getUUID(keyTuple.size() - 1); final long now = nowMilliseconds(); if (!otherSessionId.equals(this.sessionId)) { try { - final IndexBuildProto.IndexingHeartbeat otherHearbeat = IndexBuildProto.IndexingHeartbeat.parseFrom(kv.getValue()); - final long age = otherHearbeat.getHeartbeatTimeMilliseconds() - now; - if (age < TimeUnit.SECONDS.toMillis(10)) { + final IndexBuildProto.IndexingHeartbeat otherHeartbeat = IndexBuildProto.IndexingHeartbeat.parseFrom(kv.getValue()); + final long age = now - otherHeartbeat.getHeartbeatTimeMilliseconds(); + if (age > 0 && age < TimeUnit.SECONDS.toMillis(10)) { throw new SynchronizedSessionLockedException("Failed to initialize the session because of an existing session in progress"); // TODO: log details } From 30acc02e5aba3ad58857ce5fb91f9e1856924f55 Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Tue, 5 Aug 2025 19:48:06 -0400 Subject: [PATCH 03/26] Optimize heartbeat clear. Deprecate use-sync-indexing API --- .../provider/foundationdb/IndexingBase.java | 22 ++++++++++++++----- .../foundationdb/IndexingHeartbeat.java | 7 +++--- .../OnlineIndexOperationBaseBuilder.java | 2 -- .../OnlineIndexOperationConfig.java | 19 ++++++---------- .../foundationdb/OnlineIndexScrubber.java | 2 +- .../provider/foundationdb/OnlineIndexer.java | 3 +-- 6 files changed, 29 insertions(+), 26 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java index 674c4d9e91..d19878bf5f 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java @@ -150,7 +150,7 @@ protected CompletableFuture> recordIfInIndexedTypes(FDB // buildIndexAsync - the main indexing function. Builds and commits indexes asynchronously; throttling to avoid overloading the system. @SuppressWarnings("PMD.CloseResource") - public CompletableFuture buildIndexAsync(boolean markReadable, boolean useSyncLock) { + public CompletableFuture buildIndexAsync(boolean markReadable) { KeyValueLogMessage message = KeyValueLogMessage.build("build index online", LogMessageKeys.SHOULD_MARK_READABLE, markReadable); long startNanos = System.nanoTime(); @@ -179,6 +179,8 @@ public CompletableFuture buildIndexAsync(boolean markReadable, boolean use } return ret; }) + // Here: if the heartbeat was *not* cleared while marking the index readable, it would be cleared in + // these dedicated transaction. Heartbeat clearing is not a blocker but a "best effort" operation. .thenCompose(ignore -> clearHeartbeats()) .handle((ignore, exIgnore) -> { Throwable ex = indexingException.get(); @@ -322,6 +324,7 @@ public CompletableFuture markIndexReadable(boolean markReadablePlease) if (ex != null) { throw ex; } + heartbeat = null; // Here: heartbeats had been successfully cleared. No need to clear again return anythingChanged.get(); }); } @@ -332,12 +335,14 @@ private CompletableFuture markIndexReadableSingleTarget(Index index, At return getRunner().runAsync(context -> common.getRecordStoreBuilder().copyBuilder().setContext(context).openAsync() .thenCompose(store -> { + clearHeartbeatSingleTarget(store, index); return policy.shouldAllowUniquePendingState(store) ? store.markIndexReadableOrUniquePending(index) : store.markIndexReadable(index); }) ).handle((changed, ex) -> { if (ex == null) { + heartbeat = null; // Here: all heartbeats were successfully cleared withing the set readable transactions. No need to clear again. if (Boolean.TRUE.equals(changed)) { anythingChanged.set(true); } @@ -358,7 +363,7 @@ public void enforceStampOverwrite() { private CompletableFuture setIndexingTypeOrThrow(FDBRecordStore store, boolean continuedBuild) { // continuedBuild is set if this session isn't a continuation of a previous indexing IndexBuildProto.IndexBuildIndexingStamp indexingTypeStamp = getIndexingTypeStamp(store); - heartbeat = new IndexingHeartbeat(common.getUuid(), indexingTypeStamp.getMethod()); + heartbeat = new IndexingHeartbeat(common.getUuid(), indexingTypeStamp.getMethod(), common.config.getLeaseLengthMillis()); return forEachTargetIndex(index -> setIndexingTypeOrThrow(store, continuedBuild, index, indexingTypeStamp)); } @@ -850,23 +855,28 @@ private CompletableFuture updateHeartbeat(boolean validate, FDBRecordStore } private CompletableFuture clearHeartbeats() { - // Here: if the heartbeat was *not* cleared while marking the index readable, it would be cleared in - // these dedicated transaction. Heartbeat clearing is not a blocker but a "best effort" operation. if (heartbeat == null) { return AsyncUtil.DONE; } - return forEachTargetIndex(this::clearHeartbeatSingleTarget); + return forEachTargetIndex(this::clearHeartbeatSingleTarget) + .thenAccept(ignore -> heartbeat = null); } private CompletableFuture clearHeartbeatSingleTarget(Index index) { return getRunner().runAsync(context -> common.getRecordStoreBuilder().copyBuilder().setContext(context).openAsync() .thenApply(store -> { - heartbeat.clearHeartbeat(store, index); + clearHeartbeatSingleTarget(store, index); return null; })); } + private void clearHeartbeatSingleTarget(FDBRecordStore store, Index index) { + if (heartbeat != null) { + heartbeat.clearHeartbeat(store, index); + } + } + private boolean shouldValidate() { final long minimalInterval = policy.getCheckIndexingMethodFrequencyMilliseconds(); diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java index b0ade84d09..63973cb01d 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java @@ -33,17 +33,18 @@ import javax.annotation.Nonnull; import java.util.UUID; import java.util.concurrent.CompletableFuture; -import java.util.concurrent.TimeUnit; public class IndexingHeartbeat { // [prefix, xid] -> [indexing-type, genesis time, heartbeat time] final UUID sessionId; final IndexBuildProto.IndexBuildIndexingStamp.Method indexingMethod; final long genesisTimeMilliseconds; + final long leaseLength; - public IndexingHeartbeat(final UUID sessionId, IndexBuildProto.IndexBuildIndexingStamp.Method indexingMethod) { + public IndexingHeartbeat(final UUID sessionId, IndexBuildProto.IndexBuildIndexingStamp.Method indexingMethod, long leaseLength) { this.sessionId = sessionId; this.indexingMethod = indexingMethod; + this.leaseLength = leaseLength; this.genesisTimeMilliseconds = nowMilliseconds(); } @@ -94,7 +95,7 @@ private void validateNonCompetingHeartbeat(KeyValue kv) { try { final IndexBuildProto.IndexingHeartbeat otherHeartbeat = IndexBuildProto.IndexingHeartbeat.parseFrom(kv.getValue()); final long age = now - otherHeartbeat.getHeartbeatTimeMilliseconds(); - if (age > 0 && age < TimeUnit.SECONDS.toMillis(10)) { + if (age > 0 && age < leaseLength) { throw new SynchronizedSessionLockedException("Failed to initialize the session because of an existing session in progress"); // TODO: log details } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationBaseBuilder.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationBaseBuilder.java index 25b669a705..c8a0f03f39 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationBaseBuilder.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationBaseBuilder.java @@ -720,13 +720,11 @@ public B setTransactionTimeLimitMilliseconds(long timeLimitMilliseconds) { */ @API(API.Status.DEPRECATED) public B setUseSynchronizedSession(boolean useSynchronizedSession) { - configBuilder.setUseSynchronizedSession(useSynchronizedSession); return self(); } /** * Set the lease length in milliseconds if the synchronized session is used. The default value is {@link OnlineIndexOperationConfig#DEFAULT_LEASE_LENGTH_MILLIS}. - * @see #setUseSynchronizedSession(boolean) * @see com.apple.foundationdb.synchronizedsession.SynchronizedSession * @param leaseLengthMillis length between last access and lease's end time in milliseconds * @return this builder diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationConfig.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationConfig.java index 1ce11d342b..65f4fe6d84 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationConfig.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationConfig.java @@ -77,14 +77,13 @@ public class OnlineIndexOperationConfig { private final int increaseLimitAfter; private final long timeLimitMilliseconds; private final long transactionTimeLimitMilliseconds; - private final boolean useSynchronizedSession; private final long leaseLengthMillis; public static final long UNLIMITED_TIME = 0; OnlineIndexOperationConfig(int maxLimit, int initialLimit, int maxRetries, int recordsPerSecond, long progressLogIntervalMillis, int increaseLimitAfter, int maxWriteLimitBytes, long timeLimitMilliseconds, long transactionTimeLimitMilliseconds, - boolean useSynchronizedSession, long leaseLengthMillis) { + long leaseLengthMillis) { this.maxLimit = maxLimit; this.initialLimit = initialLimit; this.maxRetries = maxRetries; @@ -94,7 +93,6 @@ public class OnlineIndexOperationConfig { this.maxWriteLimitBytes = maxWriteLimitBytes; this.timeLimitMilliseconds = timeLimitMilliseconds; this.transactionTimeLimitMilliseconds = transactionTimeLimitMilliseconds; - this.useSynchronizedSession = useSynchronizedSession; this.leaseLengthMillis = leaseLengthMillis; } @@ -188,10 +186,6 @@ public static Builder newBuilder() { return new Builder(); } - public boolean shouldUseSynchronizedSession() { - return useSynchronizedSession; - } - public long getLeaseLengthMillis() { return leaseLengthMillis; } @@ -213,7 +207,6 @@ public Builder toBuilder() { .setMaxRetries(this.maxRetries) .setTimeLimitMilliseconds(timeLimitMilliseconds) .setTransactionTimeLimitMilliseconds(this.transactionTimeLimitMilliseconds) - .setUseSynchronizedSession(useSynchronizedSession) .setLeaseLengthMillis(leaseLengthMillis); } @@ -234,7 +227,6 @@ public static class Builder { private long timeLimitMilliseconds = UNLIMITED_TIME; private long transactionTimeLimitMilliseconds = DEFAULT_TRANSACTION_TIME_LIMIT; private long leaseLengthMillis = DEFAULT_LEASE_LENGTH_MILLIS; - private boolean useSynchronizedSession = true; protected Builder() { @@ -492,15 +484,18 @@ public Builder setTransactionTimeLimitMilliseconds(long timeLimitMilliseconds) { * @see SynchronizedSessionRunner * @param useSynchronizedSession use synchronize session if true, otherwise false * @return this builder + * + * @deprecated Synchronized sessions are now determined by the indexing method. */ + @API(API.Status.DEPRECATED) + @Deprecated(since = "4.4.3.0", forRemoval = true) public Builder setUseSynchronizedSession(boolean useSynchronizedSession) { - this.useSynchronizedSession = useSynchronizedSession; + // no-op return this; } /** * Set the lease length in milliseconds if the synchronized session is used. By default this is {@link #DEFAULT_LEASE_LENGTH_MILLIS}. - * @see #setUseSynchronizedSession(boolean) * @see com.apple.foundationdb.synchronizedsession.SynchronizedSession * @param leaseLengthMillis length between last access and lease's end time in milliseconds * @return this builder @@ -519,7 +514,7 @@ public Builder setLeaseLengthMillis(long leaseLengthMillis) { public OnlineIndexOperationConfig build() { return new OnlineIndexOperationConfig(maxLimit, initialLimit, maxRetries, recordsPerSecond, progressLogIntervalMillis, increaseLimitAfter, maxWriteLimitBytes, timeLimitMilliseconds, transactionTimeLimitMilliseconds, - useSynchronizedSession, leaseLengthMillis); + leaseLengthMillis); } } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexScrubber.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexScrubber.java index ced1afd22d..f1ed949da3 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexScrubber.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexScrubber.java @@ -75,7 +75,7 @@ private IndexingBase getScrubber(IndexScrubbingTools.ScrubbingType type, AtomicL @Nonnull private CompletableFuture scrubIndexAsync(IndexScrubbingTools.ScrubbingType type, AtomicLong count) { return AsyncUtil.composeHandle( - getScrubber(type, count).buildIndexAsync(false, common.config.shouldUseSynchronizedSession()), + getScrubber(type, count).buildIndexAsync(false), (ignore, ex) -> { if (ex != null) { throw FDBExceptions.wrapException(ex); diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java index 2f2e74dc51..414c143bbf 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java @@ -521,8 +521,7 @@ public CompletableFuture buildIndexAsync() { @VisibleForTesting @Nonnull CompletableFuture buildIndexAsync(boolean markReadable) { - boolean useSyncLock = (!indexingPolicy.isMutual() || fallbackToRecordsScan) && common.config.shouldUseSynchronizedSession(); - return indexingLauncher(() -> getIndexer().buildIndexAsync(markReadable, useSyncLock)); + return indexingLauncher(() -> getIndexer().buildIndexAsync(markReadable)); } /** From d4e5fb2b2069534bee4a043148b4469e802b91ab Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Tue, 5 Aug 2025 20:14:11 -0400 Subject: [PATCH 04/26] false AvoidUsingHardCodedIP warning --- .../record/provider/foundationdb/OnlineIndexOperationConfig.java | 1 + 1 file changed, 1 insertion(+) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationConfig.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationConfig.java index 65f4fe6d84..2c781e7d38 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationConfig.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationConfig.java @@ -488,6 +488,7 @@ public Builder setTransactionTimeLimitMilliseconds(long timeLimitMilliseconds) { * @deprecated Synchronized sessions are now determined by the indexing method. */ @API(API.Status.DEPRECATED) + @SuppressWarnings("PMD.AvoidUsingHardCodedIP") // version is not IP @Deprecated(since = "4.4.3.0", forRemoval = true) public Builder setUseSynchronizedSession(boolean useSynchronizedSession) { // no-op From 8a28807e57a43bb2016b426aa2b03ff1bd5e1417 Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Wed, 6 Aug 2025 13:16:12 -0400 Subject: [PATCH 05/26] Clear heartbeats after rebuild (whting the same transaction) --- .../apple/foundationdb/record/logging/LogMessageKeys.java | 2 ++ .../record/provider/foundationdb/IndexingBase.java | 6 +++++- .../record/provider/foundationdb/IndexingHeartbeat.java | 6 +++++- 3 files changed, 12 insertions(+), 2 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/logging/LogMessageKeys.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/logging/LogMessageKeys.java index 78a8987220..9e2ffc927c 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/logging/LogMessageKeys.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/logging/LogMessageKeys.java @@ -58,6 +58,7 @@ public enum LogMessageKeys { TRANSACTION_ID, TRANSACTION_NAME, AGE_SECONDS, + AGE_MILLISECONDS, CONSTITUENT, TOTAL_MICROS, // record splitting/unsplitting @@ -162,6 +163,7 @@ public enum LogMessageKeys { RECORDS_PER_SECOND, DOCUMENT, SESSION_ID, + EXISTING_SESSION_ID, INDEXER_SESSION_ID, INDEXER_ID, INDEX_STATE_PRECONDITION, diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java index d19878bf5f..e317857810 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java @@ -1027,7 +1027,11 @@ public CompletableFuture rebuildIndexAsync(@Nonnull FDBRecordStore store) return rangeSet.insertRangeAsync(null, null); })) .thenCompose(vignore -> setIndexingTypeOrThrow(store, false)) - .thenCompose(vignore -> rebuildIndexInternalAsync(store)); + .thenCompose(vignore -> rebuildIndexInternalAsync(store)) + .thenCompose(vignore -> forEachTargetIndex(index -> { + clearHeartbeatSingleTarget(store, index); + return null; + })); } abstract CompletableFuture rebuildIndexInternalAsync(FDBRecordStore store); diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java index 63973cb01d..423f7284a1 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java @@ -96,7 +96,11 @@ private void validateNonCompetingHeartbeat(KeyValue kv) { final IndexBuildProto.IndexingHeartbeat otherHeartbeat = IndexBuildProto.IndexingHeartbeat.parseFrom(kv.getValue()); final long age = now - otherHeartbeat.getHeartbeatTimeMilliseconds(); if (age > 0 && age < leaseLength) { - throw new SynchronizedSessionLockedException("Failed to initialize the session because of an existing session in progress"); + throw new SynchronizedSessionLockedException("Failed to initialize the session because of an existing session in progress") + .addLogInfo(LogMessageKeys.SESSION_ID, sessionId) + .addLogInfo(LogMessageKeys.EXISTING_SESSION_ID, otherSessionId) + .addLogInfo(LogMessageKeys.AGE_MILLISECONDS, age) + .addLogInfo(LogMessageKeys.TIME_LIMIT_MILLIS, leaseLength); // TODO: log details } } catch (InvalidProtocolBufferException e) { From 1f5c02b0ce7685d28f327099f2e32913bd83a405 Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Wed, 6 Aug 2025 17:13:52 -0400 Subject: [PATCH 06/26] Itarate targets explicitly after rebuild --- .../record/provider/foundationdb/IndexingBase.java | 8 +++++--- 1 file changed, 5 insertions(+), 3 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java index e317857810..7bdf48f968 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java @@ -1028,10 +1028,12 @@ public CompletableFuture rebuildIndexAsync(@Nonnull FDBRecordStore store) })) .thenCompose(vignore -> setIndexingTypeOrThrow(store, false)) .thenCompose(vignore -> rebuildIndexInternalAsync(store)) - .thenCompose(vignore -> forEachTargetIndex(index -> { - clearHeartbeatSingleTarget(store, index); + .thenApply(vignore -> { + for (Index index: common.getTargetIndexes()) { + clearHeartbeatSingleTarget(store, index); + } return null; - })); + }); } abstract CompletableFuture rebuildIndexInternalAsync(FDBRecordStore store); From 44d965ba6d8ed23c52ef43e75c7cee825ab5b5f0 Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Thu, 7 Aug 2025 11:33:29 -0400 Subject: [PATCH 07/26] thenApply -> whenComplete --- .../record/provider/foundationdb/IndexingBase.java | 4 +--- 1 file changed, 1 insertion(+), 3 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java index 7bdf48f968..d5f5f9141a 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java @@ -342,7 +342,6 @@ private CompletableFuture markIndexReadableSingleTarget(Index index, At }) ).handle((changed, ex) -> { if (ex == null) { - heartbeat = null; // Here: all heartbeats were successfully cleared withing the set readable transactions. No need to clear again. if (Boolean.TRUE.equals(changed)) { anythingChanged.set(true); } @@ -1028,11 +1027,10 @@ public CompletableFuture rebuildIndexAsync(@Nonnull FDBRecordStore store) })) .thenCompose(vignore -> setIndexingTypeOrThrow(store, false)) .thenCompose(vignore -> rebuildIndexInternalAsync(store)) - .thenApply(vignore -> { + .whenComplete((ignore, ignoreEx) -> { for (Index index: common.getTargetIndexes()) { clearHeartbeatSingleTarget(store, index); } - return null; }); } From 4fdefe1c83997a0912a3d001af9c60cee630a30f Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Fri, 8 Aug 2025 12:06:57 -0400 Subject: [PATCH 08/26] Add API get/clear heartbeats --- .../provider/foundationdb/IndexingBase.java | 10 +++ .../foundationdb/IndexingHeartbeat.java | 74 +++++++++++++++++-- .../provider/foundationdb/OnlineIndexer.java | 24 ++++++ 3 files changed, 103 insertions(+), 5 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java index d5f5f9141a..a8db51e786 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java @@ -1102,6 +1102,16 @@ boolean performIndexingStampOperation(@Nonnull ConcurrentHashMap> getIndexingHeartbeats(int maxCount) { + return getRunner().runAsync(context -> openRecordStore(context) + .thenCompose(store -> IndexingHeartbeat.getIndexingHeartbeats(store, common.getPrimaryIndex(), maxCount))); + } + + public CompletableFuture clearIndexingHeartbeats(long minAgenMilliseconds, int maxIteration) { + return getRunner().runAsync(context -> openRecordStore(context) + .thenCompose(store -> IndexingHeartbeat.clearIndexingHeartbeats(store, common.getPrimaryIndex(), minAgenMilliseconds, maxIteration))); + } + /** * Thrown when the indexing process fails to meet a precondition. */ diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java index 423f7284a1..3202d061c2 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java @@ -31,8 +31,11 @@ import com.google.protobuf.InvalidProtocolBufferException; 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; public class IndexingHeartbeat { // [prefix, xid] -> [indexing-type, genesis time, heartbeat time] @@ -74,7 +77,7 @@ public CompletableFuture checkAndUpdateHeartbeat(@Nonnull FDBRecordStore s if (!hasNext) { return false; } - validateNonCompetingHeartbeat(iterator.next()); + validateNonCompetingHeartbeat(iterator.next(), nowMilliseconds()); return true; })); @@ -84,13 +87,12 @@ public CompletableFuture checkAndUpdateHeartbeat(@Nonnull FDBRecordStore s } } - private void validateNonCompetingHeartbeat(KeyValue kv) { + private void validateNonCompetingHeartbeat(KeyValue kv, long now) { final Tuple keyTuple = Tuple.fromBytes(kv.getKey()); if (keyTuple.size() < 2) { // expecting 8 return; } final UUID otherSessionId = keyTuple.getUUID(keyTuple.size() - 1); - final long now = nowMilliseconds(); if (!otherSessionId.equals(this.sessionId)) { try { final IndexBuildProto.IndexingHeartbeat otherHeartbeat = IndexBuildProto.IndexingHeartbeat.parseFrom(kv.getValue()); @@ -101,7 +103,6 @@ private void validateNonCompetingHeartbeat(KeyValue kv) { .addLogInfo(LogMessageKeys.EXISTING_SESSION_ID, otherSessionId) .addLogInfo(LogMessageKeys.AGE_MILLISECONDS, age) .addLogInfo(LogMessageKeys.TIME_LIMIT_MILLIS, leaseLength); - // TODO: log details } } catch (InvalidProtocolBufferException e) { throw new RuntimeException(e); @@ -113,8 +114,71 @@ public void clearHeartbeat(@Nonnull FDBRecordStore store, @Nonnull Index index) store.ensureContextActive().clear(IndexingSubspaces.indexheartbeatSubspace(store, index, sessionId).pack()); } + public static CompletableFuture> getIndexingHeartbeats(FDBRecordStore store, Index index, int maxCount) { + final Map ret = new HashMap<>(); + final AsyncIterator iterator = heartbeatsIterator(store, index); + final AtomicInteger iterationCount = new AtomicInteger(0); + 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 Tuple keyTuple = Tuple.fromBytes(kv.getKey()); + if (keyTuple.size() < 2) { // expecting 8 + return true; // ignore, next + } + final UUID otherSessionId = keyTuple.getUUID(keyTuple.size() - 1); + try { + final IndexBuildProto.IndexingHeartbeat otherHeartbeat = IndexBuildProto.IndexingHeartbeat.parseFrom(kv.getValue()); + ret.put(otherSessionId, otherHeartbeat); + } catch (InvalidProtocolBufferException e) { + // put a NONE heartbeat to indicate an invalid item + ret.put(otherSessionId, IndexBuildProto.IndexingHeartbeat.newBuilder() + .setMethod(IndexBuildProto.IndexBuildIndexingStamp.Method.NONE) + .build()); + } + return true; + })) + .thenApply(ignore -> ret); + } + + public static CompletableFuture clearIndexingHeartbeats(@Nonnull FDBRecordStore store, @Nonnull Index index, long minAgenMilliseconds, int maxIteration) { + final AsyncIterator 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.IndexingHeartbeat otherHeartbeat = IndexBuildProto.IndexingHeartbeat.parseFrom(kv.getValue()); + // remove heartbeat if too old + shouldRemove = now + minAgenMilliseconds <= otherHeartbeat.getHeartbeatTimeMilliseconds(); + } catch (InvalidProtocolBufferException e) { + // remove heartbeat if invalid + shouldRemove = true; + } + if (shouldRemove) { + store.ensureContextActive().clear(kv.getKey()); + deleteCount.incrementAndGet(); + } + return true; + })) + .thenApply(ignore -> deleteCount.get()); + } - public AsyncIterator heartbeatsIterator(FDBRecordStore store, Index index) { + public static AsyncIterator heartbeatsIterator(FDBRecordStore store, Index index) { return store.getContext().ensureActive().snapshot().getRange(IndexingSubspaces.indexheartbeatSubspace(store, index).range()).iterator(); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java index 414c143bbf..0af21c5fbb 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java @@ -50,6 +50,7 @@ import java.util.List; import java.util.Map; import java.util.Set; +import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.atomic.AtomicLong; import java.util.function.BiFunction; @@ -604,6 +605,29 @@ private Map indexingStamp(@Null getIndexer().performIndexingStampOperation(op, id, ttlSeconds)); } + /** + * Get the current indexing heartbeats for a given index (single target or primary index). + * @param maxCount safety valve to limit number items to read. Typically set to zero to keep unlimited. + * @return map of session ids to {@link IndexBuildProto.IndexingHeartbeat} + */ + @API(API.Status.EXPERIMENTAL) + public Map getIndexingHeartbeats(int maxCount) { + return asyncToSync(FDBStoreTimer.Waits.WAIT_INDEX_READ_HEARTBEATS, + getIndexer().getIndexingHeartbeats(maxCount)); + } + + /** + * Clear old indexing heartbeats for a given index (single target or primary index). + * @param minAgenMilliseconds minimum heartbeat age (in milliseconds) to clear. + * @param maxIteration safety valve to limit number of items to check. Typically set to zero to keep unlimited + * @return number of cleared heartbeats + */ + @API(API.Status.EXPERIMENTAL) + public int clearIndexingHeartbeats(long minAgenMilliseconds, int maxIteration) { + return asyncToSync(FDBStoreTimer.Waits.WAIT_INDEX_CLEAR_HEARTBEATS, + getIndexer().clearIndexingHeartbeats(minAgenMilliseconds, maxIteration)); + } + /** * Wait for an asynchronous task to complete. This returns the result from the future or propagates * the error if the future completes exceptionally. From 829911b0d61341356d900a6541d0c358a7b5f117 Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Fri, 8 Aug 2025 17:41:06 -0400 Subject: [PATCH 09/26] Add 1st test - low level --- .../provider/foundationdb/IndexingBase.java | 2 +- .../foundationdb/IndexingHeartbeat.java | 16 +-- .../provider/foundationdb/OnlineIndexer.java | 4 +- .../src/main/proto/index_build.proto | 2 +- .../OnlineIndexingHeartbeatTest.java | 101 ++++++++++++++++++ 5 files changed, 113 insertions(+), 12 deletions(-) create mode 100644 fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java index a8db51e786..458ba02165 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java @@ -1102,7 +1102,7 @@ boolean performIndexingStampOperation(@Nonnull ConcurrentHashMap> getIndexingHeartbeats(int maxCount) { + public CompletableFuture> getIndexingHeartbeats(int maxCount) { return getRunner().runAsync(context -> openRecordStore(context) .thenCompose(store -> IndexingHeartbeat.getIndexingHeartbeats(store, common.getPrimaryIndex(), maxCount))); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java index 3202d061c2..24d148fa65 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java @@ -53,7 +53,7 @@ public IndexingHeartbeat(final UUID sessionId, IndexBuildProto.IndexBuildIndexin public void updateHeartbeat(@Nonnull FDBRecordStore store, @Nonnull Index index) { byte[] key = IndexingSubspaces.indexheartbeatSubspace(store, index, sessionId).pack(); - byte[] value = IndexBuildProto.IndexingHeartbeat.newBuilder() + byte[] value = IndexBuildProto.IndexBuildHeartbeat.newBuilder() .setMethod(indexingMethod) .setGenesisTimeMilliseconds(genesisTimeMilliseconds) .setHeartbeatTimeMilliseconds(nowMilliseconds()) @@ -95,7 +95,7 @@ private void validateNonCompetingHeartbeat(KeyValue kv, long now) { final UUID otherSessionId = keyTuple.getUUID(keyTuple.size() - 1); if (!otherSessionId.equals(this.sessionId)) { try { - final IndexBuildProto.IndexingHeartbeat otherHeartbeat = IndexBuildProto.IndexingHeartbeat.parseFrom(kv.getValue()); + final IndexBuildProto.IndexBuildHeartbeat otherHeartbeat = IndexBuildProto.IndexBuildHeartbeat.parseFrom(kv.getValue()); final long age = now - otherHeartbeat.getHeartbeatTimeMilliseconds(); if (age > 0 && age < leaseLength) { throw new SynchronizedSessionLockedException("Failed to initialize the session because of an existing session in progress") @@ -114,8 +114,8 @@ public void clearHeartbeat(@Nonnull FDBRecordStore store, @Nonnull Index index) store.ensureContextActive().clear(IndexingSubspaces.indexheartbeatSubspace(store, index, sessionId).pack()); } - public static CompletableFuture> getIndexingHeartbeats(FDBRecordStore store, Index index, int maxCount) { - final Map ret = new HashMap<>(); + public static CompletableFuture> getIndexingHeartbeats(FDBRecordStore store, Index index, int maxCount) { + final Map ret = new HashMap<>(); final AsyncIterator iterator = heartbeatsIterator(store, index); final AtomicInteger iterationCount = new AtomicInteger(0); return AsyncUtil.whileTrue(() -> iterator.onHasNext() @@ -133,11 +133,11 @@ public static CompletableFuture> ge } final UUID otherSessionId = keyTuple.getUUID(keyTuple.size() - 1); try { - final IndexBuildProto.IndexingHeartbeat otherHeartbeat = IndexBuildProto.IndexingHeartbeat.parseFrom(kv.getValue()); + final IndexBuildProto.IndexBuildHeartbeat otherHeartbeat = IndexBuildProto.IndexBuildHeartbeat.parseFrom(kv.getValue()); ret.put(otherSessionId, otherHeartbeat); } catch (InvalidProtocolBufferException e) { // put a NONE heartbeat to indicate an invalid item - ret.put(otherSessionId, IndexBuildProto.IndexingHeartbeat.newBuilder() + ret.put(otherSessionId, IndexBuildProto.IndexBuildHeartbeat.newBuilder() .setMethod(IndexBuildProto.IndexBuildIndexingStamp.Method.NONE) .build()); } @@ -162,9 +162,9 @@ public static CompletableFuture clearIndexingHeartbeats(@Nonnull FDBRec final KeyValue kv = iterator.next(); boolean shouldRemove; try { - final IndexBuildProto.IndexingHeartbeat otherHeartbeat = IndexBuildProto.IndexingHeartbeat.parseFrom(kv.getValue()); + final IndexBuildProto.IndexBuildHeartbeat otherHeartbeat = IndexBuildProto.IndexBuildHeartbeat.parseFrom(kv.getValue()); // remove heartbeat if too old - shouldRemove = now + minAgenMilliseconds <= otherHeartbeat.getHeartbeatTimeMilliseconds(); + shouldRemove = now + minAgenMilliseconds >= otherHeartbeat.getHeartbeatTimeMilliseconds(); } catch (InvalidProtocolBufferException e) { // remove heartbeat if invalid shouldRemove = true; diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java index 0af21c5fbb..befa7b04bf 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java @@ -608,10 +608,10 @@ private Map indexingStamp(@Null /** * Get the current indexing heartbeats for a given index (single target or primary index). * @param maxCount safety valve to limit number items to read. Typically set to zero to keep unlimited. - * @return map of session ids to {@link IndexBuildProto.IndexingHeartbeat} + * @return map of session ids to {@link IndexBuildProto.IndexBuildHeartbeat} */ @API(API.Status.EXPERIMENTAL) - public Map getIndexingHeartbeats(int maxCount) { + public Map getIndexingHeartbeats(int maxCount) { return asyncToSync(FDBStoreTimer.Waits.WAIT_INDEX_READ_HEARTBEATS, getIndexer().getIndexingHeartbeats(maxCount)); } diff --git a/fdb-record-layer-core/src/main/proto/index_build.proto b/fdb-record-layer-core/src/main/proto/index_build.proto index d9d1186038..8b4f4dc212 100644 --- a/fdb-record-layer-core/src/main/proto/index_build.proto +++ b/fdb-record-layer-core/src/main/proto/index_build.proto @@ -47,7 +47,7 @@ message IndexBuildIndexingStamp { optional string blockID = 7; // optional, a short string that describes the reason for the block. } - message IndexingHeartbeat { + message IndexBuildHeartbeat { required IndexBuildIndexingStamp.Method method = 1; required int64 genesisTimeMilliseconds = 2; required int64 heartbeatTimeMilliseconds = 3; diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java new file mode 100644 index 0000000000..567a95646a --- /dev/null +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java @@ -0,0 +1,101 @@ +/* + * OnlineIndexingHeartbeatTest.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.record.IndexBuildProto; +import com.apple.foundationdb.record.metadata.Index; +import com.apple.foundationdb.record.metadata.IndexOptions; +import com.apple.foundationdb.record.metadata.IndexTypes; +import com.apple.foundationdb.record.metadata.expressions.EmptyKeyExpression; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Test; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.stream.Collectors; + +import static com.apple.foundationdb.record.metadata.Key.Expressions.field; + +/** + * Verify indexing heartbeat activity (query & clear). + */ +public class OnlineIndexingHeartbeatTest extends OnlineIndexerTest { + @Test + void testHeartbeatLowLevel() { + List indexes = new ArrayList<>(); + indexes.add(new Index("indexA", field("num_value_2"), EmptyKeyExpression.EMPTY, IndexTypes.VALUE, IndexOptions.UNIQUE_OPTIONS)); + indexes.add(new Index("indexB", field("num_value_3_indexed"), IndexTypes.VALUE)); + FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(indexes); + + final int count = 10; + IndexingHeartbeat[] heartbeats = new IndexingHeartbeat[count]; + for (int i = 0; i < count; i++) { + heartbeats[i] = new IndexingHeartbeat(UUID.randomUUID(), IndexBuildProto.IndexBuildIndexingStamp.Method.BY_INDEX, 100 + i); + } + + openSimpleMetaData(hook); + try (FDBRecordContext context = openContext()) { + for (var heartbeat : heartbeats) { + heartbeat.updateHeartbeat(recordStore, indexes.get(0)); + heartbeat.updateHeartbeat(recordStore, indexes.get(1)); + } + context.commit(); + } + + // Verify query/clear operation + try (OnlineIndexer indexer = newIndexerBuilder(indexes.get(0)).build()) { + // Query, unlimited + Map queried = indexer.getIndexingHeartbeats(0); + Assertions.assertThat(queried).hasSize(count); + Assertions.assertThat(queried.keySet()) + .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(ht -> ht.sessionId).collect(Collectors.toList())); + + // Query, partial + queried = indexer.getIndexingHeartbeats(5); + Assertions.assertThat(queried).hasSize(5); + + // clear, partial + int countDeleted = indexer.clearIndexingHeartbeats(0, 7); + Assertions.assertThat(countDeleted).isEqualTo(7); + queried = indexer.getIndexingHeartbeats(5); + Assertions.assertThat(queried).hasSize(3); + } + + // Verify that the previous clear does not affect other index + try (OnlineIndexer indexer = newIndexerBuilder(indexes.get(1)).build()) { + Map queried = indexer.getIndexingHeartbeats(100); + Assertions.assertThat(queried).hasSize(count); + Assertions.assertThat(queried.keySet()) + .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(ht -> ht.sessionId).collect(Collectors.toList())); + + // clear all + int countDeleted = indexer.clearIndexingHeartbeats(0, 0); + Assertions.assertThat(countDeleted).isEqualTo(count); + + // verify empty + queried = indexer.getIndexingHeartbeats(0); + Assertions.assertThat(queried).isEmpty(); + } + } +} From 67bd2dfb70d2282f4fb4e89d45fd0dd492d1d764 Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Fri, 8 Aug 2025 21:17:17 -0400 Subject: [PATCH 10/26] Add two more tests --- .../OnlineIndexingHeartbeatTest.java | 84 ++++++++++++++++++- 1 file changed, 83 insertions(+), 1 deletion(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java index 567a95646a..7e9832884a 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java @@ -21,10 +21,12 @@ package com.apple.foundationdb.record.provider.foundationdb; import com.apple.foundationdb.record.IndexBuildProto; +import com.apple.foundationdb.record.RecordCoreException; import com.apple.foundationdb.record.metadata.Index; import com.apple.foundationdb.record.metadata.IndexOptions; import com.apple.foundationdb.record.metadata.IndexTypes; import com.apple.foundationdb.record.metadata.expressions.EmptyKeyExpression; +import com.apple.foundationdb.tuple.Tuple; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; @@ -33,14 +35,20 @@ import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; +import java.util.stream.IntStream; import static com.apple.foundationdb.record.metadata.Key.Expressions.field; +import static org.junit.jupiter.api.Assertions.assertThrows; +import static org.junit.jupiter.api.Assertions.assertTrue; /** * Verify indexing heartbeat activity (query & clear). */ -public class OnlineIndexingHeartbeatTest extends OnlineIndexerTest { +class OnlineIndexingHeartbeatTest extends OnlineIndexerTest { + + @Test void testHeartbeatLowLevel() { List indexes = new ArrayList<>(); @@ -98,4 +106,78 @@ void testHeartbeatLowLevel() { Assertions.assertThat(queried).isEmpty(); } } + + @Test + void testMutualIndexersHeartbeatsClearAfterBuild() { + // Assert that the heartbeats are cleared after building + List indexes = new ArrayList<>(); + indexes.add(new Index("indexA", field("num_value_2"), EmptyKeyExpression.EMPTY, IndexTypes.VALUE, IndexOptions.UNIQUE_OPTIONS)); + indexes.add(new Index("indexC", field("num_value_unique"), EmptyKeyExpression.EMPTY, IndexTypes.VALUE, IndexOptions.UNIQUE_OPTIONS)); + int numRecords = 77; + populateData(numRecords); + int boundarySize = 23; + final List boundariesList = getBoundariesList(numRecords, boundarySize); + FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(indexes); + openSimpleMetaData(hook); + disableAll(indexes); + + IntStream.rangeClosed(1, 5).parallel().forEach(i -> { + try (OnlineIndexer indexer = newIndexerBuilder(indexes) + .setIndexingPolicy(OnlineIndexer.IndexingPolicy.newBuilder() + .setMutualIndexingBoundaries(boundariesList)) + .build()) { + indexer.buildIndex(); + } + }); + + for (Index index : indexes) { + try (OnlineIndexer indexer = newIndexerBuilder(index).build()) { + Assertions.assertThat(indexer.getIndexingHeartbeats(0)).isEmpty(); + } + } + } + + @Test + void testMutualIndexersHeartbeatsClearAfterCrash() { + // Assert that the heartbeats are cleared after crash + List indexes = new ArrayList<>(); + indexes.add(new Index("indexA", field("num_value_2"), EmptyKeyExpression.EMPTY, IndexTypes.VALUE, IndexOptions.UNIQUE_OPTIONS)); + indexes.add(new Index("indexC", field("num_value_unique"), EmptyKeyExpression.EMPTY, IndexTypes.VALUE, IndexOptions.UNIQUE_OPTIONS)); + int numRecords = 98; + populateData(numRecords); + int boundarySize = 20; + final List boundariesList = getBoundariesList(numRecords, boundarySize); + FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(indexes); + openSimpleMetaData(hook); + disableAll(indexes); + + final String testThrowMsg = "Intentionally crash during test"; + IntStream.rangeClosed(1, 9).parallel().forEach(i -> { + final AtomicLong counter = new AtomicLong(0); + try (OnlineIndexer indexer = newIndexerBuilder(indexes) + .setIndexingPolicy(OnlineIndexer.IndexingPolicy.newBuilder() + .setMutualIndexingBoundaries(boundariesList) + .build()) + .setConfigLoader(old -> { + // Unfortunately, we cannot verify that at least one heartbeat exists from this + // block, as it would have been nesting "asyncToSync" functions. But there are other tests + // that verify the "sync lock" functionality. + if (counter.incrementAndGet() > 2) { + throw new RecordCoreException(testThrowMsg); + } + return old; + }) + .build()) { + RecordCoreException e = assertThrows(RecordCoreException.class, indexer::buildIndex); + assertTrue(e.getMessage().contains(testThrowMsg)); + } + }); + + for (Index index : indexes) { + try (OnlineIndexer indexer = newIndexerBuilder(index).build()) { + Assertions.assertThat(indexer.getIndexingHeartbeats(0)).isEmpty(); + } + } + } + } From 07d945ba0ae3fca3c686a4b7ed8369b645d02605 Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Fri, 8 Aug 2025 21:35:12 -0400 Subject: [PATCH 11/26] Add a comment.. --- .../record/provider/foundationdb/IndexingCommon.java | 1 - .../record/provider/foundationdb/IndexingSubspaces.java | 1 + 2 files changed, 1 insertion(+), 1 deletion(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingCommon.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingCommon.java index 8cc380b2bd..3eba5265b5 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingCommon.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingCommon.java @@ -49,7 +49,6 @@ @API(API.Status.INTERNAL) public class IndexingCommon { - // TODO? get uuid from caller to allow lock takeover private final UUID uuid = UUID.randomUUID(); @Nonnull private final FDBDatabaseRunner runner; diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingSubspaces.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingSubspaces.java index da885668b3..925da91e04 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingSubspaces.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingSubspaces.java @@ -209,6 +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 the clearing within the same transaction. context.clear(Range.startsWith(indexheartbeatSubspace(store, index).pack())); } } From e55888bc9998318924863ca2dcef99e5197bac5e Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Mon, 11 Aug 2025 08:40:18 -0400 Subject: [PATCH 12/26] Deprecated sync lock API functions --- .../record/provider/foundationdb/IndexingBase.java | 12 ++++++------ .../provider/foundationdb/IndexingByIndex.java | 2 +- .../provider/foundationdb/IndexingCommon.java | 8 ++++---- .../provider/foundationdb/IndexingHeartbeat.java | 14 +++++++------- .../provider/foundationdb/OnlineIndexer.java | 9 +-------- .../foundationdb/OnlineIndexerBuildIndexTest.java | 2 -- .../foundationdb/OnlineIndexingHeartbeatTest.java | 4 ++-- 7 files changed, 21 insertions(+), 30 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java index 458ba02165..77864f81ba 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java @@ -152,14 +152,14 @@ protected CompletableFuture> recordIfInIndexedTypes(FDB @SuppressWarnings("PMD.CloseResource") public CompletableFuture buildIndexAsync(boolean markReadable) { KeyValueLogMessage message = KeyValueLogMessage.build("build index online", - LogMessageKeys.SHOULD_MARK_READABLE, markReadable); + LogMessageKeys.SHOULD_MARK_READABLE, markReadable, + LogMessageKeys.INDEXER_ID, common.getIndexerId()); long startNanos = System.nanoTime(); FDBDatabaseRunner runner = getRunner(); final FDBStoreTimer timer = runner.getTimer(); if ( timer != null) { lastProgressSnapshot = StoreTimerSnapshot.from(timer); } - message.addKeyAndValue(LogMessageKeys.SESSION_ID, common.getUuid()); AtomicReference indexingException = new AtomicReference<>(null); return handleStateAndDoBuildIndexAsync(markReadable, message) .handle((ret, ex) -> { @@ -362,7 +362,7 @@ public void enforceStampOverwrite() { private CompletableFuture setIndexingTypeOrThrow(FDBRecordStore store, boolean continuedBuild) { // continuedBuild is set if this session isn't a continuation of a previous indexing IndexBuildProto.IndexBuildIndexingStamp indexingTypeStamp = getIndexingTypeStamp(store); - heartbeat = new IndexingHeartbeat(common.getUuid(), indexingTypeStamp.getMethod(), common.config.getLeaseLengthMillis()); + heartbeat = new IndexingHeartbeat(common.getIndexerId(), indexingTypeStamp.getMethod(), common.config.getLeaseLengthMillis()); return forEachTargetIndex(index -> setIndexingTypeOrThrow(store, continuedBuild, index, indexingTypeStamp)); } @@ -506,7 +506,7 @@ RecordCoreException newPartlyBuiltException(boolean continuedBuild, IndexBuildProto.IndexBuildIndexingStamp savedStamp, IndexBuildProto.IndexBuildIndexingStamp expectedStamp, Index index) { - return new PartlyBuiltException(savedStamp, expectedStamp, index, common.getUuid(), + return new PartlyBuiltException(savedStamp, expectedStamp, index, common.getIndexerId(), savedStamp.getBlock() ? "This index was partly built, and blocked" : "This index was partly built by another method"); @@ -901,7 +901,7 @@ private void validateTypeStamp(final IndexBuildProto.IndexBuildIndexingStamp typ } if (typeStamp == null || typeStamp.getMethod() != expectedTypeStamp.getMethod() || isTypeStampBlocked(typeStamp)) { throw new PartlyBuiltException(typeStamp, expectedTypeStamp, - index, common.getUuid(), "Indexing stamp had changed"); + index, common.getIndexerId(), "Indexing stamp had changed"); } } @@ -1041,7 +1041,7 @@ protected void validateOrThrowEx(boolean isValid, @Nonnull String msg) { throw new ValidationException(msg, LogMessageKeys.INDEX_NAME, common.getTargetIndexesNames(), LogMessageKeys.SOURCE_INDEX, policy.getSourceIndex(), - LogMessageKeys.INDEXER_ID, common.getUuid()); + LogMessageKeys.INDEXER_ID, common.getIndexerId()); } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingByIndex.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingByIndex.java index e2a85202df..d76916be57 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingByIndex.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingByIndex.java @@ -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 diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingCommon.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingCommon.java index 3eba5265b5..4b9b44e3fa 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingCommon.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingCommon.java @@ -49,7 +49,7 @@ @API(API.Status.INTERNAL) public class IndexingCommon { - private final UUID uuid = UUID.randomUUID(); + private final UUID indexerId = UUID.randomUUID(); @Nonnull private final FDBDatabaseRunner runner; @@ -135,8 +135,8 @@ private void fillTargetIndexers(@Nonnull List targetIndexes, @Nullable Co } } - public UUID getUuid() { - return uuid; + public UUID getIndexerId() { + return indexerId; } public List indexLogMessageKeyValues() { @@ -156,7 +156,7 @@ public List 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); diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java index 24d148fa65..7964df45c0 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java @@ -39,20 +39,20 @@ public class IndexingHeartbeat { // [prefix, xid] -> [indexing-type, genesis time, heartbeat time] - final UUID sessionId; + final UUID indexerId; final IndexBuildProto.IndexBuildIndexingStamp.Method indexingMethod; final long genesisTimeMilliseconds; final long leaseLength; - public IndexingHeartbeat(final UUID sessionId, IndexBuildProto.IndexBuildIndexingStamp.Method indexingMethod, long leaseLength) { - this.sessionId = sessionId; + public IndexingHeartbeat(final UUID indexerId, IndexBuildProto.IndexBuildIndexingStamp.Method indexingMethod, long leaseLength) { + this.indexerId = indexerId; this.indexingMethod = indexingMethod; this.leaseLength = leaseLength; this.genesisTimeMilliseconds = nowMilliseconds(); } public void updateHeartbeat(@Nonnull FDBRecordStore store, @Nonnull Index index) { - byte[] key = IndexingSubspaces.indexheartbeatSubspace(store, index, sessionId).pack(); + byte[] key = IndexingSubspaces.indexheartbeatSubspace(store, index, indexerId).pack(); byte[] value = IndexBuildProto.IndexBuildHeartbeat.newBuilder() .setMethod(indexingMethod) .setGenesisTimeMilliseconds(genesisTimeMilliseconds) @@ -93,13 +93,13 @@ private void validateNonCompetingHeartbeat(KeyValue kv, long now) { return; } final UUID otherSessionId = keyTuple.getUUID(keyTuple.size() - 1); - if (!otherSessionId.equals(this.sessionId)) { + if (!otherSessionId.equals(this.indexerId)) { try { final IndexBuildProto.IndexBuildHeartbeat otherHeartbeat = IndexBuildProto.IndexBuildHeartbeat.parseFrom(kv.getValue()); final long age = now - otherHeartbeat.getHeartbeatTimeMilliseconds(); if (age > 0 && age < leaseLength) { throw new SynchronizedSessionLockedException("Failed to initialize the session because of an existing session in progress") - .addLogInfo(LogMessageKeys.SESSION_ID, sessionId) + .addLogInfo(LogMessageKeys.SESSION_ID, indexerId) .addLogInfo(LogMessageKeys.EXISTING_SESSION_ID, otherSessionId) .addLogInfo(LogMessageKeys.AGE_MILLISECONDS, age) .addLogInfo(LogMessageKeys.TIME_LIMIT_MILLIS, leaseLength); @@ -111,7 +111,7 @@ private void validateNonCompetingHeartbeat(KeyValue kv, long now) { } public void clearHeartbeat(@Nonnull FDBRecordStore store, @Nonnull Index index) { - store.ensureContextActive().clear(IndexingSubspaces.indexheartbeatSubspace(store, index, sessionId).pack()); + store.ensureContextActive().clear(IndexingSubspaces.indexheartbeatSubspace(store, index, indexerId).pack()); } public static CompletableFuture> getIndexingHeartbeats(FDBRecordStore store, Index index, int maxCount) { diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java index befa7b04bf..f03dc00a8d 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java @@ -502,14 +502,7 @@ public static CompletableFuture checkAnyOngoingOnlineIndexBuildsAsync(@ /** * Builds an index across multiple transactions. - *

- * If it is set to use synchronized sessions, it stops with {@link com.apple.foundationdb.synchronizedsession.SynchronizedSessionLockedException} - * when there is another runner actively working on the same index. It first checks and updates index states and - * clear index data respecting the {@link IndexStatePrecondition} being set. It then builds the index across - * multiple transactions honoring the rate-limiting parameters set in the constructor of this class. It also retries - * any retriable errors that it encounters while it runs the build. At the end, it marks the index readable in the - * store. - *

+ * This is a slow and retrying operation that is intended to be executed by background processes. * @return a future that will be ready when the build has completed * @throws com.apple.foundationdb.synchronizedsession.SynchronizedSessionLockedException the build is stopped * because there may be another build running actively on this index. diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerBuildIndexTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerBuildIndexTest.java index 6ac3a98597..ecb555210b 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerBuildIndexTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerBuildIndexTest.java @@ -21,7 +21,6 @@ package com.apple.foundationdb.record.provider.foundationdb; import com.apple.foundationdb.async.AsyncUtil; -import com.apple.foundationdb.async.MoreAsyncUtil; import com.apple.foundationdb.async.RangeSet; import com.apple.foundationdb.record.IndexState; import com.apple.foundationdb.record.logging.KeyValueLogMessage; @@ -34,7 +33,6 @@ import com.apple.foundationdb.tuple.Tuple; import com.apple.test.RandomizedTestUtils; import com.google.protobuf.Message; -import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java index 7e9832884a..bc023d018a 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java @@ -77,7 +77,7 @@ void testHeartbeatLowLevel() { Map queried = indexer.getIndexingHeartbeats(0); Assertions.assertThat(queried).hasSize(count); Assertions.assertThat(queried.keySet()) - .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(ht -> ht.sessionId).collect(Collectors.toList())); + .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(ht -> ht.indexerId).collect(Collectors.toList())); // Query, partial queried = indexer.getIndexingHeartbeats(5); @@ -95,7 +95,7 @@ void testHeartbeatLowLevel() { Map queried = indexer.getIndexingHeartbeats(100); Assertions.assertThat(queried).hasSize(count); Assertions.assertThat(queried.keySet()) - .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(ht -> ht.sessionId).collect(Collectors.toList())); + .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(ht -> ht.indexerId).collect(Collectors.toList())); // clear all int countDeleted = indexer.clearIndexingHeartbeats(0, 0); From 7a98c1175729f401a1ad200cfe3e25f9099aebfa Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Mon, 11 Aug 2025 10:13:27 -0400 Subject: [PATCH 13/26] Some cleanup --- .../record/logging/LogMessageKeys.java | 3 +-- .../provider/foundationdb/IndexingBase.java | 16 ++++++++------ .../foundationdb/IndexingHeartbeat.java | 21 ++++++++++++------- .../foundationdb/IndexingSubspaces.java | 8 +++---- .../provider/foundationdb/OnlineIndexer.java | 2 +- .../src/main/proto/index_build.proto | 1 - 6 files changed, 29 insertions(+), 22 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/logging/LogMessageKeys.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/logging/LogMessageKeys.java index 9e2ffc927c..090cce3088 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/logging/LogMessageKeys.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/logging/LogMessageKeys.java @@ -163,8 +163,7 @@ public enum LogMessageKeys { RECORDS_PER_SECOND, DOCUMENT, SESSION_ID, - EXISTING_SESSION_ID, - INDEXER_SESSION_ID, + EXISTING_INDEXER_ID, INDEXER_ID, INDEX_STATE_PRECONDITION, INITIAL_INDEX_STATE, diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java index 77864f81ba..b24b991950 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java @@ -305,7 +305,6 @@ public CompletableFuture markReadableIfBuilt() { ).thenApply(ignore -> allReadable.get()), common.indexLogMessageKeyValues("IndexingBase::markReadableIfBuilt")); } - @Nonnull public CompletableFuture markIndexReadable(boolean markReadablePlease) { if (!markReadablePlease) { @@ -861,6 +860,14 @@ private CompletableFuture clearHeartbeats() { .thenAccept(ignore -> heartbeat = null); } + private void clearHeartbeats(FDBRecordStore store) { + if (heartbeat != null) { + for (Index index : common.getTargetIndexes()) { + clearHeartbeatSingleTarget(store, index); + } + } + } + private CompletableFuture clearHeartbeatSingleTarget(Index index) { return getRunner().runAsync(context -> common.getRecordStoreBuilder().copyBuilder().setContext(context).openAsync() @@ -1027,11 +1034,8 @@ public CompletableFuture rebuildIndexAsync(@Nonnull FDBRecordStore store) })) .thenCompose(vignore -> setIndexingTypeOrThrow(store, false)) .thenCompose(vignore -> rebuildIndexInternalAsync(store)) - .whenComplete((ignore, ignoreEx) -> { - for (Index index: common.getTargetIndexes()) { - clearHeartbeatSingleTarget(store, index); - } - }); + // If any of the indexes' heartbeats, for any reason, was not cleared during "mark readable", clear it here + .whenComplete((ignore, ignoreEx) -> clearHeartbeats(store)); } abstract CompletableFuture rebuildIndexInternalAsync(FDBRecordStore store); diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java index 7964df45c0..b19c157399 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java @@ -66,6 +66,7 @@ public CompletableFuture checkAndUpdateHeartbeat(@Nonnull FDBRecordStore s switch (indexingMethod) { case SCRUB_REPAIR: case MUTUAL_BY_RECORDS: + updateHeartbeat(store, index); return AsyncUtil.DONE; case BY_RECORDS: @@ -79,7 +80,11 @@ public CompletableFuture checkAndUpdateHeartbeat(@Nonnull FDBRecordStore s } validateNonCompetingHeartbeat(iterator.next(), nowMilliseconds()); return true; - })); + })) + .thenApply(ignore -> { + updateHeartbeat(store, index); + return null; + }); default: throw new IndexingBase.ValidationException("invalid indexing method", @@ -92,15 +97,15 @@ private void validateNonCompetingHeartbeat(KeyValue kv, long now) { if (keyTuple.size() < 2) { // expecting 8 return; } - final UUID otherSessionId = keyTuple.getUUID(keyTuple.size() - 1); - if (!otherSessionId.equals(this.indexerId)) { + final UUID otherIndexerId = keyTuple.getUUID(keyTuple.size() - 1); + if (!otherIndexerId.equals(this.indexerId)) { try { final IndexBuildProto.IndexBuildHeartbeat otherHeartbeat = IndexBuildProto.IndexBuildHeartbeat.parseFrom(kv.getValue()); final long age = now - otherHeartbeat.getHeartbeatTimeMilliseconds(); if (age > 0 && age < leaseLength) { throw new SynchronizedSessionLockedException("Failed to initialize the session because of an existing session in progress") - .addLogInfo(LogMessageKeys.SESSION_ID, indexerId) - .addLogInfo(LogMessageKeys.EXISTING_SESSION_ID, otherSessionId) + .addLogInfo(LogMessageKeys.INDEXER_ID, indexerId) + .addLogInfo(LogMessageKeys.EXISTING_INDEXER_ID, otherIndexerId) .addLogInfo(LogMessageKeys.AGE_MILLISECONDS, age) .addLogInfo(LogMessageKeys.TIME_LIMIT_MILLIS, leaseLength); } @@ -131,13 +136,13 @@ public static CompletableFuture> if (keyTuple.size() < 2) { // expecting 8 return true; // ignore, next } - final UUID otherSessionId = keyTuple.getUUID(keyTuple.size() - 1); + final UUID otherIndexerId = keyTuple.getUUID(keyTuple.size() - 1); try { final IndexBuildProto.IndexBuildHeartbeat otherHeartbeat = IndexBuildProto.IndexBuildHeartbeat.parseFrom(kv.getValue()); - ret.put(otherSessionId, otherHeartbeat); + ret.put(otherIndexerId, otherHeartbeat); } catch (InvalidProtocolBufferException e) { // put a NONE heartbeat to indicate an invalid item - ret.put(otherSessionId, IndexBuildProto.IndexBuildHeartbeat.newBuilder() + ret.put(otherIndexerId, IndexBuildProto.IndexBuildHeartbeat.newBuilder() .setMethod(IndexBuildProto.IndexBuildIndexingStamp.Method.NONE) .build()); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingSubspaces.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingSubspaces.java index 925da91e04..acc7363303 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingSubspaces.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingSubspaces.java @@ -100,12 +100,12 @@ public static Subspace indexheartbeatSubspace(@Nonnull FDBRecordStoreBase sto * Subspace that stores the indexing heartbeat. * @param store store * @param index index - * @param sessionId session id + * @param indexerId session id * @return subspace */ @Nonnull - public static Subspace indexheartbeatSubspace(@Nonnull FDBRecordStoreBase store, @Nonnull Index index, @Nonnull UUID sessionId) { - return indexheartbeatSubspace(store, index).subspace(Tuple.from(sessionId)); + public static Subspace indexheartbeatSubspace(@Nonnull FDBRecordStoreBase store, @Nonnull Index index, @Nonnull UUID indexerId) { + return indexheartbeatSubspace(store, index).subspace(Tuple.from(indexerId)); } /** @@ -209,7 +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 the clearing within the same transaction. + // 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())); } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java index f03dc00a8d..f0972513b8 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java @@ -1251,7 +1251,7 @@ public static class Builder { private DesiredAction ifReadable = DesiredAction.CONTINUE; private boolean doAllowUniquePendingState = false; private Set allowedTakeoverSet = null; - private long checkIndexingStampFrequency = 30_000; + private long checkIndexingStampFrequency = 10_000; private boolean useMutualIndexing = false; private List useMutualIndexingBoundaries = null; private boolean allowUnblock = false; diff --git a/fdb-record-layer-core/src/main/proto/index_build.proto b/fdb-record-layer-core/src/main/proto/index_build.proto index 8b4f4dc212..17a3bc0402 100644 --- a/fdb-record-layer-core/src/main/proto/index_build.proto +++ b/fdb-record-layer-core/src/main/proto/index_build.proto @@ -36,7 +36,6 @@ message IndexBuildIndexingStamp { MUTUAL_BY_RECORDS = 5; // scan records, build multiple target indexes, while allowing multiple indexer processes NONE = 6; // return this stamp in query when null - never to be written in the DB }; - optional Method method = 1; optional bytes source_index_subspace_key = 2; // relevant only with BY_INDEX method optional int32 source_index_last_modified_version = 3; // only with BY_INDEX method From ea2ce29abaa184d9be4cb6c11a24a953275ccd92 Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Mon, 11 Aug 2025 11:50:46 -0400 Subject: [PATCH 14/26] At genesis, first check indexing type, then heartbeat --- .../record/provider/foundationdb/IndexingBase.java | 10 +++++----- 1 file changed, 5 insertions(+), 5 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java index b24b991950..08bbfff4c2 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java @@ -363,18 +363,18 @@ private CompletableFuture setIndexingTypeOrThrow(FDBRecordStore store, boo IndexBuildProto.IndexBuildIndexingStamp indexingTypeStamp = getIndexingTypeStamp(store); heartbeat = new IndexingHeartbeat(common.getIndexerId(), indexingTypeStamp.getMethod(), common.config.getLeaseLengthMillis()); - return forEachTargetIndex(index -> setIndexingTypeOrThrow(store, continuedBuild, index, indexingTypeStamp)); + return forEachTargetIndex(index -> setIndexingTypeOrThrow(store, continuedBuild, index, indexingTypeStamp) + .thenApply(ignore -> updateHeartbeat(true, store, index))); } @Nonnull private CompletableFuture setIndexingTypeOrThrow(FDBRecordStore store, boolean continuedBuild, Index index, IndexBuildProto.IndexBuildIndexingStamp newStamp) { - final CompletableFuture checkUpdateHeartbeat = updateHeartbeat(true, store, index); if (forceStampOverwrite && !continuedBuild) { // Fresh session + overwrite = no questions asked store.saveIndexingTypeStamp(index, newStamp); - return checkUpdateHeartbeat; + return AsyncUtil.DONE ; } - return checkUpdateHeartbeat.thenCompose(ignore -> store.loadIndexingTypeStampAsync(index) + return store.loadIndexingTypeStampAsync(index) .thenCompose(savedStamp -> { if (savedStamp == null) { if (continuedBuild && newStamp.getMethod() != @@ -416,7 +416,7 @@ private CompletableFuture setIndexingTypeOrThrow(FDBRecordStore store, boo } // fall down to exception throw newPartlyBuiltException(continuedBuild, savedStamp, newStamp, index); - })); + }); } private boolean shouldAllowTypeConversionContinue(IndexBuildProto.IndexBuildIndexingStamp newStamp, IndexBuildProto.IndexBuildIndexingStamp savedStamp) { From 08badca48f8dedb5260c022e6cf5bd68e948cb46 Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Mon, 11 Aug 2025 15:45:01 -0400 Subject: [PATCH 15/26] Use subspace(...).unpack(..) --- .../provider/foundationdb/IndexingBase.java | 2 +- .../foundationdb/IndexingHeartbeat.java | 54 ++++++++----------- .../OnlineIndexingHeartbeatTest.java | 1 - 3 files changed, 24 insertions(+), 33 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java index 08bbfff4c2..3b946711f4 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java @@ -364,7 +364,7 @@ private CompletableFuture setIndexingTypeOrThrow(FDBRecordStore store, boo heartbeat = new IndexingHeartbeat(common.getIndexerId(), indexingTypeStamp.getMethod(), common.config.getLeaseLengthMillis()); return forEachTargetIndex(index -> setIndexingTypeOrThrow(store, continuedBuild, index, indexingTypeStamp) - .thenApply(ignore -> updateHeartbeat(true, store, index))); + .thenCompose(ignore -> updateHeartbeat(true, store, index))); } @Nonnull diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java index b19c157399..0c0dedb07b 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java @@ -27,7 +27,6 @@ import com.apple.foundationdb.record.logging.LogMessageKeys; import com.apple.foundationdb.record.metadata.Index; import com.apple.foundationdb.synchronizedsession.SynchronizedSessionLockedException; -import com.apple.foundationdb.tuple.Tuple; import com.google.protobuf.InvalidProtocolBufferException; import javax.annotation.Nonnull; @@ -73,12 +72,29 @@ public CompletableFuture checkAndUpdateHeartbeat(@Nonnull FDBRecordStore s case MULTI_TARGET_BY_RECORDS: case BY_INDEX: final AsyncIterator iterator = heartbeatsIterator(store, index); + final long now = nowMilliseconds(); return AsyncUtil.whileTrue(() -> iterator.onHasNext() .thenApply(hasNext -> { if (!hasNext) { return false; } - validateNonCompetingHeartbeat(iterator.next(), nowMilliseconds()); + 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) { + 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); + } + } + } catch (InvalidProtocolBufferException e) { + throw new RuntimeException(e); + } return true; })) .thenApply(ignore -> { @@ -92,29 +108,6 @@ public CompletableFuture checkAndUpdateHeartbeat(@Nonnull FDBRecordStore s } } - private void validateNonCompetingHeartbeat(KeyValue kv, long now) { - final Tuple keyTuple = Tuple.fromBytes(kv.getKey()); - if (keyTuple.size() < 2) { // expecting 8 - return; - } - final UUID otherIndexerId = keyTuple.getUUID(keyTuple.size() - 1); - if (!otherIndexerId.equals(this.indexerId)) { - try { - final IndexBuildProto.IndexBuildHeartbeat otherHeartbeat = IndexBuildProto.IndexBuildHeartbeat.parseFrom(kv.getValue()); - final long age = now - otherHeartbeat.getHeartbeatTimeMilliseconds(); - if (age > 0 && age < leaseLength) { - 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); - } - } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); - } - } - } - public void clearHeartbeat(@Nonnull FDBRecordStore store, @Nonnull Index index) { store.ensureContextActive().clear(IndexingSubspaces.indexheartbeatSubspace(store, index, indexerId).pack()); } @@ -132,11 +125,7 @@ public static CompletableFuture> return false; } final KeyValue kv = iterator.next(); - final Tuple keyTuple = Tuple.fromBytes(kv.getKey()); - if (keyTuple.size() < 2) { // expecting 8 - return true; // ignore, next - } - final UUID otherIndexerId = keyTuple.getUUID(keyTuple.size() - 1); + final UUID otherIndexerId = heartbeatKeyToIndexerId(store, index, kv.getKey()); try { final IndexBuildProto.IndexBuildHeartbeat otherHeartbeat = IndexBuildProto.IndexBuildHeartbeat.parseFrom(kv.getValue()); ret.put(otherIndexerId, otherHeartbeat); @@ -183,10 +172,13 @@ public static CompletableFuture clearIndexingHeartbeats(@Nonnull FDBRec .thenApply(ignore -> deleteCount.get()); } - public static AsyncIterator heartbeatsIterator(FDBRecordStore store, Index index) { + private static AsyncIterator heartbeatsIterator(FDBRecordStore store, Index index) { return store.getContext().ensureActive().snapshot().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(); diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java index bc023d018a..abbca7975d 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java @@ -48,7 +48,6 @@ */ class OnlineIndexingHeartbeatTest extends OnlineIndexerTest { - @Test void testHeartbeatLowLevel() { List indexes = new ArrayList<>(); From 89e071f114dc058ffa8a5346e4d6ac09e87cb50d Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Wed, 13 Aug 2025 09:19:38 -0400 Subject: [PATCH 16/26] Verify no leftover heartbeats after any singleRebuild --- .../provider/foundationdb/OnlineIndexerBuildIndexTest.java | 6 ++++++ 1 file changed, 6 insertions(+) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerBuildIndexTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerBuildIndexTest.java index ecb555210b..2d7d1cea87 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerBuildIndexTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerBuildIndexTest.java @@ -22,6 +22,7 @@ import com.apple.foundationdb.async.AsyncUtil; import com.apple.foundationdb.async.RangeSet; +import com.apple.foundationdb.record.IndexBuildProto; import com.apple.foundationdb.record.IndexState; import com.apple.foundationdb.record.logging.KeyValueLogMessage; import com.apple.foundationdb.record.logging.LogMessageKeys; @@ -45,6 +46,7 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.UUID; import java.util.concurrent.CompletableFuture; import java.util.concurrent.CompletionException; import java.util.concurrent.ThreadLocalRandom; @@ -290,6 +292,10 @@ void singleRebuild( )); } } + try (OnlineIndexer indexBuilder = newIndexerBuilder(index).build()) { + final Map heartbeats = indexBuilder.getIndexingHeartbeats(0); + assertTrue(heartbeats.isEmpty()); + } KeyValueLogMessage msg = KeyValueLogMessage.build("building index - completed", TestLogMessageKeys.INDEX, index); msg.addKeysAndValues(timer.getKeysAndValues()); LOGGER.info(msg.toString()); From aa79474cab568d10efd14973a47d83f434aa87b9 Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Wed, 13 Aug 2025 15:39:55 -0400 Subject: [PATCH 17/26] Apply Scott's requested changes --- .../provider/foundationdb/FDBRecordStore.java | 1 + .../provider/foundationdb/IndexingBase.java | 6 +- .../foundationdb/IndexingHeartbeat.java | 52 ++- .../OnlineIndexOperationConfig.java | 18 +- .../src/main/proto/index_build.proto | 2 +- .../IndexingHeartbeatLowLevelTest.java | 372 ++++++++++++++++++ .../OnlineIndexerIndexFromIndexTest.java | 69 ++++ .../foundationdb/OnlineIndexerSimpleTest.java | 28 ++ .../OnlineIndexingHeartbeatTest.java | 214 +++++++++- 9 files changed, 711 insertions(+), 51 deletions(-) create mode 100644 fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeatLowLevelTest.java diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java index a9a838ba33..e4c75e6c58 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java @@ -4966,6 +4966,7 @@ public void removeFormerIndex(FormerIndex formerIndex) { private void clearReadableIndexBuildData(Index index) { IndexingRangeSet.forIndexBuild(this, index).clear(); + IndexingHeartbeat.clearAllHeartbeats(this, index); } @SuppressWarnings("PMD.CloseResource") diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java index 3b946711f4..adc29115ca 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java @@ -361,7 +361,11 @@ public void enforceStampOverwrite() { private CompletableFuture setIndexingTypeOrThrow(FDBRecordStore store, boolean continuedBuild) { // continuedBuild is set if this session isn't a continuation of a previous indexing IndexBuildProto.IndexBuildIndexingStamp indexingTypeStamp = getIndexingTypeStamp(store); - heartbeat = new IndexingHeartbeat(common.getIndexerId(), indexingTypeStamp.getMethod(), common.config.getLeaseLengthMillis()); + final IndexBuildProto.IndexBuildIndexingStamp.Method method = indexingTypeStamp.getMethod(); + boolean allowMutual = + method == IndexBuildProto.IndexBuildIndexingStamp.Method.MUTUAL_BY_RECORDS || + method == IndexBuildProto.IndexBuildIndexingStamp.Method.SCRUB_REPAIR; + heartbeat = new IndexingHeartbeat(common.getIndexerId(), indexingTypeStamp.getMethod().toString(), common.config.getLeaseLengthMillis(), allowMutual); return forEachTargetIndex(index -> setIndexingTypeOrThrow(store, continuedBuild, index, indexingTypeStamp) .thenCompose(ignore -> updateHeartbeat(true, store, index))); diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java index 0c0dedb07b..fa0b050fcd 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java @@ -37,23 +37,25 @@ import java.util.concurrent.atomic.AtomicInteger; public class IndexingHeartbeat { - // [prefix, xid] -> [indexing-type, genesis time, heartbeat time] + // [prefix, indexerId] -> [indexing-type, genesis time, heartbeat time] final UUID indexerId; - final IndexBuildProto.IndexBuildIndexingStamp.Method indexingMethod; + final String info; final long genesisTimeMilliseconds; final long leaseLength; + final boolean allowMutual; - public IndexingHeartbeat(final UUID indexerId, IndexBuildProto.IndexBuildIndexingStamp.Method indexingMethod, long leaseLength) { + public IndexingHeartbeat(final UUID indexerId, String info, long leaseLength, boolean allowMutual) { this.indexerId = indexerId; - this.indexingMethod = indexingMethod; + 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() - .setMethod(indexingMethod) + .setInfo(info) .setGenesisTimeMilliseconds(genesisTimeMilliseconds) .setHeartbeatTimeMilliseconds(nowMilliseconds()) .build().toByteArray(); @@ -62,18 +64,14 @@ public void updateHeartbeat(@Nonnull FDBRecordStore store, @Nonnull Index index) public CompletableFuture checkAndUpdateHeartbeat(@Nonnull FDBRecordStore store, @Nonnull Index index) { // complete exceptionally if non-mutual, other exists - switch (indexingMethod) { - case SCRUB_REPAIR: - case MUTUAL_BY_RECORDS: - updateHeartbeat(store, index); - return AsyncUtil.DONE; + if (allowMutual) { + updateHeartbeat(store, index); + return AsyncUtil.DONE; + } - case BY_RECORDS: - case MULTI_TARGET_BY_RECORDS: - case BY_INDEX: - final AsyncIterator iterator = heartbeatsIterator(store, index); - final long now = nowMilliseconds(); - return AsyncUtil.whileTrue(() -> iterator.onHasNext() + final AsyncIterator iterator = heartbeatsIterator(store, index); + final long now = nowMilliseconds(); + return AsyncUtil.whileTrue(() -> iterator.onHasNext() .thenApply(hasNext -> { if (!hasNext) { return false; @@ -85,6 +83,7 @@ public CompletableFuture checkAndUpdateHeartbeat(@Nonnull FDBRecordStore s 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) @@ -97,21 +96,20 @@ public CompletableFuture checkAndUpdateHeartbeat(@Nonnull FDBRecordStore s } return true; })) - .thenApply(ignore -> { - updateHeartbeat(store, index); - return null; - }); - - default: - throw new IndexingBase.ValidationException("invalid indexing method", - LogMessageKeys.INDEXING_METHOD, indexingMethod); - } + .thenApply(ignore -> { + updateHeartbeat(store, index); + return null; + }); } 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> getIndexingHeartbeats(FDBRecordStore store, Index index, int maxCount) { final Map ret = new HashMap<>(); final AsyncIterator iterator = heartbeatsIterator(store, index); @@ -130,9 +128,9 @@ public static CompletableFuture> final IndexBuildProto.IndexBuildHeartbeat otherHeartbeat = IndexBuildProto.IndexBuildHeartbeat.parseFrom(kv.getValue()); ret.put(otherIndexerId, otherHeartbeat); } catch (InvalidProtocolBufferException e) { - // put a NONE heartbeat to indicate an invalid item + // Let the caller know about this invalid heartbeat. ret.put(otherIndexerId, IndexBuildProto.IndexBuildHeartbeat.newBuilder() - .setMethod(IndexBuildProto.IndexBuildIndexingStamp.Method.NONE) + .setInfo("<< Invalid Heartbeat >>") .build()); } return true; diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationConfig.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationConfig.java index 2c781e7d38..9a4ce2f5b1 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationConfig.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationConfig.java @@ -186,6 +186,19 @@ public static Builder newBuilder() { return new Builder(); } + + /** + * Not used anymore. + * @return always false; + * @deprecated see {@link Builder#setUseSynchronizedSession(boolean)} + */ + @API(API.Status.DEPRECATED) + @SuppressWarnings("PMD.AvoidUsingHardCodedIP") // version is not IP + @Deprecated(since = "4.4.3.0", forRemoval = true) + public boolean shouldUseSynchronizedSession() { + return false; + } + public long getLeaseLengthMillis() { return leaseLengthMillis; } @@ -496,8 +509,9 @@ public Builder setUseSynchronizedSession(boolean useSynchronizedSession) { } /** - * Set the lease length in milliseconds if the synchronized session is used. By default this is {@link #DEFAULT_LEASE_LENGTH_MILLIS}. - * @see com.apple.foundationdb.synchronizedsession.SynchronizedSession + * If the indexing session is not expected to be mutual, abort indexing if another session is active. This function + * defines the maximum age of another session's heartbeat to be considered an "active session". + * The default value is {@link #DEFAULT_LEASE_LENGTH_MILLIS}. * @param leaseLengthMillis length between last access and lease's end time in milliseconds * @return this builder */ diff --git a/fdb-record-layer-core/src/main/proto/index_build.proto b/fdb-record-layer-core/src/main/proto/index_build.proto index 17a3bc0402..c7929645cc 100644 --- a/fdb-record-layer-core/src/main/proto/index_build.proto +++ b/fdb-record-layer-core/src/main/proto/index_build.proto @@ -47,7 +47,7 @@ message IndexBuildIndexingStamp { } message IndexBuildHeartbeat { - required IndexBuildIndexingStamp.Method method = 1; + required string info = 1; required int64 genesisTimeMilliseconds = 2; required int64 heartbeatTimeMilliseconds = 3; } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeatLowLevelTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeatLowLevelTest.java new file mode 100644 index 0000000000..8d68005ecc --- /dev/null +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeatLowLevelTest.java @@ -0,0 +1,372 @@ +/* + * testIndexingHeartbeaLowLevel.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.record.IndexBuildProto; +import com.apple.foundationdb.record.RecordMetaData; +import com.apple.foundationdb.record.RecordMetaDataBuilder; +import com.apple.foundationdb.record.TestRecords1Proto; +import com.apple.foundationdb.record.metadata.Index; +import com.apple.foundationdb.record.metadata.IndexOptions; +import com.apple.foundationdb.record.metadata.IndexTypes; +import com.apple.foundationdb.record.metadata.expressions.EmptyKeyExpression; +import com.apple.foundationdb.record.metadata.expressions.GroupingKeyExpression; +import com.apple.foundationdb.record.metadata.expressions.VersionKeyExpression; +import com.apple.foundationdb.record.provider.foundationdb.keyspace.KeySpacePath; +import com.apple.foundationdb.record.test.FDBDatabaseExtension; +import com.apple.foundationdb.record.test.TestKeySpace; +import com.apple.foundationdb.record.test.TestKeySpacePathManagerExtension; +import com.google.protobuf.Descriptors; +import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.BeforeEach; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.extension.RegisterExtension; + +import javax.annotation.Nonnull; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.Map; +import java.util.UUID; +import java.util.concurrent.CompletionException; +import java.util.concurrent.TimeUnit; +import java.util.stream.Collectors; + +import static com.apple.foundationdb.record.metadata.Key.Expressions.concat; +import static com.apple.foundationdb.record.metadata.Key.Expressions.field; +import static org.junit.jupiter.api.Assertions.assertThrows; + +public class IndexingHeartbeatLowLevelTest { + @RegisterExtension + final FDBDatabaseExtension dbExtension = new FDBDatabaseExtension(); + @RegisterExtension + final TestKeySpacePathManagerExtension pathManager = new TestKeySpacePathManagerExtension(dbExtension); + FDBDatabase fdb; + KeySpacePath path; + FDBRecordStore recordStore; + RecordMetaData metaData; + + @BeforeEach + public void setUp() { + final FDBDatabaseFactory factory = dbExtension.getDatabaseFactory(); + factory.setInitialDelayMillis(2L); + factory.setMaxDelayMillis(4L); + factory.setMaxAttempts(100); + + fdb = dbExtension.getDatabase(); + fdb.setAsyncToSyncTimeout(5, TimeUnit.MINUTES); + path = pathManager.createPath(TestKeySpace.RECORD_STORE); + } + + FDBRecordContext openContext() { + FDBRecordContext context = fdb.openContext(); + FDBRecordStore.Builder builder = createStoreBuilder() + .setContext(context); + recordStore = builder.createOrOpen(FDBRecordStoreBase.StoreExistenceCheck.NONE); + metaData = recordStore.getRecordMetaData(); + return context; + } + + @Nonnull + private FDBRecordStore.Builder createStoreBuilder() { + return FDBRecordStore.newBuilder() + .setMetaDataProvider(metaData) + .setKeySpacePath(path); + } + + void openMetaData(@Nonnull Descriptors.FileDescriptor descriptor, @Nonnull FDBRecordStoreTestBase.RecordMetaDataHook hook) { + RecordMetaDataBuilder metaDataBuilder = RecordMetaData.newBuilder().setRecords(descriptor); + hook.apply(metaDataBuilder); + metaData = metaDataBuilder.getRecordMetaData(); + } + + void openMetaData(@Nonnull Descriptors.FileDescriptor descriptor) { + openMetaData(descriptor, (metaDataBuilder) -> { + }); + } + + void openSimpleMetaData(@Nonnull FDBRecordStoreTestBase.RecordMetaDataHook hook) { + openMetaData(TestRecords1Proto.getDescriptor(), hook); + } + + protected static FDBRecordStoreTestBase.RecordMetaDataHook allIndexesHook(List indexes) { + return metaDataBuilder -> { + for (Index index: indexes) { + metaDataBuilder.addIndex("MySimpleRecord", index); + } + } ; + } + + void testHeartbeatLowLevel(List indexes) { + Assertions.assertThat(indexes).hasSizeGreaterThanOrEqualTo(2); + FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(indexes); + + final int count = 10; + IndexingHeartbeat[] heartbeats = new IndexingHeartbeat[count]; + for (int i = 0; i < count; i++) { + heartbeats[i] = new IndexingHeartbeat(UUID.randomUUID(), "test", 100 + i, false); + } + + // populate heartbeats + openSimpleMetaData(hook); + try (FDBRecordContext context = openContext()) { + for (var heartbeat : heartbeats) { + heartbeat.updateHeartbeat(recordStore, indexes.get(0)); + heartbeat.updateHeartbeat(recordStore, indexes.get(1)); + } + context.commit(); + } + + // Verify query/clear operation + openSimpleMetaData(hook); + Index index = indexes.get(0); + try (FDBRecordContext context = openContext()) { + // Query, unlimited + Map queried = + IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); + Assertions.assertThat(queried).hasSize(count); + Assertions.assertThat(queried.keySet()) + .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(heartbeat -> heartbeat.indexerId).collect(Collectors.toList())); + + // Query, partial + queried = + IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 5).join(); + Assertions.assertThat(queried).hasSize(5); + + // clear, partial + int countDeleted = + IndexingHeartbeat.clearIndexingHeartbeats(recordStore, index, 0, 7).join(); + Assertions.assertThat(countDeleted).isEqualTo(7); + queried = + IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 5).join(); + Assertions.assertThat(queried).hasSize(3); + context.commit(); + } + + // Verify that the previous clear does not affect other index + openSimpleMetaData(hook); + index = indexes.get(1); + try (FDBRecordContext context = openContext()) { + Map queried = + IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 100).join(); + Assertions.assertThat(queried).hasSize(count); + Assertions.assertThat(queried.keySet()) + .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(ht -> ht.indexerId).collect(Collectors.toList())); + + // clear all + int countDeleted = + IndexingHeartbeat.clearIndexingHeartbeats(recordStore, index, 0, 0).join(); + Assertions.assertThat(countDeleted).isEqualTo(count); + + // verify empty + queried = + IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); + Assertions.assertThat(queried).isEmpty(); + context.commit(); + } + } + + @Test + void testHeartbeatLowLevelValueIndexes() { + List indexes = new ArrayList<>(); + indexes.add(new Index("indexA", field("num_value_2"), EmptyKeyExpression.EMPTY, IndexTypes.VALUE, IndexOptions.UNIQUE_OPTIONS)); + indexes.add(new Index("indexB", field("num_value_3_indexed"), IndexTypes.VALUE)); + testHeartbeatLowLevel(indexes); + } + + @Test + void testHeartbeatLowLevelSumCountIndexes() { + List indexes = new ArrayList<>(); + indexes.add(new Index("indexE", field("num_value_3_indexed").ungrouped(), IndexTypes.SUM)); + indexes.add(new Index("indexD", new GroupingKeyExpression(EmptyKeyExpression.EMPTY, 0), IndexTypes.COUNT)); + testHeartbeatLowLevel(indexes); + } + + @Test + void testHeartbeatLowLevelVersionIndexes() { + List indexes = new ArrayList<>(); + indexes.add(new Index("versionIndex1", concat(field("num_value_2"), VersionKeyExpression.VERSION), IndexTypes.VERSION)); + indexes.add(new Index("versionIndex2", concat(field("num_value_2"), VersionKeyExpression.VERSION), IndexTypes.VERSION)); + testHeartbeatLowLevel(indexes); + } + + @Test + void testCheckAndUpdateByRecords() { + Index index = new Index("indexD", new GroupingKeyExpression(EmptyKeyExpression.EMPTY, 0), IndexTypes.COUNT); + FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(List.of(index)); + IndexingHeartbeat heartbeat1 = new IndexingHeartbeat(UUID.randomUUID(), "Test", TimeUnit.SECONDS.toMillis(30), false); + + // Successfully update heartbeat + openSimpleMetaData(hook); + try (FDBRecordContext context = openContext()) { + heartbeat1.checkAndUpdateHeartbeat(recordStore, index).join(); + context.commit(); + } + + // Successfully update heartbeat + openSimpleMetaData(hook); + try (FDBRecordContext context = openContext()) { + final Map existingHeartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); + Assertions.assertThat(existingHeartbeats).hasSize(1); + heartbeat1.checkAndUpdateHeartbeat(recordStore, index).join(); + context.commit(); + } + + IndexingHeartbeat heartbeat2 = new IndexingHeartbeat(UUID.randomUUID(), "Test", TimeUnit.SECONDS.toMillis(30), false); + Assertions.assertThat(heartbeat1.indexerId).isNotEqualTo(heartbeat2.indexerId); + // Fail to create another 'BY_RECORD` heartbeat + openSimpleMetaData(hook); + try (FDBRecordContext context = openContext()) { + final Map existingHeartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); + Assertions.assertThat(existingHeartbeats).hasSize(1); + final CompletionException ex = assertThrows(CompletionException.class, () -> heartbeat2.checkAndUpdateHeartbeat(recordStore, index).join()); + Assertions.assertThat(ex.getMessage()).contains("SynchronizedSessionLockedException"); + context.commit(); + } + + // Successfully clear heartbeat1 + openSimpleMetaData(hook); + try (FDBRecordContext context = openContext()) { + final Map existingHeartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); + Assertions.assertThat(existingHeartbeats).hasSize(1); + heartbeat1.clearHeartbeat(recordStore, index); + context.commit(); + } + + // Successfully update heartbeat2 + openSimpleMetaData(hook); + try (FDBRecordContext context = openContext()) { + heartbeat2.checkAndUpdateHeartbeat(recordStore, index).join(); + context.commit(); + } + + // Successfully clear heartbeat2 + openSimpleMetaData(hook); + try (FDBRecordContext context = openContext()) { + heartbeat2.clearHeartbeat(recordStore, index); + final Map existingHeartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); + Assertions.assertThat(existingHeartbeats).isEmpty(); + context.commit(); + } + } + + @Test + void testCheckAndUpdateMutual() { + Index index = new Index("indexD", new GroupingKeyExpression(EmptyKeyExpression.EMPTY, 0), IndexTypes.COUNT); + FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(List.of(index)); + + final int count = 10; + IndexingHeartbeat[] heartbeats = new IndexingHeartbeat[count]; + for (int i = 0; i < count; i++) { + heartbeats[i] = new IndexingHeartbeat(UUID.randomUUID(), "Mutual", TimeUnit.SECONDS.toMillis(100), true); + } + + // Successfully check//update all heartbeats + openSimpleMetaData(hook); + try (FDBRecordContext context = openContext()) { + for (IndexingHeartbeat heartbeat: heartbeats) { + heartbeat.checkAndUpdateHeartbeat(recordStore, index).join(); + } + context.commit(); + } + + // Check count, clear all + openSimpleMetaData(hook); + try (FDBRecordContext context = openContext()) { + final Map existingHeartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); + Assertions.assertThat(existingHeartbeats).hasSize(count); + + for (IndexingHeartbeat heartbeat: heartbeats) { + heartbeat.clearHeartbeat(recordStore, index); + } + context.commit(); + } + + // verify cleared + openSimpleMetaData(hook); + try (FDBRecordContext context = openContext()) { + final Map existingHeartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); + Assertions.assertThat(existingHeartbeats).isEmpty(); + context.commit(); + } + } + + @Test + void testExpiredHeartbeat() throws InterruptedException { + Index index = new Index("versionIndex1", concat(field("num_value_2"), VersionKeyExpression.VERSION), IndexTypes.VERSION); + FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(List.of(index)); + IndexingHeartbeat heartbeat1 = new IndexingHeartbeat(UUID.randomUUID(), "Test", TimeUnit.SECONDS.toMillis(10), false); + + // Successfully update heartbeat1 + openSimpleMetaData(hook); + try (FDBRecordContext context = openContext()) { + heartbeat1.checkAndUpdateHeartbeat(recordStore, index).join(); + context.commit(); + } + + // Delay 20, set heartbeat2's lease to 4 + Thread.sleep(20); + IndexingHeartbeat heartbeat2 = new IndexingHeartbeat(UUID.randomUUID(), "Test", 4, false); + Assertions.assertThat(heartbeat1.indexerId).isNotEqualTo(heartbeat2.indexerId); + + // heartbeat2 successfully takes over + openSimpleMetaData(hook); + try (FDBRecordContext context = openContext()) { + final Map existingHeartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); + Assertions.assertThat(existingHeartbeats).hasSize(1); + heartbeat2.checkAndUpdateHeartbeat(recordStore, index).join(); + context.commit(); + } + } + + @Test + void testHeartbeatExpiration() throws InterruptedException { + Index index = new Index("indexD", new GroupingKeyExpression(EmptyKeyExpression.EMPTY, 0), IndexTypes.COUNT); + FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(List.of(index)); + + final IndexingHeartbeat heartbeatA = new IndexingHeartbeat(UUID.randomUUID(), "a", 500, false); + final IndexingHeartbeat heartbeatB = new IndexingHeartbeat(UUID.randomUUID(), "b", 5, false); + + // Set heartbeat A + openSimpleMetaData(hook); + try (FDBRecordContext context = openContext()) { + heartbeatA.checkAndUpdateHeartbeat(recordStore, index).join(); + context.commit(); + } + + Thread.sleep(100); + // Expect heartbeatA to expire after 5 milliseconds, and successfully set heartbeatB + openSimpleMetaData(hook); + try (FDBRecordContext context = openContext()) { + heartbeatB.checkAndUpdateHeartbeat(recordStore, index).join(); + context.commit(); + } + + // Expect heartbeatA to fail check/update + // Note: if become flakey, increase the least time of heartbeatA + openSimpleMetaData(hook); + try (FDBRecordContext context = openContext()) { + final CompletionException ex = assertThrows(CompletionException.class, () -> heartbeatA.checkAndUpdateHeartbeat(recordStore, index).join()); + Assertions.assertThat(ex.getMessage()).contains("SynchronizedSessionLockedException"); + context.commit(); + } + } +} diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerIndexFromIndexTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerIndexFromIndexTest.java index cf61bb60ae..f0663ce1f5 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerIndexFromIndexTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerIndexFromIndexTest.java @@ -29,6 +29,7 @@ import com.apple.foundationdb.record.metadata.expressions.EmptyKeyExpression; import com.apple.foundationdb.record.metadata.expressions.GroupingKeyExpression; import com.apple.foundationdb.record.metadata.expressions.KeyExpression; +import com.apple.foundationdb.synchronizedsession.SynchronizedSessionLockedException; import com.apple.test.BooleanSource; import com.google.common.collect.Comparators; import org.junit.jupiter.api.Test; @@ -38,6 +39,9 @@ import javax.annotation.Nullable; import java.util.List; import java.util.Map; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicLong; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -1154,4 +1158,69 @@ void testIndexFromIndexBlock() { assertReadable(tgtIndex); scrubAndValidate(List.of(tgtIndex)); } + + @Test + void testForbidConcurrentIndexFromIndexSessions() throws InterruptedException { + // Do not let a conversion of few indexes of an active multi-target session + final int numRecords = 59; + populateData(numRecords); + + Index sourceIndex = new Index("src_index", field("num_value_2"), EmptyKeyExpression.EMPTY, IndexTypes.VALUE, IndexOptions.UNIQUE_OPTIONS); + openSimpleMetaData(metaDataBuilder -> metaDataBuilder.addIndex("MySimpleRecord", sourceIndex)); + buildIndexClean(sourceIndex); + + // Partly build index + Index tgtIndex = new Index("tgt_index", field("num_value_3_indexed"), IndexTypes.VALUE); + FDBRecordStoreTestBase.RecordMetaDataHook hook = myHook(sourceIndex, tgtIndex); + openSimpleMetaData(hook); + + Semaphore pauseMutualBuildSemaphore = new Semaphore(1); + Semaphore startBuildingSemaphore = new Semaphore(1); + pauseMutualBuildSemaphore.acquire(); + startBuildingSemaphore.acquire(); + AtomicBoolean passed = new AtomicBoolean(false); + Thread t1 = new Thread(() -> { + // build index and pause halfway, allowing an active session test + try (OnlineIndexer indexBuilder = newIndexerBuilder(tgtIndex) + .setLeaseLengthMillis(TimeUnit.SECONDS.toMillis(20)) + .setLimit(4) + .setIndexingPolicy(OnlineIndexer.IndexingPolicy.newBuilder() + .setSourceIndex("src_index") + .forbidRecordScan()) + .setConfigLoader(old -> { + if (passed.get()) { + try { + startBuildingSemaphore.release(); + pauseMutualBuildSemaphore.acquire(); // pause to try building indexes + } catch (InterruptedException e) { + throw new RuntimeException(e); + } finally { + pauseMutualBuildSemaphore.release(); + } + } else { + passed.set(true); + } + return old; + }) + .build()) { + indexBuilder.buildIndex(); + } + }); + t1.start(); + startBuildingSemaphore.acquire(); + startBuildingSemaphore.release(); + // Try one index at a time + try (OnlineIndexer indexBuilder = newIndexerBuilder(tgtIndex) + .setIndexingPolicy(OnlineIndexer.IndexingPolicy.newBuilder() + .setSourceIndex("src_index") + .forbidRecordScan()) + .build()) { + assertThrows(SynchronizedSessionLockedException.class, indexBuilder::buildIndex); + } + // let the other thread finish indexing + pauseMutualBuildSemaphore.release(); + t1.join(); + // happy indexes assertion + assertReadable(tgtIndex); + } } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerSimpleTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerSimpleTest.java index ad51503050..58a92c58d5 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerSimpleTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerSimpleTest.java @@ -887,4 +887,32 @@ public void runWithWeakReadSemantics() { fdb.setTrackLastSeenVersionOnRead(dbTracksReadVersionOnCommit); } } + + @Test + @SuppressWarnings("removal") + void testDeprecatedSetUseSynchronizedSession() { + List records = LongStream.range(0, 20).mapToObj(val -> + TestRecords1Proto.MySimpleRecord.newBuilder().setRecNo(val).setNumValue2((int)val + 1).build() + ).collect(Collectors.toList()); + Index index = new Index("simple$value_2", field("num_value_2").ungrouped(), IndexTypes.SUM); + FDBRecordStoreTestBase.RecordMetaDataHook hook = metaDataBuilder -> metaDataBuilder.addIndex("MySimpleRecord", index); + + openSimpleMetaData(); + try (FDBRecordContext context = openContext()) { + records.forEach(recordStore::saveRecord); + context.commit(); + } + + openSimpleMetaData(hook); + disableAll(List.of(index)); + + openSimpleMetaData(hook); + try (OnlineIndexer indexBuilder = newIndexerBuilder(index) + .setUseSynchronizedSession(true) + .build()) { + indexBuilder.buildIndex(); + } + + assertReadable(index); + } } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java index abbca7975d..c110536316 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java @@ -27,15 +27,21 @@ import com.apple.foundationdb.record.metadata.IndexTypes; import com.apple.foundationdb.record.metadata.expressions.EmptyKeyExpression; import com.apple.foundationdb.tuple.Tuple; +import com.apple.test.BooleanSource; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; import java.util.ArrayList; import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.UUID; +import java.util.concurrent.Semaphore; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; +import java.util.concurrent.atomic.AtomicReference; import java.util.stream.Collectors; import java.util.stream.IntStream; @@ -58,7 +64,7 @@ void testHeartbeatLowLevel() { final int count = 10; IndexingHeartbeat[] heartbeats = new IndexingHeartbeat[count]; for (int i = 0; i < count; i++) { - heartbeats[i] = new IndexingHeartbeat(UUID.randomUUID(), IndexBuildProto.IndexBuildIndexingStamp.Method.BY_INDEX, 100 + i); + heartbeats[i] = new IndexingHeartbeat(UUID.randomUUID(), "Test", 100 + i, true); } openSimpleMetaData(hook); @@ -76,7 +82,7 @@ void testHeartbeatLowLevel() { Map queried = indexer.getIndexingHeartbeats(0); Assertions.assertThat(queried).hasSize(count); Assertions.assertThat(queried.keySet()) - .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(ht -> ht.indexerId).collect(Collectors.toList())); + .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(heartbeat -> heartbeat.indexerId).collect(Collectors.toList())); // Query, partial queried = indexer.getIndexingHeartbeats(5); @@ -106,28 +112,36 @@ void testHeartbeatLowLevel() { } } - @Test - void testMutualIndexersHeartbeatsClearAfterBuild() { + @ParameterizedTest + @BooleanSource + void testIndexersHeartbeatsClearAfterBuild(boolean mutualIndexing) { // Assert that the heartbeats are cleared after building List indexes = new ArrayList<>(); indexes.add(new Index("indexA", field("num_value_2"), EmptyKeyExpression.EMPTY, IndexTypes.VALUE, IndexOptions.UNIQUE_OPTIONS)); indexes.add(new Index("indexC", field("num_value_unique"), EmptyKeyExpression.EMPTY, IndexTypes.VALUE, IndexOptions.UNIQUE_OPTIONS)); int numRecords = 77; populateData(numRecords); - int boundarySize = 23; - final List boundariesList = getBoundariesList(numRecords, boundarySize); FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(indexes); openSimpleMetaData(hook); disableAll(indexes); - IntStream.rangeClosed(1, 5).parallel().forEach(i -> { + if (mutualIndexing) { + int boundarySize = 23; + final List boundariesList = getBoundariesList(numRecords, boundarySize); + IntStream.rangeClosed(1, 5).parallel().forEach(i -> { + try (OnlineIndexer indexer = newIndexerBuilder(indexes) + .setIndexingPolicy(OnlineIndexer.IndexingPolicy.newBuilder() + .setMutualIndexingBoundaries(boundariesList)) + .build()) { + indexer.buildIndex(); + } + }); + } else { try (OnlineIndexer indexer = newIndexerBuilder(indexes) - .setIndexingPolicy(OnlineIndexer.IndexingPolicy.newBuilder() - .setMutualIndexingBoundaries(boundariesList)) .build()) { indexer.buildIndex(); } - }); + } for (Index index : indexes) { try (OnlineIndexer indexer = newIndexerBuilder(index).build()) { @@ -136,27 +150,46 @@ void testMutualIndexersHeartbeatsClearAfterBuild() { } } - @Test - void testMutualIndexersHeartbeatsClearAfterCrash() { + @ParameterizedTest + @BooleanSource + void testIndexersHeartbeatsClearAfterCrash(boolean mutualIndexing) { // Assert that the heartbeats are cleared after crash List indexes = new ArrayList<>(); indexes.add(new Index("indexA", field("num_value_2"), EmptyKeyExpression.EMPTY, IndexTypes.VALUE, IndexOptions.UNIQUE_OPTIONS)); indexes.add(new Index("indexC", field("num_value_unique"), EmptyKeyExpression.EMPTY, IndexTypes.VALUE, IndexOptions.UNIQUE_OPTIONS)); int numRecords = 98; populateData(numRecords); - int boundarySize = 20; - final List boundariesList = getBoundariesList(numRecords, boundarySize); FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(indexes); openSimpleMetaData(hook); disableAll(indexes); final String testThrowMsg = "Intentionally crash during test"; - IntStream.rangeClosed(1, 9).parallel().forEach(i -> { - final AtomicLong counter = new AtomicLong(0); + final AtomicLong counter = new AtomicLong(0); + if (mutualIndexing) { + int boundarySize = 20; + final List boundariesList = getBoundariesList(numRecords, boundarySize); + IntStream.rangeClosed(1, 9).parallel().forEach(i -> { + try (OnlineIndexer indexer = newIndexerBuilder(indexes) + .setLimit(10) + .setIndexingPolicy(OnlineIndexer.IndexingPolicy.newBuilder() + .setMutualIndexingBoundaries(boundariesList)) + .setConfigLoader(old -> { + // Unfortunately, we cannot verify that at least one heartbeat exists from this + // block, as it would have been nesting "asyncToSync" functions. But there are other tests + // that verify the "sync lock" functionality. + if (counter.incrementAndGet() > 2) { + throw new RecordCoreException(testThrowMsg); + } + return old; + }) + .build()) { + RecordCoreException e = assertThrows(RecordCoreException.class, indexer::buildIndex); + assertTrue(e.getMessage().contains(testThrowMsg)); + } + }); + } else { try (OnlineIndexer indexer = newIndexerBuilder(indexes) - .setIndexingPolicy(OnlineIndexer.IndexingPolicy.newBuilder() - .setMutualIndexingBoundaries(boundariesList) - .build()) + .setLimit(10) .setConfigLoader(old -> { // Unfortunately, we cannot verify that at least one heartbeat exists from this // block, as it would have been nesting "asyncToSync" functions. But there are other tests @@ -170,7 +203,7 @@ void testMutualIndexersHeartbeatsClearAfterCrash() { RecordCoreException e = assertThrows(RecordCoreException.class, indexer::buildIndex); assertTrue(e.getMessage().contains(testThrowMsg)); } - }); + } for (Index index : indexes) { try (OnlineIndexer indexer = newIndexerBuilder(index).build()) { @@ -179,4 +212,145 @@ void testMutualIndexersHeartbeatsClearAfterCrash() { } } + @Test + void testMutualIndexersHeartbeatsClearAfterBuild() throws InterruptedException { + // Check heartbeats count during mutual indexing + List indexes = new ArrayList<>(); + indexes.add(new Index("indexA", field("num_value_2"), EmptyKeyExpression.EMPTY, IndexTypes.VALUE, IndexOptions.UNIQUE_OPTIONS)); + indexes.add(new Index("indexC", field("num_value_unique"), EmptyKeyExpression.EMPTY, IndexTypes.VALUE, IndexOptions.UNIQUE_OPTIONS)); + int numRecords = 77; + populateData(numRecords); + int boundarySize = 5; + final List boundariesList = getBoundariesList(numRecords, boundarySize); + FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(indexes); + openSimpleMetaData(hook); + disableAll(indexes); + + Semaphore pauseSemaphore = new Semaphore(1); + Semaphore startSemaphore = new Semaphore(1); + final AtomicInteger count = new AtomicInteger(0); + pauseSemaphore.acquire(); + startSemaphore.acquire(); + AtomicReference> heartbeats = new AtomicReference<>(); + IntStream.rangeClosed(1, 4).parallel().forEach(i -> { + if (i == 4) { + try { + startSemaphore.acquire(); + Thread.sleep(100); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + try (OnlineIndexer indexer = newIndexerBuilder(indexes).build()) { + heartbeats.set(indexer.getIndexingHeartbeats(0)); + } + startSemaphore.release(); + pauseSemaphore.release(); + } else { + AtomicInteger counter = new AtomicInteger(0); + try (OnlineIndexer indexer = newIndexerBuilder(indexes) + .setIndexingPolicy(OnlineIndexer.IndexingPolicy.newBuilder() + .setMutualIndexingBoundaries(boundariesList)) + .setConfigLoader(old -> { + if (counter.incrementAndGet() > 0) { + if (count.incrementAndGet() == 2) { + startSemaphore.release(); + } + try { + pauseSemaphore.acquire(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } finally { + pauseSemaphore.release(); + } + } + return old; + }) + .build()) { + indexer.buildIndex(); + } + } + }); + // While building, heartbeats count should have been 3 + Assertions.assertThat(heartbeats.get()).hasSize(3); + + // After building, heartbeats count should be 0 + try (OnlineIndexer indexer = newIndexerBuilder(indexes).build()) { + heartbeats.set(indexer.getIndexingHeartbeats(0)); + } + } + + + @Test + void testHeartbeatsRenewal() throws InterruptedException { + // make sure that the heartbeats behave as expected during indexing: + // single item + // same indexerId, genesis time + // monotonically increasing heartbeats + List indexes = new ArrayList<>(); + indexes.add(new Index("indexA", field("num_value_2"), EmptyKeyExpression.EMPTY, IndexTypes.VALUE, IndexOptions.UNIQUE_OPTIONS)); + int numRecords = 74; + populateData(numRecords); + FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(indexes); + openSimpleMetaData(hook); + disableAll(indexes); + final List> heartbeatsQueries = new ArrayList<>(); + + Semaphore indexerGo = new Semaphore(1); + Semaphore colectorGo = new Semaphore(1); + AtomicBoolean indexerDone = new AtomicBoolean(false); + colectorGo.acquire(); + Thread indexerThread = new Thread( () -> { + try (OnlineIndexer indexer = newIndexerBuilder(indexes) + .setLimit(10) + .setConfigLoader(old -> { + colectorGo.release(); + try { + indexerGo.acquire(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + return old; + }) + .build()) { + indexer.buildIndex(); + } + colectorGo.release(); + indexerDone.set(true); + }); + + Thread collectorThread = new Thread(() -> { + while (!indexerDone.get()) { + try { + colectorGo.acquire(); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + try (FDBRecordContext context = openContext()) { + final Map heartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, indexes.get(0), 0).join(); + heartbeatsQueries.add(heartbeats); + context.commit(); + } + indexerGo.release(); + } + }); + indexerThread.start(); + collectorThread.start(); + collectorThread.join(); + indexerThread.join(); + + Assertions.assertThat(heartbeatsQueries).hasSizeGreaterThan(5); + Assertions.assertThat(heartbeatsQueries.get(0)).hasSize(1); + final Map.Entry first = heartbeatsQueries.get(0).entrySet().iterator().next(); + Map.Entry previous = first; + for (int i = 1; i < heartbeatsQueries.size() - 1; i++) { + Assertions.assertThat(heartbeatsQueries.get(i)).hasSize(1); + final Map.Entry item = heartbeatsQueries.get(i).entrySet().iterator().next(); + Assertions.assertThat(item.getKey()).isEqualTo(first.getKey()); + Assertions.assertThat(item.getValue().getGenesisTimeMilliseconds()).isEqualTo(first.getValue().getGenesisTimeMilliseconds()); + Assertions.assertThat(item.getValue().getInfo()).isEqualTo(first.getValue().getInfo()); + Assertions.assertThat(item.getValue().getHeartbeatTimeMilliseconds()) + .isGreaterThan(previous.getValue().getHeartbeatTimeMilliseconds()); + previous = item; + } + } } From 438fb094e55e84f43423ff2d654451508c245ff7 Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Fri, 22 Aug 2025 16:24:24 -0400 Subject: [PATCH 18/26] Apply (more of) Scott's requested changes --- .../provider/foundationdb/FDBRecordStore.java | 1 + .../provider/foundationdb/IndexingBase.java | 74 +++--- .../foundationdb/IndexingHeartbeat.java | 40 +++- .../foundationdb/IndexingSubspaces.java | 8 +- .../OnlineIndexOperationBaseBuilder.java | 19 +- .../OnlineIndexOperationConfig.java | 33 +-- .../provider/foundationdb/OnlineIndexer.java | 46 +++- .../src/main/proto/index_build.proto | 7 +- .../IndexingHeartbeatLowLevelTest.java | 225 +++++++++++++----- .../OnlineIndexerBuildIndexTest.java | 8 + .../foundationdb/OnlineIndexerSimpleTest.java | 74 +++++- .../OnlineIndexingHeartbeatTest.java | 1 + gradle.properties | 2 +- 13 files changed, 375 insertions(+), 163 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java index e4c75e6c58..d4b54d272f 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java @@ -4965,6 +4965,7 @@ 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); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java index adc29115ca..9f0b89afce 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java @@ -102,6 +102,7 @@ public abstract class IndexingBase { private final long startingTimeMillis; private long lastTypeStampCheckMillis; private Map indexingMergerMap = null; + @Nullable private IndexingHeartbeat heartbeat = null; // this will stay null for index scrubbing IndexingBase(@Nonnull IndexingCommon common, @@ -157,15 +158,13 @@ public CompletableFuture buildIndexAsync(boolean markReadable) { long startNanos = System.nanoTime(); FDBDatabaseRunner runner = getRunner(); final FDBStoreTimer timer = runner.getTimer(); - if ( timer != null) { + if (timer != null) { lastProgressSnapshot = StoreTimerSnapshot.from(timer); } - AtomicReference indexingException = new AtomicReference<>(null); - return handleStateAndDoBuildIndexAsync(markReadable, message) - .handle((ret, ex) -> { - if (ex != null) { - indexingException.set(ex); - } + return MoreAsyncUtil.composeWhenComplete( + handleStateAndDoBuildIndexAsync(markReadable, message), + (result, ex) -> { + // proper log message.addKeysAndValues(indexingLogMessageKeyValues()) // add these here to pick up state accumulated during build .addKeysAndValues(common.indexLogMessageKeyValues()) .addKeyAndValue(LogMessageKeys.TOTAL_MICROS, TimeUnit.NANOSECONDS.toMicros(System.nanoTime() - startNanos)); @@ -177,20 +176,13 @@ public CompletableFuture buildIndexAsync(boolean markReadable) { message.addKeyAndValue(LogMessageKeys.RESULT, "success"); LOGGER.info(message.toString()); } - return ret; - }) - // Here: if the heartbeat was *not* cleared while marking the index readable, it would be cleared in - // these dedicated transaction. Heartbeat clearing is not a blocker but a "best effort" operation. - .thenCompose(ignore -> clearHeartbeats()) - .handle((ignore, exIgnore) -> { - Throwable ex = indexingException.get(); - if (ex instanceof RuntimeException) { - throw (RuntimeException) ex; - } else if (ex != null) { - throw new RuntimeException(ex); - } - return null; - }); + // Here: if the heartbeat was *not* cleared while marking the index readable, it would be cleared in + // these dedicated transaction. Heartbeat clearing is not a blocker but a "best effort" operation. + return clearHeartbeats() + .handle((ignoreRet, ignoreEx) -> null); + }, + getRunner().getDatabase()::mapAsyncToSyncException + ); } abstract List indexingLogMessageKeyValues(); @@ -271,7 +263,8 @@ private CompletableFuture handleStateAndDoBuildIndexAsync(boolean markRead doIndex ? buildIndexInternalAsync().thenApply(ignore -> markReadable) : AsyncUtil.READY_FALSE - ).thenCompose(this::markIndexReadable).thenApply(ignore -> null); + ).thenCompose(this::markIndexReadable + ).thenApply(ignore -> null); } private CompletableFuture markIndexesWriteOnly(boolean continueBuild, FDBRecordStore store) { @@ -317,7 +310,7 @@ public CompletableFuture markIndexReadable(boolean markReadablePlease) // Mark each index readable in its own (retriable, parallel) transaction. If one target fails to become // readable, it should not affect the others. return forEachTargetIndex(index -> - markIndexReadableSingleTarget(index, anythingChanged, runtimeExceptionAtomicReference) + markIndexReadableForIndex(index, anythingChanged, runtimeExceptionAtomicReference) ).thenApply(ignore -> { RuntimeException ex = runtimeExceptionAtomicReference.get(); if (ex != null) { @@ -328,13 +321,13 @@ public CompletableFuture markIndexReadable(boolean markReadablePlease) }); } - private CompletableFuture markIndexReadableSingleTarget(Index index, AtomicBoolean anythingChanged, - AtomicReference runtimeExceptionAtomicReference) { + private CompletableFuture markIndexReadableForIndex(Index index, AtomicBoolean anythingChanged, + AtomicReference runtimeExceptionAtomicReference) { // An extension function to reduce markIndexReadable's complexity return getRunner().runAsync(context -> common.getRecordStoreBuilder().copyBuilder().setContext(context).openAsync() .thenCompose(store -> { - clearHeartbeatSingleTarget(store, index); + clearHeartbeatForIndex(store, index); return policy.shouldAllowUniquePendingState(store) ? store.markIndexReadableOrUniquePending(index) : store.markIndexReadable(index); @@ -376,7 +369,7 @@ private CompletableFuture setIndexingTypeOrThrow(FDBRecordStore store, boo if (forceStampOverwrite && !continuedBuild) { // Fresh session + overwrite = no questions asked store.saveIndexingTypeStamp(index, newStamp); - return AsyncUtil.DONE ; + return AsyncUtil.DONE; } return store.loadIndexingTypeStampAsync(index) .thenCompose(savedStamp -> { @@ -858,30 +851,27 @@ private CompletableFuture updateHeartbeat(boolean validate, FDBRecordStore private CompletableFuture clearHeartbeats() { if (heartbeat == null) { + // Here: either silent heartbeats or heartbeats had been cleared during markReadable phase return AsyncUtil.DONE; } - return forEachTargetIndex(this::clearHeartbeatSingleTarget) - .thenAccept(ignore -> heartbeat = null); + // Here: for each index we clear (only) the heartbeat generated by this indexer. This is a quick operation that can be done in a single transaction. + return getRunner().runAsync(context -> + common.getRecordStoreBuilder().copyBuilder().setContext(context).openAsync() + .thenApply(store -> { + clearHeartbeats(store); + return null; + })); } private void clearHeartbeats(FDBRecordStore store) { if (heartbeat != null) { for (Index index : common.getTargetIndexes()) { - clearHeartbeatSingleTarget(store, index); + heartbeat.clearHeartbeat(store, index); } } } - private CompletableFuture clearHeartbeatSingleTarget(Index index) { - return getRunner().runAsync(context -> - common.getRecordStoreBuilder().copyBuilder().setContext(context).openAsync() - .thenApply(store -> { - clearHeartbeatSingleTarget(store, index); - return null; - })); - } - - private void clearHeartbeatSingleTarget(FDBRecordStore store, Index index) { + private void clearHeartbeatForIndex(FDBRecordStore store, Index index) { if (heartbeat != null) { heartbeat.clearHeartbeat(store, index); } @@ -1115,9 +1105,9 @@ public CompletableFuture> getInde .thenCompose(store -> IndexingHeartbeat.getIndexingHeartbeats(store, common.getPrimaryIndex(), maxCount))); } - public CompletableFuture clearIndexingHeartbeats(long minAgenMilliseconds, int maxIteration) { + public CompletableFuture clearIndexingHeartbeats(long minAgeMilliseconds, int maxIteration) { return getRunner().runAsync(context -> openRecordStore(context) - .thenCompose(store -> IndexingHeartbeat.clearIndexingHeartbeats(store, common.getPrimaryIndex(), minAgenMilliseconds, maxIteration))); + .thenCompose(store -> IndexingHeartbeat.clearIndexingHeartbeats(store, common.getPrimaryIndex(), minAgeMilliseconds, maxIteration))); } /** diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java index fa0b050fcd..8ac1f61061 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java @@ -21,13 +21,17 @@ 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; @@ -36,8 +40,20 @@ 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 { // [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; @@ -53,7 +69,7 @@ public IndexingHeartbeat(final UUID indexerId, String info, long leaseLength, bo } public void updateHeartbeat(@Nonnull FDBRecordStore store, @Nonnull Index index) { - byte[] key = IndexingSubspaces.indexheartbeatSubspace(store, index, indexerId).pack(); + byte[] key = IndexingSubspaces.indexHeartbeatSubspace(store, index, indexerId).pack(); byte[] value = IndexBuildProto.IndexBuildHeartbeat.newBuilder() .setInfo(info) .setGenesisTimeMilliseconds(genesisTimeMilliseconds) @@ -92,7 +108,11 @@ public CompletableFuture checkAndUpdateHeartbeat(@Nonnull FDBRecordStore s } } } catch (InvalidProtocolBufferException e) { - throw new RuntimeException(e); + if (logger.isWarnEnabled()) { + logger.warn(KeyValueLogMessage.of("Bad indexing heartbeat item", + LogMessageKeys.KEY, kv.getKey(), + LogMessageKeys.VALUE, kv.getValue())); + } } return true; })) @@ -103,11 +123,11 @@ public CompletableFuture checkAndUpdateHeartbeat(@Nonnull FDBRecordStore s } public void clearHeartbeat(@Nonnull FDBRecordStore store, @Nonnull Index index) { - store.ensureContextActive().clear(IndexingSubspaces.indexheartbeatSubspace(store, index, indexerId).pack()); + 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()); + store.ensureContextActive().clear(IndexingSubspaces.indexHeartbeatSubspace(store, index).range()); } public static CompletableFuture> getIndexingHeartbeats(FDBRecordStore store, Index index, int maxCount) { @@ -130,7 +150,9 @@ public static CompletableFuture> } catch (InvalidProtocolBufferException e) { // Let the caller know about this invalid heartbeat. ret.put(otherIndexerId, IndexBuildProto.IndexBuildHeartbeat.newBuilder() - .setInfo("<< Invalid Heartbeat >>") + .setInfo(INVALID_HEARTBEAT_INFO) + .setGenesisTimeMilliseconds(0) + .setHeartbeatTimeMilliseconds(0) .build()); } return true; @@ -138,7 +160,7 @@ public static CompletableFuture> .thenApply(ignore -> ret); } - public static CompletableFuture clearIndexingHeartbeats(@Nonnull FDBRecordStore store, @Nonnull Index index, long minAgenMilliseconds, int maxIteration) { + public static CompletableFuture clearIndexingHeartbeats(@Nonnull FDBRecordStore store, @Nonnull Index index, long minAgeMilliseconds, int maxIteration) { final AsyncIterator iterator = heartbeatsIterator(store, index); final AtomicInteger deleteCount = new AtomicInteger(0); final AtomicInteger iterationCount = new AtomicInteger(0); @@ -156,7 +178,7 @@ public static CompletableFuture clearIndexingHeartbeats(@Nonnull FDBRec try { final IndexBuildProto.IndexBuildHeartbeat otherHeartbeat = IndexBuildProto.IndexBuildHeartbeat.parseFrom(kv.getValue()); // remove heartbeat if too old - shouldRemove = now + minAgenMilliseconds >= otherHeartbeat.getHeartbeatTimeMilliseconds(); + shouldRemove = now >= otherHeartbeat.getHeartbeatTimeMilliseconds() + minAgeMilliseconds; } catch (InvalidProtocolBufferException e) { // remove heartbeat if invalid shouldRemove = true; @@ -171,11 +193,11 @@ public static CompletableFuture clearIndexingHeartbeats(@Nonnull FDBRec } private static AsyncIterator heartbeatsIterator(FDBRecordStore store, Index index) { - return store.getContext().ensureActive().snapshot().getRange(IndexingSubspaces.indexheartbeatSubspace(store, index).range()).iterator(); + 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); + return IndexingSubspaces.indexHeartbeatSubspace(store, index).unpack(key).getUUID(0); } private static long nowMilliseconds() { diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingSubspaces.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingSubspaces.java index acc7363303..7cc0ea58db 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingSubspaces.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingSubspaces.java @@ -92,7 +92,7 @@ public static Subspace indexBuildTypeSubspace(@Nonnull FDBRecordStoreBase sto * @return subspace */ @Nonnull - public static Subspace indexheartbeatSubspace(@Nonnull FDBRecordStoreBase store, @Nonnull Index index) { + public static Subspace indexHeartbeatSubspace(@Nonnull FDBRecordStoreBase store, @Nonnull Index index) { return indexBuildSubspace(store, index, INDEX_BUILD_HEARTBEAT_PREFIX); } @@ -104,8 +104,8 @@ public static Subspace indexheartbeatSubspace(@Nonnull FDBRecordStoreBase sto * @return subspace */ @Nonnull - public static Subspace indexheartbeatSubspace(@Nonnull FDBRecordStoreBase store, @Nonnull Index index, @Nonnull UUID indexerId) { - return indexheartbeatSubspace(store, index).subspace(Tuple.from(indexerId)); + public static Subspace indexHeartbeatSubspace(@Nonnull FDBRecordStoreBase store, @Nonnull Index index, @Nonnull UUID indexerId) { + return indexHeartbeatSubspace(store, index).subspace(Tuple.from(indexerId)); } /** @@ -210,6 +210,6 @@ public static void eraseAllIndexingDataButTheLock(@Nonnull FDBRecordContext cont 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())); + context.clear(Range.startsWith(indexHeartbeatSubspace(store, index).pack())); } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationBaseBuilder.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationBaseBuilder.java index c8a0f03f39..ebb2240436 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationBaseBuilder.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationBaseBuilder.java @@ -27,7 +27,6 @@ import com.apple.foundationdb.record.metadata.Index; import com.apple.foundationdb.record.metadata.MetaDataException; import com.apple.foundationdb.record.provider.common.RecordSerializer; -import com.apple.foundationdb.record.provider.foundationdb.synchronizedsession.SynchronizedSessionRunner; import com.apple.foundationdb.subspace.Subspace; import com.google.protobuf.Message; @@ -709,12 +708,21 @@ public B setTransactionTimeLimitMilliseconds(long timeLimitMilliseconds) { } /** - * Deprecated. This will soon be determined by the indexing session type + * Deprecated. Synchronization is handled by shared heartbeats and exclusive access is being determined by + * the indexing session type. + * + * When upgrading from Synchronized sessions to a version that uses Shared Heartbeats: + * During graduate code upgrade on multiple servers, there may be a state of one server indexing + * with a synchronized session lock, while another server builds the same index with an exclusive heartbeat + * "lock". If that happens: + * a) There will be no more than two concurrent active sessions (one per each lock type). + * b) The indexing sessions will conflict each other until one of the indexers will give up. While this is + * not optimal, the generated index will be valid. + * * Set the use of a synchronized session during the index operation. Synchronized sessions help performing * the multiple transactions operations in a resource efficient way. * Normally this should be {@code true}. * - * @see SynchronizedSessionRunner * @param useSynchronizedSession use synchronize session if true, otherwise false * @return this builder */ @@ -724,9 +732,8 @@ public B setUseSynchronizedSession(boolean useSynchronizedSession) { } /** - * Set the lease length in milliseconds if the synchronized session is used. The default value is {@link OnlineIndexOperationConfig#DEFAULT_LEASE_LENGTH_MILLIS}. - * @see com.apple.foundationdb.synchronizedsession.SynchronizedSession - * @param leaseLengthMillis length between last access and lease's end time in milliseconds + * Set the max age of an indexing heartbeat to define an "active" indexing session. + * @param leaseLengthMillis max heartbeat age to be considered "active". In milliseconds * @return this builder */ public B setLeaseLengthMillis(long leaseLengthMillis) { diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationConfig.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationConfig.java index 9a4ce2f5b1..318a7d36e6 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationConfig.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexOperationConfig.java @@ -21,7 +21,6 @@ package com.apple.foundationdb.record.provider.foundationdb; import com.apple.foundationdb.annotation.API; -import com.apple.foundationdb.record.provider.foundationdb.synchronizedsession.SynchronizedSessionRunner; import javax.annotation.Nonnull; @@ -53,7 +52,7 @@ public class OnlineIndexOperationConfig { */ public static final int DEFAULT_PROGRESS_LOG_INTERVAL = -1; /** - * Default synchronized session lease time in milliseconds. This allows a lock expiration, if the online operation stops unexpectedly. + * Default indexing heartbeat age, in milliseconds, to define an "active" session. */ public static final long DEFAULT_LEASE_LENGTH_MILLIS = 10_000; @@ -188,15 +187,14 @@ public static Builder newBuilder() { /** - * Not used anymore. - * @return always false; - * @deprecated see {@link Builder#setUseSynchronizedSession(boolean)} + * Deprecated. Synchronized sessions are now automatically determined by the indexing method. + * Mutual indexing and index scrubbing (if applicable) do not expect to run exclusively, other indexing methods will + * throw an exception if they another active indexing session is detected. + * @return always true; */ @API(API.Status.DEPRECATED) - @SuppressWarnings("PMD.AvoidUsingHardCodedIP") // version is not IP - @Deprecated(since = "4.4.3.0", forRemoval = true) public boolean shouldUseSynchronizedSession() { - return false; + return true; } public long getLeaseLengthMillis() { @@ -490,28 +488,23 @@ public Builder setTransactionTimeLimitMilliseconds(long timeLimitMilliseconds) { } /** - * Set the use of a synchronized session during the index operation. Synchronized sessions help performing - * the multiple transactions operation in a resource efficient way. - * Normally this should be {@code true}. - * - * @see SynchronizedSessionRunner - * @param useSynchronizedSession use synchronize session if true, otherwise false + * Deprecated. Synchronized sessions are now automatically determined by the indexing method. + * Mutual indexing and index scrubbing (if applicable) do not expect to run exclusively, other indexing methods will + * throw an exception if they another active indexing session is detected. + * @param useSynchronizedSession ignored. * @return this builder - * - * @deprecated Synchronized sessions are now determined by the indexing method. */ @API(API.Status.DEPRECATED) - @SuppressWarnings("PMD.AvoidUsingHardCodedIP") // version is not IP - @Deprecated(since = "4.4.3.0", forRemoval = true) public Builder setUseSynchronizedSession(boolean useSynchronizedSession) { // no-op return this; } /** - * If the indexing session is not expected to be mutual, abort indexing if another session is active. This function - * defines the maximum age of another session's heartbeat to be considered an "active session". + * Defines the maximum age of another session's heartbeat to be considered an active session. * The default value is {@link #DEFAULT_LEASE_LENGTH_MILLIS}. + * Mutual indexing and index scrubbing (if applicable) do not expect to run exclusively, other indexing methods will + * throw an exception if they another active indexing session is detected. * @param leaseLengthMillis length between last access and lease's end time in milliseconds * @return this builder */ diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java index f0972513b8..c9234a7b68 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java @@ -480,24 +480,39 @@ public boolean checkAnyOngoingOnlineIndexBuilds() { } /** - * Check if the index is being built by any of the {@link OnlineIndexer}s (only if they use {@link SynchronizedSession}s), - * including this {@link OnlineIndexer}. + * Check if the main index is being built. Note that with shared heartbeats, this function will now return true for any active session - mutual or exclusive. * @return a future that will complete to true if the index is being built and false otherwise */ public CompletableFuture checkAnyOngoingOnlineIndexBuildsAsync() { return runner.runAsync(context -> openRecordStore(context).thenCompose(recordStore -> - checkAnyOngoingOnlineIndexBuildsAsync(recordStore, index)), + checkAnyOngoingOnlineIndexBuildsAsync(recordStore, index, common.config.getLeaseLengthMillis())), common.indexLogMessageKeyValues("OnlineIndexer::checkAnyOngoingOnlineIndexBuilds")); } /** - * Check if the index is being built by any of {@link OnlineIndexer}s (only if they use {@link SynchronizedSession}s). + * Check if the main index is being built. Note that with shared heartbeats, this function will now return true for any active session - mutual or exclusive. + * Where "active session" is determined by an indexing heartbeat that is less than {@link OnlineIndexOperationConfig#DEFAULT_LEASE_LENGTH_MILLIS} old. * @param recordStore record store whose index builds need to be checked * @param index the index to check for ongoing index builds * @return a future that will complete to true if the index is being built and false otherwise */ public static CompletableFuture checkAnyOngoingOnlineIndexBuildsAsync(@Nonnull FDBRecordStore recordStore, @Nonnull Index index) { - return SynchronizedSession.checkActiveSessionExists(recordStore.ensureContextActive(), IndexingSubspaces.indexBuildLockSubspace(recordStore, index)); + return checkAnyOngoingOnlineIndexBuildsAsync(recordStore, index, OnlineIndexOperationConfig.DEFAULT_LEASE_LENGTH_MILLIS); + } + + /** + * Check if the main index is being built. Note that with shared heartbeats, this function will now return true for any active session - mutual or exclusive. + * @param recordStore record store whose index builds need to be checked + * @param index the index to check for ongoing index builds + * @param leasingMilliseconds max heartbeat age to be considered an "active session" + * @return a future that will complete to true if the index is being built and false otherwise + */ + public static CompletableFuture checkAnyOngoingOnlineIndexBuildsAsync(@Nonnull FDBRecordStore recordStore, @Nonnull Index index, long leasingMilliseconds) { + return IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0) + .thenApply(list -> { + long activeTime = System.currentTimeMillis() + leasingMilliseconds; + return list.values().stream().anyMatch(item -> item.getHeartbeatTimeMilliseconds() < activeTime); + }); } /** @@ -599,7 +614,14 @@ private Map indexingStamp(@Null } /** - * Get the current indexing heartbeats for a given index (single target or primary index). + * Get the current indexing heartbeats map for a given index (single target or primary index). + * Each indexing session, while active, updates a heartbeat at every transaction during the indexing's iteration. These + * heartbeats can be used to query active indexing sessions. + * Indexing sessions will attempt to clear their own heartbeat before returning (successfully or exceptionally). However, + * the heartbeat clearing may fail in the DB access level. + * When an index becomes readable any existing heartbeat will be deleted. + * Note that heartbeats that cannot be decrypted will show in the returned map as having creation time of 0 and its info will + * be set to {@link IndexingHeartbeat#INVALID_HEARTBEAT_INFO}. * @param maxCount safety valve to limit number items to read. Typically set to zero to keep unlimited. * @return map of session ids to {@link IndexBuildProto.IndexBuildHeartbeat} */ @@ -611,14 +633,16 @@ public Map getIndexingHeartbeats(int /** * Clear old indexing heartbeats for a given index (single target or primary index). - * @param minAgenMilliseconds minimum heartbeat age (in milliseconds) to clear. + * Typically, heartbeats are deleted either at the end of an indexing sessions or when the index becomes readable. This + * cleanup function can be used if, for any reason, the heartbeats could not be deleted from the database at the end of a session. + * @param minAgeMilliseconds minimum heartbeat age (time elapsed since heartbeat creation, in milliseconds) to clear. * @param maxIteration safety valve to limit number of items to check. Typically set to zero to keep unlimited * @return number of cleared heartbeats */ @API(API.Status.EXPERIMENTAL) - public int clearIndexingHeartbeats(long minAgenMilliseconds, int maxIteration) { + public int clearIndexingHeartbeats(long minAgeMilliseconds, int maxIteration) { return asyncToSync(FDBStoreTimer.Waits.WAIT_INDEX_CLEAR_HEARTBEATS, - getIndexer().clearIndexingHeartbeats(minAgenMilliseconds, maxIteration)); + getIndexer().clearIndexingHeartbeats(minAgeMilliseconds, maxIteration)); } /** @@ -1251,7 +1275,7 @@ public static class Builder { private DesiredAction ifReadable = DesiredAction.CONTINUE; private boolean doAllowUniquePendingState = false; private Set allowedTakeoverSet = null; - private long checkIndexingStampFrequency = 10_000; + private long checkIndexingStampFrequency = 5_000; private boolean useMutualIndexing = false; private List useMutualIndexingBoundaries = null; private boolean allowUnblock = false; @@ -1463,7 +1487,7 @@ public Builder setMutualIndexing() { * by other threads/processes/systems with the exact same parameters, are attempting to concurrently build this * index. To allow that, the indexer will: *
    - *
  1. Divide the records space to fragments, then iterate the fragments in a way that minimize the interference, while + *
  2. Divide the records space to fragments, then iterate the fragments in a way that minimizes the interference, while * indexing each fragment independently.
  3. *
  4. Handle indexing conflicts, when occurred.
  5. *
diff --git a/fdb-record-layer-core/src/main/proto/index_build.proto b/fdb-record-layer-core/src/main/proto/index_build.proto index c7929645cc..e037d4f078 100644 --- a/fdb-record-layer-core/src/main/proto/index_build.proto +++ b/fdb-record-layer-core/src/main/proto/index_build.proto @@ -47,9 +47,10 @@ message IndexBuildIndexingStamp { } message IndexBuildHeartbeat { - required string info = 1; - required int64 genesisTimeMilliseconds = 2; - required int64 heartbeatTimeMilliseconds = 3; + // An heartbeat is set OnlineIndexer at every transaction during the indexing iteration. + required string info = 1; // general information about the indexing session. This can be useful when querying the heartbeats. + required int64 genesisTimeMilliseconds = 2; // indexer's creation time (since epoch) + required int64 heartbeatTimeMilliseconds = 3; // last heartbeats' time (since epoch). This is the only heartbeat value that changes during online indexing. } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeatLowLevelTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeatLowLevelTest.java index 8d68005ecc..43d36b8d6b 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeatLowLevelTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeatLowLevelTest.java @@ -34,6 +34,7 @@ import com.apple.foundationdb.record.test.FDBDatabaseExtension; import com.apple.foundationdb.record.test.TestKeySpace; import com.apple.foundationdb.record.test.TestKeySpacePathManagerExtension; +import com.apple.foundationdb.synchronizedsession.SynchronizedSessionLockedException; import com.google.protobuf.Descriptors; import org.assertj.core.api.Assertions; import org.junit.jupiter.api.BeforeEach; @@ -52,7 +53,6 @@ import static com.apple.foundationdb.record.metadata.Key.Expressions.concat; import static com.apple.foundationdb.record.metadata.Key.Expressions.field; -import static org.junit.jupiter.api.Assertions.assertThrows; public class IndexingHeartbeatLowLevelTest { @RegisterExtension @@ -115,8 +115,57 @@ protected static FDBRecordStoreTestBase.RecordMetaDataHook allIndexesHook(List indexes) { - Assertions.assertThat(indexes).hasSizeGreaterThanOrEqualTo(2); + @Test + void testHeartbeatQuery() { + List indexes = new ArrayList<>(); + indexes.add(new Index("indexA", field("num_value_2"), EmptyKeyExpression.EMPTY, IndexTypes.VALUE, IndexOptions.UNIQUE_OPTIONS)); + indexes.add(new Index("indexD", new GroupingKeyExpression(EmptyKeyExpression.EMPTY, 0), IndexTypes.COUNT)); + FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(indexes); + + final int count = 23; + IndexingHeartbeat[] heartbeats = new IndexingHeartbeat[count]; + for (int i = 0; i < count; i++) { + heartbeats[i] = new IndexingHeartbeat(UUID.randomUUID(), "heartbeat" + i, 100 + i, false); + } + + // populate heartbeats + openSimpleMetaData(hook); + try (FDBRecordContext context = openContext()) { + for (var heartbeat : heartbeats) { + heartbeat.updateHeartbeat(recordStore, indexes.get(0)); + heartbeat.updateHeartbeat(recordStore, indexes.get(1)); + } + context.commit(); + } + + // Verify query operation + for (Index index: indexes) { + try (FDBRecordContext context = openContext()) { + // Query, unlimited + Map queried = + IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); + Assertions.assertThat(queried).hasSize(count); + Assertions.assertThat(queried.keySet()) + .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(heartbeat -> heartbeat.indexerId).collect(Collectors.toList())); + Assertions.assertThat(queried.values().stream().map(IndexBuildProto.IndexBuildHeartbeat::getInfo)) + .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(heartbeat -> heartbeat.info).collect(Collectors.toList())); + Assertions.assertThat(queried.values().stream().map(IndexBuildProto.IndexBuildHeartbeat::getGenesisTimeMilliseconds)) + .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(heartbeat -> heartbeat.genesisTimeMilliseconds).collect(Collectors.toList())); + + // Query, partial + queried = + IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 5).join(); + Assertions.assertThat(queried).hasSize(5); + context.commit(); + } + } + } + + @Test + void testHeartbeatLowLevelClearing() { + List indexes = new ArrayList<>(); + indexes.add(new Index("indexA", field("num_value_2"), EmptyKeyExpression.EMPTY, IndexTypes.VALUE, IndexOptions.UNIQUE_OPTIONS)); + indexes.add(new Index("indexD", new GroupingKeyExpression(EmptyKeyExpression.EMPTY, 0), IndexTypes.COUNT)); FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(indexes); final int count = 10; @@ -135,21 +184,12 @@ void testHeartbeatLowLevel(List indexes) { context.commit(); } - // Verify query/clear operation - openSimpleMetaData(hook); + // Verify clear operation Index index = indexes.get(0); try (FDBRecordContext context = openContext()) { - // Query, unlimited Map queried = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); Assertions.assertThat(queried).hasSize(count); - Assertions.assertThat(queried.keySet()) - .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(heartbeat -> heartbeat.indexerId).collect(Collectors.toList())); - - // Query, partial - queried = - IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 5).join(); - Assertions.assertThat(queried).hasSize(5); // clear, partial int countDeleted = @@ -162,14 +202,11 @@ void testHeartbeatLowLevel(List indexes) { } // Verify that the previous clear does not affect other index - openSimpleMetaData(hook); index = indexes.get(1); try (FDBRecordContext context = openContext()) { Map queried = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 100).join(); Assertions.assertThat(queried).hasSize(count); - Assertions.assertThat(queried.keySet()) - .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(ht -> ht.indexerId).collect(Collectors.toList())); // clear all int countDeleted = @@ -185,31 +222,7 @@ void testHeartbeatLowLevel(List indexes) { } @Test - void testHeartbeatLowLevelValueIndexes() { - List indexes = new ArrayList<>(); - indexes.add(new Index("indexA", field("num_value_2"), EmptyKeyExpression.EMPTY, IndexTypes.VALUE, IndexOptions.UNIQUE_OPTIONS)); - indexes.add(new Index("indexB", field("num_value_3_indexed"), IndexTypes.VALUE)); - testHeartbeatLowLevel(indexes); - } - - @Test - void testHeartbeatLowLevelSumCountIndexes() { - List indexes = new ArrayList<>(); - indexes.add(new Index("indexE", field("num_value_3_indexed").ungrouped(), IndexTypes.SUM)); - indexes.add(new Index("indexD", new GroupingKeyExpression(EmptyKeyExpression.EMPTY, 0), IndexTypes.COUNT)); - testHeartbeatLowLevel(indexes); - } - - @Test - void testHeartbeatLowLevelVersionIndexes() { - List indexes = new ArrayList<>(); - indexes.add(new Index("versionIndex1", concat(field("num_value_2"), VersionKeyExpression.VERSION), IndexTypes.VERSION)); - indexes.add(new Index("versionIndex2", concat(field("num_value_2"), VersionKeyExpression.VERSION), IndexTypes.VERSION)); - testHeartbeatLowLevel(indexes); - } - - @Test - void testCheckAndUpdateByRecords() { + void testCheckAndUpdateNonMutual() { Index index = new Index("indexD", new GroupingKeyExpression(EmptyKeyExpression.EMPTY, 0), IndexTypes.COUNT); FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(List.of(index)); IndexingHeartbeat heartbeat1 = new IndexingHeartbeat(UUID.randomUUID(), "Test", TimeUnit.SECONDS.toMillis(30), false); @@ -222,7 +235,6 @@ void testCheckAndUpdateByRecords() { } // Successfully update heartbeat - openSimpleMetaData(hook); try (FDBRecordContext context = openContext()) { final Map existingHeartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); Assertions.assertThat(existingHeartbeats).hasSize(1); @@ -232,18 +244,17 @@ void testCheckAndUpdateByRecords() { IndexingHeartbeat heartbeat2 = new IndexingHeartbeat(UUID.randomUUID(), "Test", TimeUnit.SECONDS.toMillis(30), false); Assertions.assertThat(heartbeat1.indexerId).isNotEqualTo(heartbeat2.indexerId); - // Fail to create another 'BY_RECORD` heartbeat - openSimpleMetaData(hook); + // Fail to create another non-mutual heartbeat try (FDBRecordContext context = openContext()) { final Map existingHeartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); Assertions.assertThat(existingHeartbeats).hasSize(1); - final CompletionException ex = assertThrows(CompletionException.class, () -> heartbeat2.checkAndUpdateHeartbeat(recordStore, index).join()); - Assertions.assertThat(ex.getMessage()).contains("SynchronizedSessionLockedException"); + Assertions.assertThatThrownBy(() -> heartbeat2.checkAndUpdateHeartbeat(recordStore, index).join()) + .isInstanceOf(CompletionException.class) + .hasCauseInstanceOf(SynchronizedSessionLockedException.class); context.commit(); } // Successfully clear heartbeat1 - openSimpleMetaData(hook); try (FDBRecordContext context = openContext()) { final Map existingHeartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); Assertions.assertThat(existingHeartbeats).hasSize(1); @@ -252,14 +263,12 @@ void testCheckAndUpdateByRecords() { } // Successfully update heartbeat2 - openSimpleMetaData(hook); try (FDBRecordContext context = openContext()) { heartbeat2.checkAndUpdateHeartbeat(recordStore, index).join(); context.commit(); } // Successfully clear heartbeat2 - openSimpleMetaData(hook); try (FDBRecordContext context = openContext()) { heartbeat2.clearHeartbeat(recordStore, index); final Map existingHeartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); @@ -282,26 +291,28 @@ void testCheckAndUpdateMutual() { // Successfully check//update all heartbeats openSimpleMetaData(hook); try (FDBRecordContext context = openContext()) { - for (IndexingHeartbeat heartbeat: heartbeats) { - heartbeat.checkAndUpdateHeartbeat(recordStore, index).join(); + for (int i = 0; i < 3; i++) { + for (IndexingHeartbeat heartbeat: heartbeats) { + heartbeat.checkAndUpdateHeartbeat(recordStore, index).join(); + } } context.commit(); } // Check count, clear all - openSimpleMetaData(hook); try (FDBRecordContext context = openContext()) { final Map existingHeartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); Assertions.assertThat(existingHeartbeats).hasSize(count); - for (IndexingHeartbeat heartbeat: heartbeats) { - heartbeat.clearHeartbeat(recordStore, index); + for (int i = 0; i < 3; i++) { + for (IndexingHeartbeat heartbeat: heartbeats) { + heartbeat.clearHeartbeat(recordStore, index); + } } context.commit(); } // verify cleared - openSimpleMetaData(hook); try (FDBRecordContext context = openContext()) { final Map existingHeartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); Assertions.assertThat(existingHeartbeats).isEmpty(); @@ -310,7 +321,7 @@ void testCheckAndUpdateMutual() { } @Test - void testExpiredHeartbeat() throws InterruptedException { + void testSetHeartbeatAfterOtherHeartbeatExpiration() throws InterruptedException { Index index = new Index("versionIndex1", concat(field("num_value_2"), VersionKeyExpression.VERSION), IndexTypes.VERSION); FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(List.of(index)); IndexingHeartbeat heartbeat1 = new IndexingHeartbeat(UUID.randomUUID(), "Test", TimeUnit.SECONDS.toMillis(10), false); @@ -328,7 +339,6 @@ void testExpiredHeartbeat() throws InterruptedException { Assertions.assertThat(heartbeat1.indexerId).isNotEqualTo(heartbeat2.indexerId); // heartbeat2 successfully takes over - openSimpleMetaData(hook); try (FDBRecordContext context = openContext()) { final Map existingHeartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); Assertions.assertThat(existingHeartbeats).hasSize(1); @@ -338,7 +348,7 @@ void testExpiredHeartbeat() throws InterruptedException { } @Test - void testHeartbeatExpiration() throws InterruptedException { + void testFailSetHeartbeatBeforeOtherHeartbeatExpiration() throws InterruptedException { Index index = new Index("indexD", new GroupingKeyExpression(EmptyKeyExpression.EMPTY, 0), IndexTypes.COUNT); FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(List.of(index)); @@ -353,8 +363,7 @@ void testHeartbeatExpiration() throws InterruptedException { } Thread.sleep(100); - // Expect heartbeatA to expire after 5 milliseconds, and successfully set heartbeatB - openSimpleMetaData(hook); + // heartbeatB would have respected heartbeatB's lock for 5 milliseconds only. Now successfully set itself. try (FDBRecordContext context = openContext()) { heartbeatB.checkAndUpdateHeartbeat(recordStore, index).join(); context.commit(); @@ -362,11 +371,103 @@ void testHeartbeatExpiration() throws InterruptedException { // Expect heartbeatA to fail check/update // Note: if become flakey, increase the least time of heartbeatA + try (FDBRecordContext context = openContext()) { + Assertions.assertThatThrownBy(() -> heartbeatA.checkAndUpdateHeartbeat(recordStore, index).join()) + .isInstanceOf(CompletionException.class) + .hasCauseInstanceOf(SynchronizedSessionLockedException.class); + context.commit(); + } + } + + @Test + void testHeartbeatClearOldHeartbeats() throws InterruptedException { + Index index = new Index("versionIndex1", concat(field("num_value_2"), VersionKeyExpression.VERSION), IndexTypes.VERSION); + FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(List.of(index)); + IndexingHeartbeat heartbeat1 = new IndexingHeartbeat(UUID.randomUUID(), "Test", TimeUnit.SECONDS.toMillis(10), true); + IndexingHeartbeat heartbeat2 = new IndexingHeartbeat(UUID.randomUUID(), "Test", TimeUnit.SECONDS.toMillis(10), true); + + // Successfully create heartbeat1 + openSimpleMetaData(hook); + try (FDBRecordContext context = openContext()) { + heartbeat1.checkAndUpdateHeartbeat(recordStore, index).join(); + heartbeat2.checkAndUpdateHeartbeat(recordStore, index).join(); + context.commit(); + } + + // Delay 20, clear anything older than 5 milliseconds + Thread.sleep(20); + try (FDBRecordContext context = openContext()) { + final Map existingHeartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); + Assertions.assertThat(existingHeartbeats).hasSize(2); + final Integer numDeleted = IndexingHeartbeat.clearIndexingHeartbeats(recordStore, index, 5, 0).join(); + Assertions.assertThat(numDeleted).isEqualTo(2); + context.commit(); + } + } + + @Test + void testMixedMutualNonMutualHeartbeats() { + // This scenario should never happen because of the indexing typestamp protection. Testing it anyway... + Index index = new Index("versionIndex1", concat(field("num_value_2"), VersionKeyExpression.VERSION), IndexTypes.VERSION); + FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(List.of(index)); + IndexingHeartbeat heartbeatMutual = new IndexingHeartbeat(UUID.randomUUID(), "Test", TimeUnit.SECONDS.toMillis(10), true); + IndexingHeartbeat heartbeatExclusive = new IndexingHeartbeat(UUID.randomUUID(), "Test", TimeUnit.SECONDS.toMillis(10), false); + + // lock exclusive, then successfully lock mutual. + openSimpleMetaData(hook); + try (FDBRecordContext context = openContext()) { + heartbeatExclusive.checkAndUpdateHeartbeat(recordStore, index).join(); + context.commit(); + } + + try (FDBRecordContext context = openContext()) { + heartbeatMutual.checkAndUpdateHeartbeat(recordStore, index).join(); + // and clear + heartbeatExclusive.clearHeartbeat(recordStore, index); + heartbeatMutual.clearHeartbeat(recordStore, index); + context.commit(); + } + + // lock mutual, then fail to lock exclusive + try (FDBRecordContext context = openContext()) { + final Map heartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); + Assertions.assertThat(heartbeats).isEmpty(); + heartbeatMutual.checkAndUpdateHeartbeat(recordStore, index).join(); + context.commit(); + } + + try (FDBRecordContext context = openContext()) { + Assertions.assertThatThrownBy(() -> heartbeatExclusive.checkAndUpdateHeartbeat(recordStore, index).join()) + .isInstanceOf(CompletionException.class) + .hasCauseInstanceOf(SynchronizedSessionLockedException.class); + // and clear + heartbeatMutual.clearHeartbeat(recordStore, index); + context.commit(); + } + + } + + @Test + void testUnparseableHeartbeat() { + Index index = new Index("versionIndex1", concat(field("num_value_2"), VersionKeyExpression.VERSION), IndexTypes.VERSION); + FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(List.of(index)); + + // lock exclusive, then successfully lock mutual. openSimpleMetaData(hook); try (FDBRecordContext context = openContext()) { - final CompletionException ex = assertThrows(CompletionException.class, () -> heartbeatA.checkAndUpdateHeartbeat(recordStore, index).join()); - Assertions.assertThat(ex.getMessage()).contains("SynchronizedSessionLockedException"); + byte[] key = IndexingSubspaces.indexHeartbeatSubspace(recordStore, index, UUID.randomUUID()).pack(); + byte[] value = "meaningless byte value".getBytes(); + recordStore.ensureContextActive().set(key, value); + context.commit(); + } + + IndexingHeartbeat heartbeat = new IndexingHeartbeat(UUID.randomUUID(), "Test", TimeUnit.SECONDS.toMillis(30), false); + try (FDBRecordContext context = openContext()) { + final Map existingHeartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); + Assertions.assertThat(existingHeartbeats).hasSize(1); + heartbeat.checkAndUpdateHeartbeat(recordStore, index); context.commit(); } } + } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerBuildIndexTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerBuildIndexTest.java index 2d7d1cea87..0fa2fdc353 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerBuildIndexTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerBuildIndexTest.java @@ -21,6 +21,7 @@ package com.apple.foundationdb.record.provider.foundationdb; import com.apple.foundationdb.async.AsyncUtil; +import com.apple.foundationdb.async.MoreAsyncUtil; import com.apple.foundationdb.async.RangeSet; import com.apple.foundationdb.record.IndexBuildProto; import com.apple.foundationdb.record.IndexState; @@ -34,6 +35,7 @@ import com.apple.foundationdb.tuple.Tuple; import com.apple.test.RandomizedTestUtils; import com.google.protobuf.Message; +import org.junit.jupiter.api.Assertions; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -219,6 +221,8 @@ void singleRebuild( } } + // This is also checked later in this test with indexBuilder.getIndexingHeartbeats. For now, keeping both versions. + // But at some point checkAnyOngoingOnlineIndexBuildsAsync will be deprecated. buildFuture = MoreAsyncUtil.composeWhenComplete( buildFuture, (result, ex) -> indexBuilder.checkAnyOngoingOnlineIndexBuildsAsync().thenAccept(Assertions::assertFalse), @@ -293,8 +297,12 @@ void singleRebuild( } } try (OnlineIndexer indexBuilder = newIndexerBuilder(index).build()) { + // Assert no ongoing sessions final Map heartbeats = indexBuilder.getIndexingHeartbeats(0); assertTrue(heartbeats.isEmpty()); + + // Same thing + assertFalse(indexBuilder.checkAnyOngoingOnlineIndexBuilds()); } KeyValueLogMessage msg = KeyValueLogMessage.build("building index - completed", TestLogMessageKeys.INDEX, index); msg.addKeysAndValues(timer.getKeysAndValues()); diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerSimpleTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerSimpleTest.java index 58a92c58d5..773e24a5b4 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerSimpleTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerSimpleTest.java @@ -32,12 +32,15 @@ import com.apple.foundationdb.record.metadata.IndexTypes; import com.apple.foundationdb.record.metadata.MetaDataException; import com.apple.foundationdb.record.util.pair.Pair; +import com.apple.foundationdb.synchronizedsession.SynchronizedSessionLockedException; +import com.apple.test.BooleanSource; import com.apple.test.Tags; import com.google.common.collect.ImmutableMap; import org.hamcrest.MatcherAssert; import org.hamcrest.Matchers; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.Test; +import org.junit.jupiter.params.ParameterizedTest; import javax.annotation.Nonnull; import javax.annotation.Nullable; @@ -48,6 +51,9 @@ import java.util.Queue; import java.util.concurrent.CompletableFuture; import java.util.concurrent.ExecutionException; +import java.util.concurrent.Semaphore; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; import java.util.concurrent.atomic.AtomicLong; import java.util.function.Supplier; @@ -888,9 +894,11 @@ public void runWithWeakReadSemantics() { } } - @Test + @ParameterizedTest + @BooleanSource @SuppressWarnings("removal") - void testDeprecatedSetUseSynchronizedSession() { + void testDeprecatedSetUseSynchronizedSession(boolean useSynchronizedSession) throws InterruptedException { + // regardless of useSynchronizedSession's value, the build should be exclusive List records = LongStream.range(0, 20).mapToObj(val -> TestRecords1Proto.MySimpleRecord.newBuilder().setRecNo(val).setNumValue2((int)val + 1).build() ).collect(Collectors.toList()); @@ -903,16 +911,72 @@ void testDeprecatedSetUseSynchronizedSession() { context.commit(); } + // phase 1: successfully build openSimpleMetaData(hook); disableAll(List.of(index)); - - openSimpleMetaData(hook); try (OnlineIndexer indexBuilder = newIndexerBuilder(index) - .setUseSynchronizedSession(true) + .setUseSynchronizedSession(useSynchronizedSession) .build()) { indexBuilder.buildIndex(); } + assertReadable(index); + + // Now disable and ensure exclusive build + disableAll(List.of(index)); + Semaphore pauseMutualBuildSemaphore = new Semaphore(1); + Semaphore startBuildingSemaphore = new Semaphore(1); + pauseMutualBuildSemaphore.acquire(); + startBuildingSemaphore.acquire(); + AtomicBoolean passed = new AtomicBoolean(false); + Thread t1 = new Thread(() -> { + // build index and pause halfway, allowing an active session test + try (OnlineIndexer indexBuilder = newIndexerBuilder(index) + .setLeaseLengthMillis(TimeUnit.SECONDS.toMillis(20)) + .setLimit(4) + .setConfigLoader(old -> { + if (passed.get()) { + try { + startBuildingSemaphore.release(); + pauseMutualBuildSemaphore.acquire(); // pause to try building indexes + } catch (InterruptedException e) { + throw new RuntimeException(e); + } finally { + pauseMutualBuildSemaphore.release(); + } + } else { + passed.set(true); + } + return old; + }) + .build()) { + indexBuilder.buildIndex(); + } + }); + t1.start(); + startBuildingSemaphore.acquire(); + startBuildingSemaphore.release(); + + // Fail to start another indexer + try (OnlineIndexer indexBuilder = newIndexerBuilder(index) + .build()) { + assertTrue(indexBuilder.checkAnyOngoingOnlineIndexBuildsAsync().join()); + assertThrows(SynchronizedSessionLockedException.class, indexBuilder::buildIndex); + } + // Successfully convert to a mutual indexer + try (OnlineIndexer indexBuilder = newIndexerBuilder(index) + .setIndexingPolicy(OnlineIndexer.IndexingPolicy.newBuilder() + .setMutualIndexing() + .allowTakeoverContinue() + .build()) + .build()) { + assertTrue(indexBuilder.checkAnyOngoingOnlineIndexBuildsAsync().join()); + indexBuilder.buildIndex(); + } + // let the other thread finish indexing + pauseMutualBuildSemaphore.release(); + t1.join(); + // happy indexes assertion assertReadable(index); } } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java index c110536316..30bf079d40 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java @@ -238,6 +238,7 @@ void testMutualIndexersHeartbeatsClearAfterBuild() throws InterruptedException { startSemaphore.acquire(); Thread.sleep(100); } catch (InterruptedException e) { + Thread.currentThread().interrupt(); throw new RuntimeException(e); } try (OnlineIndexer indexer = newIndexerBuilder(indexes).build()) { diff --git a/gradle.properties b/gradle.properties index 26dc08bb4b..7fc6e8c302 100644 --- a/gradle.properties +++ b/gradle.properties @@ -19,7 +19,7 @@ # rootProject.name=fdb-record-layer -version=4.5.13.0 +version=4.5.11.0 releaseBuild=false # this should be false for release branches (i.e. if there is no -SNAPSHOT on the above version) From 0e14895c16cf1523046cc8fa338b0c976fc74c2e Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Tue, 2 Sep 2025 18:34:32 -0400 Subject: [PATCH 19/26] Apply (even more of) Scott's requested changes --- .../provider/foundationdb/IndexingBase.java | 45 ++++-------- .../foundationdb/IndexingHeartbeat.java | 51 +++++++------- .../provider/foundationdb/OnlineIndexer.java | 42 ++++++----- .../src/main/proto/index_build.proto | 11 +-- .../IndexingHeartbeatLowLevelTest.java | 53 +++++++++++--- .../OnlineIndexerIndexFromIndexTest.java | 16 +---- .../OnlineIndexerMultiTargetTest.java | 33 +-------- .../foundationdb/OnlineIndexerMutualTest.java | 34 ++------- .../foundationdb/OnlineIndexerSimpleTest.java | 38 +++------- .../foundationdb/OnlineIndexerTest.java | 16 ++++- .../OnlineIndexingHeartbeatTest.java | 70 +++++++------------ 11 files changed, 170 insertions(+), 239 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java index 9f0b89afce..dc2db639cf 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java @@ -100,7 +100,6 @@ public abstract class IndexingBase { private StoreTimerSnapshot lastProgressSnapshot = null; private boolean forceStampOverwrite = false; private final long startingTimeMillis; - private long lastTypeStampCheckMillis; private Map indexingMergerMap = null; @Nullable private IndexingHeartbeat heartbeat = null; // this will stay null for index scrubbing @@ -119,7 +118,6 @@ public abstract class IndexingBase { this.isScrubber = isScrubber; this.throttle = new IndexingThrottle(common, isScrubber); this.startingTimeMillis = System.currentTimeMillis(); - this.lastTypeStampCheckMillis = startingTimeMillis; } // helper functions @@ -176,8 +174,9 @@ public CompletableFuture buildIndexAsync(boolean markReadable) { message.addKeyAndValue(LogMessageKeys.RESULT, "success"); LOGGER.info(message.toString()); } - // Here: if the heartbeat was *not* cleared while marking the index readable, it would be cleared in - // these dedicated transaction. Heartbeat clearing is not a blocker but a "best effort" operation. + // Here: if the heartbeats were not fully cleared while marking the index as readable, they will be cleared in + // this dedicated transaction. Clearing the heartbeats at the end of the indexing session is a "best effort" + // operation, hence exceptions are ignored. return clearHeartbeats() .handle((ignoreRet, ignoreEx) -> null); }, @@ -825,18 +824,17 @@ private CompletableFuture hadTransactionReachedLimits(FDBRecordStore st } private CompletableFuture validateTypeStamp(@Nonnull FDBRecordStore store) { - if (shouldValidate()) { - // check other heartbeats (if exclusive) & typestamp - final IndexBuildProto.IndexBuildIndexingStamp expectedTypeStamp = getIndexingTypeStamp(store); - return forEachTargetIndex(index -> CompletableFuture.allOf( - updateHeartbeat(true, store, index), - store.loadIndexingTypeStampAsync(index) - .thenAccept(typeStamp -> validateTypeStamp(typeStamp, expectedTypeStamp, index)) - )); - } else { - // update only - return forEachTargetIndex(index -> updateHeartbeat(false, store, index)); + // check other heartbeats (if exclusive) & typestamp + if (isScrubber) { + // Scrubber's type-stamp is never commited. It is protected by expecting a READABLE index state. + return AsyncUtil.DONE; } + final IndexBuildProto.IndexBuildIndexingStamp expectedTypeStamp = getIndexingTypeStamp(store); + return forEachTargetIndex(index -> CompletableFuture.allOf( + updateHeartbeat(true, store, index), + store.loadIndexingTypeStampAsync(index) + .thenAccept(typeStamp -> validateTypeStamp(typeStamp, expectedTypeStamp, index)) + )); } private CompletableFuture updateHeartbeat(boolean validate, FDBRecordStore store, Index index) { @@ -877,22 +875,6 @@ private void clearHeartbeatForIndex(FDBRecordStore store, Index index) { } } - - private boolean shouldValidate() { - final long minimalInterval = policy.getCheckIndexingMethodFrequencyMilliseconds(); - if (minimalInterval < 0 || isScrubber) { - return false; - } - if (minimalInterval > 0) { - final long now = System.currentTimeMillis(); - if (now < lastTypeStampCheckMillis + minimalInterval) { - return false; - } - lastTypeStampCheckMillis = now; - } - return true; - } - private void validateTypeStamp(final IndexBuildProto.IndexBuildIndexingStamp typeStamp, final IndexBuildProto.IndexBuildIndexingStamp expectedTypeStamp, Index index) { @@ -1028,7 +1010,6 @@ public CompletableFuture rebuildIndexAsync(@Nonnull FDBRecordStore store) })) .thenCompose(vignore -> setIndexingTypeOrThrow(store, false)) .thenCompose(vignore -> rebuildIndexInternalAsync(store)) - // If any of the indexes' heartbeats, for any reason, was not cleared during "mark readable", clear it here .whenComplete((ignore, ignoreEx) -> clearHeartbeats(store)); } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java index 8ac1f61061..9e7f9ad548 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java @@ -93,33 +93,34 @@ public CompletableFuture checkAndUpdateHeartbeat(@Nonnull FDBRecordStore s 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); - } - } - } catch (InvalidProtocolBufferException e) { - if (logger.isWarnEnabled()) { - logger.warn(KeyValueLogMessage.of("Bad indexing heartbeat item", - LogMessageKeys.KEY, kv.getKey(), - LogMessageKeys.VALUE, kv.getValue())); - } - } + checkSingleHeartbeat(store, index, kv, now); return true; })) - .thenApply(ignore -> { - updateHeartbeat(store, index); - return null; - }); + .thenAccept(ignore -> updateHeartbeat(store, index)); + } + + private void checkSingleHeartbeat(final @Nonnull FDBRecordStore store, final @Nonnull Index index, final KeyValue kv, final long now) { + 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); + } + } + } catch (InvalidProtocolBufferException e) { + if (logger.isWarnEnabled()) { + logger.warn(KeyValueLogMessage.of("Bad indexing heartbeat item", + LogMessageKeys.KEY, kv.getKey(), + LogMessageKeys.VALUE, kv.getValue())); + } + } } public void clearHeartbeat(@Nonnull FDBRecordStore store, @Nonnull Index index) { diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java index c9234a7b68..bb734acbb6 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java @@ -315,11 +315,11 @@ private IndexingBase getIndexer() { } /** + * Deprecated and unused. * This {@link Exception} can be thrown in the case that one calls one of the methods * that explicitly state that they are building an unbuilt range, i.e., a range of keys * that contains no keys which have yet been processed by the {@link OnlineIndexer} * during an index build. - * Deprecated and unused. */ @API(API.Status.DEPRECATED) @SuppressWarnings("serial") @@ -480,7 +480,7 @@ public boolean checkAnyOngoingOnlineIndexBuilds() { } /** - * Check if the main index is being built. Note that with shared heartbeats, this function will now return true for any active session - mutual or exclusive. + * Check if the provided index is being built. Note that with shared heartbeats, this function will now return true for any active session - mutual or exclusive. * @return a future that will complete to true if the index is being built and false otherwise */ public CompletableFuture checkAnyOngoingOnlineIndexBuildsAsync() { @@ -490,7 +490,7 @@ public CompletableFuture checkAnyOngoingOnlineIndexBuildsAsync() { } /** - * Check if the main index is being built. Note that with shared heartbeats, this function will now return true for any active session - mutual or exclusive. + * Check if the provided index is being built. Note that with shared heartbeats, this function will now return true for any active session - mutual or exclusive. * Where "active session" is determined by an indexing heartbeat that is less than {@link OnlineIndexOperationConfig#DEFAULT_LEASE_LENGTH_MILLIS} old. * @param recordStore record store whose index builds need to be checked * @param index the index to check for ongoing index builds @@ -501,7 +501,7 @@ public static CompletableFuture checkAnyOngoingOnlineIndexBuildsAsync(@ } /** - * Check if the main index is being built. Note that with shared heartbeats, this function will now return true for any active session - mutual or exclusive. + * Check if the provided index is being built. Note that with shared heartbeats, this function will now return true for any active session - mutual or exclusive. * @param recordStore record store whose index builds need to be checked * @param index the index to check for ongoing index builds * @param leasingMilliseconds max heartbeat age to be considered an "active session" @@ -517,7 +517,14 @@ public static CompletableFuture checkAnyOngoingOnlineIndexBuildsAsync(@ /** * Builds an index across multiple transactions. - * This is a slow and retrying operation that is intended to be executed by background processes. + *

+ * If the indexing session is not mutual, it will stop with {@link com.apple.foundationdb.synchronizedsession.SynchronizedSessionLockedException} + * if there is another active indexing session on the same index. It first checks and updates index states and + * clear index data respecting the {@link IndexStatePrecondition} being set. It then builds the index across + * multiple transactions honoring the rate-limiting parameters set in the constructor of this class. It also retries + * any retriable errors that it encounters while it runs the build. At the end, it marks the index readable in the + * store. + *

* @return a future that will be ready when the build has completed * @throws com.apple.foundationdb.synchronizedsession.SynchronizedSessionLockedException the build is stopped * because there may be another build running actively on this index. @@ -586,9 +593,10 @@ public Map queryIndexingStamps( /** * Block partly built indexes, preventing continuation. - * Active indexing sessions will check for this block according to {@link IndexingPolicy.Builder#checkIndexingStampFrequencyMilliseconds(long)}.} - * @param id if non null, will be added to the "indexing stamp" as an id/hint for the blocking reason. - * @param ttlSeconds if non null, the block will automatically expire after this value (in seconds). + * Active indexing sessions will check for this block during every iterating transaction, which still requires the caller to + * wait a few seconds before assuming that all indexing had stopped. + * @param id if non-null, will be added to the "indexing stamp" as an id/hint for the blocking reason. + * @param ttlSeconds if non-null, the block will automatically expire after this value (in seconds). * @return a map of target indexes and their "indexing stamps" after the change. */ @API(API.Status.EXPERIMENTAL) @@ -635,7 +643,7 @@ public Map getIndexingHeartbeats(int * Clear old indexing heartbeats for a given index (single target or primary index). * Typically, heartbeats are deleted either at the end of an indexing sessions or when the index becomes readable. This * cleanup function can be used if, for any reason, the heartbeats could not be deleted from the database at the end of a session. - * @param minAgeMilliseconds minimum heartbeat age (time elapsed since heartbeat creation, in milliseconds) to clear. + * @param minAgeMilliseconds minimum heartbeat age (time elapsed since the last heartbeat, in milliseconds) to clear. * @param maxIteration safety valve to limit number of items to check. Typically set to zero to keep unlimited * @return number of cleared heartbeats */ @@ -956,7 +964,6 @@ public static class IndexingPolicy { private final DesiredAction ifReadable; private final boolean allowUniquePendingState; private final Set allowedTakeoverSet; - private final long checkIndexingMethodFrequencyMilliseconds; private final boolean mutualIndexing; private final List mutualIndexingBoundaries; private final boolean allowUnblock; @@ -1015,7 +1022,6 @@ public enum TakeoverTypes { private IndexingPolicy(@Nullable String sourceIndex, @Nullable Object sourceIndexSubspaceKey, boolean forbidRecordScan, DesiredAction ifDisabled, DesiredAction ifWriteOnly, DesiredAction ifMismatchPrevious, DesiredAction ifReadable, boolean allowUniquePendingState, Set allowedTakeoverSet, - long checkIndexingMethodFrequencyMilliseconds, boolean mutualIndexing, List mutualIndexingBoundaries, boolean allowUnblock, String allowUnblockId, long initialMergesCountLimit, @@ -1029,7 +1035,6 @@ private IndexingPolicy(@Nullable String sourceIndex, @Nullable Object sourceInde this.ifReadable = ifReadable; this.allowUniquePendingState = allowUniquePendingState; this.allowedTakeoverSet = allowedTakeoverSet; - this.checkIndexingMethodFrequencyMilliseconds = checkIndexingMethodFrequencyMilliseconds; this.mutualIndexing = mutualIndexing; this.mutualIndexingBoundaries = mutualIndexingBoundaries; this.allowUnblock = allowUnblock; @@ -1101,7 +1106,6 @@ public Builder toBuilder() { .setIfReadable(ifReadable) .allowUniquePendingState(allowUniquePendingState) .allowTakeoverContinue(allowedTakeoverSet) - .checkIndexingStampFrequencyMilliseconds(checkIndexingMethodFrequencyMilliseconds) .setMutualIndexing(mutualIndexing) .setMutualIndexingBoundaries(mutualIndexingBoundaries) .setAllowUnblock(allowUnblock, allowUnblockId) @@ -1226,11 +1230,13 @@ public boolean shouldAllowUnblock(String stampBlockId) { } /** + * Deprecated and unused. * If negative, avoid checks. Else, minimal interval between checks. - * @return minmal interval in milliseconds. + * @return minimal interval in milliseconds. */ + @API(API.Status.DEPRECATED) public long getCheckIndexingMethodFrequencyMilliseconds() { - return this.checkIndexingMethodFrequencyMilliseconds; + return 0; } /** @@ -1275,7 +1281,6 @@ public static class Builder { private DesiredAction ifReadable = DesiredAction.CONTINUE; private boolean doAllowUniquePendingState = false; private Set allowedTakeoverSet = null; - private long checkIndexingStampFrequency = 5_000; private boolean useMutualIndexing = false; private List useMutualIndexingBoundaries = null; private boolean allowUnblock = false; @@ -1457,6 +1462,7 @@ public Builder allowTakeoverContinue(@Nullable Collection allowed } /** + * Deprecated - for better consistency, the type stamp will now be validated during every iterating transaction. * During indexing, the indexer can check the current indexing stamp and throw an exception if it had changed. * This may happen if another indexing type takes over or by an indexing block (see {@link #indexingStamp}). * The argument may be: @@ -1467,8 +1473,9 @@ public Builder allowTakeoverContinue(@Nullable Collection allowed * @param frequency : If negative, avoid checks. Else, minimal interval between checks * @return this builder. */ + @API(API.Status.DEPRECATED) public Builder checkIndexingStampFrequencyMilliseconds(long frequency) { - this.checkIndexingStampFrequency = frequency; + // No-op return this; } @@ -1577,7 +1584,6 @@ public IndexingPolicy build() { return new IndexingPolicy(sourceIndex, sourceIndexSubspaceKey, forbidRecordScan, ifDisabled, ifWriteOnly, ifMismatchPrevious, ifReadable, doAllowUniquePendingState, allowedTakeoverSet, - checkIndexingStampFrequency, useMutualIndexing, useMutualIndexingBoundaries, allowUnblock, allowUnblockId, initialMergesCountLimit, reverseScanOrder); } diff --git a/fdb-record-layer-core/src/main/proto/index_build.proto b/fdb-record-layer-core/src/main/proto/index_build.proto index e037d4f078..469917fce9 100644 --- a/fdb-record-layer-core/src/main/proto/index_build.proto +++ b/fdb-record-layer-core/src/main/proto/index_build.proto @@ -46,11 +46,12 @@ message IndexBuildIndexingStamp { optional string blockID = 7; // optional, a short string that describes the reason for the block. } - message IndexBuildHeartbeat { - // An heartbeat is set OnlineIndexer at every transaction during the indexing iteration. - required string info = 1; // general information about the indexing session. This can be useful when querying the heartbeats. - required int64 genesisTimeMilliseconds = 2; // indexer's creation time (since epoch) - required int64 heartbeatTimeMilliseconds = 3; // last heartbeats' time (since epoch). This is the only heartbeat value that changes during online indexing. +// An heartbeat is maintained by an OnlineIndexer at every transaction during the indexing iteration, to ensure there +// aren't too many concurrent processes trying to build an index +message IndexBuildHeartbeat { + optional string info = 1; // general information about the indexing session. This can be useful when querying the heartbeats. + optional int64 genesisTimeMilliseconds = 2; // indexer's creation time (since epoch) + optional int64 heartbeatTimeMilliseconds = 3; // last heartbeat's time (since epoch). This is the only heartbeat value that changes during online indexing. } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeatLowLevelTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeatLowLevelTest.java index 43d36b8d6b..0ac4611587 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeatLowLevelTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeatLowLevelTest.java @@ -162,7 +162,7 @@ void testHeartbeatQuery() { } @Test - void testHeartbeatLowLevelClearing() { + void testHeartbeatClearing() { List indexes = new ArrayList<>(); indexes.add(new Index("indexA", field("num_value_2"), EmptyKeyExpression.EMPTY, IndexTypes.VALUE, IndexOptions.UNIQUE_OPTIONS)); indexes.add(new Index("indexD", new GroupingKeyExpression(EmptyKeyExpression.EMPTY, 0), IndexTypes.COUNT)); @@ -363,7 +363,7 @@ void testFailSetHeartbeatBeforeOtherHeartbeatExpiration() throws InterruptedExce } Thread.sleep(100); - // heartbeatB would have respected heartbeatB's lock for 5 milliseconds only. Now successfully set itself. + // heartbeatB would have respected heartbeatA's lock for 5 milliseconds only. Now successfully set itself. try (FDBRecordContext context = openContext()) { heartbeatB.checkAndUpdateHeartbeat(recordStore, index).join(); context.commit(); @@ -385,6 +385,7 @@ void testHeartbeatClearOldHeartbeats() throws InterruptedException { FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(List.of(index)); IndexingHeartbeat heartbeat1 = new IndexingHeartbeat(UUID.randomUUID(), "Test", TimeUnit.SECONDS.toMillis(10), true); IndexingHeartbeat heartbeat2 = new IndexingHeartbeat(UUID.randomUUID(), "Test", TimeUnit.SECONDS.toMillis(10), true); + IndexingHeartbeat heartbeat3 = new IndexingHeartbeat(UUID.randomUUID(), "Test", TimeUnit.SECONDS.toMillis(10), true); // Successfully create heartbeat1 openSimpleMetaData(hook); @@ -403,11 +404,22 @@ void testHeartbeatClearOldHeartbeats() throws InterruptedException { Assertions.assertThat(numDeleted).isEqualTo(2); context.commit(); } + + // Now make sure that clear indexing heartbeats does not remove when it should not + try (FDBRecordContext context = openContext()) { + heartbeat3.checkAndUpdateHeartbeat(recordStore, index).join(); + final Map existingHeartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); + Assertions.assertThat(existingHeartbeats).hasSize(1); + final Integer numDeleted = IndexingHeartbeat.clearIndexingHeartbeats(recordStore, index, TimeUnit.SECONDS.toMillis(10), 0).join(); + Assertions.assertThat(numDeleted).isEqualTo(0); + context.commit(); + } + } @Test - void testMixedMutualNonMutualHeartbeats() { - // This scenario should never happen because of the indexing typestamp protection. Testing it anyway... + void testMixedNonMutualThenMutualHeartbeats() { + // This scenario should never happen because of the indexing type-stamp protection. Testing it anyway... Index index = new Index("versionIndex1", concat(field("num_value_2"), VersionKeyExpression.VERSION), IndexTypes.VERSION); FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(List.of(index)); IndexingHeartbeat heartbeatMutual = new IndexingHeartbeat(UUID.randomUUID(), "Test", TimeUnit.SECONDS.toMillis(10), true); @@ -421,14 +433,22 @@ void testMixedMutualNonMutualHeartbeats() { } try (FDBRecordContext context = openContext()) { + // Mutual: Successfully updates heartbeatMutual.checkAndUpdateHeartbeat(recordStore, index).join(); - // and clear - heartbeatExclusive.clearHeartbeat(recordStore, index); - heartbeatMutual.clearHeartbeat(recordStore, index); context.commit(); } + } + + @Test + void testMixedMutualThenNonMutualHeartbeats() { + // This scenario should never happen because of the indexing typestamp protection. Testing it anyway... + Index index = new Index("versionIndex1", concat(field("num_value_2"), VersionKeyExpression.VERSION), IndexTypes.VERSION); + FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(List.of(index)); + IndexingHeartbeat heartbeatMutual = new IndexingHeartbeat(UUID.randomUUID(), "Test", TimeUnit.SECONDS.toMillis(10), true); + IndexingHeartbeat heartbeatExclusive = new IndexingHeartbeat(UUID.randomUUID(), "Test", TimeUnit.SECONDS.toMillis(10), false); // lock mutual, then fail to lock exclusive + openSimpleMetaData(hook); try (FDBRecordContext context = openContext()) { final Map heartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); Assertions.assertThat(heartbeats).isEmpty(); @@ -444,7 +464,6 @@ void testMixedMutualNonMutualHeartbeats() { heartbeatMutual.clearHeartbeat(recordStore, index); context.commit(); } - } @Test @@ -452,7 +471,7 @@ void testUnparseableHeartbeat() { Index index = new Index("versionIndex1", concat(field("num_value_2"), VersionKeyExpression.VERSION), IndexTypes.VERSION); FDBRecordStoreTestBase.RecordMetaDataHook hook = allIndexesHook(List.of(index)); - // lock exclusive, then successfully lock mutual. + // write unparseable data where a heartbeat should exist openSimpleMetaData(hook); try (FDBRecordContext context = openContext()) { byte[] key = IndexingSubspaces.indexHeartbeatSubspace(recordStore, index, UUID.randomUUID()).pack(); @@ -465,9 +484,21 @@ void testUnparseableHeartbeat() { try (FDBRecordContext context = openContext()) { final Map existingHeartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); Assertions.assertThat(existingHeartbeats).hasSize(1); - heartbeat.checkAndUpdateHeartbeat(recordStore, index); + heartbeat.checkAndUpdateHeartbeat(recordStore, index).join(); context.commit(); } - } + // Make sure that the unparsable heartbeat can be cleared + try (FDBRecordContext context = openContext()) { + heartbeat.checkAndUpdateHeartbeat(recordStore, index).join(); + Map existingHeartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); + Assertions.assertThat(existingHeartbeats).hasSize(2); + final Integer numDeleted = IndexingHeartbeat.clearIndexingHeartbeats(recordStore, index, TimeUnit.SECONDS.toMillis(10), 0).join(); + Assertions.assertThat(numDeleted).isEqualTo(1); + existingHeartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); + Assertions.assertThat(existingHeartbeats).hasSize(1); + Assertions.assertThat(existingHeartbeats.get(heartbeat.indexerId)).isNotNull(); + context.commit(); + } + } } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerIndexFromIndexTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerIndexFromIndexTest.java index f0663ce1f5..107b90a2db 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerIndexFromIndexTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerIndexFromIndexTest.java @@ -1187,21 +1187,7 @@ void testForbidConcurrentIndexFromIndexSessions() throws InterruptedException { .setIndexingPolicy(OnlineIndexer.IndexingPolicy.newBuilder() .setSourceIndex("src_index") .forbidRecordScan()) - .setConfigLoader(old -> { - if (passed.get()) { - try { - startBuildingSemaphore.release(); - pauseMutualBuildSemaphore.acquire(); // pause to try building indexes - } catch (InterruptedException e) { - throw new RuntimeException(e); - } finally { - pauseMutualBuildSemaphore.release(); - } - } else { - passed.set(true); - } - return old; - }) + .setConfigLoader(old -> pauseAfterOnePass(old, passed, startBuildingSemaphore, pauseMutualBuildSemaphore)) .build()) { indexBuilder.buildIndex(); } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerMultiTargetTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerMultiTargetTest.java index 23c22aba80..14d20f7109 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerMultiTargetTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerMultiTargetTest.java @@ -313,7 +313,6 @@ void testMultiTargetPartlyBuildFailure(int reverseSeed) { buildIndexAndCrashHalfway(chunkSize, 2, timer, newIndexerBuilder(indexAhead) .setIndexingPolicy(OnlineIndexer.IndexingPolicy.newBuilder() .setReverseScanOrder(reverse2) - .checkIndexingStampFrequencyMilliseconds(0) .allowTakeoverContinue())); // 3. assert mismatch type stamp @@ -864,21 +863,7 @@ void testForbidConversionOfActiveMultiTarget() throws InterruptedException { try (OnlineIndexer indexBuilder = newIndexerBuilder(indexes) .setLeaseLengthMillis(TimeUnit.SECONDS.toMillis(20)) .setLimit(4) - .setConfigLoader(old -> { - if (passed.get()) { - try { - startBuildingSemaphore.release(); - pauseMutualBuildSemaphore.acquire(); // pause to try building indexes - } catch (InterruptedException e) { - throw new RuntimeException(e); - } finally { - pauseMutualBuildSemaphore.release(); - } - } else { - passed.set(true); - } - return old; - }) + .setConfigLoader(old -> pauseAfterOnePass(old, passed, startBuildingSemaphore, pauseMutualBuildSemaphore)) .build()) { indexBuilder.buildIndex(); } @@ -930,21 +915,7 @@ void testForbidConversionOfActiveMultiTargetToMutual() throws InterruptedExcepti try (OnlineIndexer indexBuilder = newIndexerBuilder(indexes) .setLeaseLengthMillis(TimeUnit.SECONDS.toMillis(20)) .setLimit(4) - .setConfigLoader(old -> { - if (passed.get()) { - try { - startBuildingSemaphore.release(); - pauseMutualBuildSemaphore.acquire(); // pause to try building indexes - } catch (InterruptedException e) { - throw new RuntimeException(e); - } finally { - pauseMutualBuildSemaphore.release(); - } - } else { - passed.set(true); - } - return old; - }) + .setConfigLoader(old -> pauseAfterOnePass(old, passed, startBuildingSemaphore, pauseMutualBuildSemaphore)) .build()) { indexBuilder.buildIndex(); } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerMutualTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerMutualTest.java index 1e96cf657a..4d05a1c7c3 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerMutualTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerMutualTest.java @@ -663,6 +663,7 @@ void testForbidConversionToMultiTarget() throws InterruptedException { Semaphore startBuildingSemaphore = new Semaphore(1); pauseMutualBuildSemaphore.acquire(); startBuildingSemaphore.acquire(); + AtomicBoolean passed = new AtomicBoolean(false); final FDBStoreTimer timer = new FDBStoreTimer(); Thread t1 = new Thread(() -> { // build indexes mutually and pause halfway, allowing an active session test @@ -672,16 +673,7 @@ void testForbidConversionToMultiTarget() throws InterruptedException { .setMutualIndexing() .build()) .setLimit(14) - .setConfigLoader(old -> { - try { - startBuildingSemaphore.release(); - pauseMutualBuildSemaphore.acquire(); // pause to try building indexes - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - pauseMutualBuildSemaphore.release(); - return old; - }) + .setConfigLoader(old -> pauseAfterOnePass(old, passed, startBuildingSemaphore, pauseMutualBuildSemaphore)) .build()) { indexBuilder.buildIndex(); } @@ -734,20 +726,7 @@ void testActiveMultiTargetConversionToMutual() throws InterruptedException { try (OnlineIndexer indexBuilder = newIndexerBuilder(indexes, timer) .setLeaseLengthMillis(TimeUnit.SECONDS.toMillis(20)) .setLimit(7) - .setConfigLoader(old -> { - if (passed.get()) { - try { - startBuildingSemaphore.release(); - pauseMutualBuildSemaphore.acquire(); // pause to try building indexes elsewhere - } catch (InterruptedException e) { - throw new RuntimeException(e); - } - pauseMutualBuildSemaphore.release(); - } else { - passed.set(true); - } - return old; - }) + .setConfigLoader(old -> pauseAfterOnePass(old, passed, startBuildingSemaphore, pauseMutualBuildSemaphore)) .build()) { indexBuilder.buildIndex(); } @@ -1699,14 +1678,9 @@ void testMutualIndexingBlockerWhileActivelyIndexing() { .setLimit(2) .setIndexingPolicy(OnlineIndexer.IndexingPolicy.newBuilder() .setMutualIndexingBoundaries(boundariesList) - .checkIndexingStampFrequencyMilliseconds(0) .build()) .setConfigLoader(old -> { - try { - Thread.sleep(10); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } + snooze(10); return old; }) .build()) { diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerSimpleTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerSimpleTest.java index 773e24a5b4..16bc65181a 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerSimpleTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerSimpleTest.java @@ -899,23 +899,16 @@ public void runWithWeakReadSemantics() { @SuppressWarnings("removal") void testDeprecatedSetUseSynchronizedSession(boolean useSynchronizedSession) throws InterruptedException { // regardless of useSynchronizedSession's value, the build should be exclusive - List records = LongStream.range(0, 20).mapToObj(val -> - TestRecords1Proto.MySimpleRecord.newBuilder().setRecNo(val).setNumValue2((int)val + 1).build() - ).collect(Collectors.toList()); Index index = new Index("simple$value_2", field("num_value_2").ungrouped(), IndexTypes.SUM); FDBRecordStoreTestBase.RecordMetaDataHook hook = metaDataBuilder -> metaDataBuilder.addIndex("MySimpleRecord", index); - - openSimpleMetaData(); - try (FDBRecordContext context = openContext()) { - records.forEach(recordStore::saveRecord); - context.commit(); - } + populateData(20); // phase 1: successfully build openSimpleMetaData(hook); disableAll(List.of(index)); try (OnlineIndexer indexBuilder = newIndexerBuilder(index) .setUseSynchronizedSession(useSynchronizedSession) + .setUseSynchronizedSession(false) .build()) { indexBuilder.buildIndex(); } @@ -933,21 +926,7 @@ void testDeprecatedSetUseSynchronizedSession(boolean useSynchronizedSession) thr try (OnlineIndexer indexBuilder = newIndexerBuilder(index) .setLeaseLengthMillis(TimeUnit.SECONDS.toMillis(20)) .setLimit(4) - .setConfigLoader(old -> { - if (passed.get()) { - try { - startBuildingSemaphore.release(); - pauseMutualBuildSemaphore.acquire(); // pause to try building indexes - } catch (InterruptedException e) { - throw new RuntimeException(e); - } finally { - pauseMutualBuildSemaphore.release(); - } - } else { - passed.set(true); - } - return old; - }) + .setConfigLoader(old -> pauseAfterOnePass(old, passed, startBuildingSemaphore, pauseMutualBuildSemaphore)) .build()) { indexBuilder.buildIndex(); } @@ -957,10 +936,13 @@ void testDeprecatedSetUseSynchronizedSession(boolean useSynchronizedSession) thr startBuildingSemaphore.release(); // Fail to start another indexer - try (OnlineIndexer indexBuilder = newIndexerBuilder(index) - .build()) { - assertTrue(indexBuilder.checkAnyOngoingOnlineIndexBuildsAsync().join()); - assertThrows(SynchronizedSessionLockedException.class, indexBuilder::buildIndex); + try (FDBRecordContext context = openContext()) { + try (OnlineIndexer indexBuilder = newIndexerBuilder(index) + .build()) { + assertTrue(OnlineIndexer.checkAnyOngoingOnlineIndexBuildsAsync(recordStore, index).join()); + assertThrows(SynchronizedSessionLockedException.class, indexBuilder::buildIndex); + } + context.commit(); } // Successfully convert to a mutual indexer diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerTest.java index 425e3a1495..8b14eb2fd1 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerTest.java @@ -40,6 +40,7 @@ import com.apple.test.Tags; import com.google.protobuf.Descriptors; import com.google.protobuf.Message; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeEach; import org.junit.jupiter.api.Tag; import org.junit.jupiter.api.extension.RegisterExtension; @@ -48,7 +49,9 @@ import javax.annotation.Nullable; import java.util.ArrayList; import java.util.List; +import java.util.concurrent.Semaphore; import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; import java.util.stream.Collectors; import java.util.stream.LongStream; @@ -369,7 +372,7 @@ protected List getBoundariesList(List records, fin return boundaries; } - protected void snooze(int millis) { + protected static void snooze(int millis) { try { Thread.sleep(millis); } catch (InterruptedException e) { @@ -377,4 +380,15 @@ protected void snooze(int millis) { throw new RuntimeException(e); } } + + protected static OnlineIndexOperationConfig pauseAfterOnePass(final OnlineIndexOperationConfig oldConfig, final AtomicBoolean passed, final Semaphore inPauseSemaphore, final Semaphore pauseSemaphore) { + if (passed.get()) { + inPauseSemaphore.release(); + Assertions.assertDoesNotThrow(() -> pauseSemaphore.acquire()); + pauseSemaphore.release(); + } else { + passed.set(true); + } + return oldConfig; + } } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java index 30bf079d40..d452f5e0e0 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java @@ -28,7 +28,7 @@ import com.apple.foundationdb.record.metadata.expressions.EmptyKeyExpression; import com.apple.foundationdb.tuple.Tuple; import com.apple.test.BooleanSource; -import org.assertj.core.api.Assertions; +import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; @@ -46,6 +46,7 @@ import java.util.stream.IntStream; import static com.apple.foundationdb.record.metadata.Key.Expressions.field; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.Assertions.assertThrows; import static org.junit.jupiter.api.Assertions.assertTrue; @@ -80,35 +81,35 @@ void testHeartbeatLowLevel() { try (OnlineIndexer indexer = newIndexerBuilder(indexes.get(0)).build()) { // Query, unlimited Map queried = indexer.getIndexingHeartbeats(0); - Assertions.assertThat(queried).hasSize(count); - Assertions.assertThat(queried.keySet()) + assertThat(queried).hasSize(count); + assertThat(queried.keySet()) .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(heartbeat -> heartbeat.indexerId).collect(Collectors.toList())); // Query, partial queried = indexer.getIndexingHeartbeats(5); - Assertions.assertThat(queried).hasSize(5); + assertThat(queried).hasSize(5); // clear, partial int countDeleted = indexer.clearIndexingHeartbeats(0, 7); - Assertions.assertThat(countDeleted).isEqualTo(7); + assertThat(countDeleted).isEqualTo(7); queried = indexer.getIndexingHeartbeats(5); - Assertions.assertThat(queried).hasSize(3); + assertThat(queried).hasSize(3); } // Verify that the previous clear does not affect other index try (OnlineIndexer indexer = newIndexerBuilder(indexes.get(1)).build()) { Map queried = indexer.getIndexingHeartbeats(100); - Assertions.assertThat(queried).hasSize(count); - Assertions.assertThat(queried.keySet()) + assertThat(queried).hasSize(count); + assertThat(queried.keySet()) .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(ht -> ht.indexerId).collect(Collectors.toList())); // clear all int countDeleted = indexer.clearIndexingHeartbeats(0, 0); - Assertions.assertThat(countDeleted).isEqualTo(count); + assertThat(countDeleted).isEqualTo(count); // verify empty queried = indexer.getIndexingHeartbeats(0); - Assertions.assertThat(queried).isEmpty(); + assertThat(queried).isEmpty(); } } @@ -145,7 +146,7 @@ void testIndexersHeartbeatsClearAfterBuild(boolean mutualIndexing) { for (Index index : indexes) { try (OnlineIndexer indexer = newIndexerBuilder(index).build()) { - Assertions.assertThat(indexer.getIndexingHeartbeats(0)).isEmpty(); + assertThat(indexer.getIndexingHeartbeats(0)).isEmpty(); } } } @@ -207,7 +208,7 @@ void testIndexersHeartbeatsClearAfterCrash(boolean mutualIndexing) { for (Index index : indexes) { try (OnlineIndexer indexer = newIndexerBuilder(index).build()) { - Assertions.assertThat(indexer.getIndexingHeartbeats(0)).isEmpty(); + assertThat(indexer.getIndexingHeartbeats(0)).isEmpty(); } } } @@ -234,13 +235,10 @@ void testMutualIndexersHeartbeatsClearAfterBuild() throws InterruptedException { AtomicReference> heartbeats = new AtomicReference<>(); IntStream.rangeClosed(1, 4).parallel().forEach(i -> { if (i == 4) { - try { + Assertions.assertDoesNotThrow(() -> { startSemaphore.acquire(); Thread.sleep(100); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - throw new RuntimeException(e); - } + }); try (OnlineIndexer indexer = newIndexerBuilder(indexes).build()) { heartbeats.set(indexer.getIndexingHeartbeats(0)); } @@ -256,13 +254,8 @@ void testMutualIndexersHeartbeatsClearAfterBuild() throws InterruptedException { if (count.incrementAndGet() == 2) { startSemaphore.release(); } - try { - pauseSemaphore.acquire(); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } finally { - pauseSemaphore.release(); - } + Assertions.assertDoesNotThrow(() -> pauseSemaphore.acquire()); + pauseSemaphore.release(); } return old; }) @@ -272,7 +265,7 @@ void testMutualIndexersHeartbeatsClearAfterBuild() throws InterruptedException { } }); // While building, heartbeats count should have been 3 - Assertions.assertThat(heartbeats.get()).hasSize(3); + assertThat(heartbeats.get()).hasSize(3); // After building, heartbeats count should be 0 try (OnlineIndexer indexer = newIndexerBuilder(indexes).build()) { @@ -280,7 +273,6 @@ void testMutualIndexersHeartbeatsClearAfterBuild() throws InterruptedException { } } - @Test void testHeartbeatsRenewal() throws InterruptedException { // make sure that the heartbeats behave as expected during indexing: @@ -305,11 +297,7 @@ void testHeartbeatsRenewal() throws InterruptedException { .setLimit(10) .setConfigLoader(old -> { colectorGo.release(); - try { - indexerGo.acquire(); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } + Assertions.assertDoesNotThrow(() -> indexerGo.acquire()); return old; }) .build()) { @@ -321,11 +309,7 @@ void testHeartbeatsRenewal() throws InterruptedException { Thread collectorThread = new Thread(() -> { while (!indexerDone.get()) { - try { - colectorGo.acquire(); - } catch (InterruptedException e) { - throw new RuntimeException(e); - } + Assertions.assertDoesNotThrow(() -> colectorGo.acquire()); try (FDBRecordContext context = openContext()) { final Map heartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, indexes.get(0), 0).join(); heartbeatsQueries.add(heartbeats); @@ -339,17 +323,17 @@ void testHeartbeatsRenewal() throws InterruptedException { collectorThread.join(); indexerThread.join(); - Assertions.assertThat(heartbeatsQueries).hasSizeGreaterThan(5); - Assertions.assertThat(heartbeatsQueries.get(0)).hasSize(1); + assertThat(heartbeatsQueries).hasSizeGreaterThan(5); + assertThat(heartbeatsQueries.get(0)).hasSize(1); final Map.Entry first = heartbeatsQueries.get(0).entrySet().iterator().next(); Map.Entry previous = first; for (int i = 1; i < heartbeatsQueries.size() - 1; i++) { - Assertions.assertThat(heartbeatsQueries.get(i)).hasSize(1); + assertThat(heartbeatsQueries.get(i)).hasSize(1); final Map.Entry item = heartbeatsQueries.get(i).entrySet().iterator().next(); - Assertions.assertThat(item.getKey()).isEqualTo(first.getKey()); - Assertions.assertThat(item.getValue().getGenesisTimeMilliseconds()).isEqualTo(first.getValue().getGenesisTimeMilliseconds()); - Assertions.assertThat(item.getValue().getInfo()).isEqualTo(first.getValue().getInfo()); - Assertions.assertThat(item.getValue().getHeartbeatTimeMilliseconds()) + assertThat(item.getKey()).isEqualTo(first.getKey()); + assertThat(item.getValue().getGenesisTimeMilliseconds()).isEqualTo(first.getValue().getGenesisTimeMilliseconds()); + assertThat(item.getValue().getInfo()).isEqualTo(first.getValue().getInfo()); + assertThat(item.getValue().getHeartbeatTimeMilliseconds()) .isGreaterThan(previous.getValue().getHeartbeatTimeMilliseconds()); previous = item; } From 6a1414c00df652d1241e32a0c6b09eeb3b9358aa Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Thu, 4 Sep 2025 12:32:28 -0400 Subject: [PATCH 20/26] Minor cleanup to make IndexingBase a little shorter --- .../provider/foundationdb/IndexingBase.java | 61 ++++++------------- .../provider/foundationdb/IndexingMerger.java | 2 +- .../foundationdb/IndexingThrottle.java | 2 +- .../IndexingHeartbeatLowLevelTest.java | 11 +--- .../foundationdb/OnlineIndexerMutualTest.java | 4 +- 5 files changed, 24 insertions(+), 56 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java index dc2db639cf..5c47733556 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java @@ -40,7 +40,6 @@ 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.record.metadata.Key; import com.apple.foundationdb.record.metadata.MetaDataException; import com.apple.foundationdb.record.provider.common.StoreTimer; import com.apple.foundationdb.record.provider.common.StoreTimerSnapshot; @@ -136,12 +135,6 @@ protected static byte[] packOrNull(@Nullable Tuple tuple) { return (tuple == null) ? null : tuple.pack(); } - // Turn a (possibly null) key into its tuple representation. - @Nullable - protected static Tuple convertOrNull(@Nullable Key.Evaluated key) { - return (key == null) ? null : key.toTuple(); - } - @Nonnull protected CompletableFuture> recordIfInIndexedTypes(FDBStoredRecord rec) { return CompletableFuture.completedFuture( rec != null && common.getAllRecordTypes().contains(rec.getRecordType()) ? rec : null); @@ -360,7 +353,7 @@ private CompletableFuture setIndexingTypeOrThrow(FDBRecordStore store, boo heartbeat = new IndexingHeartbeat(common.getIndexerId(), indexingTypeStamp.getMethod().toString(), common.config.getLeaseLengthMillis(), allowMutual); return forEachTargetIndex(index -> setIndexingTypeOrThrow(store, continuedBuild, index, indexingTypeStamp) - .thenCompose(ignore -> updateHeartbeat(true, store, index))); + .thenCompose(ignore -> updateHeartbeat(store, index))); } @Nonnull @@ -390,7 +383,7 @@ private CompletableFuture setIndexingTypeOrThrow(FDBRecordStore store, boo } if (isTypeStampBlocked(savedStamp) && !policy.shouldAllowUnblock(savedStamp.getBlockID())) { // Indexing is blocked - throw newPartlyBuiltException(continuedBuild, savedStamp, newStamp, index); + throw newPartlyBuiltException(savedStamp, newStamp, index); } if (areSimilar(newStamp, savedStamp)) { // Similar stamps, replace it @@ -411,7 +404,7 @@ private CompletableFuture setIndexingTypeOrThrow(FDBRecordStore store, boo savedStamp, continuedBuild)); } // fall down to exception - throw newPartlyBuiltException(continuedBuild, savedStamp, newStamp, index); + throw newPartlyBuiltException(savedStamp, newStamp, index); }); } @@ -455,7 +448,7 @@ private CompletableFuture throwAsByRecordsUnlessNoRecordWasScanned(boolean .toString()); } final IndexBuildProto.IndexBuildIndexingStamp fakeSavedStamp = IndexingMultiTargetByRecords.compileSingleTargetLegacyIndexingTypeStamp(); - throw newPartlyBuiltException(true, fakeSavedStamp, indexingTypeStamp, index); + throw newPartlyBuiltException(fakeSavedStamp, indexingTypeStamp, index); } @Nonnull @@ -472,7 +465,7 @@ private CompletableFuture throwUnlessNoRecordWasScanned(boolean noRecordSc return AsyncUtil.DONE; } // A force overwrite cannot be allowed when partly built - throw newPartlyBuiltException(continuedBuild, savedStamp, indexingTypeStamp, index); + throw newPartlyBuiltException(savedStamp, indexingTypeStamp, index); } @Nonnull @@ -497,10 +490,9 @@ private CompletableFuture isWriteOnlyButNoRecordScanned(FDBRecordStore }); } - RecordCoreException newPartlyBuiltException(boolean continuedBuild, - IndexBuildProto.IndexBuildIndexingStamp savedStamp, - IndexBuildProto.IndexBuildIndexingStamp expectedStamp, - Index index) { + private RecordCoreException newPartlyBuiltException(IndexBuildProto.IndexBuildIndexingStamp savedStamp, + IndexBuildProto.IndexBuildIndexingStamp expectedStamp, + Index index) { return new PartlyBuiltException(savedStamp, expectedStamp, index, common.getIndexerId(), savedStamp.getBlock() ? "This index was partly built, and blocked" : @@ -582,10 +574,6 @@ public CompletableFuture buildCommitRetryAsync(@Nonnull BiFunction validateTypeStamp(@Nonnull FDBRecordStore store) } final IndexBuildProto.IndexBuildIndexingStamp expectedTypeStamp = getIndexingTypeStamp(store); return forEachTargetIndex(index -> CompletableFuture.allOf( - updateHeartbeat(true, store, index), + updateHeartbeat(store, index), store.loadIndexingTypeStampAsync(index) .thenAccept(typeStamp -> validateTypeStamp(typeStamp, expectedTypeStamp, index)) )); } - private CompletableFuture updateHeartbeat(boolean validate, FDBRecordStore store, Index index) { - if (heartbeat != null) { - if (validate) { - return heartbeat.checkAndUpdateHeartbeat(store, index); - } - heartbeat.updateHeartbeat(store, index); - } - return AsyncUtil.DONE; + private CompletableFuture updateHeartbeat(FDBRecordStore store, Index index) { + return heartbeat == null ? + AsyncUtil.DONE : + heartbeat.checkAndUpdateHeartbeat(store, index); } private CompletableFuture clearHeartbeats() { @@ -1122,17 +1106,6 @@ public static class TimeLimitException extends RecordCoreException { } } - public static boolean isTimeLimitException(@Nullable Throwable ex) { - for (Throwable current = ex; - current != null; - current = current.getCause()) { - if (current instanceof TimeLimitException) { - return true; - } - } - return false; - } - /** * thrown when partly built by another method. */ @@ -1189,7 +1162,7 @@ public static String stampToString(IndexBuildProto.IndexBuildIndexingStamp stamp if (stamp.getBlock()) { str.append(", blocked"); String id = stamp.getBlockID(); - if (id != null && !id.isEmpty()) { + if (!id.isEmpty()) { str.append(", blockId{").append(id).append("} "); } long expirationMillis = stamp.getBlockExpireEpochMilliSeconds(); @@ -1243,11 +1216,11 @@ protected static T findException(@Nullable Throwable ex, Class classT) { return null; } - protected static boolean shouldLessenWork(@Nullable FDBException ex) { + protected static boolean doNotLessenWork(@Nullable FDBException ex) { // These error codes represent a list of errors that can occur if there is too much work to be done // in a single transaction. if (ex == null) { - return false; + return true; } final Set lessenWorkCodes = new HashSet<>(Arrays.asList( FDBError.TIMED_OUT.code(), @@ -1256,7 +1229,7 @@ protected static boolean shouldLessenWork(@Nullable FDBException ex) { FDBError.TRANSACTION_TIMED_OUT.code(), FDBError.COMMIT_READ_INCOMPLETE.code(), FDBError.TRANSACTION_TOO_LARGE.code())); - return lessenWorkCodes.contains(ex.getCode()); + return !lessenWorkCodes.contains(ex.getCode()); } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingMerger.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingMerger.java index 043c70fe20..034b056d3e 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingMerger.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingMerger.java @@ -146,7 +146,7 @@ private CompletableFuture handleFailure(final IndexDeferredMaintenanceC mergeControl.mergeHadFailed(); // report to adjust stats final FDBException ex = IndexingBase.findException(e, FDBException.class); final IndexDeferredMaintenanceControl.LastStep lastStep = mergeControl.getLastStep(); - if (!IndexingBase.shouldLessenWork(ex)) { + if (IndexingBase.doNotLessenWork(ex)) { giveUpMerging(mergeControl, e); } switch (lastStep) { diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingThrottle.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingThrottle.java index c9e76ee1ef..e0cccfab25 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingThrottle.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingThrottle.java @@ -133,7 +133,7 @@ boolean mayRetryAfterHandlingException(@Nullable FDBException fdbException, @Nullable List additionalLogMessageKeyValues, int currTries, final boolean adjustLimits) { - if (currTries >= common.config.getMaxRetries() || !IndexingBase.shouldLessenWork(fdbException)) { + if (currTries >= common.config.getMaxRetries() || IndexingBase.doNotLessenWork(fdbException)) { // Here: should not retry or no more retries. There is no real need to handle limits. return false; } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeatLowLevelTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeatLowLevelTest.java index 0ac4611587..08d77cec51 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeatLowLevelTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeatLowLevelTest.java @@ -54,7 +54,7 @@ import static com.apple.foundationdb.record.metadata.Key.Expressions.concat; import static com.apple.foundationdb.record.metadata.Key.Expressions.field; -public class IndexingHeartbeatLowLevelTest { +class IndexingHeartbeatLowLevelTest { @RegisterExtension final FDBDatabaseExtension dbExtension = new FDBDatabaseExtension(); @RegisterExtension @@ -65,7 +65,7 @@ public class IndexingHeartbeatLowLevelTest { RecordMetaData metaData; @BeforeEach - public void setUp() { + void setUp() { final FDBDatabaseFactory factory = dbExtension.getDatabaseFactory(); factory.setInitialDelayMillis(2L); factory.setMaxDelayMillis(4L); @@ -98,11 +98,6 @@ void openMetaData(@Nonnull Descriptors.FileDescriptor descriptor, @Nonnull FDBRe metaData = metaDataBuilder.getRecordMetaData(); } - void openMetaData(@Nonnull Descriptors.FileDescriptor descriptor) { - openMetaData(descriptor, (metaDataBuilder) -> { - }); - } - void openSimpleMetaData(@Nonnull FDBRecordStoreTestBase.RecordMetaDataHook hook) { openMetaData(TestRecords1Proto.getDescriptor(), hook); } @@ -411,7 +406,7 @@ void testHeartbeatClearOldHeartbeats() throws InterruptedException { final Map existingHeartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, index, 0).join(); Assertions.assertThat(existingHeartbeats).hasSize(1); final Integer numDeleted = IndexingHeartbeat.clearIndexingHeartbeats(recordStore, index, TimeUnit.SECONDS.toMillis(10), 0).join(); - Assertions.assertThat(numDeleted).isEqualTo(0); + Assertions.assertThat(numDeleted).isZero(); context.commit(); } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerMutualTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerMutualTest.java index 4d05a1c7c3..a5f24b7c8e 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerMutualTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerMutualTest.java @@ -631,7 +631,7 @@ void testByRecordsContinueAsMutualNonIdenticalList() { assertTrue(indexedMutually > 0); if (index.equals(indexes.get(3))) { // Was not built by the multi target session - assertEquals(indexedMutually, numRecords); + assertEquals(numRecords, indexedMutually); } else { // Was partly built by the multi target session assertTrue(indexedMutually < numRecords); @@ -693,7 +693,7 @@ void testForbidConversionToMultiTarget() throws InterruptedException { // must index all the records int indexedMutually = timer.getCount(FDBStoreTimer.Counts.ONLINE_INDEX_BUILDER_RECORDS_INDEXED); - assertEquals(indexedMutually, numRecords); + assertEquals(numRecords, indexedMutually); // happy indexes assertion assertReadable(indexes); From b81beaba541e10613cf79089d13cf49bf834a457 Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Thu, 4 Sep 2025 13:14:57 -0400 Subject: [PATCH 21/26] pmdMain: unused param --- .../record/provider/foundationdb/IndexingBase.java | 6 ++---- 1 file changed, 2 insertions(+), 4 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java index 5c47733556..097d37ac40 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java @@ -400,8 +400,7 @@ private CompletableFuture setIndexingTypeOrThrow(FDBRecordStore store, boo // check if partly built return isWriteOnlyButNoRecordScanned(store, index) .thenCompose(noRecordScanned -> - throwUnlessNoRecordWasScanned(noRecordScanned, store, index, newStamp, - savedStamp, continuedBuild)); + throwUnlessNoRecordWasScanned(noRecordScanned, store, index, newStamp, savedStamp)); } // fall down to exception throw newPartlyBuiltException(savedStamp, newStamp, index); @@ -456,8 +455,7 @@ private CompletableFuture throwUnlessNoRecordWasScanned(boolean noRecordSc FDBRecordStore store, Index index, IndexBuildProto.IndexBuildIndexingStamp indexingTypeStamp, - IndexBuildProto.IndexBuildIndexingStamp savedStamp, - boolean continuedBuild) { + IndexBuildProto.IndexBuildIndexingStamp savedStamp) { // Ditto (a complicated way to reduce complexity) if (noRecordScanned) { // we can safely overwrite the previous type stamp From 65e259521dadcb1abd2fa9b27e695f44edd55522 Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Sat, 6 Sep 2025 08:23:23 -0400 Subject: [PATCH 22/26] Apply PR requested changes --- .../record/provider/foundationdb/FDBRecordStore.java | 1 + .../record/provider/foundationdb/IndexingBase.java | 7 ++++--- .../record/provider/foundationdb/IndexingMerger.java | 2 +- .../provider/foundationdb/IndexingThrottle.java | 2 +- .../record/provider/foundationdb/OnlineIndexer.java | 1 + .../{ => indexing}/IndexingHeartbeat.java | 8 +++++++- .../foundationdb/OnlineIndexerSimpleTest.java | 6 +++++- .../foundationdb/OnlineIndexingHeartbeatTest.java | 5 +++-- .../{ => indexing}/IndexingHeartbeatLowLevelTest.java | 11 +++++++++-- gradle.properties | 2 +- 10 files changed, 33 insertions(+), 12 deletions(-) rename fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/{ => indexing}/IndexingHeartbeat.java (97%) rename fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/{ => indexing}/IndexingHeartbeatLowLevelTest.java (97%) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java index d4b54d272f..9734e9bcb5 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/FDBRecordStore.java @@ -83,6 +83,7 @@ import com.apple.foundationdb.record.metadata.expressions.KeyExpression; import com.apple.foundationdb.record.provider.common.DynamicMessageRecordSerializer; import com.apple.foundationdb.record.provider.common.RecordSerializer; +import com.apple.foundationdb.record.provider.foundationdb.indexing.IndexingHeartbeat; import com.apple.foundationdb.record.provider.foundationdb.indexing.IndexingRangeSet; import com.apple.foundationdb.record.provider.foundationdb.keyspace.KeySpacePath; import com.apple.foundationdb.record.provider.foundationdb.storestate.FDBRecordStoreStateCache; diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java index 097d37ac40..089116e79d 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java @@ -43,6 +43,7 @@ import com.apple.foundationdb.record.metadata.MetaDataException; import com.apple.foundationdb.record.provider.common.StoreTimer; import com.apple.foundationdb.record.provider.common.StoreTimerSnapshot; +import com.apple.foundationdb.record.provider.foundationdb.indexing.IndexingHeartbeat; import com.apple.foundationdb.record.provider.foundationdb.indexing.IndexingRangeSet; import com.apple.foundationdb.record.query.plan.RecordQueryPlanner; import com.apple.foundationdb.record.query.plan.synthetic.SyntheticRecordFromStoredRecordPlan; @@ -1214,11 +1215,11 @@ protected static T findException(@Nullable Throwable ex, Class classT) { return null; } - protected static boolean doNotLessenWork(@Nullable FDBException ex) { + protected static boolean shouldLessenWork(@Nullable FDBException ex) { // These error codes represent a list of errors that can occur if there is too much work to be done // in a single transaction. if (ex == null) { - return true; + return false; } final Set lessenWorkCodes = new HashSet<>(Arrays.asList( FDBError.TIMED_OUT.code(), @@ -1227,7 +1228,7 @@ protected static boolean doNotLessenWork(@Nullable FDBException ex) { FDBError.TRANSACTION_TIMED_OUT.code(), FDBError.COMMIT_READ_INCOMPLETE.code(), FDBError.TRANSACTION_TOO_LARGE.code())); - return !lessenWorkCodes.contains(ex.getCode()); + return lessenWorkCodes.contains(ex.getCode()); } } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingMerger.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingMerger.java index 034b056d3e..043c70fe20 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingMerger.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingMerger.java @@ -146,7 +146,7 @@ private CompletableFuture handleFailure(final IndexDeferredMaintenanceC mergeControl.mergeHadFailed(); // report to adjust stats final FDBException ex = IndexingBase.findException(e, FDBException.class); final IndexDeferredMaintenanceControl.LastStep lastStep = mergeControl.getLastStep(); - if (IndexingBase.doNotLessenWork(ex)) { + if (!IndexingBase.shouldLessenWork(ex)) { giveUpMerging(mergeControl, e); } switch (lastStep) { diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingThrottle.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingThrottle.java index e0cccfab25..c9e76ee1ef 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingThrottle.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingThrottle.java @@ -133,7 +133,7 @@ boolean mayRetryAfterHandlingException(@Nullable FDBException fdbException, @Nullable List additionalLogMessageKeyValues, int currTries, final boolean adjustLimits) { - if (currTries >= common.config.getMaxRetries() || IndexingBase.doNotLessenWork(fdbException)) { + if (currTries >= common.config.getMaxRetries() || !IndexingBase.shouldLessenWork(fdbException)) { // Here: should not retry or no more retries. There is no real need to handle limits. return false; } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java index bb734acbb6..a7645627a6 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexer.java @@ -34,6 +34,7 @@ import com.apple.foundationdb.record.metadata.MetaDataException; import com.apple.foundationdb.record.metadata.RecordType; import com.apple.foundationdb.record.provider.common.StoreTimer; +import com.apple.foundationdb.record.provider.foundationdb.indexing.IndexingHeartbeat; import com.apple.foundationdb.synchronizedsession.SynchronizedSession; import com.apple.foundationdb.tuple.Tuple; import com.google.common.annotations.VisibleForTesting; diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeat.java similarity index 97% rename from fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java rename to fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeat.java index 9e7f9ad548..147c1d52bc 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeat.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeat.java @@ -18,7 +18,7 @@ * limitations under the License. */ -package com.apple.foundationdb.record.provider.foundationdb; +package com.apple.foundationdb.record.provider.foundationdb.indexing; import com.apple.foundationdb.KeyValue; import com.apple.foundationdb.annotation.API; @@ -28,6 +28,8 @@ 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.record.provider.foundationdb.FDBRecordStore; +import com.apple.foundationdb.record.provider.foundationdb.IndexingSubspaces; import com.apple.foundationdb.synchronizedsession.SynchronizedSessionLockedException; import com.google.protobuf.InvalidProtocolBufferException; import org.slf4j.Logger; @@ -68,6 +70,10 @@ public IndexingHeartbeat(final UUID indexerId, String info, long leaseLength, bo this.genesisTimeMilliseconds = nowMilliseconds(); } + public UUID getIndexerId() { + return indexerId; + } + public void updateHeartbeat(@Nonnull FDBRecordStore store, @Nonnull Index index) { byte[] key = IndexingSubspaces.indexHeartbeatSubspace(store, index, indexerId).pack(); byte[] value = IndexBuildProto.IndexBuildHeartbeat.newBuilder() diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerSimpleTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerSimpleTest.java index 16bc65181a..d09b8a9653 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerSimpleTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerSimpleTest.java @@ -908,7 +908,10 @@ void testDeprecatedSetUseSynchronizedSession(boolean useSynchronizedSession) thr disableAll(List.of(index)); try (OnlineIndexer indexBuilder = newIndexerBuilder(index) .setUseSynchronizedSession(useSynchronizedSession) - .setUseSynchronizedSession(false) + .setConfigLoader(old -> { + assertTrue(old.shouldUseSynchronizedSession()); + return old; + }) .build()) { indexBuilder.buildIndex(); } @@ -924,6 +927,7 @@ void testDeprecatedSetUseSynchronizedSession(boolean useSynchronizedSession) thr Thread t1 = new Thread(() -> { // build index and pause halfway, allowing an active session test try (OnlineIndexer indexBuilder = newIndexerBuilder(index) + .setUseSynchronizedSession(useSynchronizedSession) .setLeaseLengthMillis(TimeUnit.SECONDS.toMillis(20)) .setLimit(4) .setConfigLoader(old -> pauseAfterOnePass(old, passed, startBuildingSemaphore, pauseMutualBuildSemaphore)) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java index d452f5e0e0..77eea25124 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java @@ -26,6 +26,7 @@ import com.apple.foundationdb.record.metadata.IndexOptions; import com.apple.foundationdb.record.metadata.IndexTypes; import com.apple.foundationdb.record.metadata.expressions.EmptyKeyExpression; +import com.apple.foundationdb.record.provider.foundationdb.indexing.IndexingHeartbeat; import com.apple.foundationdb.tuple.Tuple; import com.apple.test.BooleanSource; import org.junit.jupiter.api.Assertions; @@ -83,7 +84,7 @@ void testHeartbeatLowLevel() { Map queried = indexer.getIndexingHeartbeats(0); assertThat(queried).hasSize(count); assertThat(queried.keySet()) - .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(heartbeat -> heartbeat.indexerId).collect(Collectors.toList())); + .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(heartbeat -> heartbeat.getIndexerId()).collect(Collectors.toList())); // Query, partial queried = indexer.getIndexingHeartbeats(5); @@ -101,7 +102,7 @@ void testHeartbeatLowLevel() { Map queried = indexer.getIndexingHeartbeats(100); assertThat(queried).hasSize(count); assertThat(queried.keySet()) - .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(ht -> ht.indexerId).collect(Collectors.toList())); + .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(IndexingHeartbeat::getIndexerId).collect(Collectors.toList())); // clear all int countDeleted = indexer.clearIndexingHeartbeats(0, 0); diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeatLowLevelTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeatLowLevelTest.java similarity index 97% rename from fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeatLowLevelTest.java rename to fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeatLowLevelTest.java index 08d77cec51..c5d093d7e1 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/IndexingHeartbeatLowLevelTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeatLowLevelTest.java @@ -1,5 +1,5 @@ /* - * testIndexingHeartbeaLowLevel.java + * IndexingHeartbeatLowLevelTest.java * * This source file is part of the FoundationDB open source project * @@ -18,7 +18,7 @@ * limitations under the License. */ -package com.apple.foundationdb.record.provider.foundationdb; +package com.apple.foundationdb.record.provider.foundationdb.indexing; import com.apple.foundationdb.record.IndexBuildProto; import com.apple.foundationdb.record.RecordMetaData; @@ -30,6 +30,13 @@ import com.apple.foundationdb.record.metadata.expressions.EmptyKeyExpression; import com.apple.foundationdb.record.metadata.expressions.GroupingKeyExpression; import com.apple.foundationdb.record.metadata.expressions.VersionKeyExpression; +import com.apple.foundationdb.record.provider.foundationdb.FDBDatabase; +import com.apple.foundationdb.record.provider.foundationdb.FDBDatabaseFactory; +import com.apple.foundationdb.record.provider.foundationdb.FDBRecordContext; +import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStore; +import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStoreBase; +import com.apple.foundationdb.record.provider.foundationdb.FDBRecordStoreTestBase; +import com.apple.foundationdb.record.provider.foundationdb.IndexingSubspaces; import com.apple.foundationdb.record.provider.foundationdb.keyspace.KeySpacePath; import com.apple.foundationdb.record.test.FDBDatabaseExtension; import com.apple.foundationdb.record.test.TestKeySpace; diff --git a/gradle.properties b/gradle.properties index 7fc6e8c302..e61b228de7 100644 --- a/gradle.properties +++ b/gradle.properties @@ -19,7 +19,7 @@ # rootProject.name=fdb-record-layer -version=4.5.11.0 +version=4.6.0.0 releaseBuild=false # this should be false for release branches (i.e. if there is no -SNAPSHOT on the above version) From 6a239904bdeb4ce5950c8b9d9c28cc47737c0a98 Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Tue, 9 Sep 2025 17:34:27 -0400 Subject: [PATCH 23/26] Attempt to please some Teamscale croaks - untested (deprecated) functions and tests length --- .../foundationdb/OnlineIndexerSimpleTest.java | 44 +++++--- .../OnlineIndexingHeartbeatTest.java | 104 ++++++++++-------- 2 files changed, 87 insertions(+), 61 deletions(-) diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerSimpleTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerSimpleTest.java index d09b8a9653..b19717e6a5 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerSimpleTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexerSimpleTest.java @@ -897,28 +897,15 @@ public void runWithWeakReadSemantics() { @ParameterizedTest @BooleanSource @SuppressWarnings("removal") - void testDeprecatedSetUseSynchronizedSession(boolean useSynchronizedSession) throws InterruptedException { + void testExclusiveBuildRegardlessOfSettingDeprecatedSynchronizedSession(boolean useSynchronizedSession) throws InterruptedException { // regardless of useSynchronizedSession's value, the build should be exclusive Index index = new Index("simple$value_2", field("num_value_2").ungrouped(), IndexTypes.SUM); FDBRecordStoreTestBase.RecordMetaDataHook hook = metaDataBuilder -> metaDataBuilder.addIndex("MySimpleRecord", index); - populateData(20); - - // phase 1: successfully build + populateData(21); openSimpleMetaData(hook); disableAll(List.of(index)); - try (OnlineIndexer indexBuilder = newIndexerBuilder(index) - .setUseSynchronizedSession(useSynchronizedSession) - .setConfigLoader(old -> { - assertTrue(old.shouldUseSynchronizedSession()); - return old; - }) - .build()) { - indexBuilder.buildIndex(); - } - assertReadable(index); - // Now disable and ensure exclusive build - disableAll(List.of(index)); + // ensure exclusive build Semaphore pauseMutualBuildSemaphore = new Semaphore(1); Semaphore startBuildingSemaphore = new Semaphore(1); pauseMutualBuildSemaphore.acquire(); @@ -965,4 +952,29 @@ void testDeprecatedSetUseSynchronizedSession(boolean useSynchronizedSession) thr // happy indexes assertion assertReadable(index); } + + @ParameterizedTest + @BooleanSource + @SuppressWarnings("removal") + void testSuccessfullyBuildWithDeprecatedApiFunctions(final boolean useSynchronizedSession) { + Index index = new Index("simple$value_2", field("num_value_2").ungrouped(), IndexTypes.SUM); + FDBRecordStoreTestBase.RecordMetaDataHook hook = metaDataBuilder -> metaDataBuilder.addIndex("MySimpleRecord", index); + populateData(20); + + openSimpleMetaData(hook); + disableAll(List.of(index)); + try (OnlineIndexer indexBuilder = newIndexerBuilder(index) + .setLimit(3) + .setUseSynchronizedSession(useSynchronizedSession) // ignored value + .setConfigLoader(old -> { + assertTrue(old.shouldUseSynchronizedSession()); + return old.toBuilder().setUseSynchronizedSession(useSynchronizedSession).build(); // ignored value + }) + .setIndexingPolicy(OnlineIndexer.IndexingPolicy.newBuilder() + .checkIndexingStampFrequencyMilliseconds(useSynchronizedSession ? 5_000 : 0)) // ignored value + .build()) { + indexBuilder.buildIndex(); + } + assertReadable(index); + } } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java index 77eea25124..f987baae65 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java @@ -33,6 +33,7 @@ import org.junit.jupiter.api.Test; import org.junit.jupiter.params.ParameterizedTest; +import javax.annotation.Nonnull; import java.util.ArrayList; import java.util.Arrays; import java.util.List; @@ -246,23 +247,7 @@ void testMutualIndexersHeartbeatsClearAfterBuild() throws InterruptedException { startSemaphore.release(); pauseSemaphore.release(); } else { - AtomicInteger counter = new AtomicInteger(0); - try (OnlineIndexer indexer = newIndexerBuilder(indexes) - .setIndexingPolicy(OnlineIndexer.IndexingPolicy.newBuilder() - .setMutualIndexingBoundaries(boundariesList)) - .setConfigLoader(old -> { - if (counter.incrementAndGet() > 0) { - if (count.incrementAndGet() == 2) { - startSemaphore.release(); - } - Assertions.assertDoesNotThrow(() -> pauseSemaphore.acquire()); - pauseSemaphore.release(); - } - return old; - }) - .build()) { - indexer.buildIndex(); - } + buildIndexWithPause(indexes, boundariesList, count, startSemaphore, pauseSemaphore); } }); // While building, heartbeats count should have been 3 @@ -274,6 +259,26 @@ void testMutualIndexersHeartbeatsClearAfterBuild() throws InterruptedException { } } + private void buildIndexWithPause(final List indexes, final List boundariesList, final AtomicInteger count, final Semaphore startSemaphore, final Semaphore pauseSemaphore) { + AtomicInteger counter = new AtomicInteger(0); + try (OnlineIndexer indexer = newIndexerBuilder(indexes) + .setIndexingPolicy(OnlineIndexer.IndexingPolicy.newBuilder() + .setMutualIndexingBoundaries(boundariesList)) + .setConfigLoader(old -> { + if (counter.incrementAndGet() > 0) { + if (count.incrementAndGet() == 2) { + startSemaphore.release(); + } + Assertions.assertDoesNotThrow(() -> pauseSemaphore.acquire()); + pauseSemaphore.release(); + } + return old; + }) + .build()) { + indexer.buildIndex(); + } + } + @Test void testHeartbeatsRenewal() throws InterruptedException { // make sure that the heartbeats behave as expected during indexing: @@ -290,35 +295,11 @@ void testHeartbeatsRenewal() throws InterruptedException { final List> heartbeatsQueries = new ArrayList<>(); Semaphore indexerGo = new Semaphore(1); - Semaphore colectorGo = new Semaphore(1); + Semaphore collectorGo = new Semaphore(1); AtomicBoolean indexerDone = new AtomicBoolean(false); - colectorGo.acquire(); - Thread indexerThread = new Thread( () -> { - try (OnlineIndexer indexer = newIndexerBuilder(indexes) - .setLimit(10) - .setConfigLoader(old -> { - colectorGo.release(); - Assertions.assertDoesNotThrow(() -> indexerGo.acquire()); - return old; - }) - .build()) { - indexer.buildIndex(); - } - colectorGo.release(); - indexerDone.set(true); - }); - - Thread collectorThread = new Thread(() -> { - while (!indexerDone.get()) { - Assertions.assertDoesNotThrow(() -> colectorGo.acquire()); - try (FDBRecordContext context = openContext()) { - final Map heartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, indexes.get(0), 0).join(); - heartbeatsQueries.add(heartbeats); - context.commit(); - } - indexerGo.release(); - } - }); + collectorGo.acquire(); + Thread indexerThread = buildIndexesThread(indexes, collectorGo, indexerGo, indexerDone); + Thread collectorThread = collectHeartbeatsThread(indexerDone, collectorGo, indexes, heartbeatsQueries, indexerGo); indexerThread.start(); collectorThread.start(); collectorThread.join(); @@ -339,4 +320,37 @@ void testHeartbeatsRenewal() throws InterruptedException { previous = item; } } + + @Nonnull + private Thread collectHeartbeatsThread(final AtomicBoolean indexerDone, final Semaphore colectorGo, final List indexes, final List> heartbeatsQueries, final Semaphore indexerGo) { + return new Thread(() -> { + while (!indexerDone.get()) { + Assertions.assertDoesNotThrow(() -> colectorGo.acquire()); + try (FDBRecordContext context = openContext()) { + final Map heartbeats = IndexingHeartbeat.getIndexingHeartbeats(recordStore, indexes.get(0), 0).join(); + heartbeatsQueries.add(heartbeats); + context.commit(); + } + indexerGo.release(); + } + }); + } + + @Nonnull + private Thread buildIndexesThread(final List indexes, final Semaphore colectorGo, final Semaphore indexerGo, final AtomicBoolean indexerDone) { + return new Thread(() -> { + try (OnlineIndexer indexer = newIndexerBuilder(indexes) + .setLimit(10) + .setConfigLoader(old -> { + colectorGo.release(); + Assertions.assertDoesNotThrow(() -> indexerGo.acquire()); + return old; + }) + .build()) { + indexer.buildIndex(); + } + colectorGo.release(); + indexerDone.set(true); + }); + } } From b4acdf11cf4104af4f46034fb683093b5111f2d9 Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Thu, 11 Sep 2025 15:34:34 -0400 Subject: [PATCH 24/26] Apply Alec's requested changes --- .../provider/foundationdb/IndexingBase.java | 1 + .../indexing/IndexingHeartbeat.java | 29 +++++++------------ .../src/main/proto/index_build.proto | 2 +- .../OnlineIndexingHeartbeatTest.java | 2 +- .../IndexingHeartbeatLowLevelTest.java | 4 +-- 5 files changed, 16 insertions(+), 22 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java index 089116e79d..830edf0135 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingBase.java @@ -393,6 +393,7 @@ private CompletableFuture setIndexingTypeOrThrow(FDBRecordStore store, boo } // Here: check if type conversion is allowed if (continuedBuild && shouldAllowTypeConversionContinue(newStamp, savedStamp)) { + // Special case: partly built by another indexing method, but may be continued with the current one store.saveIndexingTypeStamp(index, newStamp); return AsyncUtil.DONE; } diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeat.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeat.java index 147c1d52bc..8095b7d3cf 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeat.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeat.java @@ -40,6 +40,7 @@ import java.util.Map; import java.util.UUID; import java.util.concurrent.CompletableFuture; +import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicInteger; /** @@ -51,14 +52,14 @@ */ @API(API.Status.INTERNAL) public class IndexingHeartbeat { - // [prefix, indexerId] -> [indexing-type, genesis time, heartbeat time] + // [prefix, indexerId] -> [creator info, create 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 createTimeMilliseconds; final long leaseLength; final boolean allowMutual; @@ -67,7 +68,7 @@ public IndexingHeartbeat(final UUID indexerId, String info, long leaseLength, bo this.info = info; this.leaseLength = leaseLength; this.allowMutual = allowMutual; - this.genesisTimeMilliseconds = nowMilliseconds(); + this.createTimeMilliseconds = nowMilliseconds(); } public UUID getIndexerId() { @@ -78,7 +79,7 @@ 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) + .setCreateTimeMilliseconds(createTimeMilliseconds) .setHeartbeatTimeMilliseconds(nowMilliseconds()) .build().toByteArray(); store.ensureContextActive().set(key, value); @@ -93,16 +94,8 @@ public CompletableFuture checkAndUpdateHeartbeat(@Nonnull FDBRecordStore s final AsyncIterator iterator = heartbeatsIterator(store, index); final long now = nowMilliseconds(); - return AsyncUtil.whileTrue(() -> iterator.onHasNext() - .thenApply(hasNext -> { - if (!hasNext) { - return false; - } - final KeyValue kv = iterator.next(); - checkSingleHeartbeat(store, index, kv, now); - return true; - })) - .thenAccept(ignore -> updateHeartbeat(store, index)); + return AsyncUtil.forEachRemaining(iterator, kv -> checkSingleHeartbeat(store, index, kv, now), store.getExecutor()) + .thenRun(() -> updateHeartbeat(store, index)); } private void checkSingleHeartbeat(final @Nonnull FDBRecordStore store, final @Nonnull Index index, final KeyValue kv, final long now) { @@ -111,7 +104,7 @@ private void checkSingleHeartbeat(final @Nonnull FDBRecordStore store, final @No 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) { + if (age > TimeUnit.DAYS.toMillis(-1) && 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) @@ -158,12 +151,12 @@ public static CompletableFuture> // Let the caller know about this invalid heartbeat. ret.put(otherIndexerId, IndexBuildProto.IndexBuildHeartbeat.newBuilder() .setInfo(INVALID_HEARTBEAT_INFO) - .setGenesisTimeMilliseconds(0) + .setCreateTimeMilliseconds(0) .setHeartbeatTimeMilliseconds(0) .build()); } return true; - })) + }), store.getExecutor()) .thenApply(ignore -> ret); } @@ -195,7 +188,7 @@ public static CompletableFuture clearIndexingHeartbeats(@Nonnull FDBRec deleteCount.incrementAndGet(); } return true; - })) + }), store.getExecutor()) .thenApply(ignore -> deleteCount.get()); } diff --git a/fdb-record-layer-core/src/main/proto/index_build.proto b/fdb-record-layer-core/src/main/proto/index_build.proto index 469917fce9..541aaf6083 100644 --- a/fdb-record-layer-core/src/main/proto/index_build.proto +++ b/fdb-record-layer-core/src/main/proto/index_build.proto @@ -50,7 +50,7 @@ message IndexBuildIndexingStamp { // aren't too many concurrent processes trying to build an index message IndexBuildHeartbeat { optional string info = 1; // general information about the indexing session. This can be useful when querying the heartbeats. - optional int64 genesisTimeMilliseconds = 2; // indexer's creation time (since epoch) + optional int64 createTimeMilliseconds = 2; // indexer's creation time (since epoch) optional int64 heartbeatTimeMilliseconds = 3; // last heartbeat's time (since epoch). This is the only heartbeat value that changes during online indexing. } diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java index f987baae65..8a3bcd055a 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/OnlineIndexingHeartbeatTest.java @@ -313,7 +313,7 @@ void testHeartbeatsRenewal() throws InterruptedException { assertThat(heartbeatsQueries.get(i)).hasSize(1); final Map.Entry item = heartbeatsQueries.get(i).entrySet().iterator().next(); assertThat(item.getKey()).isEqualTo(first.getKey()); - assertThat(item.getValue().getGenesisTimeMilliseconds()).isEqualTo(first.getValue().getGenesisTimeMilliseconds()); + assertThat(item.getValue().getCreateTimeMilliseconds()).isEqualTo(first.getValue().getCreateTimeMilliseconds()); assertThat(item.getValue().getInfo()).isEqualTo(first.getValue().getInfo()); assertThat(item.getValue().getHeartbeatTimeMilliseconds()) .isGreaterThan(previous.getValue().getHeartbeatTimeMilliseconds()); diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeatLowLevelTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeatLowLevelTest.java index c5d093d7e1..b32b37a423 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeatLowLevelTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeatLowLevelTest.java @@ -151,8 +151,8 @@ void testHeartbeatQuery() { .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(heartbeat -> heartbeat.indexerId).collect(Collectors.toList())); Assertions.assertThat(queried.values().stream().map(IndexBuildProto.IndexBuildHeartbeat::getInfo)) .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(heartbeat -> heartbeat.info).collect(Collectors.toList())); - Assertions.assertThat(queried.values().stream().map(IndexBuildProto.IndexBuildHeartbeat::getGenesisTimeMilliseconds)) - .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(heartbeat -> heartbeat.genesisTimeMilliseconds).collect(Collectors.toList())); + Assertions.assertThat(queried.values().stream().map(IndexBuildProto.IndexBuildHeartbeat::getCreateTimeMilliseconds)) + .containsExactlyInAnyOrderElementsOf(Arrays.stream(heartbeats).map(heartbeat -> heartbeat.createTimeMilliseconds).collect(Collectors.toList())); // Query, partial queried = From bbf393b03e458041e181918db11b8f78d17a479f Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Fri, 12 Sep 2025 11:55:11 -0400 Subject: [PATCH 25/26] Apply Alec's requested change --- .../record/provider/foundationdb/IndexingSubspaces.java | 4 ++-- .../provider/foundationdb/indexing/IndexingHeartbeat.java | 4 ++-- .../foundationdb/indexing/IndexingHeartbeatLowLevelTest.java | 2 +- 3 files changed, 5 insertions(+), 5 deletions(-) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingSubspaces.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingSubspaces.java index 7cc0ea58db..8c8bed3d32 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingSubspaces.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/IndexingSubspaces.java @@ -104,8 +104,8 @@ public static Subspace indexHeartbeatSubspace(@Nonnull FDBRecordStoreBase sto * @return subspace */ @Nonnull - public static Subspace indexHeartbeatSubspace(@Nonnull FDBRecordStoreBase store, @Nonnull Index index, @Nonnull UUID indexerId) { - return indexHeartbeatSubspace(store, index).subspace(Tuple.from(indexerId)); + public static byte[] indexHeartbeatSubspaceBytes(@Nonnull FDBRecordStoreBase store, @Nonnull Index index, @Nonnull UUID indexerId) { + return indexHeartbeatSubspace(store, index).subspace(Tuple.from(indexerId)).pack(); } /** diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeat.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeat.java index 8095b7d3cf..b6a7e177e7 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeat.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeat.java @@ -76,7 +76,7 @@ public UUID getIndexerId() { } public void updateHeartbeat(@Nonnull FDBRecordStore store, @Nonnull Index index) { - byte[] key = IndexingSubspaces.indexHeartbeatSubspace(store, index, indexerId).pack(); + byte[] key = IndexingSubspaces.indexHeartbeatSubspaceBytes(store, index, indexerId); byte[] value = IndexBuildProto.IndexBuildHeartbeat.newBuilder() .setInfo(info) .setCreateTimeMilliseconds(createTimeMilliseconds) @@ -123,7 +123,7 @@ private void checkSingleHeartbeat(final @Nonnull FDBRecordStore store, final @No } public void clearHeartbeat(@Nonnull FDBRecordStore store, @Nonnull Index index) { - store.ensureContextActive().clear(IndexingSubspaces.indexHeartbeatSubspace(store, index, indexerId).pack()); + store.ensureContextActive().clear(IndexingSubspaces.indexHeartbeatSubspaceBytes(store, index, indexerId)); } public static void clearAllHeartbeats(@Nonnull FDBRecordStore store, @Nonnull Index index) { diff --git a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeatLowLevelTest.java b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeatLowLevelTest.java index b32b37a423..6b517625ba 100644 --- a/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeatLowLevelTest.java +++ b/fdb-record-layer-core/src/test/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeatLowLevelTest.java @@ -476,7 +476,7 @@ void testUnparseableHeartbeat() { // write unparseable data where a heartbeat should exist openSimpleMetaData(hook); try (FDBRecordContext context = openContext()) { - byte[] key = IndexingSubspaces.indexHeartbeatSubspace(recordStore, index, UUID.randomUUID()).pack(); + byte[] key = IndexingSubspaces.indexHeartbeatSubspaceBytes(recordStore, index, UUID.randomUUID()); byte[] value = "meaningless byte value".getBytes(); recordStore.ensureContextActive().set(key, value); context.commit(); From 1d6f001f2f2fa87cf0755cfc9e5e881fcebff39b Mon Sep 17 00:00:00 2001 From: Josef Ezra Date: Fri, 12 Sep 2025 16:59:54 -0400 Subject: [PATCH 26/26] Add comment --- .../provider/foundationdb/indexing/IndexingHeartbeat.java | 2 ++ 1 file changed, 2 insertions(+) diff --git a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeat.java b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeat.java index b6a7e177e7..3b4fb21b9d 100644 --- a/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeat.java +++ b/fdb-record-layer-core/src/main/java/com/apple/foundationdb/record/provider/foundationdb/indexing/IndexingHeartbeat.java @@ -105,6 +105,8 @@ private void checkSingleHeartbeat(final @Nonnull FDBRecordStore store, final @No final IndexBuildProto.IndexBuildHeartbeat otherHeartbeat = IndexBuildProto.IndexBuildHeartbeat.parseFrom(kv.getValue()); final long age = now - otherHeartbeat.getHeartbeatTimeMilliseconds(); if (age > TimeUnit.DAYS.toMillis(-1) && age < leaseLength) { + // Note that if this heartbeat's age is more than a day in the future, it is considered as bad data. A day seems to be + // long enough to tolerate reasonable clock skews between nodes. // 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)