Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Get latest counter before attempting a take to ensure take succeeds #886

Merged
merged 3 commits into from
Jan 4, 2022
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
@@ -0,0 +1,31 @@
/*
* Copyright 2020 Amazon.com, Inc. or its affiliates.
* 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 software.amazon.kinesis.common;


public class CommonCalculations {


/**
* Convenience method for calculating renewer intervals in milliseconds.
*
* @param leaseDurationMillis Duration of a lease
* @param epsilonMillis Allow for some variance when calculating lease expirations
*/
public static long getRenewerTakerIntervalMillis(long leaseDurationMillis, long epsilonMillis) {
return leaseDurationMillis / 3 - epsilonMillis;
Copy link
Contributor

Choose a reason for hiding this comment

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

is this essentially veryOldLeaseDurationNanosMultiplier?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

yep, being extracted into a static function

Copy link
Contributor

Choose a reason for hiding this comment

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

can you use veryOldLeaseDurationNanosMultiplier instead?

Copy link
Contributor Author

Choose a reason for hiding this comment

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

it's not from my commit, do you have strong preference about it?

}
}
Original file line number Diff line number Diff line change
Expand Up @@ -14,21 +14,21 @@
*/
package software.amazon.kinesis.leases;

import java.util.Collection;
import java.util.HashSet;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.TimeUnit;
import lombok.EqualsAndHashCode;
import lombok.Getter;
import lombok.NoArgsConstructor;
import lombok.NonNull;
import lombok.Setter;
import lombok.ToString;
import lombok.experimental.Accessors;
import software.amazon.kinesis.common.HashKeyRangeForLease;
import software.amazon.kinesis.retrieval.kpl.ExtendedSequenceNumber;

import java.util.Collection;
import java.util.HashSet;
import java.util.Set;
import java.util.UUID;
import java.util.concurrent.TimeUnit;

/**
* This class contains data pertaining to a Lease. Distributed systems may use leases to partition work across a
* fleet of workers. Each unit of work (identified by a leaseKey) has a corresponding Lease. Every worker will contend
Expand All @@ -39,7 +39,7 @@
@NoArgsConstructor
@Getter
@Accessors(fluent = true)
@EqualsAndHashCode(exclude = {"concurrencyToken", "lastCounterIncrementNanos", "childShardIds", "pendingCheckpointState"})
@EqualsAndHashCode(exclude = {"concurrencyToken", "lastCounterIncrementNanos", "childShardIds", "pendingCheckpointState", "isMarkedForLeaseSteal"})
@ToString
public class Lease {
/*
Expand Down Expand Up @@ -91,6 +91,16 @@ public class Lease {
*/
private byte[] pendingCheckpointState;


/**
* Denotes whether the lease is marked for stealing. Deliberately excluded from hashCode and equals and
* not persisted in DynamoDB.
*
* @return flag for denoting lease is marked for stealing.
*/
@Setter
private boolean isMarkedForLeaseSteal;

/**
* @return count of distinct lease holders between checkpoints.
*/
Expand Down Expand Up @@ -141,6 +151,7 @@ public Lease(final String leaseKey, final String leaseOwner, final Long leaseCou
}
this.hashKeyRangeForLease = hashKeyRangeForLease;
this.pendingCheckpointState = pendingCheckpointState;
this.isMarkedForLeaseSteal = false;
}

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -14,6 +14,8 @@
*/
package software.amazon.kinesis.leases.dynamodb;

import com.google.common.util.concurrent.ThreadFactoryBuilder;

import java.util.Collection;
import java.util.Collections;
import java.util.List;
Expand All @@ -28,9 +30,6 @@
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;
import java.util.stream.Collectors;

import com.google.common.util.concurrent.ThreadFactoryBuilder;

import lombok.extern.slf4j.Slf4j;
import software.amazon.kinesis.annotations.KinesisClientInternalApi;
import software.amazon.kinesis.leases.Lease;
Expand All @@ -49,6 +48,8 @@
import software.amazon.kinesis.metrics.MetricsScope;
import software.amazon.kinesis.metrics.MetricsUtil;

import static software.amazon.kinesis.common.CommonCalculations.getRenewerTakerIntervalMillis;

