Merge remote-tracking branch 'upstream/master' into add-setclass-to-special-key-space
This commit is contained in:
commit
d270b15c89
|
@ -22,6 +22,8 @@ set(JAVA_BINDING_SRCS
|
|||
src/main/com/apple/foundationdb/directory/NoSuchDirectoryException.java
|
||||
src/main/com/apple/foundationdb/directory/package-info.java
|
||||
src/main/com/apple/foundationdb/directory/PathUtil.java
|
||||
src/main/com/apple/foundationdb/DirectBufferIterator.java
|
||||
src/main/com/apple/foundationdb/DirectBufferPool.java
|
||||
src/main/com/apple/foundationdb/FDB.java
|
||||
src/main/com/apple/foundationdb/FDBDatabase.java
|
||||
src/main/com/apple/foundationdb/FDBTransaction.java
|
||||
|
|
|
@ -305,42 +305,6 @@ JNIEXPORT jobject JNICALL Java_com_apple_foundationdb_FutureStrings_FutureString
|
|||
return arr;
|
||||
}
|
||||
|
||||
JNIEXPORT jobject JNICALL Java_com_apple_foundationdb_FutureResults_FutureResults_1getSummary(JNIEnv *jenv, jobject, jlong future) {
|
||||
if( !future ) {
|
||||
throwParamNotNull(jenv);
|
||||
return JNI_NULL;
|
||||
}
|
||||
|
||||
FDBFuture *f = (FDBFuture *)future;
|
||||
|
||||
const FDBKeyValue *kvs;
|
||||
int count;
|
||||
fdb_bool_t more;
|
||||
fdb_error_t err = fdb_future_get_keyvalue_array( f, &kvs, &count, &more );
|
||||
if( err ) {
|
||||
safeThrow( jenv, getThrowable( jenv, err ) );
|
||||
return JNI_NULL;
|
||||
}
|
||||
|
||||
jbyteArray lastKey = JNI_NULL;
|
||||
if(count) {
|
||||
lastKey = jenv->NewByteArray(kvs[count - 1].key_length);
|
||||
if( !lastKey ) {
|
||||
if( !jenv->ExceptionOccurred() )
|
||||
throwOutOfMem(jenv);
|
||||
return JNI_NULL;
|
||||
}
|
||||
|
||||
jenv->SetByteArrayRegion(lastKey, 0, kvs[count - 1].key_length, (jbyte *)kvs[count - 1].key);
|
||||
}
|
||||
|
||||
jobject result = jenv->NewObject(range_result_summary_class, range_result_summary_init, lastKey, count, (jboolean)more);
|
||||
if( jenv->ExceptionOccurred() )
|
||||
return JNI_NULL;
|
||||
|
||||
return result;
|
||||
}
|
||||
|
||||
// SOMEDAY: explore doing this more efficiently with Direct ByteBuffers
|
||||
JNIEXPORT jobject JNICALL Java_com_apple_foundationdb_FutureResults_FutureResults_1get(JNIEnv *jenv, jobject, jlong future) {
|
||||
if( !future ) {
|
||||
|
@ -640,6 +604,68 @@ JNIEXPORT jlong JNICALL Java_com_apple_foundationdb_FDBTransaction_Transaction_1
|
|||
return (jlong)f;
|
||||
}
|
||||
|
||||
JNIEXPORT void JNICALL Java_com_apple_foundationdb_FutureResults_FutureResults_1getDirect(
|
||||
JNIEnv* jenv, jobject, jlong future, jobject jbuffer, jint bufferCapacity) {
|
||||
|
||||
if( !future ) {
|
||||
throwParamNotNull(jenv);
|
||||
return;
|
||||
}
|
||||
|
||||
uint8_t* buffer = (uint8_t*)jenv->GetDirectBufferAddress(jbuffer);
|
||||
if (!buffer) {
|
||||
if (!jenv->ExceptionOccurred())
|
||||
throwRuntimeEx(jenv, "Error getting handle to native resources");
|
||||
return;
|
||||
}
|
||||
|
||||
FDBFuture* f = (FDBFuture*)future;
|
||||
const FDBKeyValue *kvs;
|
||||
int count;
|
||||
fdb_bool_t more;
|
||||
fdb_error_t err = fdb_future_get_keyvalue_array( f, &kvs, &count, &more );
|
||||
if( err ) {
|
||||
safeThrow( jenv, getThrowable( jenv, err ) );
|
||||
return;
|
||||
}
|
||||
|
||||
// Capacity for Metadata+Keys+Values
|
||||
// => sizeof(jint) for total key/value pairs
|
||||
// => sizeof(jint) to store more flag
|
||||
// => sizeof(jint) to store key length per KV pair
|
||||
// => sizeof(jint) to store value length per KV pair
|
||||
int totalCapacityNeeded = 2 * sizeof(jint);
|
||||
for(int i = 0; i < count; i++) {
|
||||
totalCapacityNeeded += kvs[i].key_length + kvs[i].value_length + 2*sizeof(jint);
|
||||
if (bufferCapacity < totalCapacityNeeded) {
|
||||
count = i; /* Only fit first `i` K/V pairs */
|
||||
more = true;
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
int offset = 0;
|
||||
|
||||
// First copy RangeResultSummary, i.e. [keyCount, more]
|
||||
memcpy(buffer + offset, &count, sizeof(jint));
|
||||
offset += sizeof(jint);
|
||||
|
||||
memcpy(buffer + offset, &more, sizeof(jint));
|
||||
offset += sizeof(jint);
|
||||
|
||||
for (int i = 0; i < count; i++) {
|
||||
memcpy(buffer + offset, &kvs[i].key_length, sizeof(jint));
|
||||
memcpy(buffer + offset + sizeof(jint), &kvs[i].value_length, sizeof(jint));
|
||||
offset += 2 * sizeof(jint);
|
||||
|
||||
memcpy(buffer + offset, kvs[i].key, kvs[i].key_length);
|
||||
offset += kvs[i].key_length;
|
||||
|
||||
memcpy(buffer + offset, kvs[i].value, kvs[i].value_length);
|
||||
offset += kvs[i].value_length;
|
||||
}
|
||||
}
|
||||
|
||||
JNIEXPORT jlong JNICALL Java_com_apple_foundationdb_FDBTransaction_Transaction_1getEstimatedRangeSizeBytes(JNIEnv *jenv, jobject, jlong tPtr,
|
||||
jbyteArray beginKeyBytes, jbyteArray endKeyBytes) {
|
||||
if( !tPtr || !beginKeyBytes || !endKeyBytes) {
|
||||
|
|
|
@ -0,0 +1,108 @@
|
|||
/*
|
||||
* DirectBufferIterator.java
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2015-2020 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;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.ByteOrder;
|
||||
import java.util.Iterator;
|
||||
import java.util.NoSuchElementException;
|
||||
|
||||
/**
|
||||
* Holds the direct buffer that is shared with JNI wrapper. A typical usage is as follows:
|
||||
*
|
||||
* The serialization format of result is =>
|
||||
* [int keyCount, boolean more, ListOf<(int keyLen, int valueLen, byte[] key, byte[] value)>]
|
||||
*/
|
||||
class DirectBufferIterator implements Iterator<KeyValue>, AutoCloseable {
|
||||
private ByteBuffer byteBuffer;
|
||||
private int current = 0;
|
||||
private int keyCount = -1;
|
||||
private boolean more = false;
|
||||
|
||||
public DirectBufferIterator(ByteBuffer buffer) {
|
||||
byteBuffer = buffer;
|
||||
byteBuffer.order(ByteOrder.nativeOrder());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
if (byteBuffer != null) {
|
||||
DirectBufferPool.getInstance().add(byteBuffer);
|
||||
byteBuffer = null;
|
||||
}
|
||||
}
|
||||
|
||||
public boolean hasResultReady() {
|
||||
return keyCount > -1;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean hasNext() {
|
||||
assert (hasResultReady());
|
||||
return current < keyCount;
|
||||
}
|
||||
|
||||
@Override
|
||||
public KeyValue next() {
|
||||
assert (hasResultReady()); // Must be called once its ready.
|
||||
if (!hasNext()) {
|
||||
throw new NoSuchElementException();
|
||||
}
|
||||
|
||||
final int keyLen = byteBuffer.getInt();
|
||||
final int valueLen = byteBuffer.getInt();
|
||||
byte[] key = new byte[keyLen];
|
||||
byteBuffer.get(key);
|
||||
|
||||
byte[] value = new byte[valueLen];
|
||||
byteBuffer.get(value);
|
||||
|
||||
current += 1;
|
||||
return new KeyValue(key, value);
|
||||
}
|
||||
|
||||
public ByteBuffer getBuffer() {
|
||||
return byteBuffer;
|
||||
}
|
||||
|
||||
public int count() {
|
||||
assert (hasResultReady());
|
||||
return keyCount;
|
||||
}
|
||||
|
||||
public boolean hasMore() {
|
||||
assert (hasResultReady());
|
||||
return more;
|
||||
}
|
||||
|
||||
public int currentIndex() {
|
||||
return current;
|
||||
}
|
||||
|
||||
public void readResultsSummary() {
|
||||
byteBuffer.rewind();
|
||||
byteBuffer.position(0);
|
||||
|
||||
keyCount = byteBuffer.getInt();
|
||||
more = byteBuffer.getInt() > 0;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,89 @@
|
|||
/*
|
||||
* DirectBufferPool.java
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2015-2020 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;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.concurrent.ArrayBlockingQueue;
|
||||
|
||||
/**
|
||||
* A singleton that manages a pool of {@link DirectByteBuffer}, that will be
|
||||
* shared by the {@link DirectBufferIterator} instances. It is responsibilty of
|
||||
* user to return the borrowed buffers.
|
||||
*/
|
||||
class DirectBufferPool {
|
||||
static final DirectBufferPool __instance = new DirectBufferPool();
|
||||
|
||||
// When tuning this, make sure that the size of the buffer,
|
||||
// is always greater than the maximum size KV allowed by FDB.
|
||||
// Current limits is :
|
||||
// 10kB for key + 100kB for value + 1 int for count + 1 int for more + 2 int for KV size
|
||||
static public final int MIN_BUFFER_SIZE = (10 + 100) * 1000 + Integer.BYTES * 4;
|
||||
|
||||
static private final int DEFAULT_NUM_BUFFERS = 128;
|
||||
static private final int DEFAULT_BUFFER_SIZE = 1024 * 512;
|
||||
|
||||
private ArrayBlockingQueue<ByteBuffer> buffers;
|
||||
private int currentBufferCapacity;
|
||||
|
||||
public DirectBufferPool() {
|
||||
resize(DEFAULT_NUM_BUFFERS, DEFAULT_BUFFER_SIZE);
|
||||
}
|
||||
|
||||
public static DirectBufferPool getInstance() {
|
||||
return __instance;
|
||||
}
|
||||
|
||||
/**
|
||||
* Resizes buffer pool with given capacity and buffer size. Throws OutOfMemory exception
|
||||
* if unable to allocate as asked.
|
||||
*/
|
||||
public synchronized void resize(int newPoolSize, int bufferSize) {
|
||||
if (bufferSize < MIN_BUFFER_SIZE) {
|
||||
throw new IllegalArgumentException("'bufferSize' must be at-least: " + MIN_BUFFER_SIZE + " bytes");
|
||||
}
|
||||
buffers = new ArrayBlockingQueue<>(newPoolSize);
|
||||
currentBufferCapacity = bufferSize;
|
||||
while (buffers.size() < newPoolSize) {
|
||||
ByteBuffer buffer = ByteBuffer.allocateDirect(bufferSize);
|
||||
buffers.add(buffer);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Requests a {@link DirectByteBuffer} from our pool. Returns null if pool is empty.
|
||||
*/
|
||||
public synchronized ByteBuffer poll() {
|
||||
return buffers.poll();
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the {@link DirectByteBuffer} that was borrowed from our pool.
|
||||
*/
|
||||
public synchronized void add(ByteBuffer buffer) {
|
||||
if (buffer.capacity() != currentBufferCapacity) {
|
||||
// This can happen when a resize is called while there are outstanding requests,
|
||||
// older buffers will be returned eventually.
|
||||
return;
|
||||
}
|
||||
|
||||
buffers.offer(buffer);
|
||||
}
|
||||
}
|
|
@ -85,6 +85,8 @@ public class FDB {
|
|||
private volatile boolean netStarted = false;
|
||||
private volatile boolean netStopped = false;
|
||||
volatile boolean warnOnUnclosed = true;
|
||||
private boolean enableDirectBufferQueries = false;
|
||||
|
||||
private boolean useShutdownHook = true;
|
||||
private Thread shutdownHook;
|
||||
private final Semaphore netRunning = new Semaphore(1);
|
||||
|
@ -229,6 +231,35 @@ public class FDB {
|
|||
return apiVersion;
|
||||
}
|
||||
|
||||
/**
|
||||
* Enables or disables use of DirectByteBuffers for getRange() queries.
|
||||
*
|
||||
* @param enabled Whether DirectByteBuffer should be used for getRange() queries.
|
||||
*/
|
||||
public void enableDirectBufferQuery(boolean enabled) {
|
||||
enableDirectBufferQueries = enabled;
|
||||
}
|
||||
|
||||
/**
|
||||
* Determines whether {@code getRange()} queries can use {@code DirectByteBuffer} from
|
||||
* {@link DirectBufferPool} to copy results.
|
||||
*
|
||||
* @return {@code true} if direct buffer queries have been enabled and {@code false} otherwise
|
||||
*/
|
||||
public boolean isDirectBufferQueriesEnabled() {
|
||||
return enableDirectBufferQueries;
|
||||
}
|
||||
|
||||
/**
|
||||
* Resizes the DirectBufferPool with given parameters, which is used by getRange() requests.
|
||||
*
|
||||
* @param poolSize Number of buffers in pool
|
||||
* @param bufferSize Size of each buffer in bytes
|
||||
*/
|
||||
public void resizeDirectBufferPool(int poolSize, int bufferSize) {
|
||||
DirectBufferPool.getInstance().resize(poolSize, bufferSize);
|
||||
}
|
||||
|
||||
/**
|
||||
* Connects to the cluster specified by the
|
||||
* <a href="/foundationdb/administration.html#default-cluster-file" target="_blank">default fdb.cluster file</a>.
|
||||
|
@ -507,4 +538,4 @@ public class FDB {
|
|||
private native boolean Error_predicate(int predicate, int code);
|
||||
|
||||
private native long Database_create(String clusterFilePath) throws FDBException;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -25,6 +25,7 @@ import java.util.concurrent.CompletionException;
|
|||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.Executor;
|
||||
import java.util.function.Function;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
import com.apple.foundationdb.async.AsyncIterable;
|
||||
import com.apple.foundationdb.async.AsyncUtil;
|
||||
|
@ -36,7 +37,6 @@ class FDBTransaction extends NativeObjectWrapper implements Transaction, OptionC
|
|||
private final TransactionOptions options;
|
||||
|
||||
private boolean transactionOwner;
|
||||
|
||||
public final ReadTransaction snapshot;
|
||||
|
||||
class ReadSnapshot implements ReadTransaction {
|
||||
|
@ -369,10 +369,11 @@ class FDBTransaction extends NativeObjectWrapper implements Transaction, OptionC
|
|||
" -- range get: (%s, %s) limit: %d, bytes: %d, mode: %d, iteration: %d, snap: %s, reverse %s",
|
||||
begin.toString(), end.toString(), rowLimit, targetBytes, streamingMode,
|
||||
iteration, Boolean.toString(isSnapshot), Boolean.toString(reverse)));*/
|
||||
return new FutureResults(Transaction_getRange(
|
||||
getPtr(), begin.getKey(), begin.orEqual(), begin.getOffset(),
|
||||
end.getKey(), end.orEqual(), end.getOffset(), rowLimit, targetBytes,
|
||||
streamingMode, iteration, isSnapshot, reverse), executor);
|
||||
return new FutureResults(
|
||||
Transaction_getRange(getPtr(), begin.getKey(), begin.orEqual(), begin.getOffset(),
|
||||
end.getKey(), end.orEqual(), end.getOffset(), rowLimit, targetBytes,
|
||||
streamingMode, iteration, isSnapshot, reverse),
|
||||
FDB.instance().isDirectBufferQueriesEnabled(), executor);
|
||||
} finally {
|
||||
pointerReadLock.unlock();
|
||||
}
|
||||
|
|
|
@ -20,12 +20,14 @@
|
|||
|
||||
package com.apple.foundationdb;
|
||||
|
||||
import java.nio.ByteBuffer;
|
||||
import java.util.concurrent.Executor;
|
||||
|
||||
class FutureResults extends NativeFuture<RangeResultInfo> {
|
||||
FutureResults(long cPtr, Executor executor) {
|
||||
FutureResults(long cPtr, boolean enableDirectBufferQueries, Executor executor) {
|
||||
super(cPtr);
|
||||
registerMarshalCallback(executor);
|
||||
this.enableDirectBufferQueries = enableDirectBufferQueries;
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -44,26 +46,28 @@ class FutureResults extends NativeFuture<RangeResultInfo> {
|
|||
return new RangeResultInfo(this);
|
||||
}
|
||||
|
||||
public RangeResultSummary getSummary() {
|
||||
try {
|
||||
pointerReadLock.lock();
|
||||
return FutureResults_getSummary(getPtr());
|
||||
}
|
||||
finally {
|
||||
pointerReadLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
public RangeResult getResults() {
|
||||
ByteBuffer buffer = enableDirectBufferQueries
|
||||
? DirectBufferPool.getInstance().poll()
|
||||
: null;
|
||||
try {
|
||||
pointerReadLock.lock();
|
||||
return FutureResults_get(getPtr());
|
||||
}
|
||||
finally {
|
||||
if (buffer != null) {
|
||||
try (DirectBufferIterator directIterator = new DirectBufferIterator(buffer)) {
|
||||
FutureResults_getDirect(getPtr(), directIterator.getBuffer(), directIterator.getBuffer().capacity());
|
||||
return new RangeResult(directIterator);
|
||||
}
|
||||
} else {
|
||||
return FutureResults_get(getPtr());
|
||||
}
|
||||
} finally {
|
||||
pointerReadLock.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
private native RangeResultSummary FutureResults_getSummary(long ptr) throws FDBException;
|
||||
private boolean enableDirectBufferQueries = false;
|
||||
|
||||
private native RangeResult FutureResults_get(long cPtr) throws FDBException;
|
||||
private native void FutureResults_getDirect(long cPtr, ByteBuffer buffer, int capacity)
|
||||
throws FDBException;
|
||||
}
|
||||
|
|
|
@ -152,8 +152,6 @@ class RangeQuery implements AsyncIterable<KeyValue> {
|
|||
@Override
|
||||
public void accept(RangeResultInfo data, Throwable error) {
|
||||
try {
|
||||
final RangeResultSummary summary;
|
||||
|
||||
if(error != null) {
|
||||
promise.completeExceptionally(error);
|
||||
if(error instanceof Error) {
|
||||
|
@ -163,7 +161,8 @@ class RangeQuery implements AsyncIterable<KeyValue> {
|
|||
return;
|
||||
}
|
||||
|
||||
summary = data.getSummary();
|
||||
final RangeResult rangeResult = data.get();
|
||||
final RangeResultSummary summary = rangeResult.getSummary();
|
||||
if(summary.lastKey == null) {
|
||||
promise.complete(Boolean.FALSE);
|
||||
return;
|
||||
|
@ -186,11 +185,11 @@ class RangeQuery implements AsyncIterable<KeyValue> {
|
|||
// If this is the first fetch or the main chunk is exhausted
|
||||
if(chunk == null || index == chunk.values.size()) {
|
||||
nextChunk = null;
|
||||
chunk = data.get();
|
||||
chunk = rangeResult;
|
||||
index = 0;
|
||||
}
|
||||
else {
|
||||
nextChunk = data.get();
|
||||
nextChunk = rangeResult;
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -51,4 +51,22 @@ class RangeResult {
|
|||
}
|
||||
this.more = more;
|
||||
}
|
||||
|
||||
RangeResult(DirectBufferIterator iterator) {
|
||||
iterator.readResultsSummary();
|
||||
more = iterator.hasMore();
|
||||
|
||||
int count = iterator.count();
|
||||
values = new ArrayList<KeyValue>(count);
|
||||
|
||||
for (int i = 0; i < count; ++i) {
|
||||
values.add(iterator.next());
|
||||
}
|
||||
}
|
||||
|
||||
public RangeResultSummary getSummary() {
|
||||
final int keyCount = values.size();
|
||||
final byte[] lastKey = keyCount > 0 ? values.get(keyCount -1).getKey() : null;
|
||||
return new RangeResultSummary(lastKey, keyCount, more);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -21,10 +21,6 @@
|
|||
package com.apple.foundationdb;
|
||||
|
||||
class RangeResultInfo {
|
||||
RangeResultSummary getSummary() {
|
||||
return f.getSummary();
|
||||
}
|
||||
|
||||
RangeResult get() {
|
||||
return f.getResults();
|
||||
}
|
||||
|
|
|
@ -57,8 +57,12 @@ RUN cd /opt/ && curl -L https://github.com/facebook/rocksdb/archive/v6.10.1.tar.
|
|||
echo "d573d2f15cdda883714f7e0bc87b814a8d4a53a82edde558f08f940e905541ee rocksdb.tar.gz" > rocksdb-sha.txt &&\
|
||||
sha256sum -c rocksdb-sha.txt && tar xf rocksdb.tar.gz && rm -rf rocksdb.tar.gz rocksdb-sha.txt
|
||||
|
||||
LABEL version=0.1.14
|
||||
ENV DOCKER_IMAGEVER=0.1.14
|
||||
# Localize time zone
|
||||
ARG TIMEZONEINFO=America/Los_Angeles
|
||||
RUN rm -f /etc/localtime && ln -s /usr/share/zoneinfo/${TIMEZONEINFO} /etc/localtime
|
||||
|
||||
LABEL version=0.1.15
|
||||
ENV DOCKER_IMAGEVER=0.1.15
|
||||
ENV JAVA_HOME=/usr/lib/jvm/java-1.8.0
|
||||
ENV CC=/opt/rh/devtoolset-8/root/usr/bin/gcc
|
||||
ENV CXX=/opt/rh/devtoolset-8/root/usr/bin/g++
|
||||
|
|
|
@ -1,4 +1,4 @@
|
|||
FROM foundationdb/foundationdb-build:0.1.14
|
||||
FROM foundationdb/foundationdb-build:0.1.15
|
||||
|
||||
USER root
|
||||
|
||||
|
@ -7,7 +7,7 @@ ADD artifacts /mnt/artifacts
|
|||
|
||||
# Install build tools for building via make
|
||||
RUN \
|
||||
yum install -y distcc-server gperf rubygems python34 libmpc-devel npm
|
||||
yum install -y distcc-server gperf rubygems python34 libmpc-devel npm cgdb
|
||||
|
||||
# Download and install llvm-10.0.0
|
||||
RUN cd / &&\
|
||||
|
@ -17,7 +17,9 @@ RUN cd / &&\
|
|||
|
||||
# Download and install gcc-9.3.0
|
||||
RUN cd / &&\
|
||||
curl -L $FDB_ARTIFACTSURL/downloads/docker/foundationdb-dev/gcc-9.3.0.tar.gz | tar -xvz
|
||||
curl -L $FDB_ARTIFACTSURL/downloads/docker/foundationdb-dev/gcc-9.3.0.tar.gz | tar -xvz &&\
|
||||
mv -iv /usr/local/bin/go /usr/local/bin/go.gcc93 &&\
|
||||
mv -iv /usr/local/bin/gofmt /usr/local/bin/gofmt.gcc93
|
||||
|
||||
# Download and install distcc 3.3.2 new centos binaries
|
||||
RUN cd / &&\
|
||||
|
@ -48,8 +50,8 @@ RUN cp -iv /usr/local/bin/clang++ /usr/local/bin/clang++.deref &&\
|
|||
ldconfig &&\
|
||||
rm -rf /mnt/artifacts
|
||||
|
||||
LABEL version=0.11.6
|
||||
ENV DOCKER_IMAGEVER=0.11.6
|
||||
LABEL version=0.11.8
|
||||
ENV DOCKER_IMAGEVER=0.11.8
|
||||
|
||||
ENV CLANGCC=/usr/local/bin/clang.de8a65ef
|
||||
ENV CLANGCXX=/usr/local/bin/clang++.de8a65ef
|
||||
|
|
|
@ -2,7 +2,7 @@ version: "3"
|
|||
|
||||
services:
|
||||
common: &common
|
||||
image: foundationdb/foundationdb-build:0.1.13
|
||||
image: foundationdb/foundationdb-build:0.1.15
|
||||
|
||||
build-setup: &build-setup
|
||||
<<: *common
|
||||
|
|
|
@ -155,8 +155,9 @@ The requirement of the new backup system raises several design challenges:
|
|||
|
||||
**Transaction Logs (TLogs)**: The transaction logs make mutations durable to disk for fast commit latencies. The logs receive commits from the proxy in version order, and only respond to the proxy once the data has been written and fsync'ed to an append only mutation log on disk. Storage servers retrieve mutations from TLogs. Once the storage servers have persisted mutations, storage servers then pop the mutations from the TLogs.
|
||||
|
||||
**Proxy**: The proxies are responsible for providing read versions, committing transactions, and tracking the storage servers responsible for each range of keys. In the old backup system, Proxies are responsible to group mutations into backup mutations and write them to the database.
|
||||
**Proxy**: The proxies are responsible for committing transactions, and tracking the storage servers responsible for each range of keys. In the old backup system, Proxies are responsible to group mutations into backup mutations and write them to the database.
|
||||
|
||||
**GrvProxy**: The GRV proxies are responsible for providing read versions.
|
||||
## System overview
|
||||
|
||||
From an end-to-end perspective, the new backup system works in the following steps:
|
||||
|
|
|
@ -6,7 +6,7 @@ FoundationDB makes your architecture flexible and easy to operate. Your applicat
|
|||
|
||||
The following diagram details the logical architecture.
|
||||
|
||||
.. image:: /images/Architecture.png
|
||||
|image0|
|
||||
|
||||
|
||||
Detailed FoundationDB Architecture
|
||||
|
@ -362,6 +362,7 @@ Documentation <https://github.com/apple/foundationdb/blob/master/design/data-dis
|
|||
`Recovery
|
||||
Documentation <https://github.com/apple/foundationdb/blob/master/design/recovery-internals.md>`__
|
||||
|
||||
.. |image0| image:: images/Architecture.png
|
||||
.. |image1| image:: images/architecture-1.jpeg
|
||||
.. |image2| image:: images/architecture-2.jpeg
|
||||
.. |image3| image:: images/architecture-3.jpeg
|
||||
|
|
|
@ -456,16 +456,20 @@ disable
|
|||
|
||||
``throttle disable auto``
|
||||
|
||||
Disables cluster auto-throttling for busy transaction tags. This does not disable any currently active throttles. To do so, run the following command after disabling auto-throttling::
|
||||
|
||||
> throttle off auto
|
||||
Disables cluster auto-throttling for busy transaction tags. This may not disable currently active throttles immediately, seconds of delay is expected.
|
||||
|
||||
list
|
||||
^^^^
|
||||
|
||||
``throttle list [LIMIT]``
|
||||
``throttle list [throttled|recommended|all] [LIMIT]``
|
||||
|
||||
Prints a list of currently active transaction tag throttles.
|
||||
Prints a list of currently active transaction tag throttles, or recommended transaction tag throttles if auto-throttling is disabled.
|
||||
|
||||
``throttled`` - list active transaction tag throttles.
|
||||
|
||||
``recommended`` - list transaction tag throttles recommended by the ratekeeper, but not active yet.
|
||||
|
||||
``all`` - list both active and recommended transaction tag throttles.
|
||||
|
||||
``LIMIT`` - The number of throttles to print. Defaults to 100.
|
||||
|
||||
|
|
|
@ -10,38 +10,38 @@ macOS
|
|||
|
||||
The macOS installation package is supported on macOS 10.7+. It includes the client and (optionally) the server.
|
||||
|
||||
* `FoundationDB-6.3.4.pkg <https://www.foundationdb.org/downloads/6.3.4/macOS/installers/FoundationDB-6.3.4.pkg>`_
|
||||
* `FoundationDB-6.3.5.pkg <https://www.foundationdb.org/downloads/6.3.5/macOS/installers/FoundationDB-6.3.5.pkg>`_
|
||||
|
||||
Ubuntu
|
||||
------
|
||||
|
||||
The Ubuntu packages are supported on 64-bit Ubuntu 12.04+, but beware of the Linux kernel bug in Ubuntu 12.x.
|
||||
|
||||
* `foundationdb-clients-6.3.4-1_amd64.deb <https://www.foundationdb.org/downloads/6.3.4/ubuntu/installers/foundationdb-clients_6.3.4-1_amd64.deb>`_
|
||||
* `foundationdb-server-6.3.4-1_amd64.deb <https://www.foundationdb.org/downloads/6.3.4/ubuntu/installers/foundationdb-server_6.3.4-1_amd64.deb>`_ (depends on the clients package)
|
||||
* `foundationdb-clients-6.3.5-1_amd64.deb <https://www.foundationdb.org/downloads/6.3.5/ubuntu/installers/foundationdb-clients_6.3.5-1_amd64.deb>`_
|
||||
* `foundationdb-server-6.3.5-1_amd64.deb <https://www.foundationdb.org/downloads/6.3.5/ubuntu/installers/foundationdb-server_6.3.5-1_amd64.deb>`_ (depends on the clients package)
|
||||
|
||||
RHEL/CentOS EL6
|
||||
---------------
|
||||
|
||||
The RHEL/CentOS EL6 packages are supported on 64-bit RHEL/CentOS 6.x.
|
||||
|
||||
* `foundationdb-clients-6.3.4-1.el6.x86_64.rpm <https://www.foundationdb.org/downloads/6.3.4/rhel6/installers/foundationdb-clients-6.3.4-1.el6.x86_64.rpm>`_
|
||||
* `foundationdb-server-6.3.4-1.el6.x86_64.rpm <https://www.foundationdb.org/downloads/6.3.4/rhel6/installers/foundationdb-server-6.3.4-1.el6.x86_64.rpm>`_ (depends on the clients package)
|
||||
* `foundationdb-clients-6.3.5-1.el6.x86_64.rpm <https://www.foundationdb.org/downloads/6.3.5/rhel6/installers/foundationdb-clients-6.3.5-1.el6.x86_64.rpm>`_
|
||||
* `foundationdb-server-6.3.5-1.el6.x86_64.rpm <https://www.foundationdb.org/downloads/6.3.5/rhel6/installers/foundationdb-server-6.3.5-1.el6.x86_64.rpm>`_ (depends on the clients package)
|
||||
|
||||
RHEL/CentOS EL7
|
||||
---------------
|
||||
|
||||
The RHEL/CentOS EL7 packages are supported on 64-bit RHEL/CentOS 7.x.
|
||||
|
||||
* `foundationdb-clients-6.3.4-1.el7.x86_64.rpm <https://www.foundationdb.org/downloads/6.3.4/rhel7/installers/foundationdb-clients-6.3.4-1.el7.x86_64.rpm>`_
|
||||
* `foundationdb-server-6.3.4-1.el7.x86_64.rpm <https://www.foundationdb.org/downloads/6.3.4/rhel7/installers/foundationdb-server-6.3.4-1.el7.x86_64.rpm>`_ (depends on the clients package)
|
||||
* `foundationdb-clients-6.3.5-1.el7.x86_64.rpm <https://www.foundationdb.org/downloads/6.3.5/rhel7/installers/foundationdb-clients-6.3.5-1.el7.x86_64.rpm>`_
|
||||
* `foundationdb-server-6.3.5-1.el7.x86_64.rpm <https://www.foundationdb.org/downloads/6.3.5/rhel7/installers/foundationdb-server-6.3.5-1.el7.x86_64.rpm>`_ (depends on the clients package)
|
||||
|
||||
Windows
|
||||
-------
|
||||
|
||||
The Windows installer is supported on 64-bit Windows XP and later. It includes the client and (optionally) the server.
|
||||
|
||||
* `foundationdb-6.3.4-x64.msi <https://www.foundationdb.org/downloads/6.3.4/windows/installers/foundationdb-6.3.4-x64.msi>`_
|
||||
* `foundationdb-6.3.5-x64.msi <https://www.foundationdb.org/downloads/6.3.5/windows/installers/foundationdb-6.3.5-x64.msi>`_
|
||||
|
||||
API Language Bindings
|
||||
=====================
|
||||
|
@ -58,18 +58,18 @@ On macOS and Windows, the FoundationDB Python API bindings are installed as part
|
|||
|
||||
If you need to use the FoundationDB Python API from other Python installations or paths, use the Python package manager ``pip`` (``pip install foundationdb``) or download the Python package:
|
||||
|
||||
* `foundationdb-6.3.4.tar.gz <https://www.foundationdb.org/downloads/6.3.4/bindings/python/foundationdb-6.3.4.tar.gz>`_
|
||||
* `foundationdb-6.3.5.tar.gz <https://www.foundationdb.org/downloads/6.3.5/bindings/python/foundationdb-6.3.5.tar.gz>`_
|
||||
|
||||
Ruby 1.9.3/2.0.0+
|
||||
-----------------
|
||||
|
||||
* `fdb-6.3.4.gem <https://www.foundationdb.org/downloads/6.3.4/bindings/ruby/fdb-6.3.4.gem>`_
|
||||
* `fdb-6.3.5.gem <https://www.foundationdb.org/downloads/6.3.5/bindings/ruby/fdb-6.3.5.gem>`_
|
||||
|
||||
Java 8+
|
||||
-------
|
||||
|
||||
* `fdb-java-6.3.4.jar <https://www.foundationdb.org/downloads/6.3.4/bindings/java/fdb-java-6.3.4.jar>`_
|
||||
* `fdb-java-6.3.4-javadoc.jar <https://www.foundationdb.org/downloads/6.3.4/bindings/java/fdb-java-6.3.4-javadoc.jar>`_
|
||||
* `fdb-java-6.3.5.jar <https://www.foundationdb.org/downloads/6.3.5/bindings/java/fdb-java-6.3.5.jar>`_
|
||||
* `fdb-java-6.3.5-javadoc.jar <https://www.foundationdb.org/downloads/6.3.5/bindings/java/fdb-java-6.3.5-javadoc.jar>`_
|
||||
|
||||
Go 1.11+
|
||||
--------
|
||||
|
|
|
@ -28,6 +28,7 @@
|
|||
"transaction",
|
||||
"resolution",
|
||||
"proxy",
|
||||
"grv_proxy",
|
||||
"master",
|
||||
"test",
|
||||
"storage_cache"
|
||||
|
@ -61,6 +62,7 @@
|
|||
"$enum":[
|
||||
"master",
|
||||
"proxy",
|
||||
"grv_proxy",
|
||||
"log",
|
||||
"storage",
|
||||
"resolver",
|
||||
|
@ -313,11 +315,18 @@
|
|||
"batch_released_transactions_per_second":0,
|
||||
"released_transactions_per_second":0,
|
||||
"throttled_tags":{
|
||||
"auto":{
|
||||
"count":0
|
||||
"auto" : {
|
||||
"busy_read" : 0,
|
||||
"busy_write" : 0,
|
||||
"count" : 0
|
||||
},
|
||||
"manual":{
|
||||
"count":0
|
||||
"manual" : {
|
||||
"count" : 0
|
||||
},
|
||||
"recommend" : {
|
||||
"busy_read" : 0,
|
||||
"busy_write" : 0,
|
||||
"count" : 0
|
||||
}
|
||||
},
|
||||
"limiting_queue_bytes_storage_server":0,
|
||||
|
@ -439,6 +448,7 @@
|
|||
"recovery_state":{
|
||||
"required_resolvers":1,
|
||||
"required_proxies":1,
|
||||
"required_grv_proxies":1,
|
||||
"name":{ // "fully_recovered" is the healthy state; other states are normal to transition through but not to persist in
|
||||
"$enum":[
|
||||
"reading_coordinated_state",
|
||||
|
|
|
@ -132,3 +132,13 @@ log_server_min_free_space Log server running out of space (approaching
|
|||
log_server_min_free_space_ratio Log server running out of space (approaching 5% limit).
|
||||
storage_server_durability_lag Storage server durable version falling behind.
|
||||
=================================== ====================================================
|
||||
|
||||
The JSON path ``cluster.qos.throttled_tags``, when it exists, is an Object containing ``"auto"`` , ``"manual"`` and ``"recommended"``. The possible fields for those object are in the following table:
|
||||
|
||||
=================================== ====================================================
|
||||
Name Description
|
||||
=================================== ====================================================
|
||||
count How many tags are throttled
|
||||
busy_read How many tags are throttled because of busy read
|
||||
busy_write How many tags are throttled because of busy write
|
||||
=================================== ====================================================
|
|
@ -2,6 +2,12 @@
|
|||
Release Notes
|
||||
#############
|
||||
|
||||
6.2.25
|
||||
======
|
||||
|
||||
* Mitigate an issue where a non-lockaware transaction that changes certain ``\xff`` "metadata" keys, committed concurrently with locking the database, can cause corruption. If a non-lockaware transaction manually sets its read version to a version where the database is locked, and changes metadata keys, this can still cause corruption. `(PR #3674) <https://github.com/apple/foundationdb/pull/3674>`_
|
||||
* Reset network connections between the proxies and satellite tlogs if the latencies are larger than 500ms. `(PR #3686) <https://github.com/apple/foundationdb/pull/3686>`_
|
||||
|
||||
6.2.24
|
||||
======
|
||||
|
||||
|
|
|
@ -5,12 +5,6 @@ Release Notes
|
|||
6.3.5
|
||||
=====
|
||||
|
||||
* Fix an issue where ``fdbcli --exec 'exclude no_wait ...'`` would incorrectly report that processes can safely be removed from the cluster. `(PR #3566) <https://github.com/apple/foundationdb/pull/3566>`_
|
||||
* When a configuration key is changed, it will always be included in ``status json`` output, even the value is reverted back to the default value. `(PR #3610) <https://github.com/apple/foundationdb/pull/3610>`_
|
||||
|
||||
6.3.4
|
||||
=====
|
||||
|
||||
Features
|
||||
--------
|
||||
|
||||
|
@ -66,6 +60,7 @@ Fixes
|
|||
* Prevent blob upload timeout if request timeout is lower than expected request time. `(PR #3533) <https://github.com/apple/foundationdb/pull/3533>`_
|
||||
* In very rare scenarios, the data distributor process would crash when being shutdown. `(PR #3530) <https://github.com/apple/foundationdb/pull/3530>`_
|
||||
* The master would die immediately if it did not have the correct cluster controller interface when recruited. [6.3.4] `(PR #3537) <https://github.com/apple/foundationdb/pull/3537>`_
|
||||
* Fix an issue where ``fdbcli --exec 'exclude no_wait ...'`` would incorrectly report that processes can safely be removed from the cluster. [6.3.5] `(PR #3566) <https://github.com/apple/foundationdb/pull/3566>`_
|
||||
|
||||
Status
|
||||
------
|
||||
|
@ -74,6 +69,7 @@ Status
|
|||
* Replaced ``cluster.database_locked`` status field with ``cluster.database_lock_state``, which contains two subfields: ``locked`` (boolean) and ``lock_uid`` (which contains the database lock uid if the database is locked). `(PR #2058) <https://github.com/apple/foundationdb/pull/2058>`_
|
||||
* Removed fields ``worst_version_lag_storage_server`` and ``limiting_version_lag_storage_server`` from the ``cluster.qos`` section. The ``worst_data_lag_storage_server`` and ``limiting_data_lag_storage_server`` objects can be used instead. `(PR #3196) <https://github.com/apple/foundationdb/pull/3196>`_
|
||||
* If a process is unable to flush trace logs to disk, the problem will now be reported via the output of ``status`` command inside ``fdbcli``. `(PR #2605) <https://github.com/apple/foundationdb/pull/2605>`_ `(PR #2820) <https://github.com/apple/foundationdb/pull/2820>`_
|
||||
* When a configuration key is changed, it will always be included in ``status json`` output, even the value is reverted back to the default value. [6.3.5] `(PR #3610) <https://github.com/apple/foundationdb/pull/3610>`_
|
||||
|
||||
Bindings
|
||||
--------
|
||||
|
@ -84,6 +80,8 @@ Bindings
|
|||
* Java: Optimize byte array comparisons in ``ByteArrayUtil``. `(PR #2823) <https://github.com/apple/foundationdb/pull/2823>`_
|
||||
* Java: Add ``FDB.disableShutdownHook`` that can be used to prevent the default shutdown hook from running. Users of this new function should make sure to call ``stopNetwork`` before terminating a client process. `(PR #2635) <https://github.com/apple/foundationdb/pull/2635>`_
|
||||
* Java: Introduced ``keyAfter`` utility function that can be used to create the immediate next key for a given byte array. `(PR #2458) <https://github.com/apple/foundationdb/pull/2458>`_
|
||||
* Java: Combined ``getSummary()`` and ``getResults()`` JNI calls for ``getRange()`` queries. [6.3.5] `(PR #3681) <https://github.com/apple/foundationdb/pull/3681>`_
|
||||
* Java: Added support to use ``DirectByteBuffers`` in ``getRange()`` requests for better performance, which can be enabled using ``FDB.enableDirectBufferQueries``. [6.3.5] `(PR #3681) <https://github.com/apple/foundationdb/pull/3681>`_
|
||||
* Golang: The ``Transact`` function will unwrap errors that have been wrapped using ``xerrors`` to determine if a retryable FoundationDB error is in the error chain. `(PR #3131) <https://github.com/apple/foundationdb/pull/3131>`_
|
||||
* Golang: Added ``Subspace.PackWithVersionstamp`` that can be used to pack a ``Tuple`` that contains a versionstamp. `(PR #2243) <https://github.com/apple/foundationdb/pull/2243>`_
|
||||
* Golang: Implement ``Stringer`` interface for ``Tuple``, ``Subspace``, ``UUID``, and ``Versionstamp``. `(PR #3032) <https://github.com/apple/foundationdb/pull/3032>`_
|
||||
|
@ -116,6 +114,7 @@ Fixes from previous versions
|
|||
|
||||
* The 6.3.1 patch release includes all fixes from the patch releases 6.2.21 and 6.2.22. :doc:`(6.2 Release Notes) </release-notes/release-notes-620>`
|
||||
* The 6.3.3 patch release includes all fixes from the patch release 6.2.23. :doc:`(6.2 Release Notes) </release-notes/release-notes-620>`
|
||||
* The 6.3.5 patch release includes all fixes from the patch releases 6.2.24 and 6.2.25. :doc:`(6.2 Release Notes) </release-notes/release-notes-620>`
|
||||
|
||||
Fixes only impacting 6.3.0+
|
||||
---------------------------
|
||||
|
@ -124,6 +123,7 @@ Fixes only impacting 6.3.0+
|
|||
* Renamed ``MIN_DELAY_STORAGE_CANDIDACY_SECONDS`` knob to ``MIN_DELAY_CC_WORST_FIT_CANDIDACY_SECONDS``. [6.3.2] `(PR #3327) <https://github.com/apple/foundationdb/pull/3327>`_
|
||||
* Refreshing TLS certificates could cause crashes. [6.3.2] `(PR #3352) <https://github.com/apple/foundationdb/pull/3352>`_
|
||||
* All storage class processes attempted to connect to the same coordinator. [6.3.2] `(PR #3361) <https://github.com/apple/foundationdb/pull/3361>`_
|
||||
* Adjusted the proxy load balancing algorithm to be based on the CPU usage of the process instead of the number of requests processed. [6.3.5] `(PR #3653) <https://github.com/apple/foundationdb/pull/3653>`_
|
||||
|
||||
Earlier release notes
|
||||
---------------------
|
||||
|
|
|
@ -96,29 +96,67 @@ enum enumRestoreType {
|
|||
//
|
||||
enum {
|
||||
// Backup constants
|
||||
OPT_DESTCONTAINER, OPT_SNAPSHOTINTERVAL, OPT_ERRORLIMIT, OPT_NOSTOPWHENDONE,
|
||||
OPT_EXPIRE_BEFORE_VERSION, OPT_EXPIRE_BEFORE_DATETIME, OPT_EXPIRE_DELETE_BEFORE_DAYS,
|
||||
OPT_EXPIRE_RESTORABLE_AFTER_VERSION, OPT_EXPIRE_RESTORABLE_AFTER_DATETIME, OPT_EXPIRE_MIN_RESTORABLE_DAYS,
|
||||
OPT_BASEURL, OPT_BLOB_CREDENTIALS, OPT_DESCRIBE_DEEP, OPT_DESCRIBE_TIMESTAMPS,
|
||||
OPT_DUMP_BEGIN, OPT_DUMP_END, OPT_JSON, OPT_DELETE_DATA, OPT_MIN_CLEANUP_SECONDS,
|
||||
OPT_DESTCONTAINER,
|
||||
OPT_SNAPSHOTINTERVAL,
|
||||
OPT_ERRORLIMIT,
|
||||
OPT_NOSTOPWHENDONE,
|
||||
OPT_EXPIRE_BEFORE_VERSION,
|
||||
OPT_EXPIRE_BEFORE_DATETIME,
|
||||
OPT_EXPIRE_DELETE_BEFORE_DAYS,
|
||||
OPT_EXPIRE_RESTORABLE_AFTER_VERSION,
|
||||
OPT_EXPIRE_RESTORABLE_AFTER_DATETIME,
|
||||
OPT_EXPIRE_MIN_RESTORABLE_DAYS,
|
||||
OPT_BASEURL,
|
||||
OPT_BLOB_CREDENTIALS,
|
||||
OPT_DESCRIBE_DEEP,
|
||||
OPT_DESCRIBE_TIMESTAMPS,
|
||||
OPT_DUMP_BEGIN,
|
||||
OPT_DUMP_END,
|
||||
OPT_JSON,
|
||||
OPT_DELETE_DATA,
|
||||
OPT_MIN_CLEANUP_SECONDS,
|
||||
OPT_USE_PARTITIONED_LOG,
|
||||
|
||||
// Backup and Restore constants
|
||||
OPT_TAGNAME, OPT_BACKUPKEYS, OPT_WAITFORDONE,
|
||||
OPT_TAGNAME,
|
||||
OPT_BACKUPKEYS,
|
||||
OPT_WAITFORDONE,
|
||||
OPT_INCREMENTALONLY,
|
||||
|
||||
// Backup Modify
|
||||
OPT_MOD_ACTIVE_INTERVAL, OPT_MOD_VERIFY_UID,
|
||||
OPT_MOD_ACTIVE_INTERVAL,
|
||||
OPT_MOD_VERIFY_UID,
|
||||
|
||||
// Restore constants
|
||||
OPT_RESTORECONTAINER, OPT_RESTORE_VERSION, OPT_RESTORE_TIMESTAMP, OPT_PREFIX_ADD, OPT_PREFIX_REMOVE, OPT_RESTORE_CLUSTERFILE_DEST, OPT_RESTORE_CLUSTERFILE_ORIG,
|
||||
OPT_RESTORECONTAINER,
|
||||
OPT_RESTORE_VERSION,
|
||||
OPT_RESTORE_TIMESTAMP,
|
||||
OPT_PREFIX_ADD,
|
||||
OPT_PREFIX_REMOVE,
|
||||
OPT_RESTORE_CLUSTERFILE_DEST,
|
||||
OPT_RESTORE_CLUSTERFILE_ORIG,
|
||||
OPT_RESTORE_BEGIN_VERSION,
|
||||
|
||||
// Shared constants
|
||||
OPT_CLUSTERFILE, OPT_QUIET, OPT_DRYRUN, OPT_FORCE,
|
||||
OPT_HELP, OPT_DEVHELP, OPT_VERSION, OPT_PARENTPID, OPT_CRASHONERROR,
|
||||
OPT_NOBUFSTDOUT, OPT_BUFSTDOUTERR, OPT_TRACE, OPT_TRACE_DIR,
|
||||
OPT_KNOB, OPT_TRACE_LOG_GROUP, OPT_MEMLIMIT, OPT_LOCALITY,
|
||||
OPT_CLUSTERFILE,
|
||||
OPT_QUIET,
|
||||
OPT_DRYRUN,
|
||||
OPT_FORCE,
|
||||
OPT_HELP,
|
||||
OPT_DEVHELP,
|
||||
OPT_VERSION,
|
||||
OPT_PARENTPID,
|
||||
OPT_CRASHONERROR,
|
||||
OPT_NOBUFSTDOUT,
|
||||
OPT_BUFSTDOUTERR,
|
||||
OPT_TRACE,
|
||||
OPT_TRACE_DIR,
|
||||
OPT_KNOB,
|
||||
OPT_TRACE_LOG_GROUP,
|
||||
OPT_MEMLIMIT,
|
||||
OPT_LOCALITY,
|
||||
|
||||
//DB constants
|
||||
// DB constants
|
||||
OPT_SOURCE_CLUSTER,
|
||||
OPT_DEST_CLUSTER,
|
||||
OPT_CLEANUP,
|
||||
|
@ -154,7 +192,7 @@ CSimpleOpt::SOption g_rgAgentOptions[] = {
|
|||
#ifndef TLS_DISABLED
|
||||
TLS_OPTION_FLAGS
|
||||
#endif
|
||||
SO_END_OF_OPTIONS
|
||||
SO_END_OF_OPTIONS
|
||||
};
|
||||
|
||||
CSimpleOpt::SOption g_rgBackupStartOptions[] = {
|
||||
|
@ -197,6 +235,7 @@ CSimpleOpt::SOption g_rgBackupStartOptions[] = {
|
|||
{ OPT_DEVHELP, "--dev-help", SO_NONE },
|
||||
{ OPT_KNOB, "--knob_", SO_REQ_SEP },
|
||||
{ OPT_BLOB_CREDENTIALS, "--blob_credentials", SO_REQ_SEP },
|
||||
{ OPT_INCREMENTALONLY, "--incremental", SO_NONE },
|
||||
#ifndef TLS_DISABLED
|
||||
TLS_OPTION_FLAGS
|
||||
#endif
|
||||
|
@ -603,6 +642,7 @@ CSimpleOpt::SOption g_rgRestoreOptions[] = {
|
|||
{ OPT_BACKUPKEYS, "--keys", SO_REQ_SEP },
|
||||
{ OPT_WAITFORDONE, "-w", SO_NONE },
|
||||
{ OPT_WAITFORDONE, "--waitfordone", SO_NONE },
|
||||
{ OPT_RESTORE_BEGIN_VERSION, "--begin_version", SO_REQ_SEP },
|
||||
{ OPT_RESTORE_VERSION, "--version", SO_REQ_SEP },
|
||||
{ OPT_RESTORE_VERSION, "-v", SO_REQ_SEP },
|
||||
{ OPT_TRACE, "--log", SO_NONE },
|
||||
|
@ -622,6 +662,7 @@ CSimpleOpt::SOption g_rgRestoreOptions[] = {
|
|||
{ OPT_HELP, "--help", SO_NONE },
|
||||
{ OPT_DEVHELP, "--dev-help", SO_NONE },
|
||||
{ OPT_BLOB_CREDENTIALS, "--blob_credentials", SO_REQ_SEP },
|
||||
{ OPT_INCREMENTALONLY, "--incremental", SO_NONE },
|
||||
#ifndef TLS_DISABLED
|
||||
TLS_OPTION_FLAGS
|
||||
#endif
|
||||
|
@ -975,6 +1016,9 @@ static void printBackupUsage(bool devhelp) {
|
|||
" remove mutations for it. By default this is set to one hour.\n");
|
||||
printf(" --delete_data\n"
|
||||
" This flag will cause cleanup to remove mutations for the most stale backup or DR.\n");
|
||||
// TODO: Enable this command-line argument once atomics are supported
|
||||
// printf(" --incremental\n"
|
||||
// " Performs incremental backup without the base backup.\n");
|
||||
#ifndef TLS_DISABLED
|
||||
printf(TLS_HELP);
|
||||
#endif
|
||||
|
@ -1032,8 +1076,11 @@ static void printRestoreUsage(bool devhelp ) {
|
|||
printf(" --trace_format FORMAT\n"
|
||||
" Select the format of the trace files. xml (the default) and json are supported.\n"
|
||||
" Has no effect unless --log is specified.\n");
|
||||
// TODO: Enable this command-line argument once atomics are supported
|
||||
// printf(" --incremental\n"
|
||||
// " Performs incremental restore without the base backup.\n");
|
||||
#ifndef TLS_DISABLED
|
||||
printf(TLS_HELP);
|
||||
printf(TLS_HELP);
|
||||
#endif
|
||||
printf(" -v DBVERSION The version at which the database will be restored.\n");
|
||||
printf(" --timestamp Instead of a numeric version, use this to specify a timestamp in %s\n", BackupAgentBase::timeFormat().c_str());
|
||||
|
@ -1721,7 +1768,8 @@ ACTOR Future<Void> submitDBBackup(Database src, Database dest, Standalone<Vector
|
|||
|
||||
ACTOR Future<Void> submitBackup(Database db, std::string url, int snapshotIntervalSeconds,
|
||||
Standalone<VectorRef<KeyRangeRef>> backupRanges, std::string tagName, bool dryRun,
|
||||
bool waitForCompletion, bool stopWhenDone, bool usePartitionedLog) {
|
||||
bool waitForCompletion, bool stopWhenDone, bool usePartitionedLog,
|
||||
bool incrementalBackupOnly) {
|
||||
try {
|
||||
state FileBackupAgent backupAgent;
|
||||
|
||||
|
@ -1766,7 +1814,7 @@ ACTOR Future<Void> submitBackup(Database db, std::string url, int snapshotInterv
|
|||
|
||||
else {
|
||||
wait(backupAgent.submitBackup(db, KeyRef(url), snapshotIntervalSeconds, tagName, backupRanges, stopWhenDone,
|
||||
usePartitionedLog));
|
||||
usePartitionedLog, incrementalBackupOnly));
|
||||
|
||||
// Wait for the backup to complete, if requested
|
||||
if (waitForCompletion) {
|
||||
|
@ -2077,7 +2125,10 @@ Reference<IBackupContainer> openBackupContainer(const char *name, std::string de
|
|||
return c;
|
||||
}
|
||||
|
||||
ACTOR Future<Void> runRestore(Database db, std::string originalClusterFile, std::string tagName, std::string container, Standalone<VectorRef<KeyRangeRef>> ranges, Version targetVersion, std::string targetTimestamp, bool performRestore, bool verbose, bool waitForDone, std::string addPrefix, std::string removePrefix) {
|
||||
ACTOR Future<Void> runRestore(Database db, std::string originalClusterFile, std::string tagName, std::string container,
|
||||
Standalone<VectorRef<KeyRangeRef>> ranges, Version beginVersion, Version targetVersion,
|
||||
std::string targetTimestamp, bool performRestore, bool verbose, bool waitForDone,
|
||||
std::string addPrefix, std::string removePrefix, bool incrementalBackupOnly) {
|
||||
if(ranges.empty()) {
|
||||
ranges.push_back_deep(ranges.arena(), normalKeys);
|
||||
}
|
||||
|
@ -2119,19 +2170,23 @@ ACTOR Future<Void> runRestore(Database db, std::string originalClusterFile, std:
|
|||
|
||||
BackupDescription desc = wait(bc->describeBackup());
|
||||
|
||||
if(!desc.maxRestorableVersion.present()) {
|
||||
if (incrementalBackupOnly && desc.contiguousLogEnd.present()) {
|
||||
targetVersion = desc.contiguousLogEnd.get() - 1;
|
||||
} else if (desc.maxRestorableVersion.present()) {
|
||||
targetVersion = desc.maxRestorableVersion.get();
|
||||
} else {
|
||||
fprintf(stderr, "The specified backup is not restorable to any version.\n");
|
||||
throw restore_error();
|
||||
}
|
||||
|
||||
targetVersion = desc.maxRestorableVersion.get();
|
||||
|
||||
if(verbose)
|
||||
printf("Using target restore version %" PRId64 "\n", targetVersion);
|
||||
}
|
||||
|
||||
if (performRestore) {
|
||||
Version restoredVersion = wait(backupAgent.restore(db, origDb, KeyRef(tagName), KeyRef(container), ranges, waitForDone, targetVersion, verbose, KeyRef(addPrefix), KeyRef(removePrefix)));
|
||||
Version restoredVersion = wait(backupAgent.restore(
|
||||
db, origDb, KeyRef(tagName), KeyRef(container), ranges, waitForDone, targetVersion, verbose,
|
||||
KeyRef(addPrefix), KeyRef(removePrefix), true, incrementalBackupOnly, beginVersion));
|
||||
|
||||
if(waitForDone && verbose) {
|
||||
// If restore is now complete then report version restored
|
||||
|
@ -2910,11 +2965,13 @@ int main(int argc, char* argv[]) {
|
|||
std::string removePrefix;
|
||||
Standalone<VectorRef<KeyRangeRef>> backupKeys;
|
||||
int maxErrors = 20;
|
||||
Version beginVersion = invalidVersion;
|
||||
Version restoreVersion = invalidVersion;
|
||||
std::string restoreTimestamp;
|
||||
bool waitForDone = false;
|
||||
bool stopWhenDone = true;
|
||||
bool usePartitionedLog = false; // Set to true to use new backup system
|
||||
bool incrementalBackupOnly = false;
|
||||
bool forceAction = false;
|
||||
bool trace = false;
|
||||
bool quietDisplay = false;
|
||||
|
@ -3167,6 +3224,10 @@ int main(int argc, char* argv[]) {
|
|||
case OPT_USE_PARTITIONED_LOG:
|
||||
usePartitionedLog = true;
|
||||
break;
|
||||
case OPT_INCREMENTALONLY:
|
||||
// TODO: Enable this command-line argument once atomics are supported
|
||||
// incrementalBackupOnly = true;
|
||||
break;
|
||||
case OPT_RESTORECONTAINER:
|
||||
restoreContainer = args->OptionArg();
|
||||
// If the url starts with '/' then prepend "file://" for backwards compatibility
|
||||
|
@ -3194,6 +3255,17 @@ int main(int argc, char* argv[]) {
|
|||
}
|
||||
break;
|
||||
}
|
||||
case OPT_RESTORE_BEGIN_VERSION: {
|
||||
const char* a = args->OptionArg();
|
||||
long long ver = 0;
|
||||
if (!sscanf(a, "%lld", &ver)) {
|
||||
fprintf(stderr, "ERROR: Could not parse database beginVersion `%s'\n", a);
|
||||
printHelpTeaser(argv[0]);
|
||||
return FDB_EXIT_ERROR;
|
||||
}
|
||||
beginVersion = ver;
|
||||
break;
|
||||
}
|
||||
case OPT_RESTORE_VERSION: {
|
||||
const char* a = args->OptionArg();
|
||||
long long ver = 0;
|
||||
|
@ -3567,7 +3639,8 @@ int main(int argc, char* argv[]) {
|
|||
// Test out the backup url to make sure it parses. Doesn't test to make sure it's actually writeable.
|
||||
openBackupContainer(argv[0], destinationContainer);
|
||||
f = stopAfter(submitBackup(db, destinationContainer, snapshotIntervalSeconds, backupKeys, tagName,
|
||||
dryRun, waitForDone, stopWhenDone, usePartitionedLog));
|
||||
dryRun, waitForDone, stopWhenDone, usePartitionedLog,
|
||||
incrementalBackupOnly));
|
||||
break;
|
||||
}
|
||||
|
||||
|
@ -3697,7 +3770,9 @@ int main(int argc, char* argv[]) {
|
|||
|
||||
switch(restoreType) {
|
||||
case RESTORE_START:
|
||||
f = stopAfter( runRestore(db, restoreClusterFileOrig, tagName, restoreContainer, backupKeys, restoreVersion, restoreTimestamp, !dryRun, !quietDisplay, waitForDone, addPrefix, removePrefix) );
|
||||
f = stopAfter(runRestore(db, restoreClusterFileOrig, tagName, restoreContainer, backupKeys,
|
||||
beginVersion, restoreVersion, restoreTimestamp, !dryRun, !quietDisplay,
|
||||
waitForDone, addPrefix, removePrefix, incrementalBackupOnly));
|
||||
break;
|
||||
case RESTORE_WAIT:
|
||||
f = stopAfter( success(ba.waitRestore(db, KeyRef(tagName), true)) );
|
||||
|
|
|
@ -469,9 +469,23 @@ void initHelp() {
|
|||
"clear a range of keys from the database",
|
||||
"All keys between BEGINKEY (inclusive) and ENDKEY (exclusive) are cleared from the database. This command will succeed even if the specified range is empty, but may fail because of conflicts." ESCAPINGK);
|
||||
helpMap["configure"] = CommandHelp(
|
||||
"configure [new] <single|double|triple|three_data_hall|three_datacenter|ssd|memory|memory-radixtree-beta|proxies=<PROXIES>|logs=<LOGS>|resolvers=<RESOLVERS>>*",
|
||||
"change the database configuration",
|
||||
"The `new' option, if present, initializes a new database with the given configuration rather than changing the configuration of an existing one. When used, both a redundancy mode and a storage engine must be specified.\n\nRedundancy mode:\n single - one copy of the data. Not fault tolerant.\n double - two copies of data (survive one failure).\n triple - three copies of data (survive two failures).\n three_data_hall - See the Admin Guide.\n three_datacenter - See the Admin Guide.\n\nStorage engine:\n ssd - B-Tree storage engine optimized for solid state disks.\n memory - Durable in-memory storage engine for small datasets.\n\nproxies=<PROXIES>: Sets the desired number of proxies in the cluster. Must be at least 1, or set to -1 which restores the number of proxies to the default value.\n\nlogs=<LOGS>: Sets the desired number of log servers in the cluster. Must be at least 1, or set to -1 which restores the number of logs to the default value.\n\nresolvers=<RESOLVERS>: Sets the desired number of resolvers in the cluster. Must be at least 1, or set to -1 which restores the number of resolvers to the default value.\n\nSee the FoundationDB Administration Guide for more information.");
|
||||
"configure [new] "
|
||||
"<single|double|triple|three_data_hall|three_datacenter|ssd|memory|memory-radixtree-beta|proxies=<PROXIES>|grv_"
|
||||
"proxies=<GRV_PROXIES>|logs=<LOGS>|resolvers=<RESOLVERS>>*",
|
||||
"change the database configuration",
|
||||
"The `new' option, if present, initializes a new database with the given configuration rather than changing "
|
||||
"the configuration of an existing one. When used, both a redundancy mode and a storage engine must be "
|
||||
"specified.\n\nRedundancy mode:\n single - one copy of the data. Not fault tolerant.\n double - two copies "
|
||||
"of data (survive one failure).\n triple - three copies of data (survive two failures).\n three_data_hall - "
|
||||
"See the Admin Guide.\n three_datacenter - See the Admin Guide.\n\nStorage engine:\n ssd - B-Tree storage "
|
||||
"engine optimized for solid state disks.\n memory - Durable in-memory storage engine for small "
|
||||
"datasets.\n\nproxies=<PROXIES>: Sets the desired number of proxies in the cluster. Must be at least 1, or set "
|
||||
"to -1 which restores the number of proxies to the default value.\n\ngrv_proxies=<GRV_PROXIES>: Sets the "
|
||||
"desired number of GRV proxies in the cluster. Must be at least 1, or set to -1 which restores the number of "
|
||||
"proxies to the default value.\n\nlogs=<LOGS>: Sets the desired number of log servers in the cluster. Must be "
|
||||
"at least 1, or set to -1 which restores the number of logs to the default value.\n\nresolvers=<RESOLVERS>: "
|
||||
"Sets the desired number of resolvers in the cluster. Must be at least 1, or set to -1 which restores the "
|
||||
"number of resolvers to the default value.\n\nSee the FoundationDB Administration Guide for more information.");
|
||||
helpMap["fileconfigure"] = CommandHelp(
|
||||
"fileconfigure [new] <FILENAME>",
|
||||
"change the database configuration from a file",
|
||||
|
@ -857,7 +871,12 @@ void printStatus(StatusObjectReader statusObj, StatusClient::StatusLevel level,
|
|||
fatalRecoveryState = true;
|
||||
|
||||
if (name == "recruiting_transaction_servers") {
|
||||
description += format("\nNeed at least %d log servers across unique zones, %d proxies and %d resolvers.", recoveryState["required_logs"].get_int(), recoveryState["required_proxies"].get_int(), recoveryState["required_resolvers"].get_int());
|
||||
description += format("\nNeed at least %d log servers across unique zones, %d proxies, "
|
||||
"%d GRV proxies and %d resolvers.",
|
||||
recoveryState["required_logs"].get_int(),
|
||||
recoveryState["required_proxies"].get_int(),
|
||||
recoveryState["required_grv_proxies"].get_int(),
|
||||
recoveryState["required_resolvers"].get_int());
|
||||
if (statusObjCluster.has("machines") && statusObjCluster.has("processes")) {
|
||||
auto numOfNonExcludedProcessesAndZones = getNumOfNonExcludedProcessAndZones(statusObjCluster);
|
||||
description += format("\nHave %d non-excluded processes on %d machines across %d zones.", numOfNonExcludedProcessesAndZones.first, getNumofNonExcludedMachines(statusObjCluster), numOfNonExcludedProcessesAndZones.second);
|
||||
|
@ -881,9 +900,8 @@ void printStatus(StatusObjectReader statusObj, StatusClient::StatusLevel level,
|
|||
}
|
||||
}
|
||||
}
|
||||
} catch (std::runtime_error&) {
|
||||
}
|
||||
catch (std::runtime_error& ){ }
|
||||
|
||||
|
||||
// Check if cluster controllable is reachable
|
||||
try {
|
||||
|
@ -1011,6 +1029,9 @@ void printStatus(StatusObjectReader statusObj, StatusClient::StatusLevel level,
|
|||
if (statusObjConfig.get("proxies", intVal))
|
||||
outputString += format("\n Desired Proxies - %d", intVal);
|
||||
|
||||
if (statusObjConfig.get("grv_proxies", intVal))
|
||||
outputString += format("\n Desired GRV Proxies - %d", intVal);
|
||||
|
||||
if (statusObjConfig.get("resolvers", intVal))
|
||||
outputString += format("\n Desired Resolvers - %d", intVal);
|
||||
|
||||
|
@ -1768,16 +1789,17 @@ ACTOR Future<bool> configure( Database db, std::vector<StringRef> tokens, Refere
|
|||
}
|
||||
|
||||
bool noChanges = conf.get().old_replication == conf.get().auto_replication &&
|
||||
conf.get().old_logs == conf.get().auto_logs &&
|
||||
conf.get().old_proxies == conf.get().auto_proxies &&
|
||||
conf.get().old_resolvers == conf.get().auto_resolvers &&
|
||||
conf.get().old_processes_with_transaction == conf.get().auto_processes_with_transaction &&
|
||||
conf.get().old_machines_with_transaction == conf.get().auto_machines_with_transaction;
|
||||
conf.get().old_logs == conf.get().auto_logs &&
|
||||
conf.get().old_proxies == conf.get().auto_proxies &&
|
||||
conf.get().old_grv_proxies == conf.get().auto_grv_proxies &&
|
||||
conf.get().old_resolvers == conf.get().auto_resolvers &&
|
||||
conf.get().old_processes_with_transaction == conf.get().auto_processes_with_transaction &&
|
||||
conf.get().old_machines_with_transaction == conf.get().auto_machines_with_transaction;
|
||||
|
||||
bool noDesiredChanges = noChanges &&
|
||||
conf.get().old_logs == conf.get().desired_logs &&
|
||||
conf.get().old_proxies == conf.get().desired_proxies &&
|
||||
conf.get().old_resolvers == conf.get().desired_resolvers;
|
||||
bool noDesiredChanges = noChanges && conf.get().old_logs == conf.get().desired_logs &&
|
||||
conf.get().old_proxies == conf.get().desired_proxies &&
|
||||
conf.get().old_grv_proxies == conf.get().desired_grv_proxies &&
|
||||
conf.get().old_resolvers == conf.get().desired_resolvers;
|
||||
|
||||
std::string outputString;
|
||||
|
||||
|
@ -1796,6 +1818,11 @@ ACTOR Future<bool> configure( Database db, std::vector<StringRef> tokens, Refere
|
|||
outputString += conf.get().auto_logs != conf.get().desired_logs ? format(" (manually set; would be %d)\n", conf.get().desired_logs) : "\n";
|
||||
outputString += format("| proxies | %16d | %16d |", conf.get().old_proxies, conf.get().auto_proxies);
|
||||
outputString += conf.get().auto_proxies != conf.get().desired_proxies ? format(" (manually set; would be %d)\n", conf.get().desired_proxies) : "\n";
|
||||
outputString += format("| grv_proxies | %16d | %16d |", conf.get().old_grv_proxies,
|
||||
conf.get().auto_grv_proxies);
|
||||
outputString += conf.get().auto_grv_proxies != conf.get().desired_grv_proxies
|
||||
? format(" (manually set; would be %d)\n", conf.get().desired_grv_proxies)
|
||||
: "\n";
|
||||
outputString += format("| resolvers | %16d | %16d |", conf.get().old_resolvers, conf.get().auto_resolvers);
|
||||
outputString += conf.get().auto_resolvers != conf.get().desired_resolvers ? format(" (manually set; would be %d)\n", conf.get().desired_resolvers) : "\n";
|
||||
outputString += format("| transaction-class processes | %16d | %16d |\n", conf.get().old_processes_with_transaction, conf.get().auto_processes_with_transaction);
|
||||
|
@ -2504,7 +2531,11 @@ void onOffGenerator(const char* text, const char *line, std::vector<std::string>
|
|||
}
|
||||
|
||||
void configureGenerator(const char* text, const char *line, std::vector<std::string>& lc) {
|
||||
const char* opts[] = {"new", "single", "double", "triple", "three_data_hall", "three_datacenter", "ssd", "ssd-1", "ssd-2", "memory", "memory-1", "memory-2", "memory-radixtree-beta", "proxies=", "logs=", "resolvers=", nullptr};
|
||||
const char* opts[] = {
|
||||
"new", "single", "double", "triple", "three_data_hall", "three_datacenter", "ssd",
|
||||
"ssd-1", "ssd-2", "memory", "memory-1", "memory-2", "memory-radixtree-beta", "proxies=",
|
||||
"grv_proxies=", "logs=", "resolvers=", nullptr
|
||||
};
|
||||
arrayGenerator(text, line, opts, lc);
|
||||
}
|
||||
|
||||
|
@ -2541,6 +2572,16 @@ void throttleGenerator(const char* text, const char *line, std::vector<std::stri
|
|||
const char* opts[] = { "auto", nullptr };
|
||||
arrayGenerator(text, line, opts, lc);
|
||||
}
|
||||
else if(tokens.size() >= 2 && tokencmp(tokens[1], "list")) {
|
||||
if(tokens.size() == 2) {
|
||||
const char* opts[] = { "throttled", "recommended", "all", nullptr };
|
||||
arrayGenerator(text, line, opts, lc);
|
||||
}
|
||||
else if(tokens.size() == 3) {
|
||||
const char* opts[] = {"LIMITS", nullptr};
|
||||
arrayGenerator(text, line, opts, lc);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void fdbcliCompCmd(std::string const& text, std::vector<std::string>& lc) {
|
||||
|
@ -2661,6 +2702,14 @@ std::vector<const char*> throttleHintGenerator(std::vector<StringRef> const& tok
|
|||
else if((tokencmp(tokens[1], "enable") || tokencmp(tokens[1], "disable")) && tokens.size() == 2) {
|
||||
return { "auto" };
|
||||
}
|
||||
else if(tokens.size() >= 2 && tokencmp(tokens[1], "list")) {
|
||||
if(tokens.size() == 2) {
|
||||
return { "[throttled|recommended|all]", "[LIMITS]" };
|
||||
}
|
||||
else if(tokens.size() == 3 && (tokencmp(tokens[2], "throttled") || tokencmp(tokens[2], "recommended") || tokencmp(tokens[2], "all"))){
|
||||
return {"[LIMITS]"};
|
||||
}
|
||||
}
|
||||
else if(tokens.size() == 2 && inArgument) {
|
||||
return { "[ARGS]" };
|
||||
}
|
||||
|
@ -4077,8 +4126,8 @@ ACTOR Future<int> cli(CLIOptions opt, LineNoise* plinenoise) {
|
|||
continue;
|
||||
}
|
||||
else if(tokencmp(tokens[1], "list")) {
|
||||
if(tokens.size() > 3) {
|
||||
printf("Usage: throttle list [LIMIT]\n");
|
||||
if(tokens.size() > 4) {
|
||||
printf("Usage: throttle list [throttled|recommended|all] [LIMIT]\n");
|
||||
printf("\n");
|
||||
printf("Lists tags that are currently throttled.\n");
|
||||
printf("The default LIMIT is 100 tags.\n");
|
||||
|
@ -4086,36 +4135,72 @@ ACTOR Future<int> cli(CLIOptions opt, LineNoise* plinenoise) {
|
|||
continue;
|
||||
}
|
||||
|
||||
state int throttleListLimit = 100;
|
||||
state bool reportThrottled = true;
|
||||
state bool reportRecommended = false;
|
||||
if(tokens.size() >= 3) {
|
||||
char *end;
|
||||
throttleListLimit = std::strtol((const char*)tokens[2].begin(), &end, 10);
|
||||
if ((tokens.size() > 3 && !std::isspace(*end)) || (tokens.size() == 3 && *end != '\0')) {
|
||||
printf("ERROR: failed to parse limit `%s'.\n", printable(tokens[2]).c_str());
|
||||
if(tokencmp(tokens[2], "recommended")) {
|
||||
reportThrottled = false; reportRecommended = true;
|
||||
}
|
||||
else if(tokencmp(tokens[2], "all")){
|
||||
reportThrottled = true; reportRecommended = true;
|
||||
}
|
||||
else if(!tokencmp(tokens[2], "throttled")){
|
||||
printf("ERROR: failed to parse `%s'.\n", printable(tokens[2]).c_str());
|
||||
is_error = true;
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
std::vector<TagThrottleInfo> tags = wait(ThrottleApi::getThrottledTags(db, throttleListLimit));
|
||||
state int throttleListLimit = 100;
|
||||
if(tokens.size() >= 4) {
|
||||
char *end;
|
||||
throttleListLimit = std::strtol((const char*)tokens[3].begin(), &end, 10);
|
||||
if ((tokens.size() > 4 && !std::isspace(*end)) || (tokens.size() == 4 && *end != '\0')) {
|
||||
printf("ERROR: failed to parse limit `%s'.\n", printable(tokens[3]).c_str());
|
||||
is_error = true;
|
||||
continue;
|
||||
}
|
||||
}
|
||||
|
||||
state std::vector<TagThrottleInfo> tags;
|
||||
if(reportThrottled && reportRecommended) {
|
||||
wait(store(tags, ThrottleApi::getThrottledTags(db, throttleListLimit, true)));
|
||||
}
|
||||
else if(reportThrottled) {
|
||||
wait(store(tags, ThrottleApi::getThrottledTags(db, throttleListLimit)));
|
||||
}
|
||||
else if(reportRecommended) {
|
||||
wait(store(tags, ThrottleApi::getRecommendedTags(db, throttleListLimit)));
|
||||
}
|
||||
|
||||
bool anyLogged = false;
|
||||
for(auto itr = tags.begin(); itr != tags.end(); ++itr) {
|
||||
if(itr->expirationTime > now()) {
|
||||
if(!anyLogged) {
|
||||
printf("Throttled tags:\n\n");
|
||||
printf(" Rate (txn/s) | Expiration (s) | Priority | Type | Tag\n");
|
||||
printf(" --------------+----------------+-----------+--------+------------------\n");
|
||||
printf(" Rate (txn/s) | Expiration (s) | Priority | Type | Reason |Tag\n");
|
||||
printf(" --------------+----------------+-----------+--------+------------+------\n");
|
||||
|
||||
anyLogged = true;
|
||||
}
|
||||
|
||||
printf(" %12d | %13ds | %9s | %6s | %s\n",
|
||||
(int)(itr->tpsRate),
|
||||
std::min((int)(itr->expirationTime-now()), (int)(itr->initialDuration)),
|
||||
transactionPriorityToString(itr->priority, false),
|
||||
itr->throttleType == TagThrottleType::AUTO ? "auto" : "manual",
|
||||
itr->tag.toString().c_str());
|
||||
std::string reasonStr = "unset";
|
||||
if(itr->reason == TagThrottledReason::MANUAL){
|
||||
reasonStr = "manual";
|
||||
}
|
||||
else if(itr->reason == TagThrottledReason::BUSY_WRITE) {
|
||||
reasonStr = "busy write";
|
||||
}
|
||||
else if(itr->reason == TagThrottledReason::BUSY_READ) {
|
||||
reasonStr = "busy read";
|
||||
}
|
||||
|
||||
printf(" %12d | %13ds | %9s | %6s | %10s |%s\n", (int)(itr->tpsRate),
|
||||
std::min((int)(itr->expirationTime - now()), (int)(itr->initialDuration)),
|
||||
transactionPriorityToString(itr->priority, false),
|
||||
itr->throttleType == TagThrottleType::AUTO ? "auto" : "manual",
|
||||
reasonStr.c_str(),
|
||||
itr->tag.toString().c_str());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -4124,7 +4209,7 @@ ACTOR Future<int> cli(CLIOptions opt, LineNoise* plinenoise) {
|
|||
printf("Usage: throttle list [LIMIT]\n");
|
||||
}
|
||||
if(!anyLogged) {
|
||||
printf("There are no throttled tags\n");
|
||||
printf("There are no %s tags\n", reportThrottled ? "throttled" : "recommended");
|
||||
}
|
||||
}
|
||||
else if(tokencmp(tokens[1], "on")) {
|
||||
|
|
|
@ -23,7 +23,7 @@
|
|||
#include "flow/UnitTest.h"
|
||||
#include "flow/actorcompiler.h" // has to be last include
|
||||
|
||||
Future<int64_t> AsyncFileBlobStoreRead::size() {
|
||||
Future<int64_t> AsyncFileBlobStoreRead::size() const {
|
||||
if(!m_size.isValid())
|
||||
m_size = m_bstore->objectSize(m_bucket, m_object);
|
||||
return m_size;
|
||||
|
|
|
@ -92,7 +92,7 @@ public:
|
|||
MD5_CTX content_md5_buf;
|
||||
};
|
||||
|
||||
virtual Future<int> read( void *data, int length, int64_t offset ) { throw file_not_readable(); }
|
||||
Future<int> read(void* data, int length, int64_t offset) override { throw file_not_readable(); }
|
||||
|
||||
ACTOR static Future<Void> write_impl(Reference<AsyncFileBlobStoreWrite> f, const uint8_t *data, int length) {
|
||||
state Part *p = f->m_parts.back().getPtr();
|
||||
|
@ -115,7 +115,7 @@ public:
|
|||
return Void();
|
||||
}
|
||||
|
||||
virtual Future<Void> write( void const *data, int length, int64_t offset ) {
|
||||
Future<Void> write(void const* data, int length, int64_t offset) override {
|
||||
if(offset != m_cursor)
|
||||
throw non_sequential_op();
|
||||
m_cursor += length;
|
||||
|
@ -123,7 +123,7 @@ public:
|
|||
return m_error.getFuture() || write_impl(Reference<AsyncFileBlobStoreWrite>::addRef(this), (const uint8_t *)data, length);
|
||||
}
|
||||
|
||||
virtual Future<Void> truncate( int64_t size ) {
|
||||
Future<Void> truncate(int64_t size) override {
|
||||
if(size != m_cursor)
|
||||
return non_sequential_op();
|
||||
return Void();
|
||||
|
@ -165,7 +165,7 @@ public:
|
|||
}
|
||||
|
||||
// Ready once all data has been sent AND acknowledged from the remote side
|
||||
virtual Future<Void> sync() {
|
||||
Future<Void> sync() override {
|
||||
// Only initiate the finish operation once, and also prevent further writing.
|
||||
if(!m_finished.isValid()) {
|
||||
m_finished = doFinishUpload(this);
|
||||
|
@ -182,25 +182,25 @@ public:
|
|||
// their size. So in the case of a write buffer that does not meet the part minimum size the part could be sent
|
||||
// but then if there is any more data written then that part needs to be sent again in its entirety. So a client
|
||||
// that calls flush often could generate far more blob store write traffic than they intend to.
|
||||
virtual Future<Void> flush() { return Void(); }
|
||||
Future<Void> flush() override { return Void(); }
|
||||
|
||||
virtual Future<int64_t> size() { return m_cursor; }
|
||||
Future<int64_t> size() const override { return m_cursor; }
|
||||
|
||||
virtual Future<Void> readZeroCopy( void** data, int* length, int64_t offset ) {
|
||||
Future<Void> readZeroCopy(void** data, int* length, int64_t offset) override {
|
||||
TraceEvent(SevError, "ReadZeroCopyNotSupported").detail("FileType", "BlobStoreWrite");
|
||||
return platform_error();
|
||||
}
|
||||
virtual void releaseZeroCopy( void* data, int length, int64_t offset ) {}
|
||||
void releaseZeroCopy(void* data, int length, int64_t offset) override {}
|
||||
|
||||
virtual int64_t debugFD() { return -1; }
|
||||
int64_t debugFD() const override { return -1; }
|
||||
|
||||
virtual ~AsyncFileBlobStoreWrite() {
|
||||
~AsyncFileBlobStoreWrite() override {
|
||||
m_upload_id.cancel();
|
||||
m_finished.cancel();
|
||||
m_parts.clear(); // Contains futures
|
||||
}
|
||||
|
||||
virtual std::string getFilename() { return m_object; }
|
||||
std::string getFilename() const override { return m_object; }
|
||||
|
||||
private:
|
||||
Reference<BlobStoreEndpoint> m_bstore;
|
||||
|
@ -259,32 +259,32 @@ public:
|
|||
virtual void addref() { ReferenceCounted<AsyncFileBlobStoreRead>::addref(); }
|
||||
virtual void delref() { ReferenceCounted<AsyncFileBlobStoreRead>::delref(); }
|
||||
|
||||
virtual Future<int> read( void *data, int length, int64_t offset );
|
||||
Future<int> read(void* data, int length, int64_t offset) override;
|
||||
|
||||
virtual Future<Void> write( void const *data, int length, int64_t offset ) { throw file_not_writable(); }
|
||||
virtual Future<Void> truncate( int64_t size ) { throw file_not_writable(); }
|
||||
Future<Void> write(void const* data, int length, int64_t offset) override { throw file_not_writable(); }
|
||||
Future<Void> truncate(int64_t size) override { throw file_not_writable(); }
|
||||
|
||||
virtual Future<Void> sync() { return Void(); }
|
||||
virtual Future<Void> flush() { return Void(); }
|
||||
Future<Void> sync() override { return Void(); }
|
||||
Future<Void> flush() override { return Void(); }
|
||||
|
||||
virtual Future<int64_t> size();
|
||||
Future<int64_t> size() const override;
|
||||
|
||||
virtual Future<Void> readZeroCopy( void** data, int* length, int64_t offset ) {
|
||||
Future<Void> readZeroCopy(void** data, int* length, int64_t offset) override {
|
||||
TraceEvent(SevError, "ReadZeroCopyNotSupported").detail("FileType", "BlobStoreRead");
|
||||
return platform_error();
|
||||
}
|
||||
virtual void releaseZeroCopy( void* data, int length, int64_t offset ) {}
|
||||
void releaseZeroCopy(void* data, int length, int64_t offset) override {}
|
||||
|
||||
virtual int64_t debugFD() { return -1; }
|
||||
int64_t debugFD() const override { return -1; }
|
||||
|
||||
virtual std::string getFilename() { return m_object; }
|
||||
std::string getFilename() const override { return m_object; }
|
||||
|
||||
virtual ~AsyncFileBlobStoreRead() {}
|
||||
|
||||
Reference<BlobStoreEndpoint> m_bstore;
|
||||
std::string m_bucket;
|
||||
std::string m_object;
|
||||
Future<int64_t> m_size;
|
||||
mutable Future<int64_t> m_size;
|
||||
|
||||
AsyncFileBlobStoreRead(Reference<BlobStoreEndpoint> bstore, std::string bucket, std::string object)
|
||||
: m_bstore(bstore), m_bucket(bucket), m_object(object) {
|
||||
|
|
|
@ -286,11 +286,19 @@ public:
|
|||
// - submit a restore on the given tagName
|
||||
// - Optionally wait for the restore's completion. Will restore_error if restore fails or is aborted.
|
||||
// restore() will return the targetVersion which will be either the valid version passed in or the max restorable version for the given url.
|
||||
Future<Version> restore(Database cx, Optional<Database> cxOrig, Key tagName, Key url, Standalone<VectorRef<KeyRangeRef>> ranges, bool waitForComplete = true, Version targetVersion = -1, bool verbose = true, Key addPrefix = Key(), Key removePrefix = Key(), bool lockDB = true);
|
||||
Future<Version> restore(Database cx, Optional<Database> cxOrig, Key tagName, Key url, bool waitForComplete = true, Version targetVersion = -1, bool verbose = true, KeyRange range = normalKeys, Key addPrefix = Key(), Key removePrefix = Key(), bool lockDB = true) {
|
||||
Future<Version> restore(Database cx, Optional<Database> cxOrig, Key tagName, Key url,
|
||||
Standalone<VectorRef<KeyRangeRef>> ranges, bool waitForComplete = true,
|
||||
Version targetVersion = -1, bool verbose = true, Key addPrefix = Key(),
|
||||
Key removePrefix = Key(), bool lockDB = true, bool incrementalBackupOnly = false,
|
||||
Version beginVersion = -1);
|
||||
Future<Version> restore(Database cx, Optional<Database> cxOrig, Key tagName, Key url, bool waitForComplete = true,
|
||||
Version targetVersion = -1, bool verbose = true, KeyRange range = normalKeys,
|
||||
Key addPrefix = Key(), Key removePrefix = Key(), bool lockDB = true,
|
||||
bool incrementalBackupOnly = false, Version beginVersion = -1) {
|
||||
Standalone<VectorRef<KeyRangeRef>> rangeRef;
|
||||
rangeRef.push_back_deep(rangeRef.arena(), range);
|
||||
return restore(cx, cxOrig, tagName, url, rangeRef, waitForComplete, targetVersion, verbose, addPrefix, removePrefix, lockDB);
|
||||
return restore(cx, cxOrig, tagName, url, rangeRef, waitForComplete, targetVersion, verbose, addPrefix,
|
||||
removePrefix, lockDB, incrementalBackupOnly, beginVersion);
|
||||
}
|
||||
Future<Version> atomicRestore(Database cx, Key tagName, Standalone<VectorRef<KeyRangeRef>> ranges, Key addPrefix = Key(), Key removePrefix = Key());
|
||||
Future<Version> atomicRestore(Database cx, Key tagName, KeyRange range = normalKeys, Key addPrefix = Key(), Key removePrefix = Key()) {
|
||||
|
@ -315,13 +323,14 @@ public:
|
|||
|
||||
Future<Void> submitBackup(Reference<ReadYourWritesTransaction> tr, Key outContainer, int snapshotIntervalSeconds,
|
||||
std::string tagName, Standalone<VectorRef<KeyRangeRef>> backupRanges,
|
||||
bool stopWhenDone = true, bool partitionedLog = false);
|
||||
bool stopWhenDone = true, bool partitionedLog = false,
|
||||
bool incrementalBackupOnly = false);
|
||||
Future<Void> submitBackup(Database cx, Key outContainer, int snapshotIntervalSeconds, std::string tagName,
|
||||
Standalone<VectorRef<KeyRangeRef>> backupRanges, bool stopWhenDone = true,
|
||||
bool partitionedLog = false) {
|
||||
bool partitionedLog = false, bool incrementalBackupOnly = false) {
|
||||
return runRYWTransactionFailIfLocked(cx, [=](Reference<ReadYourWritesTransaction> tr) {
|
||||
return submitBackup(tr, outContainer, snapshotIntervalSeconds, tagName, backupRanges, stopWhenDone,
|
||||
partitionedLog);
|
||||
partitionedLog, incrementalBackupOnly);
|
||||
});
|
||||
}
|
||||
|
||||
|
@ -810,6 +819,11 @@ public:
|
|||
return configSpace.pack(LiteralStringRef(__FUNCTION__));
|
||||
}
|
||||
|
||||
// Set to true if only requesting incremental backup without base snapshot.
|
||||
KeyBackedProperty<bool> incrementalBackupOnly() {
|
||||
return configSpace.pack(LiteralStringRef(__FUNCTION__));
|
||||
}
|
||||
|
||||
// Latest version for which all prior versions have saved by backup workers.
|
||||
KeyBackedProperty<Version> latestBackupWorkerSavedVersion() {
|
||||
return configSpace.pack(LiteralStringRef(__FUNCTION__));
|
||||
|
@ -847,17 +861,25 @@ public:
|
|||
auto workerEnabled = backupWorkerEnabled().get(tr);
|
||||
auto plogEnabled = partitionedLogEnabled().get(tr);
|
||||
auto workerVersion = latestBackupWorkerSavedVersion().get(tr);
|
||||
return map(success(lastLog) && success(firstSnapshot) && success(workerEnabled) && success(plogEnabled) && success(workerVersion), [=](Void) -> Optional<Version> {
|
||||
// The latest log greater than the oldest snapshot is the restorable version
|
||||
Optional<Version> logVersion = workerEnabled.get().present() && workerEnabled.get().get() &&
|
||||
plogEnabled.get().present() && plogEnabled.get().get()
|
||||
? workerVersion.get()
|
||||
: lastLog.get();
|
||||
if (logVersion.present() && firstSnapshot.get().present() && logVersion.get() > firstSnapshot.get().get()) {
|
||||
return std::max(logVersion.get() - 1, firstSnapshot.get().get());
|
||||
}
|
||||
return {};
|
||||
});
|
||||
auto incrementalBackup = incrementalBackupOnly().get(tr);
|
||||
return map(success(lastLog) && success(firstSnapshot) && success(workerEnabled) && success(plogEnabled) &&
|
||||
success(workerVersion) && success(incrementalBackup),
|
||||
[=](Void) -> Optional<Version> {
|
||||
// The latest log greater than the oldest snapshot is the restorable version
|
||||
Optional<Version> logVersion = workerEnabled.get().present() && workerEnabled.get().get() &&
|
||||
plogEnabled.get().present() && plogEnabled.get().get()
|
||||
? workerVersion.get()
|
||||
: lastLog.get();
|
||||
if (logVersion.present() && firstSnapshot.get().present() &&
|
||||
logVersion.get() > firstSnapshot.get().get()) {
|
||||
return std::max(logVersion.get() - 1, firstSnapshot.get().get());
|
||||
}
|
||||
if (logVersion.present() && incrementalBackup.isReady() && incrementalBackup.get().present() &&
|
||||
incrementalBackup.get().get()) {
|
||||
return logVersion.get() - 1;
|
||||
}
|
||||
return {};
|
||||
});
|
||||
}
|
||||
|
||||
KeyBackedProperty<std::vector<KeyRange>> backupRanges() {
|
||||
|
|
|
@ -27,6 +27,7 @@
|
|||
#include "flow/UnitTest.h"
|
||||
#include "flow/Hash3.h"
|
||||
#include "fdbrpc/AsyncFileReadAhead.actor.h"
|
||||
#include "fdbrpc/simulator.h"
|
||||
#include "flow/Platform.h"
|
||||
#include "fdbclient/AsyncFileBlobStore.actor.h"
|
||||
#include "fdbclient/Status.h"
|
||||
|
@ -1364,7 +1365,32 @@ public:
|
|||
return getSnapshotFileKeyRange_impl(Reference<BackupContainerFileSystem>::addRef(this), file);
|
||||
}
|
||||
|
||||
ACTOR static Future<Optional<RestorableFileSet>> getRestoreSet_impl(Reference<BackupContainerFileSystem> bc, Version targetVersion) {
|
||||
static Optional<RestorableFileSet> getRestoreSetFromLogs(std::vector<LogFile> logs, Version targetVersion,
|
||||
RestorableFileSet restorable) {
|
||||
Version end = logs.begin()->endVersion;
|
||||
computeRestoreEndVersion(logs, &restorable.logs, &end, targetVersion);
|
||||
if (end >= targetVersion) {
|
||||
restorable.continuousBeginVersion = logs.begin()->beginVersion;
|
||||
restorable.continuousEndVersion = end;
|
||||
return Optional<RestorableFileSet>(restorable);
|
||||
}
|
||||
return Optional<RestorableFileSet>();
|
||||
}
|
||||
|
||||
ACTOR static Future<Optional<RestorableFileSet>> getRestoreSet_impl(Reference<BackupContainerFileSystem> bc,
|
||||
Version targetVersion, bool logsOnly,
|
||||
Version beginVersion) {
|
||||
if (logsOnly) {
|
||||
state RestorableFileSet restorableSet;
|
||||
state std::vector<LogFile> logFiles;
|
||||
Version begin = beginVersion == invalidVersion ? 0 : beginVersion;
|
||||
wait(store(logFiles, bc->listLogFiles(begin, targetVersion, false)));
|
||||
// List logs in version order so log continuity can be analyzed
|
||||
std::sort(logFiles.begin(), logFiles.end());
|
||||
if (!logFiles.empty()) {
|
||||
return getRestoreSetFromLogs(logFiles, targetVersion, restorableSet);
|
||||
}
|
||||
}
|
||||
// Find the most recent keyrange snapshot to end at or before targetVersion
|
||||
state Optional<KeyspaceSnapshotFile> snapshot;
|
||||
std::vector<KeyspaceSnapshotFile> snapshots = wait(bc->listKeyspaceSnapshots());
|
||||
|
@ -1435,21 +1461,17 @@ public:
|
|||
|
||||
// If there are logs and the first one starts at or before the snapshot begin version then proceed
|
||||
if(!logs.empty() && logs.front().beginVersion <= snapshot.get().beginVersion) {
|
||||
Version end = logs.begin()->endVersion;
|
||||
computeRestoreEndVersion(logs, &restorable.logs, &end, targetVersion);
|
||||
if (end >= targetVersion) {
|
||||
restorable.continuousBeginVersion = logs.begin()->beginVersion;
|
||||
restorable.continuousEndVersion = end;
|
||||
return Optional<RestorableFileSet>(restorable);
|
||||
}
|
||||
return getRestoreSetFromLogs(logs, targetVersion, restorable);
|
||||
}
|
||||
}
|
||||
|
||||
return Optional<RestorableFileSet>();
|
||||
}
|
||||
|
||||
Future<Optional<RestorableFileSet>> getRestoreSet(Version targetVersion) final {
|
||||
return getRestoreSet_impl(Reference<BackupContainerFileSystem>::addRef(this), targetVersion);
|
||||
Future<Optional<RestorableFileSet>> getRestoreSet(Version targetVersion, bool logsOnly,
|
||||
Version beginVersion) final {
|
||||
return getRestoreSet_impl(Reference<BackupContainerFileSystem>::addRef(this), targetVersion, logsOnly,
|
||||
beginVersion);
|
||||
}
|
||||
|
||||
private:
|
||||
|
@ -1614,9 +1636,14 @@ public:
|
|||
std::string fullPath = joinPath(m_path, path);
|
||||
#ifndef _WIN32
|
||||
if(g_network->isSimulated()) {
|
||||
if(!fileExists(fullPath))
|
||||
if(!fileExists(fullPath)) {
|
||||
throw file_not_found();
|
||||
std::string uniquePath = fullPath + "." + deterministicRandom()->randomUniqueID().toString() + ".lnk";
|
||||
}
|
||||
|
||||
if (g_simulator.getCurrentProcess()->uid == UID()) {
|
||||
TraceEvent(SevError, "BackupContainerReadFileOnUnsetProcessID");
|
||||
}
|
||||
std::string uniquePath = fullPath + "." + g_simulator.getCurrentProcess()->uid.toString() + ".lnk";
|
||||
unlink(uniquePath.c_str());
|
||||
ASSERT(symlink(basename(path).c_str(), uniquePath.c_str()) == 0);
|
||||
fullPath = uniquePath;
|
||||
|
|
|
@ -282,7 +282,8 @@ public:
|
|||
|
||||
// Get exactly the files necessary to restore to targetVersion. Returns non-present if
|
||||
// restore to given version is not possible.
|
||||
virtual Future<Optional<RestorableFileSet>> getRestoreSet(Version targetVersion) = 0;
|
||||
virtual Future<Optional<RestorableFileSet>> getRestoreSet(Version targetVersion, bool logsOnly = false,
|
||||
Version beginVersion = -1) = 0;
|
||||
|
||||
// Get an IBackupContainer based on a container spec string
|
||||
static Reference<IBackupContainer> openContainer(std::string url);
|
||||
|
|
|
@ -21,6 +21,7 @@ set(FDBCLIENT_SRCS
|
|||
FDBOptions.h
|
||||
FDBTypes.h
|
||||
FileBackupAgent.actor.cpp
|
||||
GrvProxyInterface.h
|
||||
HTTP.actor.cpp
|
||||
IClientApi.h
|
||||
JsonBuilder.cpp
|
||||
|
|
|
@ -29,11 +29,13 @@ DatabaseConfiguration::DatabaseConfiguration()
|
|||
void DatabaseConfiguration::resetInternal() {
|
||||
// does NOT reset rawConfiguration
|
||||
initialized = false;
|
||||
masterProxyCount = resolverCount = desiredTLogCount = tLogWriteAntiQuorum = tLogReplicationFactor = storageTeamSize = desiredLogRouterCount = -1;
|
||||
proxyCount = grvProxyCount = resolverCount = desiredTLogCount = tLogWriteAntiQuorum = tLogReplicationFactor =
|
||||
storageTeamSize = desiredLogRouterCount = -1;
|
||||
tLogVersion = TLogVersion::DEFAULT;
|
||||
tLogDataStoreType = storageServerStoreType = KeyValueStoreType::END;
|
||||
tLogSpillType = TLogSpillType::DEFAULT;
|
||||
autoMasterProxyCount = CLIENT_KNOBS->DEFAULT_AUTO_PROXIES;
|
||||
autoProxyCount = CLIENT_KNOBS->DEFAULT_AUTO_PROXIES;
|
||||
autoGrvProxyCount = CLIENT_KNOBS->DEFAULT_AUTO_GRV_PROXIES;
|
||||
autoResolverCount = CLIENT_KNOBS->DEFAULT_AUTO_RESOLVERS;
|
||||
autoDesiredTLogCount = CLIENT_KNOBS->DEFAULT_AUTO_LOGS;
|
||||
usableRegions = 1;
|
||||
|
@ -168,6 +170,7 @@ bool DatabaseConfiguration::isValid() const {
|
|||
tLogReplicationFactor >= 1 &&
|
||||
storageTeamSize >= 1 &&
|
||||
getDesiredProxies() >= 1 &&
|
||||
getDesiredGrvProxies() >= 1 &&
|
||||
getDesiredLogs() >= 1 &&
|
||||
getDesiredResolvers() >= 1 &&
|
||||
tLogVersion != TLogVersion::UNSET &&
|
||||
|
@ -177,7 +180,8 @@ bool DatabaseConfiguration::isValid() const {
|
|||
tLogSpillType != TLogSpillType::UNSET &&
|
||||
!(tLogSpillType == TLogSpillType::REFERENCE && tLogVersion < TLogVersion::V3) &&
|
||||
storageServerStoreType != KeyValueStoreType::END &&
|
||||
autoMasterProxyCount >= 1 &&
|
||||
autoProxyCount >= 1 &&
|
||||
autoGrvProxyCount >= 1 &&
|
||||
autoResolverCount >= 1 &&
|
||||
autoDesiredTLogCount >= 1 &&
|
||||
storagePolicy &&
|
||||
|
@ -314,8 +318,11 @@ StatusObject DatabaseConfiguration::toJSON(bool noPolicies) const {
|
|||
if (desiredTLogCount != -1 || isOverridden("logs")) {
|
||||
result["logs"] = desiredTLogCount;
|
||||
}
|
||||
if (masterProxyCount != -1 || isOverridden("proxies")) {
|
||||
result["proxies"] = masterProxyCount;
|
||||
if (proxyCount != -1 || isOverridden("proxies")) {
|
||||
result["proxies"] = proxyCount;
|
||||
}
|
||||
if (grvProxyCount != -1 || isOverridden("grv_proxies")) {
|
||||
result["grv_proxies"] = proxyCount;
|
||||
}
|
||||
if (resolverCount != -1 || isOverridden("resolvers")) {
|
||||
result["resolvers"] = resolverCount;
|
||||
|
@ -329,8 +336,11 @@ StatusObject DatabaseConfiguration::toJSON(bool noPolicies) const {
|
|||
if (repopulateRegionAntiQuorum != 0 || isOverridden("repopulate_anti_quorum")) {
|
||||
result["repopulate_anti_quorum"] = repopulateRegionAntiQuorum;
|
||||
}
|
||||
if (autoMasterProxyCount != CLIENT_KNOBS->DEFAULT_AUTO_PROXIES || isOverridden("auto_proxies")) {
|
||||
result["auto_proxies"] = autoMasterProxyCount;
|
||||
if (autoProxyCount != CLIENT_KNOBS->DEFAULT_AUTO_PROXIES || isOverridden("auto_proxies")) {
|
||||
result["auto_proxies"] = autoProxyCount;
|
||||
}
|
||||
if (autoGrvProxyCount != CLIENT_KNOBS->DEFAULT_AUTO_GRV_PROXIES || isOverridden("auto_grv_proxies")) {
|
||||
result["auto_grv_proxies"] = autoGrvProxyCount;
|
||||
}
|
||||
if (autoResolverCount != CLIENT_KNOBS->DEFAULT_AUTO_RESOLVERS || isOverridden("auto_resolvers")) {
|
||||
result["auto_resolvers"] = autoResolverCount;
|
||||
|
@ -407,28 +417,32 @@ bool DatabaseConfiguration::setInternal(KeyRef key, ValueRef value) {
|
|||
KeyRef ck = key.removePrefix( configKeysPrefix );
|
||||
int type;
|
||||
|
||||
if (ck == LiteralStringRef("initialized")) initialized = true;
|
||||
else if (ck == LiteralStringRef("proxies")) parse(&masterProxyCount, value);
|
||||
else if (ck == LiteralStringRef("resolvers")) parse(&resolverCount, value);
|
||||
else if (ck == LiteralStringRef("logs")) parse(&desiredTLogCount, value);
|
||||
else if (ck == LiteralStringRef("log_replicas")) {
|
||||
if (ck == LiteralStringRef("initialized")) {
|
||||
initialized = true;
|
||||
} else if (ck == LiteralStringRef("proxies")) {
|
||||
parse(&proxyCount, value);
|
||||
} else if (ck == LiteralStringRef("grv_proxies")) {
|
||||
parse(&grvProxyCount, value);
|
||||
} else if (ck == LiteralStringRef("resolvers")) {
|
||||
parse(&resolverCount, value);
|
||||
} else if (ck == LiteralStringRef("logs")) {
|
||||
parse(&desiredTLogCount, value);
|
||||
} else if (ck == LiteralStringRef("log_replicas")) {
|
||||
parse(&tLogReplicationFactor, value);
|
||||
tLogWriteAntiQuorum = std::min(tLogWriteAntiQuorum, tLogReplicationFactor/2);
|
||||
}
|
||||
else if (ck == LiteralStringRef("log_anti_quorum")) {
|
||||
} else if (ck == LiteralStringRef("log_anti_quorum")) {
|
||||
parse(&tLogWriteAntiQuorum, value);
|
||||
if(tLogReplicationFactor > 0) {
|
||||
tLogWriteAntiQuorum = std::min(tLogWriteAntiQuorum, tLogReplicationFactor/2);
|
||||
tLogWriteAntiQuorum = std::min(tLogWriteAntiQuorum, tLogReplicationFactor / 2);
|
||||
}
|
||||
}
|
||||
else if (ck == LiteralStringRef("storage_replicas")) parse(&storageTeamSize, value);
|
||||
else if (ck == LiteralStringRef("log_version")) {
|
||||
} else if (ck == LiteralStringRef("storage_replicas")) {
|
||||
parse(&storageTeamSize, value);
|
||||
} else if (ck == LiteralStringRef("log_version")) {
|
||||
parse((&type), value);
|
||||
type = std::max((int)TLogVersion::MIN_RECRUITABLE, type);
|
||||
type = std::min((int)TLogVersion::MAX_SUPPORTED, type);
|
||||
tLogVersion = (TLogVersion::Version)type;
|
||||
}
|
||||
else if (ck == LiteralStringRef("log_engine")) {
|
||||
} else if (ck == LiteralStringRef("log_engine")) {
|
||||
parse((&type), value);
|
||||
tLogDataStoreType = (KeyValueStoreType::StoreType)type;
|
||||
// TODO: Remove this once Redwood works as a log engine
|
||||
|
@ -439,23 +453,44 @@ bool DatabaseConfiguration::setInternal(KeyRef key, ValueRef value) {
|
|||
if(tLogDataStoreType == KeyValueStoreType::MEMORY_RADIXTREE) {
|
||||
tLogDataStoreType = KeyValueStoreType::SSD_BTREE_V2;
|
||||
}
|
||||
} else if (ck == LiteralStringRef("log_spill")) {
|
||||
parse((&type), value);
|
||||
tLogSpillType = (TLogSpillType::SpillType)type;
|
||||
} else if (ck == LiteralStringRef("storage_engine")) {
|
||||
parse((&type), value);
|
||||
storageServerStoreType = (KeyValueStoreType::StoreType)type;
|
||||
} else if (ck == LiteralStringRef("auto_proxies")) {
|
||||
parse(&autoProxyCount, value);
|
||||
} else if (ck == LiteralStringRef("auto_grv_proxies")) {
|
||||
parse(&autoGrvProxyCount, value);
|
||||
} else if (ck == LiteralStringRef("auto_resolvers")) {
|
||||
parse(&autoResolverCount, value);
|
||||
} else if (ck == LiteralStringRef("auto_logs")) {
|
||||
parse(&autoDesiredTLogCount, value);
|
||||
} else if (ck == LiteralStringRef("storage_replication_policy")) {
|
||||
parseReplicationPolicy(&storagePolicy, value);
|
||||
} else if (ck == LiteralStringRef("log_replication_policy")) {
|
||||
parseReplicationPolicy(&tLogPolicy, value);
|
||||
} else if (ck == LiteralStringRef("log_routers")) {
|
||||
parse(&desiredLogRouterCount, value);
|
||||
} else if (ck == LiteralStringRef("remote_logs")) {
|
||||
parse(&remoteDesiredTLogCount, value);
|
||||
} else if (ck == LiteralStringRef("remote_log_replicas")) {
|
||||
parse(&remoteTLogReplicationFactor, value);
|
||||
} else if (ck == LiteralStringRef("remote_log_policy")) {
|
||||
parseReplicationPolicy(&remoteTLogPolicy, value);
|
||||
} else if (ck == LiteralStringRef("backup_worker_enabled")) {
|
||||
parse((&type), value);
|
||||
backupWorkerEnabled = (type != 0);
|
||||
} else if (ck == LiteralStringRef("usable_regions")) {
|
||||
parse(&usableRegions, value);
|
||||
} else if (ck == LiteralStringRef("repopulate_anti_quorum")) {
|
||||
parse(&repopulateRegionAntiQuorum, value);
|
||||
} else if (ck == LiteralStringRef("regions")) {
|
||||
parse(®ions, value);
|
||||
} else {
|
||||
return false;
|
||||
}
|
||||
else if (ck == LiteralStringRef("log_spill")) { parse((&type), value); tLogSpillType = (TLogSpillType::SpillType)type; }
|
||||
else if (ck == LiteralStringRef("storage_engine")) { parse((&type), value); storageServerStoreType = (KeyValueStoreType::StoreType)type; }
|
||||
else if (ck == LiteralStringRef("auto_proxies")) parse(&autoMasterProxyCount, value);
|
||||
else if (ck == LiteralStringRef("auto_resolvers")) parse(&autoResolverCount, value);
|
||||
else if (ck == LiteralStringRef("auto_logs")) parse(&autoDesiredTLogCount, value);
|
||||
else if (ck == LiteralStringRef("storage_replication_policy")) parseReplicationPolicy(&storagePolicy, value);
|
||||
else if (ck == LiteralStringRef("log_replication_policy")) parseReplicationPolicy(&tLogPolicy, value);
|
||||
else if (ck == LiteralStringRef("log_routers")) parse(&desiredLogRouterCount, value);
|
||||
else if (ck == LiteralStringRef("remote_logs")) parse(&remoteDesiredTLogCount, value);
|
||||
else if (ck == LiteralStringRef("remote_log_replicas")) parse(&remoteTLogReplicationFactor, value);
|
||||
else if (ck == LiteralStringRef("remote_log_policy")) parseReplicationPolicy(&remoteTLogPolicy, value);
|
||||
else if (ck == LiteralStringRef("backup_worker_enabled")) { parse((&type), value); backupWorkerEnabled = (type != 0); }
|
||||
else if (ck == LiteralStringRef("usable_regions")) parse(&usableRegions, value);
|
||||
else if (ck == LiteralStringRef("repopulate_anti_quorum")) parse(&repopulateRegionAntiQuorum, value);
|
||||
else if (ck == LiteralStringRef("regions")) parse(®ions, value);
|
||||
else return false;
|
||||
return true; // All of the above options currently require recovery to take effect
|
||||
}
|
||||
|
||||
|
|
|
@ -149,9 +149,11 @@ struct DatabaseConfiguration {
|
|||
return std::min(tLogReplicationFactor - 1 - tLogWriteAntiQuorum, storageTeamSize - 1);
|
||||
}
|
||||
|
||||
// MasterProxy Servers
|
||||
int32_t masterProxyCount;
|
||||
int32_t autoMasterProxyCount;
|
||||
// Proxy Servers
|
||||
int32_t proxyCount;
|
||||
int32_t autoProxyCount;
|
||||
int32_t grvProxyCount;
|
||||
int32_t autoGrvProxyCount;
|
||||
|
||||
// Resolvers
|
||||
int32_t resolverCount;
|
||||
|
@ -190,7 +192,11 @@ struct DatabaseConfiguration {
|
|||
bool isExcludedServer( NetworkAddressList ) const;
|
||||
std::set<AddressExclusion> getExcludedServers() const;
|
||||
|
||||
int32_t getDesiredProxies() const { if(masterProxyCount == -1) return autoMasterProxyCount; return masterProxyCount; }
|
||||
int32_t getDesiredProxies() const { if(proxyCount == -1) return autoProxyCount; return proxyCount; }
|
||||
int32_t getDesiredGrvProxies() const {
|
||||
if (grvProxyCount == -1) return autoGrvProxyCount;
|
||||
return grvProxyCount;
|
||||
}
|
||||
int32_t getDesiredResolvers() const { if(resolverCount == -1) return autoResolverCount; return resolverCount; }
|
||||
int32_t getDesiredLogs() const { if(desiredTLogCount == -1) return autoDesiredTLogCount; return desiredTLogCount; }
|
||||
int32_t getDesiredRemoteLogs() const { if(remoteDesiredTLogCount == -1) return getDesiredLogs(); return remoteDesiredTLogCount; }
|
||||
|
|
|
@ -68,7 +68,8 @@ struct LocationInfo : MultiInterface<ReferencedInterface<StorageServerInterface>
|
|||
}
|
||||
};
|
||||
|
||||
typedef ModelInterface<MasterProxyInterface> ProxyInfo;
|
||||
using ProxyInfo = ModelInterface<MasterProxyInterface>;
|
||||
using GrvProxyInfo = ModelInterface<GrvProxyInterface>;
|
||||
|
||||
class ClientTagThrottleData : NonCopyable {
|
||||
private:
|
||||
|
@ -160,11 +161,14 @@ public:
|
|||
void invalidateCache( const KeyRef&, bool isBackward = false );
|
||||
void invalidateCache( const KeyRangeRef& );
|
||||
|
||||
bool sampleReadTags();
|
||||
bool sampleReadTags() const;
|
||||
bool sampleOnCost(uint64_t cost) const;
|
||||
|
||||
void updateProxies();
|
||||
Reference<ProxyInfo> getMasterProxies(bool useProvisionalProxies);
|
||||
Future<Reference<ProxyInfo>> getMasterProxiesFuture(bool useProvisionalProxies);
|
||||
Future<Void> onMasterProxiesChanged();
|
||||
Reference<GrvProxyInfo> getGrvProxies(bool useProvisionalProxies);
|
||||
Future<Void> onProxiesChanged();
|
||||
Future<HealthMetrics> getHealthMetrics(bool detailed);
|
||||
|
||||
// Update the watch counter for the database
|
||||
|
@ -213,11 +217,12 @@ public:
|
|||
|
||||
// Key DB-specific information
|
||||
Reference<AsyncVar<Reference<ClusterConnectionFile>>> connectionFile;
|
||||
AsyncTrigger masterProxiesChangeTrigger;
|
||||
Future<Void> monitorMasterProxiesInfoChange;
|
||||
AsyncTrigger proxiesChangeTrigger;
|
||||
Future<Void> monitorProxiesInfoChange;
|
||||
Reference<ProxyInfo> masterProxies;
|
||||
bool provisional;
|
||||
UID masterProxiesLastChange;
|
||||
Reference<GrvProxyInfo> grvProxies;
|
||||
bool proxyProvisional;
|
||||
UID proxiesLastChange;
|
||||
LocalityData clientLocality;
|
||||
QueueModel queueModel;
|
||||
bool enableLocalityLoadBalance;
|
||||
|
@ -305,6 +310,7 @@ public:
|
|||
Counter transactionsResourceConstrained;
|
||||
Counter transactionsProcessBehind;
|
||||
Counter transactionsThrottled;
|
||||
Counter transactionsExpensiveClearCostEstCount;
|
||||
|
||||
ContinuousSample<double> latencies, readLatencies, commitLatencies, GRVLatencies, mutationsPerCommit, bytesPerCommit;
|
||||
|
||||
|
@ -337,6 +343,7 @@ public:
|
|||
HealthMetrics healthMetrics;
|
||||
double healthMetricsLastUpdated;
|
||||
double detailedHealthMetricsLastUpdated;
|
||||
Smoother smoothMidShardSize;
|
||||
|
||||
UniqueOrderedOptionList<FDBTransactionOptions> transactionDefaults;
|
||||
|
||||
|
|
|
@ -131,6 +131,9 @@ public:
|
|||
KeyBackedProperty<Key> removePrefix() {
|
||||
return configSpace.pack(LiteralStringRef(__FUNCTION__));
|
||||
}
|
||||
KeyBackedProperty<bool> incrementalBackupOnly() {
|
||||
return configSpace.pack(LiteralStringRef(__FUNCTION__));
|
||||
}
|
||||
// XXX: Remove restoreRange() once it is safe to remove. It has been changed to restoreRanges
|
||||
KeyBackedProperty<KeyRange> restoreRange() {
|
||||
return configSpace.pack(LiteralStringRef(__FUNCTION__));
|
||||
|
@ -141,6 +144,9 @@ public:
|
|||
KeyBackedProperty<Key> batchFuture() {
|
||||
return configSpace.pack(LiteralStringRef(__FUNCTION__));
|
||||
}
|
||||
KeyBackedProperty<Version> beginVersion() {
|
||||
return configSpace.pack(LiteralStringRef(__FUNCTION__));
|
||||
}
|
||||
KeyBackedProperty<Version> restoreVersion() {
|
||||
return configSpace.pack(LiteralStringRef(__FUNCTION__));
|
||||
}
|
||||
|
@ -2474,7 +2480,8 @@ namespace fileBackup {
|
|||
state Future<std::vector<KeyRange>> backupRangesFuture = config.backupRanges().getOrThrow(tr);
|
||||
state Future<Key> destUidValueFuture = config.destUidValue().getOrThrow(tr);
|
||||
state Future<Optional<bool>> partitionedLog = config.partitionedLogEnabled().get(tr);
|
||||
wait(success(backupRangesFuture) && success(destUidValueFuture) && success(partitionedLog));
|
||||
state Future<Optional<bool>> incrementalBackupOnly = config.incrementalBackupOnly().get(tr);
|
||||
wait(success(backupRangesFuture) && success(destUidValueFuture) && success(partitionedLog) && success(incrementalBackupOnly));
|
||||
std::vector<KeyRange> backupRanges = backupRangesFuture.get();
|
||||
Key destUidValue = destUidValueFuture.get();
|
||||
|
||||
|
@ -2494,7 +2501,10 @@ namespace fileBackup {
|
|||
wait(config.initNewSnapshot(tr, 0));
|
||||
|
||||
// Using priority 1 for both of these to at least start both tasks soon
|
||||
wait(success(BackupSnapshotDispatchTask::addTask(tr, taskBucket, task, 1, TaskCompletionKey::joinWith(backupFinished))));
|
||||
// Do not add snapshot task if we only want the incremental backup
|
||||
if (!incrementalBackupOnly.get().present() || !incrementalBackupOnly.get().get()) {
|
||||
wait(success(BackupSnapshotDispatchTask::addTask(tr, taskBucket, task, 1, TaskCompletionKey::joinWith(backupFinished))));
|
||||
}
|
||||
wait(success(BackupLogsDispatchTask::addTask(tr, taskBucket, task, 1, 0, beginVersion, TaskCompletionKey::joinWith(backupFinished))));
|
||||
|
||||
// If a clean stop is requested, the log and snapshot tasks will quit after the backup is restorable, then the following
|
||||
|
@ -3008,8 +3018,10 @@ namespace fileBackup {
|
|||
state int64_t remainingInBatch = Params.remainingInBatch().get(task);
|
||||
state bool addingToExistingBatch = remainingInBatch > 0;
|
||||
state Version restoreVersion;
|
||||
state Future<Optional<bool>> incrementalBackupOnly = restore.incrementalBackupOnly().get(tr);
|
||||
|
||||
wait(store(restoreVersion, restore.restoreVersion().getOrThrow(tr))
|
||||
&& success(incrementalBackupOnly)
|
||||
&& checkTaskVersion(tr->getDatabase(), task, name, version));
|
||||
|
||||
// If not adding to an existing batch then update the apply mutations end version so the mutations from the
|
||||
|
@ -3398,6 +3410,7 @@ namespace fileBackup {
|
|||
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
|
||||
state RestoreConfig restore(task);
|
||||
state Version restoreVersion;
|
||||
state Version beginVersion;
|
||||
state Reference<IBackupContainer> bc;
|
||||
|
||||
loop {
|
||||
|
@ -3408,6 +3421,8 @@ namespace fileBackup {
|
|||
wait(checkTaskVersion(tr->getDatabase(), task, name, version));
|
||||
Version _restoreVersion = wait(restore.restoreVersion().getOrThrow(tr));
|
||||
restoreVersion = _restoreVersion;
|
||||
Optional<Version> _beginVersion = wait(restore.beginVersion().get(tr));
|
||||
beginVersion = _beginVersion.present() ? _beginVersion.get() : invalidVersion;
|
||||
wait(taskBucket->keepRunning(tr, task));
|
||||
|
||||
ERestoreState oldState = wait(restore.stateEnum().getD(tr));
|
||||
|
@ -3447,14 +3462,21 @@ namespace fileBackup {
|
|||
wait(tr->onError(e));
|
||||
}
|
||||
}
|
||||
|
||||
Optional<RestorableFileSet> restorable = wait(bc->getRestoreSet(restoreVersion));
|
||||
Optional<bool> _incremental = wait(restore.incrementalBackupOnly().get(tr));
|
||||
state bool incremental = _incremental.present() ? _incremental.get() : false;
|
||||
if (beginVersion == invalidVersion) {
|
||||
beginVersion = 0;
|
||||
}
|
||||
Optional<RestorableFileSet> restorable = wait(bc->getRestoreSet(restoreVersion, incremental, beginVersion));
|
||||
if (!incremental) {
|
||||
beginVersion = restorable.get().snapshot.beginVersion;
|
||||
}
|
||||
|
||||
if(!restorable.present())
|
||||
throw restore_missing_data();
|
||||
|
||||
// First version for which log data should be applied
|
||||
Params.firstVersion().set(task, restorable.get().snapshot.beginVersion);
|
||||
Params.firstVersion().set(task, beginVersion);
|
||||
|
||||
// Convert the two lists in restorable (logs and ranges) to a single list of RestoreFiles.
|
||||
// Order does not matter, they will be put in order when written to the restoreFileMap below.
|
||||
|
@ -3463,6 +3485,7 @@ namespace fileBackup {
|
|||
for(const RangeFile &f : restorable.get().ranges) {
|
||||
files.push_back({f.version, f.fileName, true, f.blockSize, f.fileSize});
|
||||
}
|
||||
|
||||
for(const LogFile &f : restorable.get().logs) {
|
||||
files.push_back({f.beginVersion, f.fileName, false, f.blockSize, f.fileSize, f.endVersion});
|
||||
}
|
||||
|
@ -3526,6 +3549,7 @@ namespace fileBackup {
|
|||
restore.stateEnum().set(tr, ERestoreState::RUNNING);
|
||||
|
||||
// Set applyMutation versions
|
||||
|
||||
restore.setApplyBeginVersion(tr, firstVersion);
|
||||
restore.setApplyEndVersion(tr, firstVersion);
|
||||
|
||||
|
@ -3533,6 +3557,14 @@ namespace fileBackup {
|
|||
wait(success(RestoreDispatchTaskFunc::addTask(tr, taskBucket, task, 0, "", 0, CLIENT_KNOBS->RESTORE_DISPATCH_BATCH_SIZE)));
|
||||
|
||||
wait(taskBucket->finish(tr, task));
|
||||
state Future<Optional<bool>> logsOnly = restore.incrementalBackupOnly().get(tr);
|
||||
wait(success(logsOnly));
|
||||
if (logsOnly.get().present() && logsOnly.get().get()) {
|
||||
// If this is an incremental restore, we need to set the applyMutationsMapPrefix
|
||||
// to the earliest log version so no mutations are missed
|
||||
Value versionEncoded = BinaryWriter::toValue(Params.firstVersion().get(task), Unversioned());
|
||||
wait(krmSetRange(tr, restore.applyMutationsMapPrefix(), normalKeys, versionEncoded));
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
|
||||
|
@ -3760,7 +3792,7 @@ public:
|
|||
ACTOR static Future<Void> submitBackup(FileBackupAgent* backupAgent, Reference<ReadYourWritesTransaction> tr,
|
||||
Key outContainer, int snapshotIntervalSeconds, std::string tagName,
|
||||
Standalone<VectorRef<KeyRangeRef>> backupRanges, bool stopWhenDone,
|
||||
bool partitionedLog) {
|
||||
bool partitionedLog, bool incrementalBackupOnly) {
|
||||
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
||||
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
||||
tr->setOption(FDBTransactionOptions::COMMIT_ON_FIRST_PROXY);
|
||||
|
@ -3863,13 +3895,17 @@ public:
|
|||
config.backupRanges().set(tr, normalizedRanges);
|
||||
config.snapshotIntervalSeconds().set(tr, snapshotIntervalSeconds);
|
||||
config.partitionedLogEnabled().set(tr, partitionedLog);
|
||||
config.incrementalBackupOnly().set(tr, incrementalBackupOnly);
|
||||
|
||||
Key taskKey = wait(fileBackup::StartFullBackupTaskFunc::addTask(tr, backupAgent->taskBucket, uid, TaskCompletionKey::noSignal()));
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> submitRestore(FileBackupAgent* backupAgent, Reference<ReadYourWritesTransaction> tr, Key tagName, Key backupURL, Standalone<VectorRef<KeyRangeRef>> ranges, Version restoreVersion, Key addPrefix, Key removePrefix, bool lockDB, UID uid) {
|
||||
ACTOR static Future<Void> submitRestore(FileBackupAgent* backupAgent, Reference<ReadYourWritesTransaction> tr,
|
||||
Key tagName, Key backupURL, Standalone<VectorRef<KeyRangeRef>> ranges,
|
||||
Version restoreVersion, Key addPrefix, Key removePrefix, bool lockDB,
|
||||
bool incrementalBackupOnly, Version beginVersion, UID uid) {
|
||||
KeyRangeMap<int> restoreRangeSet;
|
||||
for (auto& range : ranges) {
|
||||
restoreRangeSet.insert(range, 1);
|
||||
|
@ -3917,7 +3953,7 @@ public:
|
|||
for (index = 0; index < restoreRanges.size(); index++) {
|
||||
KeyRange restoreIntoRange = KeyRangeRef(restoreRanges[index].begin, restoreRanges[index].end).removePrefix(removePrefix).withPrefix(addPrefix);
|
||||
Standalone<RangeResultRef> existingRows = wait(tr->getRange(restoreIntoRange, 1));
|
||||
if (existingRows.size() > 0) {
|
||||
if (existingRows.size() > 0 && !incrementalBackupOnly) {
|
||||
throw restore_destination_not_empty();
|
||||
}
|
||||
}
|
||||
|
@ -3934,6 +3970,8 @@ public:
|
|||
restore.sourceContainer().set(tr, bc);
|
||||
restore.stateEnum().set(tr, ERestoreState::QUEUED);
|
||||
restore.restoreVersion().set(tr, restoreVersion);
|
||||
restore.incrementalBackupOnly().set(tr, incrementalBackupOnly);
|
||||
restore.beginVersion().set(tr, beginVersion);
|
||||
if (BUGGIFY && restoreRanges.size() == 1) {
|
||||
restore.restoreRange().set(tr, restoreRanges[0]);
|
||||
}
|
||||
|
@ -4451,7 +4489,8 @@ public:
|
|||
ACTOR static Future<Version> restore(FileBackupAgent* backupAgent, Database cx, Optional<Database> cxOrig,
|
||||
Key tagName, Key url, Standalone<VectorRef<KeyRangeRef>> ranges,
|
||||
bool waitForComplete, Version targetVersion, bool verbose, Key addPrefix,
|
||||
Key removePrefix, bool lockDB, UID randomUid) {
|
||||
Key removePrefix, bool lockDB, bool incrementalBackupOnly,
|
||||
Version beginVersion, UID randomUid) {
|
||||
state Reference<IBackupContainer> bc = IBackupContainer::openContainer(url.toString());
|
||||
|
||||
state BackupDescription desc = wait(bc->describeBackup());
|
||||
|
@ -4463,7 +4502,12 @@ public:
|
|||
if(targetVersion == invalidVersion && desc.maxRestorableVersion.present())
|
||||
targetVersion = desc.maxRestorableVersion.get();
|
||||
|
||||
Optional<RestorableFileSet> restoreSet = wait(bc->getRestoreSet(targetVersion));
|
||||
if (targetVersion == invalidVersion && incrementalBackupOnly && desc.contiguousLogEnd.present()) {
|
||||
targetVersion = desc.contiguousLogEnd.get() - 1;
|
||||
}
|
||||
|
||||
Optional<RestorableFileSet> restoreSet =
|
||||
wait(bc->getRestoreSet(targetVersion, incrementalBackupOnly, beginVersion));
|
||||
|
||||
if(!restoreSet.present()) {
|
||||
TraceEvent(SevWarn, "FileBackupAgentRestoreNotPossible")
|
||||
|
@ -4482,7 +4526,8 @@ public:
|
|||
try {
|
||||
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
||||
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
||||
wait(submitRestore(backupAgent, tr, tagName, url, ranges, targetVersion, addPrefix, removePrefix, lockDB, randomUid));
|
||||
wait(submitRestore(backupAgent, tr, tagName, url, ranges, targetVersion, addPrefix, removePrefix,
|
||||
lockDB, incrementalBackupOnly, beginVersion, randomUid));
|
||||
wait(tr->commit());
|
||||
break;
|
||||
} catch(Error &e) {
|
||||
|
@ -4619,7 +4664,7 @@ public:
|
|||
} else {
|
||||
TraceEvent("AS_StartRestore");
|
||||
Version ver = wait(restore(backupAgent, cx, cx, tagName, KeyRef(bc->getURL()), ranges, true, -1, true,
|
||||
addPrefix, removePrefix, true, randomUid));
|
||||
addPrefix, removePrefix, true, false, invalidVersion, randomUid));
|
||||
return ver;
|
||||
}
|
||||
}
|
||||
|
@ -4656,8 +4701,13 @@ Future<Void> FileBackupAgent::atomicParallelRestore(Database cx, Key tagName, St
|
|||
return FileBackupAgentImpl::atomicParallelRestore(this, cx, tagName, ranges, addPrefix, removePrefix);
|
||||
}
|
||||
|
||||
Future<Version> FileBackupAgent::restore(Database cx, Optional<Database> cxOrig, Key tagName, Key url, Standalone<VectorRef<KeyRangeRef>> ranges, bool waitForComplete, Version targetVersion, bool verbose, Key addPrefix, Key removePrefix, bool lockDB) {
|
||||
return FileBackupAgentImpl::restore(this, cx, cxOrig, tagName, url, ranges, waitForComplete, targetVersion, verbose, addPrefix, removePrefix, lockDB, deterministicRandom()->randomUniqueID());
|
||||
Future<Version> FileBackupAgent::restore(Database cx, Optional<Database> cxOrig, Key tagName, Key url,
|
||||
Standalone<VectorRef<KeyRangeRef>> ranges, bool waitForComplete,
|
||||
Version targetVersion, bool verbose, Key addPrefix, Key removePrefix,
|
||||
bool lockDB, bool incrementalBackupOnly, Version beginVersion) {
|
||||
return FileBackupAgentImpl::restore(this, cx, cxOrig, tagName, url, ranges, waitForComplete, targetVersion, verbose,
|
||||
addPrefix, removePrefix, lockDB, incrementalBackupOnly, beginVersion,
|
||||
deterministicRandom()->randomUniqueID());
|
||||
}
|
||||
|
||||
Future<Version> FileBackupAgent::atomicRestore(Database cx, Key tagName, Standalone<VectorRef<KeyRangeRef>> ranges, Key addPrefix, Key removePrefix) {
|
||||
|
@ -4683,9 +4733,9 @@ Future<ERestoreState> FileBackupAgent::waitRestore(Database cx, Key tagName, boo
|
|||
Future<Void> FileBackupAgent::submitBackup(Reference<ReadYourWritesTransaction> tr, Key outContainer,
|
||||
int snapshotIntervalSeconds, std::string tagName,
|
||||
Standalone<VectorRef<KeyRangeRef>> backupRanges, bool stopWhenDone,
|
||||
bool partitionedLog) {
|
||||
bool partitionedLog, bool incrementalBackupOnly) {
|
||||
return FileBackupAgentImpl::submitBackup(this, tr, outContainer, snapshotIntervalSeconds, tagName, backupRanges,
|
||||
stopWhenDone, partitionedLog);
|
||||
stopWhenDone, partitionedLog, incrementalBackupOnly);
|
||||
}
|
||||
|
||||
Future<Void> FileBackupAgent::discontinueBackup(Reference<ReadYourWritesTransaction> tr, Key tagName){
|
||||
|
|
|
@ -0,0 +1,64 @@
|
|||
|
||||
/*
|
||||
* GrvProxyInterface.h
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2020 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.
|
||||
*/
|
||||
|
||||
#ifndef FDBCLIENT_GRVPROXYINTERFACE_H
|
||||
#define FDBCLIENT_GRVPROXYINTERFACE_H
|
||||
#pragma once
|
||||
|
||||
// GrvProxy is proxy primarily specializing on serving GetReadVersion. It also serves health metrics since it communicates
|
||||
// with RateKeeper to gather health information of the cluster.
|
||||
struct GrvProxyInterface {
|
||||
constexpr static FileIdentifier file_identifier = 8743216;
|
||||
|
||||
Optional<Key> processId;
|
||||
bool provisional;
|
||||
|
||||
RequestStream< struct GetReadVersionRequest > getConsistentReadVersion; // Returns a version which (1) is committed, and (2) is >= the latest version reported committed (by a commit response) when this request was sent
|
||||
// (at some point between when this request is sent and when its response is received, the latest version reported committed)
|
||||
RequestStream<ReplyPromise<Void>> waitFailure; // reports heartbeat to master.
|
||||
RequestStream< struct GetHealthMetricsRequest > getHealthMetrics;
|
||||
|
||||
UID id() const { return getConsistentReadVersion.getEndpoint().token; }
|
||||
std::string toString() const { return id().shortString(); }
|
||||
bool operator == (GrvProxyInterface const& r) const { return id() == r.id(); }
|
||||
bool operator != (GrvProxyInterface const& r) const { return id() != r.id(); }
|
||||
NetworkAddress address() const { return getConsistentReadVersion.getEndpoint().getPrimaryAddress(); }
|
||||
|
||||
template <class Archive>
|
||||
void serialize(Archive& ar) {
|
||||
serializer(ar, processId, provisional, getConsistentReadVersion);
|
||||
if( Archive::isDeserializing ) {
|
||||
waitFailure = RequestStream<ReplyPromise<Void>>( getConsistentReadVersion.getEndpoint().getAdjustedEndpoint(1) );
|
||||
getHealthMetrics = RequestStream< struct GetHealthMetricsRequest >( getConsistentReadVersion.getEndpoint().getAdjustedEndpoint(2) );
|
||||
}
|
||||
}
|
||||
|
||||
void initEndpoints() {
|
||||
std::vector<std::pair<FlowReceiver*, TaskPriority>> streams;
|
||||
streams.push_back(getConsistentReadVersion.getReceiver(TaskPriority::ReadSocket));
|
||||
streams.push_back(waitFailure.getReceiver());
|
||||
streams.push_back(getHealthMetrics.getReceiver());
|
||||
FlowTransport::transport().addEndpoints(streams);
|
||||
}
|
||||
};
|
||||
|
||||
|
||||
#endif // FDBCLIENT_GRVPROXYINTERFACE_H
|
|
@ -52,7 +52,8 @@ void ClientKnobs::initialize(bool randomize) {
|
|||
init( COORDINATOR_RECONNECTION_DELAY, 1.0 );
|
||||
init( CLIENT_EXAMPLE_AMOUNT, 20 );
|
||||
init( MAX_CLIENT_STATUS_AGE, 1.0 );
|
||||
init( MAX_PROXY_CONNECTIONS, 5 ); if( randomize && BUGGIFY ) MAX_PROXY_CONNECTIONS = 1;
|
||||
init( MAX_MASTER_PROXY_CONNECTIONS, 5 ); if( randomize && BUGGIFY ) MAX_MASTER_PROXY_CONNECTIONS = 1;
|
||||
init( MAX_GRV_PROXY_CONNECTIONS, 3 ); if( randomize && BUGGIFY ) MAX_GRV_PROXY_CONNECTIONS = 1;
|
||||
init( STATUS_IDLE_TIMEOUT, 120.0 );
|
||||
|
||||
// wrong_shard_server sometimes comes from the only nonfailed server, so we need to avoid a fast spin
|
||||
|
@ -65,6 +66,8 @@ void ClientKnobs::initialize(bool randomize) {
|
|||
init( BACKOFF_GROWTH_RATE, 2.0 );
|
||||
init( RESOURCE_CONSTRAINED_MAX_BACKOFF, 30.0 );
|
||||
init( PROXY_COMMIT_OVERHEAD_BYTES, 23 ); //The size of serializing 7 tags (3 primary, 3 remote, 1 log router) + 2 for the tag length
|
||||
init( SHARD_STAT_SMOOTH_AMOUNT, 5.0 );
|
||||
init( INIT_MID_SHARD_BYTES, 200000 ); if( randomize && BUGGIFY ) INIT_MID_SHARD_BYTES = 40000; // The same value as SERVER_KNOBS->MIN_SHARD_BYTES
|
||||
|
||||
init( TRANSACTION_SIZE_LIMIT, 1e7 );
|
||||
init( KEY_SIZE_LIMIT, 1e4 );
|
||||
|
@ -89,7 +92,8 @@ void ClientKnobs::initialize(bool randomize) {
|
|||
init( STORAGE_METRICS_TOO_MANY_SHARDS_DELAY, 15.0 );
|
||||
init( AGGREGATE_HEALTH_METRICS_MAX_STALENESS, 0.5 );
|
||||
init( DETAILED_HEALTH_METRICS_MAX_STALENESS, 5.0 );
|
||||
init( TAG_ENCODE_KEY_SERVERS, true ); if( randomize && BUGGIFY ) TAG_ENCODE_KEY_SERVERS = false;
|
||||
init( MID_SHARD_SIZE_MAX_STALENESS, 10.0 );
|
||||
init( TAG_ENCODE_KEY_SERVERS, false ); if( randomize && BUGGIFY ) TAG_ENCODE_KEY_SERVERS = true;
|
||||
|
||||
//KeyRangeMap
|
||||
init( KRM_GET_RANGE_LIMIT, 1e5 ); if( randomize && BUGGIFY ) KRM_GET_RANGE_LIMIT = 10;
|
||||
|
@ -168,6 +172,7 @@ void ClientKnobs::initialize(bool randomize) {
|
|||
|
||||
// Configuration
|
||||
init( DEFAULT_AUTO_PROXIES, 3 );
|
||||
init( DEFAULT_AUTO_GRV_PROXIES, 1 );
|
||||
init( DEFAULT_AUTO_RESOLVERS, 1 );
|
||||
init( DEFAULT_AUTO_LOGS, 3 );
|
||||
|
||||
|
@ -227,6 +232,9 @@ void ClientKnobs::initialize(bool randomize) {
|
|||
// transaction tags
|
||||
init( MAX_TAGS_PER_TRANSACTION, 5 );
|
||||
init( MAX_TRANSACTION_TAG_LENGTH, 16 );
|
||||
init( COMMIT_SAMPLE_COST, 100 ); if( randomize && BUGGIFY ) COMMIT_SAMPLE_COST = 10;
|
||||
init( WRITE_COST_BYTE_FACTOR, 16384 ); if( randomize && BUGGIFY ) WRITE_COST_BYTE_FACTOR = 4096;
|
||||
init( INCOMPLETE_SHARD_PLUS, 4096 );
|
||||
init( READ_TAG_SAMPLE_RATE, 0.01 ); if( randomize && BUGGIFY ) READ_TAG_SAMPLE_RATE = 1.0; // Communicated to clients from cluster
|
||||
init( TAG_THROTTLE_SMOOTHING_WINDOW, 2.0 );
|
||||
init( TAG_THROTTLE_RECHECK_INTERVAL, 5.0 ); if( randomize && BUGGIFY ) TAG_THROTTLE_RECHECK_INTERVAL = 0.0;
|
||||
|
|
|
@ -27,8 +27,6 @@
|
|||
|
||||
class ClientKnobs : public Knobs {
|
||||
public:
|
||||
int BYTE_LIMIT_UNLIMITED;
|
||||
int ROW_LIMIT_UNLIMITED;
|
||||
|
||||
int TOO_MANY; // FIXME: this should really be split up so we can control these more specifically
|
||||
|
||||
|
@ -48,7 +46,8 @@ public:
|
|||
double COORDINATOR_RECONNECTION_DELAY;
|
||||
int CLIENT_EXAMPLE_AMOUNT;
|
||||
double MAX_CLIENT_STATUS_AGE;
|
||||
int MAX_PROXY_CONNECTIONS;
|
||||
int MAX_MASTER_PROXY_CONNECTIONS;
|
||||
int MAX_GRV_PROXY_CONNECTIONS;
|
||||
double STATUS_IDLE_TIMEOUT;
|
||||
|
||||
// wrong_shard_server sometimes comes from the only nonfailed server, so we need to avoid a fast spin
|
||||
|
@ -60,6 +59,8 @@ public:
|
|||
double BACKOFF_GROWTH_RATE;
|
||||
double RESOURCE_CONSTRAINED_MAX_BACKOFF;
|
||||
int PROXY_COMMIT_OVERHEAD_BYTES;
|
||||
double SHARD_STAT_SMOOTH_AMOUNT;
|
||||
int INIT_MID_SHARD_BYTES;
|
||||
|
||||
int TRANSACTION_SIZE_LIMIT;
|
||||
int64_t KEY_SIZE_LIMIT;
|
||||
|
@ -85,6 +86,7 @@ public:
|
|||
double STORAGE_METRICS_TOO_MANY_SHARDS_DELAY;
|
||||
double AGGREGATE_HEALTH_METRICS_MAX_STALENESS;
|
||||
double DETAILED_HEALTH_METRICS_MAX_STALENESS;
|
||||
double MID_SHARD_SIZE_MAX_STALENESS;
|
||||
bool TAG_ENCODE_KEY_SERVERS;
|
||||
|
||||
//KeyRangeMap
|
||||
|
@ -166,6 +168,7 @@ public:
|
|||
|
||||
// Configuration
|
||||
int32_t DEFAULT_AUTO_PROXIES;
|
||||
int32_t DEFAULT_AUTO_GRV_PROXIES;
|
||||
int32_t DEFAULT_AUTO_RESOLVERS;
|
||||
int32_t DEFAULT_AUTO_LOGS;
|
||||
|
||||
|
@ -214,6 +217,9 @@ public:
|
|||
// transaction tags
|
||||
int MAX_TRANSACTION_TAG_LENGTH;
|
||||
int MAX_TAGS_PER_TRANSACTION;
|
||||
int COMMIT_SAMPLE_COST; // The expectation of sampling is every COMMIT_SAMPLE_COST sample once
|
||||
int WRITE_COST_BYTE_FACTOR;
|
||||
int INCOMPLETE_SHARD_PLUS; // The size of (possible) incomplete shard when estimate clear range
|
||||
double READ_TAG_SAMPLE_RATE; // Communicated to clients from cluster
|
||||
double TAG_THROTTLE_SMOOTHING_WINDOW;
|
||||
double TAG_THROTTLE_RECHECK_INTERVAL;
|
||||
|
|
|
@ -78,7 +78,9 @@ std::map<std::string, std::string> configForToken( std::string const& mode ) {
|
|||
std::string key = mode.substr(0, pos);
|
||||
std::string value = mode.substr(pos+1);
|
||||
|
||||
if( (key == "logs" || key == "proxies" || key == "resolvers" || key == "remote_logs" || key == "log_routers" || key == "usable_regions" || key == "repopulate_anti_quorum") && isInteger(value) ) {
|
||||
if ((key == "logs" || key == "proxies" || key == "grv_proxies" || key == "resolvers" || key == "remote_logs" ||
|
||||
key == "log_routers" || key == "usable_regions" || key == "repopulate_anti_quorum") &&
|
||||
isInteger(value)) {
|
||||
out[p+key] = value;
|
||||
}
|
||||
|
||||
|
@ -653,7 +655,12 @@ ConfigureAutoResult parseConfig( StatusObject const& status ) {
|
|||
oldMachinesWithTransaction.insert(machineId);
|
||||
}
|
||||
|
||||
if(processClass.classType() == ProcessClass::TransactionClass || processClass.classType() == ProcessClass::ProxyClass || processClass.classType() == ProcessClass::ResolutionClass || processClass.classType() == ProcessClass::StatelessClass || processClass.classType() == ProcessClass::LogClass) {
|
||||
if (processClass.classType() == ProcessClass::TransactionClass ||
|
||||
processClass.classType() == ProcessClass::ProxyClass ||
|
||||
processClass.classType() == ProcessClass::GrvProxyClass ||
|
||||
processClass.classType() == ProcessClass::ResolutionClass ||
|
||||
processClass.classType() == ProcessClass::StatelessClass ||
|
||||
processClass.classType() == ProcessClass::LogClass) {
|
||||
oldTransactionProcesses++;
|
||||
}
|
||||
|
||||
|
@ -682,6 +689,7 @@ ConfigureAutoResult parseConfig( StatusObject const& status ) {
|
|||
std::set<std::string> machinesWithStorage;
|
||||
int totalTransactionProcesses = 0;
|
||||
int existingProxyCount = 0;
|
||||
int existingGrvProxyCount = 0;
|
||||
int existingResolverCount = 0;
|
||||
int existingStatelessCount = 0;
|
||||
for( auto& it : machine_processes ) {
|
||||
|
@ -690,12 +698,15 @@ ConfigureAutoResult parseConfig( StatusObject const& status ) {
|
|||
totalTransactionProcesses++;
|
||||
machinesWithTransaction.insert(it.first);
|
||||
}
|
||||
if(proc.second == ProcessClass::StatelessClass) {
|
||||
if (proc.second == ProcessClass::StatelessClass) {
|
||||
existingStatelessCount++;
|
||||
}
|
||||
if(proc.second == ProcessClass::ProxyClass) {
|
||||
existingProxyCount++;
|
||||
}
|
||||
if (proc.second == ProcessClass::GrvProxyClass) {
|
||||
existingGrvProxyCount++;
|
||||
}
|
||||
if(proc.second == ProcessClass::ResolutionClass) {
|
||||
existingResolverCount++;
|
||||
}
|
||||
|
@ -723,7 +734,7 @@ ConfigureAutoResult parseConfig( StatusObject const& status ) {
|
|||
resolverCount = result.old_resolvers;
|
||||
}
|
||||
|
||||
result.desired_proxies = std::min( 12, processCount / 15 );
|
||||
result.desired_proxies = std::max(std::min(12, processCount / 15), 1);
|
||||
int proxyCount;
|
||||
if (!statusObjConfig.get("proxies", result.old_proxies)) {
|
||||
result.old_proxies = CLIENT_KNOBS->DEFAULT_AUTO_PROXIES;
|
||||
|
@ -735,6 +746,19 @@ ConfigureAutoResult parseConfig( StatusObject const& status ) {
|
|||
proxyCount = result.old_proxies;
|
||||
}
|
||||
|
||||
// Need to configure a good number.
|
||||
result.desired_grv_proxies = std::max(std::min(4, processCount / 20), 1);
|
||||
int grvProxyCount;
|
||||
if (!statusObjConfig.get("grv_proxies", result.old_grv_proxies)) {
|
||||
result.old_grv_proxies = CLIENT_KNOBS->DEFAULT_AUTO_GRV_PROXIES;
|
||||
statusObjConfig.get("auto_grv_proxies", result.old_grv_proxies);
|
||||
result.auto_grv_proxies = result.desired_grv_proxies;
|
||||
grvProxyCount = result.auto_grv_proxies;
|
||||
} else {
|
||||
result.auto_grv_proxies = result.old_grv_proxies;
|
||||
grvProxyCount = result.old_grv_proxies;
|
||||
}
|
||||
|
||||
result.desired_logs = std::min( 12, processCount / 20 );
|
||||
result.desired_logs = std::max( result.desired_logs, log_replication + 1 );
|
||||
result.desired_logs = std::min<int>( result.desired_logs, machine_processes.size() );
|
||||
|
@ -752,6 +776,7 @@ ConfigureAutoResult parseConfig( StatusObject const& status ) {
|
|||
logCount = std::max(logCount, log_replication);
|
||||
|
||||
totalTransactionProcesses += std::min(existingProxyCount, proxyCount);
|
||||
totalTransactionProcesses += std::min(existingGrvProxyCount, grvProxyCount);
|
||||
totalTransactionProcesses += std::min(existingResolverCount, resolverCount);
|
||||
totalTransactionProcesses += existingStatelessCount;
|
||||
|
||||
|
@ -767,7 +792,7 @@ ConfigureAutoResult parseConfig( StatusObject const& status ) {
|
|||
}
|
||||
}
|
||||
|
||||
int desiredTotalTransactionProcesses = logCount + resolverCount + proxyCount;
|
||||
int desiredTotalTransactionProcesses = logCount + resolverCount + proxyCount + grvProxyCount;
|
||||
|
||||
//add machines with all transaction class until we have enough processes and enough machines
|
||||
for( auto& it : count_processes ) {
|
||||
|
@ -829,12 +854,15 @@ ACTOR Future<ConfigurationResult::Type> autoConfig( Database cx, ConfigureAutoRe
|
|||
if(conf.address_class.size())
|
||||
tr.set(processClassChangeKey, deterministicRandom()->randomUniqueID().toString());
|
||||
|
||||
if(conf.auto_logs != conf.old_logs)
|
||||
if (conf.auto_logs != conf.old_logs)
|
||||
tr.set(configKeysPrefix.toString() + "auto_logs", format("%d", conf.auto_logs));
|
||||
|
||||
if(conf.auto_proxies != conf.old_proxies)
|
||||
tr.set(configKeysPrefix.toString() + "auto_proxies", format("%d", conf.auto_proxies));
|
||||
|
||||
if (conf.auto_grv_proxies != conf.old_grv_proxies)
|
||||
tr.set(configKeysPrefix.toString() + "auto_grv_proxies", format("%d", conf.auto_grv_proxies));
|
||||
|
||||
if(conf.auto_resolvers != conf.old_resolvers)
|
||||
tr.set(configKeysPrefix.toString() + "auto_resolvers", format("%d", conf.auto_resolvers));
|
||||
|
||||
|
|
|
@ -87,6 +87,7 @@ struct ConfigureAutoResult {
|
|||
|
||||
std::string old_replication;
|
||||
int32_t old_proxies;
|
||||
int32_t old_grv_proxies;
|
||||
int32_t old_resolvers;
|
||||
int32_t old_logs;
|
||||
int32_t old_processes_with_transaction;
|
||||
|
@ -94,19 +95,22 @@ struct ConfigureAutoResult {
|
|||
|
||||
std::string auto_replication;
|
||||
int32_t auto_proxies;
|
||||
int32_t auto_grv_proxies;
|
||||
int32_t auto_resolvers;
|
||||
int32_t auto_logs;
|
||||
int32_t auto_processes_with_transaction;
|
||||
int32_t auto_machines_with_transaction;
|
||||
|
||||
int32_t desired_proxies;
|
||||
int32_t desired_grv_proxies;
|
||||
int32_t desired_resolvers;
|
||||
int32_t desired_logs;
|
||||
|
||||
ConfigureAutoResult() : processes(-1), machines(-1),
|
||||
old_proxies(-1), old_resolvers(-1), old_logs(-1), old_processes_with_transaction(-1), old_machines_with_transaction(-1),
|
||||
auto_proxies(-1), auto_resolvers(-1), auto_logs(-1), auto_processes_with_transaction(-1), auto_machines_with_transaction(-1),
|
||||
desired_proxies(-1), desired_resolvers(-1), desired_logs(-1) {}
|
||||
ConfigureAutoResult()
|
||||
: processes(-1), machines(-1), old_proxies(-1), old_grv_proxies(-1), old_resolvers(-1), old_logs(-1),
|
||||
old_processes_with_transaction(-1), old_machines_with_transaction(-1), auto_proxies(-1), auto_grv_proxies(-1),
|
||||
auto_resolvers(-1), auto_logs(-1), auto_processes_with_transaction(-1), auto_machines_with_transaction(-1),
|
||||
desired_proxies(-1), desired_grv_proxies(-1), desired_resolvers(-1), desired_logs(-1) {}
|
||||
|
||||
bool isValid() const { return processes != -1; }
|
||||
};
|
||||
|
|
|
@ -34,6 +34,7 @@
|
|||
|
||||
#include "fdbrpc/Stats.h"
|
||||
#include "fdbrpc/TimedRequest.h"
|
||||
#include "GrvProxyInterface.h"
|
||||
|
||||
struct MasterProxyInterface {
|
||||
constexpr static FileIdentifier file_identifier = 8954922;
|
||||
|
@ -99,14 +100,19 @@ struct MasterProxyInterface {
|
|||
struct ClientDBInfo {
|
||||
constexpr static FileIdentifier file_identifier = 5355080;
|
||||
UID id; // Changes each time anything else changes
|
||||
vector< MasterProxyInterface > proxies;
|
||||
vector< GrvProxyInterface > grvProxies;
|
||||
vector< MasterProxyInterface > masterProxies;
|
||||
Optional<MasterProxyInterface> firstProxy; //not serialized, used for commitOnFirstProxy when the proxies vector has been shrunk
|
||||
double clientTxnInfoSampleRate;
|
||||
int64_t clientTxnInfoSizeLimit;
|
||||
Optional<Value> forward;
|
||||
double transactionTagSampleRate;
|
||||
double transactionTagSampleCost;
|
||||
|
||||
ClientDBInfo() : clientTxnInfoSampleRate(std::numeric_limits<double>::infinity()), clientTxnInfoSizeLimit(-1), transactionTagSampleRate(CLIENT_KNOBS->READ_TAG_SAMPLE_RATE) {}
|
||||
ClientDBInfo()
|
||||
: clientTxnInfoSampleRate(std::numeric_limits<double>::infinity()), clientTxnInfoSizeLimit(-1),
|
||||
transactionTagSampleRate(CLIENT_KNOBS->READ_TAG_SAMPLE_RATE),
|
||||
transactionTagSampleCost(CLIENT_KNOBS->COMMIT_SAMPLE_COST) {}
|
||||
|
||||
bool operator == (ClientDBInfo const& r) const { return id == r.id; }
|
||||
bool operator != (ClientDBInfo const& r) const { return id != r.id; }
|
||||
|
@ -116,7 +122,8 @@ struct ClientDBInfo {
|
|||
if constexpr (!is_fb_function<Archive>) {
|
||||
ASSERT(ar.protocolVersion().isValid());
|
||||
}
|
||||
serializer(ar, proxies, id, clientTxnInfoSampleRate, clientTxnInfoSizeLimit, forward, transactionTagSampleRate);
|
||||
serializer(ar, grvProxies, masterProxies, id, clientTxnInfoSampleRate, clientTxnInfoSizeLimit, forward,
|
||||
transactionTagSampleRate, transactionTagSampleCost);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -155,7 +162,7 @@ struct CommitTransactionRequest : TimedRequest {
|
|||
ReplyPromise<CommitID> reply;
|
||||
uint32_t flags;
|
||||
Optional<UID> debugID;
|
||||
Optional<TransactionCommitCostEstimation> commitCostEstimation;
|
||||
Optional<ClientTrCommitCostEstimation> commitCostEstimation;
|
||||
Optional<TagSet> tagSet;
|
||||
|
||||
CommitTransactionRequest() : flags(0) {}
|
||||
|
@ -184,6 +191,7 @@ struct GetReadVersionReply : public BasicLoadBalancedReply {
|
|||
Version version;
|
||||
bool locked;
|
||||
Optional<Value> metadataVersion;
|
||||
int64_t midShardSize = 0;
|
||||
|
||||
TransactionTagMap<ClientTagThrottleLimits> tagThrottleInfo;
|
||||
|
||||
|
@ -191,7 +199,7 @@ struct GetReadVersionReply : public BasicLoadBalancedReply {
|
|||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, BasicLoadBalancedReply::recentRequests, version, locked, metadataVersion, tagThrottleInfo);
|
||||
serializer(ar, BasicLoadBalancedReply::processBusyTime, version, locked, metadataVersion, tagThrottleInfo, midShardSize);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -296,11 +304,27 @@ struct GetKeyServerLocationsRequest {
|
|||
}
|
||||
};
|
||||
|
||||
struct GetRawCommittedVersionReply {
|
||||
constexpr static FileIdentifier file_identifier = 1314732;
|
||||
Optional<UID> debugID;
|
||||
Version version;
|
||||
bool locked;
|
||||
Optional<Value> metadataVersion;
|
||||
Version minKnownCommittedVersion;
|
||||
|
||||
GetRawCommittedVersionReply(): debugID(Optional<UID>()), version(invalidVersion), locked(false), metadataVersion(Optional<Value>()), minKnownCommittedVersion(invalidVersion) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize( Ar& ar ) {
|
||||
serializer(ar, debugID, version, locked, metadataVersion, minKnownCommittedVersion);
|
||||
}
|
||||
};
|
||||
|
||||
struct GetRawCommittedVersionRequest {
|
||||
constexpr static FileIdentifier file_identifier = 12954034;
|
||||
SpanID spanContext;
|
||||
Optional<UID> debugID;
|
||||
ReplyPromise<GetReadVersionReply> reply;
|
||||
ReplyPromise<GetRawCommittedVersionReply> reply;
|
||||
|
||||
explicit GetRawCommittedVersionRequest(SpanID spanContext, Optional<UID> const& debugID = Optional<UID>()) : spanContext(spanContext), debugID(debugID) {}
|
||||
explicit GetRawCommittedVersionRequest() : spanContext(), debugID() {}
|
||||
|
|
|
@ -623,7 +623,10 @@ ACTOR Future<Void> getClientInfoFromLeader( Reference<AsyncVar<Optional<ClusterC
|
|||
req.knownClientInfoID = clientData->clientInfo->get().read().id;
|
||||
choose {
|
||||
when( ClientDBInfo ni = wait( brokenPromiseToNever( knownLeader->get().get().clientInterface.openDatabase.getReply( req ) ) ) ) {
|
||||
TraceEvent("MonitorLeaderForProxiesGotClientInfo", knownLeader->get().get().clientInterface.id()).detail("Proxy0", ni.proxies.size() ? ni.proxies[0].id() : UID()).detail("ClientID", ni.id);
|
||||
TraceEvent("MonitorLeaderForProxiesGotClientInfo", knownLeader->get().get().clientInterface.id())
|
||||
.detail("MasterProxy0", ni.masterProxies.size() ? ni.masterProxies[0].id() : UID())
|
||||
.detail("GrvProxy0", ni.grvProxies.size() ? ni.grvProxies[0].id() : UID())
|
||||
.detail("ClientID", ni.id);
|
||||
clientData->clientInfo->set(CachedSerialization<ClientDBInfo>(ni));
|
||||
}
|
||||
when( wait( knownLeader->onChange() ) ) {}
|
||||
|
@ -678,23 +681,40 @@ ACTOR Future<Void> monitorLeaderForProxies( Key clusterKey, vector<NetworkAddres
|
|||
}
|
||||
}
|
||||
|
||||
void shrinkProxyList( ClientDBInfo& ni, std::vector<UID>& lastProxyUIDs, std::vector<MasterProxyInterface>& lastProxies ) {
|
||||
if(ni.proxies.size() > CLIENT_KNOBS->MAX_PROXY_CONNECTIONS) {
|
||||
std::vector<UID> proxyUIDs;
|
||||
for(auto& proxy : ni.proxies) {
|
||||
proxyUIDs.push_back(proxy.id());
|
||||
void shrinkProxyList( ClientDBInfo& ni, std::vector<UID>& lastMasterProxyUIDs, std::vector<MasterProxyInterface>& lastMasterProxies,
|
||||
std::vector<UID>& lastGrvProxyUIDs, std::vector<GrvProxyInterface>& lastGrvProxies) {
|
||||
if(ni.masterProxies.size() > CLIENT_KNOBS->MAX_MASTER_PROXY_CONNECTIONS) {
|
||||
std::vector<UID> masterProxyUIDs;
|
||||
for(auto& masterProxy : ni.masterProxies) {
|
||||
masterProxyUIDs.push_back(masterProxy.id());
|
||||
}
|
||||
if(proxyUIDs != lastProxyUIDs) {
|
||||
lastProxyUIDs = proxyUIDs;
|
||||
lastProxies = ni.proxies;
|
||||
deterministicRandom()->randomShuffle(lastProxies);
|
||||
lastProxies.resize(CLIENT_KNOBS->MAX_PROXY_CONNECTIONS);
|
||||
for(int i = 0; i < lastProxies.size(); i++) {
|
||||
TraceEvent("ConnectedProxy").detail("Proxy", lastProxies[i].id());
|
||||
if(masterProxyUIDs != lastMasterProxyUIDs) {
|
||||
lastMasterProxyUIDs.swap(masterProxyUIDs);
|
||||
lastMasterProxies = ni.masterProxies;
|
||||
deterministicRandom()->randomShuffle(lastMasterProxies);
|
||||
lastMasterProxies.resize(CLIENT_KNOBS->MAX_MASTER_PROXY_CONNECTIONS);
|
||||
for(int i = 0; i < lastMasterProxies.size(); i++) {
|
||||
TraceEvent("ConnectedMasterProxy").detail("MasterProxy", lastMasterProxies[i].id());
|
||||
}
|
||||
}
|
||||
ni.firstProxy = ni.proxies[0];
|
||||
ni.proxies = lastProxies;
|
||||
ni.firstProxy = ni.masterProxies[0];
|
||||
ni.masterProxies = lastMasterProxies;
|
||||
}
|
||||
if(ni.grvProxies.size() > CLIENT_KNOBS->MAX_GRV_PROXY_CONNECTIONS) {
|
||||
std::vector<UID> grvProxyUIDs;
|
||||
for(auto& grvProxy : ni.grvProxies) {
|
||||
grvProxyUIDs.push_back(grvProxy.id());
|
||||
}
|
||||
if(grvProxyUIDs != lastGrvProxyUIDs) {
|
||||
lastGrvProxyUIDs.swap(grvProxyUIDs);
|
||||
lastGrvProxies = ni.grvProxies;
|
||||
deterministicRandom()->randomShuffle(lastGrvProxies);
|
||||
lastGrvProxies.resize(CLIENT_KNOBS->MAX_GRV_PROXY_CONNECTIONS);
|
||||
for(int i = 0; i < lastGrvProxies.size(); i++) {
|
||||
TraceEvent("ConnectedGrvProxy").detail("GrvProxy", lastGrvProxies[i].id());
|
||||
}
|
||||
}
|
||||
ni.grvProxies = lastGrvProxies;
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -707,6 +727,8 @@ ACTOR Future<MonitorLeaderInfo> monitorProxiesOneGeneration( Reference<ClusterCo
|
|||
state Optional<double> incorrectTime;
|
||||
state std::vector<UID> lastProxyUIDs;
|
||||
state std::vector<MasterProxyInterface> lastProxies;
|
||||
state std::vector<UID> lastGrvProxyUIDs;
|
||||
state std::vector<GrvProxyInterface> lastGrvProxies;
|
||||
|
||||
deterministicRandom()->randomShuffle(addrs);
|
||||
loop {
|
||||
|
@ -758,7 +780,7 @@ ACTOR Future<MonitorLeaderInfo> monitorProxiesOneGeneration( Reference<ClusterCo
|
|||
connFile->notifyConnected();
|
||||
|
||||
auto& ni = rep.get().mutate();
|
||||
shrinkProxyList(ni, lastProxyUIDs, lastProxies);
|
||||
shrinkProxyList(ni, lastProxyUIDs, lastProxies, lastGrvProxyUIDs, lastGrvProxies);
|
||||
clientInfo->set( ni );
|
||||
successIdx = idx;
|
||||
} else {
|
||||
|
|
|
@ -67,7 +67,8 @@ Future<Void> monitorLeaderForProxies( Value const& key, vector<NetworkAddress> c
|
|||
|
||||
Future<Void> monitorProxies( Reference<AsyncVar<Reference<ClusterConnectionFile>>> const& connFile, Reference<AsyncVar<ClientDBInfo>> const& clientInfo, Reference<ReferencedObject<Standalone<VectorRef<ClientVersionRef>>>> const& supportedVersions, Key const& traceLogGroup );
|
||||
|
||||
void shrinkProxyList( ClientDBInfo& ni, std::vector<UID>& lastProxyUIDs, std::vector<MasterProxyInterface>& lastProxies );
|
||||
void shrinkProxyList( ClientDBInfo& ni, std::vector<UID>& lastMasterProxyUIDs, std::vector<MasterProxyInterface>& lastMasterProxies,
|
||||
std::vector<UID>& lastGrvProxyUIDs, std::vector<GrvProxyInterface>& lastGrvProxies);
|
||||
|
||||
#ifndef __INTEL_COMPILER
|
||||
#pragma region Implementation
|
||||
|
|
|
@ -483,14 +483,17 @@ ACTOR static Future<Void> clientStatusUpdateActor(DatabaseContext *cx) {
|
|||
}
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> monitorMasterProxiesChange(Reference<AsyncVar<ClientDBInfo>> clientDBInfo, AsyncTrigger *triggerVar) {
|
||||
ACTOR static Future<Void> monitorProxiesChange(Reference<AsyncVar<ClientDBInfo>> clientDBInfo, AsyncTrigger *triggerVar) {
|
||||
state vector< MasterProxyInterface > curProxies;
|
||||
curProxies = clientDBInfo->get().proxies;
|
||||
state vector< GrvProxyInterface > curGrvProxies;
|
||||
curProxies = clientDBInfo->get().masterProxies;
|
||||
curGrvProxies = clientDBInfo->get().grvProxies;
|
||||
|
||||
loop{
|
||||
wait(clientDBInfo->onChange());
|
||||
if (clientDBInfo->get().proxies != curProxies) {
|
||||
curProxies = clientDBInfo->get().proxies;
|
||||
if (clientDBInfo->get().masterProxies != curProxies || clientDBInfo->get().grvProxies != curGrvProxies) {
|
||||
curProxies = clientDBInfo->get().masterProxies;
|
||||
curGrvProxies = clientDBInfo->get().grvProxies;
|
||||
triggerVar->trigger();
|
||||
}
|
||||
}
|
||||
|
@ -671,9 +674,9 @@ ACTOR static Future<HealthMetrics> getHealthMetricsActor(DatabaseContext *cx, bo
|
|||
CLIENT_KNOBS->DETAILED_HEALTH_METRICS_MAX_STALENESS;
|
||||
loop {
|
||||
choose {
|
||||
when(wait(cx->onMasterProxiesChanged())) {}
|
||||
when(wait(cx->onProxiesChanged())) {}
|
||||
when(GetHealthMetricsReply rep =
|
||||
wait(basicLoadBalance(cx->getMasterProxies(false), &MasterProxyInterface::getHealthMetrics,
|
||||
wait(basicLoadBalance(cx->getGrvProxies(false), &GrvProxyInterface::getHealthMetrics,
|
||||
GetHealthMetricsRequest(sendDetailedRequest)))) {
|
||||
cx->healthMetrics.update(rep.healthMetrics, detailed, true);
|
||||
if (detailed) {
|
||||
|
@ -846,9 +849,8 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<ClusterConnectionF
|
|||
bool switchable)
|
||||
: connectionFile(connectionFile), clientInfo(clientInfo), clientInfoMonitor(clientInfoMonitor), taskID(taskID),
|
||||
clientLocality(clientLocality), enableLocalityLoadBalance(enableLocalityLoadBalance), lockAware(lockAware),
|
||||
apiVersion(apiVersion), switchable(switchable), provisional(false), cc("TransactionMetrics"),
|
||||
transactionReadVersions("ReadVersions", cc),
|
||||
transactionReadVersionsThrottled("ReadVersionsThrottled", cc),
|
||||
apiVersion(apiVersion), switchable(switchable), proxyProvisional(false), cc("TransactionMetrics"),
|
||||
transactionReadVersions("ReadVersions", cc), transactionReadVersionsThrottled("ReadVersionsThrottled", cc),
|
||||
transactionReadVersionsCompleted("ReadVersionsCompleted", cc),
|
||||
transactionReadVersionBatches("ReadVersionBatches", cc),
|
||||
transactionBatchReadVersions("BatchPriorityReadVersions", cc),
|
||||
|
@ -875,9 +877,13 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<ClusterConnectionF
|
|||
transactionsProcessBehind("ProcessBehind", cc), outstandingWatches(0), latencies(1000), readLatencies(1000),
|
||||
commitLatencies(1000), GRVLatencies(1000), mutationsPerCommit(1000), bytesPerCommit(1000), mvCacheInsertLocation(0),
|
||||
healthMetricsLastUpdated(0), detailedHealthMetricsLastUpdated(0), internal(internal),
|
||||
smoothMidShardSize(CLIENT_KNOBS->SHARD_STAT_SMOOTH_AMOUNT),
|
||||
transactionsExpensiveClearCostEstCount("ExpensiveClearCostEstCount", cc),
|
||||
specialKeySpace(std::make_unique<SpecialKeySpace>(specialKeys.begin, specialKeys.end, /* test */ false)) {
|
||||
dbId = deterministicRandom()->randomUniqueID();
|
||||
connected = clientInfo->get().proxies.size() ? Void() : clientInfo->onChange();
|
||||
connected = (clientInfo->get().masterProxies.size() && clientInfo->get().grvProxies.size())
|
||||
? Void()
|
||||
: clientInfo->onChange();
|
||||
|
||||
metadataVersionCache.resize(CLIENT_KNOBS->METADATA_VERSION_CACHE_SIZE);
|
||||
maxOutstandingWatches = CLIENT_KNOBS->DEFAULT_MAX_OUTSTANDING_WATCHES;
|
||||
|
@ -892,9 +898,12 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<ClusterConnectionF
|
|||
getValueSubmitted.init(LiteralStringRef("NativeAPI.GetValueSubmitted"));
|
||||
getValueCompleted.init(LiteralStringRef("NativeAPI.GetValueCompleted"));
|
||||
|
||||
monitorMasterProxiesInfoChange = monitorMasterProxiesChange(clientInfo, &masterProxiesChangeTrigger);
|
||||
monitorProxiesInfoChange = monitorProxiesChange(clientInfo, &proxiesChangeTrigger);
|
||||
clientStatusUpdater.actor = clientStatusUpdateActor(this);
|
||||
cacheListMonitor = monitorCacheList(this);
|
||||
|
||||
smoothMidShardSize.reset(CLIENT_KNOBS->INIT_MID_SHARD_BYTES);
|
||||
|
||||
if (apiVersionAtLeast(700)) {
|
||||
registerSpecialKeySpaceModule(SpecialKeySpace::MODULE::ERRORMSG, SpecialKeySpace::IMPLTYPE::READONLY,
|
||||
std::make_unique<SingleSpecialKeyImpl>(
|
||||
|
@ -1000,22 +1009,36 @@ DatabaseContext::DatabaseContext(Reference<AsyncVar<Reference<ClusterConnectionF
|
|||
}
|
||||
}
|
||||
|
||||
DatabaseContext::DatabaseContext( const Error &err ) : deferredError(err), cc("TransactionMetrics"), transactionReadVersions("ReadVersions", cc), transactionReadVersionsThrottled("ReadVersionsThrottled", cc),
|
||||
transactionReadVersionsCompleted("ReadVersionsCompleted", cc), transactionReadVersionBatches("ReadVersionBatches", cc), transactionBatchReadVersions("BatchPriorityReadVersions", cc),
|
||||
transactionDefaultReadVersions("DefaultPriorityReadVersions", cc), transactionImmediateReadVersions("ImmediatePriorityReadVersions", cc),
|
||||
transactionBatchReadVersionsCompleted("BatchPriorityReadVersionsCompleted", cc), transactionDefaultReadVersionsCompleted("DefaultPriorityReadVersionsCompleted", cc),
|
||||
transactionImmediateReadVersionsCompleted("ImmediatePriorityReadVersionsCompleted", cc), transactionLogicalReads("LogicalUncachedReads", cc), transactionPhysicalReads("PhysicalReadRequests", cc),
|
||||
transactionPhysicalReadsCompleted("PhysicalReadRequestsCompleted", cc), transactionGetKeyRequests("GetKeyRequests", cc), transactionGetValueRequests("GetValueRequests", cc),
|
||||
transactionGetRangeRequests("GetRangeRequests", cc), transactionWatchRequests("WatchRequests", cc), transactionGetAddressesForKeyRequests("GetAddressesForKeyRequests", cc),
|
||||
transactionBytesRead("BytesRead", cc), transactionKeysRead("KeysRead", cc), transactionMetadataVersionReads("MetadataVersionReads", cc), transactionCommittedMutations("CommittedMutations", cc),
|
||||
transactionCommittedMutationBytes("CommittedMutationBytes", cc), transactionSetMutations("SetMutations", cc), transactionClearMutations("ClearMutations", cc),
|
||||
transactionAtomicMutations("AtomicMutations", cc), transactionsCommitStarted("CommitStarted", cc), transactionsCommitCompleted("CommitCompleted", cc),
|
||||
transactionKeyServerLocationRequests("KeyServerLocationRequests", cc), transactionKeyServerLocationRequestsCompleted("KeyServerLocationRequestsCompleted", cc), transactionsTooOld("TooOld", cc),
|
||||
transactionsFutureVersions("FutureVersions", cc), transactionsNotCommitted("NotCommitted", cc), transactionsMaybeCommitted("MaybeCommitted", cc),
|
||||
transactionsResourceConstrained("ResourceConstrained", cc), transactionsThrottled("Throttled", cc), transactionsProcessBehind("ProcessBehind", cc), latencies(1000), readLatencies(1000), commitLatencies(1000),
|
||||
GRVLatencies(1000), mutationsPerCommit(1000), bytesPerCommit(1000),
|
||||
internal(false) {}
|
||||
|
||||
DatabaseContext::DatabaseContext(const Error& err)
|
||||
: deferredError(err), cc("TransactionMetrics"), transactionReadVersions("ReadVersions", cc),
|
||||
transactionReadVersionsThrottled("ReadVersionsThrottled", cc),
|
||||
transactionReadVersionsCompleted("ReadVersionsCompleted", cc),
|
||||
transactionReadVersionBatches("ReadVersionBatches", cc),
|
||||
transactionBatchReadVersions("BatchPriorityReadVersions", cc),
|
||||
transactionDefaultReadVersions("DefaultPriorityReadVersions", cc),
|
||||
transactionImmediateReadVersions("ImmediatePriorityReadVersions", cc),
|
||||
transactionBatchReadVersionsCompleted("BatchPriorityReadVersionsCompleted", cc),
|
||||
transactionDefaultReadVersionsCompleted("DefaultPriorityReadVersionsCompleted", cc),
|
||||
transactionImmediateReadVersionsCompleted("ImmediatePriorityReadVersionsCompleted", cc),
|
||||
transactionLogicalReads("LogicalUncachedReads", cc), transactionPhysicalReads("PhysicalReadRequests", cc),
|
||||
transactionPhysicalReadsCompleted("PhysicalReadRequestsCompleted", cc),
|
||||
transactionGetKeyRequests("GetKeyRequests", cc), transactionGetValueRequests("GetValueRequests", cc),
|
||||
transactionGetRangeRequests("GetRangeRequests", cc), transactionWatchRequests("WatchRequests", cc),
|
||||
transactionGetAddressesForKeyRequests("GetAddressesForKeyRequests", cc), transactionBytesRead("BytesRead", cc),
|
||||
transactionKeysRead("KeysRead", cc), transactionMetadataVersionReads("MetadataVersionReads", cc),
|
||||
transactionCommittedMutations("CommittedMutations", cc),
|
||||
transactionCommittedMutationBytes("CommittedMutationBytes", cc), transactionSetMutations("SetMutations", cc),
|
||||
transactionClearMutations("ClearMutations", cc), transactionAtomicMutations("AtomicMutations", cc),
|
||||
transactionsCommitStarted("CommitStarted", cc), transactionsCommitCompleted("CommitCompleted", cc),
|
||||
transactionKeyServerLocationRequests("KeyServerLocationRequests", cc),
|
||||
transactionKeyServerLocationRequestsCompleted("KeyServerLocationRequestsCompleted", cc),
|
||||
transactionsTooOld("TooOld", cc), transactionsFutureVersions("FutureVersions", cc),
|
||||
transactionsNotCommitted("NotCommitted", cc), transactionsMaybeCommitted("MaybeCommitted", cc),
|
||||
transactionsResourceConstrained("ResourceConstrained", cc), transactionsThrottled("Throttled", cc),
|
||||
transactionsProcessBehind("ProcessBehind", cc), latencies(1000), readLatencies(1000), commitLatencies(1000),
|
||||
GRVLatencies(1000), mutationsPerCommit(1000), bytesPerCommit(1000),
|
||||
smoothMidShardSize(CLIENT_KNOBS->SHARD_STAT_SMOOTH_AMOUNT),
|
||||
transactionsExpensiveClearCostEstCount("ExpensiveClearCostEstCount", cc), internal(false) {}
|
||||
|
||||
Database DatabaseContext::create(Reference<AsyncVar<ClientDBInfo>> clientInfo, Future<Void> clientInfoMonitor, LocalityData clientLocality, bool enableLocalityLoadBalance, TaskPriority taskID, bool lockAware, int apiVersion, bool switchable) {
|
||||
return Database( new DatabaseContext( Reference<AsyncVar<Reference<ClusterConnectionFile>>>(), clientInfo, clientInfoMonitor, taskID, clientLocality, enableLocalityLoadBalance, lockAware, true, apiVersion, switchable ) );
|
||||
|
@ -1023,7 +1046,7 @@ Database DatabaseContext::create(Reference<AsyncVar<ClientDBInfo>> clientInfo, F
|
|||
|
||||
DatabaseContext::~DatabaseContext() {
|
||||
cacheListMonitor.cancel();
|
||||
monitorMasterProxiesInfoChange.cancel();
|
||||
monitorProxiesInfoChange.cancel();
|
||||
for(auto it = server_interf.begin(); it != server_interf.end(); it = server_interf.erase(it))
|
||||
it->second->notifyContextDestroyed();
|
||||
ASSERT_ABORT( server_interf.empty() );
|
||||
|
@ -1102,14 +1125,19 @@ void DatabaseContext::invalidateCache( const KeyRangeRef& keys ) {
|
|||
locationCache.insert(KeyRangeRef(begin, end), Reference<LocationInfo>());
|
||||
}
|
||||
|
||||
Future<Void> DatabaseContext::onMasterProxiesChanged() {
|
||||
return this->masterProxiesChangeTrigger.onTrigger();
|
||||
Future<Void> DatabaseContext::onProxiesChanged() {
|
||||
return this->proxiesChangeTrigger.onTrigger();
|
||||
}
|
||||
|
||||
bool DatabaseContext::sampleReadTags() {
|
||||
bool DatabaseContext::sampleReadTags() const {
|
||||
return clientInfo->get().transactionTagSampleRate > 0 && deterministicRandom()->random01() <= clientInfo->get().transactionTagSampleRate;
|
||||
}
|
||||
|
||||
bool DatabaseContext::sampleOnCost(uint64_t cost) const {
|
||||
if (clientInfo->get().transactionTagSampleCost <= 0) return false;
|
||||
return deterministicRandom()->random01() <= (double)cost / clientInfo->get().transactionTagSampleCost;
|
||||
}
|
||||
|
||||
int64_t extractIntOption( Optional<StringRef> value, int64_t minValue, int64_t maxValue ) {
|
||||
validateOptionValue(value, true);
|
||||
if( value.get().size() != 8 ) {
|
||||
|
@ -1146,8 +1174,10 @@ void DatabaseContext::setOption( FDBDatabaseOptions::Option option, Optional<Str
|
|||
break;
|
||||
case FDBDatabaseOptions::MACHINE_ID:
|
||||
clientLocality = LocalityData( clientLocality.processId(), value.present() ? Standalone<StringRef>(value.get()) : Optional<Standalone<StringRef>>(), clientLocality.machineId(), clientLocality.dcId() );
|
||||
if( clientInfo->get().proxies.size() )
|
||||
masterProxies = Reference<ProxyInfo>( new ProxyInfo( clientInfo->get().proxies ) );
|
||||
if( clientInfo->get().masterProxies.size() )
|
||||
masterProxies = Reference<ProxyInfo>( new ProxyInfo( clientInfo->get().masterProxies) );
|
||||
if( clientInfo->get().grvProxies.size() )
|
||||
grvProxies = Reference<GrvProxyInfo>( new GrvProxyInfo( clientInfo->get().grvProxies ) );
|
||||
server_interf.clear();
|
||||
locationCache.insert( allKeys, Reference<LocationInfo>() );
|
||||
break;
|
||||
|
@ -1156,8 +1186,10 @@ void DatabaseContext::setOption( FDBDatabaseOptions::Option option, Optional<Str
|
|||
break;
|
||||
case FDBDatabaseOptions::DATACENTER_ID:
|
||||
clientLocality = LocalityData(clientLocality.processId(), clientLocality.zoneId(), clientLocality.machineId(), value.present() ? Standalone<StringRef>(value.get()) : Optional<Standalone<StringRef>>());
|
||||
if( clientInfo->get().proxies.size() )
|
||||
masterProxies = Reference<ProxyInfo>( new ProxyInfo( clientInfo->get().proxies ));
|
||||
if( clientInfo->get().masterProxies.size() )
|
||||
masterProxies = Reference<ProxyInfo>( new ProxyInfo( clientInfo->get().masterProxies));
|
||||
if( clientInfo->get().grvProxies.size() )
|
||||
grvProxies = Reference<GrvProxyInfo>( new GrvProxyInfo( clientInfo->get().grvProxies ));
|
||||
server_interf.clear();
|
||||
locationCache.insert( allKeys, Reference<LocationInfo>() );
|
||||
break;
|
||||
|
@ -1199,11 +1231,13 @@ ACTOR static Future<Void> switchConnectionFileImpl(Reference<ClusterConnectionFi
|
|||
|
||||
// Reset state from former cluster.
|
||||
self->masterProxies.clear();
|
||||
self->grvProxies.clear();
|
||||
self->minAcceptableReadVersion = std::numeric_limits<Version>::max();
|
||||
self->invalidateCache(allKeys);
|
||||
|
||||
auto clearedClientInfo = self->clientInfo->get();
|
||||
clearedClientInfo.proxies.clear();
|
||||
clearedClientInfo.masterProxies.clear();
|
||||
clearedClientInfo.grvProxies.clear();
|
||||
clearedClientInfo.id = deterministicRandom()->randomUniqueID();
|
||||
self->clientInfo->set(clearedClientInfo);
|
||||
self->connectionFile->set(connFile);
|
||||
|
@ -1534,28 +1568,49 @@ void stopNetwork() {
|
|||
closeTraceFile();
|
||||
}
|
||||
|
||||
Reference<ProxyInfo> DatabaseContext::getMasterProxies(bool useProvisionalProxies) {
|
||||
if (masterProxiesLastChange != clientInfo->get().id) {
|
||||
masterProxiesLastChange = clientInfo->get().id;
|
||||
masterProxies.clear();
|
||||
if( clientInfo->get().proxies.size() ) {
|
||||
masterProxies = Reference<ProxyInfo>( new ProxyInfo( clientInfo->get().proxies ));
|
||||
provisional = clientInfo->get().proxies[0].provisional;
|
||||
}
|
||||
void DatabaseContext::updateProxies() {
|
||||
if (proxiesLastChange == clientInfo->get().id) return;
|
||||
proxiesLastChange = clientInfo->get().id;
|
||||
masterProxies.clear();
|
||||
grvProxies.clear();
|
||||
bool masterProxyProvisional = false, grvProxyProvisional = false;
|
||||
if (clientInfo->get().masterProxies.size()) {
|
||||
masterProxies = Reference<ProxyInfo>(new ProxyInfo(clientInfo->get().masterProxies));
|
||||
masterProxyProvisional = clientInfo->get().masterProxies[0].provisional;
|
||||
}
|
||||
if(provisional && !useProvisionalProxies) {
|
||||
if (clientInfo->get().grvProxies.size()) {
|
||||
grvProxies = Reference<GrvProxyInfo>(new GrvProxyInfo(clientInfo->get().grvProxies));
|
||||
grvProxyProvisional = clientInfo->get().grvProxies[0].provisional;
|
||||
}
|
||||
if (clientInfo->get().masterProxies.size() && clientInfo->get().grvProxies.size()) {
|
||||
ASSERT(masterProxyProvisional == grvProxyProvisional);
|
||||
proxyProvisional = masterProxyProvisional;
|
||||
}
|
||||
}
|
||||
|
||||
Reference<ProxyInfo> DatabaseContext::getMasterProxies(bool useProvisionalProxies) {
|
||||
updateProxies();
|
||||
if (proxyProvisional && !useProvisionalProxies) {
|
||||
return Reference<ProxyInfo>();
|
||||
}
|
||||
return masterProxies;
|
||||
}
|
||||
|
||||
Reference<GrvProxyInfo> DatabaseContext::getGrvProxies(bool useProvisionalProxies) {
|
||||
updateProxies();
|
||||
if (proxyProvisional && !useProvisionalProxies) {
|
||||
return Reference<GrvProxyInfo>();
|
||||
}
|
||||
return grvProxies;
|
||||
}
|
||||
|
||||
//Actor which will wait until the MultiInterface<MasterProxyInterface> returned by the DatabaseContext cx is not NULL
|
||||
ACTOR Future<Reference<ProxyInfo>> getMasterProxiesFuture(DatabaseContext *cx, bool useProvisionalProxies) {
|
||||
loop{
|
||||
Reference<ProxyInfo> proxies = cx->getMasterProxies(useProvisionalProxies);
|
||||
if (proxies)
|
||||
return proxies;
|
||||
wait( cx->onMasterProxiesChanged() );
|
||||
wait( cx->onProxiesChanged() );
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -1687,8 +1742,8 @@ ACTOR Future<pair<KeyRange, Reference<LocationInfo>>> getKeyLocation_internal(Da
|
|||
loop {
|
||||
++cx->transactionKeyServerLocationRequests;
|
||||
choose {
|
||||
when ( wait( cx->onMasterProxiesChanged() ) ) {}
|
||||
when(GetKeyServerLocationsReply rep = wait(basicLoadBalance(
|
||||
when (wait(cx->onProxiesChanged())) {}
|
||||
when (GetKeyServerLocationsReply rep = wait(basicLoadBalance(
|
||||
cx->getMasterProxies(info.useProvisionalProxies), &MasterProxyInterface::getKeyServersLocations,
|
||||
GetKeyServerLocationsRequest(span.context, key, Optional<KeyRef>(), 100, isBackward, key.arena()),
|
||||
TaskPriority::DefaultPromiseEndpoint))) {
|
||||
|
@ -1736,8 +1791,8 @@ ACTOR Future<vector<pair<KeyRange, Reference<LocationInfo>>>> getKeyRangeLocatio
|
|||
loop {
|
||||
++cx->transactionKeyServerLocationRequests;
|
||||
choose {
|
||||
when ( wait( cx->onMasterProxiesChanged() ) ) {}
|
||||
when(GetKeyServerLocationsReply _rep = wait(basicLoadBalance(
|
||||
when ( wait( cx->onProxiesChanged() ) ) {}
|
||||
when ( GetKeyServerLocationsReply _rep = wait(basicLoadBalance(
|
||||
cx->getMasterProxies(info.useProvisionalProxies), &MasterProxyInterface::getKeyServersLocations,
|
||||
GetKeyServerLocationsRequest(span.context, keys.begin, keys.end, limit, reverse, keys.arena()),
|
||||
TaskPriority::DefaultPromiseEndpoint))) {
|
||||
|
@ -1998,12 +2053,12 @@ ACTOR Future<Version> waitForCommittedVersion( Database cx, Version version, Spa
|
|||
try {
|
||||
loop {
|
||||
choose {
|
||||
when ( wait( cx->onMasterProxiesChanged() ) ) {}
|
||||
when(GetReadVersionReply v = wait(basicLoadBalance(
|
||||
cx->getMasterProxies(false), &MasterProxyInterface::getConsistentReadVersion,
|
||||
when ( wait( cx->onProxiesChanged() ) ) {}
|
||||
when( GetReadVersionReply v = wait(basicLoadBalance(
|
||||
cx->getGrvProxies(false), &GrvProxyInterface::getConsistentReadVersion,
|
||||
GetReadVersionRequest(span.context, 0, TransactionPriority::IMMEDIATE), cx->taskID))) {
|
||||
cx->minAcceptableReadVersion = std::min(cx->minAcceptableReadVersion, v.version);
|
||||
|
||||
if (v.midShardSize > 0) cx->smoothMidShardSize.setTotal(v.midShardSize);
|
||||
if (v.version >= version)
|
||||
return v.version;
|
||||
// SOMEDAY: Do the wait on the server side, possibly use less expensive source of committed version (causal consistency is not needed for this purpose)
|
||||
|
@ -2021,9 +2076,9 @@ ACTOR Future<Version> getRawVersion( Database cx, SpanID spanContext ) {
|
|||
state Span span("NAPI:getRawVersion"_loc, { spanContext });
|
||||
loop {
|
||||
choose {
|
||||
when ( wait( cx->onMasterProxiesChanged() ) ) {}
|
||||
when(GetReadVersionReply v =
|
||||
wait(basicLoadBalance(cx->getMasterProxies(false), &MasterProxyInterface::getConsistentReadVersion,
|
||||
when ( wait( cx->onProxiesChanged() ) ) {}
|
||||
when ( GetReadVersionReply v =
|
||||
wait(basicLoadBalance(cx->getGrvProxies(false), &GrvProxyInterface::getConsistentReadVersion,
|
||||
GetReadVersionRequest(spanContext, 0, TransactionPriority::IMMEDIATE), cx->taskID))) {
|
||||
return v.version;
|
||||
}
|
||||
|
@ -3019,7 +3074,6 @@ void Transaction::clear( const KeyRef& key, bool addConflictRange ) {
|
|||
data[key.size()] = 0;
|
||||
t.mutations.emplace_back(req.arena, MutationRef::ClearRange, KeyRef(data, key.size()),
|
||||
KeyRef(data, key.size() + 1));
|
||||
|
||||
if(addConflictRange)
|
||||
t.write_conflict_ranges.emplace_back(req.arena, KeyRef(data, key.size()), KeyRef(data, key.size() + 1));
|
||||
}
|
||||
|
@ -3302,33 +3356,73 @@ void Transaction::setupWatches() {
|
|||
}
|
||||
}
|
||||
|
||||
ACTOR Future<TransactionCommitCostEstimation> estimateCommitCosts(Transaction* self,
|
||||
CommitTransactionRef* transaction) {
|
||||
state MutationRef* it = transaction->mutations.begin();
|
||||
state MutationRef* end = transaction->mutations.end();
|
||||
state TransactionCommitCostEstimation trCommitCosts;
|
||||
state KeyRange keyRange;
|
||||
for (; it != end; ++it) {
|
||||
if (it->type == MutationRef::Type::SetValue) {
|
||||
trCommitCosts.bytesWrite += it->expectedSize();
|
||||
trCommitCosts.numWrite++;
|
||||
} else if (it->isAtomicOp()) {
|
||||
trCommitCosts.bytesAtomicWrite += it->expectedSize();
|
||||
trCommitCosts.numAtomicWrite++;
|
||||
ACTOR Future<Optional<ClientTrCommitCostEstimation>> estimateCommitCosts(Transaction* self,
|
||||
CommitTransactionRef const* transaction) {
|
||||
state ClientTrCommitCostEstimation trCommitCosts;
|
||||
state KeyRangeRef keyRange;
|
||||
state int i = 0;
|
||||
|
||||
for (; i < transaction->mutations.size(); ++i) {
|
||||
auto* it = &transaction->mutations[i];
|
||||
|
||||
if (it->type == MutationRef::Type::SetValue || it->isAtomicOp()) {
|
||||
trCommitCosts.opsCount++;
|
||||
trCommitCosts.writeCosts += getWriteOperationCost(it->expectedSize());
|
||||
} else if (it->type == MutationRef::Type::ClearRange) {
|
||||
trCommitCosts.numClear++;
|
||||
keyRange = KeyRange(KeyRangeRef(it->param1, it->param2));
|
||||
trCommitCosts.opsCount++;
|
||||
keyRange = KeyRangeRef(it->param1, it->param2);
|
||||
if (self->options.expensiveClearCostEstimation) {
|
||||
StorageMetrics m = wait(self->getStorageMetrics(keyRange, std::numeric_limits<int>::max()));
|
||||
trCommitCosts.bytesClearEst += m.bytes;
|
||||
StorageMetrics m = wait(self->getStorageMetrics(keyRange, CLIENT_KNOBS->TOO_MANY));
|
||||
trCommitCosts.clearIdxCosts.emplace_back(i, getWriteOperationCost(m.bytes));
|
||||
trCommitCosts.writeCosts += getWriteOperationCost(m.bytes);
|
||||
++trCommitCosts.expensiveCostEstCount;
|
||||
++self->getDatabase()->transactionsExpensiveClearCostEstCount;
|
||||
}
|
||||
else {
|
||||
std::vector<pair<KeyRange, Reference<LocationInfo>>> locations = wait(getKeyRangeLocations(
|
||||
self->getDatabase(), keyRange, std::numeric_limits<int>::max(), false, &StorageServerInterface::getShardState, self->info));
|
||||
trCommitCosts.numClearShards += locations.size();
|
||||
std::vector<pair<KeyRange, Reference<LocationInfo>>> locations =
|
||||
wait(getKeyRangeLocations(self->getDatabase(), keyRange, CLIENT_KNOBS->TOO_MANY, false,
|
||||
&StorageServerInterface::getShardState, self->info));
|
||||
if (locations.empty()) continue;
|
||||
|
||||
uint64_t bytes = 0;
|
||||
if (locations.size() == 1) {
|
||||
bytes = CLIENT_KNOBS->INCOMPLETE_SHARD_PLUS;
|
||||
} else { // small clear on the boundary will hit two shards but be much smaller than the shard size
|
||||
bytes = CLIENT_KNOBS->INCOMPLETE_SHARD_PLUS * 2 +
|
||||
(locations.size() - 2) * (int64_t)self->getDatabase()->smoothMidShardSize.smoothTotal();
|
||||
}
|
||||
|
||||
trCommitCosts.clearIdxCosts.emplace_back(i, getWriteOperationCost(bytes));
|
||||
trCommitCosts.writeCosts += getWriteOperationCost(bytes);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
// sample on written bytes
|
||||
if (!self->getDatabase()->sampleOnCost(trCommitCosts.writeCosts)) return Optional<ClientTrCommitCostEstimation>();
|
||||
|
||||
// sample clear op: the expectation of #sampledOp is every COMMIT_SAMPLE_COST sample once
|
||||
// we also scale the cost of mutations whose cost is less than COMMIT_SAMPLE_COST as scaledCost =
|
||||
// min(COMMIT_SAMPLE_COST, cost) If we have 4 transactions: A - 100 1-cost mutations: E[sampled ops] = 1, E[sampled
|
||||
// cost] = 100 B - 1 100-cost mutation: E[sampled ops] = 1, E[sampled cost] = 100 C - 50 2-cost mutations: E[sampled
|
||||
// ops] = 1, E[sampled cost] = 100 D - 1 150-cost mutation and 150 1-cost mutations: E[sampled ops] = 3, E[sampled
|
||||
// cost] = 150cost * 1 + 150 * 100cost * 0.01 = 300
|
||||
ASSERT(trCommitCosts.writeCosts > 0);
|
||||
std::deque<std::pair<int, uint64_t>> newClearIdxCosts;
|
||||
for (const auto& [idx, cost] : trCommitCosts.clearIdxCosts) {
|
||||
if (trCommitCosts.writeCosts >= CLIENT_KNOBS->COMMIT_SAMPLE_COST) {
|
||||
double mul = trCommitCosts.writeCosts / std::max(1.0, (double)CLIENT_KNOBS->COMMIT_SAMPLE_COST);
|
||||
if (deterministicRandom()->random01() < cost * mul / trCommitCosts.writeCosts) {
|
||||
newClearIdxCosts.emplace_back(
|
||||
idx, cost < CLIENT_KNOBS->COMMIT_SAMPLE_COST ? CLIENT_KNOBS->COMMIT_SAMPLE_COST : cost);
|
||||
}
|
||||
} else if (deterministicRandom()->random01() < (double)cost / trCommitCosts.writeCosts) {
|
||||
newClearIdxCosts.emplace_back(
|
||||
idx, cost < CLIENT_KNOBS->COMMIT_SAMPLE_COST ? CLIENT_KNOBS->COMMIT_SAMPLE_COST : cost);
|
||||
}
|
||||
}
|
||||
|
||||
trCommitCosts.clearIdxCosts.swap(newClearIdxCosts);
|
||||
return trCommitCosts;
|
||||
}
|
||||
|
||||
|
@ -3348,11 +3442,11 @@ ACTOR static Future<Void> tryCommit( Database cx, Reference<TransactionLogInfo>
|
|||
commit_unknown_result()});
|
||||
}
|
||||
|
||||
if (!req.tagSet.present()) {
|
||||
wait(store(req.transaction.read_snapshot, readVersion));
|
||||
if (req.tagSet.present() && tr->options.priority < TransactionPriority::IMMEDIATE) {
|
||||
wait(store(req.transaction.read_snapshot, readVersion) &&
|
||||
store(req.commitCostEstimation, estimateCommitCosts(tr, &req.transaction)));
|
||||
} else {
|
||||
req.commitCostEstimation = TransactionCommitCostEstimation();
|
||||
wait(store(req.transaction.read_snapshot, readVersion) && store(req.commitCostEstimation.get(), estimateCommitCosts(tr, &req.transaction)));
|
||||
wait(store(req.transaction.read_snapshot, readVersion));
|
||||
}
|
||||
|
||||
startTime = now();
|
||||
|
@ -3369,7 +3463,7 @@ ACTOR static Future<Void> tryCommit( Database cx, Reference<TransactionLogInfo>
|
|||
if(cx->clientInfo->get().firstProxy.present()) {
|
||||
reply = throwErrorOr ( brokenPromiseToMaybeDelivered ( cx->clientInfo->get().firstProxy.get().commit.tryGetReply(req) ) );
|
||||
} else {
|
||||
const std::vector<MasterProxyInterface>& proxies = cx->clientInfo->get().proxies;
|
||||
const std::vector<MasterProxyInterface>& proxies = cx->clientInfo->get().masterProxies;
|
||||
reply = proxies.size() ? throwErrorOr ( brokenPromiseToMaybeDelivered ( proxies[0].commit.tryGetReply(req) ) ) : Never();
|
||||
}
|
||||
} else {
|
||||
|
@ -3377,7 +3471,7 @@ ACTOR static Future<Void> tryCommit( Database cx, Reference<TransactionLogInfo>
|
|||
}
|
||||
|
||||
choose {
|
||||
when ( wait( cx->onMasterProxiesChanged() ) ) {
|
||||
when ( wait( cx->onProxiesChanged() ) ) {
|
||||
reply.cancel();
|
||||
throw request_maybe_delivered();
|
||||
}
|
||||
|
@ -3677,7 +3771,7 @@ void Transaction::setOption( FDBTransactionOptions::Option option, Optional<Stri
|
|||
if (info.debugID.present()) {
|
||||
TraceEvent(SevInfo, "TransactionBeingTraced")
|
||||
.detail("DebugTransactionID", trLogInfo->identifier)
|
||||
.detail("ServerTraceID", info.debugID.get().first());
|
||||
.detail("ServerTraceID", info.debugID.get());
|
||||
|
||||
}
|
||||
break;
|
||||
|
@ -3713,7 +3807,7 @@ void Transaction::setOption( FDBTransactionOptions::Option option, Optional<Stri
|
|||
if (trLogInfo && !trLogInfo->identifier.empty()) {
|
||||
TraceEvent(SevInfo, "TransactionBeingTraced")
|
||||
.detail("DebugTransactionID", trLogInfo->identifier)
|
||||
.detail("ServerTraceID", info.debugID.get().first());
|
||||
.detail("ServerTraceID", info.debugID.get());
|
||||
}
|
||||
break;
|
||||
|
||||
|
@ -3780,7 +3874,7 @@ void Transaction::setOption( FDBTransactionOptions::Option option, Optional<Stri
|
|||
validateOptionValue(value, false);
|
||||
options.reportConflictingKeys = true;
|
||||
break;
|
||||
|
||||
|
||||
case FDBTransactionOptions::EXPENSIVE_CLEAR_COST_ESTIMATION_ENABLE:
|
||||
validateOptionValue(value, false);
|
||||
options.expensiveClearCostEstimation = true;
|
||||
|
@ -3802,8 +3896,8 @@ ACTOR Future<GetReadVersionReply> getConsistentReadVersion(SpanID parentSpan, Da
|
|||
loop {
|
||||
state GetReadVersionRequest req( span.context, transactionCount, priority, flags, tags, debugID );
|
||||
choose {
|
||||
when ( wait( cx->onMasterProxiesChanged() ) ) {}
|
||||
when ( GetReadVersionReply v = wait( basicLoadBalance( cx->getMasterProxies(flags & GetReadVersionRequest::FLAG_USE_PROVISIONAL_PROXIES), &MasterProxyInterface::getConsistentReadVersion, req, cx->taskID ) ) ) {
|
||||
when ( wait( cx->onProxiesChanged() ) ) {}
|
||||
when ( GetReadVersionReply v = wait( basicLoadBalance( cx->getGrvProxies(flags & GetReadVersionRequest::FLAG_USE_PROVISIONAL_PROXIES), &GrvProxyInterface::getConsistentReadVersion, req, cx->taskID ) ) ) {
|
||||
if(tags.size() != 0) {
|
||||
auto &priorityThrottledTags = cx->throttledTags[priority];
|
||||
for(auto& tag : tags) {
|
||||
|
@ -4107,9 +4201,9 @@ Future<Void> Transaction::onError( Error const& e ) {
|
|||
|
||||
return e;
|
||||
}
|
||||
ACTOR Future<StorageMetrics> getStorageMetricsLargeKeyRange(Database cx, KeyRangeRef keys);
|
||||
ACTOR Future<StorageMetrics> getStorageMetricsLargeKeyRange(Database cx, KeyRange keys);
|
||||
|
||||
ACTOR Future<StorageMetrics> doGetStorageMetrics(Database cx, KeyRangeRef keys, Reference<LocationInfo> locationInfo) {
|
||||
ACTOR Future<StorageMetrics> doGetStorageMetrics(Database cx, KeyRange keys, Reference<LocationInfo> locationInfo) {
|
||||
loop {
|
||||
try {
|
||||
WaitMetricsRequest req(keys, StorageMetrics(), StorageMetrics());
|
||||
|
@ -4131,7 +4225,7 @@ ACTOR Future<StorageMetrics> doGetStorageMetrics(Database cx, KeyRangeRef keys,
|
|||
}
|
||||
}
|
||||
|
||||
ACTOR Future<StorageMetrics> getStorageMetricsLargeKeyRange(Database cx, KeyRangeRef keys) {
|
||||
ACTOR Future<StorageMetrics> getStorageMetricsLargeKeyRange(Database cx, KeyRange keys) {
|
||||
state Span span("NAPI:GetStorageMetricsLargeKeyRange"_loc);
|
||||
vector<pair<KeyRange, Reference<LocationInfo>>> locations = wait(
|
||||
getKeyRangeLocations(cx, keys, std::numeric_limits<int>::max(), false, &StorageServerInterface::waitMetrics,
|
||||
|
@ -4243,13 +4337,22 @@ ACTOR Future<Standalone<VectorRef<ReadHotRangeWithMetrics>>> getReadHotRanges(Da
|
|||
}
|
||||
|
||||
wait(waitForAll(fReplies));
|
||||
Standalone<VectorRef<ReadHotRangeWithMetrics>> results;
|
||||
|
||||
for (int i = 0; i < nLocs; i++)
|
||||
results.append(results.arena(), fReplies[i].get().readHotRanges.begin(),
|
||||
fReplies[i].get().readHotRanges.size());
|
||||
if(nLocs == 1) {
|
||||
TEST(true); // Single-shard read hot range request
|
||||
return fReplies[0].get().readHotRanges;
|
||||
}
|
||||
else {
|
||||
TEST(true); // Multi-shard read hot range request
|
||||
Standalone<VectorRef<ReadHotRangeWithMetrics>> results;
|
||||
for (int i = 0; i < nLocs; i++) {
|
||||
results.append(results.arena(), fReplies[i].get().readHotRanges.begin(),
|
||||
fReplies[i].get().readHotRanges.size());
|
||||
results.arena().dependsOn(fReplies[i].get().readHotRanges.arena());
|
||||
}
|
||||
|
||||
return results;
|
||||
return results;
|
||||
}
|
||||
} catch (Error& e) {
|
||||
if (e.code() != error_code_wrong_shard_server && e.code() != error_code_all_alternatives_failed) {
|
||||
TraceEvent(SevError, "GetReadHotSubRangesError").error(e);
|
||||
|
@ -4338,7 +4441,7 @@ ACTOR Future<Standalone<VectorRef<DDMetricsRef>>> waitDataDistributionMetricsLis
|
|||
state Future<Void> clientTimeout = delay(5.0);
|
||||
loop {
|
||||
choose {
|
||||
when(wait(cx->onMasterProxiesChanged())) {}
|
||||
when(wait(cx->onProxiesChanged())) {}
|
||||
when(ErrorOr<GetDDMetricsReply> rep =
|
||||
wait(errorOr(basicLoadBalance(cx->getMasterProxies(false), &MasterProxyInterface::getDDMetrics,
|
||||
GetDDMetricsRequest(keys, shardLimit))))) {
|
||||
|
@ -4445,7 +4548,7 @@ ACTOR Future<Void> snapCreate(Database cx, Standalone<StringRef> snapCmd, UID sn
|
|||
try {
|
||||
loop {
|
||||
choose {
|
||||
when(wait(cx->onMasterProxiesChanged())) {}
|
||||
when(wait(cx->onProxiesChanged())) {}
|
||||
when(wait(basicLoadBalance(cx->getMasterProxies(false), &MasterProxyInterface::proxySnapReq, ProxySnapRequest(snapCmd, snapUID, snapUID), cx->taskID, true /*atmostOnce*/ ))) {
|
||||
TraceEvent("SnapCreateExit")
|
||||
.detail("SnapCmd", snapCmd.toString())
|
||||
|
@ -4472,7 +4575,7 @@ ACTOR Future<bool> checkSafeExclusions(Database cx, vector<AddressExclusion> exc
|
|||
try {
|
||||
loop {
|
||||
choose {
|
||||
when(wait(cx->onMasterProxiesChanged())) {}
|
||||
when(wait(cx->onProxiesChanged())) {}
|
||||
when(ExclusionSafetyCheckReply _ddCheck =
|
||||
wait(basicLoadBalance(cx->getMasterProxies(false), &MasterProxyInterface::exclusionSafetyCheckReq,
|
||||
req, cx->taskID))) {
|
||||
|
|
|
@ -357,5 +357,8 @@ ACTOR Future<Void> snapCreate(Database cx, Standalone<StringRef> snapCmd, UID sn
|
|||
// Checks with Data Distributor that it is safe to mark all servers in exclusions as failed
|
||||
ACTOR Future<bool> checkSafeExclusions(Database cx, vector<AddressExclusion> exclusions);
|
||||
|
||||
inline uint64_t getWriteOperationCost(uint64_t bytes) {
|
||||
return bytes / std::max(1, CLIENT_KNOBS->WRITE_COST_BYTE_FACTOR) + 1;
|
||||
}
|
||||
#include "flow/unactorcompiler.h"
|
||||
#endif
|
||||
|
|
|
@ -112,7 +112,7 @@ struct RestoreRoleInterface {
|
|||
|
||||
UID id() const { return nodeID; }
|
||||
|
||||
std::string toString() {
|
||||
std::string toString() const {
|
||||
std::stringstream ss;
|
||||
ss << "Role:" << getRoleStr(role) << " interfID:" << nodeID.toString();
|
||||
return ss.str();
|
||||
|
@ -201,7 +201,7 @@ struct RestoreApplierInterface : RestoreRoleInterface {
|
|||
collectRestoreRoleInterfaces, finishRestore);
|
||||
}
|
||||
|
||||
std::string toString() { return nodeID.toString(); }
|
||||
std::string toString() const { return nodeID.toString(); }
|
||||
};
|
||||
|
||||
struct RestoreControllerInterface : RestoreRoleInterface {
|
||||
|
@ -226,7 +226,7 @@ struct RestoreControllerInterface : RestoreRoleInterface {
|
|||
serializer(ar, *(RestoreRoleInterface*)this, samples);
|
||||
}
|
||||
|
||||
std::string toString() { return nodeID.toString(); }
|
||||
std::string toString() const { return nodeID.toString(); }
|
||||
};
|
||||
|
||||
// RestoreAsset uniquely identifies the work unit done by restore roles;
|
||||
|
@ -249,29 +249,31 @@ struct RestoreAsset {
|
|||
Key addPrefix;
|
||||
Key removePrefix;
|
||||
|
||||
int batchIndex; // for progress tracking and performance investigation
|
||||
|
||||
RestoreAsset() = default;
|
||||
|
||||
// Q: Can we simply use uid for == and use different comparison rule for less than operator.
|
||||
// The ordering of RestoreAsset may change, will that affect correctness or performance?
|
||||
bool operator==(const RestoreAsset& r) const {
|
||||
return beginVersion == r.beginVersion && endVersion == r.endVersion && range == r.range &&
|
||||
fileIndex == r.fileIndex && partitionId == r.partitionId && filename == r.filename &&
|
||||
return batchIndex == r.batchIndex && beginVersion == r.beginVersion && endVersion == r.endVersion &&
|
||||
range == r.range && fileIndex == r.fileIndex && partitionId == r.partitionId && filename == r.filename &&
|
||||
offset == r.offset && len == r.len && addPrefix == r.addPrefix && removePrefix == r.removePrefix;
|
||||
}
|
||||
bool operator!=(const RestoreAsset& r) const {
|
||||
return !(*this == r);
|
||||
}
|
||||
bool operator<(const RestoreAsset& r) const {
|
||||
return std::make_tuple(fileIndex, filename, offset, len, beginVersion, endVersion, range.begin, range.end,
|
||||
addPrefix, removePrefix) < std::make_tuple(r.fileIndex, r.filename, r.offset, r.len,
|
||||
r.beginVersion, r.endVersion, r.range.begin,
|
||||
r.range.end, r.addPrefix, r.removePrefix);
|
||||
return std::make_tuple(batchIndex, fileIndex, filename, offset, len, beginVersion, endVersion, range.begin,
|
||||
range.end, addPrefix, removePrefix) <
|
||||
std::make_tuple(r.batchIndex, r.fileIndex, r.filename, r.offset, r.len, r.beginVersion, r.endVersion,
|
||||
r.range.begin, r.range.end, r.addPrefix, r.removePrefix);
|
||||
}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, uid, beginVersion, endVersion, range, filename, fileIndex, partitionId, offset, len, addPrefix,
|
||||
removePrefix);
|
||||
removePrefix, batchIndex);
|
||||
}
|
||||
|
||||
std::string toString() const {
|
||||
|
@ -279,7 +281,8 @@ struct RestoreAsset {
|
|||
ss << "UID:" << uid.toString() << " begin:" << beginVersion << " end:" << endVersion
|
||||
<< " range:" << range.toString() << " filename:" << filename << " fileIndex:" << fileIndex
|
||||
<< " partitionId:" << partitionId << " offset:" << offset << " len:" << len
|
||||
<< " addPrefix:" << addPrefix.toString() << " removePrefix:" << removePrefix.toString();
|
||||
<< " addPrefix:" << addPrefix.toString() << " removePrefix:" << removePrefix.toString()
|
||||
<< " BatchIndex:" << batchIndex;
|
||||
return ss.str();
|
||||
}
|
||||
|
||||
|
@ -342,7 +345,7 @@ struct LoadingParam {
|
|||
serializer(ar, isRangeFile, url, rangeVersion, blockSize, asset);
|
||||
}
|
||||
|
||||
std::string toString() {
|
||||
std::string toString() const {
|
||||
std::stringstream str;
|
||||
str << "isRangeFile:" << isRangeFile << " url:" << url.toString()
|
||||
<< " rangeVersion:" << (rangeVersion.present() ? rangeVersion.get() : -1) << " blockSize:" << blockSize
|
||||
|
@ -370,7 +373,7 @@ struct RestoreRecruitRoleReply : TimedRequest {
|
|||
serializer(ar, id, role, loader, applier);
|
||||
}
|
||||
|
||||
std::string toString() {
|
||||
std::string toString() const {
|
||||
std::stringstream ss;
|
||||
ss << "roleInterf role:" << getRoleStr(role) << " replyID:" << id.toString();
|
||||
if (loader.present()) {
|
||||
|
@ -402,14 +405,14 @@ struct RestoreRecruitRoleRequest : TimedRequest {
|
|||
serializer(ar, ci, role, nodeIndex, reply);
|
||||
}
|
||||
|
||||
std::string printable() {
|
||||
std::string printable() const {
|
||||
std::stringstream ss;
|
||||
ss << "RestoreRecruitRoleRequest Role:" << getRoleStr(role) << " NodeIndex:" << nodeIndex
|
||||
<< " RestoreController:" << ci.id().toString();
|
||||
return ss.str();
|
||||
}
|
||||
|
||||
std::string toString() { return printable(); }
|
||||
std::string toString() const { return printable(); }
|
||||
};
|
||||
|
||||
// Static info. across version batches
|
||||
|
@ -431,7 +434,7 @@ struct RestoreSysInfoRequest : TimedRequest {
|
|||
serializer(ar, sysInfo, rangeVersions, reply);
|
||||
}
|
||||
|
||||
std::string toString() {
|
||||
std::string toString() const {
|
||||
std::stringstream ss;
|
||||
ss << "RestoreSysInfoRequest "
|
||||
<< "rangeVersions.size:" << rangeVersions.size();
|
||||
|
@ -456,7 +459,7 @@ struct RestoreSamplesRequest : TimedRequest {
|
|||
serializer(ar, id, batchIndex, samples, reply);
|
||||
}
|
||||
|
||||
std::string toString() {
|
||||
std::string toString() const {
|
||||
std::stringstream ss;
|
||||
ss << "ID:" << id.toString() << " BatchIndex:" << batchIndex << " samples:" << samples.size();
|
||||
return ss.str();
|
||||
|
@ -477,7 +480,7 @@ struct RestoreLoadFileReply : TimedRequest {
|
|||
serializer(ar, param, isDuplicated);
|
||||
}
|
||||
|
||||
std::string toString() {
|
||||
std::string toString() const {
|
||||
std::stringstream ss;
|
||||
ss << "LoadingParam:" << param.toString() << " isDuplicated:" << isDuplicated;
|
||||
return ss.str();
|
||||
|
@ -496,12 +499,14 @@ struct RestoreLoadFileRequest : TimedRequest {
|
|||
RestoreLoadFileRequest() = default;
|
||||
explicit RestoreLoadFileRequest(int batchIndex, LoadingParam& param) : batchIndex(batchIndex), param(param){};
|
||||
|
||||
bool operator<(RestoreLoadFileRequest const& rhs) const { return batchIndex > rhs.batchIndex; }
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, batchIndex, param, reply);
|
||||
}
|
||||
|
||||
std::string toString() {
|
||||
std::string toString() const {
|
||||
std::stringstream ss;
|
||||
ss << "RestoreLoadFileRequest batchIndex:" << batchIndex << " param:" << param.toString();
|
||||
return ss.str();
|
||||
|
@ -521,12 +526,14 @@ struct RestoreSendMutationsToAppliersRequest : TimedRequest {
|
|||
explicit RestoreSendMutationsToAppliersRequest(int batchIndex, std::map<Key, UID> rangeToApplier, bool useRangeFile)
|
||||
: batchIndex(batchIndex), rangeToApplier(rangeToApplier), useRangeFile(useRangeFile) {}
|
||||
|
||||
bool operator<(RestoreSendMutationsToAppliersRequest const& rhs) const { return batchIndex > rhs.batchIndex; }
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, batchIndex, rangeToApplier, useRangeFile, reply);
|
||||
}
|
||||
|
||||
std::string toString() {
|
||||
std::string toString() const {
|
||||
std::stringstream ss;
|
||||
ss << "RestoreSendMutationsToAppliersRequest batchIndex:" << batchIndex
|
||||
<< " keyToAppliers.size:" << rangeToApplier.size() << " useRangeFile:" << useRangeFile;
|
||||
|
@ -552,10 +559,10 @@ struct RestoreSendVersionedMutationsRequest : TimedRequest {
|
|||
: batchIndex(batchIndex), asset(asset), msgIndex(msgIndex), isRangeFile(isRangeFile),
|
||||
versionedMutations(versionedMutations) {}
|
||||
|
||||
std::string toString() {
|
||||
std::string toString() const {
|
||||
std::stringstream ss;
|
||||
ss << "VersionBatchIndex:" << batchIndex << "RestoreAsset:" << asset.toString() << " msgIndex:" << msgIndex
|
||||
<< " isRangeFile:" << isRangeFile << " versionedMutations.size:" << versionedMutations.size();
|
||||
ss << "VersionBatchIndex:" << batchIndex << " msgIndex:" << msgIndex << " isRangeFile:" << isRangeFile
|
||||
<< " versionedMutations.size:" << versionedMutations.size() << " RestoreAsset:" << asset.toString();
|
||||
return ss.str();
|
||||
}
|
||||
|
||||
|
@ -580,7 +587,7 @@ struct RestoreVersionBatchRequest : TimedRequest {
|
|||
serializer(ar, batchIndex, reply);
|
||||
}
|
||||
|
||||
std::string toString() {
|
||||
std::string toString() const {
|
||||
std::stringstream ss;
|
||||
ss << "RestoreVersionBatchRequest batchIndex:" << batchIndex;
|
||||
return ss.str();
|
||||
|
@ -602,7 +609,7 @@ struct RestoreFinishRequest : TimedRequest {
|
|||
serializer(ar, terminate, reply);
|
||||
}
|
||||
|
||||
std::string toString() {
|
||||
std::string toString() const {
|
||||
std::stringstream ss;
|
||||
ss << "RestoreFinishRequest terminate:" << terminate;
|
||||
return ss.str();
|
||||
|
|
|
@ -48,6 +48,7 @@ const KeyRef JSONSchemas::statusSchema = LiteralStringRef(R"statusSchema(
|
|||
"transaction",
|
||||
"resolution",
|
||||
"proxy",
|
||||
"grv_proxy",
|
||||
"master",
|
||||
"test",
|
||||
"storage_cache"
|
||||
|
@ -84,6 +85,7 @@ const KeyRef JSONSchemas::statusSchema = LiteralStringRef(R"statusSchema(
|
|||
"$enum":[
|
||||
"master",
|
||||
"proxy",
|
||||
"grv_proxy",
|
||||
"log",
|
||||
"storage",
|
||||
"resolver",
|
||||
|
@ -189,6 +191,13 @@ const KeyRef JSONSchemas::statusSchema = LiteralStringRef(R"statusSchema(
|
|||
"estimated_cost":{
|
||||
"hz": 0.0
|
||||
}
|
||||
},
|
||||
"busiest_write_tag":{
|
||||
"tag": "",
|
||||
"fractional_cost": 0.0,
|
||||
"estimated_cost":{
|
||||
"hz": 0.0
|
||||
}
|
||||
}
|
||||
}
|
||||
],
|
||||
|
@ -341,11 +350,14 @@ const KeyRef JSONSchemas::statusSchema = LiteralStringRef(R"statusSchema(
|
|||
"batch_released_transactions_per_second":0,
|
||||
"released_transactions_per_second":0,
|
||||
"throttled_tags":{
|
||||
"auto":{
|
||||
"count":0
|
||||
"auto" : {
|
||||
"busy_read" : 0,
|
||||
"busy_write" : 0,
|
||||
"count" : 0,
|
||||
"recommended_only": 0
|
||||
},
|
||||
"manual":{
|
||||
"count":0
|
||||
"manual" : {
|
||||
"count" : 0
|
||||
}
|
||||
},
|
||||
"limiting_queue_bytes_storage_server":0,
|
||||
|
@ -475,6 +487,7 @@ const KeyRef JSONSchemas::statusSchema = LiteralStringRef(R"statusSchema(
|
|||
"recovery_state":{
|
||||
"required_resolvers":1,
|
||||
"required_proxies":1,
|
||||
"required_grv_proxies":1,
|
||||
"name":{
|
||||
"$enum":[
|
||||
"reading_coordinated_state",
|
||||
|
@ -663,9 +676,11 @@ const KeyRef JSONSchemas::statusSchema = LiteralStringRef(R"statusSchema(
|
|||
}
|
||||
],
|
||||
"auto_proxies":3,
|
||||
"auto_grv_proxies":1,
|
||||
"auto_resolvers":1,
|
||||
"auto_logs":3,
|
||||
"proxies":5,
|
||||
"grv_proxies":1,
|
||||
"backup_worker_enabled":1
|
||||
},
|
||||
"data":{
|
||||
|
@ -865,9 +880,11 @@ const KeyRef JSONSchemas::clusterConfigurationSchema = LiteralStringRef(R"config
|
|||
"memory"
|
||||
]},
|
||||
"auto_proxies":3,
|
||||
"auto_grv_proxies":1,
|
||||
"auto_resolvers":1,
|
||||
"auto_logs":3,
|
||||
"proxies":5
|
||||
"grv_proxies":1
|
||||
})configSchema");
|
||||
|
||||
const KeyRef JSONSchemas::latencyBandConfigurationSchema = LiteralStringRef(R"configSchema(
|
||||
|
|
|
@ -906,6 +906,7 @@ std::pair<MetricNameRef, KeyRef> decodeMetricConfKey( KeyRef const& prefix, KeyR
|
|||
const KeyRef maxUIDKey = LiteralStringRef("\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff\xff");
|
||||
|
||||
const KeyRef databaseLockedKey = LiteralStringRef("\xff/dbLocked");
|
||||
const KeyRef databaseLockedKeyEnd = LiteralStringRef("\xff/dbLocked\x00");
|
||||
const KeyRef metadataVersionKey = LiteralStringRef("\xff/metadataVersion");
|
||||
const KeyRef metadataVersionKeyEnd = LiteralStringRef("\xff/metadataVersion\x00");
|
||||
const KeyRef metadataVersionRequiredValue = LiteralStringRef("\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00\x00");
|
||||
|
|
|
@ -351,6 +351,7 @@ extern const KeyRef metricConfPrefix;
|
|||
extern const KeyRef maxUIDKey;
|
||||
|
||||
extern const KeyRef databaseLockedKey;
|
||||
extern const KeyRef databaseLockedKeyEnd;
|
||||
extern const KeyRef metadataVersionKey;
|
||||
extern const KeyRef metadataVersionKeyEnd;
|
||||
extern const KeyRef metadataVersionRequiredValue;
|
||||
|
|
|
@ -110,6 +110,31 @@ TagThrottleValue TagThrottleValue::fromValue(const ValueRef& value) {
|
|||
}
|
||||
|
||||
namespace ThrottleApi {
|
||||
ACTOR Future<bool> getValidAutoEnabled(Transaction* tr, Database db) {
|
||||
state bool result;
|
||||
loop {
|
||||
Optional<Value> value = wait(tr->get(tagThrottleAutoEnabledKey));
|
||||
if(!value.present()) {
|
||||
tr->reset();
|
||||
wait(delay(CLIENT_KNOBS->DEFAULT_BACKOFF));
|
||||
continue;
|
||||
}
|
||||
else if(value.get() == LiteralStringRef("1")) {
|
||||
result = true;
|
||||
}
|
||||
else if(value.get() == LiteralStringRef("0")) {
|
||||
result = false;
|
||||
}
|
||||
else {
|
||||
TraceEvent(SevWarnAlways, "InvalidAutoTagThrottlingValue", db->dbId).detail("Value", value.get());
|
||||
tr->reset();
|
||||
wait(delay(CLIENT_KNOBS->DEFAULT_BACKOFF));
|
||||
continue;
|
||||
}
|
||||
return result;
|
||||
};
|
||||
}
|
||||
|
||||
void signalThrottleChange(Transaction &tr) {
|
||||
tr.atomicOp(tagThrottleSignalKey, LiteralStringRef("XXXXXXXXXX\x00\x00\x00\x00"), MutationRef::SetVersionstampedValue);
|
||||
}
|
||||
|
@ -146,12 +171,16 @@ namespace ThrottleApi {
|
|||
return Void();
|
||||
}
|
||||
|
||||
ACTOR Future<std::vector<TagThrottleInfo>> getThrottledTags(Database db, int limit) {
|
||||
ACTOR Future<std::vector<TagThrottleInfo>> getThrottledTags(Database db, int limit, bool containsRecommend) {
|
||||
state Transaction tr(db);
|
||||
|
||||
state bool reportAuto = containsRecommend;
|
||||
loop {
|
||||
try {
|
||||
Standalone<RangeResultRef> throttles = wait(tr.getRange(tagThrottleKeys, limit));
|
||||
if (!containsRecommend) {
|
||||
wait(store(reportAuto, getValidAutoEnabled(&tr, db)));
|
||||
}
|
||||
Standalone<RangeResultRef> throttles = wait(tr.getRange(
|
||||
reportAuto ? tagThrottleKeys : KeyRangeRef(tagThrottleKeysPrefix, tagThrottleAutoKeysPrefix), limit));
|
||||
std::vector<TagThrottleInfo> results;
|
||||
for(auto throttle : throttles) {
|
||||
results.push_back(TagThrottleInfo(TagThrottleKey::fromKey(throttle.key), TagThrottleValue::fromValue(throttle.value)));
|
||||
|
@ -164,13 +193,41 @@ namespace ThrottleApi {
|
|||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> throttleTags(Database db, TagSet tags, double tpsRate, double initialDuration, TagThrottleType throttleType, TransactionPriority priority, Optional<double> expirationTime) {
|
||||
ACTOR Future<std::vector<TagThrottleInfo>> getRecommendedTags(Database db, int limit) {
|
||||
state Transaction tr(db);
|
||||
loop {
|
||||
try {
|
||||
bool enableAuto = wait(getValidAutoEnabled(&tr, db));
|
||||
if(enableAuto) {
|
||||
return std::vector<TagThrottleInfo>();
|
||||
}
|
||||
|
||||
Standalone<RangeResultRef> throttles = wait(tr.getRange(KeyRangeRef(tagThrottleAutoKeysPrefix, tagThrottleKeys.end), limit));
|
||||
std::vector<TagThrottleInfo> results;
|
||||
for(auto throttle : throttles) {
|
||||
results.push_back(TagThrottleInfo(TagThrottleKey::fromKey(throttle.key), TagThrottleValue::fromValue(throttle.value)));
|
||||
}
|
||||
return results;
|
||||
}
|
||||
catch(Error& e) {
|
||||
wait(tr.onError(e));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> throttleTags(Database db, TagSet tags, double tpsRate, double initialDuration,
|
||||
TagThrottleType throttleType, TransactionPriority priority, Optional<double> expirationTime,
|
||||
Optional<TagThrottledReason> reason) {
|
||||
state Transaction tr(db);
|
||||
state Key key = TagThrottleKey(tags, throttleType, priority).toKey();
|
||||
|
||||
ASSERT(initialDuration > 0);
|
||||
|
||||
TagThrottleValue throttle(tpsRate, expirationTime.present() ? expirationTime.get() : 0, initialDuration);
|
||||
if(throttleType == TagThrottleType::MANUAL) {
|
||||
reason = TagThrottledReason::MANUAL;
|
||||
}
|
||||
TagThrottleValue throttle(tpsRate, expirationTime.present() ? expirationTime.get() : 0, initialDuration,
|
||||
reason.present() ? reason.get() : TagThrottledReason::UNSET);
|
||||
BinaryWriter wr(IncludeVersion(ProtocolVersion::withTagThrottleValue()));
|
||||
wr << throttle;
|
||||
state Value value = wr.toValue();
|
||||
|
|
|
@ -115,6 +115,13 @@ enum class TagThrottleType : uint8_t {
|
|||
AUTO
|
||||
};
|
||||
|
||||
enum class TagThrottledReason: uint8_t {
|
||||
UNSET = 0,
|
||||
MANUAL,
|
||||
BUSY_READ,
|
||||
BUSY_WRITE
|
||||
};
|
||||
|
||||
struct TagThrottleKey {
|
||||
TagSet tags;
|
||||
TagThrottleType throttleType;
|
||||
|
@ -132,17 +139,26 @@ struct TagThrottleValue {
|
|||
double tpsRate;
|
||||
double expirationTime;
|
||||
double initialDuration;
|
||||
TagThrottledReason reason;
|
||||
|
||||
TagThrottleValue() : tpsRate(0), expirationTime(0), initialDuration(0) {}
|
||||
TagThrottleValue(double tpsRate, double expirationTime, double initialDuration)
|
||||
: tpsRate(tpsRate), expirationTime(expirationTime), initialDuration(initialDuration) {}
|
||||
TagThrottleValue() : tpsRate(0), expirationTime(0), initialDuration(0), reason(TagThrottledReason::UNSET) {}
|
||||
TagThrottleValue(double tpsRate, double expirationTime, double initialDuration, TagThrottledReason reason)
|
||||
: tpsRate(tpsRate), expirationTime(expirationTime), initialDuration(initialDuration), reason(reason) {}
|
||||
|
||||
static TagThrottleValue fromValue(const ValueRef& value);
|
||||
|
||||
//To change this serialization, ProtocolVersion::TagThrottleValue must be updated, and downgrades need to be considered
|
||||
template<class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, tpsRate, expirationTime, initialDuration);
|
||||
if(ar.protocolVersion().hasTagThrottleValueReason()) {
|
||||
serializer(ar, tpsRate, expirationTime, initialDuration, reinterpret_cast<uint8_t&>(reason));
|
||||
}
|
||||
else if(ar.protocolVersion().hasTagThrottleValue()) {
|
||||
serializer(ar, tpsRate, expirationTime, initialDuration);
|
||||
if(ar.isDeserializing) {
|
||||
reason = TagThrottledReason::UNSET;
|
||||
}
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -153,12 +169,13 @@ struct TagThrottleInfo {
|
|||
double tpsRate;
|
||||
double expirationTime;
|
||||
double initialDuration;
|
||||
TagThrottledReason reason;
|
||||
|
||||
TagThrottleInfo(TransactionTag tag, TagThrottleType throttleType, TransactionPriority priority, double tpsRate, double expirationTime, double initialDuration)
|
||||
: tag(tag), throttleType(throttleType), priority(priority), tpsRate(tpsRate), expirationTime(expirationTime), initialDuration(initialDuration) {}
|
||||
TagThrottleInfo(TransactionTag tag, TagThrottleType throttleType, TransactionPriority priority, double tpsRate, double expirationTime, double initialDuration, TagThrottledReason reason = TagThrottledReason::UNSET)
|
||||
: tag(tag), throttleType(throttleType), priority(priority), tpsRate(tpsRate), expirationTime(expirationTime), initialDuration(initialDuration), reason(reason) {}
|
||||
|
||||
TagThrottleInfo(TagThrottleKey key, TagThrottleValue value)
|
||||
: throttleType(key.throttleType), priority(key.priority), tpsRate(value.tpsRate), expirationTime(value.expirationTime), initialDuration(value.initialDuration)
|
||||
TagThrottleInfo(TagThrottleKey key, TagThrottleValue value)
|
||||
: throttleType(key.throttleType), priority(key.priority), tpsRate(value.tpsRate), expirationTime(value.expirationTime), initialDuration(value.initialDuration), reason(value.reason)
|
||||
{
|
||||
ASSERT(key.tags.size() == 1); // Multiple tags per throttle is not currently supported
|
||||
tag = *key.tags.begin();
|
||||
|
@ -166,10 +183,12 @@ struct TagThrottleInfo {
|
|||
};
|
||||
|
||||
namespace ThrottleApi {
|
||||
Future<std::vector<TagThrottleInfo>> getThrottledTags(Database const& db, int const& limit);
|
||||
Future<std::vector<TagThrottleInfo>> getThrottledTags(Database const& db, int const& limit, bool const& containsRecommend = false);
|
||||
Future<std::vector<TagThrottleInfo>> getRecommendedTags(Database const& db, int const& limit);
|
||||
|
||||
Future<Void> throttleTags(Database const& db, TagSet const& tags, double const& tpsRate, double const& initialDuration,
|
||||
TagThrottleType const& throttleType, TransactionPriority const& priority, Optional<double> const& expirationTime = Optional<double>());
|
||||
TagThrottleType const& throttleType, TransactionPriority const& priority, Optional<double> const& expirationTime = Optional<double>(),
|
||||
Optional<TagThrottledReason> const& reason = Optional<TagThrottledReason>());
|
||||
|
||||
Future<bool> unthrottleTags(Database const& db, TagSet const& tags, Optional<TagThrottleType> const& throttleType, Optional<TransactionPriority> const& priority);
|
||||
|
||||
|
@ -187,4 +206,6 @@ using TransactionTagMap = std::unordered_map<TransactionTag, Value, std::hash<Tr
|
|||
template<class Value>
|
||||
using PrioritizedTransactionTagMap = std::map<TransactionPriority, TransactionTagMap<Value>>;
|
||||
|
||||
template <class Value>
|
||||
using UIDTransactionTagMap = std::unordered_map<UID, TransactionTagMap<Value>>;
|
||||
#endif
|
|
@ -234,17 +234,17 @@ public:
|
|||
ACTOR static Future<bool> taskVerify(Reference<TaskBucket> tb, Reference<ReadYourWritesTransaction> tr, Reference<Task> task) {
|
||||
|
||||
if (task->params.find(Task::reservedTaskParamValidKey) == task->params.end()) {
|
||||
TraceEvent("TB_TaskVerifyInvalidTask")
|
||||
.detail("Task", task->params[Task::reservedTaskParamKeyType])
|
||||
.detail("ReservedTaskParamValidKey", "missing");
|
||||
TraceEvent("TaskBucketTaskVerifyInvalidTask")
|
||||
.detail("Task", task->params[Task::reservedTaskParamKeyType])
|
||||
.detail("ReservedTaskParamValidKey", "missing");
|
||||
return false;
|
||||
}
|
||||
|
||||
if (task->params.find(Task::reservedTaskParamValidValue) == task->params.end()) {
|
||||
TraceEvent("TB_TaskVerifyInvalidTask")
|
||||
.detail("Task", task->params[Task::reservedTaskParamKeyType])
|
||||
.detail("ReservedTaskParamValidKey", task->params[Task::reservedTaskParamValidKey])
|
||||
.detail("ReservedTaskParamValidValue", "missing");
|
||||
TraceEvent("TaskBucketTaskVerifyInvalidTask")
|
||||
.detail("Task", task->params[Task::reservedTaskParamKeyType])
|
||||
.detail("ReservedTaskParamValidKey", task->params[Task::reservedTaskParamValidKey])
|
||||
.detail("ReservedTaskParamValidValue", "missing");
|
||||
return false;
|
||||
}
|
||||
|
||||
|
@ -253,20 +253,20 @@ public:
|
|||
Optional<Value> keyValue = wait(tr->get(task->params[Task::reservedTaskParamValidKey]));
|
||||
|
||||
if (!keyValue.present()) {
|
||||
TraceEvent("TB_TaskVerifyInvalidTask")
|
||||
.detail("Task", task->params[Task::reservedTaskParamKeyType])
|
||||
.detail("ReservedTaskParamValidKey", task->params[Task::reservedTaskParamValidKey])
|
||||
.detail("ReservedTaskParamValidValue", task->params[Task::reservedTaskParamValidValue])
|
||||
.detail("KeyValue", "missing");
|
||||
TraceEvent("TaskBucketTaskVerifyInvalidTask")
|
||||
.detail("Task", task->params[Task::reservedTaskParamKeyType])
|
||||
.detail("ReservedTaskParamValidKey", task->params[Task::reservedTaskParamValidKey])
|
||||
.detail("ReservedTaskParamValidValue", task->params[Task::reservedTaskParamValidValue])
|
||||
.detail("KeyValue", "missing");
|
||||
return false;
|
||||
}
|
||||
|
||||
if (keyValue.get().compare(StringRef(task->params[Task::reservedTaskParamValidValue]))) {
|
||||
TraceEvent("TB_TaskVerifyAbortedTask")
|
||||
.detail("Task", task->params[Task::reservedTaskParamKeyType])
|
||||
.detail("ReservedTaskParamValidKey", task->params[Task::reservedTaskParamValidKey])
|
||||
.detail("ReservedTaskParamValidValue", task->params[Task::reservedTaskParamValidValue])
|
||||
.detail("KeyValue", keyValue.get());
|
||||
TraceEvent("TaskBucketTaskVerifyAbortedTask")
|
||||
.detail("Task", task->params[Task::reservedTaskParamKeyType])
|
||||
.detail("ReservedTaskParamValidKey", task->params[Task::reservedTaskParamValidKey])
|
||||
.detail("ReservedTaskParamValidValue", task->params[Task::reservedTaskParamValidValue])
|
||||
.detail("KeyValue", keyValue.get());
|
||||
return false;
|
||||
}
|
||||
|
||||
|
@ -332,10 +332,10 @@ public:
|
|||
|
||||
if(now() - start > 300) {
|
||||
TraceEvent(SevWarnAlways, "TaskBucketLongExtend")
|
||||
.detail("Duration", now() - start)
|
||||
.detail("TaskUID", task->key)
|
||||
.detail("TaskType", task->params[Task::reservedTaskParamKeyType])
|
||||
.detail("Priority", task->getPriority());
|
||||
.detail("Duration", now() - start)
|
||||
.detail("TaskUID", task->key)
|
||||
.detail("TaskType", task->params[Task::reservedTaskParamKeyType])
|
||||
.detail("Priority", task->getPriority());
|
||||
}
|
||||
// Take the extendMutex lock until we either succeed or stop trying to extend due to failure
|
||||
wait(task->extendMutex.take());
|
||||
|
@ -402,19 +402,19 @@ public:
|
|||
}));
|
||||
}
|
||||
} catch(Error &e) {
|
||||
TraceEvent(SevWarn, "TB_ExecuteFailure")
|
||||
.error(e)
|
||||
.detail("TaskUID", task->key)
|
||||
.detail("TaskType", task->params[Task::reservedTaskParamKeyType].printable())
|
||||
.detail("Priority", task->getPriority());
|
||||
TraceEvent(SevWarn, "TaskBucketExecuteFailure")
|
||||
.error(e)
|
||||
.detail("TaskUID", task->key)
|
||||
.detail("TaskType", task->params[Task::reservedTaskParamKeyType].printable())
|
||||
.detail("Priority", task->getPriority());
|
||||
try {
|
||||
wait(taskFunc->handleError(cx, task, e));
|
||||
} catch(Error &e) {
|
||||
TraceEvent(SevWarn, "TB_ExecuteFailureLogErrorFailed")
|
||||
.error(e) // output handleError() error instead of original task error
|
||||
.detail("TaskUID", task->key.printable())
|
||||
.detail("TaskType", task->params[Task::reservedTaskParamKeyType].printable())
|
||||
.detail("Priority", task->getPriority());
|
||||
TraceEvent(SevWarn, "TaskBucketExecuteFailureLogErrorFailed")
|
||||
.error(e) // output handleError() error instead of original task error
|
||||
.detail("TaskUID", task->key.printable())
|
||||
.detail("TaskType", task->params[Task::reservedTaskParamKeyType].printable())
|
||||
.detail("Priority", task->getPriority());
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -727,14 +727,17 @@ public:
|
|||
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
|
||||
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
|
||||
Standalone<RangeResultRef> values = wait(tr->getRange(subspace.range(), CLIENT_KNOBS->TOO_MANY));
|
||||
TraceEvent("TaskBucket").detail("DebugPrintRange", "Print DB Range").detail("Key", subspace.key()).detail("Count", values.size()).detail("Msg", msg);
|
||||
|
||||
TraceEvent("TaskBucketDebugPrintRange")
|
||||
.detail("Key", subspace.key())
|
||||
.detail("Count", values.size())
|
||||
.detail("Msg", msg);
|
||||
|
||||
/*printf("debugPrintRange key: (%d) %s\n", values.size(), printable(subspace.key()).c_str());
|
||||
for (auto & s : values) {
|
||||
printf(" key: %-40s value: %s\n", printable(s.key).c_str(), s.value.c_str());
|
||||
TraceEvent("TaskBucket").detail("DebugPrintRange", msg)
|
||||
.detail("Key", s.key)
|
||||
.detail("Value", s.value);
|
||||
printf(" key: %-40s value: %s\n", printable(s.key).c_str(), s.value.c_str());
|
||||
TraceEvent("TaskBucketDebugPrintKV").detail("Msg", msg)
|
||||
.detail("Key", s.key)
|
||||
.detail("Value", s.value);
|
||||
}*/
|
||||
|
||||
return Void();
|
||||
|
@ -870,9 +873,9 @@ ACTOR static Future<Key> actorAddTask(TaskBucket* tb, Reference<ReadYourWritesTr
|
|||
Optional<Value> validationValue = wait(tr->get(validationKey));
|
||||
|
||||
if (!validationValue.present()) {
|
||||
TraceEvent(SevError, "TB_AddTaskInvalidKey")
|
||||
.detail("Task", task->params[Task::reservedTaskParamKeyType])
|
||||
.detail("ValidationKey", validationKey);
|
||||
TraceEvent(SevError, "TaskBucketAddTaskInvalidKey")
|
||||
.detail("Task", task->params[Task::reservedTaskParamKeyType])
|
||||
.detail("ValidationKey", validationKey);
|
||||
throw invalid_option_value();
|
||||
}
|
||||
|
||||
|
@ -1138,9 +1141,9 @@ public:
|
|||
Optional<Value> validationValue = wait(tr->get(validationKey));
|
||||
|
||||
if (!validationValue.present()) {
|
||||
TraceEvent(SevError, "TB_OnSetAddTaskInvalidKey")
|
||||
.detail("Task", task->params[Task::reservedTaskParamKeyType])
|
||||
.detail("ValidationKey", validationKey);
|
||||
TraceEvent(SevError, "TaskBucketOnSetAddTaskInvalidKey")
|
||||
.detail("Task", task->params[Task::reservedTaskParamKeyType])
|
||||
.detail("ValidationKey", validationKey);
|
||||
throw invalid_option_value();
|
||||
}
|
||||
|
||||
|
|
|
@ -35,6 +35,20 @@
|
|||
class FutureBucket;
|
||||
class TaskFuture;
|
||||
|
||||
// A Task is a set of key=value parameters that constitute a unit of work for a TaskFunc to perform.
|
||||
// The parameter keys are specific to the TaskFunc that the Task is for, except for a set of reserved
|
||||
// parameter keys which are used by TaskBucket to determine which TaskFunc to run and provide
|
||||
// several other core features of TaskBucket.
|
||||
//
|
||||
// Task Life Cycle
|
||||
// 1. Task is created in database transaction.
|
||||
// 2. An executor (see TaskBucket class) will reserve an begin executing the task
|
||||
// 3. Task's _execute() function is run. This is non-transactional, and can run indefinitely.
|
||||
// 4. If the executor loses contact with FDB, another executor may begin at step 2. The first
|
||||
// Task execution can detect this by checking the result of keepRunning() periodically.
|
||||
// 5. Once a Task execution's _execute() call returns, the _finish() step is called.
|
||||
// _finish() is transactional and is guaraunteed to never be called more than once for the
|
||||
// same Task
|
||||
class Task : public ReferenceCounted<Task> {
|
||||
public:
|
||||
Task(Value type = StringRef(), uint32_t version = 0, Value done = StringRef(), unsigned int priority = 0);
|
||||
|
@ -53,13 +67,18 @@ public:
|
|||
Map<Key, Value> params; // SOMEDAY: use one arena?
|
||||
|
||||
// New reserved task parameter keys should be added in ReservedTaskParams below instead of here.
|
||||
// Task priority, determines execution order relative to other queued Tasks
|
||||
static Key reservedTaskParamKeyPriority;
|
||||
// Name of the registered TaskFunc that this Task is for
|
||||
static Key reservedTaskParamKeyType;
|
||||
static Key reservedTaskParamKeyAddTask;
|
||||
static Key reservedTaskParamKeyDone;
|
||||
static Key reservedTaskParamKeyFuture;
|
||||
static Key reservedTaskParamKeyBlockID;
|
||||
static Key reservedTaskParamKeyVersion;
|
||||
|
||||
// Optional parameters that specify a database Key that must have a specific Value in order for the Task
|
||||
// to be executed (for _execute() or _finish() to be called) OR for keepRunning() to return true for the Task.
|
||||
static Key reservedTaskParamValidKey;
|
||||
static Key reservedTaskParamValidValue;
|
||||
|
||||
|
@ -74,6 +93,7 @@ public:
|
|||
}
|
||||
};
|
||||
|
||||
// TaskParam is a convenience class to make storing non-string types into Task Parameters easier.
|
||||
template <typename T>
|
||||
class TaskParam {
|
||||
public:
|
||||
|
@ -103,6 +123,23 @@ struct ReservedTaskParams {
|
|||
|
||||
class FutureBucket;
|
||||
|
||||
// A TaskBucket is a subspace in which a set of Tasks and TaskFutures exists. Within the subspace, there
|
||||
// are several other subspaces including
|
||||
// available - Tasks that are waiting to run at default priority
|
||||
// available_prioritized - Tasks with priorities 0 through max priority, all higher than default
|
||||
// timeouts - Tasks that are currently running and are scheduled to timeout a specific FDB commit version.
|
||||
// futures - TaskFutures that have not been completed
|
||||
//
|
||||
// One or more processes must instantiate a TaskBucket call run() or doOne() repeatedly in order for Tasks
|
||||
// in the TaskBucket to make progress. The calling process is directly used to execute the Task code.
|
||||
//
|
||||
// Tasks are added to a TaskBucket with addTask(), and this can be done at any time but is typically done
|
||||
// in the _finish() step of a Task. This makes the completion of one Task and the creation of its one or
|
||||
// more child Tasks atomic.
|
||||
//
|
||||
// While a TaskBucket instance is executing a task, there is timeout set for the Task and periodically the
|
||||
// executor will extend this timeout in the database. If this fails to happen, then another TaskBucket
|
||||
// instance may declare the Task a failure and move it back to the available subspace.
|
||||
class TaskBucket : public ReferenceCounted<TaskBucket> {
|
||||
public:
|
||||
TaskBucket(const Subspace& subspace, bool sysAccess = false, bool priorityBatch = false, bool lockAware = false);
|
||||
|
|
|
@ -160,7 +160,7 @@ public:
|
|||
return openFiles[filename].get();
|
||||
}
|
||||
|
||||
virtual Future<int> read( void* data, int length, int64_t offset ) {
|
||||
Future<int> read(void* data, int length, int64_t offset) override {
|
||||
++countFileCacheReads;
|
||||
++countCacheReads;
|
||||
if (offset + length > this->length) {
|
||||
|
@ -190,17 +190,15 @@ public:
|
|||
return Void();
|
||||
}
|
||||
|
||||
virtual Future<Void> write( void const* data, int length, int64_t offset ) {
|
||||
Future<Void> write(void const* data, int length, int64_t offset) override {
|
||||
return write_impl(this, data, length, offset);
|
||||
}
|
||||
|
||||
virtual Future<Void> readZeroCopy( void** data, int* length, int64_t offset );
|
||||
virtual void releaseZeroCopy( void* data, int length, int64_t offset );
|
||||
Future<Void> readZeroCopy(void** data, int* length, int64_t offset) override;
|
||||
void releaseZeroCopy(void* data, int length, int64_t offset) override;
|
||||
|
||||
// This waits for previously started truncates to finish and then truncates
|
||||
virtual Future<Void> truncate( int64_t size ) {
|
||||
return truncate_impl(this, size);
|
||||
}
|
||||
Future<Void> truncate(int64_t size) override { return truncate_impl(this, size); }
|
||||
|
||||
// This is the 'real' truncate that does the actual removal of cache blocks and then shortens the file
|
||||
Future<Void> changeFileSize( int64_t size );
|
||||
|
@ -215,21 +213,13 @@ public:
|
|||
return Void();
|
||||
}
|
||||
|
||||
virtual Future<Void> sync() {
|
||||
return waitAndSync( this, flush() );
|
||||
}
|
||||
Future<Void> sync() override { return waitAndSync(this, flush()); }
|
||||
|
||||
virtual Future<int64_t> size() {
|
||||
return length;
|
||||
}
|
||||
Future<int64_t> size() const override { return length; }
|
||||
|
||||
virtual int64_t debugFD() {
|
||||
return uncached->debugFD();
|
||||
}
|
||||
int64_t debugFD() const override { return uncached->debugFD(); }
|
||||
|
||||
virtual std::string getFilename() {
|
||||
return filename;
|
||||
}
|
||||
std::string getFilename() const override { return filename; }
|
||||
|
||||
virtual void addref() {
|
||||
ReferenceCounted<AsyncFileCached>::addref();
|
||||
|
@ -337,7 +327,7 @@ private:
|
|||
}
|
||||
}
|
||||
|
||||
virtual Future<Void> flush();
|
||||
Future<Void> flush() override;
|
||||
|
||||
Future<Void> quiesce();
|
||||
|
||||
|
@ -356,7 +346,7 @@ private:
|
|||
};
|
||||
|
||||
struct AFCPage : public EvictablePage, public FastAllocated<AFCPage> {
|
||||
virtual bool evict() {
|
||||
bool evict() override {
|
||||
if ( notReading.isReady() && notFlushing.isReady() && !dirty && !zeroCopyRefCount && !truncated ) {
|
||||
owner->remove_page( this );
|
||||
delete this;
|
||||
|
|
|
@ -115,26 +115,26 @@ public:
|
|||
virtual void addref() { ReferenceCounted<AsyncFileEIO>::addref(); }
|
||||
virtual void delref() { ReferenceCounted<AsyncFileEIO>::delref(); }
|
||||
|
||||
virtual int64_t debugFD() { return fd; }
|
||||
int64_t debugFD() const override { return fd; }
|
||||
|
||||
virtual Future<int> read( void* data, int length, int64_t offset ) {
|
||||
Future<int> read(void* data, int length, int64_t offset) override {
|
||||
++countFileLogicalReads;
|
||||
++countLogicalReads;
|
||||
return read_impl(fd, data, length, offset);
|
||||
}
|
||||
virtual Future<Void> write( void const* data, int length, int64_t offset ) // Copies data synchronously
|
||||
Future<Void> write(void const* data, int length, int64_t offset) override // Copies data synchronously
|
||||
{
|
||||
++countFileLogicalWrites;
|
||||
++countLogicalWrites;
|
||||
//Standalone<StringRef> copy = StringRef((const uint8_t*)data, length);
|
||||
return write_impl( fd, err, StringRef((const uint8_t*)data, length), offset );
|
||||
}
|
||||
virtual Future<Void> truncate( int64_t size ) {
|
||||
Future<Void> truncate(int64_t size) override {
|
||||
++countFileLogicalWrites;
|
||||
++countLogicalWrites;
|
||||
return truncate_impl( fd, err, size );
|
||||
}
|
||||
virtual Future<Void> sync() {
|
||||
Future<Void> sync() override {
|
||||
++countFileLogicalWrites;
|
||||
++countLogicalWrites;
|
||||
auto fsync = sync_impl( fd, err );
|
||||
|
@ -147,14 +147,12 @@ public:
|
|||
|
||||
return fsync;
|
||||
}
|
||||
virtual Future<int64_t> size() {
|
||||
Future<int64_t> size() const override {
|
||||
++countFileLogicalReads;
|
||||
++countLogicalReads;
|
||||
return size_impl(fd);
|
||||
}
|
||||
virtual std::string getFilename() {
|
||||
return filename;
|
||||
}
|
||||
std::string getFilename() const override { return filename; }
|
||||
|
||||
ACTOR static Future<Void> async_fsync_parent( std::string filename ) {
|
||||
std::string folder = parentDirectory( filename );
|
||||
|
@ -227,11 +225,11 @@ private:
|
|||
int fd, flags;
|
||||
Reference<ErrorInfo> err;
|
||||
std::string filename;
|
||||
Int64MetricHandle countFileLogicalWrites;
|
||||
Int64MetricHandle countFileLogicalReads;
|
||||
mutable Int64MetricHandle countFileLogicalWrites;
|
||||
mutable Int64MetricHandle countFileLogicalReads;
|
||||
|
||||
Int64MetricHandle countLogicalWrites;
|
||||
Int64MetricHandle countLogicalReads;
|
||||
mutable Int64MetricHandle countLogicalWrites;
|
||||
mutable Int64MetricHandle countLogicalReads;
|
||||
|
||||
AsyncFileEIO( int fd, int flags, std::string const& filename ) : fd(fd), flags(flags), filename(filename), err(new ErrorInfo) {
|
||||
if( !g_network->isSimulated() ) {
|
||||
|
|
|
@ -182,7 +182,7 @@ public:
|
|||
virtual void addref() { ReferenceCounted<AsyncFileKAIO>::addref(); }
|
||||
virtual void delref() { ReferenceCounted<AsyncFileKAIO>::delref(); }
|
||||
|
||||
virtual Future<int> read( void* data, int length, int64_t offset ) {
|
||||
Future<int> read(void* data, int length, int64_t offset) override {
|
||||
++countFileLogicalReads;
|
||||
++countLogicalReads;
|
||||
//printf("%p Begin logical read\n", getCurrentCoro());
|
||||
|
@ -205,7 +205,7 @@ public:
|
|||
|
||||
return result;
|
||||
}
|
||||
virtual Future<Void> write( void const* data, int length, int64_t offset ) {
|
||||
Future<Void> write(void const* data, int length, int64_t offset) override {
|
||||
++countFileLogicalWrites;
|
||||
++countLogicalWrites;
|
||||
//printf("%p Begin logical write\n", getCurrentCoro());
|
||||
|
@ -234,7 +234,7 @@ public:
|
|||
#ifndef FALLOC_FL_ZERO_RANGE
|
||||
#define FALLOC_FL_ZERO_RANGE 0x10
|
||||
#endif
|
||||
virtual Future<Void> zeroRange( int64_t offset, int64_t length ) override {
|
||||
Future<Void> zeroRange(int64_t offset, int64_t length) override {
|
||||
bool success = false;
|
||||
if (ctx.fallocateZeroSupported) {
|
||||
int rc = fallocate( fd, FALLOC_FL_ZERO_RANGE, offset, length );
|
||||
|
@ -247,7 +247,7 @@ public:
|
|||
}
|
||||
return success ? Void() : IAsyncFile::zeroRange(offset, length);
|
||||
}
|
||||
virtual Future<Void> truncate( int64_t size ) {
|
||||
Future<Void> truncate(int64_t size) override {
|
||||
++countFileLogicalWrites;
|
||||
++countLogicalWrites;
|
||||
|
||||
|
@ -308,7 +308,7 @@ public:
|
|||
return Void();
|
||||
}
|
||||
|
||||
virtual Future<Void> sync() {
|
||||
Future<Void> sync() override {
|
||||
++countFileLogicalWrites;
|
||||
++countLogicalWrites;
|
||||
|
||||
|
@ -340,13 +340,9 @@ public:
|
|||
|
||||
return fsync;
|
||||
}
|
||||
virtual Future<int64_t> size() { return nextFileSize; }
|
||||
virtual int64_t debugFD() {
|
||||
return fd;
|
||||
}
|
||||
virtual std::string getFilename() {
|
||||
return filename;
|
||||
}
|
||||
Future<int64_t> size() const override { return nextFileSize; }
|
||||
int64_t debugFD() const override { return fd; }
|
||||
std::string getFilename() const override { return filename; }
|
||||
~AsyncFileKAIO() {
|
||||
close(fd);
|
||||
|
||||
|
|
|
@ -87,42 +87,42 @@ public:
|
|||
ReferenceCounted<AsyncFileDetachable>::delref();
|
||||
}
|
||||
|
||||
Future<int> read(void *data, int length, int64_t offset) {
|
||||
Future<int> read(void* data, int length, int64_t offset) override {
|
||||
if( !file.getPtr() || g_simulator.getCurrentProcess()->shutdownSignal.getFuture().isReady() )
|
||||
return io_error().asInjectedFault();
|
||||
return sendErrorOnShutdown( file->read( data, length, offset ) );
|
||||
}
|
||||
|
||||
Future<Void> write(void const *data, int length, int64_t offset) {
|
||||
Future<Void> write(void const* data, int length, int64_t offset) override {
|
||||
if( !file.getPtr() || g_simulator.getCurrentProcess()->shutdownSignal.getFuture().isReady() )
|
||||
return io_error().asInjectedFault();
|
||||
return sendErrorOnShutdown( file->write( data, length, offset ) );
|
||||
}
|
||||
|
||||
Future<Void> truncate(int64_t size) {
|
||||
|
||||
Future<Void> truncate(int64_t size) override {
|
||||
if( !file.getPtr() || g_simulator.getCurrentProcess()->shutdownSignal.getFuture().isReady() )
|
||||
return io_error().asInjectedFault();
|
||||
return sendErrorOnShutdown( file->truncate( size ) );
|
||||
}
|
||||
|
||||
Future<Void> sync() {
|
||||
Future<Void> sync() override {
|
||||
if( !file.getPtr() || g_simulator.getCurrentProcess()->shutdownSignal.getFuture().isReady() )
|
||||
return io_error().asInjectedFault();
|
||||
return sendErrorOnShutdown( file->sync() );
|
||||
}
|
||||
|
||||
Future<int64_t> size() {
|
||||
Future<int64_t> size() const override {
|
||||
if( !file.getPtr() || g_simulator.getCurrentProcess()->shutdownSignal.getFuture().isReady() )
|
||||
return io_error().asInjectedFault();
|
||||
return sendErrorOnShutdown( file->size() );
|
||||
}
|
||||
|
||||
int64_t debugFD() {
|
||||
int64_t debugFD() const override {
|
||||
if( !file.getPtr() )
|
||||
throw io_error().asInjectedFault();
|
||||
return file->debugFD();
|
||||
}
|
||||
std::string getFilename() {
|
||||
std::string getFilename() const override {
|
||||
if( !file.getPtr() )
|
||||
throw io_error().asInjectedFault();
|
||||
return file->getFilename();
|
||||
|
@ -137,7 +137,7 @@ public:
|
|||
std::string filename;
|
||||
|
||||
//An approximation of the size of the file; .size() should be used instead of this variable in most cases
|
||||
int64_t approximateSize;
|
||||
mutable int64_t approximateSize;
|
||||
|
||||
//The address of the machine that opened the file
|
||||
NetworkAddress openedAddress;
|
||||
|
@ -263,13 +263,11 @@ public:
|
|||
}
|
||||
|
||||
//Passes along reads straight to the underlying file, waiting for any outstanding changes that could affect the results
|
||||
Future<int> read(void *data, int length, int64_t offset) {
|
||||
return read(this, data, length, offset);
|
||||
}
|
||||
Future<int> read(void* data, int length, int64_t offset) override { return read(this, data, length, offset); }
|
||||
|
||||
//Writes data to the file. Writes are delayed a random amount of time before being
|
||||
//passed to the underlying file
|
||||
Future<Void> write(void const *data, int length, int64_t offset) {
|
||||
Future<Void> write(void const* data, int length, int64_t offset) override {
|
||||
//TraceEvent("AsyncFileNonDurable_Write", id).detail("Filename", filename).detail("Offset", offset).detail("Length", length);
|
||||
if(length == 0) {
|
||||
TraceEvent(SevWarnAlways, "AsyncFileNonDurable_EmptyModification", id).detail("Filename", filename);
|
||||
|
@ -283,10 +281,10 @@ public:
|
|||
writeEnded.send(write(this, writeStarted, writeEnded.getFuture(), data, length, offset));
|
||||
return writeStarted.getFuture();
|
||||
}
|
||||
|
||||
|
||||
//Truncates the file. Truncates are delayed a random amount of time before being
|
||||
//passed to the underlying file
|
||||
Future<Void> truncate(int64_t size) {
|
||||
Future<Void> truncate(int64_t size) override {
|
||||
//TraceEvent("AsyncFileNonDurable_Truncate", id).detail("Filename", filename).detail("Offset", size);
|
||||
debugFileTruncate("AsyncFileNonDurableTruncate", filename, size);
|
||||
|
||||
|
@ -306,17 +304,11 @@ public:
|
|||
}
|
||||
|
||||
//Passes along size requests to the underlying file, augmenting with any writes past the end of the file
|
||||
Future<int64_t> size() {
|
||||
return size(this);
|
||||
}
|
||||
Future<int64_t> size() const override { return size(this); }
|
||||
|
||||
int64_t debugFD() {
|
||||
return file->debugFD();
|
||||
}
|
||||
int64_t debugFD() const override { return file->debugFD(); }
|
||||
|
||||
std::string getFilename() {
|
||||
return file->getFilename();
|
||||
}
|
||||
std::string getFilename() const override { return file->getFilename(); }
|
||||
|
||||
//Forces a non-durable sync (some writes are not made or made incorrectly)
|
||||
//This is used when the file should 'die' without first completing its operations
|
||||
|
@ -358,7 +350,7 @@ private:
|
|||
}
|
||||
|
||||
//Checks if the file is killed. If so, then the current sync is completed if running and then an error is thrown
|
||||
ACTOR Future<Void> checkKilled(AsyncFileNonDurable *self, std::string context) {
|
||||
ACTOR static Future<Void> checkKilled(AsyncFileNonDurable const* self, std::string context) {
|
||||
if(self->killed.isSet()) {
|
||||
//TraceEvent("AsyncFileNonDurable_KilledInCheck", self->id).detail("In", context).detail("Filename", self->filename);
|
||||
wait(self->killComplete.getFuture());
|
||||
|
@ -372,14 +364,14 @@ private:
|
|||
|
||||
//Passes along reads straight to the underlying file, waiting for any outstanding changes that could affect the results
|
||||
ACTOR Future<int> onRead(AsyncFileNonDurable *self, void *data, int length, int64_t offset) {
|
||||
wait(self->checkKilled(self, "Read"));
|
||||
wait(checkKilled(self, "Read"));
|
||||
vector<Future<Void>> priorModifications = self->getModificationsAndInsert(offset, length);
|
||||
wait(waitForAll(priorModifications));
|
||||
state Future<int> readFuture = self->file->read(data, length, offset);
|
||||
wait( success( readFuture ) || self->killed.getFuture() );
|
||||
|
||||
// throws if we were killed
|
||||
wait(self->checkKilled(self, "ReadEnd"));
|
||||
wait(checkKilled(self, "ReadEnd"));
|
||||
|
||||
debugFileCheck("AsyncFileNonDurableRead", self->filename, data, offset, length);
|
||||
|
||||
|
@ -421,7 +413,7 @@ private:
|
|||
|
||||
try {
|
||||
//TraceEvent("AsyncFileNonDurable_Write", self->id).detail("Delay", delayDuration).detail("Filename", self->filename).detail("WriteLength", length).detail("Offset", offset);
|
||||
wait(self->checkKilled(self, "Write"));
|
||||
wait(checkKilled(self, "Write"));
|
||||
|
||||
Future<Void> writeEnded = wait(ownFuture);
|
||||
std::vector<Future<Void>> priorModifications = self->getModificationsAndInsert(offset, length, true, writeEnded);
|
||||
|
@ -543,7 +535,7 @@ private:
|
|||
|
||||
try {
|
||||
//TraceEvent("AsyncFileNonDurable_Truncate", self->id).detail("Delay", delayDuration).detail("Filename", self->filename);
|
||||
wait(self->checkKilled(self, "Truncate"));
|
||||
wait(checkKilled(self, "Truncate"));
|
||||
|
||||
Future<Void> truncateEnded = wait(ownFuture);
|
||||
std::vector<Future<Void>> priorModifications = self->getModificationsAndInsert(size, -1, true, truncateEnded);
|
||||
|
@ -600,8 +592,8 @@ private:
|
|||
wait(waitUntilDiskReady(self->diskParameters, 0, true) || self->killed.getFuture());
|
||||
}
|
||||
|
||||
wait(self->checkKilled(self, durable ? "Sync" : "Kill"));
|
||||
|
||||
wait(checkKilled(self, durable ? "Sync" : "Kill"));
|
||||
|
||||
if(!durable)
|
||||
self->killed.send( Void() );
|
||||
|
||||
|
@ -653,7 +645,7 @@ private:
|
|||
}
|
||||
//A killed file cannot be allowed to report that it successfully synced
|
||||
else {
|
||||
wait(self->checkKilled(self, "SyncEnd"));
|
||||
wait(checkKilled(self, "SyncEnd"));
|
||||
wait(self->file->sync());
|
||||
//TraceEvent("AsyncFileNonDurable_ImplSyncEnd", self->id).detail("Filename", self->filename).detail("Durable", durable);
|
||||
}
|
||||
|
@ -679,13 +671,13 @@ private:
|
|||
}
|
||||
|
||||
//Passes along size requests to the underlying file, augmenting with any writes past the end of the file
|
||||
ACTOR Future<int64_t> onSize(AsyncFileNonDurable *self) {
|
||||
ACTOR static Future<int64_t> onSize(AsyncFileNonDurable const* self) {
|
||||
//TraceEvent("AsyncFileNonDurable_Size", self->id).detail("Filename", self->filename);
|
||||
wait(self->checkKilled(self, "Size"));
|
||||
wait(checkKilled(self, "Size"));
|
||||
state Future<int64_t> sizeFuture = self->file->size();
|
||||
wait( success( sizeFuture ) || self->killed.getFuture() );
|
||||
|
||||
wait(self->checkKilled(self, "SizeEnd"));
|
||||
wait(checkKilled(self, "SizeEnd"));
|
||||
|
||||
//Include any modifications which extend past the end of the file
|
||||
uint64_t maxModification = self->pendingModifications.lastItem().begin();
|
||||
|
@ -693,14 +685,14 @@ private:
|
|||
return self->approximateSize;
|
||||
}
|
||||
|
||||
ACTOR Future<int64_t> size(AsyncFileNonDurable *self) {
|
||||
ACTOR static Future<int64_t> size(AsyncFileNonDurable const* self) {
|
||||
state ISimulator::ProcessInfo* currentProcess = g_simulator.getCurrentProcess();
|
||||
state TaskPriority currentTaskID = g_network->getCurrentTask();
|
||||
|
||||
wait( g_simulator.onMachine( currentProcess ) );
|
||||
|
||||
try {
|
||||
state int64_t rep = wait( self->onSize( self ) );
|
||||
state int64_t rep = wait(onSize(self));
|
||||
wait( g_simulator.onProcess( currentProcess, currentTaskID ) );
|
||||
|
||||
return rep;
|
||||
|
|
|
@ -155,27 +155,27 @@ public:
|
|||
return wpos;
|
||||
}
|
||||
|
||||
virtual Future<int> read( void *data, int length, int64_t offset ) {
|
||||
Future<int> read(void* data, int length, int64_t offset) override {
|
||||
return read_impl(Reference<AsyncFileReadAheadCache>::addRef(this), data, length, offset);
|
||||
}
|
||||
|
||||
virtual Future<Void> write( void const *data, int length, int64_t offset ) { throw file_not_writable(); }
|
||||
virtual Future<Void> truncate( int64_t size ) { throw file_not_writable(); }
|
||||
Future<Void> write(void const* data, int length, int64_t offset) override { throw file_not_writable(); }
|
||||
Future<Void> truncate(int64_t size) override { throw file_not_writable(); }
|
||||
|
||||
virtual Future<Void> sync() { return Void(); }
|
||||
virtual Future<Void> flush() { return Void(); }
|
||||
Future<Void> sync() override { return Void(); }
|
||||
Future<Void> flush() override { return Void(); }
|
||||
|
||||
virtual Future<int64_t> size() { return m_f->size(); }
|
||||
Future<int64_t> size() const override { return m_f->size(); }
|
||||
|
||||
virtual Future<Void> readZeroCopy( void** data, int* length, int64_t offset ) {
|
||||
Future<Void> readZeroCopy(void** data, int* length, int64_t offset) override {
|
||||
TraceEvent(SevError, "ReadZeroCopyNotSupported").detail("FileType", "ReadAheadCache");
|
||||
return platform_error();
|
||||
}
|
||||
virtual void releaseZeroCopy( void* data, int length, int64_t offset ) {}
|
||||
void releaseZeroCopy(void* data, int length, int64_t offset) override {}
|
||||
|
||||
virtual int64_t debugFD() { return -1; }
|
||||
int64_t debugFD() const override { return -1; }
|
||||
|
||||
virtual std::string getFilename() { return m_f->getFilename(); }
|
||||
std::string getFilename() const override { return m_f->getFilename(); }
|
||||
|
||||
virtual ~AsyncFileReadAheadCache() {
|
||||
for(auto &it : m_blocks) {
|
||||
|
|
|
@ -87,7 +87,7 @@ public:
|
|||
virtual void addref() { ReferenceCounted<AsyncFileWinASIO>::addref(); }
|
||||
virtual void delref() { ReferenceCounted<AsyncFileWinASIO>::delref(); }
|
||||
|
||||
virtual int64_t debugFD() { return (int64_t)file.native_handle(); }
|
||||
int64_t debugFD() const override { return (int64_t)(const_cast<decltype(file)&>(file).native_handle()); }
|
||||
|
||||
static void onReadReady( Promise<int> onReady, const boost::system::error_code& error, size_t bytesRead ) {
|
||||
if (error) {
|
||||
|
@ -116,7 +116,7 @@ public:
|
|||
}
|
||||
}
|
||||
|
||||
virtual Future<int> read( void* data, int length, int64_t offset ) {
|
||||
Future<int> read(void* data, int length, int64_t offset) override {
|
||||
// the size call is set inline
|
||||
auto end = this->size().get();
|
||||
//TraceEvent("WinAsyncRead").detail("Offset", offset).detail("Length", length).detail("FileSize", end).detail("FileName", filename);
|
||||
|
@ -128,7 +128,7 @@ public:
|
|||
|
||||
return result.getFuture();
|
||||
}
|
||||
virtual Future<Void> write( void const* data, int length, int64_t offset ) {
|
||||
Future<Void> write(void const* data, int length, int64_t offset) override {
|
||||
/*
|
||||
FIXME
|
||||
if ( length + offset >= fileValidData ) {
|
||||
|
@ -139,7 +139,7 @@ public:
|
|||
boost::asio::async_write_at( file, offset, boost::asio::const_buffers_1( data, length ), boost::bind( &onWriteReady, result, length, boost::asio::placeholders::error, boost::asio::placeholders::bytes_transferred ) );
|
||||
return result.getFuture();
|
||||
}
|
||||
virtual Future<Void> truncate( int64_t size ) {
|
||||
Future<Void> truncate(int64_t size) override {
|
||||
// FIXME: Possibly use SetFileInformationByHandle( file.native_handle(), FileEndOfFileInfo, ... ) instead
|
||||
if (!SetFilePointerEx( file.native_handle(), *(LARGE_INTEGER*)&size, NULL, FILE_BEGIN ))
|
||||
throw io_error();
|
||||
|
@ -147,7 +147,7 @@ public:
|
|||
throw io_error();
|
||||
return Void();
|
||||
}
|
||||
virtual Future<Void> sync() {
|
||||
Future<Void> sync() override {
|
||||
// FIXME: Do FlushFileBuffers in a worker thread (using g_network->createThreadPool)?
|
||||
if (!FlushFileBuffers( file.native_handle() )) throw io_error();
|
||||
|
||||
|
@ -159,14 +159,12 @@ public:
|
|||
|
||||
return Void();
|
||||
}
|
||||
virtual Future<int64_t> size() {
|
||||
Future<int64_t> size() const override {
|
||||
LARGE_INTEGER s;
|
||||
if (!GetFileSizeEx(file.native_handle(), &s)) throw io_error();
|
||||
if (!GetFileSizeEx(const_cast<decltype(file)&>(file).native_handle(), &s)) throw io_error();
|
||||
return *(int64_t*)&s;
|
||||
}
|
||||
virtual std::string getFilename() {
|
||||
return filename;
|
||||
}
|
||||
std::string getFilename() const override { return filename; }
|
||||
|
||||
~AsyncFileWinASIO() { }
|
||||
|
||||
|
|
|
@ -60,10 +60,10 @@ public:
|
|||
|
||||
Future<Void> sync() { return m_f->sync(); }
|
||||
Future<Void> flush() { return m_f->flush(); }
|
||||
Future<int64_t> size() { return m_f->size(); }
|
||||
std::string getFilename() { return m_f->getFilename(); }
|
||||
Future<int64_t> size() const override { return m_f->size(); }
|
||||
std::string getFilename() const override { return m_f->getFilename(); }
|
||||
void releaseZeroCopy( void* data, int length, int64_t offset ) { return m_f->releaseZeroCopy(data, length, offset); }
|
||||
int64_t debugFD() { return m_f->debugFD(); }
|
||||
int64_t debugFD() const override { return m_f->debugFD(); }
|
||||
|
||||
AsyncFileWriteChecker(Reference<IAsyncFile> f) : m_f(f) {
|
||||
// Initialize the static history budget the first time (and only the first time) a file is opened.
|
||||
|
|
|
@ -551,6 +551,7 @@ ACTOR Future<Void> connectionKeeper( Reference<Peer> self,
|
|||
}
|
||||
} else {
|
||||
self->outgoingConnectionIdle = false;
|
||||
self->lastConnectTime = now();
|
||||
}
|
||||
|
||||
firstConnFailedTime.reset();
|
||||
|
@ -724,7 +725,7 @@ void Peer::onIncomingConnection( Reference<Peer> self, Reference<IConnection> co
|
|||
compatibleAddr = transport->localAddresses.secondaryAddress.get();
|
||||
}
|
||||
|
||||
if ( !destination.isPublic() || outgoingConnectionIdle || destination > compatibleAddr ) {
|
||||
if ( !destination.isPublic() || outgoingConnectionIdle || destination > compatibleAddr || (lastConnectTime > 1.0 && now() - lastConnectTime > FLOW_KNOBS->ALWAYS_ACCEPT_DELAY) ) {
|
||||
// Keep the new connection
|
||||
TraceEvent("IncomingConnection", conn->getDebugID())
|
||||
.suppressFor(1.0)
|
||||
|
|
|
@ -63,8 +63,8 @@ public:
|
|||
virtual Future<Void> truncate( int64_t size ) = 0;
|
||||
virtual Future<Void> sync() = 0;
|
||||
virtual Future<Void> flush() { return Void(); } // Sends previous writes to the OS if they have been buffered in memory, but does not make them power safe
|
||||
virtual Future<int64_t> size() = 0;
|
||||
virtual std::string getFilename() = 0;
|
||||
virtual Future<int64_t> size() const = 0;
|
||||
virtual std::string getFilename() const = 0;
|
||||
|
||||
// Attempt to read the *length bytes at offset without copying. If successful, a pointer to the
|
||||
// requested bytes is written to *data, and the number of bytes successfully read is
|
||||
|
@ -80,7 +80,7 @@ public:
|
|||
virtual Future<Void> readZeroCopy( void** data, int* length, int64_t offset ) { return io_error(); }
|
||||
virtual void releaseZeroCopy( void* data, int length, int64_t offset ) {}
|
||||
|
||||
virtual int64_t debugFD() = 0;
|
||||
virtual int64_t debugFD() const = 0;
|
||||
};
|
||||
|
||||
typedef void (*runCycleFuncPtr)();
|
||||
|
|
|
@ -458,8 +458,8 @@ Future< REPLY_TYPE(Request) > loadBalance(
|
|||
// Subclasses must initialize all members in their default constructors
|
||||
// Subclasses must serialize all members
|
||||
struct BasicLoadBalancedReply {
|
||||
int recentRequests;
|
||||
BasicLoadBalancedReply() : recentRequests(0) {}
|
||||
int processBusyTime;
|
||||
BasicLoadBalancedReply() : processBusyTime(0) {}
|
||||
};
|
||||
|
||||
Optional<BasicLoadBalancedReply> getBasicLoadBalancedReply(const BasicLoadBalancedReply *reply);
|
||||
|
@ -528,7 +528,7 @@ Future< REPLY_TYPE(Request) > basicLoadBalance(
|
|||
if(result.present()) {
|
||||
Optional<BasicLoadBalancedReply> loadBalancedReply = getBasicLoadBalancedReply(&result.get());
|
||||
if(loadBalancedReply.present()) {
|
||||
alternatives->updateRecent( useAlt, loadBalancedReply.get().recentRequests );
|
||||
alternatives->updateRecent( useAlt, loadBalancedReply.get().processBusyTime );
|
||||
}
|
||||
|
||||
return result.get();
|
||||
|
|
|
@ -71,6 +71,29 @@ ProcessClass::Fitness ProcessClass::machineClassFitness( ClusterRole role ) cons
|
|||
return ProcessClass::GoodFit;
|
||||
case ProcessClass::UnsetClass:
|
||||
return ProcessClass::UnsetFit;
|
||||
case ProcessClass::GrvProxyClass:
|
||||
return ProcessClass::OkayFit;
|
||||
case ProcessClass::ResolutionClass:
|
||||
return ProcessClass::OkayFit;
|
||||
case ProcessClass::TransactionClass:
|
||||
return ProcessClass::OkayFit;
|
||||
case ProcessClass::CoordinatorClass:
|
||||
case ProcessClass::TesterClass:
|
||||
case ProcessClass::StorageCacheClass:
|
||||
return ProcessClass::NeverAssign;
|
||||
default:
|
||||
return ProcessClass::WorstFit;
|
||||
}
|
||||
case ProcessClass::GrvProxy:
|
||||
switch (_class) {
|
||||
case ProcessClass::GrvProxyClass:
|
||||
return ProcessClass::BestFit;
|
||||
case ProcessClass::StatelessClass:
|
||||
return ProcessClass::GoodFit;
|
||||
case ProcessClass::UnsetClass:
|
||||
return ProcessClass::UnsetFit;
|
||||
case ProcessClass::ProxyClass:
|
||||
return ProcessClass::OkayFit;
|
||||
case ProcessClass::ResolutionClass:
|
||||
return ProcessClass::OkayFit;
|
||||
case ProcessClass::TransactionClass:
|
||||
|
@ -171,6 +194,8 @@ ProcessClass::Fitness ProcessClass::machineClassFitness( ClusterRole role ) cons
|
|||
return ProcessClass::OkayFit;
|
||||
case ProcessClass::ProxyClass:
|
||||
return ProcessClass::OkayFit;
|
||||
case ProcessClass::GrvProxyClass:
|
||||
return ProcessClass::OkayFit;
|
||||
case ProcessClass::LogRouterClass:
|
||||
return ProcessClass::OkayFit;
|
||||
case ProcessClass::CoordinatorClass:
|
||||
|
|
|
@ -33,7 +33,8 @@ struct ProcessClass {
|
|||
TransactionClass,
|
||||
ResolutionClass,
|
||||
TesterClass,
|
||||
ProxyClass,
|
||||
ProxyClass, // Process class of CommitProxy
|
||||
GrvProxyClass,
|
||||
MasterClass,
|
||||
StatelessClass,
|
||||
LogClass,
|
||||
|
@ -49,7 +50,21 @@ struct ProcessClass {
|
|||
};
|
||||
|
||||
enum Fitness { BestFit, GoodFit, UnsetFit, OkayFit, WorstFit, ExcludeFit, NeverAssign }; //cannot be larger than 7 because of leader election mask
|
||||
enum ClusterRole { Storage, TLog, Proxy, Master, Resolver, LogRouter, ClusterController, DataDistributor, Ratekeeper, StorageCache, Backup, NoRole };
|
||||
enum ClusterRole {
|
||||
Storage,
|
||||
TLog,
|
||||
Proxy,
|
||||
GrvProxy,
|
||||
Master,
|
||||
Resolver,
|
||||
LogRouter,
|
||||
ClusterController,
|
||||
DataDistributor,
|
||||
Ratekeeper,
|
||||
StorageCache,
|
||||
Backup,
|
||||
NoRole
|
||||
};
|
||||
enum ClassSource { CommandLineSource, AutoSource, DBSource, InvalidSource = -1 };
|
||||
int16_t _class;
|
||||
int16_t _source;
|
||||
|
@ -63,6 +78,7 @@ public:
|
|||
else if (s=="transaction") _class = TransactionClass;
|
||||
else if (s=="resolution") _class = ResolutionClass;
|
||||
else if (s=="proxy") _class = ProxyClass;
|
||||
else if (s=="grv_proxy") _class = GrvProxyClass;
|
||||
else if (s=="master") _class = MasterClass;
|
||||
else if (s=="test") _class = TesterClass;
|
||||
else if (s=="unset") _class = UnsetClass;
|
||||
|
@ -84,6 +100,7 @@ public:
|
|||
else if (classStr=="transaction") _class = TransactionClass;
|
||||
else if (classStr=="resolution") _class = ResolutionClass;
|
||||
else if (classStr=="proxy") _class = ProxyClass;
|
||||
else if (classStr=="grv_proxy") _class = GrvProxyClass;
|
||||
else if (classStr=="master") _class = MasterClass;
|
||||
else if (classStr=="test") _class = TesterClass;
|
||||
else if (classStr=="unset") _class = UnsetClass;
|
||||
|
@ -121,6 +138,7 @@ public:
|
|||
case TransactionClass: return "transaction";
|
||||
case ResolutionClass: return "resolution";
|
||||
case ProxyClass: return "proxy";
|
||||
case GrvProxyClass: return "grv_proxy";
|
||||
case MasterClass: return "master";
|
||||
case TesterClass: return "test";
|
||||
case StatelessClass: return "stateless";
|
||||
|
|
|
@ -67,10 +67,10 @@ struct AlternativeInfo {
|
|||
T interf;
|
||||
double probability;
|
||||
double cumulativeProbability;
|
||||
int recentRequests;
|
||||
int processBusyTime;
|
||||
double lastUpdate;
|
||||
|
||||
AlternativeInfo(T const& interf, double probability, double cumulativeProbability) : interf(interf), probability(probability), cumulativeProbability(cumulativeProbability), recentRequests(-1), lastUpdate(0) {}
|
||||
AlternativeInfo(T const& interf, double probability, double cumulativeProbability) : interf(interf), probability(probability), cumulativeProbability(cumulativeProbability), processBusyTime(-1), lastUpdate(0) {}
|
||||
|
||||
bool operator < (double const& r) const {
|
||||
return cumulativeProbability < r;
|
||||
|
@ -105,26 +105,28 @@ public:
|
|||
return std::lower_bound( alternatives.begin(), alternatives.end(), deterministicRandom()->random01() ) - alternatives.begin();
|
||||
}
|
||||
|
||||
void updateRecent( int index, int recentRequests ) {
|
||||
alternatives[index].recentRequests = recentRequests;
|
||||
void updateRecent( int index, int processBusyTime ) {
|
||||
alternatives[index].processBusyTime = processBusyTime;
|
||||
alternatives[index].lastUpdate = now();
|
||||
}
|
||||
|
||||
void updateProbabilities() {
|
||||
double totalRequests = 0;
|
||||
double totalBusyTime = 0;
|
||||
for(auto& it : alternatives) {
|
||||
totalRequests += it.recentRequests;
|
||||
totalBusyTime += it.processBusyTime;
|
||||
if(now() - it.lastUpdate > FLOW_KNOBS->BASIC_LOAD_BALANCE_UPDATE_RATE/2.0) {
|
||||
return;
|
||||
}
|
||||
}
|
||||
if(totalRequests < 1000) {
|
||||
|
||||
//Do not update probabilities if the average proxy busyness is less than 5%
|
||||
if(totalBusyTime < FLOW_KNOBS->BASIC_LOAD_BALANCE_MIN_AMOUNT*alternatives.size()) {
|
||||
return;
|
||||
}
|
||||
|
||||
double totalProbability = 0;
|
||||
for(auto& it : alternatives) {
|
||||
it.probability += (1.0/alternatives.size()-(it.recentRequests/totalRequests))*FLOW_KNOBS->BASIC_LOAD_BALANCE_MAX_CHANGE;
|
||||
it.probability += (1.0/alternatives.size()-(it.processBusyTime/totalBusyTime))*FLOW_KNOBS->BASIC_LOAD_BALANCE_MAX_CHANGE;
|
||||
it.probability = std::max(it.probability, 1/(FLOW_KNOBS->BASIC_LOAD_BALANCE_MAX_PROB*alternatives.size()));
|
||||
it.probability = std::min(it.probability, FLOW_KNOBS->BASIC_LOAD_BALANCE_MAX_PROB/alternatives.size());
|
||||
totalProbability += it.probability;
|
||||
|
|
|
@ -164,6 +164,11 @@ public:
|
|||
i.decrementNonEnd();
|
||||
return iterator(i);
|
||||
}
|
||||
const_iterator lastItem() const {
|
||||
auto i(map.lastItem());
|
||||
i.decrementNonEnd();
|
||||
return const_iterator(i);
|
||||
}
|
||||
int size() const { return map.size() - 1; } // We always have one range bounded by two entries
|
||||
iterator randomRange() { return iterator(map.index(deterministicRandom()->randomInt(0, map.size() - 1))); }
|
||||
const_iterator randomRange() const {
|
||||
|
@ -275,4 +280,4 @@ void RangeMap<Key,Val,Range,Metric,MetricFunc>::insert( const Range& keys, const
|
|||
map.insert(beginPair, true, mf(beginPair));
|
||||
}
|
||||
|
||||
#endif
|
||||
#endif
|
||||
|
|
|
@ -53,29 +53,17 @@ bool IReplicationPolicy::validateFull(
|
|||
|
||||
if (!solved) {
|
||||
if (validate(totalSolution, fromServers)) {
|
||||
if (g_replicationdebug > 2) {
|
||||
printf("Error: Validate unsolved policy with%3lu also servers and%3lu solution servers\n", alsoServers.size(), solutionSet.size());
|
||||
}
|
||||
valid = false;
|
||||
}
|
||||
else if (validate(fromServers->getGroupEntries(), fromServers)) {
|
||||
if (g_replicationdebug > 2) {
|
||||
printf("Error: Validated unsolved policy with all%5d servers\n", fromServers->size());
|
||||
}
|
||||
valid = false;
|
||||
}
|
||||
}
|
||||
else if (!validate(totalSolution, fromServers)) {
|
||||
if (g_replicationdebug > 2) {
|
||||
printf("Error: Failed to validate solved policy with%3lu also servers and%3lu solution servers\n", alsoServers.size(), solutionSet.size());
|
||||
}
|
||||
valid = false;
|
||||
}
|
||||
else if (solutionSet.empty()) {
|
||||
if (!validate(alsoServers, fromServers)) {
|
||||
if (g_replicationdebug > 2) {
|
||||
printf("Error: Failed to validate policy with only%3lu also servers\n", alsoServers.size());
|
||||
}
|
||||
valid = false;
|
||||
}
|
||||
}
|
||||
|
@ -85,14 +73,7 @@ bool IReplicationPolicy::validateFull(
|
|||
totalSolution[lastSolutionIndex] = totalSolution.back();
|
||||
totalSolution.pop_back();
|
||||
for (int index = 0; index < solutionSet.size() && index < totalSolution.size(); index ++) {
|
||||
if (g_replicationdebug > 3) {
|
||||
auto fromServer = fromServers->getRecordViaEntry(missingEntry);
|
||||
printf("Test remove entry: %s test:%3d of%3lu\n", fromServers->getEntryInfo(missingEntry).c_str(), index+1, solutionSet.size());
|
||||
}
|
||||
if (validate(totalSolution, fromServers)) {
|
||||
if (g_replicationdebug > 2) {
|
||||
printf("Invalid extra entry: %s\n", fromServers->getEntryInfo(missingEntry).c_str());
|
||||
}
|
||||
valid = false;
|
||||
break;
|
||||
}
|
||||
|
@ -119,9 +100,6 @@ bool PolicyOne::selectReplicas(
|
|||
itemsUsed ++;
|
||||
totalUsed ++;
|
||||
}
|
||||
if (g_replicationdebug > 0) {
|
||||
printf("PolicyOne used:%5d results:%3d from %3d servers\n", totalUsed, itemsUsed, fromServers->size());
|
||||
}
|
||||
return (totalUsed > 0);
|
||||
}
|
||||
|
||||
|
@ -205,50 +183,16 @@ bool PolicyAcross::validate(
|
|||
}
|
||||
}
|
||||
if (validMap.size() < _count) {
|
||||
if (g_replicationdebug > 3) {
|
||||
printf("Across too few values:%3lu <%2d key: %-7s policy: %-10s => %s\n", validMap.size(), _count, _attribKey.c_str(), _policy->name().c_str(), _policy->info().c_str());
|
||||
}
|
||||
valid = false;
|
||||
}
|
||||
else {
|
||||
if (g_replicationdebug > 3) {
|
||||
printf("Across check values:%9lu key: %-7s solutions:%2lu count:%2d policy: %-10s => %s\n", validMap.size(), _attribKey.c_str(), solutionSet.size(), _count, _policy->name().c_str(), _policy->info().c_str());
|
||||
for (auto& itValue : validMap) {
|
||||
printf(" value: (%3d) %-10s\n", itValue.first._id, fromServers->valueText(itValue.first).c_str());
|
||||
}
|
||||
}
|
||||
for (auto& itValid : validMap) {
|
||||
// itValid.second is the vector of LocalityEntries that belong to the same locality
|
||||
if (_policy->validate(itValid.second, fromServers)) {
|
||||
if (g_replicationdebug > 4) {
|
||||
printf("Across valid solution: %6lu key: %-7s count:%3d of%3d value: (%3d) %-10s policy: %-10s => "
|
||||
"%s\n",
|
||||
itValid.second.size(), _attribKey.c_str(), count + 1, _count, itValid.first._id,
|
||||
fromServers->valueText(itValid.first).c_str(), _policy->name().c_str(),
|
||||
_policy->info().c_str());
|
||||
if (g_replicationdebug > 5) {
|
||||
for (auto& entry : itValid.second) {
|
||||
printf(" entry: %s\n", fromServers->getEntryInfo(entry).c_str());
|
||||
}
|
||||
}
|
||||
}
|
||||
count ++;
|
||||
} else if (g_replicationdebug > 4) {
|
||||
printf("Across invalid solution:%5lu key: %-7s value: (%3d) %-10s policy: %-10s => %s\n", itValid.second.size(), _attribKey.c_str(), itValid.first._id, fromServers->valueText(itValid.first).c_str(), _policy->name().c_str(), _policy->info().c_str());
|
||||
if (g_replicationdebug > 5) {
|
||||
for (auto& entry : itValid.second) {
|
||||
printf(" entry: %s\n", fromServers->getEntryInfo(entry).c_str());
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
if (count < _count) {
|
||||
if (g_replicationdebug > 3) {
|
||||
printf("Across failed solution: %3lu key: %-7s values:%3lu count: %d=%d policy: %-10s => %s\n", solutionSet.size(), _attribKey.c_str(), validMap.size(), count, _count, _policy->name().c_str(), _policy->info().c_str());
|
||||
for (auto& entry : solutionSet) {
|
||||
printf(" entry: %s\n", fromServers->getEntryInfo(entry).c_str());
|
||||
}
|
||||
}
|
||||
valid = false;
|
||||
}
|
||||
}
|
||||
|
@ -277,9 +221,6 @@ bool PolicyAcross::selectReplicas(
|
|||
_newResults.clear();
|
||||
_addedResults.resize(_arena, 0);
|
||||
|
||||
if (g_replicationdebug > 0) {
|
||||
printf("Across !also:%4lu key: %-7s policy: %-10s => %s\n", alsoServers.size(), _attribKey.c_str(), _policy->name().c_str(), _policy->info().c_str());
|
||||
}
|
||||
for (auto& alsoServer : alsoServers) {
|
||||
auto value = fromServers->getValueViaGroupKey(alsoServer, groupIndexKey);
|
||||
if (value.present()) {
|
||||
|
@ -287,16 +228,6 @@ bool PolicyAcross::selectReplicas(
|
|||
if ((lowerBound == _usedValues.end()) || (*lowerBound != value.get())) {
|
||||
//_selected is a set of processes that have the same indexKey and indexValue (value)
|
||||
_selected = fromServers->restrict(indexKey, value.get());
|
||||
if (g_replicationdebug > 0) {
|
||||
if (_selected->size() > 0) {
|
||||
// entry is the locality entry info (entryValue) from the to-be-selected team member alsoServer
|
||||
printf("Across !select key: %-7s value: (%3d) %-10s entry: %s\n", _attribKey.c_str(),
|
||||
value.get()._id, fromServers->valueText(value.get()).c_str(),
|
||||
fromServers->getEntryInfo(alsoServer).c_str());
|
||||
} else {
|
||||
printf("Across !select empty\n");
|
||||
}
|
||||
}
|
||||
if (_selected->size()) {
|
||||
// Pass only the also array item which are valid for the value
|
||||
resultsSize = _newResults.size();
|
||||
|
@ -321,11 +252,6 @@ bool PolicyAcross::selectReplicas(
|
|||
if ((count < _count) && (_addedResults.size())) {
|
||||
// Sort the added results array
|
||||
std::sort(_addedResults.begin(), _addedResults.end(), PolicyAcross::compareAddedResults);
|
||||
|
||||
if (g_replicationdebug > 0) {
|
||||
printf("Across !add sets key: %-7s sets:%3d results:%3lu count:%3d of%3d\n", _attribKey.c_str(), _addedResults.size(), _newResults.size(), count, _count);
|
||||
}
|
||||
|
||||
if (g_replicationdebug > 0) {
|
||||
LocalitySet::staticDisplayEntries(fromServers, alsoServers, "also");
|
||||
LocalitySet::staticDisplayEntries(fromServers, results, "results");
|
||||
|
@ -334,9 +260,6 @@ bool PolicyAcross::selectReplicas(
|
|||
|
||||
for (auto& addedResult : _addedResults) {
|
||||
count ++;
|
||||
if (g_replicationdebug > 0) {
|
||||
printf("Across !add set key: %-7s count:%3d of%3d results:%3d index:%3d\n", _attribKey.c_str(), count, _count, addedResult.first, addedResult.second);
|
||||
}
|
||||
results.reserve(results.size() + addedResult.first);
|
||||
results.insert(results.end(), _newResults.begin()+addedResult.second, _newResults.begin()+addedResult.second+addedResult.first);
|
||||
if (count >= _count) break;
|
||||
|
@ -349,9 +272,6 @@ bool PolicyAcross::selectReplicas(
|
|||
// Cannot find replica from the least used alsoServers, now try to find replicas from all servers
|
||||
// Process the remaining values
|
||||
if (count < _count) {
|
||||
if (g_replicationdebug > 0) {
|
||||
printf("Across items:%4d key: %-7s policy: %-10s => %s count:%3d of%3d\n", fromServers->size(), _attribKey.c_str(), _policy->name().c_str(), _policy->info().c_str(), count, _count);
|
||||
}
|
||||
int recordIndex;
|
||||
// Use mutable array so that swaps does not affect actual element array
|
||||
auto& mutableArray = fromServers->getMutableEntries();
|
||||
|
@ -367,20 +287,8 @@ bool PolicyAcross::selectReplicas(
|
|||
if ((lowerBound == _usedValues.end()) || (*lowerBound != value.get())) {
|
||||
_selected = fromServers->restrict(indexKey, value.get());
|
||||
if (_selected->size()) {
|
||||
if (g_replicationdebug > 5) {
|
||||
printf("Across select:%3d key: %-7s value: (%3d) %-10s entry: %s index:%4d\n",
|
||||
fromServers->size() - checksLeft + 1, _attribKey.c_str(), value.get()._id,
|
||||
fromServers->valueText(value.get()).c_str(),
|
||||
fromServers->getEntryInfo(entry).c_str(), recordIndex);
|
||||
}
|
||||
if (_policy->selectReplicas(_selected, emptyEntryArray, results))
|
||||
{
|
||||
if (g_replicationdebug > 5) {
|
||||
printf("Across added:%4d key: %-7s value: (%3d) %-10s policy: %-10s => %s needed:%3d\n",
|
||||
count + 1, _attribKey.c_str(), value.get()._id,
|
||||
fromServers->valueText(value.get()).c_str(), _policy->name().c_str(),
|
||||
_policy->info().c_str(), _count);
|
||||
}
|
||||
count ++;
|
||||
if (count >= _count) break;
|
||||
_usedValues.insert(lowerBound, value.get());
|
||||
|
@ -395,13 +303,9 @@ bool PolicyAcross::selectReplicas(
|
|||
}
|
||||
// Clear the return array, if not satified
|
||||
if (count < _count) {
|
||||
if (g_replicationdebug > 0) printf("Across result count: %d < %d requested\n", count, _count);
|
||||
results.resize(resultsInit);
|
||||
count = 0;
|
||||
}
|
||||
if (g_replicationdebug > 0) {
|
||||
printf("Across used:%5lu results:%3d from %3d items key: %-7s policy: %-10s => %s\n", results.size()-resultsInit, count, fromServers->size(), _attribKey.c_str(), _policy->name().c_str(), _policy->info().c_str());
|
||||
}
|
||||
return (count >= _count);
|
||||
}
|
||||
|
||||
|
|
|
@ -478,31 +478,21 @@ public:
|
|||
virtual void addref() { ReferenceCounted<SimpleFile>::addref(); }
|
||||
virtual void delref() { ReferenceCounted<SimpleFile>::delref(); }
|
||||
|
||||
virtual int64_t debugFD() { return (int64_t)h; }
|
||||
int64_t debugFD() const override { return (int64_t)h; }
|
||||
|
||||
virtual Future<int> read( void* data, int length, int64_t offset ) {
|
||||
return read_impl( this, data, length, offset );
|
||||
}
|
||||
Future<int> read(void* data, int length, int64_t offset) override { return read_impl(this, data, length, offset); }
|
||||
|
||||
virtual Future<Void> write( void const* data, int length, int64_t offset ) {
|
||||
Future<Void> write(void const* data, int length, int64_t offset) override {
|
||||
return write_impl( this, StringRef((const uint8_t*)data, length), offset );
|
||||
}
|
||||
|
||||
virtual Future<Void> truncate( int64_t size ) {
|
||||
return truncate_impl( this, size );
|
||||
}
|
||||
Future<Void> truncate(int64_t size) override { return truncate_impl(this, size); }
|
||||
|
||||
virtual Future<Void> sync() {
|
||||
return sync_impl( this );
|
||||
}
|
||||
Future<Void> sync() override { return sync_impl(this); }
|
||||
|
||||
virtual Future<int64_t> size() {
|
||||
return size_impl( this );
|
||||
}
|
||||
Future<int64_t> size() const override { return size_impl(this); }
|
||||
|
||||
virtual std::string getFilename() {
|
||||
return actualFilename;
|
||||
}
|
||||
std::string getFilename() const override { return actualFilename; }
|
||||
|
||||
~SimpleFile() {
|
||||
_close( h );
|
||||
|
@ -667,7 +657,7 @@ private:
|
|||
return Void();
|
||||
}
|
||||
|
||||
ACTOR static Future<int64_t> size_impl( SimpleFile* self ) {
|
||||
ACTOR static Future<int64_t> size_impl(SimpleFile const* self) {
|
||||
state UID opId = deterministicRandom()->randomUniqueID();
|
||||
if (randLog)
|
||||
fprintf(randLog, "SFS1 %s %s %s\n", self->dbgId.shortString().c_str(), self->filename.c_str(), opId.shortString().c_str());
|
||||
|
|
|
@ -66,12 +66,16 @@ public:
|
|||
uint64_t fault_injection_r;
|
||||
double fault_injection_p1, fault_injection_p2;
|
||||
|
||||
UID uid;
|
||||
|
||||
ProcessInfo(const char* name, LocalityData locality, ProcessClass startingClass, NetworkAddressList addresses,
|
||||
INetworkConnections* net, const char* dataFolder, const char* coordinationFolder)
|
||||
: name(name), locality(locality), startingClass(startingClass), addresses(addresses),
|
||||
address(addresses.address), dataFolder(dataFolder), network(net), coordinationFolder(coordinationFolder),
|
||||
failed(false), excluded(false), rebooting(false), fault_injection_p1(0), fault_injection_p2(0),
|
||||
fault_injection_r(0), machine(0), cleared(false) {}
|
||||
fault_injection_r(0), machine(0), cleared(false) {
|
||||
uid = deterministicRandom()->randomUniqueID();
|
||||
}
|
||||
|
||||
Future<KillType> onShutdown() { return shutdownSignal.getFuture(); }
|
||||
|
||||
|
@ -86,7 +90,7 @@ public:
|
|||
return ss.str();
|
||||
}
|
||||
|
||||
// Returns true if the class represents an acceptable worker
|
||||
// Return true if the class type is suitable for stateful roles, such as tLog and StorageServer.
|
||||
bool isAvailableClass() const {
|
||||
switch (startingClass._class) {
|
||||
case ProcessClass::UnsetClass: return true;
|
||||
|
@ -94,9 +98,13 @@ public:
|
|||
case ProcessClass::TransactionClass: return true;
|
||||
case ProcessClass::ResolutionClass: return false;
|
||||
case ProcessClass::ProxyClass: return false;
|
||||
case ProcessClass::MasterClass: return false;
|
||||
case ProcessClass::TesterClass: return false;
|
||||
case ProcessClass::StatelessClass: return false;
|
||||
case ProcessClass::GrvProxyClass:
|
||||
return false;
|
||||
case ProcessClass::MasterClass:
|
||||
return false;
|
||||
case ProcessClass::TesterClass:
|
||||
return false;
|
||||
case ProcessClass::StatelessClass: return false;
|
||||
case ProcessClass::LogClass: return true;
|
||||
case ProcessClass::LogRouterClass: return false;
|
||||
case ProcessClass::ClusterControllerClass: return false;
|
||||
|
|
|
@ -296,7 +296,7 @@ void applyMetadataMutations(UID const& dbgid, Arena& arena, VectorRef<MutationRe
|
|||
}
|
||||
else if (m.param1 == minRequiredCommitVersionKey) {
|
||||
Version requested = BinaryReader::fromStringRef<Version>(m.param2, Unversioned());
|
||||
TraceEvent("MinRequiredCommitVersion", dbgid).detail("Min", requested).detail("Current", popVersion).detail("HasConf", !!confChange);
|
||||
TraceEvent("MinRequiredCommitVersion", dbgid).detail("Min", requested).detail("Current", popVersion);
|
||||
if(!initialCommit) txnStateStore->set(KeyValueRef(m.param1, m.param2));
|
||||
confChange = true;
|
||||
TEST(true); // Recovering at a higher version.
|
||||
|
|
|
@ -431,9 +431,9 @@ struct BackupData {
|
|||
GetReadVersionRequest request(span.context, 1, TransactionPriority::DEFAULT,
|
||||
GetReadVersionRequest::FLAG_USE_MIN_KNOWN_COMMITTED_VERSION);
|
||||
choose {
|
||||
when(wait(self->cx->onMasterProxiesChanged())) {}
|
||||
when(GetReadVersionReply reply = wait(basicLoadBalance(self->cx->getMasterProxies(false),
|
||||
&MasterProxyInterface::getConsistentReadVersion,
|
||||
when(wait(self->cx->onProxiesChanged())) {}
|
||||
when(GetReadVersionReply reply = wait(basicLoadBalance(self->cx->getGrvProxies(false),
|
||||
&GrvProxyInterface::getConsistentReadVersion,
|
||||
request, self->cx->taskID))) {
|
||||
return reply.version;
|
||||
}
|
||||
|
|
|
@ -23,6 +23,7 @@ set(FDBSERVER_SRCS
|
|||
fdbserver.actor.cpp
|
||||
FDBExecHelper.actor.cpp
|
||||
FDBExecHelper.actor.h
|
||||
GrvProxyServer.actor.cpp
|
||||
IDiskQueue.h
|
||||
IKeyValueStore.h
|
||||
IPager.h
|
||||
|
@ -151,12 +152,14 @@ set(FDBSERVER_SRCS
|
|||
workloads/Fuzz.cpp
|
||||
workloads/FuzzApiCorrectness.actor.cpp
|
||||
workloads/HealthMetricsApi.actor.cpp
|
||||
workloads/IncrementalBackup.actor.cpp
|
||||
workloads/Increment.actor.cpp
|
||||
workloads/IndexScan.actor.cpp
|
||||
workloads/Inventory.actor.cpp
|
||||
workloads/KVStoreTest.actor.cpp
|
||||
workloads/KillRegion.actor.cpp
|
||||
workloads/LockDatabase.actor.cpp
|
||||
workloads/LockDatabaseFrequently.actor.cpp
|
||||
workloads/LocalRatekeeper.actor.cpp
|
||||
workloads/LogMetrics.actor.cpp
|
||||
workloads/LowLatency.actor.cpp
|
||||
|
|
|
@ -414,7 +414,6 @@ public:
|
|||
TraceEvent("GetTLogTeamDone").detail("Completed", bCompleted).detail("Policy", policy->info()).detail("Results", results.size()).detail("Processes", logServerSet->size()).detail("Workers", id_worker.size())
|
||||
.detail("Required", required).detail("Desired", desired).detail("RatingTests",SERVER_KNOBS->POLICY_RATING_TESTS).detail("PolicyGenerations",SERVER_KNOBS->POLICY_GENERATIONS);
|
||||
|
||||
|
||||
return results;
|
||||
}
|
||||
|
||||
|
@ -514,20 +513,29 @@ public:
|
|||
throw no_more_servers();
|
||||
}
|
||||
|
||||
vector<WorkerDetails> getWorkersForRoleInDatacenter(Optional<Standalone<StringRef>> const& dcId, ProcessClass::ClusterRole role, int amount, DatabaseConfiguration const& conf, std::map< Optional<Standalone<StringRef>>, int>& id_used, Optional<WorkerFitnessInfo> minWorker = Optional<WorkerFitnessInfo>(), bool checkStable = false ) {
|
||||
std::map<std::pair<ProcessClass::Fitness,int>, std::pair<vector<WorkerDetails>,vector<WorkerDetails>>> fitness_workers;
|
||||
vector<WorkerDetails> getWorkersForRoleInDatacenter(
|
||||
Optional<Standalone<StringRef>> const& dcId, ProcessClass::ClusterRole role, int amount,
|
||||
DatabaseConfiguration const& conf, std::map<Optional<Standalone<StringRef>>, int>& id_used,
|
||||
Optional<WorkerFitnessInfo> minWorker = Optional<WorkerFitnessInfo>(), bool checkStable = false) {
|
||||
std::map<std::pair<ProcessClass::Fitness, int>, std::pair<vector<WorkerDetails>, vector<WorkerDetails>>>
|
||||
fitness_workers;
|
||||
vector<WorkerDetails> results;
|
||||
if(minWorker.present()) {
|
||||
if (minWorker.present()) {
|
||||
results.push_back(minWorker.get().worker);
|
||||
}
|
||||
if (amount <= results.size()) {
|
||||
return results;
|
||||
}
|
||||
|
||||
for( auto& it : id_worker ) {
|
||||
for (auto& it : id_worker) {
|
||||
auto fitness = it.second.details.processClass.machineClassFitness( role );
|
||||
if( workerAvailable(it.second, checkStable) && !conf.isExcludedServer(it.second.details.interf.addresses()) && it.second.details.interf.locality.dcId() == dcId &&
|
||||
( !minWorker.present() || ( it.second.details.interf.id() != minWorker.get().worker.interf.id() && ( fitness < minWorker.get().fitness || (fitness == minWorker.get().fitness && id_used[it.first] <= minWorker.get().used ) ) ) ) ) {
|
||||
if (workerAvailable(it.second, checkStable) &&
|
||||
!conf.isExcludedServer(it.second.details.interf.addresses()) &&
|
||||
it.second.details.interf.locality.dcId() == dcId &&
|
||||
(!minWorker.present() ||
|
||||
(it.second.details.interf.id() != minWorker.get().worker.interf.id() &&
|
||||
(fitness < minWorker.get().fitness ||
|
||||
(fitness == minWorker.get().fitness && id_used[it.first] <= minWorker.get().used))))) {
|
||||
if (isLongLivedStateless(it.first)) {
|
||||
fitness_workers[ std::make_pair(fitness, id_used[it.first]) ].second.push_back(it.second.details);
|
||||
} else {
|
||||
|
@ -543,8 +551,7 @@ public:
|
|||
for( int i=0; i < w.size(); i++ ) {
|
||||
results.push_back(w[i]);
|
||||
id_used[w[i].interf.locality.processId()]++;
|
||||
if( results.size() == amount )
|
||||
return results;
|
||||
if (results.size() == amount) return results;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -618,19 +625,27 @@ public:
|
|||
|
||||
struct RoleFitnessPair {
|
||||
RoleFitness proxy;
|
||||
RoleFitness grvProxy;
|
||||
RoleFitness resolver;
|
||||
|
||||
RoleFitnessPair() {}
|
||||
RoleFitnessPair(RoleFitness const& proxy, RoleFitness const& resolver) : proxy(proxy), resolver(resolver) {}
|
||||
RoleFitnessPair(RoleFitness const& proxy, RoleFitness const& grvProxy, RoleFitness const& resolver)
|
||||
: proxy(proxy), grvProxy(grvProxy), resolver(resolver) {}
|
||||
|
||||
bool operator < (RoleFitnessPair const& r) const {
|
||||
if(proxy.betterFitness(r.proxy)) {
|
||||
bool operator<(RoleFitnessPair const& r) const {
|
||||
if (proxy.betterFitness(r.proxy)) {
|
||||
return true;
|
||||
}
|
||||
if(r.proxy.betterFitness(proxy)) {
|
||||
return false;
|
||||
}
|
||||
if(resolver.betterFitness(r.resolver)) {
|
||||
if (grvProxy.betterFitness(r.grvProxy)) {
|
||||
return true;
|
||||
}
|
||||
if (r.grvProxy.betterFitness(grvProxy)) {
|
||||
return false;
|
||||
}
|
||||
if (resolver.betterFitness(r.resolver)) {
|
||||
return true;
|
||||
}
|
||||
if(r.resolver.betterFitness(resolver)) {
|
||||
|
@ -639,13 +654,18 @@ public:
|
|||
if(proxy.count != r.proxy.count) {
|
||||
return proxy.count > r.proxy.count;
|
||||
}
|
||||
if (grvProxy.count != r.grvProxy.count) {
|
||||
return grvProxy.count > r.grvProxy.count;
|
||||
}
|
||||
return resolver.count > r.resolver.count;
|
||||
}
|
||||
bool operator>(RoleFitnessPair const& r) const { return r < *this; }
|
||||
bool operator<=(RoleFitnessPair const& r) const { return !(*this > r); }
|
||||
bool operator>=(RoleFitnessPair const& r) const { return !(*this < r); }
|
||||
|
||||
bool operator == (RoleFitnessPair const& r) const { return proxy == r.proxy && resolver == r.resolver; }
|
||||
bool operator==(RoleFitnessPair const& r) const {
|
||||
return proxy == r.proxy && grvProxy == r.grvProxy && resolver == r.resolver;
|
||||
}
|
||||
};
|
||||
|
||||
std::set<Optional<Standalone<StringRef>>> getDatacenters( DatabaseConfiguration const& conf, bool checkStable = false ) {
|
||||
|
@ -725,23 +745,31 @@ public:
|
|||
}
|
||||
|
||||
std::vector<WorkerDetails> satelliteLogs;
|
||||
if(region.satelliteTLogReplicationFactor > 0 && req.configuration.usableRegions > 1) {
|
||||
satelliteLogs = getWorkersForSatelliteLogs( req.configuration, region, remoteRegion, id_used, result.satelliteFallback );
|
||||
if (region.satelliteTLogReplicationFactor > 0 && req.configuration.usableRegions > 1) {
|
||||
satelliteLogs =
|
||||
getWorkersForSatelliteLogs(req.configuration, region, remoteRegion, id_used, result.satelliteFallback);
|
||||
for(int i = 0; i < satelliteLogs.size(); i++) {
|
||||
result.satelliteTLogs.push_back(satelliteLogs[i].interf);
|
||||
}
|
||||
}
|
||||
|
||||
auto first_resolver = getWorkerForRoleInDatacenter( dcId, ProcessClass::Resolver, ProcessClass::ExcludeFit, req.configuration, id_used );
|
||||
auto first_proxy = getWorkerForRoleInDatacenter( dcId, ProcessClass::Proxy, ProcessClass::ExcludeFit, req.configuration, id_used );
|
||||
auto first_proxy = getWorkerForRoleInDatacenter(dcId, ProcessClass::Proxy, ProcessClass::ExcludeFit,
|
||||
req.configuration, id_used);
|
||||
auto first_grv_proxy = getWorkerForRoleInDatacenter(dcId, ProcessClass::GrvProxy, ProcessClass::ExcludeFit,
|
||||
req.configuration, id_used);
|
||||
auto first_resolver = getWorkerForRoleInDatacenter(dcId, ProcessClass::Resolver, ProcessClass::ExcludeFit,
|
||||
req.configuration, id_used);
|
||||
|
||||
auto proxies = getWorkersForRoleInDatacenter( dcId, ProcessClass::Proxy, req.configuration.getDesiredProxies(), req.configuration, id_used, first_proxy );
|
||||
auto proxies = getWorkersForRoleInDatacenter(dcId, ProcessClass::Proxy, req.configuration.getDesiredProxies(),
|
||||
req.configuration, id_used, first_proxy);
|
||||
auto grv_proxies =
|
||||
getWorkersForRoleInDatacenter(dcId, ProcessClass::GrvProxy, req.configuration.getDesiredGrvProxies(),
|
||||
req.configuration, id_used, first_grv_proxy);
|
||||
auto resolvers = getWorkersForRoleInDatacenter( dcId, ProcessClass::Resolver, req.configuration.getDesiredResolvers(), req.configuration, id_used, first_resolver );
|
||||
|
||||
for (int i = 0; i < proxies.size(); i++) result.masterProxies.push_back(proxies[i].interf);
|
||||
for (int i = 0; i < grv_proxies.size(); i++) result.grvProxies.push_back(grv_proxies[i].interf);
|
||||
for(int i = 0; i < resolvers.size(); i++)
|
||||
result.resolvers.push_back(resolvers[i].interf);
|
||||
for(int i = 0; i < proxies.size(); i++)
|
||||
result.proxies.push_back(proxies[i].interf);
|
||||
|
||||
if(req.maxOldLogRouters > 0) {
|
||||
if(tlogs.size() == 1) {
|
||||
|
@ -765,11 +793,22 @@ public:
|
|||
[](const WorkerDetails& w) { return w.interf; });
|
||||
}
|
||||
|
||||
if( !goodRecruitmentTime.isReady() &&
|
||||
( RoleFitness(SERVER_KNOBS->EXPECTED_TLOG_FITNESS, req.configuration.getDesiredLogs(), ProcessClass::TLog).betterCount(RoleFitness(tlogs, ProcessClass::TLog)) ||
|
||||
( region.satelliteTLogReplicationFactor > 0 && req.configuration.usableRegions > 1 && RoleFitness(SERVER_KNOBS->EXPECTED_TLOG_FITNESS, req.configuration.getDesiredSatelliteLogs(dcId), ProcessClass::TLog).betterCount(RoleFitness(satelliteLogs, ProcessClass::TLog)) ) ||
|
||||
RoleFitness(SERVER_KNOBS->EXPECTED_PROXY_FITNESS, req.configuration.getDesiredProxies(), ProcessClass::Proxy).betterCount(RoleFitness(proxies, ProcessClass::Proxy)) ||
|
||||
RoleFitness(SERVER_KNOBS->EXPECTED_RESOLVER_FITNESS, req.configuration.getDesiredResolvers(), ProcessClass::Resolver).betterCount(RoleFitness(resolvers, ProcessClass::Resolver)) ) ) {
|
||||
if (!goodRecruitmentTime.isReady() &&
|
||||
(RoleFitness(SERVER_KNOBS->EXPECTED_TLOG_FITNESS, req.configuration.getDesiredLogs(), ProcessClass::TLog)
|
||||
.betterCount(RoleFitness(tlogs, ProcessClass::TLog)) ||
|
||||
(region.satelliteTLogReplicationFactor > 0 && req.configuration.usableRegions > 1 &&
|
||||
RoleFitness(SERVER_KNOBS->EXPECTED_TLOG_FITNESS, req.configuration.getDesiredSatelliteLogs(dcId),
|
||||
ProcessClass::TLog)
|
||||
.betterCount(RoleFitness(satelliteLogs, ProcessClass::TLog))) ||
|
||||
RoleFitness(SERVER_KNOBS->EXPECTED_PROXY_FITNESS, req.configuration.getDesiredProxies(),
|
||||
ProcessClass::Proxy)
|
||||
.betterCount(RoleFitness(proxies, ProcessClass::Proxy)) ||
|
||||
RoleFitness(SERVER_KNOBS->EXPECTED_GRV_PROXY_FITNESS, req.configuration.getDesiredGrvProxies(),
|
||||
ProcessClass::GrvProxy)
|
||||
.betterCount(RoleFitness(grv_proxies, ProcessClass::GrvProxy)) ||
|
||||
RoleFitness(SERVER_KNOBS->EXPECTED_RESOLVER_FITNESS, req.configuration.getDesiredResolvers(),
|
||||
ProcessClass::Resolver)
|
||||
.betterCount(RoleFitness(resolvers, ProcessClass::Resolver)))) {
|
||||
return operation_failed();
|
||||
}
|
||||
|
||||
|
@ -848,15 +887,15 @@ public:
|
|||
if(tlogs.size() == 1) {
|
||||
result.oldLogRouters.push_back(tlogs[0].interf);
|
||||
} else {
|
||||
for(int i = 0; i < tlogs.size(); i++) {
|
||||
if(tlogs[i].interf.locality.processId() != clusterControllerProcessId) {
|
||||
for (int i = 0; i < tlogs.size(); i++) {
|
||||
if (tlogs[i].interf.locality.processId() != clusterControllerProcessId) {
|
||||
result.oldLogRouters.push_back(tlogs[i].interf);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if(req.recruitSeedServers) {
|
||||
if (req.recruitSeedServers) {
|
||||
auto primaryStorageServers = getWorkersForSeedServers( req.configuration, req.configuration.storagePolicy );
|
||||
for(int i = 0; i < primaryStorageServers.size(); i++)
|
||||
result.storageServers.push_back(primaryStorageServers[i].interf);
|
||||
|
@ -872,21 +911,37 @@ public:
|
|||
try {
|
||||
//SOMEDAY: recruitment in other DCs besides the clusterControllerDcID will not account for the processes used by the master and cluster controller properly.
|
||||
auto used = id_used;
|
||||
auto first_resolver = getWorkerForRoleInDatacenter( dcId, ProcessClass::Resolver, ProcessClass::ExcludeFit, req.configuration, used );
|
||||
auto first_proxy = getWorkerForRoleInDatacenter( dcId, ProcessClass::Proxy, ProcessClass::ExcludeFit, req.configuration, used );
|
||||
auto first_proxy = getWorkerForRoleInDatacenter(dcId, ProcessClass::Proxy, ProcessClass::ExcludeFit,
|
||||
req.configuration, used);
|
||||
auto first_grv_proxy = getWorkerForRoleInDatacenter(
|
||||
dcId, ProcessClass::GrvProxy, ProcessClass::ExcludeFit, req.configuration, used);
|
||||
auto first_resolver = getWorkerForRoleInDatacenter(
|
||||
dcId, ProcessClass::Resolver, ProcessClass::ExcludeFit, req.configuration, used);
|
||||
|
||||
auto proxies = getWorkersForRoleInDatacenter( dcId, ProcessClass::Proxy, req.configuration.getDesiredProxies(), req.configuration, used, first_proxy );
|
||||
auto proxies =
|
||||
getWorkersForRoleInDatacenter(dcId, ProcessClass::Proxy, req.configuration.getDesiredProxies(),
|
||||
req.configuration, used, first_proxy);
|
||||
auto grv_proxies = getWorkersForRoleInDatacenter(dcId, ProcessClass::GrvProxy,
|
||||
req.configuration.getDesiredGrvProxies(),
|
||||
req.configuration, used, first_grv_proxy);
|
||||
auto resolvers = getWorkersForRoleInDatacenter( dcId, ProcessClass::Resolver, req.configuration.getDesiredResolvers(), req.configuration, used, first_resolver );
|
||||
|
||||
RoleFitnessPair fitness( RoleFitness(proxies, ProcessClass::Proxy), RoleFitness(resolvers, ProcessClass::Resolver) );
|
||||
RoleFitnessPair fitness(RoleFitness(proxies, ProcessClass::Proxy),
|
||||
RoleFitness(grv_proxies, ProcessClass::GrvProxy),
|
||||
RoleFitness(resolvers, ProcessClass::Resolver));
|
||||
|
||||
if(dcId == clusterControllerDcId) {
|
||||
bestFitness = fitness;
|
||||
bestDC = dcId;
|
||||
for(int i = 0; i < resolvers.size(); i++)
|
||||
for (int i = 0; i < resolvers.size(); i++) {
|
||||
result.resolvers.push_back(resolvers[i].interf);
|
||||
for(int i = 0; i < proxies.size(); i++)
|
||||
result.proxies.push_back(proxies[i].interf);
|
||||
}
|
||||
for (int i = 0; i < proxies.size(); i++) {
|
||||
result.masterProxies.push_back(proxies[i].interf);
|
||||
}
|
||||
for (int i = 0; i < grv_proxies.size(); i++) {
|
||||
result.grvProxies.push_back(grv_proxies[i].interf);
|
||||
}
|
||||
|
||||
if (req.configuration.backupWorkerEnabled) {
|
||||
const int nBackup = std::max<int>(tlogs.size(), req.maxOldLogRouters);
|
||||
|
@ -903,7 +958,8 @@ public:
|
|||
bestFitness = fitness;
|
||||
numEquivalent = 1;
|
||||
bestDC = dcId;
|
||||
} else if( fitness == bestFitness && deterministicRandom()->random01() < 1.0/++numEquivalent ) {
|
||||
} else if (fitness == bestFitness &&
|
||||
deterministicRandom()->random01() < 1.0 / ++numEquivalent) {
|
||||
bestDC = dcId;
|
||||
}
|
||||
}
|
||||
|
@ -922,15 +978,30 @@ public:
|
|||
}
|
||||
//If this cluster controller dies, do not prioritize recruiting the next one in the same DC
|
||||
desiredDcIds.set(vector<Optional<Key>>());
|
||||
TraceEvent("FindWorkersForConfig").detail("Replication", req.configuration.tLogReplicationFactor)
|
||||
.detail("DesiredLogs", req.configuration.getDesiredLogs()).detail("ActualLogs", result.tLogs.size())
|
||||
.detail("DesiredProxies", req.configuration.getDesiredProxies()).detail("ActualProxies", result.proxies.size())
|
||||
.detail("DesiredResolvers", req.configuration.getDesiredResolvers()).detail("ActualResolvers", result.resolvers.size());
|
||||
TraceEvent("FindWorkersForConfig")
|
||||
.detail("Replication", req.configuration.tLogReplicationFactor)
|
||||
.detail("DesiredLogs", req.configuration.getDesiredLogs())
|
||||
.detail("ActualLogs", result.tLogs.size())
|
||||
.detail("DesiredProxies", req.configuration.getDesiredProxies())
|
||||
.detail("ActualProxies", result.masterProxies.size())
|
||||
.detail("DesiredGrvProxies", req.configuration.getDesiredGrvProxies())
|
||||
.detail("ActualGrvProxies", result.grvProxies.size())
|
||||
.detail("DesiredResolvers", req.configuration.getDesiredResolvers())
|
||||
.detail("ActualResolvers", result.resolvers.size());
|
||||
|
||||
if( !goodRecruitmentTime.isReady() &&
|
||||
( RoleFitness(SERVER_KNOBS->EXPECTED_TLOG_FITNESS, req.configuration.getDesiredLogs(), ProcessClass::TLog).betterCount(RoleFitness(tlogs, ProcessClass::TLog)) ||
|
||||
RoleFitness(SERVER_KNOBS->EXPECTED_PROXY_FITNESS, req.configuration.getDesiredProxies(), ProcessClass::Proxy).betterCount(bestFitness.proxy) ||
|
||||
RoleFitness(SERVER_KNOBS->EXPECTED_RESOLVER_FITNESS, req.configuration.getDesiredResolvers(), ProcessClass::Resolver).betterCount(bestFitness.resolver) ) ) {
|
||||
if (!goodRecruitmentTime.isReady() &&
|
||||
(RoleFitness(SERVER_KNOBS->EXPECTED_TLOG_FITNESS, req.configuration.getDesiredLogs(),
|
||||
ProcessClass::TLog)
|
||||
.betterCount(RoleFitness(tlogs, ProcessClass::TLog)) ||
|
||||
RoleFitness(SERVER_KNOBS->EXPECTED_PROXY_FITNESS, req.configuration.getDesiredProxies(),
|
||||
ProcessClass::Proxy)
|
||||
.betterCount(bestFitness.proxy) ||
|
||||
RoleFitness(SERVER_KNOBS->EXPECTED_GRV_PROXY_FITNESS, req.configuration.getDesiredGrvProxies(),
|
||||
ProcessClass::GrvProxy)
|
||||
.betterCount(bestFitness.grvProxy) ||
|
||||
RoleFitness(SERVER_KNOBS->EXPECTED_RESOLVER_FITNESS, req.configuration.getDesiredResolvers(),
|
||||
ProcessClass::Resolver)
|
||||
.betterCount(bestFitness.resolver))) {
|
||||
throw operation_failed();
|
||||
}
|
||||
|
||||
|
@ -958,6 +1029,8 @@ public:
|
|||
|
||||
getWorkerForRoleInDatacenter( regions[0].dcId, ProcessClass::Resolver, ProcessClass::ExcludeFit, db.config, id_used, true );
|
||||
getWorkerForRoleInDatacenter( regions[0].dcId, ProcessClass::Proxy, ProcessClass::ExcludeFit, db.config, id_used, true );
|
||||
getWorkerForRoleInDatacenter(regions[0].dcId, ProcessClass::GrvProxy, ProcessClass::ExcludeFit, db.config,
|
||||
id_used, true);
|
||||
|
||||
vector<Optional<Key>> dcPriority;
|
||||
dcPriority.push_back(regions[0].dcId);
|
||||
|
@ -1026,15 +1099,14 @@ public:
|
|||
|
||||
if(logSet.isLocal && logSet.locality == tagLocalitySatellite) {
|
||||
satellite_tlogs.push_back(tlogWorker->second.details);
|
||||
}
|
||||
else if(logSet.isLocal) {
|
||||
} else if (logSet.isLocal) {
|
||||
tlogs.push_back(tlogWorker->second.details);
|
||||
} else {
|
||||
remote_tlogs.push_back(tlogWorker->second.details);
|
||||
}
|
||||
}
|
||||
|
||||
for( auto& it : logSet.logRouters ) {
|
||||
for (auto& it : logSet.logRouters) {
|
||||
auto tlogWorker = id_worker.find(it.interf().filteredLocality.processId());
|
||||
if ( tlogWorker == id_worker.end() )
|
||||
return false;
|
||||
|
@ -1059,13 +1131,24 @@ public:
|
|||
|
||||
// Get proxy classes
|
||||
std::vector<WorkerDetails> proxyClasses;
|
||||
for(auto& it : dbi.client.proxies ) {
|
||||
auto proxyWorker = id_worker.find(it.processId);
|
||||
if ( proxyWorker == id_worker.end() )
|
||||
for(auto& it : dbi.client.masterProxies) {
|
||||
auto masterProxyWorker = id_worker.find(it.processId);
|
||||
if ( masterProxyWorker == id_worker.end() )
|
||||
return false;
|
||||
if ( proxyWorker->second.priorityInfo.isExcluded )
|
||||
if ( masterProxyWorker->second.priorityInfo.isExcluded )
|
||||
return true;
|
||||
proxyClasses.push_back(proxyWorker->second.details);
|
||||
proxyClasses.push_back(masterProxyWorker->second.details);
|
||||
}
|
||||
|
||||
// Get grv proxy classes
|
||||
std::vector<WorkerDetails> grvProxyClasses;
|
||||
for(auto& it : dbi.client.grvProxies ) {
|
||||
auto grvProxyWorker = id_worker.find(it.processId);
|
||||
if ( grvProxyWorker == id_worker.end() )
|
||||
return false;
|
||||
if ( grvProxyWorker->second.priorityInfo.isExcluded )
|
||||
return true;
|
||||
grvProxyClasses.push_back(grvProxyWorker->second.details);
|
||||
}
|
||||
|
||||
// Get resolver classes
|
||||
|
@ -1146,7 +1229,7 @@ public:
|
|||
|
||||
int32_t oldSatelliteRegionFit = std::numeric_limits<int32_t>::max();
|
||||
for(auto& it : satellite_tlogs) {
|
||||
if(satellite_priority.count(it.interf.locality.dcId())) {
|
||||
if (satellite_priority.count(it.interf.locality.dcId())) {
|
||||
oldSatelliteRegionFit = std::min(oldSatelliteRegionFit, satellite_priority[it.interf.locality.dcId()]);
|
||||
} else {
|
||||
oldSatelliteRegionFit = -1;
|
||||
|
@ -1162,13 +1245,11 @@ public:
|
|||
}
|
||||
}
|
||||
|
||||
if(oldSatelliteFallback && !newSatelliteFallback)
|
||||
return true;
|
||||
if (oldSatelliteFallback && !newSatelliteFallback) return true;
|
||||
if(!oldSatelliteFallback && newSatelliteFallback)
|
||||
return false;
|
||||
|
||||
if(oldSatelliteRegionFit < newSatelliteRegionFit)
|
||||
return true;
|
||||
if (oldSatelliteRegionFit < newSatelliteRegionFit) return true;
|
||||
if(oldSatelliteRegionFit > newSatelliteRegionFit)
|
||||
return false;
|
||||
|
||||
|
@ -1188,7 +1269,12 @@ public:
|
|||
int oldRouterCount = oldTLogFit.count * std::max<int>(1, db.config.desiredLogRouterCount / std::max(1,oldTLogFit.count));
|
||||
int newRouterCount = newTLogFit.count * std::max<int>(1, db.config.desiredLogRouterCount / std::max(1,newTLogFit.count));
|
||||
RoleFitness oldLogRoutersFit(log_routers, ProcessClass::LogRouter);
|
||||
RoleFitness newLogRoutersFit((db.config.usableRegions > 1 && dbi.recoveryState == RecoveryState::FULLY_RECOVERED) ? getWorkersForRoleInDatacenter( *remoteDC.begin(), ProcessClass::LogRouter, newRouterCount, db.config, id_used, Optional<WorkerFitnessInfo>(), true ) : log_routers, ProcessClass::LogRouter);
|
||||
RoleFitness newLogRoutersFit(
|
||||
(db.config.usableRegions > 1 && dbi.recoveryState == RecoveryState::FULLY_RECOVERED)
|
||||
? getWorkersForRoleInDatacenter(*remoteDC.begin(), ProcessClass::LogRouter, newRouterCount, db.config,
|
||||
id_used, Optional<WorkerFitnessInfo>(), true)
|
||||
: log_routers,
|
||||
ProcessClass::LogRouter);
|
||||
|
||||
if(oldLogRoutersFit.count < oldRouterCount) {
|
||||
oldLogRoutersFit.worstFit = ProcessClass::NeverAssign;
|
||||
|
@ -1197,17 +1283,31 @@ public:
|
|||
newLogRoutersFit.worstFit = ProcessClass::NeverAssign;
|
||||
}
|
||||
if(oldLogRoutersFit < newLogRoutersFit) return false;
|
||||
// Check proxy/resolver fitness
|
||||
RoleFitnessPair oldInFit(RoleFitness(proxyClasses, ProcessClass::Proxy), RoleFitness(resolverClasses, ProcessClass::Resolver));
|
||||
|
||||
auto first_resolver = getWorkerForRoleInDatacenter( clusterControllerDcId, ProcessClass::Resolver, ProcessClass::ExcludeFit, db.config, id_used, true );
|
||||
auto first_proxy = getWorkerForRoleInDatacenter( clusterControllerDcId, ProcessClass::Proxy, ProcessClass::ExcludeFit, db.config, id_used, true );
|
||||
// Check proxy/grvProxy/resolver fitness
|
||||
RoleFitnessPair oldInFit(RoleFitness(proxyClasses, ProcessClass::Proxy),
|
||||
RoleFitness(grvProxyClasses, ProcessClass::GrvProxy),
|
||||
RoleFitness(resolverClasses, ProcessClass::Resolver));
|
||||
|
||||
auto proxies = getWorkersForRoleInDatacenter( clusterControllerDcId, ProcessClass::Proxy, db.config.getDesiredProxies(), db.config, id_used, first_proxy, true );
|
||||
auto first_proxy = getWorkerForRoleInDatacenter(clusterControllerDcId, ProcessClass::Proxy,
|
||||
ProcessClass::ExcludeFit, db.config, id_used, true);
|
||||
auto first_grv_proxy = getWorkerForRoleInDatacenter(clusterControllerDcId, ProcessClass::GrvProxy,
|
||||
ProcessClass::ExcludeFit, db.config, id_used, true);
|
||||
auto first_resolver = getWorkerForRoleInDatacenter(clusterControllerDcId, ProcessClass::Resolver,
|
||||
ProcessClass::ExcludeFit, db.config, id_used, true);
|
||||
auto proxies =
|
||||
getWorkersForRoleInDatacenter(clusterControllerDcId, ProcessClass::Proxy, db.config.getDesiredProxies(),
|
||||
db.config, id_used, first_proxy, true);
|
||||
auto grv_proxies =
|
||||
getWorkersForRoleInDatacenter(clusterControllerDcId, ProcessClass::GrvProxy,
|
||||
db.config.getDesiredGrvProxies(), db.config, id_used, first_grv_proxy, true);
|
||||
auto resolvers = getWorkersForRoleInDatacenter( clusterControllerDcId, ProcessClass::Resolver, db.config.getDesiredResolvers(), db.config, id_used, first_resolver, true );
|
||||
|
||||
RoleFitnessPair newInFit(RoleFitness(proxies, ProcessClass::Proxy), RoleFitness(resolvers, ProcessClass::Resolver));
|
||||
if(oldInFit.proxy.betterFitness(newInFit.proxy) || oldInFit.resolver.betterFitness(newInFit.resolver)) {
|
||||
RoleFitnessPair newInFit(RoleFitness(proxies, ProcessClass::Proxy),
|
||||
RoleFitness(grv_proxies, ProcessClass::GrvProxy),
|
||||
RoleFitness(resolvers, ProcessClass::Resolver));
|
||||
if (oldInFit.proxy.betterFitness(newInFit.proxy) || oldInFit.grvProxy.betterFitness(newInFit.grvProxy) ||
|
||||
oldInFit.resolver.betterFitness(newInFit.resolver)) {
|
||||
return false;
|
||||
}
|
||||
|
||||
|
@ -1228,6 +1328,8 @@ public:
|
|||
.detail("NewTLogFit", newTLogFit.toString())
|
||||
.detail("OldProxyFit", oldInFit.proxy.toString())
|
||||
.detail("NewProxyFit", newInFit.proxy.toString())
|
||||
.detail("OldGrvProxyFit", oldInFit.grvProxy.toString())
|
||||
.detail("NewGrvProxyFit", newInFit.grvProxy.toString())
|
||||
.detail("OldResolverFit", oldInFit.resolver.toString())
|
||||
.detail("NewResolverFit", newInFit.resolver.toString())
|
||||
.detail("OldSatelliteFit", oldSatelliteTLogFit.toString())
|
||||
|
@ -1256,7 +1358,10 @@ public:
|
|||
if (tlog.present() && tlog.interf().filteredLocality.processId() == processId) return true;
|
||||
}
|
||||
}
|
||||
for (const MasterProxyInterface& interf : dbInfo.client.proxies) {
|
||||
for (const MasterProxyInterface& interf : dbInfo.client.masterProxies) {
|
||||
if (interf.processId == processId) return true;
|
||||
}
|
||||
for (const GrvProxyInterface& interf : dbInfo.client.grvProxies) {
|
||||
if (interf.processId == processId) return true;
|
||||
}
|
||||
for (const ResolverInterface& interf: dbInfo.resolvers) {
|
||||
|
@ -1288,7 +1393,11 @@ public:
|
|||
}
|
||||
}
|
||||
}
|
||||
for (const MasterProxyInterface& interf : dbInfo.client.proxies) {
|
||||
for (const MasterProxyInterface& interf : dbInfo.client.masterProxies) {
|
||||
ASSERT(interf.processId.present());
|
||||
idUsed[interf.processId]++;
|
||||
}
|
||||
for (const GrvProxyInterface& interf : dbInfo.client.grvProxies) {
|
||||
ASSERT(interf.processId.present());
|
||||
idUsed[interf.processId]++;
|
||||
}
|
||||
|
@ -1858,7 +1967,8 @@ void clusterRegisterMaster( ClusterControllerData* self, RegisterMasterRequest c
|
|||
.detail("Resolvers", req.resolvers.size())
|
||||
.detail("RecoveryState", (int)req.recoveryState)
|
||||
.detail("RegistrationCount", req.registrationCount)
|
||||
.detail("Proxies", req.proxies.size())
|
||||
.detail("MasterProxies", req.masterProxies.size())
|
||||
.detail("GrvProxies", req.grvProxies.size())
|
||||
.detail("RecoveryCount", req.recoveryCount)
|
||||
.detail("Stalled", req.recoveryStalled)
|
||||
.detail("OldestBackupEpoch", req.logSystemConfig.oldestBackupEpoch);
|
||||
|
@ -1912,11 +2022,12 @@ void clusterRegisterMaster( ClusterControllerData* self, RegisterMasterRequest c
|
|||
}
|
||||
|
||||
// Construct the client information
|
||||
if (db->clientInfo->get().proxies != req.proxies) {
|
||||
if (db->clientInfo->get().masterProxies != req.masterProxies || db->clientInfo->get().grvProxies != req.grvProxies) {
|
||||
isChanged = true;
|
||||
ClientDBInfo clientInfo;
|
||||
clientInfo.id = deterministicRandom()->randomUniqueID();
|
||||
clientInfo.proxies = req.proxies;
|
||||
clientInfo.masterProxies = req.masterProxies;
|
||||
clientInfo.grvProxies = req.grvProxies;
|
||||
clientInfo.clientTxnInfoSampleRate = db->clientInfo->get().clientTxnInfoSampleRate;
|
||||
clientInfo.clientTxnInfoSizeLimit = db->clientInfo->get().clientTxnInfoSizeLimit;
|
||||
db->clientInfo->set( clientInfo );
|
||||
|
@ -2773,10 +2884,7 @@ ACTOR Future<Void> dbInfoUpdater( ClusterControllerData* self ) {
|
|||
TraceEvent("DBInfoStartBroadcast", self->id);
|
||||
choose {
|
||||
when(std::vector<Endpoint> notUpdated = wait( broadcastDBInfoRequest(req, SERVER_KNOBS->DBINFO_SEND_AMOUNT, Optional<Endpoint>(), false) )) {
|
||||
TraceEvent("DBInfoFinishBroadcast", self->id);
|
||||
for(auto &it : notUpdated) {
|
||||
TraceEvent("DBInfoNotUpdated", self->id).detail("Addr", it.getPrimaryAddress());
|
||||
}
|
||||
TraceEvent("DBInfoFinishBroadcast", self->id).detail("NotUpdated", notUpdated.size());
|
||||
if(notUpdated.size()) {
|
||||
self->updateDBInfoEndpoints.insert(notUpdated.begin(), notUpdated.end());
|
||||
self->updateDBInfo.trigger();
|
||||
|
|
|
@ -512,10 +512,10 @@ ACTOR Future<Reference<InitialDataDistribution>> getInitialDataDistribution( Dat
|
|||
beginKey = keyServers.end()[-1].key;
|
||||
break;
|
||||
} catch (Error& e) {
|
||||
wait( tr.onError(e) );
|
||||
TraceEvent("GetInitialTeamsKeyServersRetry", distributorId).error(e);
|
||||
|
||||
wait(tr.onError(e));
|
||||
ASSERT(!succeeded); //We shouldn't be retrying if we have already started modifying result in this loop
|
||||
TraceEvent("GetInitialTeamsKeyServersRetry", distributorId);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -884,8 +884,8 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
|
|||
// If unhealthy team is majority, we may not find an ok dest in this while loop
|
||||
Reference<IDataDistributionTeam> dest = deterministicRandom()->randomChoice(self->teams);
|
||||
|
||||
bool ok = dest->isHealthy() &&
|
||||
(!req.preferLowerUtilization || dest->hasHealthyAvailableSpace(self->medianAvailableSpace));
|
||||
bool ok = dest->isHealthy() && (!req.preferLowerUtilization ||
|
||||
dest->hasHealthyAvailableSpace(self->medianAvailableSpace));
|
||||
|
||||
for(int i=0; ok && i<randomTeams.size(); i++) {
|
||||
if (randomTeams[i]->getServerIDs() == dest->getServerIDs()) {
|
||||
|
@ -1458,9 +1458,11 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
|
|||
if (!shouldPrint) return;
|
||||
// Record all team collections IDs
|
||||
for (int i = 0; i < teamCollections.size(); ++i) {
|
||||
TraceEvent("TraceAllInfo", distributorId)
|
||||
.detail("TeamCollectionIndex", i)
|
||||
.detail("Primary", teamCollections[i]->primary);
|
||||
if (teamCollections[i] != nullptr) {
|
||||
TraceEvent("TraceAllInfo", distributorId)
|
||||
.detail("TeamCollectionIndex", i)
|
||||
.detail("Primary", teamCollections[i]->primary);
|
||||
}
|
||||
}
|
||||
|
||||
TraceEvent("TraceAllInfo", distributorId).detail("Primary", primary);
|
||||
|
@ -2075,7 +2077,7 @@ struct DDTeamCollection : ReferenceCounted<DDTeamCollection> {
|
|||
for (auto& server : serverTeam) {
|
||||
score += server_info[server]->teams.size();
|
||||
}
|
||||
TraceEvent("BuildServerTeams")
|
||||
TraceEvent(SevDebug, "BuildServerTeams")
|
||||
.detail("Score", score)
|
||||
.detail("BestScore", bestScore)
|
||||
.detail("TeamSize", serverTeam.size())
|
||||
|
@ -2779,7 +2781,7 @@ ACTOR Future<Void> machineTeamRemover(DDTeamCollection* self) {
|
|||
// To avoid removing machine teams too fast, which is unlikely happen though
|
||||
wait( delay(SERVER_KNOBS->TR_REMOVE_MACHINE_TEAM_DELAY, TaskPriority::DataDistribution) );
|
||||
|
||||
wait(waitUntilHealthy(self));
|
||||
wait(waitUntilHealthy(self, SERVER_KNOBS->TR_REMOVE_SERVER_TEAM_EXTRA_DELAY));
|
||||
// Wait for the badTeamRemover() to avoid the potential race between adding the bad team (add the team tracker)
|
||||
// and remove bad team (cancel the team tracker).
|
||||
wait(self->badTeamRemover);
|
||||
|
@ -4484,12 +4486,12 @@ ACTOR Future<Void> monitorBatchLimitedTime(Reference<AsyncVar<ServerDBInfo>> db,
|
|||
loop {
|
||||
wait( delay(SERVER_KNOBS->METRIC_UPDATE_RATE) );
|
||||
|
||||
state Reference<ProxyInfo> proxies(new ProxyInfo(db->get().client.proxies));
|
||||
state Reference<GrvProxyInfo> grvProxies(new GrvProxyInfo(db->get().client.grvProxies));
|
||||
|
||||
choose {
|
||||
when (wait(db->onChange())) {}
|
||||
when (GetHealthMetricsReply reply = wait(proxies->size() ?
|
||||
basicLoadBalance(proxies, &MasterProxyInterface::getHealthMetrics, GetHealthMetricsRequest(false))
|
||||
when (GetHealthMetricsReply reply = wait(grvProxies->size() ?
|
||||
basicLoadBalance(grvProxies, &GrvProxyInterface::getHealthMetrics, GetHealthMetricsRequest(false))
|
||||
: Never())) {
|
||||
if (reply.healthMetrics.batchLimited) {
|
||||
*lastLimited = now();
|
||||
|
@ -4975,6 +4977,37 @@ ACTOR Future<Void> cacheServerWatcher(Database* db) {
|
|||
}
|
||||
}
|
||||
|
||||
static int64_t getMedianShardSize(VectorRef<DDMetricsRef> metricVec) {
|
||||
std::nth_element(metricVec.begin(), metricVec.begin() + metricVec.size() / 2, metricVec.end(),
|
||||
[](const DDMetricsRef& d1, const DDMetricsRef& d2) { return d1.shardBytes < d2.shardBytes; });
|
||||
return metricVec[metricVec.size() / 2].shardBytes;
|
||||
}
|
||||
|
||||
ACTOR Future<Void> ddGetMetrics(GetDataDistributorMetricsRequest req,
|
||||
PromiseStream<GetMetricsListRequest> getShardMetricsList) {
|
||||
ErrorOr<Standalone<VectorRef<DDMetricsRef>>> result = wait(
|
||||
errorOr(brokenPromiseToNever(getShardMetricsList.getReply(GetMetricsListRequest(req.keys, req.shardLimit)))));
|
||||
|
||||
if (result.isError()) {
|
||||
req.reply.sendError(result.getError());
|
||||
} else {
|
||||
GetDataDistributorMetricsReply rep;
|
||||
if (!req.midOnly) {
|
||||
rep.storageMetricsList = result.get();
|
||||
} else {
|
||||
auto& metricVec = result.get();
|
||||
if (metricVec.empty())
|
||||
rep.midShardSize = 0;
|
||||
else {
|
||||
rep.midShardSize = getMedianShardSize(metricVec.contents());
|
||||
}
|
||||
}
|
||||
req.reply.send(rep);
|
||||
}
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR Future<Void> dataDistributor(DataDistributorInterface di, Reference<AsyncVar<struct ServerDBInfo>> db ) {
|
||||
state Reference<DataDistributorData> self( new DataDistributorData(db, di.id()) );
|
||||
state Future<Void> collection = actorCollection( self->addActor.getFuture() );
|
||||
|
@ -5000,16 +5033,8 @@ ACTOR Future<Void> dataDistributor(DataDistributorInterface di, Reference<AsyncV
|
|||
TraceEvent("DataDistributorHalted", di.id()).detail("ReqID", req.requesterID);
|
||||
break;
|
||||
}
|
||||
when ( state GetDataDistributorMetricsRequest req = waitNext(di.dataDistributorMetrics.getFuture()) ) {
|
||||
ErrorOr<Standalone<VectorRef<DDMetricsRef>>> result = wait(errorOr(brokenPromiseToNever(
|
||||
getShardMetricsList.getReply(GetMetricsListRequest(req.keys, req.shardLimit)))));
|
||||
if ( result.isError() ) {
|
||||
req.reply.sendError(result.getError());
|
||||
} else {
|
||||
GetDataDistributorMetricsReply rep;
|
||||
rep.storageMetricsList = result.get();
|
||||
req.reply.send(rep);
|
||||
}
|
||||
when(GetDataDistributorMetricsRequest req = waitNext(di.dataDistributorMetrics.getFuture())) {
|
||||
actors.add(ddGetMetrics(req, getShardMetricsList));
|
||||
}
|
||||
when(DistributorSnapRequest snapReq = waitNext(di.distributorSnapReq.getFuture())) {
|
||||
actors.add(ddSnapCreate(snapReq, db));
|
||||
|
|
|
@ -70,12 +70,13 @@ struct HaltDataDistributorRequest {
|
|||
struct GetDataDistributorMetricsReply {
|
||||
constexpr static FileIdentifier file_identifier = 1284337;
|
||||
Standalone<VectorRef<DDMetricsRef>> storageMetricsList;
|
||||
Optional<int64_t> midShardSize;
|
||||
|
||||
GetDataDistributorMetricsReply() {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar,storageMetricsList);
|
||||
serializer(ar, storageMetricsList, midShardSize);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -84,13 +85,15 @@ struct GetDataDistributorMetricsRequest {
|
|||
KeyRange keys;
|
||||
int shardLimit;
|
||||
ReplyPromise<struct GetDataDistributorMetricsReply> reply;
|
||||
bool midOnly = false;
|
||||
|
||||
GetDataDistributorMetricsRequest() {}
|
||||
explicit GetDataDistributorMetricsRequest(KeyRange const& keys, const int shardLimit) : keys(keys), shardLimit(shardLimit) {}
|
||||
explicit GetDataDistributorMetricsRequest(KeyRange const& keys, const int shardLimit, bool midOnly = false)
|
||||
: keys(keys), shardLimit(shardLimit), midOnly(midOnly) {}
|
||||
|
||||
template<class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, keys, shardLimit, reply);
|
||||
serializer(ar, keys, shardLimit, reply, midOnly);
|
||||
}
|
||||
};
|
||||
|
||||
|
|
|
@ -0,0 +1,740 @@
|
|||
/*
|
||||
* GrvProxyServer.actor.cpp
|
||||
*
|
||||
* This source file is part of the FoundationDB open source project
|
||||
*
|
||||
* Copyright 2013-2020 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.
|
||||
*/
|
||||
|
||||
#include "fdbclient/Notified.h"
|
||||
#include "fdbserver/LogSystem.h"
|
||||
#include "fdbserver/LogSystemDiskQueueAdapter.h"
|
||||
#include "fdbclient/MasterProxyInterface.h"
|
||||
#include "fdbclient/GrvProxyInterface.h"
|
||||
#include "fdbserver/WaitFailure.h"
|
||||
#include "fdbserver/WorkerInterface.actor.h"
|
||||
#include "flow/flow.h"
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
||||
struct GrvProxyStats {
|
||||
CounterCollection cc;
|
||||
Counter txnRequestIn, txnRequestOut, txnRequestErrors;
|
||||
Counter txnStartIn, txnStartOut, txnStartBatch;
|
||||
Counter txnSystemPriorityStartIn, txnSystemPriorityStartOut;
|
||||
Counter txnBatchPriorityStartIn, txnBatchPriorityStartOut;
|
||||
Counter txnDefaultPriorityStartIn, txnDefaultPriorityStartOut;
|
||||
Counter txnThrottled;
|
||||
|
||||
LatencyBands grvLatencyBands;
|
||||
LatencySample grvLatencySample;
|
||||
|
||||
Future<Void> logger;
|
||||
|
||||
explicit GrvProxyStats(UID id)
|
||||
: cc("GrvProxyStats", id.toString()), txnRequestIn("TxnRequestIn", cc), txnRequestOut("TxnRequestOut", cc),
|
||||
txnRequestErrors("TxnRequestErrors", cc), txnStartIn("TxnStartIn", cc), txnStartOut("TxnStartOut", cc),
|
||||
txnStartBatch("TxnStartBatch", cc), txnSystemPriorityStartIn("TxnSystemPriorityStartIn", cc),
|
||||
txnSystemPriorityStartOut("TxnSystemPriorityStartOut", cc),
|
||||
txnBatchPriorityStartIn("TxnBatchPriorityStartIn", cc),
|
||||
txnBatchPriorityStartOut("TxnBatchPriorityStartOut", cc),
|
||||
txnDefaultPriorityStartIn("TxnDefaultPriorityStartIn", cc),
|
||||
txnDefaultPriorityStartOut("TxnDefaultPriorityStartOut", cc), txnThrottled("TxnThrottled", cc),
|
||||
grvLatencySample("GRVLatencyMetrics", id, SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
grvLatencyBands("GRVLatencyMetrics", id, SERVER_KNOBS->STORAGE_LOGGING_DELAY) {
|
||||
logger = traceCounters("GrvProxyMetrics", id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, &cc, "GrvProxyMetrics");
|
||||
}
|
||||
};
|
||||
|
||||
struct GrvTransactionRateInfo {
|
||||
double rate;
|
||||
double limit;
|
||||
double budget;
|
||||
|
||||
bool disabled;
|
||||
|
||||
Smoother smoothRate;
|
||||
Smoother smoothReleased;
|
||||
|
||||
GrvTransactionRateInfo(double rate) : rate(rate), limit(0), budget(0), disabled(true), smoothRate(SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW),
|
||||
smoothReleased(SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW) {}
|
||||
|
||||
void reset() {
|
||||
// Determine the number of transactions that this proxy is allowed to release
|
||||
// Roughly speaking, this is done by computing the number of transactions over some historical window that we could
|
||||
// have started but didn't, and making that our limit. More precisely, we track a smoothed rate limit and release rate,
|
||||
// the difference of which is the rate of additional transactions that we could have released based on that window.
|
||||
// Then we multiply by the window size to get a number of transactions.
|
||||
//
|
||||
// Limit can be negative in the event that we are releasing more transactions than we are allowed (due to the use of
|
||||
// our budget or because of higher priority transactions).
|
||||
double releaseRate = smoothRate.smoothTotal() - smoothReleased.smoothRate();
|
||||
limit = SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW * releaseRate;
|
||||
}
|
||||
|
||||
bool canStart(int64_t numAlreadyStarted, int64_t count) {
|
||||
return numAlreadyStarted + count <= std::min(limit + budget, SERVER_KNOBS->START_TRANSACTION_MAX_TRANSACTIONS_TO_START);
|
||||
}
|
||||
|
||||
void updateBudget(int64_t numStartedAtPriority, bool queueEmptyAtPriority, double elapsed) {
|
||||
// Update the budget to accumulate any extra capacity available or remove any excess that was used.
|
||||
// The actual delta is the portion of the limit we didn't use multiplied by the fraction of the window that elapsed.
|
||||
//
|
||||
// We may have exceeded our limit due to the budget or because of higher priority transactions, in which case this
|
||||
// delta will be negative. The delta can also be negative in the event that our limit was negative, which can happen
|
||||
// if we had already started more transactions in our window than our rate would have allowed.
|
||||
//
|
||||
// This budget has the property that when the budget is required to start transactions (because batches are big),
|
||||
// the sum limit+budget will increase linearly from 0 to the batch size over time and decrease by the batch size
|
||||
// upon starting a batch. In other words, this works equivalently to a model where we linearly accumulate budget over
|
||||
// time in the case that our batches are too big to take advantage of the window based limits.
|
||||
budget = std::max(0.0, budget + elapsed * (limit - numStartedAtPriority) / SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW);
|
||||
|
||||
// If we are emptying out the queue of requests, then we don't need to carry much budget forward
|
||||
// If we did keep accumulating budget, then our responsiveness to changes in workflow could be compromised
|
||||
if(queueEmptyAtPriority) {
|
||||
budget = std::min(budget, SERVER_KNOBS->START_TRANSACTION_MAX_EMPTY_QUEUE_BUDGET);
|
||||
}
|
||||
|
||||
smoothReleased.addDelta(numStartedAtPriority);
|
||||
}
|
||||
|
||||
void disable() {
|
||||
disabled = true;
|
||||
rate = 0;
|
||||
smoothRate.reset(0);
|
||||
}
|
||||
|
||||
void setRate(double rate) {
|
||||
ASSERT(rate >= 0 && rate != std::numeric_limits<double>::infinity() && !std::isnan(rate));
|
||||
|
||||
this->rate = rate;
|
||||
if(disabled) {
|
||||
smoothRate.reset(rate);
|
||||
disabled = false;
|
||||
}
|
||||
else {
|
||||
smoothRate.setTotal(rate);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
struct GrvProxyData {
|
||||
GrvProxyInterface proxy;
|
||||
UID dbgid;
|
||||
|
||||
GrvProxyStats stats;
|
||||
MasterInterface master;
|
||||
RequestStream<GetReadVersionRequest> getConsistentReadVersion;
|
||||
Reference<ILogSystem> logSystem;
|
||||
|
||||
Database cx;
|
||||
Reference<AsyncVar<ServerDBInfo>> db;
|
||||
|
||||
Optional<LatencyBandConfig> latencyBandConfig;
|
||||
double lastStartCommit;
|
||||
double lastCommitLatency;
|
||||
int updateCommitRequests;
|
||||
NotifiedDouble lastCommitTime;
|
||||
|
||||
Version minKnownCommittedVersion; // we should ask master for this version.
|
||||
|
||||
void updateLatencyBandConfig(Optional<LatencyBandConfig> newLatencyBandConfig) {
|
||||
if(newLatencyBandConfig.present() != latencyBandConfig.present()
|
||||
|| (newLatencyBandConfig.present() && newLatencyBandConfig.get().grvConfig != latencyBandConfig.get().grvConfig))
|
||||
{
|
||||
TraceEvent("LatencyBandGrvUpdatingConfig").detail("Present", newLatencyBandConfig.present());
|
||||
stats.grvLatencyBands.clearBands();
|
||||
if(newLatencyBandConfig.present()) {
|
||||
for(auto band : newLatencyBandConfig.get().grvConfig.bands) {
|
||||
stats.grvLatencyBands.addThreshold(band);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
latencyBandConfig = newLatencyBandConfig;
|
||||
}
|
||||
|
||||
GrvProxyData(UID dbgid, MasterInterface master, RequestStream<GetReadVersionRequest> getConsistentReadVersion, Reference<AsyncVar<ServerDBInfo>> db)
|
||||
: dbgid(dbgid), stats(dbgid), master(master), getConsistentReadVersion(getConsistentReadVersion), cx(openDBOnServer(db, TaskPriority::DefaultEndpoint, true, true)), db(db),
|
||||
lastStartCommit(0), lastCommitLatency(SERVER_KNOBS->REQUIRED_MIN_RECOVERY_DURATION), updateCommitRequests(0), lastCommitTime(0), minKnownCommittedVersion(invalidVersion)
|
||||
{}
|
||||
};
|
||||
|
||||
ACTOR Future<Void> healthMetricsRequestServer(GrvProxyInterface grvProxy, GetHealthMetricsReply* healthMetricsReply, GetHealthMetricsReply* detailedHealthMetricsReply)
|
||||
{
|
||||
loop {
|
||||
choose {
|
||||
when(GetHealthMetricsRequest req =
|
||||
waitNext(grvProxy.getHealthMetrics.getFuture()))
|
||||
{
|
||||
if (req.detailed)
|
||||
req.reply.send(*detailedHealthMetricsReply);
|
||||
else
|
||||
req.reply.send(*healthMetricsReply);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
ACTOR Future<Void> getRate(UID myID, Reference<AsyncVar<ServerDBInfo>> db, int64_t* inTransactionCount,
|
||||
int64_t* inBatchTransactionCount, GrvTransactionRateInfo* transactionRateInfo,
|
||||
GrvTransactionRateInfo* batchTransactionRateInfo, GetHealthMetricsReply* healthMetricsReply,
|
||||
GetHealthMetricsReply* detailedHealthMetricsReply,
|
||||
TransactionTagMap<uint64_t>* transactionTagCounter,
|
||||
PrioritizedTransactionTagMap<ClientTagThrottleLimits>* throttledTags) {
|
||||
state Future<Void> nextRequestTimer = Never();
|
||||
state Future<Void> leaseTimeout = Never();
|
||||
state Future<GetRateInfoReply> reply = Never();
|
||||
state double lastDetailedReply = 0.0; // request detailed metrics immediately
|
||||
state bool expectingDetailedReply = false;
|
||||
state int64_t lastTC = 0;
|
||||
|
||||
if (db->get().ratekeeper.present()) nextRequestTimer = Void();
|
||||
loop choose {
|
||||
when ( wait( db->onChange() ) ) {
|
||||
if ( db->get().ratekeeper.present() ) {
|
||||
TraceEvent("ProxyRatekeeperChanged", myID)
|
||||
.detail("RKID", db->get().ratekeeper.get().id());
|
||||
nextRequestTimer = Void(); // trigger GetRate request
|
||||
} else {
|
||||
TraceEvent("ProxyRatekeeperDied", myID);
|
||||
nextRequestTimer = Never();
|
||||
reply = Never();
|
||||
}
|
||||
}
|
||||
when ( wait( nextRequestTimer ) ) {
|
||||
nextRequestTimer = Never();
|
||||
bool detailed = now() - lastDetailedReply > SERVER_KNOBS->DETAILED_METRIC_UPDATE_RATE;
|
||||
|
||||
TransactionTagMap<uint64_t> tagCounts;
|
||||
for(auto itr : *throttledTags) {
|
||||
for(auto priorityThrottles : itr.second) {
|
||||
tagCounts[priorityThrottles.first] = (*transactionTagCounter)[priorityThrottles.first];
|
||||
}
|
||||
}
|
||||
reply = brokenPromiseToNever(db->get().ratekeeper.get().getRateInfo.getReply(GetRateInfoRequest(
|
||||
myID, *inTransactionCount, *inBatchTransactionCount, *transactionTagCounter, detailed)));
|
||||
transactionTagCounter->clear();
|
||||
expectingDetailedReply = detailed;
|
||||
}
|
||||
when ( GetRateInfoReply rep = wait(reply) ) {
|
||||
reply = Never();
|
||||
|
||||
transactionRateInfo->setRate(rep.transactionRate);
|
||||
batchTransactionRateInfo->setRate(rep.batchTransactionRate);
|
||||
//TraceEvent("GrvProxyRate", myID).detail("Rate", rep.transactionRate).detail("BatchRate", rep.batchTransactionRate).detail("Lease", rep.leaseDuration).detail("ReleasedTransactions", *inTransactionCount - lastTC);
|
||||
lastTC = *inTransactionCount;
|
||||
leaseTimeout = delay(rep.leaseDuration);
|
||||
nextRequestTimer = delayJittered(rep.leaseDuration / 2);
|
||||
healthMetricsReply->update(rep.healthMetrics, expectingDetailedReply, true);
|
||||
if (expectingDetailedReply) {
|
||||
detailedHealthMetricsReply->update(rep.healthMetrics, true, true);
|
||||
lastDetailedReply = now();
|
||||
}
|
||||
|
||||
// Replace our throttles with what was sent by ratekeeper. Because we do this,
|
||||
// we are not required to expire tags out of the map
|
||||
if(rep.throttledTags.present()) {
|
||||
*throttledTags = std::move(rep.throttledTags.get());
|
||||
}
|
||||
}
|
||||
when ( wait( leaseTimeout ) ) {
|
||||
transactionRateInfo->disable();
|
||||
batchTransactionRateInfo->disable();
|
||||
TraceEvent(SevWarn, "GrvProxyRateLeaseExpired", myID).suppressFor(5.0);
|
||||
//TraceEvent("GrvProxyRate", myID).detail("Rate", 0.0).detail("BatchRate", 0.0).detail("Lease", 0);
|
||||
leaseTimeout = Never();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> queueGetReadVersionRequests(
|
||||
Reference<AsyncVar<ServerDBInfo>> db, SpannedDeque<GetReadVersionRequest>* systemQueue,
|
||||
SpannedDeque<GetReadVersionRequest>* defaultQueue, SpannedDeque<GetReadVersionRequest>* batchQueue,
|
||||
FutureStream<GetReadVersionRequest> readVersionRequests, PromiseStream<Void> GRVTimer, double* lastGRVTime,
|
||||
double* GRVBatchTime, FutureStream<double> normalGRVLatency, GrvProxyStats* stats,
|
||||
GrvTransactionRateInfo* batchRateInfo, TransactionTagMap<uint64_t>* transactionTagCounter) {
|
||||
loop choose{
|
||||
when(GetReadVersionRequest req = waitNext(readVersionRequests)) {
|
||||
//WARNING: this code is run at a high priority, so it needs to do as little work as possible
|
||||
if( stats->txnRequestIn.getValue() - stats->txnRequestOut.getValue() > SERVER_KNOBS->START_TRANSACTION_MAX_QUEUE_SIZE ) {
|
||||
++stats->txnRequestErrors;
|
||||
//FIXME: send an error instead of giving an unreadable version when the client can support the error: req.reply.sendError(proxy_memory_limit_exceeded());
|
||||
GetReadVersionReply rep;
|
||||
rep.version = 1;
|
||||
rep.locked = true;
|
||||
req.reply.send(rep);
|
||||
TraceEvent(SevWarnAlways, "ProxyGRVThresholdExceeded").suppressFor(60);
|
||||
} else {
|
||||
// TODO: check whether this is reasonable to do in the fast path
|
||||
for(auto tag : req.tags) {
|
||||
(*transactionTagCounter)[tag.first] += tag.second;
|
||||
}
|
||||
|
||||
if (req.debugID.present())
|
||||
g_traceBatch.addEvent("TransactionDebug", req.debugID.get().first(), "GrvProxyServer.queueTransactionStartRequests.Before");
|
||||
|
||||
if (systemQueue->empty() && defaultQueue->empty() && batchQueue->empty()) {
|
||||
forwardPromise(GRVTimer, delayJittered(std::max(0.0, *GRVBatchTime - (now() - *lastGRVTime)), TaskPriority::ProxyGRVTimer));
|
||||
}
|
||||
|
||||
++stats->txnRequestIn;
|
||||
stats->txnStartIn += req.transactionCount;
|
||||
if (req.priority >= TransactionPriority::IMMEDIATE) {
|
||||
stats->txnSystemPriorityStartIn += req.transactionCount;
|
||||
systemQueue->push_back(req);
|
||||
systemQueue->span.addParent(req.spanContext);
|
||||
} else if (req.priority >= TransactionPriority::DEFAULT) {
|
||||
stats->txnDefaultPriorityStartIn += req.transactionCount;
|
||||
defaultQueue->push_back(req);
|
||||
defaultQueue->span.addParent(req.spanContext);
|
||||
} else {
|
||||
// Return error for batch_priority GRV requests
|
||||
int64_t proxiesCount = std::max((int)db->get().client.grvProxies.size(), 1);
|
||||
if (batchRateInfo->rate <= (1.0 / proxiesCount)) {
|
||||
req.reply.sendError(batch_transaction_throttled());
|
||||
stats->txnThrottled += req.transactionCount;
|
||||
continue;
|
||||
}
|
||||
|
||||
stats->txnBatchPriorityStartIn += req.transactionCount;
|
||||
batchQueue->push_back(req);
|
||||
batchQueue->span.addParent(req.spanContext);
|
||||
}
|
||||
}
|
||||
}
|
||||
// dynamic batching monitors reply latencies
|
||||
when(double reply_latency = waitNext(normalGRVLatency)) {
|
||||
double target_latency = reply_latency * SERVER_KNOBS->START_TRANSACTION_BATCH_INTERVAL_LATENCY_FRACTION;
|
||||
*GRVBatchTime = std::max(
|
||||
SERVER_KNOBS->START_TRANSACTION_BATCH_INTERVAL_MIN,
|
||||
std::min(SERVER_KNOBS->START_TRANSACTION_BATCH_INTERVAL_MAX,
|
||||
target_latency * SERVER_KNOBS->START_TRANSACTION_BATCH_INTERVAL_SMOOTHER_ALPHA +
|
||||
*GRVBatchTime * (1 - SERVER_KNOBS->START_TRANSACTION_BATCH_INTERVAL_SMOOTHER_ALPHA)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> updateLastCommit(GrvProxyData* self, Optional<UID> debugID = Optional<UID>()) {
|
||||
state double confirmStart = now();
|
||||
self->lastStartCommit = confirmStart;
|
||||
self->updateCommitRequests++;
|
||||
wait(self->logSystem->confirmEpochLive(debugID));
|
||||
self->updateCommitRequests--;
|
||||
self->lastCommitLatency = now()-confirmStart;
|
||||
self->lastCommitTime = std::max(self->lastCommitTime.get(), confirmStart);
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR Future<Void> lastCommitUpdater(GrvProxyData* self, PromiseStream<Future<Void>> addActor) {
|
||||
loop {
|
||||
double interval = std::max(SERVER_KNOBS->MIN_CONFIRM_INTERVAL, (SERVER_KNOBS->REQUIRED_MIN_RECOVERY_DURATION - self->lastCommitLatency)/2.0);
|
||||
double elapsed = now()-self->lastStartCommit;
|
||||
if(elapsed < interval) {
|
||||
wait( delay(interval + 0.0001 - elapsed) );
|
||||
} else {
|
||||
// May want to change the default value of MAX_COMMIT_UPDATES since we don't have
|
||||
if(self->updateCommitRequests < SERVER_KNOBS->MAX_COMMIT_UPDATES) {
|
||||
addActor.send(updateLastCommit(self));
|
||||
} else {
|
||||
TraceEvent(g_network->isSimulated() ? SevInfo : SevWarnAlways, "TooManyLastCommitUpdates").suppressFor(1.0);
|
||||
self->lastStartCommit = now();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<GetReadVersionReply> getLiveCommittedVersion(SpanID parentSpan, GrvProxyData* grvProxyData, uint32_t flags, Optional<UID> debugID,
|
||||
int transactionCount, int systemTransactionCount, int defaultPriTransactionCount, int batchPriTransactionCount)
|
||||
{
|
||||
// Returns a version which (1) is committed, and (2) is >= the latest version reported committed (by a commit response) when this request was sent
|
||||
// (1) The version returned is the committedVersion of some proxy at some point before the request returns, so it is committed.
|
||||
// (2) No proxy on our list reported committed a higher version before this request was received, because then its committedVersion would have been higher,
|
||||
// and no other proxy could have already committed anything without first ending the epoch
|
||||
state Span span("GP:getLiveCommittedVersion"_loc, parentSpan);
|
||||
++grvProxyData->stats.txnStartBatch;
|
||||
state Future<GetRawCommittedVersionReply> replyFromMasterFuture;
|
||||
replyFromMasterFuture = grvProxyData->master.getLiveCommittedVersion.getReply(
|
||||
GetRawCommittedVersionRequest(span.context, debugID), TaskPriority::GetLiveCommittedVersionReply);
|
||||
|
||||
if (!SERVER_KNOBS->ALWAYS_CAUSAL_READ_RISKY && !(flags&GetReadVersionRequest::FLAG_CAUSAL_READ_RISKY)) {
|
||||
wait(updateLastCommit(grvProxyData, debugID));
|
||||
} else if (SERVER_KNOBS->REQUIRED_MIN_RECOVERY_DURATION > 0 &&
|
||||
now() - SERVER_KNOBS->REQUIRED_MIN_RECOVERY_DURATION > grvProxyData->lastCommitTime.get()) {
|
||||
wait(grvProxyData->lastCommitTime.whenAtLeast(now() - SERVER_KNOBS->REQUIRED_MIN_RECOVERY_DURATION));
|
||||
}
|
||||
|
||||
if (debugID.present()) {
|
||||
g_traceBatch.addEvent("TransactionDebug", debugID.get().first(), "GrvProxyServer.getLiveCommittedVersion.confirmEpochLive");
|
||||
}
|
||||
|
||||
GetRawCommittedVersionReply repFromMaster = wait(replyFromMasterFuture);
|
||||
grvProxyData->minKnownCommittedVersion = std::max(grvProxyData->minKnownCommittedVersion, repFromMaster.minKnownCommittedVersion);
|
||||
|
||||
GetReadVersionReply rep;
|
||||
rep.version = repFromMaster.version;
|
||||
rep.locked = repFromMaster.locked;
|
||||
rep.metadataVersion = repFromMaster.metadataVersion;
|
||||
rep.processBusyTime = 1e6 * (g_network->isSimulated() ? deterministicRandom()->random01() : g_network->networkInfo.metrics.lastRunLoopBusyness);
|
||||
|
||||
|
||||
if (debugID.present()) {
|
||||
g_traceBatch.addEvent("TransactionDebug", debugID.get().first(), "GrvProxyServer.getLiveCommittedVersion.After");
|
||||
}
|
||||
|
||||
grvProxyData->stats.txnStartOut += transactionCount;
|
||||
grvProxyData->stats.txnSystemPriorityStartOut += systemTransactionCount;
|
||||
grvProxyData->stats.txnDefaultPriorityStartOut += defaultPriTransactionCount;
|
||||
grvProxyData->stats.txnBatchPriorityStartOut += batchPriTransactionCount;
|
||||
|
||||
return rep;
|
||||
}
|
||||
|
||||
ACTOR Future<Void> sendGrvReplies(Future<GetReadVersionReply> replyFuture, std::vector<GetReadVersionRequest> requests,
|
||||
GrvProxyStats* stats, Version minKnownCommittedVersion,
|
||||
PrioritizedTransactionTagMap<ClientTagThrottleLimits> throttledTags,
|
||||
int64_t midShardSize = 0) {
|
||||
GetReadVersionReply _reply = wait(replyFuture);
|
||||
GetReadVersionReply reply = _reply;
|
||||
Version replyVersion = reply.version;
|
||||
|
||||
double end = g_network->timer();
|
||||
for(GetReadVersionRequest const& request : requests) {
|
||||
double duration = end - request.requestTime();
|
||||
if(request.priority == TransactionPriority::DEFAULT) {
|
||||
stats->grvLatencySample.addMeasurement(duration);
|
||||
}
|
||||
|
||||
if(request.priority >= TransactionPriority::DEFAULT) {
|
||||
stats->grvLatencyBands.addMeasurement(duration);
|
||||
}
|
||||
|
||||
if (request.flags & GetReadVersionRequest::FLAG_USE_MIN_KNOWN_COMMITTED_VERSION) {
|
||||
// Only backup worker may infrequently use this flag.
|
||||
reply.version = minKnownCommittedVersion;
|
||||
}
|
||||
else {
|
||||
reply.version = replyVersion;
|
||||
}
|
||||
reply.midShardSize = midShardSize;
|
||||
reply.tagThrottleInfo.clear();
|
||||
|
||||
if(!request.tags.empty()) {
|
||||
auto& priorityThrottledTags = throttledTags[request.priority];
|
||||
for(auto tag : request.tags) {
|
||||
auto tagItr = priorityThrottledTags.find(tag.first);
|
||||
if(tagItr != priorityThrottledTags.end()) {
|
||||
if(tagItr->second.expiration > now()) {
|
||||
if(tagItr->second.tpsRate == std::numeric_limits<double>::max()) {
|
||||
TEST(true); // Auto TPS rate is unlimited
|
||||
}
|
||||
else {
|
||||
TEST(true); // Proxy returning tag throttle
|
||||
reply.tagThrottleInfo[tag.first] = tagItr->second;
|
||||
}
|
||||
}
|
||||
else {
|
||||
// This isn't required, but we might as well
|
||||
TEST(true); // Proxy expiring tag throttle
|
||||
priorityThrottledTags.erase(tagItr);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
request.reply.send(reply);
|
||||
++stats->txnRequestOut;
|
||||
}
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR Future<Void> monitorDDMetricsChanges(int64_t* midShardSize, Reference<AsyncVar<ServerDBInfo>> db) {
|
||||
state Future<Void> nextRequestTimer = Never();
|
||||
state Future<GetDataDistributorMetricsReply> nextReply = Never();
|
||||
|
||||
if (db->get().distributor.present()) nextRequestTimer = Void();
|
||||
loop {
|
||||
try {
|
||||
choose {
|
||||
when(wait(db->onChange())) {
|
||||
if (db->get().distributor.present()) {
|
||||
TraceEvent("DataDistributorChanged", db->get().id)
|
||||
.detail("DDID", db->get().distributor.get().id());
|
||||
nextRequestTimer = Void();
|
||||
} else {
|
||||
TraceEvent("DataDistributorDied", db->get().id);
|
||||
nextRequestTimer = Never();
|
||||
}
|
||||
nextReply = Never();
|
||||
}
|
||||
when(wait(nextRequestTimer)) {
|
||||
nextRequestTimer = Never();
|
||||
if (db->get().distributor.present()) {
|
||||
nextReply = brokenPromiseToNever(db->get().distributor.get().dataDistributorMetrics.getReply(
|
||||
GetDataDistributorMetricsRequest(normalKeys, CLIENT_KNOBS->TOO_MANY, true)));
|
||||
} else
|
||||
nextReply = Never();
|
||||
}
|
||||
when(GetDataDistributorMetricsReply reply = wait(nextReply)) {
|
||||
nextReply = Never();
|
||||
ASSERT(reply.midShardSize.present());
|
||||
*midShardSize = reply.midShardSize.get();
|
||||
nextRequestTimer = delay(CLIENT_KNOBS->MID_SHARD_SIZE_MAX_STALENESS);
|
||||
}
|
||||
}
|
||||
} catch (Error& e) {
|
||||
TraceEvent("DDMidShardSizeUpdateFail").error(e);
|
||||
if (e.code() != error_code_timed_out && e.code() != error_code_dd_not_found) throw;
|
||||
nextRequestTimer = delay(CLIENT_KNOBS->MID_SHARD_SIZE_MAX_STALENESS);
|
||||
nextReply = Never();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> transactionStarter(GrvProxyInterface proxy, Reference<AsyncVar<ServerDBInfo>> db,
|
||||
PromiseStream<Future<Void>> addActor, GrvProxyData* grvProxyData,
|
||||
GetHealthMetricsReply* healthMetricsReply,
|
||||
GetHealthMetricsReply* detailedHealthMetricsReply) {
|
||||
state double lastGRVTime = 0;
|
||||
state PromiseStream<Void> GRVTimer;
|
||||
state double GRVBatchTime = SERVER_KNOBS->START_TRANSACTION_BATCH_INTERVAL_MIN;
|
||||
|
||||
state int64_t transactionCount = 0;
|
||||
state int64_t batchTransactionCount = 0;
|
||||
state GrvTransactionRateInfo normalRateInfo(10);
|
||||
state GrvTransactionRateInfo batchRateInfo(0);
|
||||
|
||||
state SpannedDeque<GetReadVersionRequest> systemQueue("GP:transactionStarterSystemQueue"_loc);
|
||||
state SpannedDeque<GetReadVersionRequest> defaultQueue("GP:transactionStarterDefaultQueue"_loc);
|
||||
state SpannedDeque<GetReadVersionRequest> batchQueue("GP:transactionStarterBatchQueue"_loc);
|
||||
|
||||
state TransactionTagMap<uint64_t> transactionTagCounter;
|
||||
state PrioritizedTransactionTagMap<ClientTagThrottleLimits> throttledTags;
|
||||
|
||||
state PromiseStream<double> normalGRVLatency;
|
||||
state Span span;
|
||||
|
||||
state int64_t midShardSize = SERVER_KNOBS->MIN_SHARD_BYTES;
|
||||
addActor.send(monitorDDMetricsChanges(&midShardSize, db));
|
||||
|
||||
addActor.send(getRate(proxy.id(), db, &transactionCount, &batchTransactionCount, &normalRateInfo, &batchRateInfo, healthMetricsReply, detailedHealthMetricsReply, &transactionTagCounter, &throttledTags));
|
||||
addActor.send(queueGetReadVersionRequests(db, &systemQueue, &defaultQueue, &batchQueue,
|
||||
proxy.getConsistentReadVersion.getFuture(), GRVTimer, &lastGRVTime,
|
||||
&GRVBatchTime, normalGRVLatency.getFuture(), &grvProxyData->stats,
|
||||
&batchRateInfo, &transactionTagCounter));
|
||||
|
||||
while (std::find(db->get().client.grvProxies.begin(), db->get().client.grvProxies.end(), proxy) ==
|
||||
db->get().client.grvProxies.end()) {
|
||||
wait(db->onChange());
|
||||
}
|
||||
|
||||
ASSERT(db->get().recoveryState >= RecoveryState::ACCEPTING_COMMITS); // else potentially we could return uncommitted read versions from master.
|
||||
TraceEvent("GrvProxyReadyForTxnStarts", proxy.id());
|
||||
|
||||
loop{
|
||||
waitNext(GRVTimer.getFuture());
|
||||
// Select zero or more transactions to start
|
||||
double t = now();
|
||||
double elapsed = now() - lastGRVTime;
|
||||
lastGRVTime = t;
|
||||
|
||||
// Resolve a possible indeterminate multiplication with infinite transaction rate
|
||||
if (elapsed == 0) {
|
||||
elapsed = 1e-15;
|
||||
}
|
||||
|
||||
normalRateInfo.reset();
|
||||
batchRateInfo.reset();
|
||||
|
||||
int transactionsStarted[2] = {0,0};
|
||||
int systemTransactionsStarted[2] = {0,0};
|
||||
int defaultPriTransactionsStarted[2] = { 0, 0 };
|
||||
int batchPriTransactionsStarted[2] = { 0, 0 };
|
||||
|
||||
vector<vector<GetReadVersionRequest>> start(2); // start[0] is transactions starting with !(flags&CAUSAL_READ_RISKY), start[1] is transactions starting with flags&CAUSAL_READ_RISKY
|
||||
Optional<UID> debugID;
|
||||
|
||||
int requestsToStart = 0;
|
||||
|
||||
while (requestsToStart < SERVER_KNOBS->START_TRANSACTION_MAX_REQUESTS_TO_START) {
|
||||
SpannedDeque<GetReadVersionRequest>* transactionQueue;
|
||||
if(!systemQueue.empty()) {
|
||||
transactionQueue = &systemQueue;
|
||||
} else if(!defaultQueue.empty()) {
|
||||
transactionQueue = &defaultQueue;
|
||||
} else if(!batchQueue.empty()) {
|
||||
transactionQueue = &batchQueue;
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
transactionQueue->span.swap(span);
|
||||
|
||||
auto& req = transactionQueue->front();
|
||||
int tc = req.transactionCount;
|
||||
|
||||
if(req.priority < TransactionPriority::DEFAULT && !batchRateInfo.canStart(transactionsStarted[0] + transactionsStarted[1], tc)) {
|
||||
break;
|
||||
}
|
||||
else if(req.priority < TransactionPriority::IMMEDIATE && !normalRateInfo.canStart(transactionsStarted[0] + transactionsStarted[1], tc)) {
|
||||
break;
|
||||
}
|
||||
|
||||
if (req.debugID.present()) {
|
||||
if (!debugID.present()) debugID = nondeterministicRandom()->randomUniqueID();
|
||||
g_traceBatch.addAttach("TransactionAttachID", req.debugID.get().first(), debugID.get().first());
|
||||
}
|
||||
|
||||
transactionsStarted[req.flags&1] += tc;
|
||||
if (req.priority >= TransactionPriority::IMMEDIATE)
|
||||
systemTransactionsStarted[req.flags & 1] += tc;
|
||||
else if (req.priority >= TransactionPriority::DEFAULT)
|
||||
defaultPriTransactionsStarted[req.flags & 1] += tc;
|
||||
else
|
||||
batchPriTransactionsStarted[req.flags & 1] += tc;
|
||||
|
||||
start[req.flags & 1].push_back(std::move(req));
|
||||
static_assert(GetReadVersionRequest::FLAG_CAUSAL_READ_RISKY == 1, "Implementation dependent on flag value");
|
||||
transactionQueue->pop_front();
|
||||
requestsToStart++;
|
||||
}
|
||||
|
||||
if (!systemQueue.empty() || !defaultQueue.empty() || !batchQueue.empty()) {
|
||||
forwardPromise(GRVTimer, delayJittered(SERVER_KNOBS->START_TRANSACTION_BATCH_QUEUE_CHECK_INTERVAL, TaskPriority::ProxyGRVTimer));
|
||||
}
|
||||
|
||||
/*TraceEvent("GRVBatch", proxy.id())
|
||||
.detail("Elapsed", elapsed)
|
||||
.detail("NTransactionToStart", nTransactionsToStart)
|
||||
.detail("TransactionRate", transactionRate)
|
||||
.detail("TransactionQueueSize", transactionQueue.size())
|
||||
.detail("NumTransactionsStarted", transactionsStarted[0] + transactionsStarted[1])
|
||||
.detail("NumSystemTransactionsStarted", systemTransactionsStarted[0] + systemTransactionsStarted[1])
|
||||
.detail("NumNonSystemTransactionsStarted", transactionsStarted[0] + transactionsStarted[1] -
|
||||
systemTransactionsStarted[0] - systemTransactionsStarted[1])
|
||||
.detail("TransactionBudget", transactionBudget)
|
||||
.detail("BatchTransactionBudget", batchTransactionBudget);*/
|
||||
|
||||
int systemTotalStarted = systemTransactionsStarted[0] + systemTransactionsStarted[1];
|
||||
int normalTotalStarted = defaultPriTransactionsStarted[0] + defaultPriTransactionsStarted[1];
|
||||
int batchTotalStarted = batchPriTransactionsStarted[0] + batchPriTransactionsStarted[1];
|
||||
|
||||
transactionCount += transactionsStarted[0] + transactionsStarted[1];
|
||||
batchTransactionCount += batchTotalStarted;
|
||||
|
||||
normalRateInfo.updateBudget(systemTotalStarted + normalTotalStarted, systemQueue.empty() && defaultQueue.empty(), elapsed);
|
||||
batchRateInfo.updateBudget(systemTotalStarted + normalTotalStarted + batchTotalStarted, systemQueue.empty() && defaultQueue.empty() && batchQueue.empty(), elapsed);
|
||||
|
||||
if (debugID.present()) {
|
||||
g_traceBatch.addEvent("TransactionDebug", debugID.get().first(),
|
||||
"GrvProxyServer.transactionStarter.AskLiveCommittedVersionFromMaster");
|
||||
}
|
||||
|
||||
for (int i = 0; i < start.size(); i++) {
|
||||
if (start[i].size()) {
|
||||
Future<GetReadVersionReply> readVersionReply = getLiveCommittedVersion(
|
||||
span.context, grvProxyData, i, debugID, transactionsStarted[i], systemTransactionsStarted[i], defaultPriTransactionsStarted[i], batchPriTransactionsStarted[i]);
|
||||
addActor.send(sendGrvReplies(readVersionReply, start[i], &grvProxyData->stats,
|
||||
grvProxyData->minKnownCommittedVersion, throttledTags, midShardSize));
|
||||
|
||||
// Use normal priority transaction's GRV latency to dynamically calculate transaction batching interval.
|
||||
if (i == 0) {
|
||||
addActor.send(timeReply(readVersionReply, normalGRVLatency));
|
||||
}
|
||||
}
|
||||
}
|
||||
span = Span(span.location);
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> grvProxyServerCore(
|
||||
GrvProxyInterface proxy,
|
||||
MasterInterface master,
|
||||
Reference<AsyncVar<ServerDBInfo>> db)
|
||||
{
|
||||
state GrvProxyData grvProxyData(proxy.id(), master, proxy.getConsistentReadVersion, db);
|
||||
|
||||
state PromiseStream<Future<Void>> addActor;
|
||||
state Future<Void> onError = transformError( actorCollection(addActor.getFuture()), broken_promise(), master_tlog_failed() );
|
||||
|
||||
state GetHealthMetricsReply healthMetricsReply;
|
||||
state GetHealthMetricsReply detailedHealthMetricsReply;
|
||||
|
||||
addActor.send( waitFailureServer(proxy.waitFailure.getFuture()) );
|
||||
addActor.send( traceRole(Role::GRV_PROXY, proxy.id()) );
|
||||
|
||||
// Wait until we can load the "real" logsystem, since we don't support switching them currently
|
||||
while (!(grvProxyData.db->get().master.id() == master.id() && grvProxyData.db->get().recoveryState >= RecoveryState::RECOVERY_TRANSACTION)) {
|
||||
wait(grvProxyData.db->onChange());
|
||||
}
|
||||
// Do we need to wait for any db info change? Yes. To update latency band.
|
||||
state Future<Void> dbInfoChange = grvProxyData.db->onChange();
|
||||
grvProxyData.logSystem = ILogSystem::fromServerDBInfo(proxy.id(), grvProxyData.db->get(), false, addActor);
|
||||
|
||||
grvProxyData.updateLatencyBandConfig(grvProxyData.db->get().latencyBandConfig);
|
||||
|
||||
addActor.send(transactionStarter(proxy, grvProxyData.db, addActor, &grvProxyData, &healthMetricsReply,
|
||||
&detailedHealthMetricsReply));
|
||||
addActor.send(healthMetricsRequestServer(proxy, &healthMetricsReply, &detailedHealthMetricsReply));
|
||||
|
||||
if(SERVER_KNOBS->REQUIRED_MIN_RECOVERY_DURATION > 0) {
|
||||
addActor.send(lastCommitUpdater(&grvProxyData, addActor));
|
||||
}
|
||||
|
||||
loop choose{
|
||||
when( wait( dbInfoChange ) ) {
|
||||
dbInfoChange = grvProxyData.db->onChange();
|
||||
|
||||
if(grvProxyData.db->get().master.id() == master.id() && grvProxyData.db->get().recoveryState >= RecoveryState::RECOVERY_TRANSACTION) {
|
||||
grvProxyData.logSystem = ILogSystem::fromServerDBInfo(proxy.id(), grvProxyData.db->get(), false, addActor);
|
||||
}
|
||||
grvProxyData.updateLatencyBandConfig(grvProxyData.db->get().latencyBandConfig);
|
||||
}
|
||||
when(wait(onError)) {}
|
||||
}
|
||||
|
||||
}
|
||||
|
||||
ACTOR Future<Void> checkRemoved(Reference<AsyncVar<ServerDBInfo>> db, uint64_t recoveryCount, GrvProxyInterface myInterface) {
|
||||
loop{
|
||||
if (db->get().recoveryCount >= recoveryCount && !std::count(db->get().client.grvProxies.begin(), db->get().client.grvProxies.end(), myInterface)) {
|
||||
throw worker_removed();
|
||||
}
|
||||
wait(db->onChange());
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> grvProxyServer(
|
||||
GrvProxyInterface proxy,
|
||||
InitializeGrvProxyRequest req,
|
||||
Reference<AsyncVar<ServerDBInfo>> db)
|
||||
{
|
||||
try {
|
||||
state Future<Void> core = grvProxyServerCore(proxy, req.master, db);
|
||||
wait(core || checkRemoved(db, req.recoveryCount, proxy));
|
||||
}
|
||||
catch (Error& e) {
|
||||
TraceEvent("GrvProxyTerminated", proxy.id()).error(e, true);
|
||||
|
||||
if (e.code() != error_code_worker_removed && e.code() != error_code_tlog_stopped &&
|
||||
e.code() != error_code_master_tlog_failed && e.code() != error_code_coordinators_changed &&
|
||||
e.code() != error_code_coordinated_state_conflict && e.code() != error_code_new_coordinators_timed_out) {
|
||||
throw;
|
||||
}
|
||||
}
|
||||
return Void();
|
||||
}
|
|
@ -105,6 +105,10 @@ public:
|
|||
// Free pageID to be used again after the commit that moves oldestVersion past v
|
||||
virtual void freePage(LogicalPageID pageID, Version v) = 0;
|
||||
|
||||
// If id is remapped, delete the original as of version v and return the page it was remapped to. The caller
|
||||
// is then responsible for referencing and deleting the returned page ID.
|
||||
virtual LogicalPageID detachRemappedPage(LogicalPageID id, Version v) = 0;
|
||||
|
||||
// Returns the latest data (regardless of version) for a page by LogicalPageID
|
||||
// The data returned will be the later of
|
||||
// - the most recent committed atomic
|
||||
|
@ -133,7 +137,7 @@ public:
|
|||
|
||||
virtual StorageBytes getStorageBytes() const = 0;
|
||||
|
||||
// Count of pages in use by the pager client
|
||||
// Count of pages in use by the pager client (including retained old page versions)
|
||||
virtual Future<int64_t> getUserPageCount() = 0;
|
||||
|
||||
// Future returned is ready when pager has been initialized from disk and is ready for reads and writes.
|
||||
|
|
|
@ -2,6 +2,7 @@
|
|||
|
||||
#include <rocksdb/db.h>
|
||||
#include <rocksdb/options.h>
|
||||
#include <rocksdb/utilities/table_properties_collectors.h>
|
||||
#include "flow/flow.h"
|
||||
#include "flow/IThreadPool.h"
|
||||
|
||||
|
@ -22,14 +23,23 @@ StringRef toStringRef(rocksdb::Slice s) {
|
|||
return StringRef(reinterpret_cast<const uint8_t*>(s.data()), s.size());
|
||||
}
|
||||
|
||||
rocksdb::Options getOptions() {
|
||||
rocksdb::Options options;
|
||||
options.create_if_missing = true;
|
||||
rocksdb::ColumnFamilyOptions getCFOptions() {
|
||||
rocksdb::ColumnFamilyOptions options;
|
||||
options.level_compaction_dynamic_level_bytes = true;
|
||||
options.OptimizeLevelStyleCompaction(SERVER_KNOBS->ROCKSDB_MEMTABLE_BYTES);
|
||||
// Compact sstables when there's too much deleted stuff.
|
||||
options.table_properties_collector_factories = { rocksdb::NewCompactOnDeletionCollectorFactory(128, 1) };
|
||||
return options;
|
||||
}
|
||||
|
||||
rocksdb::ColumnFamilyOptions getCFOptions() {
|
||||
return {};
|
||||
rocksdb::Options getOptions() {
|
||||
rocksdb::Options options({}, getCFOptions());
|
||||
options.avoid_unnecessary_blocking_io = true;
|
||||
options.create_if_missing = true;
|
||||
if (SERVER_KNOBS->ROCKSDB_BACKGROUND_PARALLELISM > 0) {
|
||||
options.IncreaseParallelism(SERVER_KNOBS->ROCKSDB_BACKGROUND_PARALLELISM);
|
||||
}
|
||||
return options;
|
||||
}
|
||||
|
||||
struct RocksDBKeyValueStore : IKeyValueStore {
|
||||
|
@ -119,7 +129,6 @@ struct RocksDBKeyValueStore : IKeyValueStore {
|
|||
|
||||
struct Reader : IThreadPoolReceiver {
|
||||
DB& db;
|
||||
rocksdb::ReadOptions readOptions;
|
||||
|
||||
explicit Reader(DB& db) : db(db) {}
|
||||
|
||||
|
@ -141,7 +150,7 @@ struct RocksDBKeyValueStore : IKeyValueStore {
|
|||
traceBatch.get().addEvent("GetValueDebug", a.debugID.get().first(), "Reader.Before");
|
||||
}
|
||||
rocksdb::PinnableSlice value;
|
||||
auto s = db->Get(readOptions, db->DefaultColumnFamily(), toSlice(a.key), &value);
|
||||
auto s = db->Get({}, db->DefaultColumnFamily(), toSlice(a.key), &value);
|
||||
if (a.debugID.present()) {
|
||||
traceBatch.get().addEvent("GetValueDebug", a.debugID.get().first(), "Reader.After");
|
||||
traceBatch.get().dump();
|
||||
|
@ -172,7 +181,7 @@ struct RocksDBKeyValueStore : IKeyValueStore {
|
|||
traceBatch.get().addEvent("GetValuePrefixDebug", a.debugID.get().first(),
|
||||
"Reader.Before"); //.detail("TaskID", g_network->getCurrentTask());
|
||||
}
|
||||
auto s = db->Get(readOptions, db->DefaultColumnFamily(), toSlice(a.key), &value);
|
||||
auto s = db->Get({}, db->DefaultColumnFamily(), toSlice(a.key), &value);
|
||||
if (a.debugID.present()) {
|
||||
traceBatch.get().addEvent("GetValuePrefixDebug", a.debugID.get().first(),
|
||||
"Reader.After"); //.detail("TaskID", g_network->getCurrentTask());
|
||||
|
@ -195,33 +204,51 @@ struct RocksDBKeyValueStore : IKeyValueStore {
|
|||
virtual double getTimeEstimate() { return SERVER_KNOBS->READ_RANGE_TIME_ESTIMATE; }
|
||||
};
|
||||
void action(ReadRangeAction& a) {
|
||||
auto cursor = std::unique_ptr<rocksdb::Iterator>(db->NewIterator(readOptions));
|
||||
Standalone<RangeResultRef> result;
|
||||
if (a.rowLimit == 0 || a.byteLimit == 0) {
|
||||
a.result.send(result);
|
||||
}
|
||||
int accumulatedBytes = 0;
|
||||
rocksdb::Status s;
|
||||
if (a.rowLimit >= 0) {
|
||||
rocksdb::ReadOptions options;
|
||||
auto endSlice = toSlice(a.keys.end);
|
||||
options.iterate_upper_bound = &endSlice;
|
||||
auto cursor = std::unique_ptr<rocksdb::Iterator>(db->NewIterator(options));
|
||||
cursor->Seek(toSlice(a.keys.begin));
|
||||
while (cursor->Valid() && toStringRef(cursor->key()) < a.keys.end && result.size() < a.rowLimit &&
|
||||
accumulatedBytes < a.byteLimit) {
|
||||
while (cursor->Valid() && toStringRef(cursor->key()) < a.keys.end) {
|
||||
KeyValueRef kv(toStringRef(cursor->key()), toStringRef(cursor->value()));
|
||||
accumulatedBytes += sizeof(KeyValueRef) + kv.expectedSize();
|
||||
result.push_back_deep(result.arena(), kv);
|
||||
// Calling `cursor->Next()` is potentially expensive, so short-circut here just in case.
|
||||
if (result.size() >= a.rowLimit || accumulatedBytes >= a.byteLimit) {
|
||||
break;
|
||||
}
|
||||
cursor->Next();
|
||||
}
|
||||
s = cursor->status();
|
||||
} else {
|
||||
rocksdb::ReadOptions options;
|
||||
auto beginSlice = toSlice(a.keys.begin);
|
||||
options.iterate_lower_bound = &beginSlice;
|
||||
auto cursor = std::unique_ptr<rocksdb::Iterator>(db->NewIterator(options));
|
||||
cursor->SeekForPrev(toSlice(a.keys.end));
|
||||
if (cursor->Valid() && toStringRef(cursor->key()) == a.keys.end) {
|
||||
cursor->Prev();
|
||||
}
|
||||
|
||||
while (cursor->Valid() && toStringRef(cursor->key()) >= a.keys.begin && result.size() < -a.rowLimit &&
|
||||
accumulatedBytes < a.byteLimit) {
|
||||
while (cursor->Valid() && toStringRef(cursor->key()) >= a.keys.begin) {
|
||||
KeyValueRef kv(toStringRef(cursor->key()), toStringRef(cursor->value()));
|
||||
accumulatedBytes += sizeof(KeyValueRef) + kv.expectedSize();
|
||||
result.push_back_deep(result.arena(), kv);
|
||||
// Calling `cursor->Prev()` is potentially expensive, so short-circut here just in case.
|
||||
if (result.size() >= -a.rowLimit || accumulatedBytes >= a.byteLimit) {
|
||||
break;
|
||||
}
|
||||
cursor->Prev();
|
||||
}
|
||||
s = cursor->status();
|
||||
}
|
||||
auto s = cursor->status();
|
||||
|
||||
if (!s.ok()) {
|
||||
TraceEvent(SevError, "RocksDBError").detail("Error", s.ToString()).detail("Method", "ReadRange");
|
||||
}
|
||||
|
|
|
@ -93,9 +93,14 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
|
|||
init( PEEK_RESET_INTERVAL, 300.0 ); if ( randomize && BUGGIFY ) PEEK_RESET_INTERVAL = 20.0;
|
||||
init( PEEK_MAX_LATENCY, 0.5 ); if ( randomize && BUGGIFY ) PEEK_MAX_LATENCY = 0.0;
|
||||
init( PEEK_COUNT_SMALL_MESSAGES, false ); if ( randomize && BUGGIFY ) PEEK_COUNT_SMALL_MESSAGES = true;
|
||||
init( PEEK_STATS_INTERVAL, 10.0 );
|
||||
init( PEEK_STATS_SLOW_AMOUNT, 0 );
|
||||
init( PEEK_STATS_INTERVAL, 10.0 );
|
||||
init( PEEK_STATS_SLOW_AMOUNT, 2 );
|
||||
init( PEEK_STATS_SLOW_RATIO, 0.5 );
|
||||
init( PUSH_RESET_INTERVAL, 300.0 ); if ( randomize && BUGGIFY ) PUSH_RESET_INTERVAL = 20.0;
|
||||
init( PUSH_MAX_LATENCY, 0.5 ); if ( randomize && BUGGIFY ) PUSH_MAX_LATENCY = 0.0;
|
||||
init( PUSH_STATS_INTERVAL, 10.0 );
|
||||
init( PUSH_STATS_SLOW_AMOUNT, 2 );
|
||||
init( PUSH_STATS_SLOW_RATIO, 0.5 );
|
||||
|
||||
// disk snapshot max timeout, to be put in TLog, storage and coordinator nodes
|
||||
init( SNAP_CREATE_MAX_TIMEOUT, 300.0 );
|
||||
|
@ -236,7 +241,7 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
|
|||
init( DD_VALIDATE_LOCALITY, true ); if( randomize && BUGGIFY ) DD_VALIDATE_LOCALITY = false;
|
||||
init( DD_CHECK_INVALID_LOCALITY_DELAY, 60 ); if( randomize && BUGGIFY ) DD_CHECK_INVALID_LOCALITY_DELAY = 1 + deterministicRandom()->random01() * 600;
|
||||
init( DD_ENABLE_VERBOSE_TRACING, false ); if( randomize && BUGGIFY ) DD_ENABLE_VERBOSE_TRACING = true;
|
||||
init( DD_SS_FAILURE_VERSIONLAG, 250000000 );
|
||||
init( DD_SS_FAILURE_VERSIONLAG, 250000000 );
|
||||
init( DD_SS_ALLOWED_VERSIONLAG, 200000000 ); if( randomize && BUGGIFY ) { DD_SS_FAILURE_VERSIONLAG = deterministicRandom()->randomInt(15000000, 500000000); DD_SS_ALLOWED_VERSIONLAG = 0.75 * DD_SS_FAILURE_VERSIONLAG; }
|
||||
init( DD_SS_STUCK_TIME_LIMIT, 300.0 ); if( randomize && BUGGIFY ) { DD_SS_STUCK_TIME_LIMIT = 200.0 + deterministicRandom()->random01() * 100.0; }
|
||||
|
||||
|
@ -308,6 +313,10 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
|
|||
// KeyValueStoreMemory
|
||||
init( REPLACE_CONTENTS_BYTES, 1e5 );
|
||||
|
||||
// KeyValueStoreRocksDB
|
||||
init( ROCKSDB_BACKGROUND_PARALLELISM, 0 );
|
||||
init( ROCKSDB_MEMTABLE_BYTES, 512 * 1024 * 1024 );
|
||||
|
||||
// Leader election
|
||||
bool longLeaderElection = randomize && BUGGIFY;
|
||||
init( MAX_NOTIFICATIONS, 100000 );
|
||||
|
@ -319,7 +328,7 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
|
|||
init( POLLING_FREQUENCY, 2.0 ); if( longLeaderElection ) POLLING_FREQUENCY = 8.0;
|
||||
init( HEARTBEAT_FREQUENCY, 0.5 ); if( longLeaderElection ) HEARTBEAT_FREQUENCY = 1.0;
|
||||
|
||||
// Master Proxy
|
||||
// Master Proxy and GRV Proxy
|
||||
init( START_TRANSACTION_BATCH_INTERVAL_MIN, 1e-6 );
|
||||
init( START_TRANSACTION_BATCH_INTERVAL_MAX, 0.010 );
|
||||
init( START_TRANSACTION_BATCH_INTERVAL_LATENCY_FRACTION, 0.5 );
|
||||
|
@ -365,6 +374,7 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
|
|||
init( PROXY_COMPUTE_BUCKETS, 20000 );
|
||||
init( PROXY_COMPUTE_GROWTH_RATE, 0.01 );
|
||||
init( TXN_STATE_SEND_AMOUNT, 4 );
|
||||
init( REPORT_TRANSACTION_COST_ESTIMATION_DELAY, 0.1 );
|
||||
|
||||
// Master Server
|
||||
// masterCommitter() in the master server will allow lower priority tasks (e.g. DataDistibution)
|
||||
|
@ -429,6 +439,7 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
|
|||
init( EXPECTED_TLOG_FITNESS, ProcessClass::UnsetFit );
|
||||
init( EXPECTED_LOG_ROUTER_FITNESS, ProcessClass::UnsetFit );
|
||||
init( EXPECTED_PROXY_FITNESS, ProcessClass::UnsetFit );
|
||||
init( EXPECTED_GRV_PROXY_FITNESS, ProcessClass::UnsetFit );
|
||||
init( EXPECTED_RESOLVER_FITNESS, ProcessClass::UnsetFit );
|
||||
init( RECRUITMENT_TIMEOUT, 600 ); if( randomize && BUGGIFY ) RECRUITMENT_TIMEOUT = deterministicRandom()->coinflip() ? 60.0 : 1.0;
|
||||
|
||||
|
@ -525,7 +536,7 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
|
|||
init( BYTES_READ_UNITS_PER_SAMPLE, 100000 ); // 100K bytes
|
||||
init( READ_HOT_SUB_RANGE_CHUNK_SIZE, 10000000); // 10MB
|
||||
init( EMPTY_READ_PENALTY, 20 ); // 20 bytes
|
||||
init( READ_SAMPLING_ENABLED, true ); if ( randomize && BUGGIFY ) READ_SAMPLING_ENABLED = false;// enable/disable read sampling
|
||||
init( READ_SAMPLING_ENABLED, false ); if ( randomize && BUGGIFY ) READ_SAMPLING_ENABLED = true;// enable/disable read sampling
|
||||
|
||||
//Storage Server
|
||||
init( STORAGE_LOGGING_DELAY, 5.0 );
|
||||
|
@ -555,9 +566,9 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
|
|||
init( BEHIND_CHECK_COUNT, 2 );
|
||||
init( BEHIND_CHECK_VERSIONS, 5 * VERSIONS_PER_SECOND );
|
||||
init( WAIT_METRICS_WRONG_SHARD_CHANCE, isSimulated ? 1.0 : 0.1 );
|
||||
init( MIN_TAG_PAGES_READ_RATE, 1.0e4 ); if( randomize && BUGGIFY ) MIN_TAG_PAGES_READ_RATE = 0;
|
||||
init( READ_TAG_MEASUREMENT_INTERVAL, 30.0 ); if( randomize && BUGGIFY ) READ_TAG_MEASUREMENT_INTERVAL = 1.0;
|
||||
init( OPERATION_COST_BYTE_FACTOR, 16384 ); if( randomize && BUGGIFY ) OPERATION_COST_BYTE_FACTOR = 4096;
|
||||
init( MIN_TAG_PAGES_RATE, 1.0e4 ); if( randomize && BUGGIFY ) MIN_TAG_PAGES_RATE = 0;
|
||||
init( TAG_MEASUREMENT_INTERVAL, 30.0 ); if( randomize && BUGGIFY ) TAG_MEASUREMENT_INTERVAL = 1.0;
|
||||
init( READ_COST_BYTE_FACTOR, 16384 ); if( randomize && BUGGIFY ) READ_COST_BYTE_FACTOR = 4096;
|
||||
init( PREFIX_COMPRESS_KVS_MEM_SNAPSHOTS, true ); if( randomize && BUGGIFY ) PREFIX_COMPRESS_KVS_MEM_SNAPSHOTS = false;
|
||||
|
||||
//Wait Failure
|
||||
|
@ -573,7 +584,7 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
|
|||
init( TRACE_LOG_FLUSH_FAILURE_CHECK_INTERVAL_SECONDS, 10 );
|
||||
init( TRACE_LOG_PING_TIMEOUT_SECONDS, 5.0 );
|
||||
init( MIN_DELAY_CC_WORST_FIT_CANDIDACY_SECONDS, 10.0 );
|
||||
init( MAX_DELAY_CC_WORST_FIT_CANDIDACY_SECONDS, 30.0 );
|
||||
init( MAX_DELAY_CC_WORST_FIT_CANDIDACY_SECONDS, 30.0 );
|
||||
init( DBINFO_FAILED_DELAY, 1.0 );
|
||||
|
||||
// Test harness
|
||||
|
@ -608,13 +619,13 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
|
|||
init( FASTRESTORE_FAILURE_TIMEOUT, 3600 );
|
||||
init( FASTRESTORE_HEARTBEAT_INTERVAL, 60 );
|
||||
init( FASTRESTORE_SAMPLING_PERCENT, 100 ); if( randomize && BUGGIFY ) { FASTRESTORE_SAMPLING_PERCENT = deterministicRandom()->random01() * 100; }
|
||||
init( FASTRESTORE_NUM_LOADERS, 2 ); if( randomize && BUGGIFY ) { FASTRESTORE_NUM_LOADERS = deterministicRandom()->random01() * 10 + 1; }
|
||||
init( FASTRESTORE_NUM_LOADERS, 3 ); if( randomize && BUGGIFY ) { FASTRESTORE_NUM_LOADERS = deterministicRandom()->random01() * 10 + 1; }
|
||||
init( FASTRESTORE_NUM_APPLIERS, 3 ); if( randomize && BUGGIFY ) { FASTRESTORE_NUM_APPLIERS = deterministicRandom()->random01() * 10 + 1; }
|
||||
init( FASTRESTORE_TXN_BATCH_MAX_BYTES, 1024.0 ); if( randomize && BUGGIFY ) { FASTRESTORE_TXN_BATCH_MAX_BYTES = deterministicRandom()->random01() * 1024.0 * 1024.0 + 1.0; }
|
||||
init( FASTRESTORE_VERSIONBATCH_MAX_BYTES, 10.0 * 1024.0 * 1024.0 ); if( randomize && BUGGIFY ) { FASTRESTORE_VERSIONBATCH_MAX_BYTES = deterministicRandom()->random01() * 10.0 * 1024.0 * 1024.0 * 1024.0; } // too small value may increase chance of TooManyFile error
|
||||
init( FASTRESTORE_VB_PARALLELISM, 5 ); if( randomize && BUGGIFY ) { FASTRESTORE_VB_PARALLELISM = deterministicRandom()->random01() * 20 + 1; }
|
||||
init( FASTRESTORE_TXN_BATCH_MAX_BYTES, 1024.0 * 1024.0 ); if( randomize && BUGGIFY ) { FASTRESTORE_TXN_BATCH_MAX_BYTES = deterministicRandom()->random01() * 1024.0 * 1024.0 + 1.0; }
|
||||
init( FASTRESTORE_VERSIONBATCH_MAX_BYTES, 10.0 * 1024.0 * 1024.0 ); if( randomize && BUGGIFY ) { FASTRESTORE_VERSIONBATCH_MAX_BYTES = deterministicRandom()->random01() < 0.2 ? 5 * 1024 : deterministicRandom()->random01() < 0.4 ? 100 * 1024 * 1024 : deterministicRandom()->random01() * 1000.0 * 1024.0 * 1024.0; } // too small value may increase chance of TooManyFile error
|
||||
init( FASTRESTORE_VB_PARALLELISM, 5 ); if( randomize && BUGGIFY ) { FASTRESTORE_VB_PARALLELISM = deterministicRandom()->random01() < 0.2 ? 2 : deterministicRandom()->random01() * 10 + 1; }
|
||||
init( FASTRESTORE_VB_MONITOR_DELAY, 30 ); if( randomize && BUGGIFY ) { FASTRESTORE_VB_MONITOR_DELAY = deterministicRandom()->random01() * 20 + 1; }
|
||||
init( FASTRESTORE_VB_LAUNCH_DELAY, 5 ); if( randomize && BUGGIFY ) { FASTRESTORE_VB_LAUNCH_DELAY = deterministicRandom()->random01() * 60 + 1; }
|
||||
init( FASTRESTORE_VB_LAUNCH_DELAY, 1.0 ); if( randomize && BUGGIFY ) { FASTRESTORE_VB_LAUNCH_DELAY = deterministicRandom()->random01() < 0.2 ? 0.1 : deterministicRandom()->random01() * 10.0 + 1; }
|
||||
init( FASTRESTORE_ROLE_LOGGING_DELAY, 5 ); if( randomize && BUGGIFY ) { FASTRESTORE_ROLE_LOGGING_DELAY = deterministicRandom()->random01() * 60 + 1; }
|
||||
init( FASTRESTORE_UPDATE_PROCESS_STATS_INTERVAL, 5 ); if( randomize && BUGGIFY ) { FASTRESTORE_UPDATE_PROCESS_STATS_INTERVAL = deterministicRandom()->random01() * 60 + 1; }
|
||||
init( FASTRESTORE_ATOMICOP_WEIGHT, 1 ); if( randomize && BUGGIFY ) { FASTRESTORE_ATOMICOP_WEIGHT = deterministicRandom()->random01() * 200 + 1; }
|
||||
|
@ -628,7 +639,7 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
|
|||
init( FASTRESTORE_HEARTBEAT_DELAY, 10 ); if( randomize && BUGGIFY ) { FASTRESTORE_HEARTBEAT_DELAY = deterministicRandom()->random01() * 120 + 2; }
|
||||
init( FASTRESTORE_HEARTBEAT_MAX_DELAY, 10 ); if( randomize && BUGGIFY ) { FASTRESTORE_HEARTBEAT_MAX_DELAY = FASTRESTORE_HEARTBEAT_DELAY * 10; }
|
||||
init( FASTRESTORE_APPLIER_FETCH_KEYS_SIZE, 100 ); if( randomize && BUGGIFY ) { FASTRESTORE_APPLIER_FETCH_KEYS_SIZE = deterministicRandom()->random01() * 10240 + 1; }
|
||||
init( FASTRESTORE_LOADER_SEND_MUTATION_MSG_BYTES, 1.0 * 1024.0 * 1024.0 ); if( randomize && BUGGIFY ) { FASTRESTORE_LOADER_SEND_MUTATION_MSG_BYTES = deterministicRandom()->random01() * 10.0 * 1024.0 * 1024.0 + 1; }
|
||||
init( FASTRESTORE_LOADER_SEND_MUTATION_MSG_BYTES, 1.0 * 1024.0 * 1024.0 ); if( randomize && BUGGIFY ) { FASTRESTORE_LOADER_SEND_MUTATION_MSG_BYTES = deterministicRandom()->random01() < 0.2 ? 1024 : deterministicRandom()->random01() * 5.0 * 1024.0 * 1024.0 + 1; }
|
||||
init( FASTRESTORE_GET_RANGE_VERSIONS_EXPENSIVE, false ); if( randomize && BUGGIFY ) { FASTRESTORE_GET_RANGE_VERSIONS_EXPENSIVE = deterministicRandom()->random01() < 0.5 ? true : false; }
|
||||
init( FASTRESTORE_REQBATCH_PARALLEL, 50 ); if( randomize && BUGGIFY ) { FASTRESTORE_REQBATCH_PARALLEL = deterministicRandom()->random01() * 100 + 1; }
|
||||
init( FASTRESTORE_REQBATCH_LOG, false ); if( randomize && BUGGIFY ) { FASTRESTORE_REQBATCH_LOG = deterministicRandom()->random01() < 0.2 ? true : false; }
|
||||
|
@ -639,6 +650,17 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
|
|||
init( FASTRESTORE_USE_RANGE_FILE, true ); // Perf test only: set it to false will cause simulation failure
|
||||
init( FASTRESTORE_USE_LOG_FILE, true ); // Perf test only: set it to false will cause simulation failure
|
||||
init( FASTRESTORE_SAMPLE_MSG_BYTES, 1048576 ); if( randomize && BUGGIFY ) { FASTRESTORE_SAMPLE_MSG_BYTES = deterministicRandom()->random01() * 2048;}
|
||||
init( FASTRESTORE_SCHED_UPDATE_DELAY, 0.1 ); if( randomize && BUGGIFY ) { FASTRESTORE_SCHED_UPDATE_DELAY = deterministicRandom()->random01() * 2;}
|
||||
init( FASTRESTORE_SCHED_TARGET_CPU_PERCENT, 70 ); if( randomize && BUGGIFY ) { FASTRESTORE_SCHED_TARGET_CPU_PERCENT = deterministicRandom()->random01() * 100 + 50;} // simulate cpu usage can be larger than 100
|
||||
init( FASTRESTORE_SCHED_MAX_CPU_PERCENT, 90 ); if( randomize && BUGGIFY ) { FASTRESTORE_SCHED_MAX_CPU_PERCENT = FASTRESTORE_SCHED_TARGET_CPU_PERCENT + deterministicRandom()->random01() * 100;}
|
||||
init( FASTRESTORE_SCHED_INFLIGHT_LOAD_REQS, 50 ); if( randomize && BUGGIFY ) { FASTRESTORE_SCHED_INFLIGHT_LOAD_REQS = deterministicRandom()->random01() < 0.2 ? 1 : deterministicRandom()->random01() * 30 + 1;}
|
||||
init( FASTRESTORE_SCHED_INFLIGHT_SEND_REQS, 3 ); if( randomize && BUGGIFY ) { FASTRESTORE_SCHED_INFLIGHT_SEND_REQS = deterministicRandom()->random01() < 0.2 ? 1 : deterministicRandom()->random01() * 10 + 1;}
|
||||
init( FASTRESTORE_SCHED_LOAD_REQ_BATCHSIZE, 5 ); if( randomize && BUGGIFY ) { FASTRESTORE_SCHED_LOAD_REQ_BATCHSIZE = deterministicRandom()->random01() < 0.2 ? 1 : deterministicRandom()->random01() * 10 + 1;}
|
||||
init( FASTRESTORE_SCHED_INFLIGHT_SENDPARAM_THRESHOLD, 10 ); if( randomize && BUGGIFY ) { FASTRESTORE_SCHED_INFLIGHT_SENDPARAM_THRESHOLD = deterministicRandom()->random01() < 0.2 ? 1 : deterministicRandom()->random01() * 15 + 1;}
|
||||
init( FASTRESTORE_SCHED_SEND_FUTURE_VB_REQS_BATCH, 2 ); if( randomize && BUGGIFY ) { FASTRESTORE_SCHED_SEND_FUTURE_VB_REQS_BATCH = deterministicRandom()->random01() < 0.2 ? 1 : deterministicRandom()->random01() * 15 + 1;}
|
||||
init( FASTRESTORE_NUM_TRACE_EVENTS, 100 ); if( randomize && BUGGIFY ) { FASTRESTORE_NUM_TRACE_EVENTS = deterministicRandom()->random01() < 0.2 ? 1 : deterministicRandom()->random01() * 500 + 1;}
|
||||
init( FASTRESTORE_EXPENSIVE_VALIDATION, false ); if( randomize && BUGGIFY ) { FASTRESTORE_EXPENSIVE_VALIDATION = deterministicRandom()->random01() < 0.5 ? true : false;}
|
||||
|
||||
|
||||
init( REDWOOD_DEFAULT_PAGE_SIZE, 4096 );
|
||||
init( REDWOOD_KVSTORE_CONCURRENT_READS, 64 );
|
||||
|
@ -650,13 +672,18 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
|
|||
init( REDWOOD_REMAP_CLEANUP_WINDOW, 50 );
|
||||
init( REDWOOD_REMAP_CLEANUP_LAG, 0.1 );
|
||||
init( REDWOOD_LOGGING_INTERVAL, 5.0 );
|
||||
|
||||
|
||||
// Server request latency measurement
|
||||
init( LATENCY_SAMPLE_SIZE, 100000 );
|
||||
init( LATENCY_METRICS_LOGGING_INTERVAL, 60.0 );
|
||||
|
||||
// clang-format on
|
||||
|
||||
if(clientKnobs)
|
||||
clientKnobs->IS_ACCEPTABLE_DELAY = clientKnobs->IS_ACCEPTABLE_DELAY*std::min(MAX_READ_TRANSACTION_LIFE_VERSIONS, MAX_WRITE_TRANSACTION_LIFE_VERSIONS)/(5.0*VERSIONS_PER_SECOND);
|
||||
if (clientKnobs) {
|
||||
clientKnobs->IS_ACCEPTABLE_DELAY =
|
||||
clientKnobs->IS_ACCEPTABLE_DELAY *
|
||||
std::min(MAX_READ_TRANSACTION_LIFE_VERSIONS, MAX_WRITE_TRANSACTION_LIFE_VERSIONS) /
|
||||
(5.0 * VERSIONS_PER_SECOND);
|
||||
clientKnobs->INIT_MID_SHARD_BYTES = MIN_SHARD_BYTES;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -94,6 +94,11 @@ public:
|
|||
double PEEK_STATS_INTERVAL;
|
||||
double PEEK_STATS_SLOW_AMOUNT;
|
||||
double PEEK_STATS_SLOW_RATIO;
|
||||
double PUSH_RESET_INTERVAL;
|
||||
double PUSH_MAX_LATENCY;
|
||||
double PUSH_STATS_INTERVAL;
|
||||
double PUSH_STATS_SLOW_AMOUNT;
|
||||
double PUSH_STATS_SLOW_RATIO;
|
||||
|
||||
// Data distribution queue
|
||||
double HEALTH_POLL_TIME;
|
||||
|
@ -243,6 +248,10 @@ public:
|
|||
// KeyValueStoreMemory
|
||||
int64_t REPLACE_CONTENTS_BYTES;
|
||||
|
||||
// KeyValueStoreRocksDB
|
||||
int ROCKSDB_BACKGROUND_PARALLELISM;
|
||||
int64_t ROCKSDB_MEMTABLE_BYTES;
|
||||
|
||||
// Leader election
|
||||
int MAX_NOTIFICATIONS;
|
||||
int MIN_NOTIFICATIONS;
|
||||
|
@ -295,6 +304,7 @@ public:
|
|||
int PROXY_COMPUTE_BUCKETS;
|
||||
double PROXY_COMPUTE_GROWTH_RATE;
|
||||
int TXN_STATE_SEND_AMOUNT;
|
||||
double REPORT_TRANSACTION_COST_ESTIMATION_DELAY;
|
||||
|
||||
// Master Server
|
||||
double COMMIT_SLEEP_TIME;
|
||||
|
@ -359,6 +369,7 @@ public:
|
|||
int EXPECTED_TLOG_FITNESS;
|
||||
int EXPECTED_LOG_ROUTER_FITNESS;
|
||||
int EXPECTED_PROXY_FITNESS;
|
||||
int EXPECTED_GRV_PROXY_FITNESS;
|
||||
int EXPECTED_RESOLVER_FITNESS;
|
||||
double RECRUITMENT_TIMEOUT;
|
||||
int DBINFO_SEND_AMOUNT;
|
||||
|
@ -484,9 +495,9 @@ public:
|
|||
int BEHIND_CHECK_COUNT;
|
||||
int64_t BEHIND_CHECK_VERSIONS;
|
||||
double WAIT_METRICS_WRONG_SHARD_CHANCE;
|
||||
int64_t MIN_TAG_PAGES_READ_RATE;
|
||||
double READ_TAG_MEASUREMENT_INTERVAL;
|
||||
int64_t OPERATION_COST_BYTE_FACTOR;
|
||||
int64_t MIN_TAG_PAGES_RATE;
|
||||
double TAG_MEASUREMENT_INTERVAL;
|
||||
int64_t READ_COST_BYTE_FACTOR;
|
||||
bool PREFIX_COMPRESS_KVS_MEM_SNAPSHOTS;
|
||||
|
||||
//Wait Failure
|
||||
|
@ -546,7 +557,7 @@ public:
|
|||
// FASTRESTORE_VB_PARALLELISM is the number of concurrently running version batches
|
||||
int64_t FASTRESTORE_VB_PARALLELISM;
|
||||
int64_t FASTRESTORE_VB_MONITOR_DELAY; // How quickly monitor finished version batch
|
||||
int64_t FASTRESTORE_VB_LAUNCH_DELAY;
|
||||
double FASTRESTORE_VB_LAUNCH_DELAY;
|
||||
int64_t FASTRESTORE_ROLE_LOGGING_DELAY;
|
||||
int64_t FASTRESTORE_UPDATE_PROCESS_STATS_INTERVAL; // How quickly to update process metrics for restore
|
||||
int64_t FASTRESTORE_ATOMICOP_WEIGHT; // workload amplication factor for atomic op
|
||||
|
@ -571,6 +582,16 @@ public:
|
|||
bool FASTRESTORE_USE_RANGE_FILE; // use range file in backup
|
||||
bool FASTRESTORE_USE_LOG_FILE; // use log file in backup
|
||||
int64_t FASTRESTORE_SAMPLE_MSG_BYTES; // sample message desired size
|
||||
double FASTRESTORE_SCHED_UPDATE_DELAY; // delay in seconds in updating process metrics
|
||||
int FASTRESTORE_SCHED_TARGET_CPU_PERCENT; // release as many requests as possible when cpu usage is below the knob
|
||||
int FASTRESTORE_SCHED_MAX_CPU_PERCENT; // max cpu percent when scheduler shall not release non-urgent requests
|
||||
int FASTRESTORE_SCHED_INFLIGHT_LOAD_REQS; // number of inflight requests to load backup files
|
||||
int FASTRESTORE_SCHED_INFLIGHT_SEND_REQS; // number of inflight requests for loaders to send mutations to appliers
|
||||
int FASTRESTORE_SCHED_LOAD_REQ_BATCHSIZE; // number of load request to release at once
|
||||
int FASTRESTORE_SCHED_INFLIGHT_SENDPARAM_THRESHOLD; // we can send future VB requests if it is less than this knob
|
||||
int FASTRESTORE_SCHED_SEND_FUTURE_VB_REQS_BATCH; // number of future VB sendLoadingParam requests to process at once
|
||||
int FASTRESTORE_NUM_TRACE_EVENTS;
|
||||
bool FASTRESTORE_EXPENSIVE_VALIDATION; // when set true, performance will be heavily affected
|
||||
|
||||
int REDWOOD_DEFAULT_PAGE_SIZE; // Page size for new Redwood files
|
||||
int REDWOOD_KVSTORE_CONCURRENT_READS; // Max number of simultaneous point or range reads in progress.
|
||||
|
|
|
@ -37,12 +37,22 @@ struct DBCoreState;
|
|||
struct TLogSet;
|
||||
struct CoreTLogSet;
|
||||
|
||||
struct ConnectionResetInfo : public ReferenceCounted<ConnectionResetInfo> {
|
||||
double lastReset;
|
||||
Future<Void> resetCheck;
|
||||
int slowReplies;
|
||||
int fastReplies;
|
||||
|
||||
ConnectionResetInfo() : lastReset(now()), slowReplies(0), fastReplies(0), resetCheck(Void()) {}
|
||||
};
|
||||
|
||||
// The set of tLog servers, logRouters and backupWorkers for a log tag
|
||||
class LogSet : NonCopyable, public ReferenceCounted<LogSet> {
|
||||
public:
|
||||
std::vector<Reference<AsyncVar<OptionalInterface<TLogInterface>>>> logServers;
|
||||
std::vector<Reference<AsyncVar<OptionalInterface<TLogInterface>>>> logRouters;
|
||||
std::vector<Reference<AsyncVar<OptionalInterface<BackupInterface>>>> backupWorkers;
|
||||
std::vector<Reference<ConnectionResetInfo>> connectionResetTrackers;
|
||||
int32_t tLogWriteAntiQuorum;
|
||||
int32_t tLogReplicationFactor;
|
||||
std::vector< LocalityData > tLogLocalities; // Stores the localities of the log servers
|
||||
|
|
|
@ -141,7 +141,7 @@ ACTOR Future<Void> resetChecker( ILogSystem::ServerPeekCursor* self, NetworkAddr
|
|||
self->unknownReplies = 0;
|
||||
self->fastReplies = 0;
|
||||
wait(delay(SERVER_KNOBS->PEEK_STATS_INTERVAL));
|
||||
TraceEvent("SlowPeekStats").detail("SlowReplies", self->slowReplies).detail("FastReplies", self->fastReplies).detail("UnknownReplies", self->unknownReplies);
|
||||
TraceEvent("SlowPeekStats").detail("PeerAddress", addr).detail("SlowReplies", self->slowReplies).detail("FastReplies", self->fastReplies).detail("UnknownReplies", self->unknownReplies);
|
||||
if(self->slowReplies >= SERVER_KNOBS->PEEK_STATS_SLOW_AMOUNT && self->slowReplies/double(self->slowReplies+self->fastReplies) >= SERVER_KNOBS->PEEK_STATS_SLOW_RATIO) {
|
||||
FlowTransport::transport().resetConnection(addr);
|
||||
self->lastReset = now();
|
||||
|
|
|
@ -186,15 +186,17 @@ struct ReportRawCommittedVersionRequest {
|
|||
Version version;
|
||||
bool locked;
|
||||
Optional<Value> metadataVersion;
|
||||
Version minKnownCommittedVersion;
|
||||
|
||||
ReplyPromise<Void> reply;
|
||||
|
||||
ReportRawCommittedVersionRequest() : version(invalidVersion), locked(false) {}
|
||||
ReportRawCommittedVersionRequest(Version version, bool locked, Optional<Value> metadataVersion) : version(version), locked(locked), metadataVersion(metadataVersion) {}
|
||||
ReportRawCommittedVersionRequest() : version(invalidVersion), locked(false), minKnownCommittedVersion(0) {}
|
||||
ReportRawCommittedVersionRequest(Version version, bool locked, Optional<Value> metadataVersion, Version minKnownCommittedVersion)
|
||||
: version(version), locked(locked), metadataVersion(metadataVersion), minKnownCommittedVersion(minKnownCommittedVersion) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, version, locked, metadataVersion, reply);
|
||||
serializer(ar, version, locked, metadataVersion, minKnownCommittedVersion, reply);
|
||||
}
|
||||
};
|
||||
|
||||
|
|
|
@ -21,29 +21,26 @@
|
|||
#include <algorithm>
|
||||
#include <tuple>
|
||||
|
||||
#include <fdbclient/DatabaseContext.h>
|
||||
#include "fdbclient/Atomic.h"
|
||||
#include "fdbclient/DatabaseConfiguration.h"
|
||||
#include "fdbclient/FDBTypes.h"
|
||||
#include "fdbclient/KeyRangeMap.h"
|
||||
#include "fdbclient/Knobs.h"
|
||||
#include "fdbclient/MasterProxyInterface.h"
|
||||
#include "fdbclient/NativeAPI.actor.h"
|
||||
#include "fdbclient/Notified.h"
|
||||
#include "fdbclient/SystemData.h"
|
||||
#include "fdbrpc/sim_validation.h"
|
||||
#include "fdbrpc/Stats.h"
|
||||
#include "fdbserver/ApplyMetadataMutation.h"
|
||||
#include "fdbserver/ConflictSet.h"
|
||||
#include "fdbserver/DataDistributorInterface.h"
|
||||
#include "fdbserver/FDBExecHelper.actor.h"
|
||||
#include "fdbserver/IKeyValueStore.h"
|
||||
#include "fdbserver/Knobs.h"
|
||||
#include "fdbserver/LatencyBandConfig.h"
|
||||
#include "fdbserver/LogSystem.h"
|
||||
#include "fdbserver/LogSystemDiskQueueAdapter.h"
|
||||
#include "fdbserver/MasterInterface.h"
|
||||
#include "fdbserver/MutationTracking.h"
|
||||
#include "fdbserver/ProxyCommitData.actor.h"
|
||||
#include "fdbserver/RatekeeperInterface.h"
|
||||
#include "fdbserver/RecoveryState.h"
|
||||
#include "fdbserver/ServerDBInfo.h"
|
||||
#include "fdbserver/WaitFailure.h"
|
||||
|
@ -51,7 +48,7 @@
|
|||
#include "flow/ActorCollection.h"
|
||||
#include "flow/IRandom.h"
|
||||
#include "flow/Knobs.h"
|
||||
#include "flow/TDMetric.actor.h"
|
||||
#include "flow/Trace.h"
|
||||
#include "flow/Tracing.h"
|
||||
|
||||
#include "flow/actorcompiler.h" // This must be the last #include.
|
||||
|
@ -79,227 +76,6 @@ ACTOR Future<Void> broadcastTxnRequest(TxnStateRequest req, int sendAmount, bool
|
|||
return Void();
|
||||
}
|
||||
|
||||
struct TransactionRateInfo {
|
||||
double rate;
|
||||
double limit;
|
||||
double budget;
|
||||
|
||||
bool disabled;
|
||||
|
||||
Smoother smoothRate;
|
||||
Smoother smoothReleased;
|
||||
|
||||
TransactionRateInfo(double rate) : rate(rate), limit(0), budget(0), disabled(true), smoothRate(SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW),
|
||||
smoothReleased(SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW) {}
|
||||
|
||||
void reset() {
|
||||
// Determine the number of transactions that this proxy is allowed to release
|
||||
// Roughly speaking, this is done by computing the number of transactions over some historical window that we could
|
||||
// have started but didn't, and making that our limit. More precisely, we track a smoothed rate limit and release rate,
|
||||
// the difference of which is the rate of additional transactions that we could have released based on that window.
|
||||
// Then we multiply by the window size to get a number of transactions.
|
||||
//
|
||||
// Limit can be negative in the event that we are releasing more transactions than we are allowed (due to the use of
|
||||
// our budget or because of higher priority transactions).
|
||||
double releaseRate = smoothRate.smoothTotal() - smoothReleased.smoothRate();
|
||||
limit = SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW * releaseRate;
|
||||
}
|
||||
|
||||
bool canStart(int64_t numAlreadyStarted, int64_t count) {
|
||||
return numAlreadyStarted + count <= std::min(limit + budget, SERVER_KNOBS->START_TRANSACTION_MAX_TRANSACTIONS_TO_START);
|
||||
}
|
||||
|
||||
void updateBudget(int64_t numStartedAtPriority, bool queueEmptyAtPriority, double elapsed) {
|
||||
// Update the budget to accumulate any extra capacity available or remove any excess that was used.
|
||||
// The actual delta is the portion of the limit we didn't use multiplied by the fraction of the window that elapsed.
|
||||
//
|
||||
// We may have exceeded our limit due to the budget or because of higher priority transactions, in which case this
|
||||
// delta will be negative. The delta can also be negative in the event that our limit was negative, which can happen
|
||||
// if we had already started more transactions in our window than our rate would have allowed.
|
||||
//
|
||||
// This budget has the property that when the budget is required to start transactions (because batches are big),
|
||||
// the sum limit+budget will increase linearly from 0 to the batch size over time and decrease by the batch size
|
||||
// upon starting a batch. In other words, this works equivalently to a model where we linearly accumulate budget over
|
||||
// time in the case that our batches are too big to take advantage of the window based limits.
|
||||
budget = std::max(0.0, budget + elapsed * (limit - numStartedAtPriority) / SERVER_KNOBS->START_TRANSACTION_RATE_WINDOW);
|
||||
|
||||
// If we are emptying out the queue of requests, then we don't need to carry much budget forward
|
||||
// If we did keep accumulating budget, then our responsiveness to changes in workflow could be compromised
|
||||
if(queueEmptyAtPriority) {
|
||||
budget = std::min(budget, SERVER_KNOBS->START_TRANSACTION_MAX_EMPTY_QUEUE_BUDGET);
|
||||
}
|
||||
|
||||
smoothReleased.addDelta(numStartedAtPriority);
|
||||
}
|
||||
|
||||
void disable() {
|
||||
disabled = true;
|
||||
rate = 0;
|
||||
smoothRate.reset(0);
|
||||
}
|
||||
|
||||
void setRate(double rate) {
|
||||
ASSERT(rate >= 0 && rate != std::numeric_limits<double>::infinity() && !std::isnan(rate));
|
||||
|
||||
this->rate = rate;
|
||||
if(disabled) {
|
||||
smoothRate.reset(rate);
|
||||
disabled = false;
|
||||
}
|
||||
else {
|
||||
smoothRate.setTotal(rate);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
ACTOR Future<Void> getRate(UID myID, Reference<AsyncVar<ServerDBInfo>> db, int64_t* inTransactionCount,
|
||||
int64_t* inBatchTransactionCount, TransactionRateInfo* transactionRateInfo,
|
||||
TransactionRateInfo* batchTransactionRateInfo, GetHealthMetricsReply* healthMetricsReply,
|
||||
GetHealthMetricsReply* detailedHealthMetricsReply,
|
||||
TransactionTagMap<uint64_t>* transactionTagCounter,
|
||||
PrioritizedTransactionTagMap<ClientTagThrottleLimits>* throttledTags,
|
||||
TransactionTagMap<TransactionCommitCostEstimation>* transactionTagCommitCostEst) {
|
||||
state Future<Void> nextRequestTimer = Never();
|
||||
state Future<Void> leaseTimeout = Never();
|
||||
state Future<GetRateInfoReply> reply = Never();
|
||||
state double lastDetailedReply = 0.0; // request detailed metrics immediately
|
||||
state bool expectingDetailedReply = false;
|
||||
state int64_t lastTC = 0;
|
||||
|
||||
if (db->get().ratekeeper.present()) nextRequestTimer = Void();
|
||||
loop choose {
|
||||
when ( wait( db->onChange() ) ) {
|
||||
if ( db->get().ratekeeper.present() ) {
|
||||
TraceEvent("ProxyRatekeeperChanged", myID)
|
||||
.detail("RKID", db->get().ratekeeper.get().id());
|
||||
nextRequestTimer = Void(); // trigger GetRate request
|
||||
} else {
|
||||
TraceEvent("ProxyRatekeeperDied", myID);
|
||||
nextRequestTimer = Never();
|
||||
reply = Never();
|
||||
}
|
||||
}
|
||||
when ( wait( nextRequestTimer ) ) {
|
||||
nextRequestTimer = Never();
|
||||
bool detailed = now() - lastDetailedReply > SERVER_KNOBS->DETAILED_METRIC_UPDATE_RATE;
|
||||
|
||||
TransactionTagMap<uint64_t> tagCounts;
|
||||
for(auto itr : *throttledTags) {
|
||||
for(auto priorityThrottles : itr.second) {
|
||||
tagCounts[priorityThrottles.first] = (*transactionTagCounter)[priorityThrottles.first];
|
||||
}
|
||||
}
|
||||
reply = brokenPromiseToNever(db->get().ratekeeper.get().getRateInfo.getReply(
|
||||
GetRateInfoRequest(myID, *inTransactionCount, *inBatchTransactionCount, tagCounts,
|
||||
*transactionTagCommitCostEst, detailed)));
|
||||
transactionTagCounter->clear();
|
||||
transactionTagCommitCostEst->clear();
|
||||
expectingDetailedReply = detailed;
|
||||
}
|
||||
when ( GetRateInfoReply rep = wait(reply) ) {
|
||||
reply = Never();
|
||||
|
||||
transactionRateInfo->setRate(rep.transactionRate);
|
||||
batchTransactionRateInfo->setRate(rep.batchTransactionRate);
|
||||
//TraceEvent("MasterProxyRate", myID).detail("Rate", rep.transactionRate).detail("BatchRate", rep.batchTransactionRate).detail("Lease", rep.leaseDuration).detail("ReleasedTransactions", *inTransactionCount - lastTC);
|
||||
lastTC = *inTransactionCount;
|
||||
leaseTimeout = delay(rep.leaseDuration);
|
||||
nextRequestTimer = delayJittered(rep.leaseDuration / 2);
|
||||
healthMetricsReply->update(rep.healthMetrics, expectingDetailedReply, true);
|
||||
if (expectingDetailedReply) {
|
||||
detailedHealthMetricsReply->update(rep.healthMetrics, true, true);
|
||||
lastDetailedReply = now();
|
||||
}
|
||||
|
||||
// Replace our throttles with what was sent by ratekeeper. Because we do this,
|
||||
// we are not required to expire tags out of the map
|
||||
if(rep.throttledTags.present()) {
|
||||
*throttledTags = std::move(rep.throttledTags.get());
|
||||
}
|
||||
}
|
||||
when ( wait( leaseTimeout ) ) {
|
||||
transactionRateInfo->disable();
|
||||
batchTransactionRateInfo->disable();
|
||||
TraceEvent(SevWarn, "MasterProxyRateLeaseExpired", myID).suppressFor(5.0);
|
||||
//TraceEvent("MasterProxyRate", myID).detail("Rate", 0.0).detail("BatchRate", 0.0).detail("Lease", 0);
|
||||
leaseTimeout = Never();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> queueTransactionStartRequests(
|
||||
Reference<AsyncVar<ServerDBInfo>> db,
|
||||
SpannedDeque<GetReadVersionRequest> *systemQueue,
|
||||
SpannedDeque<GetReadVersionRequest> *defaultQueue,
|
||||
SpannedDeque<GetReadVersionRequest> *batchQueue,
|
||||
FutureStream<GetReadVersionRequest> readVersionRequests,
|
||||
PromiseStream<Void> GRVTimer, double *lastGRVTime,
|
||||
double *GRVBatchTime, FutureStream<double> replyTimes,
|
||||
ProxyStats* stats, TransactionRateInfo* batchRateInfo,
|
||||
TransactionTagMap<uint64_t>* transactionTagCounter)
|
||||
{
|
||||
loop choose{
|
||||
when(GetReadVersionRequest req = waitNext(readVersionRequests)) {
|
||||
//WARNING: this code is run at a high priority, so it needs to do as little work as possible
|
||||
stats->addRequest();
|
||||
if( stats->txnRequestIn.getValue() - stats->txnRequestOut.getValue() > SERVER_KNOBS->START_TRANSACTION_MAX_QUEUE_SIZE ) {
|
||||
++stats->txnRequestErrors;
|
||||
//FIXME: send an error instead of giving an unreadable version when the client can support the error: req.reply.sendError(proxy_memory_limit_exceeded());
|
||||
GetReadVersionReply rep;
|
||||
rep.version = 1;
|
||||
rep.locked = true;
|
||||
req.reply.send(rep);
|
||||
TraceEvent(SevWarnAlways, "ProxyGRVThresholdExceeded").suppressFor(60);
|
||||
} else {
|
||||
// TODO: check whether this is reasonable to do in the fast path
|
||||
for(auto tag : req.tags) {
|
||||
(*transactionTagCounter)[tag.first] += tag.second;
|
||||
}
|
||||
|
||||
if (req.debugID.present())
|
||||
g_traceBatch.addEvent("TransactionDebug", req.debugID.get().first(), "MasterProxyServer.queueTransactionStartRequests.Before");
|
||||
|
||||
if (systemQueue->empty() && defaultQueue->empty() && batchQueue->empty()) {
|
||||
forwardPromise(GRVTimer, delayJittered(std::max(0.0, *GRVBatchTime - (now() - *lastGRVTime)), TaskPriority::ProxyGRVTimer));
|
||||
}
|
||||
|
||||
++stats->txnRequestIn;
|
||||
stats->txnStartIn += req.transactionCount;
|
||||
if (req.priority >= TransactionPriority::IMMEDIATE) {
|
||||
stats->txnSystemPriorityStartIn += req.transactionCount;
|
||||
systemQueue->push_back(req);
|
||||
systemQueue->span.addParent(req.spanContext);
|
||||
} else if (req.priority >= TransactionPriority::DEFAULT) {
|
||||
stats->txnDefaultPriorityStartIn += req.transactionCount;
|
||||
defaultQueue->push_back(req);
|
||||
defaultQueue->span.addParent(req.spanContext);
|
||||
} else {
|
||||
// Return error for batch_priority GRV requests
|
||||
int64_t proxiesCount = std::max((int)db->get().client.proxies.size(), 1);
|
||||
if (batchRateInfo->rate <= (1.0 / proxiesCount)) {
|
||||
req.reply.sendError(batch_transaction_throttled());
|
||||
stats->txnThrottled += req.transactionCount;
|
||||
continue;
|
||||
}
|
||||
|
||||
stats->txnBatchPriorityStartIn += req.transactionCount;
|
||||
batchQueue->push_back(req);
|
||||
batchQueue->span.addParent(req.spanContext);
|
||||
}
|
||||
}
|
||||
}
|
||||
// dynamic batching monitors reply latencies
|
||||
when(double reply_latency = waitNext(replyTimes)) {
|
||||
double target_latency = reply_latency * SERVER_KNOBS->START_TRANSACTION_BATCH_INTERVAL_LATENCY_FRACTION;
|
||||
*GRVBatchTime = std::max(
|
||||
SERVER_KNOBS->START_TRANSACTION_BATCH_INTERVAL_MIN,
|
||||
std::min(SERVER_KNOBS->START_TRANSACTION_BATCH_INTERVAL_MAX,
|
||||
target_latency * SERVER_KNOBS->START_TRANSACTION_BATCH_INTERVAL_SMOOTHER_ALPHA +
|
||||
*GRVBatchTime * (1 - SERVER_KNOBS->START_TRANSACTION_BATCH_INTERVAL_SMOOTHER_ALPHA)));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR void discardCommit(UID id, Future<LogSystemDiskQueueAdapter::CommitMessage> fcm, Future<Void> dummyCommitState) {
|
||||
ASSERT(!dummyCommitState.isReady());
|
||||
LogSystemDiskQueueAdapter::CommitMessage cm = wait(fcm);
|
||||
|
@ -340,7 +116,8 @@ struct ResolutionRequestBuilder {
|
|||
return *out;
|
||||
}
|
||||
|
||||
void addTransaction(CommitTransactionRef& trIn, int transactionNumberInBatch) {
|
||||
void addTransaction(CommitTransactionRequest& trRequest, int transactionNumberInBatch) {
|
||||
auto& trIn = trRequest.transaction;
|
||||
// SOMEDAY: There are a couple of unnecessary O( # resolvers ) steps here
|
||||
outTr.assign(requests.size(), NULL);
|
||||
ASSERT( transactionNumberInBatch >= 0 && transactionNumberInBatch < 32768 );
|
||||
|
@ -358,6 +135,13 @@ struct ResolutionRequestBuilder {
|
|||
getOutTransaction(0, trIn.read_snapshot).mutations.push_back(requests[0].arena, m);
|
||||
}
|
||||
}
|
||||
if (isTXNStateTransaction && !trRequest.isLockAware()) {
|
||||
// This mitigates https://github.com/apple/foundationdb/issues/3647. Since this transaction is not lock
|
||||
// aware, if this transaction got a read version then \xff/dbLocked must not have been set at this
|
||||
// transaction's read snapshot. If that changes by commit time, then it won't commit on any proxy because of
|
||||
// a conflict. A client could set a read version manually so this isn't totally bulletproof.
|
||||
trIn.read_conflict_ranges.push_back(trRequest.arena, KeyRangeRef(databaseLockedKey, databaseLockedKeyEnd));
|
||||
}
|
||||
std::vector<std::vector<int>> rCRIndexMap(
|
||||
requests.size()); // [resolver_index][read_conflict_range_index_on_the_resolver]
|
||||
// -> read_conflict_range's original index
|
||||
|
@ -426,7 +210,6 @@ ACTOR Future<Void> commitBatcher(ProxyCommitData *commitData, PromiseStream<std:
|
|||
choose{
|
||||
when(CommitTransactionRequest req = waitNext(in)) {
|
||||
//WARNING: this code is run at a high priority, so it needs to do as little work as possible
|
||||
commitData->stats.addRequest();
|
||||
int bytes = getBytes(req);
|
||||
|
||||
// Drop requests if memory is under severe pressure
|
||||
|
@ -819,7 +602,7 @@ ACTOR Future<Void> getResolution(CommitBatchContext* self) {
|
|||
int conflictRangeCount = 0;
|
||||
self->maxTransactionBytes = 0;
|
||||
for (int t = 0; t < trs.size(); t++) {
|
||||
requests.addTransaction(trs[t].transaction, t);
|
||||
requests.addTransaction(trs[t], t);
|
||||
conflictRangeCount +=
|
||||
trs[t].transaction.read_conflict_ranges.size() + trs[t].transaction.write_conflict_ranges.size();
|
||||
//TraceEvent("MPTransactionDump", self->dbgid).detail("Snapshot", trs[t].transaction.read_snapshot);
|
||||
|
@ -1016,6 +799,8 @@ ACTOR Future<Void> assignMutationsToStorageServers(CommitBatchContext* self) {
|
|||
continue;
|
||||
}
|
||||
|
||||
state bool checkSample = trs[self->transactionNum].commitCostEstimation.present();
|
||||
state Optional<ClientTrCommitCostEstimation>* trCost = &trs[self->transactionNum].commitCostEstimation;
|
||||
state int mutationNum = 0;
|
||||
state VectorRef<MutationRef>* pMutations = &trs[self->transactionNum].transaction.mutations;
|
||||
for (; mutationNum < pMutations->size(); mutationNum++) {
|
||||
|
@ -1038,6 +823,25 @@ ACTOR Future<Void> assignMutationsToStorageServers(CommitBatchContext* self) {
|
|||
if (isSingleKeyMutation((MutationRef::Type) m.type)) {
|
||||
auto& tags = pProxyCommitData->tagsForKey(m.param1);
|
||||
|
||||
// sample single key mutation based on cost
|
||||
// the expectation of sampling is every COMMIT_SAMPLE_COST sample once
|
||||
if (checkSample) {
|
||||
double totalCosts = trCost->get().writeCosts;
|
||||
double cost = getWriteOperationCost(m.expectedSize());
|
||||
double mul = std::max(1.0, totalCosts / std::max(1.0, (double)CLIENT_KNOBS->COMMIT_SAMPLE_COST));
|
||||
ASSERT(totalCosts > 0);
|
||||
double prob = mul * cost / totalCosts;
|
||||
|
||||
if (deterministicRandom()->random01() < prob) {
|
||||
for (const auto& ssInfo : pProxyCommitData->keyInfo[m.param1].src_info) {
|
||||
auto id = ssInfo->interf.id();
|
||||
// scale cost
|
||||
cost = cost < CLIENT_KNOBS->COMMIT_SAMPLE_COST ? CLIENT_KNOBS->COMMIT_SAMPLE_COST : cost;
|
||||
pProxyCommitData->updateSSTagCost(id, trs[self->transactionNum].tagSet.get(), m, cost);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if(pProxyCommitData->singleKeyMutationEvent->enabled) {
|
||||
KeyRangeRef shard = pProxyCommitData->keyInfo.rangeContaining(m.param1).range();
|
||||
pProxyCommitData->singleKeyMutationEvent->tag1 = (int64_t)tags[0].id;
|
||||
|
@ -1066,6 +870,17 @@ ACTOR Future<Void> assignMutationsToStorageServers(CommitBatchContext* self) {
|
|||
|
||||
ranges.begin().value().populateTags();
|
||||
self->toCommit.addTags(ranges.begin().value().tags);
|
||||
|
||||
// check whether clear is sampled
|
||||
if (checkSample && !trCost->get().clearIdxCosts.empty() &&
|
||||
trCost->get().clearIdxCosts[0].first == mutationNum) {
|
||||
for (const auto& ssInfo : ranges.begin().value().src_info) {
|
||||
auto id = ssInfo->interf.id();
|
||||
pProxyCommitData->updateSSTagCost(id, trs[self->transactionNum].tagSet.get(), m,
|
||||
trCost->get().clearIdxCosts[0].second);
|
||||
}
|
||||
trCost->get().clearIdxCosts.pop_front();
|
||||
}
|
||||
}
|
||||
else {
|
||||
TEST(true); //A clear range extends past a shard boundary
|
||||
|
@ -1073,6 +888,17 @@ ACTOR Future<Void> assignMutationsToStorageServers(CommitBatchContext* self) {
|
|||
for (auto r : ranges) {
|
||||
r.value().populateTags();
|
||||
allSources.insert(r.value().tags.begin(), r.value().tags.end());
|
||||
|
||||
// check whether clear is sampled
|
||||
if (checkSample && !trCost->get().clearIdxCosts.empty() &&
|
||||
trCost->get().clearIdxCosts[0].first == mutationNum) {
|
||||
for (const auto& ssInfo : r.value().src_info) {
|
||||
auto id = ssInfo->interf.id();
|
||||
pProxyCommitData->updateSSTagCost(id, trs[self->transactionNum].tagSet.get(), m,
|
||||
trCost->get().clearIdxCosts[0].second);
|
||||
}
|
||||
trCost->get().clearIdxCosts.pop_front();
|
||||
}
|
||||
}
|
||||
DEBUG_MUTATION("ProxyCommit", self->commitVersion, m).detail("Dbgid", pProxyCommitData->dbgid).detail("To", allSources).detail("Mutation", m);
|
||||
|
||||
|
@ -1121,6 +947,11 @@ ACTOR Future<Void> assignMutationsToStorageServers(CommitBatchContext* self) {
|
|||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (checkSample) {
|
||||
self->pProxyCommitData->stats.txnExpensiveClearCostEstCount +=
|
||||
trs[self->transactionNum].commitCostEstimation.get().expensiveCostEstCount;
|
||||
}
|
||||
}
|
||||
|
||||
return Void();
|
||||
|
@ -1189,9 +1020,9 @@ ACTOR Future<Void> postResolution(CommitBatchContext* self) {
|
|||
wait(yield());
|
||||
break;
|
||||
}
|
||||
when(GetReadVersionReply v = wait(pProxyCommitData->getConsistentReadVersion.getReply(
|
||||
GetReadVersionRequest(waitVersionSpan.context, 0, TransactionPriority::IMMEDIATE,
|
||||
GetReadVersionRequest::FLAG_CAUSAL_READ_RISKY)))) {
|
||||
when(wait(pProxyCommitData->cx->onProxiesChanged())) {}
|
||||
when(GetRawCommittedVersionReply v = wait(pProxyCommitData->master.getLiveCommittedVersion.getReply(
|
||||
GetRawCommittedVersionRequest(waitVersionSpan.context, debugID), TaskPriority::GetLiveCommittedVersionReply))) {
|
||||
if(v.version > pProxyCommitData->committedVersion.get()) {
|
||||
pProxyCommitData->locked = v.locked;
|
||||
pProxyCommitData->metadataVersion = v.metadataVersion;
|
||||
|
@ -1311,12 +1142,11 @@ ACTOR Future<Void> reply(CommitBatchContext* self) {
|
|||
// by reporting commit version first before updating self->committedVersion. Otherwise, a client may get a commit
|
||||
// version that the master is not aware of, and next GRV request may get a version less than self->committedVersion.
|
||||
TEST(pProxyCommitData->committedVersion.get() > self->commitVersion); // A later version was reported committed first
|
||||
if ( self->commitVersion > pProxyCommitData->committedVersion.get()) {
|
||||
if (self->commitVersion >= pProxyCommitData->committedVersion.get()) {
|
||||
wait(pProxyCommitData->master.reportLiveCommittedVersion.getReply(
|
||||
ReportRawCommittedVersionRequest(
|
||||
self->commitVersion,
|
||||
self->lockedAfter,
|
||||
self->metadataVersionAfter), TaskPriority::ProxyMasterVersionReply));
|
||||
ReportRawCommittedVersionRequest(self->commitVersion, self->lockedAfter, self->metadataVersionAfter,
|
||||
pProxyCommitData->minKnownCommittedVersion),
|
||||
TaskPriority::ProxyMasterVersionReply));
|
||||
}
|
||||
if( self->commitVersion > pProxyCommitData->committedVersion.get() ) {
|
||||
pProxyCommitData->locked = self->lockedAfter;
|
||||
|
@ -1339,15 +1169,6 @@ ACTOR Future<Void> reply(CommitBatchContext* self) {
|
|||
if (self->committed[t] == ConflictBatch::TransactionCommitted && (!self->locked || tr.isLockAware())) {
|
||||
ASSERT_WE_THINK(self->commitVersion != invalidVersion);
|
||||
tr.reply.send(CommitID(self->commitVersion, t, self->metadataVersionAfter));
|
||||
|
||||
// aggregate commit cost estimation if committed
|
||||
ASSERT(tr.commitCostEstimation.present() == tr.tagSet.present());
|
||||
if (tr.tagSet.present()) {
|
||||
TransactionCommitCostEstimation& costEstimation = tr.commitCostEstimation.get();
|
||||
for (auto& tag : tr.tagSet.get()) {
|
||||
pProxyCommitData->transactionTagCommitCostEst[tag] += costEstimation;
|
||||
}
|
||||
}
|
||||
}
|
||||
else if (self->committed[t] == ConflictBatch::TransactionTooOld) {
|
||||
tr.reply.sendError(transaction_too_old());
|
||||
|
@ -1440,6 +1261,7 @@ ACTOR Future<Void> commitBatch(
|
|||
|
||||
context.pProxyCommitData->lastVersionTime = context.startTime;
|
||||
++context.pProxyCommitData->stats.commitBatchIn;
|
||||
context.setupTraceBatch();
|
||||
|
||||
/////// Phase 1: Pre-resolution processing (CPU bound except waiting for a version # which is separately pipelined and *should* be available by now (unless empty commit); ordered; currently atomic but could yield)
|
||||
wait(CommitBatch::preresolutionProcessing(&context));
|
||||
|
@ -1459,271 +1281,6 @@ ACTOR Future<Void> commitBatch(
|
|||
return Void();
|
||||
}
|
||||
|
||||
ACTOR Future<Void> updateLastCommit(ProxyCommitData* self, Optional<UID> debugID = Optional<UID>()) {
|
||||
state double confirmStart = now();
|
||||
self->lastStartCommit = confirmStart;
|
||||
self->updateCommitRequests++;
|
||||
wait(self->logSystem->confirmEpochLive(debugID));
|
||||
self->updateCommitRequests--;
|
||||
self->lastCommitLatency = now()-confirmStart;
|
||||
self->lastCommitTime = std::max(self->lastCommitTime.get(), confirmStart);
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR Future<GetReadVersionReply> getLiveCommittedVersion(SpanID parentSpan, ProxyCommitData* commitData, uint32_t flags, Optional<UID> debugID,
|
||||
int transactionCount, int systemTransactionCount, int defaultPriTransactionCount, int batchPriTransactionCount)
|
||||
{
|
||||
// Returns a version which (1) is committed, and (2) is >= the latest version reported committed (by a commit response) when this request was sent
|
||||
// (1) The version returned is the committedVersion of some proxy at some point before the request returns, so it is committed.
|
||||
// (2) No proxy on our list reported committed a higher version before this request was received, because then its committedVersion would have been higher,
|
||||
// and no other proxy could have already committed anything without first ending the epoch
|
||||
state Span span("MP:getLiveCommittedVersion"_loc, parentSpan);
|
||||
++commitData->stats.txnStartBatch;
|
||||
state Future<GetReadVersionReply> replyFromMasterFuture = commitData->master.getLiveCommittedVersion.getReply(
|
||||
GetRawCommittedVersionRequest(span.context, debugID), TaskPriority::GetLiveCommittedVersionReply);
|
||||
|
||||
if (!SERVER_KNOBS->ALWAYS_CAUSAL_READ_RISKY && !(flags&GetReadVersionRequest::FLAG_CAUSAL_READ_RISKY)) {
|
||||
wait(updateLastCommit(commitData, debugID));
|
||||
} else if (SERVER_KNOBS->REQUIRED_MIN_RECOVERY_DURATION > 0 && now() - SERVER_KNOBS->REQUIRED_MIN_RECOVERY_DURATION > commitData->lastCommitTime.get()) {
|
||||
wait(commitData->lastCommitTime.whenAtLeast(now() - SERVER_KNOBS->REQUIRED_MIN_RECOVERY_DURATION));
|
||||
}
|
||||
|
||||
if (debugID.present()) {
|
||||
g_traceBatch.addEvent("TransactionDebug", debugID.get().first(), "MasterProxyServer.getLiveCommittedVersion.confirmEpochLive");
|
||||
}
|
||||
|
||||
state GetReadVersionReply rep;
|
||||
rep.locked = commitData->locked;
|
||||
rep.metadataVersion = commitData->metadataVersion;
|
||||
rep.version = commitData->committedVersion.get();
|
||||
|
||||
GetReadVersionReply replyFromMaster = wait(replyFromMasterFuture);
|
||||
if (replyFromMaster.version > rep.version) {
|
||||
rep = replyFromMaster;
|
||||
}
|
||||
rep.recentRequests = commitData->stats.getRecentRequests();
|
||||
|
||||
if (debugID.present()) {
|
||||
g_traceBatch.addEvent("TransactionDebug", debugID.get().first(), "MasterProxyServer.getLiveCommittedVersion.After");
|
||||
}
|
||||
|
||||
commitData->stats.txnStartOut += transactionCount;
|
||||
commitData->stats.txnSystemPriorityStartOut += systemTransactionCount;
|
||||
commitData->stats.txnDefaultPriorityStartOut += defaultPriTransactionCount;
|
||||
commitData->stats.txnBatchPriorityStartOut += batchPriTransactionCount;
|
||||
|
||||
return rep;
|
||||
}
|
||||
|
||||
ACTOR Future<Void> sendGrvReplies(Future<GetReadVersionReply> replyFuture, std::vector<GetReadVersionRequest> requests,
|
||||
ProxyStats* stats, Version minKnownCommittedVersion, PrioritizedTransactionTagMap<ClientTagThrottleLimits> throttledTags) {
|
||||
GetReadVersionReply _reply = wait(replyFuture);
|
||||
GetReadVersionReply reply = _reply;
|
||||
Version replyVersion = reply.version;
|
||||
|
||||
double end = g_network->timer();
|
||||
for(GetReadVersionRequest const& request : requests) {
|
||||
double duration = end - request.requestTime();
|
||||
if(request.priority == TransactionPriority::DEFAULT) {
|
||||
stats->grvLatencySample.addMeasurement(duration);
|
||||
}
|
||||
if(request.priority >= TransactionPriority::DEFAULT) {
|
||||
stats->grvLatencyBands.addMeasurement(duration);
|
||||
}
|
||||
|
||||
if (request.flags & GetReadVersionRequest::FLAG_USE_MIN_KNOWN_COMMITTED_VERSION) {
|
||||
// Only backup worker may infrequently use this flag.
|
||||
reply.version = minKnownCommittedVersion;
|
||||
}
|
||||
else {
|
||||
reply.version = replyVersion;
|
||||
}
|
||||
|
||||
reply.tagThrottleInfo.clear();
|
||||
|
||||
if(!request.tags.empty()) {
|
||||
auto& priorityThrottledTags = throttledTags[request.priority];
|
||||
for(auto tag : request.tags) {
|
||||
auto tagItr = priorityThrottledTags.find(tag.first);
|
||||
if(tagItr != priorityThrottledTags.end()) {
|
||||
if(tagItr->second.expiration > now()) {
|
||||
if(tagItr->second.tpsRate == std::numeric_limits<double>::max()) {
|
||||
TEST(true); // Auto TPS rate is unlimited
|
||||
}
|
||||
else {
|
||||
TEST(true); // Proxy returning tag throttle
|
||||
reply.tagThrottleInfo[tag.first] = tagItr->second;
|
||||
}
|
||||
}
|
||||
else {
|
||||
// This isn't required, but we might as well
|
||||
TEST(true); // Proxy expiring tag throttle
|
||||
priorityThrottledTags.erase(tagItr);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
request.reply.send(reply);
|
||||
++stats->txnRequestOut;
|
||||
}
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> transactionStarter(
|
||||
MasterProxyInterface proxy,
|
||||
Reference<AsyncVar<ServerDBInfo>> db,
|
||||
PromiseStream<Future<Void>> addActor,
|
||||
ProxyCommitData* commitData, GetHealthMetricsReply* healthMetricsReply,
|
||||
GetHealthMetricsReply* detailedHealthMetricsReply)
|
||||
{
|
||||
state double lastGRVTime = 0;
|
||||
state PromiseStream<Void> GRVTimer;
|
||||
state double GRVBatchTime = SERVER_KNOBS->START_TRANSACTION_BATCH_INTERVAL_MIN;
|
||||
|
||||
state int64_t transactionCount = 0;
|
||||
state int64_t batchTransactionCount = 0;
|
||||
state TransactionRateInfo normalRateInfo(10);
|
||||
state TransactionRateInfo batchRateInfo(0);
|
||||
|
||||
state SpannedDeque<GetReadVersionRequest> systemQueue("MP:transactionStarterSystemQueue"_loc);
|
||||
state SpannedDeque<GetReadVersionRequest> defaultQueue("MP:transactionStarterDefaultQueue"_loc);
|
||||
state SpannedDeque<GetReadVersionRequest> batchQueue("MP:transactionStarterBatchQueue"_loc);
|
||||
|
||||
state TransactionTagMap<uint64_t> transactionTagCounter;
|
||||
state PrioritizedTransactionTagMap<ClientTagThrottleLimits> throttledTags;
|
||||
|
||||
state PromiseStream<double> replyTimes;
|
||||
state Span span;
|
||||
|
||||
addActor.send(getRate(proxy.id(), db, &transactionCount, &batchTransactionCount, &normalRateInfo, &batchRateInfo,
|
||||
healthMetricsReply, detailedHealthMetricsReply, &transactionTagCounter, &throttledTags,
|
||||
&(commitData->transactionTagCommitCostEst)));
|
||||
addActor.send(queueTransactionStartRequests(db, &systemQueue, &defaultQueue, &batchQueue, proxy.getConsistentReadVersion.getFuture(),
|
||||
GRVTimer, &lastGRVTime, &GRVBatchTime, replyTimes.getFuture(), &commitData->stats, &batchRateInfo,
|
||||
&transactionTagCounter));
|
||||
|
||||
// Get a list of the other proxies that go together with us
|
||||
while (std::find(db->get().client.proxies.begin(), db->get().client.proxies.end(), proxy) == db->get().client.proxies.end())
|
||||
wait(db->onChange());
|
||||
|
||||
ASSERT(db->get().recoveryState >= RecoveryState::ACCEPTING_COMMITS); // else potentially we could return uncommitted read versions (since self->committedVersion is only a committed version if this recovery succeeds)
|
||||
|
||||
TraceEvent("ProxyReadyForTxnStarts", proxy.id());
|
||||
|
||||
loop{
|
||||
waitNext(GRVTimer.getFuture());
|
||||
// Select zero or more transactions to start
|
||||
double t = now();
|
||||
double elapsed = now() - lastGRVTime;
|
||||
lastGRVTime = t;
|
||||
|
||||
if(elapsed == 0) elapsed = 1e-15; // resolve a possible indeterminant multiplication with infinite transaction rate
|
||||
|
||||
normalRateInfo.reset();
|
||||
batchRateInfo.reset();
|
||||
|
||||
int transactionsStarted[2] = {0,0};
|
||||
int systemTransactionsStarted[2] = {0,0};
|
||||
int defaultPriTransactionsStarted[2] = { 0, 0 };
|
||||
int batchPriTransactionsStarted[2] = { 0, 0 };
|
||||
|
||||
vector<vector<GetReadVersionRequest>> start(2); // start[0] is transactions starting with !(flags&CAUSAL_READ_RISKY), start[1] is transactions starting with flags&CAUSAL_READ_RISKY
|
||||
Optional<UID> debugID;
|
||||
|
||||
int requestsToStart = 0;
|
||||
|
||||
while (requestsToStart < SERVER_KNOBS->START_TRANSACTION_MAX_REQUESTS_TO_START) {
|
||||
SpannedDeque<GetReadVersionRequest>* transactionQueue;
|
||||
if(!systemQueue.empty()) {
|
||||
transactionQueue = &systemQueue;
|
||||
} else if(!defaultQueue.empty()) {
|
||||
transactionQueue = &defaultQueue;
|
||||
} else if(!batchQueue.empty()) {
|
||||
transactionQueue = &batchQueue;
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
transactionQueue->span.swap(span);
|
||||
|
||||
auto& req = transactionQueue->front();
|
||||
int tc = req.transactionCount;
|
||||
|
||||
if(req.priority < TransactionPriority::DEFAULT && !batchRateInfo.canStart(transactionsStarted[0] + transactionsStarted[1], tc)) {
|
||||
break;
|
||||
}
|
||||
else if(req.priority < TransactionPriority::IMMEDIATE && !normalRateInfo.canStart(transactionsStarted[0] + transactionsStarted[1], tc)) {
|
||||
break;
|
||||
}
|
||||
|
||||
if (req.debugID.present()) {
|
||||
if (!debugID.present()) debugID = nondeterministicRandom()->randomUniqueID();
|
||||
g_traceBatch.addAttach("TransactionAttachID", req.debugID.get().first(), debugID.get().first());
|
||||
}
|
||||
|
||||
transactionsStarted[req.flags&1] += tc;
|
||||
if (req.priority >= TransactionPriority::IMMEDIATE)
|
||||
systemTransactionsStarted[req.flags & 1] += tc;
|
||||
else if (req.priority >= TransactionPriority::DEFAULT)
|
||||
defaultPriTransactionsStarted[req.flags & 1] += tc;
|
||||
else
|
||||
batchPriTransactionsStarted[req.flags & 1] += tc;
|
||||
|
||||
start[req.flags & 1].emplace_back(std::move(req));
|
||||
static_assert(GetReadVersionRequest::FLAG_CAUSAL_READ_RISKY == 1, "Implementation dependent on flag value");
|
||||
transactionQueue->pop_front();
|
||||
requestsToStart++;
|
||||
}
|
||||
|
||||
if (!systemQueue.empty() || !defaultQueue.empty() || !batchQueue.empty()) {
|
||||
forwardPromise(GRVTimer, delayJittered(SERVER_KNOBS->START_TRANSACTION_BATCH_QUEUE_CHECK_INTERVAL, TaskPriority::ProxyGRVTimer));
|
||||
}
|
||||
|
||||
/*TraceEvent("GRVBatch", proxy.id())
|
||||
.detail("Elapsed", elapsed)
|
||||
.detail("NTransactionToStart", nTransactionsToStart)
|
||||
.detail("TransactionRate", transactionRate)
|
||||
.detail("TransactionQueueSize", transactionQueue.size())
|
||||
.detail("NumTransactionsStarted", transactionsStarted[0] + transactionsStarted[1])
|
||||
.detail("NumSystemTransactionsStarted", systemTransactionsStarted[0] + systemTransactionsStarted[1])
|
||||
.detail("NumNonSystemTransactionsStarted", transactionsStarted[0] + transactionsStarted[1] -
|
||||
systemTransactionsStarted[0] - systemTransactionsStarted[1])
|
||||
.detail("TransactionBudget", transactionBudget)
|
||||
.detail("BatchTransactionBudget", batchTransactionBudget);*/
|
||||
|
||||
int systemTotalStarted = systemTransactionsStarted[0] + systemTransactionsStarted[1];
|
||||
int normalTotalStarted = defaultPriTransactionsStarted[0] + defaultPriTransactionsStarted[1];
|
||||
int batchTotalStarted = batchPriTransactionsStarted[0] + batchPriTransactionsStarted[1];
|
||||
|
||||
transactionCount += transactionsStarted[0] + transactionsStarted[1];
|
||||
batchTransactionCount += batchTotalStarted;
|
||||
|
||||
normalRateInfo.updateBudget(systemTotalStarted + normalTotalStarted, systemQueue.empty() && defaultQueue.empty(), elapsed);
|
||||
batchRateInfo.updateBudget(systemTotalStarted + normalTotalStarted + batchTotalStarted, systemQueue.empty() && defaultQueue.empty() && batchQueue.empty(), elapsed);
|
||||
|
||||
if (debugID.present()) {
|
||||
g_traceBatch.addEvent("TransactionDebug", debugID.get().first(), "MasterProxyServer.masterProxyServerCore.Broadcast");
|
||||
}
|
||||
|
||||
for (int i = 0; i < start.size(); i++) {
|
||||
if (start[i].size()) {
|
||||
Future<GetReadVersionReply> readVersionReply = getLiveCommittedVersion(
|
||||
span.context, commitData, i, debugID, transactionsStarted[i], systemTransactionsStarted[i],
|
||||
defaultPriTransactionsStarted[i], batchPriTransactionsStarted[i]);
|
||||
addActor.send(sendGrvReplies(readVersionReply, start[i], &commitData->stats,
|
||||
commitData->minKnownCommittedVersion, throttledTags));
|
||||
|
||||
// for now, base dynamic batching on the time for normal requests (not read_risky)
|
||||
if (i == 0) {
|
||||
addActor.send(timeReply(readVersionReply, replyTimes));
|
||||
}
|
||||
}
|
||||
}
|
||||
span = Span(span.location);
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR static Future<Void> doKeyServerLocationRequest( GetKeyServerLocationsRequest req, ProxyCommitData* commitData ) {
|
||||
// We can't respond to these requests until we have valid txnStateStore
|
||||
wait(commitData->validState.getFuture());
|
||||
|
@ -1775,7 +1332,6 @@ ACTOR static Future<Void> readRequestServer( MasterProxyInterface proxy, Promise
|
|||
loop {
|
||||
GetKeyServerLocationsRequest req = waitNext(proxy.getKeyServersLocations.getFuture());
|
||||
//WARNING: this code is run at a high priority, so it needs to do as little work as possible
|
||||
commitData->stats.addRequest();
|
||||
if(req.limit != CLIENT_KNOBS->STORAGE_METRICS_SHARD_LIMIT && //Always do data distribution requests
|
||||
commitData->stats.keyServerLocationIn.getValue() - commitData->stats.keyServerLocationOut.getValue() > SERVER_KNOBS->KEY_LOCATION_MAX_QUEUE_SIZE) {
|
||||
++commitData->stats.keyServerLocationErrors;
|
||||
|
@ -1857,30 +1413,20 @@ ACTOR static Future<Void> rejoinServer( MasterProxyInterface proxy, ProxyCommitD
|
|||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> healthMetricsRequestServer(MasterProxyInterface proxy, GetHealthMetricsReply* healthMetricsReply, GetHealthMetricsReply* detailedHealthMetricsReply)
|
||||
{
|
||||
loop {
|
||||
choose {
|
||||
when(GetHealthMetricsRequest req =
|
||||
waitNext(proxy.getHealthMetrics.getFuture()))
|
||||
{
|
||||
if (req.detailed)
|
||||
req.reply.send(*detailedHealthMetricsReply);
|
||||
else
|
||||
req.reply.send(*healthMetricsReply);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> ddMetricsRequestServer(MasterProxyInterface proxy, Reference<AsyncVar<ServerDBInfo>> db)
|
||||
{
|
||||
loop {
|
||||
choose {
|
||||
when(state GetDDMetricsRequest req = waitNext(proxy.getDDMetrics.getFuture()))
|
||||
{
|
||||
ErrorOr<GetDataDistributorMetricsReply> reply = wait(errorOr(db->get().distributor.get().dataDistributorMetrics.getReply(GetDataDistributorMetricsRequest(req.keys, req.shardLimit))));
|
||||
if ( reply.isError() ) {
|
||||
if (!db->get().distributor.present()) {
|
||||
req.reply.sendError(dd_not_found());
|
||||
continue;
|
||||
}
|
||||
ErrorOr<GetDataDistributorMetricsReply> reply =
|
||||
wait(errorOr(db->get().distributor.get().dataDistributorMetrics.getReply(
|
||||
GetDataDistributorMetricsRequest(req.keys, req.shardLimit))));
|
||||
if (reply.isError()) {
|
||||
req.reply.sendError(reply.getError());
|
||||
} else {
|
||||
GetDDMetricsReply newReply;
|
||||
|
@ -1949,23 +1495,6 @@ ACTOR Future<Void> monitorRemoteCommitted(ProxyCommitData* self) {
|
|||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> lastCommitUpdater(ProxyCommitData* self, PromiseStream<Future<Void>> addActor) {
|
||||
loop {
|
||||
double interval = std::max(SERVER_KNOBS->MIN_CONFIRM_INTERVAL, (SERVER_KNOBS->REQUIRED_MIN_RECOVERY_DURATION - self->lastCommitLatency)/2.0);
|
||||
double elapsed = now()-self->lastStartCommit;
|
||||
if(elapsed < interval) {
|
||||
wait( delay(interval + 0.0001 - elapsed) );
|
||||
} else {
|
||||
if(self->updateCommitRequests < SERVER_KNOBS->MAX_COMMIT_UPDATES) {
|
||||
addActor.send(updateLastCommit(self));
|
||||
} else {
|
||||
TraceEvent(g_network->isSimulated() ? SevInfo : SevWarnAlways, "TooManyLastCommitUpdates").suppressFor(1.0);
|
||||
self->lastStartCommit = now();
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> proxySnapCreate(ProxySnapRequest snapReq, ProxyCommitData* commitData) {
|
||||
TraceEvent("SnapMasterProxy_SnapReqEnter")
|
||||
.detail("SnapPayload", snapReq.snapPayload)
|
||||
|
@ -2011,7 +1540,7 @@ ACTOR Future<Void> proxySnapCreate(ProxySnapRequest snapReq, ProxyCommitData* co
|
|||
// send a snap request to DD
|
||||
if (!commitData->db->get().distributor.present()) {
|
||||
TraceEvent(SevWarnAlways, "DataDistributorNotPresent").detail("Operation", "SnapRequest");
|
||||
throw operation_failed();
|
||||
throw dd_not_found();
|
||||
}
|
||||
state Future<ErrorOr<Void>> ddSnapReq =
|
||||
commitData->db->get().distributor.get().distributorSnapReq.tryGetReply(DistributorSnapRequest(snapReq.snapPayload, snapReq.snapUID));
|
||||
|
@ -2070,6 +1599,38 @@ ACTOR Future<Void> proxyCheckSafeExclusion(Reference<AsyncVar<ServerDBInfo>> db,
|
|||
return Void();
|
||||
}
|
||||
|
||||
ACTOR Future<Void> reportTxnTagCommitCost(UID myID, Reference<AsyncVar<ServerDBInfo>> db,
|
||||
UIDTransactionTagMap<TransactionCommitCostEstimation>* ssTrTagCommitCost) {
|
||||
state Future<Void> nextRequestTimer = Never();
|
||||
state Future<Void> nextReply = Never();
|
||||
if (db->get().ratekeeper.present()) nextRequestTimer = Void();
|
||||
loop choose {
|
||||
when(wait(db->onChange())) {
|
||||
if (db->get().ratekeeper.present()) {
|
||||
TraceEvent("ProxyRatekeeperChanged", myID).detail("RKID", db->get().ratekeeper.get().id());
|
||||
nextRequestTimer = Void();
|
||||
} else {
|
||||
TraceEvent("ProxyRatekeeperDied", myID);
|
||||
nextRequestTimer = Never();
|
||||
}
|
||||
}
|
||||
when(wait(nextRequestTimer)) {
|
||||
nextRequestTimer = Never();
|
||||
if (db->get().ratekeeper.present()) {
|
||||
nextReply = brokenPromiseToNever(db->get().ratekeeper.get().reportCommitCostEstimation.getReply(
|
||||
ReportCommitCostEstimationRequest(*ssTrTagCommitCost)));
|
||||
} else {
|
||||
nextReply = Never();
|
||||
}
|
||||
}
|
||||
when(wait(nextReply)) {
|
||||
nextReply = Never();
|
||||
ssTrTagCommitCost->clear();
|
||||
nextRequestTimer = delay(SERVER_KNOBS->REPORT_TRANSACTION_COST_ESTIMATION_DELAY);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> masterProxyServerCore(
|
||||
MasterProxyInterface proxy,
|
||||
MasterInterface master,
|
||||
|
@ -2128,24 +1689,19 @@ ACTOR Future<Void> masterProxyServerCore(
|
|||
TraceEvent(SevInfo, "CommitBatchesMemoryLimit").detail("BytesLimit", commitBatchesMemoryLimit);
|
||||
|
||||
addActor.send(monitorRemoteCommitted(&commitData));
|
||||
addActor.send(transactionStarter(proxy, commitData.db, addActor, &commitData, &healthMetricsReply, &detailedHealthMetricsReply));
|
||||
addActor.send(readRequestServer(proxy, addActor, &commitData));
|
||||
addActor.send(rejoinServer(proxy, &commitData));
|
||||
addActor.send(healthMetricsRequestServer(proxy, &healthMetricsReply, &detailedHealthMetricsReply));
|
||||
addActor.send(ddMetricsRequestServer(proxy, db));
|
||||
addActor.send(reportTxnTagCommitCost(proxy.id(), db, &commitData.ssTrTagCommitCost));
|
||||
|
||||
// wait for txnStateStore recovery
|
||||
wait(success(commitData.txnStateStore->readValue(StringRef())));
|
||||
|
||||
if(SERVER_KNOBS->REQUIRED_MIN_RECOVERY_DURATION > 0) {
|
||||
addActor.send(lastCommitUpdater(&commitData, addActor));
|
||||
}
|
||||
|
||||
int commitBatchByteLimit =
|
||||
(int)std::min<double>(SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_BYTES_MAX,
|
||||
std::max<double>(SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_BYTES_MIN,
|
||||
SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_BYTES_SCALE_BASE *
|
||||
pow(commitData.db->get().client.proxies.size(),
|
||||
pow(commitData.db->get().client.masterProxies.size(),
|
||||
SERVER_KNOBS->COMMIT_TRANSACTION_BATCH_BYTES_SCALE_POWER)));
|
||||
|
||||
commitBatcherActor = commitBatcher(&commitData, batchedCommits, proxy.commit.getFuture(), commitBatchByteLimit, commitBatchesMemoryLimit);
|
||||
|
@ -2270,7 +1826,7 @@ ACTOR Future<Void> masterProxyServerCore(
|
|||
|
||||
ACTOR Future<Void> checkRemoved(Reference<AsyncVar<ServerDBInfo>> db, uint64_t recoveryCount, MasterProxyInterface myInterface) {
|
||||
loop{
|
||||
if (db->get().recoveryCount >= recoveryCount && !std::count(db->get().client.proxies.begin(), db->get().client.proxies.end(), myInterface)) {
|
||||
if (db->get().recoveryCount >= recoveryCount && !std::count(db->get().client.masterProxies.begin(), db->get().client.masterProxies.end(), myInterface)) {
|
||||
throw worker_removed();
|
||||
}
|
||||
wait(db->onChange());
|
||||
|
|
|
@ -1300,6 +1300,11 @@ namespace oldTLog_4_6 {
|
|||
DUMPTOKEN( recruited.lock );
|
||||
DUMPTOKEN( recruited.getQueuingMetrics );
|
||||
DUMPTOKEN( recruited.confirmRunning );
|
||||
DUMPTOKEN( recruited.waitFailure );
|
||||
DUMPTOKEN( recruited.recoveryFinished );
|
||||
DUMPTOKEN( recruited.disablePopRequest );
|
||||
DUMPTOKEN( recruited.enablePopRequest );
|
||||
DUMPTOKEN( recruited.snapRequest );
|
||||
|
||||
logData = Reference<LogData>( new LogData(self, recruited) );
|
||||
logData->stopped = true;
|
||||
|
|
|
@ -2136,6 +2136,11 @@ ACTOR Future<Void> restorePersistentState( TLogData* self, LocalityData locality
|
|||
DUMPTOKEN( recruited.lock );
|
||||
DUMPTOKEN( recruited.getQueuingMetrics );
|
||||
DUMPTOKEN( recruited.confirmRunning );
|
||||
DUMPTOKEN( recruited.waitFailure );
|
||||
DUMPTOKEN( recruited.recoveryFinished );
|
||||
DUMPTOKEN( recruited.disablePopRequest );
|
||||
DUMPTOKEN( recruited.enablePopRequest );
|
||||
DUMPTOKEN( recruited.snapRequest );
|
||||
|
||||
//We do not need the remoteTag, because we will not be loading any additional data
|
||||
logData = Reference<LogData>( new LogData(self, recruited, Tag(), true, id_logRouterTags[id1], id_txsTags[id1], UID(), std::vector<Tag>(), "Restored") );
|
||||
|
@ -2303,6 +2308,11 @@ ACTOR Future<Void> tLogStart( TLogData* self, InitializeTLogRequest req, Localit
|
|||
DUMPTOKEN( recruited.lock );
|
||||
DUMPTOKEN( recruited.getQueuingMetrics );
|
||||
DUMPTOKEN( recruited.confirmRunning );
|
||||
DUMPTOKEN( recruited.waitFailure );
|
||||
DUMPTOKEN( recruited.recoveryFinished );
|
||||
DUMPTOKEN( recruited.disablePopRequest );
|
||||
DUMPTOKEN( recruited.enablePopRequest );
|
||||
DUMPTOKEN( recruited.snapRequest );
|
||||
|
||||
for(auto it : self->id_data) {
|
||||
if( !it.second->stopped ) {
|
||||
|
|
|
@ -49,91 +49,42 @@ struct ApplyMutationsData {
|
|||
|
||||
struct ProxyStats {
|
||||
CounterCollection cc;
|
||||
Counter txnRequestIn, txnRequestOut, txnRequestErrors;
|
||||
Counter txnStartIn, txnStartOut, txnStartBatch;
|
||||
Counter txnSystemPriorityStartIn, txnSystemPriorityStartOut;
|
||||
Counter txnBatchPriorityStartIn, txnBatchPriorityStartOut;
|
||||
Counter txnDefaultPriorityStartIn, txnDefaultPriorityStartOut;
|
||||
Counter txnCommitIn, txnCommitVersionAssigned, txnCommitResolving, txnCommitResolved, txnCommitOut,
|
||||
txnCommitOutSuccess, txnCommitErrors;
|
||||
Counter txnConflicts;
|
||||
Counter txnThrottled;
|
||||
Counter commitBatchIn, commitBatchOut;
|
||||
Counter mutationBytes;
|
||||
Counter mutations;
|
||||
Counter conflictRanges;
|
||||
Counter keyServerLocationIn, keyServerLocationOut, keyServerLocationErrors;
|
||||
Counter txnExpensiveClearCostEstCount;
|
||||
Version lastCommitVersionAssigned;
|
||||
|
||||
LatencySample commitLatencySample;
|
||||
LatencySample grvLatencySample;
|
||||
|
||||
LatencyBands commitLatencyBands;
|
||||
LatencyBands grvLatencyBands;
|
||||
|
||||
Future<Void> logger;
|
||||
|
||||
int recentRequests;
|
||||
Deque<int> requestBuckets;
|
||||
double lastBucketBegin;
|
||||
double bucketInterval;
|
||||
|
||||
void updateRequestBuckets() {
|
||||
while (now() - lastBucketBegin > bucketInterval) {
|
||||
lastBucketBegin += bucketInterval;
|
||||
recentRequests -= requestBuckets.front();
|
||||
requestBuckets.pop_front();
|
||||
requestBuckets.push_back(0);
|
||||
}
|
||||
}
|
||||
|
||||
void addRequest() {
|
||||
updateRequestBuckets();
|
||||
++recentRequests;
|
||||
++requestBuckets.back();
|
||||
}
|
||||
|
||||
int getRecentRequests() {
|
||||
updateRequestBuckets();
|
||||
return recentRequests * FLOW_KNOBS->BASIC_LOAD_BALANCE_UPDATE_RATE /
|
||||
(FLOW_KNOBS->BASIC_LOAD_BALANCE_UPDATE_RATE - (lastBucketBegin + bucketInterval - now()));
|
||||
}
|
||||
|
||||
explicit ProxyStats(UID id, Version* pVersion, NotifiedVersion* pCommittedVersion,
|
||||
int64_t* commitBatchesMemBytesCountPtr)
|
||||
: cc("ProxyStats", id.toString()), recentRequests(0), lastBucketBegin(now()),
|
||||
bucketInterval(FLOW_KNOBS->BASIC_LOAD_BALANCE_UPDATE_RATE / FLOW_KNOBS->BASIC_LOAD_BALANCE_BUCKETS),
|
||||
txnRequestIn("TxnRequestIn", cc), txnRequestOut("TxnRequestOut", cc), txnRequestErrors("TxnRequestErrors", cc),
|
||||
txnStartIn("TxnStartIn", cc), txnStartOut("TxnStartOut", cc), txnStartBatch("TxnStartBatch", cc),
|
||||
txnSystemPriorityStartIn("TxnSystemPriorityStartIn", cc),
|
||||
txnSystemPriorityStartOut("TxnSystemPriorityStartOut", cc),
|
||||
txnBatchPriorityStartIn("TxnBatchPriorityStartIn", cc),
|
||||
txnBatchPriorityStartOut("TxnBatchPriorityStartOut", cc),
|
||||
txnDefaultPriorityStartIn("TxnDefaultPriorityStartIn", cc),
|
||||
txnDefaultPriorityStartOut("TxnDefaultPriorityStartOut", cc), txnCommitIn("TxnCommitIn", cc),
|
||||
txnCommitVersionAssigned("TxnCommitVersionAssigned", cc), txnCommitResolving("TxnCommitResolving", cc),
|
||||
txnCommitResolved("TxnCommitResolved", cc), txnCommitOut("TxnCommitOut", cc),
|
||||
txnCommitOutSuccess("TxnCommitOutSuccess", cc), txnCommitErrors("TxnCommitErrors", cc),
|
||||
txnConflicts("TxnConflicts", cc), txnThrottled("TxnThrottled", cc), commitBatchIn("CommitBatchIn", cc),
|
||||
: cc("ProxyStats", id.toString()),
|
||||
txnCommitIn("TxnCommitIn", cc), txnCommitVersionAssigned("TxnCommitVersionAssigned", cc),
|
||||
txnCommitResolving("TxnCommitResolving", cc), txnCommitResolved("TxnCommitResolved", cc),
|
||||
txnCommitOut("TxnCommitOut", cc), txnCommitOutSuccess("TxnCommitOutSuccess", cc),
|
||||
txnCommitErrors("TxnCommitErrors", cc), txnConflicts("TxnConflicts", cc), commitBatchIn("CommitBatchIn", cc),
|
||||
commitBatchOut("CommitBatchOut", cc), mutationBytes("MutationBytes", cc), mutations("Mutations", cc),
|
||||
conflictRanges("ConflictRanges", cc), keyServerLocationIn("KeyServerLocationIn", cc),
|
||||
keyServerLocationOut("KeyServerLocationOut", cc), keyServerLocationErrors("KeyServerLocationErrors", cc),
|
||||
lastCommitVersionAssigned(0),
|
||||
lastCommitVersionAssigned(0), txnExpensiveClearCostEstCount("ExpensiveClearCostEstCount", cc),
|
||||
commitLatencySample("CommitLatencyMetrics", id, SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
grvLatencySample("GRVLatencyMetrics", id, SERVER_KNOBS->LATENCY_METRICS_LOGGING_INTERVAL,
|
||||
SERVER_KNOBS->LATENCY_SAMPLE_SIZE),
|
||||
commitLatencyBands("CommitLatencyMetrics", id, SERVER_KNOBS->STORAGE_LOGGING_DELAY),
|
||||
grvLatencyBands("GRVLatencyMetrics", id, SERVER_KNOBS->STORAGE_LOGGING_DELAY) {
|
||||
commitLatencyBands("CommitLatencyMetrics", id, SERVER_KNOBS->STORAGE_LOGGING_DELAY) {
|
||||
specialCounter(cc, "LastAssignedCommitVersion", [this]() { return this->lastCommitVersionAssigned; });
|
||||
specialCounter(cc, "Version", [pVersion]() { return *pVersion; });
|
||||
specialCounter(cc, "CommittedVersion", [pCommittedVersion]() { return pCommittedVersion->get(); });
|
||||
specialCounter(cc, "CommitBatchesMemBytesCount",
|
||||
[commitBatchesMemBytesCountPtr]() { return *commitBatchesMemBytesCountPtr; });
|
||||
logger = traceCounters("ProxyMetrics", id, SERVER_KNOBS->WORKER_LOGGING_INTERVAL, &cc, "ProxyMetrics");
|
||||
for (int i = 0; i < FLOW_KNOBS->BASIC_LOAD_BALANCE_BUCKETS; i++) {
|
||||
requestBuckets.push_back(0);
|
||||
}
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -190,7 +141,7 @@ struct ProxyCommitData {
|
|||
NotifiedDouble lastCommitTime;
|
||||
|
||||
vector<double> commitComputePerOperation;
|
||||
TransactionTagMap<TransactionCommitCostEstimation> transactionTagCommitCostEst;
|
||||
UIDTransactionTagMap<TransactionCommitCostEstimation> ssTrTagCommitCost;
|
||||
|
||||
// The tag related to a storage server rarely change, so we keep a vector of tags for each key range to be slightly
|
||||
// more CPU efficient. When a tag related to a storage server does change, we empty out all of these vectors to
|
||||
|
@ -199,13 +150,7 @@ struct ProxyCommitData {
|
|||
auto& tags = keyInfo[key].tags;
|
||||
if (!tags.size()) {
|
||||
auto& r = keyInfo.rangeContaining(key).value();
|
||||
for (auto info : r.src_info) {
|
||||
r.tags.push_back(info->tag);
|
||||
}
|
||||
for (auto info : r.dest_info) {
|
||||
r.tags.push_back(info->tag);
|
||||
}
|
||||
uniquify(r.tags);
|
||||
r.populateTags();
|
||||
return r.tags;
|
||||
}
|
||||
return tags;
|
||||
|
@ -222,18 +167,6 @@ struct ProxyCommitData {
|
|||
}
|
||||
|
||||
void updateLatencyBandConfig(Optional<LatencyBandConfig> newLatencyBandConfig) {
|
||||
if (newLatencyBandConfig.present() != latencyBandConfig.present() ||
|
||||
(newLatencyBandConfig.present() &&
|
||||
newLatencyBandConfig.get().grvConfig != latencyBandConfig.get().grvConfig)) {
|
||||
TraceEvent("LatencyBandGrvUpdatingConfig").detail("Present", newLatencyBandConfig.present());
|
||||
stats.grvLatencyBands.clearBands();
|
||||
if (newLatencyBandConfig.present()) {
|
||||
for (auto band : newLatencyBandConfig.get().grvConfig.bands) {
|
||||
stats.grvLatencyBands.addThreshold(band);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
if (newLatencyBandConfig.present() != latencyBandConfig.present() ||
|
||||
(newLatencyBandConfig.present() &&
|
||||
newLatencyBandConfig.get().commitConfig != latencyBandConfig.get().commitConfig)) {
|
||||
|
@ -249,6 +182,18 @@ struct ProxyCommitData {
|
|||
latencyBandConfig = newLatencyBandConfig;
|
||||
}
|
||||
|
||||
void updateSSTagCost(const UID& id, const TagSet& tagSet, MutationRef m, int cost) {
|
||||
auto [it, _] = ssTrTagCommitCost.try_emplace(id, TransactionTagMap<TransactionCommitCostEstimation>());
|
||||
|
||||
for (auto& tag : tagSet) {
|
||||
auto& costItem = it->second[tag];
|
||||
if (m.isAtomicOp() || m.type == MutationRef::Type::SetValue || m.type == MutationRef::Type::ClearRange) {
|
||||
costItem.opsSum++;
|
||||
costItem.costSum += cost;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ProxyCommitData(UID dbgid, MasterInterface master, RequestStream<GetReadVersionRequest> getConsistentReadVersion,
|
||||
Version recoveryTransactionVersion, RequestStream<CommitTransactionRequest> commit,
|
||||
Reference<AsyncVar<ServerDBInfo>> db, bool firstProxy)
|
||||
|
|
|
@ -97,17 +97,21 @@ struct StorageQueueInfo {
|
|||
Smoother smoothTotalSpace;
|
||||
limitReason_t limitReason;
|
||||
|
||||
Optional<TransactionTag> busiestTag;
|
||||
double busiestTagFractionalBusyness;
|
||||
double busiestTagRate;
|
||||
Optional<TransactionTag> busiestReadTag, busiestWriteTag;
|
||||
double busiestReadTagFractionalBusyness = 0, busiestWriteTagFractionalBusyness = 0;
|
||||
double busiestReadTagRate = 0, busiestWriteTagRate = 0;
|
||||
|
||||
// refresh periodically
|
||||
TransactionTagMap<TransactionCommitCostEstimation> tagCostEst;
|
||||
uint64_t totalWriteCosts = 0;
|
||||
int totalWriteOps = 0;
|
||||
|
||||
StorageQueueInfo(UID id, LocalityData locality)
|
||||
: valid(false), id(id), locality(locality), smoothDurableBytes(SERVER_KNOBS->SMOOTHING_AMOUNT),
|
||||
smoothInputBytes(SERVER_KNOBS->SMOOTHING_AMOUNT), verySmoothDurableBytes(SERVER_KNOBS->SLOW_SMOOTHING_AMOUNT),
|
||||
smoothDurableVersion(SERVER_KNOBS->SMOOTHING_AMOUNT),
|
||||
smoothLatestVersion(SERVER_KNOBS->SMOOTHING_AMOUNT), smoothFreeSpace(SERVER_KNOBS->SMOOTHING_AMOUNT),
|
||||
smoothTotalSpace(SERVER_KNOBS->SMOOTHING_AMOUNT), limitReason(limitReason_t::unlimited), busiestTagFractionalBusyness(0),
|
||||
busiestTagRate(0) {
|
||||
smoothDurableVersion(SERVER_KNOBS->SMOOTHING_AMOUNT), smoothLatestVersion(SERVER_KNOBS->SMOOTHING_AMOUNT),
|
||||
smoothFreeSpace(SERVER_KNOBS->SMOOTHING_AMOUNT), smoothTotalSpace(SERVER_KNOBS->SMOOTHING_AMOUNT),
|
||||
limitReason(limitReason_t::unlimited) {
|
||||
// FIXME: this is a tacky workaround for a potential uninitialized use in trackStorageServerQueueInfo
|
||||
lastReply.instanceID = -1;
|
||||
}
|
||||
|
@ -338,7 +342,7 @@ public:
|
|||
return Optional<ClientTagThrottleLimits>();
|
||||
}
|
||||
|
||||
PrioritizedTransactionTagMap<ClientTagThrottleLimits> getClientRates() {
|
||||
PrioritizedTransactionTagMap<ClientTagThrottleLimits> getClientRates(bool autoThrottlingEnabled) {
|
||||
PrioritizedTransactionTagMap<ClientTagThrottleLimits> clientRates;
|
||||
|
||||
for(auto tagItr = tagData.begin(); tagItr != tagData.end();) {
|
||||
|
@ -401,14 +405,18 @@ public:
|
|||
}
|
||||
|
||||
tagPresent = true;
|
||||
auto result = clientRates[TransactionPriority::DEFAULT].try_emplace(tagItr->first, adjustedRate, autoItr->second.limits.expiration);
|
||||
if(!result.second && result.first->second.tpsRate > adjustedRate) {
|
||||
result.first->second = ClientTagThrottleLimits(adjustedRate, autoItr->second.limits.expiration);
|
||||
if (autoThrottlingEnabled) {
|
||||
auto result = clientRates[TransactionPriority::DEFAULT].try_emplace(
|
||||
tagItr->first, adjustedRate, autoItr->second.limits.expiration);
|
||||
if (!result.second && result.first->second.tpsRate > adjustedRate) {
|
||||
result.first->second =
|
||||
ClientTagThrottleLimits(adjustedRate, autoItr->second.limits.expiration);
|
||||
} else {
|
||||
TEST(true); // Auto throttle overriden by manual throttle
|
||||
}
|
||||
clientRates[TransactionPriority::BATCH][tagItr->first] =
|
||||
ClientTagThrottleLimits(0, autoItr->second.limits.expiration);
|
||||
}
|
||||
else {
|
||||
TEST(true); // Auto throttle overriden by manual throttle
|
||||
}
|
||||
clientRates[TransactionPriority::BATCH][tagItr->first] = ClientTagThrottleLimits(0, autoItr->second.limits.expiration);
|
||||
}
|
||||
else {
|
||||
ASSERT(autoItr->second.limits.expiration <= now());
|
||||
|
@ -481,6 +489,7 @@ public:
|
|||
TransactionTagMap<RkTagThrottleData> autoThrottledTags;
|
||||
TransactionTagMap<std::map<TransactionPriority, RkTagThrottleData>> manualThrottledTags;
|
||||
TransactionTagMap<RkTagData> tagData;
|
||||
uint32_t busyReadTagCount = 0, busyWriteTagCount = 0;
|
||||
};
|
||||
|
||||
struct RatekeeperLimits {
|
||||
|
@ -546,6 +555,7 @@ struct RatekeeperData {
|
|||
|
||||
double lastWarning;
|
||||
double lastSSListFetchedTimestamp;
|
||||
double lastBusiestCommitTagPick;
|
||||
|
||||
RkTagThrottleCollection throttledTags;
|
||||
uint64_t throttledTagChangeId;
|
||||
|
@ -565,7 +575,7 @@ struct RatekeeperData {
|
|||
smoothBatchReleasedTransactions(SERVER_KNOBS->SMOOTHING_AMOUNT),
|
||||
smoothTotalDurableBytes(SERVER_KNOBS->SLOW_SMOOTHING_AMOUNT),
|
||||
actualTpsMetric(LiteralStringRef("Ratekeeper.ActualTPS")), lastWarning(0), lastSSListFetchedTimestamp(now()),
|
||||
throttledTagChangeId(0),
|
||||
throttledTagChangeId(0), lastBusiestCommitTagPick(0),
|
||||
normalLimits(TransactionPriority::DEFAULT, "", SERVER_KNOBS->TARGET_BYTES_PER_STORAGE_SERVER,
|
||||
SERVER_KNOBS->SPRING_BYTES_STORAGE_SERVER, SERVER_KNOBS->TARGET_BYTES_PER_TLOG,
|
||||
SERVER_KNOBS->SPRING_BYTES_TLOG, SERVER_KNOBS->MAX_TL_SS_VERSION_DIFFERENCE,
|
||||
|
@ -583,8 +593,7 @@ struct RatekeeperData {
|
|||
ACTOR Future<Void> trackStorageServerQueueInfo( RatekeeperData* self, StorageServerInterface ssi ) {
|
||||
self->storageQueueInfo.insert( mapPair(ssi.id(), StorageQueueInfo(ssi.id(), ssi.locality) ) );
|
||||
state Map<UID, StorageQueueInfo>::iterator myQueueInfo = self->storageQueueInfo.find(ssi.id());
|
||||
TraceEvent("RkTracking", self->id)
|
||||
.detail("StorageServer", ssi.id());
|
||||
TraceEvent("RkTracking", self->id).detail("StorageServer", ssi.id()).detail("Locality", ssi.locality.toString());
|
||||
try {
|
||||
loop {
|
||||
ErrorOr<StorageQueuingMetricsReply> reply = wait( ssi.getQueuingMetrics.getReplyUnlessFailedFor( StorageQueuingMetricsRequest(), 0, 0 ) ); // SOMEDAY: or tryGetReply?
|
||||
|
@ -611,9 +620,9 @@ ACTOR Future<Void> trackStorageServerQueueInfo( RatekeeperData* self, StorageSer
|
|||
myQueueInfo->value.smoothLatestVersion.setTotal(reply.get().version);
|
||||
}
|
||||
|
||||
myQueueInfo->value.busiestTag = reply.get().busiestTag;
|
||||
myQueueInfo->value.busiestTagFractionalBusyness = reply.get().busiestTagFractionalBusyness;
|
||||
myQueueInfo->value.busiestTagRate = reply.get().busiestTagRate;
|
||||
myQueueInfo->value.busiestReadTag = reply.get().busiestTag;
|
||||
myQueueInfo->value.busiestReadTagFractionalBusyness = reply.get().busiestTagFractionalBusyness;
|
||||
myQueueInfo->value.busiestReadTagRate = reply.get().busiestTagRate;
|
||||
} else {
|
||||
if(myQueueInfo->value.valid) {
|
||||
TraceEvent("RkStorageServerDidNotRespond", self->id)
|
||||
|
@ -787,6 +796,8 @@ ACTOR Future<Void> monitorThrottlingChanges(RatekeeperData *self) {
|
|||
TraceEvent(SevWarnAlways, "InvalidAutoTagThrottlingValue", self->id).detail("Value", autoThrottlingEnabled.get().get());
|
||||
}
|
||||
self->autoThrottlingEnabled = SERVER_KNOBS->AUTO_TAG_THROTTLING_ENABLED;
|
||||
if(!committed)
|
||||
tr.set(tagThrottleAutoEnabledKey, LiteralStringRef(self->autoThrottlingEnabled ? "1" : "0"));
|
||||
}
|
||||
|
||||
RkTagThrottleCollection updatedTagThrottles;
|
||||
|
@ -814,6 +825,12 @@ ACTOR Future<Void> monitorThrottlingChanges(RatekeeperData *self) {
|
|||
|
||||
if(tagKey.throttleType == TagThrottleType::AUTO) {
|
||||
updatedTagThrottles.autoThrottleTag(self->id, tag, 0, tagValue.tpsRate, tagValue.expirationTime);
|
||||
if(tagValue.reason == TagThrottledReason::BUSY_READ){
|
||||
updatedTagThrottles.busyReadTagCount ++;
|
||||
}
|
||||
else if(tagValue.reason == TagThrottledReason::BUSY_WRITE) {
|
||||
updatedTagThrottles.busyWriteTagCount ++;
|
||||
}
|
||||
}
|
||||
else {
|
||||
updatedTagThrottles.manualThrottleTag(self->id, tag, tagKey.priority, tagValue.tpsRate, tagValue.expirationTime, oldLimits);
|
||||
|
@ -840,17 +857,83 @@ ACTOR Future<Void> monitorThrottlingChanges(RatekeeperData *self) {
|
|||
}
|
||||
}
|
||||
|
||||
void tryAutoThrottleTag(RatekeeperData *self, StorageQueueInfo const& ss) {
|
||||
if(ss.busiestTag.present() && ss.busiestTagFractionalBusyness > SERVER_KNOBS->AUTO_THROTTLE_TARGET_TAG_BUSYNESS && ss.busiestTagRate > SERVER_KNOBS->MIN_TAG_COST) {
|
||||
TEST(true); // Transaction tag auto-throttled
|
||||
|
||||
Optional<double> clientRate = self->throttledTags.autoThrottleTag(self->id, ss.busiestTag.get(), ss.busiestTagFractionalBusyness);
|
||||
if(clientRate.present()) {
|
||||
TagSet tags;
|
||||
tags.addTag(ss.busiestTag.get());
|
||||
|
||||
self->addActor.send(ThrottleApi::throttleTags(self->db, tags, clientRate.get(), SERVER_KNOBS->AUTO_TAG_THROTTLE_DURATION, TagThrottleType::AUTO, TransactionPriority::DEFAULT, now() + SERVER_KNOBS->AUTO_TAG_THROTTLE_DURATION));
|
||||
Future<Void> refreshStorageServerCommitCost(RatekeeperData* self) {
|
||||
if (self->lastBusiestCommitTagPick == 0) { // the first call should be skipped
|
||||
self->lastBusiestCommitTagPick = now();
|
||||
return Void();
|
||||
}
|
||||
double elapsed = now() - self->lastBusiestCommitTagPick;
|
||||
// for each SS, select the busiest commit tag from ssTrTagCommitCost
|
||||
for (auto it = self->storageQueueInfo.begin(); it != self->storageQueueInfo.end(); ++it) {
|
||||
it->value.busiestWriteTag.reset();
|
||||
TransactionTag busiestTag;
|
||||
TransactionCommitCostEstimation maxCost;
|
||||
double maxRate = 0, maxBusyness = 0;
|
||||
for(const auto& [tag, cost] : it->value.tagCostEst) {
|
||||
double rate = cost.getCostSum() / elapsed;
|
||||
if(rate > maxRate) {
|
||||
busiestTag = tag;
|
||||
maxRate = rate;
|
||||
maxCost = cost;
|
||||
}
|
||||
}
|
||||
if (maxRate > SERVER_KNOBS->MIN_TAG_PAGES_RATE) {
|
||||
it->value.busiestWriteTag = busiestTag;
|
||||
// TraceEvent("RefreshSSCommitCost").detail("TotalWriteCost", it->value.totalWriteCost).detail("TotalWriteOps",it->value.totalWriteOps);
|
||||
ASSERT(it->value.totalWriteCosts > 0);
|
||||
maxBusyness = double(maxCost.getCostSum()) / it->value.totalWriteCosts;
|
||||
it->value.busiestWriteTagFractionalBusyness = maxBusyness;
|
||||
it->value.busiestWriteTagRate = maxRate;
|
||||
}
|
||||
|
||||
TraceEvent("BusiestWriteTag", it->key)
|
||||
.detail("Elapsed", elapsed)
|
||||
.detail("Tag", printable(busiestTag))
|
||||
.detail("TagOps", maxCost.getOpsSum())
|
||||
.detail("TagCost", maxCost.getCostSum())
|
||||
.detail("TotalCost", it->value.totalWriteCosts)
|
||||
.detail("Reported", it->value.busiestWriteTag.present())
|
||||
.trackLatest(it->key.toString() + "/BusiestWriteTag");
|
||||
|
||||
// reset statistics
|
||||
it->value.tagCostEst.clear();
|
||||
it->value.totalWriteOps = 0;
|
||||
it->value.totalWriteCosts = 0;
|
||||
}
|
||||
self->lastBusiestCommitTagPick = now();
|
||||
return Void();
|
||||
}
|
||||
|
||||
void tryAutoThrottleTag(RatekeeperData* self, TransactionTag tag, double rate, double busyness,
|
||||
TagThrottledReason reason) {
|
||||
if (busyness > SERVER_KNOBS->AUTO_THROTTLE_TARGET_TAG_BUSYNESS && rate > SERVER_KNOBS->MIN_TAG_COST) {
|
||||
TEST(true); // Transaction tag auto-throttled
|
||||
Optional<double> clientRate = self->throttledTags.autoThrottleTag(self->id, tag, busyness);
|
||||
if (clientRate.present()) {
|
||||
TagSet tags;
|
||||
tags.addTag(tag);
|
||||
|
||||
self->addActor.send(ThrottleApi::throttleTags(
|
||||
self->db, tags, clientRate.get(), SERVER_KNOBS->AUTO_TAG_THROTTLE_DURATION, TagThrottleType::AUTO,
|
||||
TransactionPriority::DEFAULT, now() + SERVER_KNOBS->AUTO_TAG_THROTTLE_DURATION, reason));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
void tryAutoThrottleTag(RatekeeperData* self, StorageQueueInfo& ss, int64_t storageQueue,
|
||||
int64_t storageDurabilityLag) {
|
||||
// TODO: reasonable criteria for write satuation should be investigated in experiment
|
||||
// if (ss.busiestWriteTag.present() && storageQueue > SERVER_KNOBS->AUTO_TAG_THROTTLE_STORAGE_QUEUE_BYTES &&
|
||||
// storageDurabilityLag > SERVER_KNOBS->AUTO_TAG_THROTTLE_DURABILITY_LAG_VERSIONS) {
|
||||
// // write-saturated
|
||||
// tryAutoThrottleTag(self, ss.busiestWriteTag.get(), ss.busiestWriteTagRate,
|
||||
//ss.busiestWriteTagFractionalBusyness); } else
|
||||
if (ss.busiestReadTag.present() &&
|
||||
(storageQueue > SERVER_KNOBS->AUTO_TAG_THROTTLE_STORAGE_QUEUE_BYTES ||
|
||||
storageDurabilityLag > SERVER_KNOBS->AUTO_TAG_THROTTLE_DURABILITY_LAG_VERSIONS)) {
|
||||
// read saturated
|
||||
tryAutoThrottleTag(self, ss.busiestReadTag.get(), ss.busiestReadTagRate, ss.busiestReadTagFractionalBusyness,
|
||||
TagThrottledReason::BUSY_READ);
|
||||
}
|
||||
}
|
||||
|
||||
|
@ -921,8 +1004,8 @@ void updateRate(RatekeeperData* self, RatekeeperLimits* limits) {
|
|||
|
||||
double targetRateRatio = std::min(( storageQueue - targetBytes + springBytes ) / (double)springBytes, 2.0);
|
||||
|
||||
if(limits->priority == TransactionPriority::DEFAULT && (storageQueue > SERVER_KNOBS->AUTO_TAG_THROTTLE_STORAGE_QUEUE_BYTES || storageDurabilityLag > SERVER_KNOBS->AUTO_TAG_THROTTLE_DURABILITY_LAG_VERSIONS)) {
|
||||
tryAutoThrottleTag(self, ss);
|
||||
if (limits->priority == TransactionPriority::DEFAULT) {
|
||||
tryAutoThrottleTag(self, ss, storageQueue, storageDurabilityLag);
|
||||
}
|
||||
|
||||
double inputRate = ss.smoothInputBytes.smoothRate();
|
||||
|
@ -990,10 +1073,10 @@ void updateRate(RatekeeperData* self, RatekeeperLimits* limits) {
|
|||
limits->tpsLimit = ss->first;
|
||||
reasonID = storageTpsLimitReverseIndex.begin()->second->id; // Although we aren't controlling based on the worst SS, we still report it as the limiting process
|
||||
limitReason = ssReasons[reasonID];
|
||||
|
||||
break;
|
||||
}
|
||||
|
||||
// Calculate limited durability lag
|
||||
int64_t limitingDurabilityLag = 0;
|
||||
|
||||
std::set<Optional<Standalone<StringRef>>> ignoredDurabilityLagMachines;
|
||||
|
@ -1184,7 +1267,7 @@ void updateRate(RatekeeperData* self, RatekeeperLimits* limits) {
|
|||
.detail("ReleasedBatchTPS", self->smoothBatchReleasedTransactions.smoothRate())
|
||||
.detail("TPSBasis", actualTps)
|
||||
.detail("StorageServers", sscount)
|
||||
.detail("Proxies", self->proxyInfo.size())
|
||||
.detail("GrvProxies", self->proxyInfo.size())
|
||||
.detail("TLogs", tlcount)
|
||||
.detail("WorstFreeSpaceStorageServer", worstFreeSpaceStorageServer)
|
||||
.detail("WorstFreeSpaceTLog", worstFreeSpaceTLog)
|
||||
|
@ -1197,11 +1280,27 @@ void updateRate(RatekeeperData* self, RatekeeperLimits* limits) {
|
|||
.detail("WorstStorageServerDurabilityLag", worstDurabilityLag)
|
||||
.detail("LimitingStorageServerDurabilityLag", limitingDurabilityLag)
|
||||
.detail("TagsAutoThrottled", self->throttledTags.autoThrottleCount())
|
||||
.detail("TagsAutoThrottledBusyRead", self->throttledTags.busyReadTagCount)
|
||||
.detail("TagsAutoThrottledBusyWrite", self->throttledTags.busyWriteTagCount)
|
||||
.detail("TagsManuallyThrottled", self->throttledTags.manualThrottleCount())
|
||||
.detail("AutoThrottlingEnabled", self->autoThrottlingEnabled)
|
||||
.trackLatest(name);
|
||||
}
|
||||
}
|
||||
|
||||
static void updateCommitCostEstimation(RatekeeperData* self,
|
||||
UIDTransactionTagMap<TransactionCommitCostEstimation> const& costEstimation) {
|
||||
for (auto it = self->storageQueueInfo.begin(); it != self->storageQueueInfo.end(); ++it) {
|
||||
auto tagCostIt = costEstimation.find(it->key);
|
||||
if (tagCostIt == costEstimation.end()) continue;
|
||||
for (const auto& [tagName, cost] : tagCostIt->second) {
|
||||
it->value.tagCostEst[tagName] += cost;
|
||||
it->value.totalWriteCosts += cost.getCostSum();
|
||||
it->value.totalWriteOps += cost.getOpsSum();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
ACTOR Future<Void> configurationMonitor(RatekeeperData *self) {
|
||||
loop {
|
||||
state ReadYourWritesTransaction tr(self->db);
|
||||
|
@ -1244,6 +1343,9 @@ ACTOR Future<Void> ratekeeper(RatekeeperInterface rkInterf, Reference<AsyncVar<S
|
|||
self.addActor.send( traceRole(Role::RATEKEEPER, rkInterf.id()) );
|
||||
|
||||
self.addActor.send(monitorThrottlingChanges(&self));
|
||||
RatekeeperData* selfPtr = &self; // let flow compiler capture self
|
||||
self.addActor.send(
|
||||
recurring([selfPtr]() { refreshStorageServerCommitCost(selfPtr); }, SERVER_KNOBS->TAG_MEASUREMENT_INTERVAL));
|
||||
|
||||
TraceEvent("RkTLogQueueSizeParameters", rkInterf.id()).detail("Target", SERVER_KNOBS->TARGET_BYTES_PER_TLOG).detail("Spring", SERVER_KNOBS->SPRING_BYTES_TLOG)
|
||||
.detail("Rate", (SERVER_KNOBS->TARGET_BYTES_PER_TLOG - SERVER_KNOBS->SPRING_BYTES_TLOG) / ((((double)SERVER_KNOBS->MAX_READ_TRANSACTION_LIFE_VERSIONS) / SERVER_KNOBS->VERSIONS_PER_SECOND) + 2.0));
|
||||
|
@ -1279,17 +1381,13 @@ ACTOR Future<Void> ratekeeper(RatekeeperInterface rkInterf, Reference<AsyncVar<S
|
|||
GetRateInfoReply reply;
|
||||
|
||||
auto& p = self.proxyInfo[ req.requesterID ];
|
||||
//TraceEvent("RKMPU", req.requesterID).detail("TRT", req.totalReleasedTransactions).detail("Last", p.first).detail("Delta", req.totalReleasedTransactions - p.first);
|
||||
//TraceEvent("RKMPU", req.requesterID).detail("TRT", req.totalReleasedTransactions).detail("Last", p.totalTransactions).detail("Delta", req.totalReleasedTransactions - p.totalTransactions);
|
||||
if (p.totalTransactions > 0) {
|
||||
self.smoothReleasedTransactions.addDelta( req.totalReleasedTransactions - p.totalTransactions );
|
||||
|
||||
for(auto tag : req.throttledTagCounts) {
|
||||
self.throttledTags.addRequests(tag.first, tag.second);
|
||||
}
|
||||
// TODO process commitCostEstimation
|
||||
// for (const auto &[tagName, cost] : req.throttledTagCommitCostEst) {
|
||||
//
|
||||
// }
|
||||
}
|
||||
if(p.batchTransactions > 0) {
|
||||
self.smoothBatchReleasedTransactions.addDelta( req.batchReleasedTransactions - p.batchTransactions );
|
||||
|
@ -1303,11 +1401,11 @@ ACTOR Future<Void> ratekeeper(RatekeeperInterface rkInterf, Reference<AsyncVar<S
|
|||
reply.batchTransactionRate = self.batchLimits.tpsLimit / self.proxyInfo.size();
|
||||
reply.leaseDuration = SERVER_KNOBS->METRIC_UPDATE_RATE;
|
||||
|
||||
if(p.lastThrottledTagChangeId != self.throttledTagChangeId || now() < p.lastTagPushTime + SERVER_KNOBS->TAG_THROTTLE_PUSH_INTERVAL) {
|
||||
if(p.lastThrottledTagChangeId != self.throttledTagChangeId || now() > p.lastTagPushTime + SERVER_KNOBS->TAG_THROTTLE_PUSH_INTERVAL) {
|
||||
p.lastThrottledTagChangeId = self.throttledTagChangeId;
|
||||
p.lastTagPushTime = now();
|
||||
|
||||
reply.throttledTags = self.throttledTags.getClientRates();
|
||||
reply.throttledTags = self.throttledTags.getClientRates(self.autoThrottlingEnabled);
|
||||
TEST(reply.throttledTags.present() && reply.throttledTags.get().size() > 0); // Returning tag throttles to a proxy
|
||||
}
|
||||
|
||||
|
@ -1322,6 +1420,10 @@ ACTOR Future<Void> ratekeeper(RatekeeperInterface rkInterf, Reference<AsyncVar<S
|
|||
TraceEvent("RatekeeperHalted", rkInterf.id()).detail("ReqID", req.requesterID);
|
||||
break;
|
||||
}
|
||||
when(ReportCommitCostEstimationRequest req = waitNext(rkInterf.reportCommitCostEstimation.getFuture())) {
|
||||
updateCommitCostEstimation(&self, req.ssTrTagCommitCost);
|
||||
req.reply.send(Void());
|
||||
}
|
||||
when (wait(err.getFuture())) {}
|
||||
when (wait(dbInfo->onChange())) {
|
||||
if( tlogInterfs != dbInfo->get().logSystemConfig.allLocalLogs() ) {
|
||||
|
|
|
@ -30,6 +30,7 @@ struct RatekeeperInterface {
|
|||
RequestStream<ReplyPromise<Void>> waitFailure;
|
||||
RequestStream<struct GetRateInfoRequest> getRateInfo;
|
||||
RequestStream<struct HaltRatekeeperRequest> haltRatekeeper;
|
||||
RequestStream<struct ReportCommitCostEstimationRequest> reportCommitCostEstimation;
|
||||
struct LocalityData locality;
|
||||
UID myId;
|
||||
|
||||
|
@ -48,7 +49,7 @@ struct RatekeeperInterface {
|
|||
|
||||
template <class Archive>
|
||||
void serialize(Archive& ar) {
|
||||
serializer(ar, waitFailure, getRateInfo, haltRatekeeper, locality, myId);
|
||||
serializer(ar, waitFailure, getRateInfo, haltRatekeeper, reportCommitCostEstimation, locality, myId);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -76,31 +77,35 @@ struct ClientTagThrottleLimits {
|
|||
};
|
||||
|
||||
struct TransactionCommitCostEstimation {
|
||||
int numWrite = 0;
|
||||
int numAtomicWrite = 0;
|
||||
int numClear = 0;
|
||||
int numClearShards = 0;
|
||||
uint64_t bytesWrite = 0;
|
||||
uint64_t bytesAtomicWrite = 0;
|
||||
uint64_t bytesClearEst = 0;
|
||||
int opsSum = 0;
|
||||
uint64_t costSum = 0;
|
||||
|
||||
uint64_t getCostSum() const { return costSum; }
|
||||
int getOpsSum() const { return opsSum; }
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, bytesWrite, bytesClearEst, bytesAtomicWrite, numWrite, numAtomicWrite, numClear, numClearShards);
|
||||
serializer(ar, opsSum, costSum);
|
||||
}
|
||||
|
||||
TransactionCommitCostEstimation& operator+=(const TransactionCommitCostEstimation& other) {
|
||||
numWrite += other.numWrite;
|
||||
numAtomicWrite += other.numAtomicWrite;
|
||||
numClear += other.numClear;
|
||||
bytesWrite += other.bytesWrite;
|
||||
bytesAtomicWrite += other.numAtomicWrite;
|
||||
numClearShards += other.numClearShards;
|
||||
bytesClearEst += other.bytesClearEst;
|
||||
opsSum += other.opsSum;
|
||||
costSum += other.costSum;
|
||||
return *this;
|
||||
}
|
||||
};
|
||||
|
||||
struct ClientTrCommitCostEstimation {
|
||||
int opsCount = 0;
|
||||
uint64_t writeCosts = 0;
|
||||
std::deque<std::pair<int, uint64_t>> clearIdxCosts;
|
||||
uint32_t expensiveCostEstCount = 0;
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, opsCount, writeCosts, clearIdxCosts, expensiveCostEstCount);
|
||||
}
|
||||
};
|
||||
|
||||
struct GetRateInfoReply {
|
||||
constexpr static FileIdentifier file_identifier = 7845006;
|
||||
double transactionRate;
|
||||
|
@ -123,21 +128,20 @@ struct GetRateInfoRequest {
|
|||
int64_t batchReleasedTransactions;
|
||||
|
||||
TransactionTagMap<uint64_t> throttledTagCounts;
|
||||
TransactionTagMap<TransactionCommitCostEstimation> throttledTagCommitCostEst;
|
||||
bool detailed;
|
||||
ReplyPromise<struct GetRateInfoReply> reply;
|
||||
|
||||
GetRateInfoRequest() {}
|
||||
GetRateInfoRequest(UID const& requesterID, int64_t totalReleasedTransactions, int64_t batchReleasedTransactions,
|
||||
TransactionTagMap<uint64_t> throttledTagCounts,
|
||||
TransactionTagMap<TransactionCommitCostEstimation> throttledTagCommitCostEst, bool detailed)
|
||||
TransactionTagMap<uint64_t> throttledTagCounts, bool detailed)
|
||||
: requesterID(requesterID), totalReleasedTransactions(totalReleasedTransactions),
|
||||
batchReleasedTransactions(batchReleasedTransactions), throttledTagCounts(throttledTagCounts),
|
||||
throttledTagCommitCostEst(throttledTagCommitCostEst), detailed(detailed) {}
|
||||
detailed(detailed) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, requesterID, totalReleasedTransactions, batchReleasedTransactions, throttledTagCounts, detailed, reply, throttledTagCommitCostEst);
|
||||
serializer(ar, requesterID, totalReleasedTransactions, batchReleasedTransactions, throttledTagCounts, detailed,
|
||||
reply);
|
||||
}
|
||||
};
|
||||
|
||||
|
@ -155,4 +159,19 @@ struct HaltRatekeeperRequest {
|
|||
}
|
||||
};
|
||||
|
||||
struct ReportCommitCostEstimationRequest {
|
||||
constexpr static FileIdentifier file_identifier = 8314904;
|
||||
UIDTransactionTagMap<TransactionCommitCostEstimation> ssTrTagCommitCost;
|
||||
ReplyPromise<Void> reply;
|
||||
|
||||
ReportCommitCostEstimationRequest() {}
|
||||
ReportCommitCostEstimationRequest(UIDTransactionTagMap<TransactionCommitCostEstimation> ssTrTagCommitCost)
|
||||
: ssTrTagCommitCost(ssTrTagCommitCost) {}
|
||||
|
||||
template <class Ar>
|
||||
void serialize(Ar& ar) {
|
||||
serializer(ar, ssTrTagCommitCost, reply);
|
||||
}
|
||||
};
|
||||
|
||||
#endif //FDBSERVER_RATEKEEPERINTERFACE_H
|
||||
|
|
|
@ -77,6 +77,7 @@ ACTOR Future<Void> restoreApplierCore(RestoreApplierInterface applierInterf, int
|
|||
}
|
||||
when(RestoreFinishRequest req = waitNext(applierInterf.finishRestore.getFuture())) {
|
||||
requestTypeStr = "finishRestore";
|
||||
actors.clear(false); // cancel all pending actors
|
||||
handleFinishRestoreRequest(req, self);
|
||||
if (req.terminate) {
|
||||
exitRole = Void();
|
||||
|
@ -88,6 +89,7 @@ ACTOR Future<Void> restoreApplierCore(RestoreApplierInterface applierInterf, int
|
|||
break;
|
||||
}
|
||||
}
|
||||
TraceEvent("RestoreApplierCore", self->id()).detail("Request", requestTypeStr); // For debug only
|
||||
} catch (Error& e) {
|
||||
TraceEvent(SevWarn, "FastRestoreApplierError", self->id())
|
||||
.detail("RequestType", requestTypeStr)
|
||||
|
@ -108,12 +110,23 @@ ACTOR Future<Void> restoreApplierCore(RestoreApplierInterface applierInterf, int
|
|||
// Only one actor can process mutations from the same file.
|
||||
ACTOR static Future<Void> handleSendMutationVectorRequest(RestoreSendVersionedMutationsRequest req,
|
||||
Reference<RestoreApplierData> self) {
|
||||
state Reference<ApplierBatchData> batchData = self->batch[req.batchIndex];
|
||||
// Assume: processedFileState[req.asset] will not be erased while the actor is active.
|
||||
// Note: Insert new items into processedFileState will not invalidate the reference.
|
||||
state NotifiedVersion& curMsgIndex = batchData->processedFileState[req.asset];
|
||||
state Reference<ApplierBatchData> batchData; // initialized as nullptr
|
||||
state bool printTrace = false;
|
||||
state NotifiedVersion* curMsgIndex = nullptr;
|
||||
|
||||
if (req.batchIndex <= self->finishedBatch.get()) { // Handle duplicate request from batchIndex that has finished
|
||||
TraceEvent(SevWarn, "FastRestoreApplierRestoreSendVersionedMutationsRequestTooLate")
|
||||
.detail("RequestBatchIndex", req.batchIndex)
|
||||
.detail("FinishedBatchIndex", self->finishedBatch.get());
|
||||
req.reply.send(RestoreCommonReply(self->id(), true));
|
||||
ASSERT_WE_THINK(false); // Test to see if simulation can reproduce this
|
||||
return Void();
|
||||
}
|
||||
|
||||
batchData = self->batch[req.batchIndex];
|
||||
|
||||
ASSERT(batchData.isValid());
|
||||
ASSERT(self->finishedBatch.get() < req.batchIndex);
|
||||
// wait(delay(0.0, TaskPriority::RestoreApplierReceiveMutations)); // This hurts performance from 100MB/s to 60MB/s
|
||||
// on circus
|
||||
|
||||
|
@ -121,11 +134,11 @@ ACTOR static Future<Void> handleSendMutationVectorRequest(RestoreSendVersionedMu
|
|||
// Trace when the receive phase starts at a VB and when it finishes.
|
||||
// This can help check if receiveMutations block applyMutation phase.
|
||||
// If so, we need more sophisticated scheduler to ensure priority execution
|
||||
printTrace = (batchData->receiveMutationReqs % 100 == 1);
|
||||
printTrace = (batchData->receiveMutationReqs % SERVER_KNOBS->FASTRESTORE_NUM_TRACE_EVENTS == 0);
|
||||
TraceEvent(printTrace ? SevInfo : SevFRDebugInfo, "FastRestoreApplierPhaseReceiveMutations", self->id())
|
||||
.detail("BatchIndex", req.batchIndex)
|
||||
.detail("RestoreAsset", req.asset.toString())
|
||||
.detail("RestoreAssetMesssageIndex", curMsgIndex.get())
|
||||
.detail("RestoreAssetMesssageIndex", batchData->processedFileState[req.asset].get())
|
||||
.detail("Request", req.toString())
|
||||
.detail("CurrentMemory", getSystemStatistics().processMemory)
|
||||
.detail("PreviousVersionBatchState", batchData->vbState.get())
|
||||
|
@ -133,11 +146,16 @@ ACTOR static Future<Void> handleSendMutationVectorRequest(RestoreSendVersionedMu
|
|||
|
||||
wait(isSchedulable(self, req.batchIndex, __FUNCTION__));
|
||||
|
||||
wait(curMsgIndex.whenAtLeast(req.msgIndex - 1));
|
||||
ASSERT(batchData.isValid());
|
||||
ASSERT(req.batchIndex > self->finishedBatch.get());
|
||||
// Assume: processedFileState[req.asset] will not be erased while the actor is active.
|
||||
// Note: Insert new items into processedFileState will not invalidate the reference.
|
||||
curMsgIndex = &batchData->processedFileState[req.asset];
|
||||
wait(curMsgIndex->whenAtLeast(req.msgIndex - 1));
|
||||
batchData->vbState = ApplierVersionBatchState::RECEIVE_MUTATIONS;
|
||||
|
||||
state bool isDuplicated = true;
|
||||
if (curMsgIndex.get() == req.msgIndex - 1) {
|
||||
if (curMsgIndex->get() == req.msgIndex - 1) {
|
||||
isDuplicated = false;
|
||||
|
||||
for (int mIndex = 0; mIndex < req.versionedMutations.size(); mIndex++) {
|
||||
|
@ -165,14 +183,14 @@ ACTOR static Future<Void> handleSendMutationVectorRequest(RestoreSendVersionedMu
|
|||
ASSERT(versionedMutation.mutation.type != MutationRef::SetVersionstampedKey &&
|
||||
versionedMutation.mutation.type != MutationRef::SetVersionstampedValue);
|
||||
}
|
||||
curMsgIndex.set(req.msgIndex);
|
||||
curMsgIndex->set(req.msgIndex);
|
||||
}
|
||||
|
||||
req.reply.send(RestoreCommonReply(self->id(), isDuplicated));
|
||||
TraceEvent(printTrace ? SevInfo : SevFRDebugInfo, "FastRestoreApplierPhaseReceiveMutationsDone", self->id())
|
||||
.detail("BatchIndex", req.batchIndex)
|
||||
.detail("RestoreAsset", req.asset.toString())
|
||||
.detail("ProcessedMessageIndex", curMsgIndex.get())
|
||||
.detail("ProcessedMessageIndex", curMsgIndex->get())
|
||||
.detail("Request", req.toString());
|
||||
return Void();
|
||||
}
|
||||
|
@ -185,7 +203,7 @@ ACTOR static Future<Void> applyClearRangeMutations(Standalone<VectorRef<KeyRange
|
|||
state int retries = 0;
|
||||
state double numOps = 0;
|
||||
wait(delay(delayTime + deterministicRandom()->random01() * delayTime));
|
||||
TraceEvent(delayTime > 5 ? SevWarnAlways : SevInfo, "FastRestoreApplierClearRangeMutationsStart", applierID)
|
||||
TraceEvent(delayTime > 5 ? SevWarnAlways : SevDebug, "FastRestoreApplierClearRangeMutationsStart", applierID)
|
||||
.detail("BatchIndex", batchIndex)
|
||||
.detail("Ranges", ranges.size())
|
||||
.detail("DelayTime", delayTime);
|
||||
|
@ -296,7 +314,7 @@ ACTOR static Future<Void> getAndComputeStagingKeys(
|
|||
for (auto& key : incompleteStagingKeys) {
|
||||
if (!fValues[i].get().present()) { // Key not exist in DB
|
||||
// if condition: fValues[i].Valid() && fValues[i].isReady() && !fValues[i].isError() &&
|
||||
TraceEvent(SevWarn, "FastRestoreApplierGetAndComputeStagingKeysNoBaseValueInDB", applierID)
|
||||
TraceEvent(SevDebug, "FastRestoreApplierGetAndComputeStagingKeysNoBaseValueInDB", applierID)
|
||||
.suppressFor(5.0)
|
||||
.detail("BatchIndex", batchIndex)
|
||||
.detail("Key", key.first)
|
||||
|
@ -304,7 +322,7 @@ ACTOR static Future<Void> getAndComputeStagingKeys(
|
|||
.detail("PendingMutations", key.second->second.pendingMutations.size())
|
||||
.detail("StagingKeyType", getTypeString(key.second->second.type));
|
||||
for (auto& vm : key.second->second.pendingMutations) {
|
||||
TraceEvent(SevWarn, "FastRestoreApplierGetAndComputeStagingKeysNoBaseValueInDB")
|
||||
TraceEvent(SevDebug, "FastRestoreApplierGetAndComputeStagingKeysNoBaseValueInDB")
|
||||
.detail("PendingMutationVersion", vm.first.toString())
|
||||
.detail("PendingMutation", vm.second.toString());
|
||||
}
|
||||
|
@ -398,6 +416,7 @@ ACTOR static Future<Void> precomputeMutationsResult(Reference<ApplierBatchData>
|
|||
std::map<Key, std::map<Key, StagingKey>::iterator> incompleteStagingKeys;
|
||||
std::map<Key, StagingKey>::iterator stagingKeyIter = batchData->stagingKeys.begin();
|
||||
int numKeysInBatch = 0;
|
||||
int numGetTxns = 0;
|
||||
double delayTime = 0; // Start transactions at different time to avoid overwhelming FDB.
|
||||
for (; stagingKeyIter != batchData->stagingKeys.end(); stagingKeyIter++) {
|
||||
if (!stagingKeyIter->second.hasBaseValue()) {
|
||||
|
@ -407,12 +426,14 @@ ACTOR static Future<Void> precomputeMutationsResult(Reference<ApplierBatchData>
|
|||
if (numKeysInBatch == SERVER_KNOBS->FASTRESTORE_APPLIER_FETCH_KEYS_SIZE) {
|
||||
fGetAndComputeKeys.push_back(getAndComputeStagingKeys(incompleteStagingKeys, delayTime, cx, applierID,
|
||||
batchIndex, &batchData->counters));
|
||||
numGetTxns++;
|
||||
delayTime += SERVER_KNOBS->FASTRESTORE_TXN_EXTRA_DELAY;
|
||||
numKeysInBatch = 0;
|
||||
incompleteStagingKeys.clear();
|
||||
}
|
||||
}
|
||||
if (numKeysInBatch > 0) {
|
||||
numGetTxns++;
|
||||
fGetAndComputeKeys.push_back(getAndComputeStagingKeys(incompleteStagingKeys, delayTime, cx, applierID,
|
||||
batchIndex, &batchData->counters));
|
||||
}
|
||||
|
@ -420,7 +441,8 @@ ACTOR static Future<Void> precomputeMutationsResult(Reference<ApplierBatchData>
|
|||
TraceEvent("FastRestoreApplerPhasePrecomputeMutationsResult", applierID)
|
||||
.detail("BatchIndex", batchIndex)
|
||||
.detail("Step", "Compute the other staging keys")
|
||||
.detail("StagingKeys", batchData->stagingKeys.size());
|
||||
.detail("StagingKeys", batchData->stagingKeys.size())
|
||||
.detail("GetStagingKeyBatchTxns", numGetTxns);
|
||||
// Pre-compute pendingMutations to other keys in stagingKeys that has base value
|
||||
for (stagingKeyIter = batchData->stagingKeys.begin(); stagingKeyIter != batchData->stagingKeys.end();
|
||||
stagingKeyIter++) {
|
||||
|
@ -577,8 +599,10 @@ ACTOR static Future<Void> handleApplyToDBRequest(RestoreVersionBatchRequest req,
|
|||
wait(self->finishedBatch.whenAtLeast(req.batchIndex - 1));
|
||||
|
||||
state bool isDuplicated = true;
|
||||
if (self->finishedBatch.get() == req.batchIndex - 1) {
|
||||
if (self->finishedBatch.get() ==
|
||||
req.batchIndex - 1) { // duplicate request from earlier version batch will be ignored
|
||||
Reference<ApplierBatchData> batchData = self->batch[req.batchIndex];
|
||||
ASSERT(batchData.isValid());
|
||||
TraceEvent("FastRestoreApplierPhaseHandleApplyToDBRunning", self->id())
|
||||
.detail("BatchIndex", req.batchIndex)
|
||||
.detail("FinishedBatch", self->finishedBatch.get())
|
||||
|
@ -604,7 +628,7 @@ ACTOR static Future<Void> handleApplyToDBRequest(RestoreVersionBatchRequest req,
|
|||
// Avoid setting finishedBatch when finishedBatch > req.batchIndex
|
||||
if (self->finishedBatch.get() == req.batchIndex - 1) {
|
||||
self->finishedBatch.set(req.batchIndex);
|
||||
self->batch[req.batchIndex]->vbState = ApplierVersionBatchState::DONE;
|
||||
// self->batch[req.batchIndex]->vbState = ApplierVersionBatchState::DONE;
|
||||
// Free memory for the version batch
|
||||
self->batch.erase(req.batchIndex);
|
||||
if (self->delayedActors > 0) {
|
||||
|
|
|
@ -115,7 +115,7 @@ struct StagingKey {
|
|||
// Precompute the final value of the key.
|
||||
// TODO: Look at the last LogMessageVersion, if it set or clear, we can ignore the rest of versions.
|
||||
void precomputeResult(const char* context, UID applierID, int batchIndex) {
|
||||
TraceEvent(SevDebug, "FastRestoreApplierPrecomputeResult", applierID)
|
||||
TraceEvent(SevFRMutationInfo, "FastRestoreApplierPrecomputeResult", applierID)
|
||||
.detail("BatchIndex", batchIndex)
|
||||
.detail("Context", context)
|
||||
.detail("Version", version.toString())
|
||||
|
|
|
@ -84,6 +84,7 @@ ACTOR Future<Void> sampleBackups(Reference<RestoreControllerData> self, RestoreC
|
|||
ASSERT(req.batchIndex <= self->batch.size()); // batchIndex starts from 1
|
||||
|
||||
Reference<ControllerBatchData> batch = self->batch[req.batchIndex];
|
||||
ASSERT(batch.isValid());
|
||||
if (batch->sampleMsgs.find(req.id) != batch->sampleMsgs.end()) {
|
||||
req.reply.send(RestoreCommonReply(req.id));
|
||||
continue;
|
||||
|
@ -164,7 +165,10 @@ ACTOR Future<Void> recruitRestoreRoles(Reference<RestoreWorkerData> controllerWo
|
|||
break;
|
||||
}
|
||||
|
||||
TraceEvent("FastRestoreController", controllerData->id()).detail("WorkerNode", workerInterf.first);
|
||||
TraceEvent("FastRestoreController", controllerData->id())
|
||||
.detail("WorkerNode", workerInterf.first)
|
||||
.detail("NodeRole", role)
|
||||
.detail("NodeIndex", nodeIndex);
|
||||
requests.emplace_back(workerInterf.first,
|
||||
RestoreRecruitRoleRequest(controllerWorker->controllerInterf.get(), role, nodeIndex));
|
||||
nodeIndex++;
|
||||
|
@ -300,7 +304,7 @@ ACTOR static Future<Version> processRestoreRequest(Reference<RestoreControllerDa
|
|||
state std::vector<RestoreFileFR> logFiles;
|
||||
state std::vector<RestoreFileFR> allFiles;
|
||||
state Version minRangeVersion = MAX_VERSION;
|
||||
state ActorCollection actors(false);
|
||||
state Future<Void> error = actorCollection(self->addActor.getFuture());
|
||||
|
||||
self->initBackupContainer(request.url);
|
||||
|
||||
|
@ -356,7 +360,7 @@ ACTOR static Future<Version> processRestoreRequest(Reference<RestoreControllerDa
|
|||
}
|
||||
}
|
||||
|
||||
actors.add(monitorFinishedVersion(self, request));
|
||||
self->addActor.send(monitorFinishedVersion(self, request));
|
||||
state std::vector<VersionBatch>::iterator versionBatch = versionBatches.begin();
|
||||
for (; versionBatch != versionBatches.end(); versionBatch++) {
|
||||
while (self->runningVersionBatches.get() >= SERVER_KNOBS->FASTRESTORE_VB_PARALLELISM && !releaseVBOutOfOrder) {
|
||||
|
@ -378,7 +382,11 @@ ACTOR static Future<Version> processRestoreRequest(Reference<RestoreControllerDa
|
|||
wait(delay(SERVER_KNOBS->FASTRESTORE_VB_LAUNCH_DELAY));
|
||||
}
|
||||
|
||||
wait(waitForAll(fBatches));
|
||||
try {
|
||||
wait(waitForAll(fBatches) || error);
|
||||
} catch (Error& e) {
|
||||
TraceEvent(SevError, "FastRestoreControllerDispatchVersionBatchesUnexpectedError").error(e);
|
||||
}
|
||||
|
||||
TraceEvent("FastRestoreController").detail("RestoreToVersion", request.targetVersion);
|
||||
return request.targetVersion;
|
||||
|
@ -436,6 +444,7 @@ ACTOR static Future<Void> loadFilesOnLoaders(Reference<ControllerBatchData> batc
|
|||
: std::min(versionBatch.endVersion, request.targetVersion + 1);
|
||||
param.asset.addPrefix = request.addPrefix;
|
||||
param.asset.removePrefix = request.removePrefix;
|
||||
param.asset.batchIndex = batchIndex;
|
||||
|
||||
TraceEvent("FastRestoreControllerPhaseLoadFiles")
|
||||
.detail("BatchIndex", batchIndex)
|
||||
|
|
|
@ -149,6 +149,10 @@ struct RestoreControllerData : RestoreRoleData, public ReferenceCounted<RestoreC
|
|||
|
||||
std::map<UID, double> rolesHeartBeatTime; // Key: role id; Value: most recent time controller receives heart beat
|
||||
|
||||
// addActor: add to actorCollection so that when an actor has error, the ActorCollection can catch the error.
|
||||
// addActor is used to create the actorCollection when the RestoreController is created
|
||||
PromiseStream<Future<Void>> addActor;
|
||||
|
||||
void addref() { return ReferenceCounted<RestoreControllerData>::addref(); }
|
||||
void delref() { return ReferenceCounted<RestoreControllerData>::delref(); }
|
||||
|
||||
|
@ -217,6 +221,7 @@ struct RestoreControllerData : RestoreRoleData, public ReferenceCounted<RestoreC
|
|||
}
|
||||
|
||||
TraceEvent("FastRestoreVersionBatchesSummary")
|
||||
.detail("VersionBatches", versionBatches.size())
|
||||
.detail("LogFiles", logFiles)
|
||||
.detail("RangeFiles", rangeFiles)
|
||||
.detail("LogBytes", logSize)
|
||||
|
@ -310,6 +315,7 @@ struct RestoreControllerData : RestoreRoleData, public ReferenceCounted<RestoreC
|
|||
}
|
||||
}
|
||||
} else {
|
||||
// TODO: Check why this may happen?!
|
||||
TraceEvent(SevError, "FastRestoreBuildVersionBatch")
|
||||
.detail("RangeIndex", rangeIdx)
|
||||
.detail("RangeFiles", rangeFiles.size())
|
||||
|
|
|
@ -50,9 +50,11 @@ void handleRestoreSysInfoRequest(const RestoreSysInfoRequest& req, Reference<Res
|
|||
ACTOR Future<Void> handleLoadFileRequest(RestoreLoadFileRequest req, Reference<RestoreLoaderData> self);
|
||||
ACTOR Future<Void> handleSendMutationsRequest(RestoreSendMutationsToAppliersRequest req,
|
||||
Reference<RestoreLoaderData> self);
|
||||
ACTOR Future<Void> sendMutationsToApplier(VersionedMutationsMap* pkvOps, int batchIndex, RestoreAsset asset,
|
||||
bool isRangeFile, std::map<Key, UID>* pRangeToApplier,
|
||||
std::map<UID, RestoreApplierInterface>* pApplierInterfaces);
|
||||
ACTOR Future<Void> sendMutationsToApplier(
|
||||
std::priority_queue<RestoreLoaderSchedSendLoadParamRequest>* sendLoadParamQueue,
|
||||
std::map<int, int>* inflightSendLoadParamReqs, NotifiedVersion* finishedBatch, VersionedMutationsMap* pkvOps,
|
||||
int batchIndex, RestoreAsset asset, bool isRangeFile, std::map<Key, UID>* pRangeToApplier,
|
||||
std::map<UID, RestoreApplierInterface>* pApplierInterfaces);
|
||||
ACTOR static Future<Void> _parseLogFileToMutationsOnLoader(NotifiedVersion* pProcessedFileOffset,
|
||||
SerializedMutationListMap* mutationMap,
|
||||
Reference<IBackupContainer> bc, RestoreAsset asset);
|
||||
|
@ -62,16 +64,169 @@ ACTOR static Future<Void> _parseRangeFileToMutationsOnLoader(
|
|||
Reference<IBackupContainer> bc, Version version, RestoreAsset asset);
|
||||
ACTOR Future<Void> handleFinishVersionBatchRequest(RestoreVersionBatchRequest req, Reference<RestoreLoaderData> self);
|
||||
|
||||
// Dispatch requests based on node's business (i.e, cpu usage for now) and requests' priorities
|
||||
// Requests for earlier version batches are preferred; which is equivalent to
|
||||
// sendMuttionsRequests are preferred than loadingFileRequests
|
||||
ACTOR Future<Void> dispatchRequests(Reference<RestoreLoaderData> self) {
|
||||
try {
|
||||
state int curVBInflightReqs = 0;
|
||||
state int sendLoadParams = 0;
|
||||
state int lastLoadReqs = 0;
|
||||
loop {
|
||||
TraceEvent(SevDebug, "FastRestoreLoaderDispatchRequests", self->id())
|
||||
.detail("SendingQueue", self->sendingQueue.size())
|
||||
.detail("LoadingQueue", self->loadingQueue.size())
|
||||
.detail("SendingLoadParamQueue", self->sendLoadParamQueue.size())
|
||||
.detail("InflightSendingReqs", self->inflightSendingReqs)
|
||||
.detail("InflightSendingReqsThreshold", SERVER_KNOBS->FASTRESTORE_SCHED_INFLIGHT_SEND_REQS)
|
||||
.detail("InflightLoadingReqs", self->inflightLoadingReqs)
|
||||
.detail("InflightLoadingReqsThreshold", SERVER_KNOBS->FASTRESTORE_SCHED_INFLIGHT_LOAD_REQS)
|
||||
.detail("LastLoadFileRequests", lastLoadReqs)
|
||||
.detail("LoadFileRequestsBatchThreshold", SERVER_KNOBS->FASTRESTORE_SCHED_LOAD_REQ_BATCHSIZE)
|
||||
.detail("LastDispatchSendLoadParamReqsForCurrentVB", curVBInflightReqs)
|
||||
.detail("LastDispatchSendLoadParamReqsForFutureVB", sendLoadParams)
|
||||
.detail("CpuUsage", self->cpuUsage)
|
||||
.detail("TargetCpuUsage", SERVER_KNOBS->FASTRESTORE_SCHED_TARGET_CPU_PERCENT)
|
||||
.detail("MaxCpuUsage", SERVER_KNOBS->FASTRESTORE_SCHED_MAX_CPU_PERCENT);
|
||||
|
||||
// TODO: Pop old requests whose version batch <= finishedBatch.get()
|
||||
// TODO2: Simulate delayed request can be too old by introducing artificial delay
|
||||
if (SERVER_KNOBS->FASTRESTORE_EXPENSIVE_VALIDATION) {
|
||||
// Sanity check: All requests before and in finishedBatch must have been processed; otherwise,
|
||||
// those requests may cause segmentation fault after applier remove the batch data
|
||||
if (!self->loadingQueue.empty() && self->loadingQueue.top().batchIndex <= self->finishedBatch.get()) {
|
||||
// Still has pending requests from earlier batchIndex and current batchIndex, which should not
|
||||
// happen
|
||||
TraceEvent(SevError, "FastRestoreLoaderSchedulerHasOldLoadFileRequests")
|
||||
.detail("FinishedBatchIndex", self->finishedBatch.get())
|
||||
.detail("PendingRequest", self->loadingQueue.top().toString());
|
||||
}
|
||||
if (!self->sendingQueue.empty() && self->sendingQueue.top().batchIndex <= self->finishedBatch.get()) {
|
||||
TraceEvent(SevError, "FastRestoreLoaderSchedulerHasOldSendRequests")
|
||||
.detail("FinishedBatchIndex", self->finishedBatch.get())
|
||||
.detail("PendingRequest", self->sendingQueue.top().toString());
|
||||
}
|
||||
if (!self->sendLoadParamQueue.empty() &&
|
||||
self->sendLoadParamQueue.top().batchIndex <= self->finishedBatch.get()) {
|
||||
TraceEvent(SevError, "FastRestoreLoaderSchedulerHasOldSendLoadParamRequests")
|
||||
.detail("FinishedBatchIndex", self->finishedBatch.get())
|
||||
.detail("PendingRequest", self->sendLoadParamQueue.top().toString());
|
||||
}
|
||||
}
|
||||
|
||||
if (!self->sendingQueue.empty()) {
|
||||
// Only release one sendMutationRequest at a time because it sends all data for a version batch
|
||||
// and it takes large amount of resource
|
||||
const RestoreSendMutationsToAppliersRequest& req = self->sendingQueue.top();
|
||||
// Dispatch the request if it is the next version batch to process or if cpu usage is low
|
||||
if (req.batchIndex - 1 == self->finishedSendingVB ||
|
||||
self->cpuUsage < SERVER_KNOBS->FASTRESTORE_SCHED_TARGET_CPU_PERCENT) {
|
||||
self->addActor.send(handleSendMutationsRequest(req, self));
|
||||
self->sendingQueue.pop();
|
||||
}
|
||||
}
|
||||
// When shall the node pause the process of other requests, e.g., load file requests
|
||||
// TODO: Revisit if we should have (self->inflightSendingReqs > 0 && self->inflightLoadingReqs > 0)
|
||||
if ((self->inflightSendingReqs > 0 && self->inflightLoadingReqs > 0) &&
|
||||
(self->inflightSendingReqs >= SERVER_KNOBS->FASTRESTORE_SCHED_INFLIGHT_SEND_REQS ||
|
||||
self->inflightLoadingReqs >= SERVER_KNOBS->FASTRESTORE_SCHED_INFLIGHT_LOAD_REQS ||
|
||||
(self->inflightSendingReqs >= 1 &&
|
||||
self->cpuUsage >= SERVER_KNOBS->FASTRESTORE_SCHED_TARGET_CPU_PERCENT) ||
|
||||
self->cpuUsage >= SERVER_KNOBS->FASTRESTORE_SCHED_MAX_CPU_PERCENT)) {
|
||||
if (self->inflightSendingReqs >= SERVER_KNOBS->FASTRESTORE_SCHED_INFLIGHT_SEND_REQS) {
|
||||
TraceEvent(SevWarn, "FastRestoreLoaderTooManyInflightRequests")
|
||||
.detail("VersionBatchesBlockedAtSendingMutationsToAppliers", self->inflightSendingReqs)
|
||||
.detail("CpuUsage", self->cpuUsage)
|
||||
.detail("InflightSendingReq", self->inflightSendingReqs)
|
||||
.detail("InflightSendingReqThreshold", SERVER_KNOBS->FASTRESTORE_SCHED_INFLIGHT_SEND_REQS)
|
||||
.detail("InflightLoadingReq", self->inflightLoadingReqs)
|
||||
.detail("InflightLoadingReqThreshold", SERVER_KNOBS->FASTRESTORE_SCHED_INFLIGHT_LOAD_REQS);
|
||||
}
|
||||
wait(delay(SERVER_KNOBS->FASTRESTORE_SCHED_UPDATE_DELAY));
|
||||
updateProcessStats(self);
|
||||
continue;
|
||||
}
|
||||
// Dispatch queued requests of sending mutations per loading param
|
||||
while (!self->sendLoadParamQueue.empty()) { // dispatch current VB first
|
||||
const RestoreLoaderSchedSendLoadParamRequest& req = self->sendLoadParamQueue.top();
|
||||
if (req.batchIndex - 1 > self->finishedSendingVB) { // future VB
|
||||
break;
|
||||
} else {
|
||||
req.toSched.send(Void());
|
||||
self->sendLoadParamQueue.pop();
|
||||
}
|
||||
}
|
||||
sendLoadParams = 0;
|
||||
curVBInflightReqs = self->inflightSendLoadParamReqs[self->finishedSendingVB + 1];
|
||||
while (!self->sendLoadParamQueue.empty()) {
|
||||
const RestoreLoaderSchedSendLoadParamRequest& req = self->sendLoadParamQueue.top();
|
||||
if (curVBInflightReqs >= SERVER_KNOBS->FASTRESTORE_SCHED_INFLIGHT_SENDPARAM_THRESHOLD ||
|
||||
sendLoadParams >= SERVER_KNOBS->FASTRESTORE_SCHED_SEND_FUTURE_VB_REQS_BATCH) {
|
||||
// Too many future VB requests are released
|
||||
break;
|
||||
} else {
|
||||
req.toSched.send(Void());
|
||||
self->sendLoadParamQueue.pop();
|
||||
sendLoadParams++;
|
||||
}
|
||||
}
|
||||
|
||||
// Dispatch loading backup file requests
|
||||
lastLoadReqs = 0;
|
||||
while (!self->loadingQueue.empty()) {
|
||||
if (lastLoadReqs >= SERVER_KNOBS->FASTRESTORE_SCHED_LOAD_REQ_BATCHSIZE) {
|
||||
break;
|
||||
}
|
||||
const RestoreLoadFileRequest& req = self->loadingQueue.top();
|
||||
if (req.batchIndex <= self->finishedBatch.get()) {
|
||||
TraceEvent(SevError, "FastRestoreLoaderDispatchRestoreLoadFileRequestTooOld")
|
||||
.detail("FinishedBatchIndex", self->finishedBatch.get())
|
||||
.detail("RequestBatchIndex", req.batchIndex);
|
||||
req.reply.send(RestoreLoadFileReply(req.param, true));
|
||||
self->loadingQueue.pop();
|
||||
ASSERT(false); // Check if this ever happens easily
|
||||
} else {
|
||||
self->addActor.send(handleLoadFileRequest(req, self));
|
||||
self->loadingQueue.pop();
|
||||
lastLoadReqs++;
|
||||
}
|
||||
}
|
||||
|
||||
if (self->cpuUsage >= SERVER_KNOBS->FASTRESTORE_SCHED_TARGET_CPU_PERCENT) {
|
||||
wait(delay(SERVER_KNOBS->FASTRESTORE_SCHED_UPDATE_DELAY));
|
||||
}
|
||||
updateProcessStats(self);
|
||||
|
||||
if (self->loadingQueue.empty() && self->sendingQueue.empty() && self->sendLoadParamQueue.empty()) {
|
||||
TraceEvent(SevDebug, "FastRestoreLoaderDispatchRequestsWaitOnRequests", self->id())
|
||||
.detail("HasPendingRequests", self->hasPendingRequests->get());
|
||||
self->hasPendingRequests->set(false);
|
||||
wait(self->hasPendingRequests->onChange()); // CAREFUL:Improper req release may cause restore stuck here
|
||||
}
|
||||
}
|
||||
} catch (Error& e) {
|
||||
if (e.code() != error_code_actor_cancelled) {
|
||||
TraceEvent(SevError, "FastRestoreLoaderDispatchRequests").error(e, true);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
return Void();
|
||||
}
|
||||
|
||||
ACTOR Future<Void> restoreLoaderCore(RestoreLoaderInterface loaderInterf, int nodeIndex, Database cx,
|
||||
RestoreControllerInterface ci) {
|
||||
state Reference<RestoreLoaderData> self =
|
||||
Reference<RestoreLoaderData>(new RestoreLoaderData(loaderInterf.id(), nodeIndex, ci));
|
||||
state ActorCollection actors(false);
|
||||
state Future<Void> error = actorCollection(self->addActor.getFuture());
|
||||
state ActorCollection actors(false); // actors whose errors can be ignored
|
||||
state Future<Void> exitRole = Never();
|
||||
state bool hasQueuedRequests = false;
|
||||
|
||||
actors.add(updateProcessMetrics(self));
|
||||
actors.add(traceProcessMetrics(self, "RestoreLoader"));
|
||||
|
||||
self->addActor.send(dispatchRequests(self));
|
||||
|
||||
loop {
|
||||
state std::string requestTypeStr = "[Init]";
|
||||
|
||||
|
@ -87,12 +242,20 @@ ACTOR Future<Void> restoreLoaderCore(RestoreLoaderInterface loaderInterf, int no
|
|||
}
|
||||
when(RestoreLoadFileRequest req = waitNext(loaderInterf.loadFile.getFuture())) {
|
||||
requestTypeStr = "loadFile";
|
||||
hasQueuedRequests = !self->loadingQueue.empty() || !self->sendingQueue.empty();
|
||||
self->initBackupContainer(req.param.url);
|
||||
actors.add(handleLoadFileRequest(req, self));
|
||||
self->loadingQueue.push(req);
|
||||
if (!hasQueuedRequests) {
|
||||
self->hasPendingRequests->set(true);
|
||||
}
|
||||
}
|
||||
when(RestoreSendMutationsToAppliersRequest req = waitNext(loaderInterf.sendMutations.getFuture())) {
|
||||
requestTypeStr = "sendMutations";
|
||||
actors.add(handleSendMutationsRequest(req, self));
|
||||
hasQueuedRequests = !self->loadingQueue.empty() || !self->sendingQueue.empty();
|
||||
self->sendingQueue.push(req);
|
||||
if (!hasQueuedRequests) {
|
||||
self->hasPendingRequests->set(true);
|
||||
}
|
||||
}
|
||||
when(RestoreVersionBatchRequest req = waitNext(loaderInterf.initVersionBatch.getFuture())) {
|
||||
requestTypeStr = "initVersionBatch";
|
||||
|
@ -114,6 +277,7 @@ ACTOR Future<Void> restoreLoaderCore(RestoreLoaderInterface loaderInterf, int no
|
|||
TraceEvent("FastRestoreLoaderCoreExitRole", self->id());
|
||||
break;
|
||||
}
|
||||
when(wait(error)) { TraceEvent("FastRestoreLoaderActorCollectionError", self->id()); }
|
||||
}
|
||||
} catch (Error& e) {
|
||||
TraceEvent(e.code() == error_code_broken_promise ? SevError : SevWarnAlways, "FastRestoreLoaderError",
|
||||
|
@ -190,8 +354,9 @@ ACTOR static Future<Void> _parsePartitionedLogFileOnLoader(
|
|||
int rLen = wait(file->read(mutateString(buf), asset.len, asset.offset));
|
||||
if (rLen != asset.len) throw restore_bad_read();
|
||||
|
||||
TraceEvent("FastRestoreLoader")
|
||||
.detail("DecodingLogFile", asset.filename)
|
||||
TraceEvent("FastRestoreLoaderDecodingLogFile")
|
||||
.detail("BatchIndex", asset.batchIndex)
|
||||
.detail("Filename", asset.filename)
|
||||
.detail("Offset", asset.offset)
|
||||
.detail("Length", asset.len);
|
||||
|
||||
|
@ -285,6 +450,7 @@ ACTOR static Future<Void> _parsePartitionedLogFileOnLoader(
|
|||
} catch (Error& e) {
|
||||
TraceEvent(SevWarn, "FileRestoreCorruptLogFileBlock")
|
||||
.error(e)
|
||||
.detail("BatchIndex", asset.batchIndex)
|
||||
.detail("Filename", file->getFilename())
|
||||
.detail("BlockOffset", asset.offset)
|
||||
.detail("BlockLen", asset.len);
|
||||
|
@ -306,8 +472,9 @@ ACTOR Future<Void> _processLoadingParam(KeyRangeMap<Version>* pRangeVersions, Lo
|
|||
state std::map<LoadingParam, VersionedMutationsMap>::iterator kvOpsPerLPIter = batchData->kvOpsPerLP.end();
|
||||
state std::map<LoadingParam, SampledMutationsVec>::iterator samplesIter = batchData->sampleMutations.end();
|
||||
|
||||
// Q: How to record the param's fields inside LoadingParam Refer to storageMetrics
|
||||
TraceEvent("FastRestoreLoaderProcessLoadingParam", loaderID).detail("LoadingParam", param.toString());
|
||||
TraceEvent("FastRestoreLoaderProcessLoadingParam", loaderID)
|
||||
.detail("BatchIndex", param.asset.batchIndex)
|
||||
.detail("LoadingParam", param.toString());
|
||||
ASSERT(param.blockSize > 0);
|
||||
ASSERT(param.asset.offset % param.blockSize == 0); // Parse file must be at block boundary.
|
||||
ASSERT(batchData->kvOpsPerLP.find(param) == batchData->kvOpsPerLP.end());
|
||||
|
@ -345,7 +512,9 @@ ACTOR Future<Void> _processLoadingParam(KeyRangeMap<Version>* pRangeVersions, Lo
|
|||
param.asset);
|
||||
}
|
||||
|
||||
TraceEvent("FastRestoreLoaderProcessLoadingParamDone", loaderID).detail("LoadingParam", param.toString());
|
||||
TraceEvent("FastRestoreLoaderProcessLoadingParamDone", loaderID)
|
||||
.detail("BatchIndex", param.asset.batchIndex)
|
||||
.detail("LoadingParam", param.toString());
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
@ -356,6 +525,7 @@ ACTOR Future<Void> handleLoadFileRequest(RestoreLoadFileRequest req, Reference<R
|
|||
state bool isDuplicated = true;
|
||||
state bool printTrace = false;
|
||||
ASSERT(batchData.isValid());
|
||||
ASSERT(req.batchIndex > self->finishedBatch.get());
|
||||
bool paramExist = batchData->processedFileParams.find(req.param) != batchData->processedFileParams.end();
|
||||
bool isReady = paramExist ? batchData->processedFileParams[req.param].isReady() : false;
|
||||
|
||||
|
@ -380,6 +550,7 @@ ACTOR Future<Void> handleLoadFileRequest(RestoreLoadFileRequest req, Reference<R
|
|||
ASSERT(batchData->sampleMutations.find(req.param) == batchData->sampleMutations.end());
|
||||
batchData->processedFileParams[req.param] =
|
||||
_processLoadingParam(&self->rangeVersions, req.param, batchData, self->id(), self->bc);
|
||||
self->inflightLoadingReqs++;
|
||||
isDuplicated = false;
|
||||
} else {
|
||||
TraceEvent(SevFRDebugInfo, "FastRestoreLoadFile", self->id())
|
||||
|
@ -424,6 +595,7 @@ ACTOR Future<Void> handleLoadFileRequest(RestoreLoadFileRequest req, Reference<R
|
|||
}
|
||||
|
||||
// Ack restore controller the param is processed
|
||||
self->inflightLoadingReqs--;
|
||||
req.reply.send(RestoreLoadFileReply(req.param, isDuplicated));
|
||||
TraceEvent(printTrace ? SevInfo : SevFRDebugInfo, "FastRestoreLoaderPhaseLoadFileDone", self->id())
|
||||
.detail("BatchIndex", req.batchIndex)
|
||||
|
@ -436,16 +608,29 @@ ACTOR Future<Void> handleLoadFileRequest(RestoreLoadFileRequest req, Reference<R
|
|||
// Do not need to block on low memory usage because this actor should not increase memory usage.
|
||||
ACTOR Future<Void> handleSendMutationsRequest(RestoreSendMutationsToAppliersRequest req,
|
||||
Reference<RestoreLoaderData> self) {
|
||||
state Reference<LoaderBatchData> batchData = self->batch[req.batchIndex];
|
||||
state Reference<LoaderBatchStatus> batchStatus = self->status[req.batchIndex];
|
||||
state Reference<LoaderBatchData> batchData;
|
||||
state Reference<LoaderBatchStatus> batchStatus;
|
||||
state bool isDuplicated = true;
|
||||
|
||||
if (req.batchIndex <= self->finishedBatch.get()) {
|
||||
TraceEvent(SevWarn, "FastRestoreLoaderRestoreSendMutationsToAppliersRequestTooOld")
|
||||
.detail("FinishedBatchIndex", self->finishedBatch.get())
|
||||
.detail("RequestBatchIndex", req.batchIndex);
|
||||
req.reply.send(RestoreCommonReply(self->id(), isDuplicated));
|
||||
return Void();
|
||||
}
|
||||
|
||||
batchData = self->batch[req.batchIndex];
|
||||
batchStatus = self->status[req.batchIndex];
|
||||
ASSERT(batchData.isValid() && batchStatus.isValid());
|
||||
// Loader destroy batchData once the batch finishes and self->finishedBatch.set(req.batchIndex);
|
||||
ASSERT(req.batchIndex > self->finishedBatch.get());
|
||||
TraceEvent("FastRestoreLoaderPhaseSendMutations", self->id())
|
||||
.detail("BatchIndex", req.batchIndex)
|
||||
.detail("UseRangeFile", req.useRangeFile)
|
||||
.detail("LoaderSendStatus", batchStatus->toString());
|
||||
// Loader destroy batchData once the batch finishes and self->finishedBatch.set(req.batchIndex);
|
||||
ASSERT(self->finishedBatch.get() < req.batchIndex);
|
||||
// The VB must finish loading phase before it can send mutations; update finishedLoadingVB for scheduler
|
||||
self->finishedLoadingVB = std::max(self->finishedLoadingVB, req.batchIndex);
|
||||
|
||||
// Ensure each file is sent exactly once by using batchStatus->sendAllLogs and batchStatus->sendAllRanges
|
||||
if (!req.useRangeFile) {
|
||||
|
@ -485,17 +670,20 @@ ACTOR Future<Void> handleSendMutationsRequest(RestoreSendMutationsToAppliersRequ
|
|||
}
|
||||
|
||||
if (!isDuplicated) {
|
||||
self->inflightSendingReqs++;
|
||||
vector<Future<Void>> fSendMutations;
|
||||
batchData->rangeToApplier = req.rangeToApplier;
|
||||
for (auto& [loadParam, kvOps] : batchData->kvOpsPerLP) {
|
||||
if (loadParam.isRangeFile == req.useRangeFile) {
|
||||
// Send the parsed mutation to applier who will apply the mutation to DB
|
||||
fSendMutations.push_back(sendMutationsToApplier(&kvOps, req.batchIndex, loadParam.asset,
|
||||
loadParam.isRangeFile, &batchData->rangeToApplier,
|
||||
&self->appliersInterf));
|
||||
fSendMutations.push_back(
|
||||
sendMutationsToApplier(&self->sendLoadParamQueue, &self->inflightSendLoadParamReqs,
|
||||
&self->finishedBatch, &kvOps, req.batchIndex, loadParam.asset,
|
||||
loadParam.isRangeFile, &batchData->rangeToApplier, &self->appliersInterf));
|
||||
}
|
||||
}
|
||||
wait(waitForAll(fSendMutations));
|
||||
self->inflightSendingReqs--;
|
||||
if (req.useRangeFile) {
|
||||
batchStatus->sendAllRanges = Void(); // Finish sending kvs parsed from range files
|
||||
} else {
|
||||
|
@ -504,6 +692,7 @@ ACTOR Future<Void> handleSendMutationsRequest(RestoreSendMutationsToAppliersRequ
|
|||
if ((batchStatus->sendAllRanges.present() && batchStatus->sendAllRanges.get().isReady()) &&
|
||||
(batchStatus->sendAllLogs.present() && batchStatus->sendAllLogs.get().isReady())) {
|
||||
// Both log and range files have been sent.
|
||||
self->finishedSendingVB = std::max(self->finishedSendingVB, req.batchIndex);
|
||||
batchData->kvOpsPerLP.clear();
|
||||
}
|
||||
}
|
||||
|
@ -534,9 +723,11 @@ void buildApplierRangeMap(KeyRangeMap<UID>* krMap, std::map<Key, UID>* pRangeToA
|
|||
// isRangeFile: is pkvOps from range file? Let receiver (applier) know if the mutation is log mutation;
|
||||
// pRangeToApplier: range to applierID mapping, deciding which applier is responsible for which range
|
||||
// pApplierInterfaces: applier interfaces to send the mutations to
|
||||
ACTOR Future<Void> sendMutationsToApplier(VersionedMutationsMap* pkvOps, int batchIndex, RestoreAsset asset,
|
||||
bool isRangeFile, std::map<Key, UID>* pRangeToApplier,
|
||||
std::map<UID, RestoreApplierInterface>* pApplierInterfaces) {
|
||||
ACTOR Future<Void> sendMutationsToApplier(
|
||||
std::priority_queue<RestoreLoaderSchedSendLoadParamRequest>* sendLoadParamQueue,
|
||||
std::map<int, int>* inflightSendLoadParamReqs, NotifiedVersion* finishedBatch, VersionedMutationsMap* pkvOps,
|
||||
int batchIndex, RestoreAsset asset, bool isRangeFile, std::map<Key, UID>* pRangeToApplier,
|
||||
std::map<UID, RestoreApplierInterface>* pApplierInterfaces) {
|
||||
state VersionedMutationsMap& kvOps = *pkvOps;
|
||||
state VersionedMutationsMap::iterator kvOp = kvOps.begin();
|
||||
state int kvCount = 0;
|
||||
|
@ -545,6 +736,20 @@ ACTOR Future<Void> sendMutationsToApplier(VersionedMutationsMap* pkvOps, int bat
|
|||
state std::vector<UID> applierIDs = getApplierIDs(*pRangeToApplier);
|
||||
state double msgSize = 0; // size of mutations in the message
|
||||
|
||||
// Wait for scheduler to kick it off
|
||||
Promise<Void> toSched;
|
||||
sendLoadParamQueue->push(RestoreLoaderSchedSendLoadParamRequest(batchIndex, toSched, now()));
|
||||
wait(toSched.getFuture());
|
||||
if (finishedBatch->get() >= batchIndex) {
|
||||
TraceEvent(SevError, "FastRestoreLoaderSendMutationToApplierLateRequest")
|
||||
.detail("FinishedBatchIndex", finishedBatch->get())
|
||||
.detail("RequestBatchIndex", batchIndex);
|
||||
ASSERT(false);
|
||||
return Void();
|
||||
}
|
||||
|
||||
(*inflightSendLoadParamReqs)[batchIndex]++;
|
||||
|
||||
TraceEvent("FastRestoreLoaderSendMutationToApplier")
|
||||
.detail("IsRangeFile", isRangeFile)
|
||||
.detail("EndVersion", asset.endVersion)
|
||||
|
@ -641,7 +846,7 @@ ACTOR Future<Void> sendMutationsToApplier(VersionedMutationsMap* pkvOps, int bat
|
|||
applierID, RestoreSendVersionedMutationsRequest(batchIndex, asset, msgIndex, isRangeFile,
|
||||
applierVersionedMutationsBuffer[applierID]));
|
||||
}
|
||||
TraceEvent(SevDebug, "FastRestoreLoaderSendMutationToApplier")
|
||||
TraceEvent(SevInfo, "FastRestoreLoaderSendMutationToApplier")
|
||||
.detail("MessageIndex", msgIndex)
|
||||
.detail("RestoreAsset", asset.toString())
|
||||
.detail("Requests", requests.size());
|
||||
|
@ -665,7 +870,7 @@ ACTOR Future<Void> sendMutationsToApplier(VersionedMutationsMap* pkvOps, int bat
|
|||
RestoreSendVersionedMutationsRequest(batchIndex, asset, msgIndex, isRangeFile,
|
||||
applierVersionedMutationsBuffer[applierID]));
|
||||
}
|
||||
TraceEvent(SevDebug, "FastRestoreLoaderSendMutationToApplier")
|
||||
TraceEvent(SevInfo, "FastRestoreLoaderSendMutationToApplier")
|
||||
.detail("MessageIndex", msgIndex)
|
||||
.detail("RestoreAsset", asset.toString())
|
||||
.detail("Requests", requests.size());
|
||||
|
@ -674,11 +879,22 @@ ACTOR Future<Void> sendMutationsToApplier(VersionedMutationsMap* pkvOps, int bat
|
|||
}
|
||||
wait(waitForAll(fSends));
|
||||
|
||||
kvOps = VersionedMutationsMap(); // Free memory for parsed mutations at the restore asset.
|
||||
TraceEvent("FastRestoreLoaderSendMutationToAppliers")
|
||||
.detail("BatchIndex", batchIndex)
|
||||
.detail("RestoreAsset", asset.toString())
|
||||
.detail("Mutations", kvCount);
|
||||
(*inflightSendLoadParamReqs)[batchIndex]--;
|
||||
|
||||
if (finishedBatch->get() < batchIndex) {
|
||||
kvOps = VersionedMutationsMap(); // Free memory for parsed mutations at the restore asset.
|
||||
TraceEvent("FastRestoreLoaderSendMutationToApplierDone")
|
||||
.detail("BatchIndex", batchIndex)
|
||||
.detail("RestoreAsset", asset.toString())
|
||||
.detail("Mutations", kvCount);
|
||||
} else {
|
||||
TraceEvent(SevWarnAlways, "FastRestoreLoaderSendMutationToApplierDoneTooLate")
|
||||
.detail("BatchIndex", batchIndex)
|
||||
.detail("FinishedBatchIndex", finishedBatch->get())
|
||||
.detail("RestoreAsset", asset.toString())
|
||||
.detail("Mutations", kvCount);
|
||||
}
|
||||
|
||||
return Void();
|
||||
}
|
||||
|
||||
|
@ -779,6 +995,7 @@ void _parseSerializedMutation(KeyRangeMap<Version>* pRangeVersions,
|
|||
SerializedMutationListMap& mutationMap = *pmutationMap;
|
||||
|
||||
TraceEvent(SevFRMutationInfo, "FastRestoreLoaderParseSerializedLogMutation")
|
||||
.detail("BatchIndex", asset.batchIndex)
|
||||
.detail("RestoreAsset", asset.toString());
|
||||
|
||||
Arena tempArena;
|
||||
|
@ -881,6 +1098,7 @@ ACTOR static Future<Void> _parseRangeFileToMutationsOnLoader(
|
|||
state SampledMutationsVec& sampleMutations = samplesIter->second;
|
||||
|
||||
TraceEvent(SevFRDebugInfo, "FastRestoreDecodedRangeFile")
|
||||
.detail("BatchIndex", asset.batchIndex)
|
||||
.detail("Filename", asset.filename)
|
||||
.detail("Version", version)
|
||||
.detail("BeginVersion", asset.beginVersion)
|
||||
|
@ -895,8 +1113,9 @@ ACTOR static Future<Void> _parseRangeFileToMutationsOnLoader(
|
|||
try {
|
||||
Standalone<VectorRef<KeyValueRef>> kvs =
|
||||
wait(fileBackup::decodeRangeFileBlock(inFile, asset.offset, asset.len));
|
||||
TraceEvent("FastRestoreLoader")
|
||||
.detail("DecodedRangeFile", asset.filename)
|
||||
TraceEvent("FastRestoreLoaderDecodedRangeFile")
|
||||
.detail("BatchIndex", asset.batchIndex)
|
||||
.detail("Filename", asset.filename)
|
||||
.detail("DataSize", kvs.contents().size());
|
||||
blockData = kvs;
|
||||
} catch (Error& e) {
|
||||
|
@ -951,6 +1170,7 @@ ACTOR static Future<Void> _parseRangeFileToMutationsOnLoader(
|
|||
// We cache all kv operations into kvOps, and apply all kv operations later in one place
|
||||
auto it = kvOps.insert(std::make_pair(msgVersion, MutationsVec()));
|
||||
TraceEvent(SevFRMutationInfo, "FastRestoreDecodeRangeFile")
|
||||
.detail("BatchIndex", asset.batchIndex)
|
||||
.detail("CommitVersion", version)
|
||||
.detail("ParsedMutationKV", m.toString());
|
||||
|
||||
|
@ -980,6 +1200,7 @@ ACTOR static Future<Void> _parseLogFileToMutationsOnLoader(NotifiedVersion* pPro
|
|||
state Standalone<VectorRef<KeyValueRef>> data =
|
||||
wait(parallelFileRestore::decodeLogFileBlock(inFile, asset.offset, asset.len));
|
||||
TraceEvent("FastRestoreLoaderDecodeLogFile")
|
||||
.detail("BatchIndex", asset.batchIndex)
|
||||
.detail("RestoreAsset", asset.toString())
|
||||
.detail("DataSize", data.contents().size());
|
||||
|
||||
|
@ -1017,6 +1238,25 @@ ACTOR Future<Void> handleFinishVersionBatchRequest(RestoreVersionBatchRequest re
|
|||
.detail("RequestedBatchIndex", req.batchIndex);
|
||||
wait(self->finishedBatch.whenAtLeast(req.batchIndex - 1));
|
||||
if (self->finishedBatch.get() == req.batchIndex - 1) {
|
||||
// Sanity check: All requests before and in this batchIndex must have been processed; otherwise,
|
||||
// those requests may cause segmentation fault after applier remove the batch data
|
||||
while (!self->loadingQueue.empty() && self->loadingQueue.top().batchIndex <= req.batchIndex) {
|
||||
// Still has pending requests from earlier batchIndex and current batchIndex, which should not happen
|
||||
TraceEvent(SevWarn, "FastRestoreLoaderHasPendingLoadFileRequests")
|
||||
.detail("PendingRequest", self->loadingQueue.top().toString());
|
||||
self->loadingQueue.pop();
|
||||
}
|
||||
while (!self->sendingQueue.empty() && self->sendingQueue.top().batchIndex <= req.batchIndex) {
|
||||
TraceEvent(SevWarn, "FastRestoreLoaderHasPendingSendRequests")
|
||||
.detail("PendingRequest", self->sendingQueue.top().toString());
|
||||
self->sendingQueue.pop();
|
||||
}
|
||||
while (!self->sendLoadParamQueue.empty() && self->sendLoadParamQueue.top().batchIndex <= req.batchIndex) {
|
||||
TraceEvent(SevWarn, "FastRestoreLoaderHasPendingSendLoadParamRequests")
|
||||
.detail("PendingRequest", self->sendLoadParamQueue.top().toString());
|
||||
self->sendLoadParamQueue.pop();
|
||||
}
|
||||
|
||||
self->finishedBatch.set(req.batchIndex);
|
||||
// Clean up batchData
|
||||
self->batch.erase(req.batchIndex);
|
||||
|
|
|
@ -93,7 +93,8 @@ struct LoaderBatchData : public ReferenceCounted<LoaderBatchData> {
|
|||
oldLogMutations("OldLogMutations", cc) {}
|
||||
} counters;
|
||||
|
||||
explicit LoaderBatchData(UID nodeID, int batchIndex) : counters(this, nodeID, batchIndex), vbState(LoaderVersionBatchState::NOT_INIT) {
|
||||
explicit LoaderBatchData(UID nodeID, int batchIndex)
|
||||
: counters(this, nodeID, batchIndex), vbState(LoaderVersionBatchState::NOT_INIT), loadFileReqs(0) {
|
||||
pollMetrics = traceCounters(format("FastRestoreLoaderMetrics%d", batchIndex), nodeID,
|
||||
SERVER_KNOBS->FASTRESTORE_ROLE_LOGGING_DELAY, &counters.cc,
|
||||
nodeID.toString() + "/RestoreLoaderMetrics/" + std::to_string(batchIndex));
|
||||
|
@ -118,7 +119,7 @@ struct LoaderBatchStatus : public ReferenceCounted<LoaderBatchStatus> {
|
|||
void addref() { return ReferenceCounted<LoaderBatchStatus>::addref(); }
|
||||
void delref() { return ReferenceCounted<LoaderBatchStatus>::delref(); }
|
||||
|
||||
std::string toString() {
|
||||
std::string toString() const {
|
||||
std::stringstream ss;
|
||||
ss << "sendAllRanges: "
|
||||
<< (!sendAllRanges.present() ? "invalid" : (sendAllRanges.get().isReady() ? "ready" : "notReady"))
|
||||
|
@ -128,6 +129,29 @@ struct LoaderBatchStatus : public ReferenceCounted<LoaderBatchStatus> {
|
|||
}
|
||||
};
|
||||
|
||||
// Each request for each loadingParam, so that scheduler can control which requests in which version batch to send first
|
||||
struct RestoreLoaderSchedSendLoadParamRequest {
|
||||
int batchIndex;
|
||||
Promise<Void> toSched;
|
||||
double start;
|
||||
|
||||
explicit RestoreLoaderSchedSendLoadParamRequest(int batchIndex, Promise<Void> toSched, double start)
|
||||
: batchIndex(batchIndex), toSched(toSched), start(start){};
|
||||
RestoreLoaderSchedSendLoadParamRequest() = default;
|
||||
|
||||
bool operator<(RestoreLoaderSchedSendLoadParamRequest const& rhs) const {
|
||||
return batchIndex > rhs.batchIndex || (batchIndex == rhs.batchIndex && start > rhs.start);
|
||||
}
|
||||
|
||||
std::string toString() const {
|
||||
std::stringstream ss;
|
||||
ss << "RestoreLoaderSchedSendLoadParamRequest: "
|
||||
<< " batchIndex:" << batchIndex << " toSchedFutureIsReady:" << toSched.getFuture().isReady()
|
||||
<< " start:" << start;
|
||||
return ss.str();
|
||||
}
|
||||
};
|
||||
|
||||
struct RestoreLoaderData : RestoreRoleData, public ReferenceCounted<RestoreLoaderData> {
|
||||
// buffered data per version batch
|
||||
std::map<int, Reference<LoaderBatchData>> batch;
|
||||
|
@ -139,13 +163,32 @@ struct RestoreLoaderData : RestoreRoleData, public ReferenceCounted<RestoreLoade
|
|||
Reference<IBackupContainer> bc; // Backup container is used to read backup files
|
||||
Key bcUrl; // The url used to get the bc
|
||||
|
||||
// Request scheduler
|
||||
std::priority_queue<RestoreLoadFileRequest> loadingQueue; // request queue of loading files
|
||||
std::priority_queue<RestoreSendMutationsToAppliersRequest>
|
||||
sendingQueue; // request queue of sending mutations to appliers
|
||||
std::priority_queue<RestoreLoaderSchedSendLoadParamRequest> sendLoadParamQueue;
|
||||
int finishedLoadingVB; // the max version batch index that finished loading file phase
|
||||
int finishedSendingVB; // the max version batch index that finished sending mutations phase
|
||||
int inflightSendingReqs; // number of sendingMutations requests released
|
||||
int inflightLoadingReqs; // number of load backup file requests released
|
||||
std::map<int, int> inflightSendLoadParamReqs; // key: batchIndex, value: inflightSendLoadParamReqs
|
||||
|
||||
Reference<AsyncVar<bool>> hasPendingRequests; // are there pending requests for loader
|
||||
|
||||
// addActor: add to actorCollection so that when an actor has error, the ActorCollection can catch the error.
|
||||
// addActor is used to create the actorCollection when the RestoreController is created
|
||||
PromiseStream<Future<Void>> addActor;
|
||||
|
||||
void addref() { return ReferenceCounted<RestoreLoaderData>::addref(); }
|
||||
void delref() { return ReferenceCounted<RestoreLoaderData>::delref(); }
|
||||
|
||||
explicit RestoreLoaderData(UID loaderInterfID, int assignedIndex, RestoreControllerInterface ci) : ci(ci) {
|
||||
explicit RestoreLoaderData(UID loaderInterfID, int assignedIndex, RestoreControllerInterface ci)
|
||||
: ci(ci), finishedLoadingVB(0), finishedSendingVB(0), inflightSendingReqs(0), inflightLoadingReqs(0) {
|
||||
nodeID = loaderInterfID;
|
||||
nodeIndex = assignedIndex;
|
||||
role = RestoreRole::Loader;
|
||||
hasPendingRequests = Reference<AsyncVar<bool>>(new AsyncVar<bool>(false));
|
||||
}
|
||||
|
||||
~RestoreLoaderData() = default;
|
||||
|
|
|
@ -81,7 +81,17 @@ void updateProcessStats(Reference<RestoreRoleData> self) {
|
|||
if (g_network->isSimulated()) {
|
||||
// memUsage and cpuUsage are not relevant in the simulator,
|
||||
// and relying on the actual values could break seed determinism
|
||||
self->cpuUsage = 100.0;
|
||||
if (deterministicRandom()->random01() < 0.2) { // not fully utilized cpu
|
||||
self->cpuUsage = deterministicRandom()->random01() * SERVER_KNOBS->FASTRESTORE_SCHED_TARGET_CPU_PERCENT;
|
||||
} else if (deterministicRandom()->random01() < 0.6) { // achieved target cpu but cpu is not busy
|
||||
self->cpuUsage = SERVER_KNOBS->FASTRESTORE_SCHED_TARGET_CPU_PERCENT +
|
||||
deterministicRandom()->random01() * (SERVER_KNOBS->FASTRESTORE_SCHED_MAX_CPU_PERCENT -
|
||||
SERVER_KNOBS->FASTRESTORE_SCHED_TARGET_CPU_PERCENT);
|
||||
} else { // reach desired max cpu usage; use max cpu as 200 to simulate incorrect cpu profiling
|
||||
self->cpuUsage =
|
||||
SERVER_KNOBS->FASTRESTORE_SCHED_MAX_CPU_PERCENT +
|
||||
deterministicRandom()->random01() * (200 - SERVER_KNOBS->FASTRESTORE_SCHED_MAX_CPU_PERCENT);
|
||||
}
|
||||
self->memory = 100.0;
|
||||
self->residentMemory = 100.0;
|
||||
return;
|
||||
|
@ -109,7 +119,12 @@ ACTOR Future<Void> isSchedulable(Reference<RestoreRoleData> self, int actorBatch
|
|||
if (g_network->isSimulated() && BUGGIFY) {
|
||||
// Intentionally randomly block actors for low memory reason.
|
||||
// memory will be larger than threshold when deterministicRandom()->random01() > 1/2
|
||||
memory = SERVER_KNOBS->FASTRESTORE_MEMORY_THRESHOLD_MB_SOFT * 2 * deterministicRandom()->random01();
|
||||
if (deterministicRandom()->random01() < 0.4) { // enough memory
|
||||
memory = SERVER_KNOBS->FASTRESTORE_MEMORY_THRESHOLD_MB_SOFT * deterministicRandom()->random01();
|
||||
} else { // used too much memory, needs throttling
|
||||
memory = SERVER_KNOBS->FASTRESTORE_MEMORY_THRESHOLD_MB_SOFT +
|
||||
deterministicRandom()->random01() * SERVER_KNOBS->FASTRESTORE_MEMORY_THRESHOLD_MB_SOFT;
|
||||
}
|
||||
}
|
||||
if (memory < memoryThresholdBytes || self->finishedBatch.get() + 1 == actorBatchIndex) {
|
||||
if (memory >= memoryThresholdBytes) {
|
||||
|
@ -163,10 +178,15 @@ ACTOR Future<Void> traceRoleVersionBatchProgress(Reference<RestoreRoleData> self
|
|||
loop {
|
||||
int batchIndex = self->finishedBatch.get();
|
||||
int maxBatchIndex = self->versionBatchId.get();
|
||||
int maxPrintBatchIndex = batchIndex + SERVER_KNOBS->FASTRESTORE_VB_PARALLELISM;
|
||||
|
||||
TraceEvent ev("FastRestoreVersionBatchProgressState", self->nodeID);
|
||||
ev.detail("Role", role).detail("Node", self->nodeID).detail("FinishedBatch", batchIndex).detail("InitializedBatch", maxBatchIndex);
|
||||
while (batchIndex <= maxBatchIndex) {
|
||||
if (batchIndex > maxPrintBatchIndex) {
|
||||
ev.detail("SkipVersionBatches", maxBatchIndex - batchIndex + 1);
|
||||
break;
|
||||
}
|
||||
std::stringstream typeName;
|
||||
typeName << "VersionBatch" << batchIndex;
|
||||
ev.detail(typeName.str(), self->getVersionBatchState(batchIndex));
|
||||
|
|
|
@ -124,6 +124,7 @@ public:
|
|||
virtual std::string describeNode() = 0;
|
||||
};
|
||||
|
||||
void updateProcessStats(Reference<RestoreRoleData> self);
|
||||
ACTOR Future<Void> updateProcessMetrics(Reference<RestoreRoleData> self);
|
||||
ACTOR Future<Void> traceProcessMetrics(Reference<RestoreRoleData> self, std::string role);
|
||||
ACTOR Future<Void> traceRoleVersionBatchProgress(Reference<RestoreRoleData> self, std::string role);
|
||||
|
|
|
@ -75,8 +75,9 @@ ACTOR Future<Void> handlerTerminateWorkerRequest(RestoreSimpleRequest req, Refer
|
|||
// Future: Multiple roles in a restore worker
|
||||
void handleRecruitRoleRequest(RestoreRecruitRoleRequest req, Reference<RestoreWorkerData> self,
|
||||
ActorCollection* actors, Database cx) {
|
||||
// Already recruited a role
|
||||
// Future: Allow multiple restore roles on a restore worker. The design should easily allow this.
|
||||
ASSERT(!self->loaderInterf.present() || !self->applierInterf.present()); // Only one role per worker for now
|
||||
// Already recruited a role
|
||||
if (self->loaderInterf.present()) {
|
||||
ASSERT(req.role == RestoreRole::Loader);
|
||||
req.reply.send(RestoreRecruitRoleReply(self->id(), RestoreRole::Loader, self->loaderInterf.get()));
|
||||
|
|
|
@ -19,6 +19,7 @@
|
|||
*/
|
||||
|
||||
#include <fstream>
|
||||
#include <ostream>
|
||||
#include "fdbrpc/simulator.h"
|
||||
#include "fdbclient/DatabaseContext.h"
|
||||
#include "fdbserver/TesterInterface.actor.h"
|
||||
|
@ -732,7 +733,8 @@ void SimulationConfig::generateNormalConfig(int minimumReplication, int minimumR
|
|||
bool generateFearless = simple ? false : (minimumRegions > 1 || deterministicRandom()->random01() < 0.5);
|
||||
datacenters = simple ? 1 : ( generateFearless ? ( minimumReplication > 0 || deterministicRandom()->random01() < 0.5 ? 4 : 6 ) : deterministicRandom()->randomInt( 1, 4 ) );
|
||||
if (deterministicRandom()->random01() < 0.25) db.desiredTLogCount = deterministicRandom()->randomInt(1,7);
|
||||
if (deterministicRandom()->random01() < 0.25) db.masterProxyCount = deterministicRandom()->randomInt(1,7);
|
||||
if (deterministicRandom()->random01() < 0.25) db.proxyCount = deterministicRandom()->randomInt(1, 7);
|
||||
if (deterministicRandom()->random01() < 0.25) db.grvProxyCount = deterministicRandom()->randomInt(1, 4);
|
||||
if (deterministicRandom()->random01() < 0.25) db.resolverCount = deterministicRandom()->randomInt(1,7);
|
||||
int storage_engine_type = deterministicRandom()->randomInt(0, 4);
|
||||
switch (storage_engine_type) {
|
||||
|
@ -768,7 +770,8 @@ void SimulationConfig::generateNormalConfig(int minimumReplication, int minimumR
|
|||
// set_config("memory-radixtree-beta");
|
||||
if(simple) {
|
||||
db.desiredTLogCount = 1;
|
||||
db.masterProxyCount = 1;
|
||||
db.proxyCount = 1;
|
||||
db.grvProxyCount = 1;
|
||||
db.resolverCount = 1;
|
||||
}
|
||||
int replication_type = simple ? 1 : ( std::max(minimumReplication, datacenters > 4 ? deterministicRandom()->randomInt(1,3) : std::min(deterministicRandom()->randomInt(0,6), 3)) );
|
||||
|
|
|
@ -400,7 +400,7 @@ struct RolesInfo {
|
|||
latencyStats.setKeyRawNumber("p95", metrics.getValue("P95"));
|
||||
latencyStats.setKeyRawNumber("p99", metrics.getValue("P99"));
|
||||
latencyStats.setKeyRawNumber("p99.9", metrics.getValue("P99.9"));
|
||||
|
||||
|
||||
return latencyStats;
|
||||
}
|
||||
|
||||
|
@ -511,6 +511,30 @@ struct RolesInfo {
|
|||
obj["busiest_read_tag"] = busiestReadTagObj;
|
||||
}
|
||||
}
|
||||
|
||||
TraceEventFields const& busiestWriteTag = metrics.at("BusiestWriteTag");
|
||||
if(busiestWriteTag.size()) {
|
||||
int64_t tagCost = busiestWriteTag.getInt64("TagCost");
|
||||
|
||||
if(tagCost > 0) {
|
||||
JsonBuilderObject busiestWriteTagObj;
|
||||
|
||||
int64_t totalCost = busiestWriteTag.getInt64("TotalCost");
|
||||
ASSERT(totalCost > 0);
|
||||
|
||||
busiestWriteTagObj["tag"] = busiestWriteTag.getValue("Tag");
|
||||
busiestWriteTagObj["fractional_cost"] = (double)tagCost / totalCost;
|
||||
|
||||
double elapsed = busiestWriteTag.getDouble("Elapsed");
|
||||
if(elapsed > 0) {
|
||||
JsonBuilderObject estimatedCostObj;
|
||||
estimatedCostObj["hz"] = tagCost / elapsed;
|
||||
busiestWriteTagObj["estimated_cost"] = estimatedCostObj;
|
||||
}
|
||||
|
||||
obj["busiest_write_tag"] = busiestWriteTagObj;
|
||||
}
|
||||
}
|
||||
} catch (Error& e) {
|
||||
if(e.code() != error_code_attribute_not_found)
|
||||
throw e;
|
||||
|
@ -551,6 +575,28 @@ struct RolesInfo {
|
|||
return roles.insert( std::make_pair(iface.address(), obj ))->second;
|
||||
}
|
||||
JsonBuilderObject& addRole(std::string const& role, MasterProxyInterface& iface, EventMap const& metrics) {
|
||||
JsonBuilderObject obj;
|
||||
obj["id"] = iface.id().shortString();
|
||||
obj["role"] = role;
|
||||
try {
|
||||
TraceEventFields const& commitLatencyMetrics = metrics.at("CommitLatencyMetrics");
|
||||
if(commitLatencyMetrics.size()) {
|
||||
obj["commit_latency_statistics"] = addLatencyStatistics(commitLatencyMetrics);
|
||||
}
|
||||
|
||||
TraceEventFields const& commitLatencyBands = metrics.at("CommitLatencyBands");
|
||||
if(commitLatencyBands.size()) {
|
||||
obj["commit_latency_bands"] = addLatencyBandInfo(commitLatencyBands);
|
||||
}
|
||||
} catch (Error &e) {
|
||||
if(e.code() != error_code_attribute_not_found) {
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
|
||||
return roles.insert( std::make_pair(iface.address(), obj ))->second;
|
||||
}
|
||||
JsonBuilderObject& addRole(std::string const& role, GrvProxyInterface& iface, EventMap const& metrics) {
|
||||
JsonBuilderObject obj;
|
||||
obj["id"] = iface.id().shortString();
|
||||
obj["role"] = role;
|
||||
|
@ -567,16 +613,6 @@ struct RolesInfo {
|
|||
if(grvLatencyBands.size()) {
|
||||
obj["grv_latency_bands"] = addLatencyBandInfo(grvLatencyBands);
|
||||
}
|
||||
|
||||
TraceEventFields const& commitLatencyMetrics = metrics.at("CommitLatencyMetrics");
|
||||
if(commitLatencyMetrics.size()) {
|
||||
obj["commit_latency_statistics"] = addLatencyStatistics(commitLatencyMetrics);
|
||||
}
|
||||
|
||||
TraceEventFields const& commitLatencyBands = metrics.at("CommitLatencyBands");
|
||||
if(commitLatencyBands.size()) {
|
||||
obj["commit_latency_bands"] = addLatencyBandInfo(commitLatencyBands);
|
||||
}
|
||||
} catch (Error &e) {
|
||||
if(e.code() != error_code_attribute_not_found) {
|
||||
throw e;
|
||||
|
@ -610,7 +646,9 @@ ACTOR static Future<JsonBuilderObject> processStatusFetcher(
|
|||
WorkerEvents mMetrics, WorkerEvents nMetrics, WorkerEvents errors, WorkerEvents traceFileOpenErrors,
|
||||
WorkerEvents programStarts, std::map<std::string, std::vector<JsonBuilderObject>> processIssues,
|
||||
vector<std::pair<StorageServerInterface, EventMap>> storageServers,
|
||||
vector<std::pair<TLogInterface, EventMap>> tLogs, vector<std::pair<MasterProxyInterface, EventMap>> proxies,
|
||||
vector<std::pair<TLogInterface, EventMap>> tLogs,
|
||||
vector<std::pair<MasterProxyInterface, EventMap>> proxies,
|
||||
vector<std::pair<GrvProxyInterface, EventMap>> grvProxies,
|
||||
ServerCoordinators coordinators, Database cx, Optional<DatabaseConfiguration> configuration,
|
||||
Optional<Key> healthyZone, std::set<std::string>* incomplete_reasons) {
|
||||
|
||||
|
@ -704,6 +742,12 @@ ACTOR static Future<JsonBuilderObject> processStatusFetcher(
|
|||
wait(yield());
|
||||
}
|
||||
|
||||
state std::vector<std::pair<GrvProxyInterface, EventMap>>::iterator grvProxy;
|
||||
for(grvProxy = grvProxies.begin(); grvProxy != grvProxies.end(); ++grvProxy) {
|
||||
roles.addRole("grv_proxy", grvProxy->first, grvProxy->second);
|
||||
wait(yield());
|
||||
}
|
||||
|
||||
state std::vector<std::pair<TLogInterface, EventMap>>::iterator log;
|
||||
state Version maxTLogVersion = 0;
|
||||
|
||||
|
@ -1021,12 +1065,14 @@ ACTOR static Future<JsonBuilderObject> recoveryStateStatusFetcher(WorkerDetails
|
|||
if (mStatusCode == RecoveryStatus::recruiting_transaction_servers) {
|
||||
int requiredLogs = atoi( md.getValue("RequiredTLogs").c_str() );
|
||||
int requiredProxies = atoi( md.getValue("RequiredProxies").c_str() );
|
||||
int requiredGrvProxies = atoi(md.getValue("RequiredGrvProxies").c_str());
|
||||
int requiredResolvers = atoi( md.getValue("RequiredResolvers").c_str() );
|
||||
//int requiredProcesses = std::max(requiredLogs, std::max(requiredResolvers, requiredProxies));
|
||||
//int requiredMachines = std::max(requiredLogs, 1);
|
||||
|
||||
message["required_logs"] = requiredLogs;
|
||||
message["required_proxies"] = requiredProxies;
|
||||
message["required_grv_proxies"] = requiredGrvProxies;
|
||||
message["required_resolvers"] = requiredResolvers;
|
||||
} else if (mStatusCode == RecoveryStatus::locking_old_transaction_servers) {
|
||||
message["missing_logs"] = md.getValue("MissingIDs").c_str();
|
||||
|
@ -1583,10 +1629,34 @@ static Future<vector<std::pair<iface, EventMap>>> getServerMetrics(vector<iface>
|
|||
return results;
|
||||
}
|
||||
|
||||
ACTOR static Future<vector<std::pair<StorageServerInterface, EventMap>>> getStorageServersAndMetrics(Database cx, std::unordered_map<NetworkAddress, WorkerInterface> address_workers) {
|
||||
vector<StorageServerInterface> servers = wait(timeoutError(getStorageServers(cx, true), 5.0));
|
||||
vector<std::pair<StorageServerInterface, EventMap>> results = wait(
|
||||
getServerMetrics(servers, address_workers, std::vector<std::string>{ "StorageMetrics", "ReadLatencyMetrics", "ReadLatencyBands", "BusiestReadTag" }));
|
||||
ACTOR template <class iface>
|
||||
static Future<vector<TraceEventFields>> getServerBusiestWriteTags(vector<iface> servers, std::unordered_map<NetworkAddress, WorkerInterface> address_workers, WorkerDetails rkWorker) {
|
||||
state vector<Future<Optional<TraceEventFields>>> futures;
|
||||
for (const auto& s : servers) {
|
||||
futures.push_back(latestEventOnWorker(rkWorker.interf, s.id().toString() + "/BusiestWriteTag"));
|
||||
}
|
||||
wait(waitForAll(futures));
|
||||
|
||||
vector<TraceEventFields> result(servers.size());
|
||||
for(int i = 0; i < servers.size(); ++ i) {
|
||||
if(futures[i].get().present()) {
|
||||
result[i] = futures[i].get().get();
|
||||
}
|
||||
}
|
||||
return result;
|
||||
}
|
||||
|
||||
ACTOR static Future<vector<std::pair<StorageServerInterface, EventMap>>> getStorageServersAndMetrics(Database cx, std::unordered_map<NetworkAddress, WorkerInterface> address_workers, WorkerDetails rkWorker) {
|
||||
state vector<StorageServerInterface> servers = wait(timeoutError(getStorageServers(cx, true), 5.0));
|
||||
state vector<std::pair<StorageServerInterface, EventMap>> results;
|
||||
state vector<TraceEventFields> busiestWriteTags;
|
||||
wait(store(results, getServerMetrics(servers, address_workers,std::vector<std::string>{ "StorageMetrics", "ReadLatencyMetrics","ReadLatencyBands", "BusiestReadTag" }))
|
||||
&& store(busiestWriteTags, getServerBusiestWriteTags(servers, address_workers, rkWorker)));
|
||||
|
||||
ASSERT(busiestWriteTags.size() == results.size());
|
||||
for(int i = 0; i < busiestWriteTags.size(); ++ i) {
|
||||
results[i].second.emplace("BusiestWriteTag", busiestWriteTags[i]);
|
||||
}
|
||||
|
||||
return results;
|
||||
}
|
||||
|
@ -1601,11 +1671,17 @@ ACTOR static Future<vector<std::pair<TLogInterface, EventMap>>> getTLogsAndMetri
|
|||
|
||||
ACTOR static Future<vector<std::pair<MasterProxyInterface, EventMap>>> getProxiesAndMetrics(Reference<AsyncVar<ServerDBInfo>> db, std::unordered_map<NetworkAddress, WorkerInterface> address_workers) {
|
||||
vector<std::pair<MasterProxyInterface, EventMap>> results = wait(getServerMetrics(
|
||||
db->get().client.proxies, address_workers, std::vector<std::string>{ "GRVLatencyMetrics", "CommitLatencyMetrics", "GRVLatencyBands", "CommitLatencyBands" }));
|
||||
db->get().client.masterProxies, address_workers, std::vector<std::string>{ "CommitLatencyMetrics", "CommitLatencyBands" }));
|
||||
|
||||
return results;
|
||||
}
|
||||
|
||||
ACTOR static Future<vector<std::pair<GrvProxyInterface, EventMap>>> getGrvProxiesAndMetrics(Reference<AsyncVar<ServerDBInfo>> db, std::unordered_map<NetworkAddress, WorkerInterface> address_workers) {
|
||||
vector<std::pair<GrvProxyInterface, EventMap>> results = wait(getServerMetrics(
|
||||
db->get().client.grvProxies, address_workers, std::vector<std::string>{ "GRVLatencyMetrics", "GRVLatencyBands" }));
|
||||
return results;
|
||||
}
|
||||
|
||||
static int getExtraTLogEligibleZones(const vector<WorkerDetails>& workers, const DatabaseConfiguration& configuration) {
|
||||
std::set<StringRef> allZones;
|
||||
std::map<Key,std::set<StringRef>> dcId_zone;
|
||||
|
@ -1679,42 +1755,55 @@ ACTOR static Future<JsonBuilderObject> workloadStatusFetcher(Reference<AsyncVar<
|
|||
|
||||
// Writes and conflicts
|
||||
try {
|
||||
vector<Future<TraceEventFields>> proxyStatFutures;
|
||||
state vector<Future<TraceEventFields>> proxyStatFutures;
|
||||
state vector<Future<TraceEventFields>> grvProxyStatFutures;
|
||||
std::map<NetworkAddress, WorkerDetails> workersMap;
|
||||
for (auto const& w : workers) {
|
||||
workersMap[w.interf.address()] = w;
|
||||
}
|
||||
for (auto &p : db->get().client.proxies) {
|
||||
for (auto &p : db->get().client.masterProxies) {
|
||||
auto worker = getWorker(workersMap, p.address());
|
||||
if (worker.present())
|
||||
proxyStatFutures.push_back(timeoutError(worker.get().interf.eventLogRequest.getReply(EventLogRequest(LiteralStringRef("ProxyMetrics"))), 1.0));
|
||||
else
|
||||
throw all_alternatives_failed(); // We need data from all proxies for this result to be trustworthy
|
||||
}
|
||||
vector<TraceEventFields> proxyStats = wait(getAll(proxyStatFutures));
|
||||
for (auto &p : db->get().client.grvProxies) {
|
||||
auto worker = getWorker(workersMap, p.address());
|
||||
if (worker.present())
|
||||
grvProxyStatFutures.push_back(timeoutError(worker.get().interf.eventLogRequest.getReply(EventLogRequest(LiteralStringRef("GrvProxyMetrics"))), 1.0));
|
||||
else
|
||||
throw all_alternatives_failed(); // We need data from all proxies for this result to be trustworthy
|
||||
}
|
||||
state vector<TraceEventFields> proxyStats = wait(getAll(proxyStatFutures));
|
||||
state vector<TraceEventFields> grvProxyStats = wait(getAll(grvProxyStatFutures));
|
||||
|
||||
StatusCounter mutations;
|
||||
StatusCounter mutationBytes;
|
||||
StatusCounter txnConflicts;
|
||||
StatusCounter txnStartOut;
|
||||
StatusCounter txnSystemPriorityStartOut;
|
||||
StatusCounter txnDefaultPriorityStartOut;
|
||||
StatusCounter txnBatchPriorityStartOut;
|
||||
|
||||
StatusCounter mutations;
|
||||
StatusCounter mutationBytes;
|
||||
StatusCounter txnConflicts;
|
||||
StatusCounter txnCommitOutSuccess;
|
||||
StatusCounter txnKeyLocationOut;
|
||||
StatusCounter txnMemoryErrors;
|
||||
|
||||
for (auto &gps : grvProxyStats) {
|
||||
txnStartOut.updateValues( StatusCounter(gps.getValue("TxnStartOut")) );
|
||||
txnSystemPriorityStartOut.updateValues(StatusCounter(gps.getValue("TxnSystemPriorityStartOut")));
|
||||
txnDefaultPriorityStartOut.updateValues(StatusCounter(gps.getValue("TxnDefaultPriorityStartOut")));
|
||||
txnBatchPriorityStartOut.updateValues(StatusCounter(gps.getValue("TxnBatchPriorityStartOut")));
|
||||
txnMemoryErrors.updateValues(StatusCounter(gps.getValue("TxnRequestErrors")));
|
||||
}
|
||||
|
||||
for (auto &ps : proxyStats) {
|
||||
mutations.updateValues( StatusCounter(ps.getValue("Mutations")) );
|
||||
mutationBytes.updateValues( StatusCounter(ps.getValue("MutationBytes")) );
|
||||
txnConflicts.updateValues( StatusCounter(ps.getValue("TxnConflicts")) );
|
||||
txnStartOut.updateValues( StatusCounter(ps.getValue("TxnStartOut")) );
|
||||
txnSystemPriorityStartOut.updateValues(StatusCounter(ps.getValue("TxnSystemPriorityStartOut")));
|
||||
txnDefaultPriorityStartOut.updateValues(StatusCounter(ps.getValue("TxnDefaultPriorityStartOut")));
|
||||
txnBatchPriorityStartOut.updateValues(StatusCounter(ps.getValue("TxnBatchPriorityStartOut")));
|
||||
txnCommitOutSuccess.updateValues( StatusCounter(ps.getValue("TxnCommitOutSuccess")) );
|
||||
txnKeyLocationOut.updateValues( StatusCounter(ps.getValue("KeyServerLocationOut")) );
|
||||
txnMemoryErrors.updateValues( StatusCounter(ps.getValue("TxnRequestErrors")) );
|
||||
txnMemoryErrors.updateValues( StatusCounter(ps.getValue("KeyServerLocationErrors")) );
|
||||
txnMemoryErrors.updateValues( StatusCounter(ps.getValue("TxnCommitErrors")) );
|
||||
}
|
||||
|
@ -1745,11 +1834,14 @@ ACTOR static Future<JsonBuilderObject> workloadStatusFetcher(Reference<AsyncVar<
|
|||
state TraceEventFields ratekeeper = wait( timeoutError(rkWorker.interf.eventLogRequest.getReply( EventLogRequest(LiteralStringRef("RkUpdate") ) ), 1.0) );
|
||||
TraceEventFields batchRatekeeper = wait( timeoutError(rkWorker.interf.eventLogRequest.getReply( EventLogRequest(LiteralStringRef("RkUpdateBatch") ) ), 1.0) );
|
||||
|
||||
bool autoThrottlingEnabled = ratekeeper.getInt("AutoThrottlingEnabled");
|
||||
double tpsLimit = ratekeeper.getDouble("TPSLimit");
|
||||
double batchTpsLimit = batchRatekeeper.getDouble("TPSLimit");
|
||||
double transPerSec = ratekeeper.getDouble("ReleasedTPS");
|
||||
double batchTransPerSec = ratekeeper.getDouble("ReleasedBatchTPS");
|
||||
int autoThrottledTags = ratekeeper.getInt("TagsAutoThrottled");
|
||||
int autoThrottledTagsBusyRead = ratekeeper.getInt("TagsAutoThrottledBusyRead");
|
||||
int autoThrottledTagsBusyWrite = ratekeeper.getInt("TagsAutoThrottledBusyWrite");
|
||||
int manualThrottledTags = ratekeeper.getInt("TagsManuallyThrottled");
|
||||
int ssCount = ratekeeper.getInt("StorageServers");
|
||||
int tlogCount = ratekeeper.getInt("TLogs");
|
||||
|
@ -1781,6 +1873,15 @@ ACTOR static Future<JsonBuilderObject> workloadStatusFetcher(Reference<AsyncVar<
|
|||
JsonBuilderObject throttledTagsObj;
|
||||
JsonBuilderObject autoThrottledTagsObj;
|
||||
autoThrottledTagsObj["count"] = autoThrottledTags;
|
||||
autoThrottledTagsObj["busy_read"] = autoThrottledTagsBusyRead;
|
||||
autoThrottledTagsObj["busy_write"] = autoThrottledTagsBusyWrite;
|
||||
if(autoThrottlingEnabled) {
|
||||
autoThrottledTagsObj["recommended_only"] = 0;
|
||||
}
|
||||
else {
|
||||
autoThrottledTagsObj["recommended_only"] = 1;
|
||||
}
|
||||
|
||||
throttledTagsObj["auto"] = autoThrottledTagsObj;
|
||||
|
||||
JsonBuilderObject manualThrottledTagsObj;
|
||||
|
@ -2340,6 +2441,7 @@ ACTOR Future<StatusReply> clusterGetStatus(
|
|||
state vector<std::pair<StorageServerInterface, EventMap>> storageServers;
|
||||
state vector<std::pair<TLogInterface, EventMap>> tLogs;
|
||||
state vector<std::pair<MasterProxyInterface, EventMap>> proxies;
|
||||
state vector<std::pair<GrvProxyInterface, EventMap>> grvProxies;
|
||||
state JsonBuilderObject qos;
|
||||
state JsonBuilderObject data_overlay;
|
||||
|
||||
|
@ -2400,9 +2502,10 @@ ACTOR Future<StatusReply> clusterGetStatus(
|
|||
address_workers[worker.interf.address()] = worker.interf;
|
||||
}
|
||||
|
||||
state Future<ErrorOr<vector<std::pair<StorageServerInterface, EventMap>>>> storageServerFuture = errorOr(getStorageServersAndMetrics(cx, address_workers));
|
||||
state Future<ErrorOr<vector<std::pair<StorageServerInterface, EventMap>>>> storageServerFuture = errorOr(getStorageServersAndMetrics(cx, address_workers, rkWorker));
|
||||
state Future<ErrorOr<vector<std::pair<TLogInterface, EventMap>>>> tLogFuture = errorOr(getTLogsAndMetrics(db, address_workers));
|
||||
state Future<ErrorOr<vector<std::pair<MasterProxyInterface, EventMap>>>> proxyFuture = errorOr(getProxiesAndMetrics(db, address_workers));
|
||||
state Future<ErrorOr<vector<std::pair<GrvProxyInterface, EventMap>>>> grvProxyFuture = errorOr(getGrvProxiesAndMetrics(db, address_workers));
|
||||
|
||||
state int minReplicasRemaining = -1;
|
||||
state Future<Optional<Value>> primaryDCFO = getActivePrimaryDC(cx, &messages);
|
||||
|
@ -2492,6 +2595,15 @@ ACTOR Future<StatusReply> clusterGetStatus(
|
|||
else {
|
||||
messages.push_back(JsonBuilder::makeMessage("proxies_error", "Timed out trying to retrieve proxies."));
|
||||
}
|
||||
|
||||
// ...also grv proxies
|
||||
ErrorOr<vector<std::pair<GrvProxyInterface, EventMap>>> _grvProxies = wait(grvProxyFuture);
|
||||
if (_grvProxies.present()) {
|
||||
grvProxies = _grvProxies.get();
|
||||
}
|
||||
else {
|
||||
messages.push_back(JsonBuilder::makeMessage("grv_proxies_error", "Timed out trying to retrieve grv proxies."));
|
||||
}
|
||||
wait( waitForAll(warningFutures) );
|
||||
}
|
||||
else {
|
||||
|
@ -2505,7 +2617,7 @@ ACTOR Future<StatusReply> clusterGetStatus(
|
|||
JsonBuilderObject processStatus = wait(processStatusFetcher(db, workers, pMetrics, mMetrics, networkMetrics,
|
||||
latestError, traceFileOpenErrors, programStarts,
|
||||
processIssues, storageServers, tLogs, proxies,
|
||||
coordinators, cx, configuration,
|
||||
grvProxies, coordinators, cx, configuration,
|
||||
loadResult.present() ? loadResult.get().healthyZone : Optional<Key>(),
|
||||
&status_incomplete_reasons));
|
||||
statusObj["processes"] = processStatus;
|
||||
|
|
|
@ -1235,7 +1235,7 @@ ACTOR Future<Void> fetchKeys( StorageCacheData *data, AddingCacheRange* cacheRan
|
|||
try {
|
||||
TEST(true); // Fetching keys for transferred cacheRange
|
||||
|
||||
state Standalone<RangeResultRef> this_block = wait( tryFetchRange( data->cx, fetchVersion, keys, GetRangeLimits( CLIENT_KNOBS->ROW_LIMIT_UNLIMITED, fetchBlockBytes ), &isTooOld ) );
|
||||
state Standalone<RangeResultRef> this_block = wait( tryFetchRange( data->cx, fetchVersion, keys, GetRangeLimits( GetRangeLimits::ROW_LIMIT_UNLIMITED, fetchBlockBytes ), &isTooOld ) );
|
||||
|
||||
state int expectedSize = (int)this_block.expectedSize() + (8-(int)sizeof(KeyValueRef))*this_block.size();
|
||||
|
||||
|
|
Some files were not shown because too many files have changed in this diff Show More
Loading…
Reference in New Issue