/**
* LeaseCoordinator abstracts away LeaseTaker and LeaseRenewer from the application code that's using leasing. It owns
* the scheduling of the two previously mentioned components as well as informing LeaseRenewer when LeaseTaker takes new
Expand Down Expand Up @@ -156,7 +157,7 @@ public DynamoDBLeaseCoordinator(final LeaseRefresher leaseRefresher,
.withMaxLeasesToStealAtOneTime(maxLeasesToStealAtOneTime);
this.leaseRenewer = new DynamoDBLeaseRenewer(
leaseRefresher, workerIdentifier, leaseDurationMillis, leaseRenewalThreadpool, metricsFactory);
this.renewerIntervalMillis = leaseDurationMillis / 3 - epsilonMillis;
this.renewerIntervalMillis = getRenewerTakerIntervalMillis(leaseDurationMillis, epsilonMillis);
this.takerIntervalMillis = (leaseDurationMillis + epsilonMillis) * 2;
if (initialLeaseTableReadCapacity <= 0) {
throw new IllegalArgumentException("readCapacity should be >= 1");
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -42,6 +42,8 @@
import software.amazon.kinesis.metrics.MetricsScope;
import software.amazon.kinesis.metrics.MetricsUtil;

import static software.amazon.kinesis.common.CommonCalculations.getRenewerTakerIntervalMillis;

/**
* An implementation of {@link LeaseTaker} that uses DynamoDB via {@link LeaseRefresher}.
*/
Expand All @@ -50,7 +52,7 @@
public class DynamoDBLeaseTaker implements LeaseTaker {
private static final int TAKE_RETRIES = 3;
private static final int SCAN_RETRIES = 1;
private long veryOldLeaseDurationNanosMultiplier = 3;
private static final double RENEWAL_SLACK_PERCENTAGE = 0.5;

// See note on TAKE_LEASES_DIMENSION(Callable) for why we have this callable.
private static final Callable<Long> SYSTEM_CLOCK_CALLABLE = System::nanoTime;
Expand All @@ -60,20 +62,23 @@ public class DynamoDBLeaseTaker implements LeaseTaker {
private final LeaseRefresher leaseRefresher;
private final String workerIdentifier;
private final long leaseDurationNanos;
private final long leaseRenewalIntervalMillis;
private final MetricsFactory metricsFactory;

final Map<String, Lease> allLeases = new HashMap<>();
// TODO: Remove these defaults and use the defaults in the config
private int maxLeasesForWorker = Integer.MAX_VALUE;
private int maxLeasesToStealAtOneTime = 1;

private long veryOldLeaseDurationNanosMultiplier = 3;
private long lastScanTimeNanos = 0L;

final Map<String, Lease> allLeases = new HashMap<>();

public DynamoDBLeaseTaker(LeaseRefresher leaseRefresher, String workerIdentifier, long leaseDurationMillis,
final MetricsFactory metricsFactory) {
this.leaseRefresher = leaseRefresher;
this.workerIdentifier = workerIdentifier;
this.leaseRenewalIntervalMillis = getRenewerTakerIntervalMillis(leaseDurationMillis, 0);
this.leaseDurationNanos = TimeUnit.MILLISECONDS.toNanos(leaseDurationMillis);
this.metricsFactory = metricsFactory;
}
Expand Down Expand Up @@ -139,12 +144,12 @@ public Map<String, Lease> takeLeases() throws DependencyException, InvalidStateE
* Internal implementation of TAKE_LEASES_DIMENSION. Takes a callable that can provide the time to enable test cases
* without Thread.sleep. Takes a callable instead of a raw time value because the time needs to be computed as-of
* immediately after the scan.
*
*
* @param timeProvider
* Callable that will supply the time
*
*
* @return map of lease key to taken lease
*
*
* @throws DependencyException
* @throws InvalidStateException
*/
Expand All @@ -156,6 +161,7 @@ synchronized Map<String, Lease> takeLeases(Callable<Long> timeProvider)
final MetricsScope scope = MetricsUtil.createMetricsWithOperation(metricsFactory, TAKE_LEASES_DIMENSION);

long startTime = System.currentTimeMillis();
long updateAllLeasesTotalTimeMillis;
boolean success = false;

ProvisionedThroughputException lastException = null;
Expand All @@ -173,19 +179,23 @@ synchronized Map<String, Lease> takeLeases(Callable<Long> timeProvider)
}
}
} finally {
updateAllLeasesTotalTimeMillis = System.currentTimeMillis() - startTime;
MetricsUtil.addWorkerIdentifier(scope, workerIdentifier);
MetricsUtil.addSuccessAndLatency(scope, "ListLeases", success, startTime, MetricsLevel.DETAILED);
}


if (lastException != null) {
log.error("Worker {} could not scan leases table, aborting TAKE_LEASES_DIMENSION. Exception caught by"
+ " last retry:", workerIdentifier, lastException);
+ " last retry:", workerIdentifier, lastException);
return takenLeases;
}

List<Lease> expiredLeases = getExpiredLeases();

Set<Lease> leasesToTake = computeLeasesToTake(expiredLeases);
leasesToTake = updateStaleLeasesWithLatestState(updateAllLeasesTotalTimeMillis, leasesToTake);

Set<String> untakenLeaseKeys = new HashSet<>();

for (Lease lease : leasesToTake) {
Expand Down Expand Up @@ -233,6 +243,33 @@ synchronized Map<String, Lease> takeLeases(Callable<Long> timeProvider)
return takenLeases;
}

/**
* If update all leases takes longer than the lease renewal time,
* we fetch the latest lease info for the given leases that are marked for lease steal.
* If nothing is found (or any transient network error occurs),
* we default to the last known state of the lease
*
* @param updateAllLeasesEndTime How long it takes for update all leases to complete
* @return set of leases to take.
*/
private Set<Lease> updateStaleLeasesWithLatestState(long updateAllLeasesEndTime,
Set<Lease> leasesToTake) {
if (updateAllLeasesEndTime > leaseRenewalIntervalMillis * RENEWAL_SLACK_PERCENTAGE) {
leasesToTake = leasesToTake.stream().map(lease -> {
if (lease.isMarkedForLeaseSteal()) {
try {
return leaseRefresher.getLease(lease.leaseKey());
} catch (DependencyException | InvalidStateException | ProvisionedThroughputException e) {
Copy link
Contributor

@avahuang0429 avahuang0429 Dec 27, 2021

Choose a reason for hiding this comment

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

#765 (comment)

this needs to be addressed

The log line information doesn't really make sense. You can keep the original log line as is, but explain why we would run into getting these exception in the first place

Copy link
Contributor Author

Choose a reason for hiding this comment

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

i think while refreshing the stale lease explains why we get this exception, following the suggestion from Ashwing Like while we tried to update the stale leases in #765 (comment). And the line defaulting to existing lease explains the current status... do you want more context on the failure?

Copy link
Contributor

Choose a reason for hiding this comment

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

"Unable to retrieve the current lease while refreshing the stale lease" this means that we know for a fact that the leases being refreshed are stale while in fact they can be just the same as in DDB. This is not accurate. You can explain in comment naming different cases we run into exceptions.

"Like while we tried to update the stale leases" <- i am not sure this is the right reason why we would run into such exception. He might have confused it with a conditional update; while this is in fact just a get ddb request. I am guessing his intention is we explain how we would run into dependencyException and invalidStateException

Copy link
Contributor

@ashwing ashwing Dec 30, 2021

Choose a reason for hiding this comment

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

Here we are trying to update the local copy of the leases with latest info in the event of leases going stale. Note that we already fetched the leases from ddb table just a while ago, but now we want to get their latest state in order to successfully steal the leases. Having a message like "Failed to fetch latest state of the lease that needs to be stolen" would help.

Copy link
Contributor

Choose a reason for hiding this comment

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

can you add leasekey to the msg

Copy link
Contributor Author

Choose a reason for hiding this comment

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

leaseKey is added to the logline

log.warn("Failed to fetch latest state of the lease {} that needs to be stolen, "
Copy link
Contributor Author

Choose a reason for hiding this comment

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

added leaseKey in the logline

+ "defaulting to existing lease", lease.leaseKey(), e);
}
}
return lease;
}).collect(Collectors.toSet());
}
return leasesToTake;
}

/** Package access for testing purposes.
*
* @param strings
Expand All @@ -249,17 +286,17 @@ static String stringJoin(Collection<String> strings, String delimiter) {
builder.append(string);
needDelimiter = true;
}

return builder.toString();
}

/**
* Scan all leases and update lastRenewalTime. Add new leases and delete old leases.
*
*
* @param timeProvider callable that supplies the current time
*
*
* @return list of expired leases, possibly empty, never null.
*
*
* @throws ProvisionedThroughputException if listLeases fails due to lack of provisioned throughput
* @throws InvalidStateException if the lease table does not exist
* @throws DependencyException if listLeases fails in an unexpected way
Expand Down Expand Up @@ -336,7 +373,7 @@ private List<Lease> getExpiredLeases() {

/**
* Compute the number of leases I should try to take based on the state of the system.
*
*
* @param expiredLeases list of leases we determined to be expired
* @return set of leases to take.
*/
Expand Down Expand Up @@ -461,11 +498,11 @@ private Set<Lease> computeLeasesToTake(List<Lease> expiredLeases) {
/**
* Choose leases to steal by randomly selecting one or more (up to max) from the most loaded worker.
* Stealing rules:
*
*
* Steal up to maxLeasesToStealAtOneTime leases from the most loaded worker if
* a) he has > target leases and I need >= 1 leases : steal min(leases needed, maxLeasesToStealAtOneTime)
* b) he has == target leases and I need > 1 leases : steal 1
*
*
* @param leaseCounts map of workerIdentifier to lease count
* @param needed # of leases needed to reach the target leases for the worker
* @param target target # of leases per worker
Expand Down Expand Up @@ -530,15 +567,16 @@ private List<Lease> chooseLeasesToSteal(Map<String, Integer> leaseCounts, int ne
// Return random ones
Collections.shuffle(candidates);
int toIndex = Math.min(candidates.size(), numLeasesToSteal);
leasesToSteal.addAll(candidates.subList(0, toIndex));

leasesToSteal.addAll(candidates.subList(0, toIndex).stream()
.map(lease -> lease.isMarkedForLeaseSteal(true))
.collect(Collectors.toList()));
return leasesToSteal;
}

/**
* Count leases by host. Always includes myself, but otherwise only includes hosts that are currently holding
* leases.
*
*
* @param expiredLeases list of leases that are currently expired
* @return map of workerIdentifier to lease count
*/
Expand All @@ -563,11 +601,7 @@ Map<String, Integer> computeLeaseCounts(List<Lease> expiredLeases) {
}

// If I have no leases, I wasn't represented in leaseCounts. Let's fix that.
Integer myCount = leaseCounts.get(workerIdentifier);
if (myCount == null) {
myCount = 0;
leaseCounts.put(workerIdentifier, myCount);
}
leaseCounts.putIfAbsent(workerIdentifier, 0);

return leaseCounts;
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -15,7 +15,9 @@
package software.amazon.kinesis.leases.dynamodb;

import java.util.Collection;
import java.util.List;
import java.util.Map;
import java.util.stream.Collectors;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
Expand All @@ -26,6 +28,7 @@
import software.amazon.kinesis.leases.exceptions.LeasingException;
import software.amazon.kinesis.metrics.NullMetricsFactory;
import static org.hamcrest.CoreMatchers.equalTo;
import static org.junit.Assert.assertEquals;
import static org.junit.Assert.assertThat;

@RunWith(MockitoJUnitRunner.class)
Expand Down Expand Up @@ -150,6 +153,33 @@ public void testGetAllLeases() throws LeasingException {
assertThat(addedLeases.values().containsAll(allLeases), equalTo(true));
}


/**
* Sets the leaseDurationMillis to 0, ensuring a get request to update the existing lease after computing
* leases to take
*/
@Test
public void testSlowGetAllLeases() throws LeasingException {
long leaseDurationMillis = 0;
taker = new DynamoDBLeaseTaker(leaseRefresher,
"foo",
leaseDurationMillis,
new NullMetricsFactory());
TestHarnessBuilder builder = new TestHarnessBuilder(leaseRefresher);

Map<String, Lease> addedLeases = builder.withLease("1", "bar")
.withLease("2", "bar")
.withLease("5", "foo")
.build();

assertThat(taker.allLeases().size(), equalTo(0));
taker.takeLeases();

Collection<Lease> allLeases = taker.allLeases();
assertThat(allLeases.size(), equalTo(addedLeases.size()));
assertEquals(addedLeases.values().size(), allLeases.size());
}

/**
* Verify that LeaseTaker does not steal when it's only short 1 lease and the other worker is at target. Set up a
* scenario where there are 4 leases held by two servers, and a third server with one lease. The third server should
Expand Down Expand Up @@ -189,7 +219,7 @@ public void testSteal() throws LeasingException {
builder.build();

// Assert that one lease was stolen from baz.
Map<String, Lease> takenLeases = builder.takeMutateAssert(taker, 1);
Map<String, Lease> takenLeases = builder.stealMutateAssert(taker, 1);

// Assert that it was one of baz's leases (shardId != 1)
String shardIdStolen = takenLeases.keySet().iterator().next();
Expand Down
Loading