Merge remote-tracking branch 'upstream/release-6.3' into rocksdb-unsafe-fsync

This commit is contained in:
Daniel Smith 2020-10-09 16:53:05 +00:00
commit 6e287eb0d1
118 changed files with 4025 additions and 1184 deletions

View File

@ -18,7 +18,7 @@
# limitations under the License.
cmake_minimum_required(VERSION 3.13)
project(foundationdb
VERSION 6.3.5
VERSION 6.3.10
DESCRIPTION "FoundationDB is a scalable, fault-tolerant, ordered key-value store with full ACID transactions."
HOMEPAGE_URL "http://www.foundationdb.org/"
LANGUAGES C CXX ASM)

View File

@ -153,7 +153,7 @@ void fdb_future_destroy( FDBFuture* f ) {
extern "C" DLLEXPORT
fdb_error_t fdb_future_block_until_ready( FDBFuture* f ) {
CATCH_AND_RETURN( TSAVB(f)->blockUntilReady(); );
CATCH_AND_RETURN(TSAVB(f)->blockUntilReadyCheckOnMainThread(););
}
fdb_bool_t fdb_future_is_error_v22( FDBFuture* f ) {

View File

@ -78,8 +78,9 @@ type Subspace interface {
// FoundationDB keys (corresponding to the prefix of this Subspace).
fdb.KeyConvertible
// All Subspaces implement fdb.ExactRange and fdb.Range, and describe all
// keys logically in this Subspace.
// All Subspaces implement fdb.ExactRange and fdb.Range, and describe all
// keys strictly within the subspace that encode tuples. Specifically,
// this will include all keys in [prefix + '\x00', prefix + '\xff').
fdb.ExactRange
}

View File

@ -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

View File

@ -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) {

View File

@ -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;
}
}

View File

@ -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);
}
}

View File

@ -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;
}
}

View File

@ -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();
}

View File

@ -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;
}

View File

@ -152,8 +152,6 @@ class RangeQuery implements AsyncIterable<KeyValue>, Iterable<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>, Iterable<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>, Iterable<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;
}
}

View File

@ -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);
}
}

View File

@ -21,10 +21,6 @@
package com.apple.foundationdb;
class RangeResultInfo {
RangeResultSummary getSummary() {
return f.getSummary();
}
RangeResult get() {
return f.getResults();
}

View File

@ -45,20 +45,24 @@ RUN cd /tmp && curl -L https://github.com/ninja-build/ninja/archive/v1.9.0.zip -
cd .. && rm -rf ninja-1.9.0 ninja.zip
# install openssl
RUN cd /tmp && curl -L https://www.openssl.org/source/openssl-1.1.1d.tar.gz -o openssl.tar.gz &&\
echo "1e3a91bc1f9dfce01af26026f856e064eab4c8ee0a8f457b5ae30b40b8b711f2 openssl.tar.gz" > openssl-sha.txt &&\
RUN cd /tmp && curl -L https://www.openssl.org/source/openssl-1.1.1h.tar.gz -o openssl.tar.gz &&\
echo "5c9ca8774bd7b03e5784f26ae9e9e6d749c9da2438545077e6b3d755a06595d9 openssl.tar.gz" > openssl-sha.txt &&\
sha256sum -c openssl-sha.txt && tar -xzf openssl.tar.gz &&\
cd openssl-1.1.1d && scl enable devtoolset-8 -- ./config CFLAGS="-fPIC -O3" --prefix=/usr/local &&\
cd openssl-1.1.1h && scl enable devtoolset-8 -- ./config CFLAGS="-fPIC -O3" --prefix=/usr/local &&\
scl enable devtoolset-8 -- make -j`nproc` && scl enable devtoolset-8 -- make -j1 install &&\
ln -sv /usr/local/lib64/lib*.so.1.1 /usr/lib64/ &&\
cd /tmp/ && rm -rf /tmp/openssl-1.1.1d /tmp/openssl.tar.gz
cd /tmp/ && rm -rf /tmp/openssl-1.1.1h /tmp/openssl.tar.gz
RUN cd /opt/ && curl -L https://github.com/facebook/rocksdb/archive/v6.10.1.tar.gz -o rocksdb.tar.gz &&\
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.17
ENV DOCKER_IMAGEVER=0.1.17
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++

View File

@ -1,4 +1,4 @@
FROM foundationdb/foundationdb-build:0.1.14
FROM foundationdb/foundationdb-build:0.1.17
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

View File

@ -2,7 +2,7 @@ version: "3"
services:
common: &common
image: foundationdb/foundationdb-build:0.1.13
image: foundationdb/foundationdb-build:0.1.17
build-setup: &build-setup
<<: *common

View File

@ -59,11 +59,14 @@ else()
set(ROCKSDB_LIBRARIES
${BINARY_DIR}/librocksdb.a)
ExternalProject_Get_Property(rocksdb SOURCE_DIR)
set (ROCKSDB_INCLUDE_DIR "${SOURCE_DIR}/include")
set(ROCKSDB_FOUND TRUE)
endif()
message(STATUS "Found RocksDB library: ${ROCKSDB_LIBRARIES}")
message(STATUS "Found RocksDB includes: ${ROCKSDB_INCLUDE_DIRS}")
message(STATUS "Found RocksDB includes: ${ROCKSDB_INCLUDE_DIR}")
mark_as_advanced(
ROCKSDB_LIBRARIES

View File

@ -12,7 +12,7 @@ endif()
# SSL
################################################################################
include(CheckSymbolExists)
set(DISABLE_TLS OFF CACHE BOOL "Don't try to find OpenSSL and always build without TLS support")
if(DISABLE_TLS)
set(WITH_TLS OFF)
@ -107,7 +107,9 @@ endif()
################################################################################
set(SSD_ROCKSDB_EXPERIMENTAL OFF CACHE BOOL "Build with experimental RocksDB support")
if (SSD_ROCKSDB_EXPERIMENTAL)
# RocksDB is currently enabled by default for GCC but does not build with the latest
# Clang.
if (SSD_ROCKSDB_EXPERIMENTAL OR GCC)
set(WITH_ROCKSDB_EXPERIMENTAL ON)
else()
set(WITH_ROCKSDB_EXPERIMENTAL OFF)

View File

@ -1,6 +1,5 @@
#!/bin/bash
SCRIPTDIR="$( cd "$( dirname "${BASH_SOURCE[0]}" )" && pwd )"
pkill fdbserver
ulimit -S -c unlimited
unset FDB_NETWORK_OPTION_EXTERNAL_CLIENT_DIRECTORY
@ -8,4 +7,4 @@ WORKDIR="$(pwd)/tmp/$$"
if [ ! -d "${WORKDIR}" ] ; then
mkdir -p "${WORKDIR}"
fi
DEBUGLEVEL=0 DISPLAYERROR=1 RANDOMTEST=1 WORKDIR="${WORKDIR}" FDBSERVERPORT="${PORT_FDBSERVER:-4500}" ${SCRIPTDIR}/bindingTestScript.sh 1
DEBUGLEVEL=0 DISPLAYERROR=1 RANDOMTEST=1 WORKDIR="${WORKDIR}" ${SCRIPTDIR}/bindingTestScript.sh 1

View File

@ -7,7 +7,7 @@ SCRIPTID="${$}"
SAVEONERROR="${SAVEONERROR:-1}"
PYTHONDIR="${BINDIR}/tests/python"
testScript="${BINDIR}/tests/bindingtester/run_binding_tester.sh"
VERSION="1.6"
VERSION="1.9"
source ${SCRIPTDIR}/localClusterStart.sh
@ -23,19 +23,22 @@ cycles="${1}"
if [ "${DEBUGLEVEL}" -gt 0 ]
then
echo "Work dir: ${WORKDIR}"
echo "Bin dir: ${BINDIR}"
echo "Log dir: ${LOGDIR}"
echo "Python path: ${PYTHONDIR}"
echo "Lib dir: ${LIBDIR}"
echo "Server port: ${FDBSERVERPORT}"
echo "Script Id: ${SCRIPTID}"
echo "Version: ${VERSION}"
echo "Work dir: ${WORKDIR}"
echo "Bin dir: ${BINDIR}"
echo "Log dir: ${LOGDIR}"
echo "Python path: ${PYTHONDIR}"
echo "Lib dir: ${LIBDIR}"
echo "Cluster String: ${FDBCLUSTERTEXT}"
echo "Script Id: ${SCRIPTID}"
echo "Version: ${VERSION}"
fi
# Begin the cluster using the logic in localClusterStart.sh.
startCluster
# Stop the cluster on exit
trap "stopCluster" EXIT
# Display user message
if [ "${status}" -ne 0 ]; then
:
@ -58,8 +61,8 @@ fi
# Display directory and log information, if an error occurred
if [ "${status}" -ne 0 ]
then
ls "${WORKDIR}" > "${LOGDIR}/dir.log"
ps -eafw > "${LOGDIR}/process-preclean.log"
ls "${WORKDIR}" &> "${LOGDIR}/dir.log"
ps -eafwH &> "${LOGDIR}/process-preclean.log"
if [ -f "${FDBCONF}" ]; then
cp -f "${FDBCONF}" "${LOGDIR}/"
fi
@ -71,10 +74,15 @@ fi
# Save debug information files, environment, and log information, if an error occurred
if [ "${status}" -ne 0 ] && [ "${SAVEONERROR}" -gt 0 ]; then
ps -eafw > "${LOGDIR}/process-exit.log"
netstat -na > "${LOGDIR}/netstat.log"
df -h > "${LOGDIR}/disk.log"
env > "${LOGDIR}/env.log"
ps -eafwH &> "${LOGDIR}/process-exit.log"
netstat -na &> "${LOGDIR}/netstat.log"
df -h &> "${LOGDIR}/disk.log"
env &> "${LOGDIR}/env.log"
fi
# Stop the cluster
if stopCluster; then
unset FDBSERVERID
fi
exit "${status}"

View File

@ -5,311 +5,398 @@ WORKDIR="${WORKDIR:-${SCRIPTDIR}/tmp/fdb.work}"
LOGDIR="${WORKDIR}/log"
ETCDIR="${WORKDIR}/etc"
BINDIR="${BINDIR:-${SCRIPTDIR}}"
FDBSERVERPORT="${FDBSERVERPORT:-4500}"
FDBPORTSTART="${FDBPORTSTART:-4000}"
FDBPORTTOTAL="${FDBPORTTOTAL:-1000}"
SERVERCHECKS="${SERVERCHECKS:-10}"
CONFIGUREWAIT="${CONFIGUREWAIT:-240}"
FDBCONF="${ETCDIR}/fdb.cluster"
LOGFILE="${LOGFILE:-${LOGDIR}/startcluster.log}"
AUDITCLUSTER="${AUDITCLUSTER:-0}"
AUDITLOG="${AUDITLOG:-/tmp/audit-cluster.log}"
# Initialize the variables
status=0
messagetime=0
messagecount=0
# Do nothing, if cluster string is already defined
if [ -n "${FDBCLUSTERTEXT}" ]
then
:
# Otherwise, define the cluster text
else
# Define a random ip address and port on localhost
if [ -z "${IPADDRESS}" ]; then
let index2="${RANDOM} % 256"
let index3="${RANDOM} % 256"
let index4="(${RANDOM} % 255) + 1"
IPADDRESS="127.${index2}.${index3}.${index4}"
fi
if [ -z "${FDBPORT}" ]; then
let FDBPORT="(${RANDOM} % ${FDBPORTTOTAL}) + ${FDBPORTSTART}"
fi
FDBCLUSTERTEXT="${IPADDRESS}:${FDBPORT}"
fi
function log
{
local status=0
if [ "$#" -lt 1 ]
then
echo "Usage: log <message> [echo]"
echo
echo "Logs the message and timestamp to LOGFILE (${LOGFILE}) and, if the"
echo "second argument is either not present or is set to 1, stdout."
let status="${status} + 1"
else
# Log to stdout.
if [ "$#" -lt 2 ] || [ "${2}" -ge 1 ]
then
echo "${1}"
fi
local status=0
if [ "$#" -lt 1 ]
then
echo "Usage: log <message> [echo]"
echo
echo "Logs the message and timestamp to LOGFILE (${LOGFILE}) and, if the"
echo "second argument is either not present or is set to 1, stdout."
let status="${status} + 1"
else
# Log to stdout.
if [ "$#" -lt 2 ] || [ "${2}" -ge 1 ]
then
echo "${1}"
fi
# Log to file.
datestr=$(date +"%Y-%m-%d %H:%M:%S (%s)")
dir=$(dirname "${LOGFILE}")
if ! [ -d "${dir}" ] && ! mkdir -p "${dir}"
then
echo "Could not create directory to log output."
let status="${status} + 1"
elif ! [ -f "${LOGFILE}" ] && ! touch "${LOGFILE}"
then
echo "Could not create file ${LOGFILE} to log output."
let status="${status} + 1"
elif ! echo "[ ${datestr} ] ${1}" >> "${LOGFILE}"
then
echo "Could not log output to ${LOGFILE}."
let status="${status} + 1"
fi
fi
# Log to file.
datestr=$(date +"%Y-%m-%d %H:%M:%S (%s)")
dir=$(dirname "${LOGFILE}")
if ! [ -d "${dir}" ] && ! mkdir -p "${dir}"
then
echo "Could not create directory to log output."
let status="${status} + 1"
elif ! [ -f "${LOGFILE}" ] && ! touch "${LOGFILE}"
then
echo "Could not create file ${LOGFILE} to log output."
let status="${status} + 1"
elif ! echo "[ ${datestr} ] ${1}" >> "${LOGFILE}"
then
echo "Could not log output to ${LOGFILE}."
let status="${status} + 1"
fi
fi
return "${status}"
return "${status}"
}
# Display a message for the user.
function displayMessage
{
local status=0
local status=0
if [ "$#" -lt 1 ]
then
echo "displayMessage <message>"
let status="${status} + 1"
elif ! log "${1}" 0
then
log "Could not write message to file."
else
# Increment the message counter
let messagecount="${messagecount} + 1"
if [ "$#" -lt 1 ]
then
echo "displayMessage <message>"
let status="${status} + 1"
elif ! log "${1}" 0
then
log "Could not write message to file."
else
# Increment the message counter
let messagecount="${messagecount} + 1"
# Display successful message, if previous message
if [ "${messagecount}" -gt 1 ]
then
# Determine the amount of transpired time
let timespent="${SECONDS}-${messagetime}"
# Display successful message, if previous message
if [ "${messagecount}" -gt 1 ]
then
# Determine the amount of transpired time
let timespent="${SECONDS}-${messagetime}"
if [ "${DEBUGLEVEL}" -gt 0 ]; then
printf "... done in %3d seconds\n" "${timespent}"
fi
fi
if [ "${DEBUGLEVEL}" -gt 0 ]; then
printf "... done in %3d seconds\n" "${timespent}"
fi
fi
# Display message
if [ "${DEBUGLEVEL}" -gt 0 ]; then
printf "%-16s %-35s " "$(date "+%F %H-%M-%S")" "$1"
fi
# Display message
if [ "${DEBUGLEVEL}" -gt 0 ]; then
printf "%-16s %-35s " "$(date "+%F %H-%M-%S")" "$1"
fi
# Update the variables
messagetime="${SECONDS}"
fi
# Update the variables
messagetime="${SECONDS}"
fi
return "${status}"
return "${status}"
}
# Create the directories used by the server.
function createDirectories {
# Display user message
if ! displayMessage "Creating directories"
then
echo 'Failed to display user message'
let status="${status} + 1"
elif ! mkdir -p "${LOGDIR}" "${ETCDIR}"
then
log "Failed to create directories"
let status="${status} + 1"
# Display user message
elif ! displayMessage "Setting file permissions"
then
log 'Failed to display user message'
let status="${status} + 1"
elif ! chmod 755 "${BINDIR}/fdbserver" "${BINDIR}/fdbcli"
then
log "Failed to set file permissions"
let status="${status} + 1"
else
while read filepath
do
if [ -f "${filepath}" ] && [ ! -x "${filepath}" ]
then
# if [ "${DEBUGLEVEL}" -gt 1 ]; then
# log " Enable executable: ${filepath}"
# fi
log " Enable executable: ${filepath}" "${DEBUGLEVEL}"
if ! chmod 755 "${filepath}"
then
log "Failed to set executable for file: ${filepath}"
let status="${status} + 1"
fi
fi
done < <(find "${BINDIR}" -iname '*.py' -o -iname '*.rb' -o -iname 'fdb_flow_tester' -o -iname '_stacktester' -o -iname '*.js' -o -iname '*.sh' -o -iname '*.ksh')
fi
function createDirectories
{
local status=0
return ${status}
# Display user message
if ! displayMessage "Creating directories"
then
echo 'Failed to display user message'
let status="${status} + 1"
elif ! mkdir -p "${LOGDIR}" "${ETCDIR}"
then
log "Failed to create directories"
let status="${status} + 1"
# Display user message
elif ! displayMessage "Setting file permissions"
then
log 'Failed to display user message'
let status="${status} + 1"
elif ! chmod 755 "${BINDIR}/fdbserver" "${BINDIR}/fdbcli"
then
log "Failed to set file permissions"
let status="${status} + 1"
else
while read filepath
do
if [ -f "${filepath}" ] && [ ! -x "${filepath}" ]
then
# if [ "${DEBUGLEVEL}" -gt 1 ]; then
# log " Enable executable: ${filepath}"
# fi
log " Enable executable: ${filepath}" "${DEBUGLEVEL}"
if ! chmod 755 "${filepath}"
then
log "Failed to set executable for file: ${filepath}"
let status="${status} + 1"
fi
fi
done < <(find "${BINDIR}" -iname '*.py' -o -iname '*.rb' -o -iname 'fdb_flow_tester' -o -iname '_stacktester' -o -iname '*.js' -o -iname '*.sh' -o -iname '*.ksh')
fi
return ${status}
}
# Create a cluster file for the local cluster.
function createClusterFile {
if [ "${status}" -ne 0 ]; then
:
# Display user message
elif ! displayMessage "Creating Fdb Cluster file"
then
log 'Failed to display user message'
let status="${status} + 1"
else
description=$(LC_CTYPE=C tr -dc A-Za-z0-9 < /dev/urandom 2> /dev/null | head -c 8)
random_str=$(LC_CTYPE=C tr -dc A-Za-z0-9 < /dev/urandom 2> /dev/null | head -c 8)
echo "$description:$random_str@127.0.0.1:${FDBSERVERPORT}" > "${FDBCONF}"
fi
function createClusterFile
{
local status=0
if [ "${status}" -ne 0 ]; then
:
elif ! chmod 0664 "${FDBCONF}"; then
log "Failed to set permissions on fdbconf: ${FDBCONF}"
let status="${status} + 1"
fi
if [ "${status}" -ne 0 ]; then
:
# Display user message
elif ! displayMessage "Creating Fdb Cluster file"
then
log 'Failed to display user message'
let status="${status} + 1"
else
description=$(LC_CTYPE=C tr -dc A-Za-z0-9 < /dev/urandom 2> /dev/null | head -c 8)
random_str=$(LC_CTYPE=C tr -dc A-Za-z0-9 < /dev/urandom 2> /dev/null | head -c 8)
echo "${description}:${random_str}@${FDBCLUSTERTEXT}" > "${FDBCONF}"
fi
return ${status}
if [ "${status}" -ne 0 ]; then
:
elif ! chmod 0664 "${FDBCONF}"; then
log "Failed to set permissions on fdbconf: ${FDBCONF}"
let status="${status} + 1"
fi
return ${status}
}
# Stop the Cluster from running.
function stopCluster
{
local status=0
# Add an audit entry, if enabled
if [ "${AUDITCLUSTER}" -gt 0 ]; then
printf '%-15s (%6s) Stopping cluster %-20s (%6s): %s\n' "$(date +'%Y-%m-%d %H:%M:%S')" "${$}" "${FDBCLUSTERTEXT}" "${FDBSERVERID}" >> "${AUDITLOG}"
fi
if [ -z "${FDBSERVERID}" ]; then
log 'FDB Server process is not defined'
let status="${status} + 1"
elif ! kill -0 "${FDBSERVERID}"; then
log "Failed to locate FDB Server process (${FDBSERVERID})"
let status="${status} + 1"
elif "${BINDIR}/fdbcli" -C "${FDBCONF}" --exec "kill; kill ${FDBCLUSTERTEXT}; sleep 3" --timeout 120 &>> "${LOGDIR}/fdbcli-kill.log"
then
# Ensure that process is dead
if ! kill -0 "${FDBSERVERID}" 2> /dev/null; then
log "Killed cluster (${FDBSERVERID}) via cli"
elif ! kill -9 "${FDBSERVERID}"; then
log "Failed to kill FDB Server process (${FDBSERVERID}) via cli or kill command"
let status="${status} + 1"
else
log "Forcibly killed FDB Server process (${FDBSERVERID}) since cli failed"
fi
elif ! kill -9 "${FDBSERVERID}"; then
log "Failed to forcibly kill FDB Server process (${FDBSERVERID})"
let status="${status} + 1"
else
log "Forcibly killed FDB Server process (${FDBSERVERID})"
fi
return "${status}"
}
# Start the server running.
function startFdbServer {
if [ "${status}" -ne 0 ]; then
:
elif ! displayMessage "Starting Fdb Server"
then
log 'Failed to display user message'
let status="${status} + 1"
function startFdbServer
{
local status=0
elif ! "${BINDIR}/fdbserver" -C "${FDBCONF}" -p "auto:${FDBSERVERPORT}" -L "${LOGDIR}" -d "${WORKDIR}/fdb/$$" &> "${LOGDIR}/fdbserver.log" &
then
log "Failed to start FDB Server"
# Maybe the server is already running
FDBSERVERID="$(pidof fdbserver)"
let status="${status} + 1"
else
FDBSERVERID="${!}"
fi
# Add an audit entry, if enabled
if [ "${AUDITCLUSTER}" -gt 0 ]; then
printf '%-15s (%6s) Starting cluster %-20s\n' "$(date +'%Y-%m-%d %H:%M:%S')" "${$}" "${FDBCLUSTERTEXT}" >> "${AUDITLOG}"
fi
if ! kill -0 ${FDBSERVERID} ; then
log "FDB Server start failed."
let status="${status} + 1"
fi
if ! displayMessage "Starting Fdb Server"
then
log 'Failed to display user message'
let status="${status} + 1"
return ${status}
else
"${BINDIR}/fdbserver" --knob_disable_posix_kernel_aio=1 -C "${FDBCONF}" -p "${FDBCLUSTERTEXT}" -L "${LOGDIR}" -d "${WORKDIR}/fdb/${$}" &> "${LOGDIR}/fdbserver.log" &
if [ "${?}" -ne 0 ]
then
log "Failed to start FDB Server"
let status="${status} + 1"
else
FDBSERVERID="${!}"
fi
fi
if [ -z "${FDBSERVERID}" ]; then
log "FDB Server start failed because no process"
let status="${status} + 1"
elif ! kill -0 "${FDBSERVERID}" ; then
log "FDB Server start failed because process terminated unexpectedly"
let status="${status} + 1"
fi
return ${status}
}
function getStatus {
if [ "${status}" -ne 0 ]; then
:
elif ! date &>> "${LOGDIR}/fdbclient.log"
then
log 'Failed to get date'
let status="${status} + 1"
elif ! "${BINDIR}/fdbcli" -C "${FDBCONF}" --exec 'status json' --timeout 120 &>> "${LOGDIR}/fdbclient.log"
then
log 'Failed to get status from fdbcli'
let status="${status} + 1"
elif ! date &>> "${LOGDIR}/fdbclient.log"
then
log 'Failed to get date'
let status="${status} + 1"
fi
function getStatus
{
local status=0
return ${status}
if [ "${status}" -ne 0 ]; then
:
elif ! date &>> "${LOGDIR}/fdbclient.log"
then
log 'Failed to get date'
let status="${status} + 1"
elif ! "${BINDIR}/fdbcli" -C "${FDBCONF}" --exec 'status json' --timeout 120 &>> "${LOGDIR}/fdbclient.log"
then
log 'Failed to get status from fdbcli'
let status="${status} + 1"
elif ! date &>> "${LOGDIR}/fdbclient.log"
then
log 'Failed to get date'
let status="${status} + 1"
fi
return ${status}
}
# Verify that the cluster is available.
function verifyAvailable {
# Verify that the server is running.
if ! kill -0 "${FDBSERVERID}"
then
log "FDB server process (${FDBSERVERID}) is not running"
let status="${status} + 1"
return 1
function verifyAvailable
{
local status=0
# Display user message.
elif ! displayMessage "Checking cluster availability"
then
log 'Failed to display user message'
let status="${status} + 1"
return 1
# Determine if status json says the database is available.
else
avail=`"${BINDIR}/fdbcli" -C "${FDBCONF}" --exec 'status json' --timeout 10 2> /dev/null | grep -E '"database_available"|"available"' | grep 'true'`
log "Avail value: ${avail}" "${DEBUGLEVEL}"
if [[ -n "${avail}" ]] ; then
return 0
else
return 1
fi
fi
if [ -z "${FDBSERVERID}" ]; then
log "FDB Server process is not defined."
let status="${status} + 1"
# Verify that the server is running.
elif ! kill -0 "${FDBSERVERID}"
then
log "FDB server process (${FDBSERVERID}) is not running"
let status="${status} + 1"
# Display user message.
elif ! displayMessage "Checking cluster availability"
then
log 'Failed to display user message'
let status="${status} + 1"
# Determine if status json says the database is available.
else
avail=`"${BINDIR}/fdbcli" -C "${FDBCONF}" --exec 'status json' --timeout "${SERVERCHECKS}" 2> /dev/null | grep -E '"database_available"|"available"' | grep 'true'`
log "Avail value: ${avail}" "${DEBUGLEVEL}"
if [[ -n "${avail}" ]] ; then
:
else
let status="${status} + 1"
fi
fi
return "${status}"
}
# Configure the database on the server.
function createDatabase {
if [ "${status}" -ne 0 ]; then
:
# Ensure that the server is running
elif ! kill -0 "${FDBSERVERID}"
then
log "FDB server process: (${FDBSERVERID}) is not running"
let status="${status} + 1"
function createDatabase
{
local status=0
# Display user message
elif ! displayMessage "Creating database"
then
log 'Failed to display user message'
let status="${status} + 1"
elif ! echo "Client log:" &> "${LOGDIR}/fdbclient.log"
then
log 'Failed to create fdbclient.log'
let status="${status} + 1"
elif ! getStatus
then
log 'Failed to get status'
let status="${status} + 1"
if [ "${status}" -ne 0 ]; then
:
# Ensure that the server is running
elif ! kill -0 "${FDBSERVERID}"
then
log "FDB server process: (${FDBSERVERID}) is not running"
let status="${status} + 1"
# Configure the database.
else
"${BINDIR}/fdbcli" -C "${FDBCONF}" --exec 'configure new single memory; status' --timeout 240 --log --log-dir "${LOGDIR}" &>> "${LOGDIR}/fdbclient.log"
# Display user message
elif ! displayMessage "Creating database"
then
log 'Failed to display user message'
let status="${status} + 1"
elif ! echo "Client log:" &> "${LOGDIR}/fdbclient.log"
then
log 'Failed to create fdbclient.log'
let status="${status} + 1"
elif ! getStatus
then
log 'Failed to get status'
let status="${status} + 1"
if ! displayMessage "Checking if config succeeded"
then
log 'Failed to display user message.'
fi
# Configure the database.
else
"${BINDIR}/fdbcli" -C "${FDBCONF}" --exec 'configure new single memory; status' --timeout "${CONFIGUREWAIT}" --log --log-dir "${LOGDIR}" &>> "${LOGDIR}/fdbclient.log"
iteration=0
while [[ "${iteration}" -lt 10 ]] && ! verifyAvailable
do
log "Database not created (iteration ${iteration})."
let iteration="${iteration} + 1"
done
if ! displayMessage "Checking if config succeeded"
then
log 'Failed to display user message.'
fi
if ! verifyAvailable
then
log "Failed to create database via cli"
getStatus
cat "${LOGDIR}/fdbclient.log"
log "Ignoring -- moving on"
#let status="${status} + 1"
fi
fi
iteration=0
while [[ "${iteration}" -lt "${SERVERCHECKS}" ]] && ! verifyAvailable
do
log "Database not created (iteration ${iteration})."
let iteration="${iteration} + 1"
done
return ${status}
if ! verifyAvailable
then
log "Failed to create database via cli"
getStatus
cat "${LOGDIR}/fdbclient.log"
log "Ignoring -- moving on"
#let status="${status} + 1"
fi
fi
return ${status}
}
# Begin the local cluster from scratch.
function startCluster {
if [ "${status}" -ne 0 ]; then
:
elif ! createDirectories
then
log "Could not create directories."
let status="${status} + 1"
elif ! createClusterFile
then
log "Could not create cluster file."
let status="${status} + 1"
elif ! startFdbServer
then
log "Could not start FDB server."
let status="${status} + 1"
elif ! createDatabase
then
log "Could not create database."
let status="${status} + 1"
fi
function startCluster
{
local status=0
return ${status}
if [ "${status}" -ne 0 ]; then
:
elif ! createDirectories
then
log "Could not create directories."
let status="${status} + 1"
elif ! createClusterFile
then
log "Could not create cluster file."
let status="${status} + 1"
elif ! startFdbServer
then
log "Could not start FDB server."
let status="${status} + 1"
elif ! createDatabase
then
log "Could not create database."
let status="${status} + 1"
fi
return ${status}
}

View File

@ -263,9 +263,9 @@ See :ref:`developer-guide-programming-with-futures` for further (language-indepe
.. function:: fdb_error_t fdb_future_block_until_ready(FDBFuture* future)
Blocks the calling thread until the given Future is ready. It will return success even if the Future is set to an error -- you must call :func:`fdb_future_get_error()` to determine that. :func:`fdb_future_block_until_ready()` will return an error only in exceptional conditions (e.g. out of memory or other operating system resources).
Blocks the calling thread until the given Future is ready. It will return success even if the Future is set to an error -- you must call :func:`fdb_future_get_error()` to determine that. :func:`fdb_future_block_until_ready()` will return an error only in exceptional conditions (e.g. deadlock detected, out of memory or other operating system resources).
.. warning:: Never call this function from a callback passed to :func:`fdb_future_set_callback()`. This may block the thread on which :func:`fdb_run_network()` was invoked, resulting in a deadlock.
.. warning:: Never call this function from a callback passed to :func:`fdb_future_set_callback()`. This may block the thread on which :func:`fdb_run_network()` was invoked, resulting in a deadlock. In some cases the client can detect the deadlock and throw a ``blocked_from_network_thread`` error.
.. function:: fdb_bool_t fdb_future_is_ready(FDBFuture* future)

View File

@ -114,8 +114,12 @@ FoundationDB may return the following error codes from API functions. If you nee
+-----------------------------------------------+-----+--------------------------------------------------------------------------------+
| transaction_read_only | 2023| Attempted to commit a transaction specified as read-only |
+-----------------------------------------------+-----+--------------------------------------------------------------------------------+
| invalid_cache_eviction_policy | 2024| Invalid cache eviction policy, only random and lru are supported |
+-----------------------------------------------+-----+--------------------------------------------------------------------------------+
| network_cannot_be_restarted | 2025| Network can only be started once |
+-----------------------------------------------+-----+--------------------------------------------------------------------------------+
| blocked_from_network_thread | 2026| Detected a deadlock in a callback called from the network thread |
+-----------------------------------------------+-----+--------------------------------------------------------------------------------+
| incompatible_protocol_version | 2100| Incompatible protocol version |
+-----------------------------------------------+-----+--------------------------------------------------------------------------------+
| transaction_too_large | 2101| Transaction exceeds byte limit |

View File

@ -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.9.pkg <https://www.foundationdb.org/downloads/6.3.9/macOS/installers/FoundationDB-6.3.9.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.9-1_amd64.deb <https://www.foundationdb.org/downloads/6.3.9/ubuntu/installers/foundationdb-clients_6.3.9-1_amd64.deb>`_
* `foundationdb-server-6.3.9-1_amd64.deb <https://www.foundationdb.org/downloads/6.3.9/ubuntu/installers/foundationdb-server_6.3.9-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.9-1.el6.x86_64.rpm <https://www.foundationdb.org/downloads/6.3.9/rhel6/installers/foundationdb-clients-6.3.9-1.el6.x86_64.rpm>`_
* `foundationdb-server-6.3.9-1.el6.x86_64.rpm <https://www.foundationdb.org/downloads/6.3.9/rhel6/installers/foundationdb-server-6.3.9-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.9-1.el7.x86_64.rpm <https://www.foundationdb.org/downloads/6.3.9/rhel7/installers/foundationdb-clients-6.3.9-1.el7.x86_64.rpm>`_
* `foundationdb-server-6.3.9-1.el7.x86_64.rpm <https://www.foundationdb.org/downloads/6.3.9/rhel7/installers/foundationdb-server-6.3.9-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.9-x64.msi <https://www.foundationdb.org/downloads/6.3.9/windows/installers/foundationdb-6.3.9-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.9.tar.gz <https://www.foundationdb.org/downloads/6.3.9/bindings/python/foundationdb-6.3.9.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.9.gem <https://www.foundationdb.org/downloads/6.3.9/bindings/ruby/fdb-6.3.9.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.9.jar <https://www.foundationdb.org/downloads/6.3.9/bindings/java/fdb-java-6.3.9.jar>`_
* `fdb-java-6.3.9-javadoc.jar <https://www.foundationdb.org/downloads/6.3.9/bindings/java/fdb-java-6.3.9-javadoc.jar>`_
Go 1.11+
--------

View File

@ -2,53 +2,47 @@
Release Notes
#############
6.2.24
6.2.26
======
Features
--------
* Fixed undefined behavior in configuring supported FoundationDB versions while starting up a client. `(PR #3849) <https://github.com/apple/foundationdb/pull/3849>`_
* Updated OpenSSL to version 1.1.1h. `(PR #3809) <https://github.com/apple/foundationdb/pull/3809>`_
* Attempt to detect when calling :func:`fdb_future_block_until_ready` would cause a deadlock, and throw ``blocked_from_network_thread`` if it would definitely cause a deadlock. `(PR #3786) <https://github.com/apple/foundationdb/pull/3786>`_
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
======
* Added the ``suspend`` command to ``fdbcli`` which kills a process and prevents it from rejoining the cluster for a specified duration. `(PR #3550) <https://github.com/apple/foundationdb/pull/3550>`_
6.2.23
======
Fixes
-----
* When configured with ``usable_regions=2`` data distribution could temporarily lower the replication of a shard when moving it. `(PR #3487) <https://github.com/apple/foundationdb/pull/3487>`_
* Prevent data distribution from running out of memory by fetching the source servers for too many shards in parallel. `(PR #3487) <https://github.com/apple/foundationdb/pull/3487>`_
* Reset network connections between log routers and satellite tlogs if the latencies are larger than 500ms. `(PR #3487) <https://github.com/apple/foundationdb/pull/3487>`_
Status
------
* Added per-process server request latency statistics reported in the role section of relevant processes. These are named ``grv_latency_statistics`` and ``commit_latency_statistics`` on proxy roles and ``read_latency_statistics`` on storage roles. `(PR #3480) <https://github.com/apple/foundationdb/pull/3480>`_
* Added ``cluster.active_primary_dc`` that indicates which datacenter is serving as the primary datacenter in multi-region setups. `(PR #3320) <https://github.com/apple/foundationdb/pull/3320>`_
6.2.22
======
Fixes
-----
* Coordinator class processes could be recruited as the cluster controller. `(PR #3282) <https://github.com/apple/foundationdb/pull/3282>`_
* HTTPS requests made by backup would fail (introduced in 6.2.21). `(PR #3284) <https://github.com/apple/foundationdb/pull/3284>`_
6.2.21
======
Fixes
-----
* HTTPS requests made by backup could hang indefinitely. `(PR #3027) <https://github.com/apple/foundationdb/pull/3027>`_
* ``fdbrestore`` prefix options required exactly a single hyphen instead of the standard two. `(PR #3056) <https://github.com/apple/foundationdb/pull/3056>`_
* Commits could stall on a newly elected proxy because of inaccurate compute estimates. `(PR #3123) <https://github.com/apple/foundationdb/pull/3123>`_
* A transaction class process with a bad disk could be repeatedly recruited as a transaction log. `(PR #3268) <https://github.com/apple/foundationdb/pull/3268>`_
* Fix a potential race condition that could lead to undefined behavior when connecting to a database using the multi-version client API. `(PR #3265) <https://github.com/apple/foundationdb/pull/3265>`_
Features
--------
* Added the ``getversion`` command to ``fdbcli`` which returns the current read version of the cluster. `(PR #2882) <https://github.com/apple/foundationdb/pull/2882>`_
* Added the ``advanceversion`` command to ``fdbcli`` which increases the current version of a cluster. `(PR #2965) <https://github.com/apple/foundationdb/pull/2965>`_
* Added the ``lock`` and ``unlock`` commands to ``fdbcli`` which lock or unlock a cluster. `(PR #2890) <https://github.com/apple/foundationdb/pull/2890>`_
@ -56,9 +50,6 @@ Features
6.2.20
======
Fixes
-----
* In rare scenarios, clients could send corrupted data to the server. `(PR #2976) <https://github.com/apple/foundationdb/pull/2976>`_
* Internal tools like ``fdbbackup`` are no longer tracked as clients in status (introduced in 6.2.18) `(PR #2849) <https://github.com/apple/foundationdb/pull/2849>`_
* Changed TLS error handling to match the behavior of 6.2.15. `(PR #2993) <https://github.com/apple/foundationdb/pull/2993>`_ `(PR #2977) <https://github.com/apple/foundationdb/pull/2977>`_
@ -66,9 +57,6 @@ Fixes
6.2.19
======
Fixes
-----
* Protect the proxies from running out of memory when bombarded with requests from clients. `(PR #2812) <https://github.com/apple/foundationdb/pull/2812>`_.
* One process with a ``proxy`` class would not become the first proxy when put with other ``stateless`` class processes. `(PR #2819) <https://github.com/apple/foundationdb/pull/2819>`_.
* If a transaction log stalled on a disk operation during recruitment the cluster would become unavailable until the process died. `(PR #2815) <https://github.com/apple/foundationdb/pull/2815>`_.
@ -76,70 +64,37 @@ Fixes
* Prevent the cluster from having too many active generations as a safety measure against repeated failures. `(PR #2814) <https://github.com/apple/foundationdb/pull/2814>`_.
* ``fdbcli`` status JSON could become truncated because of unprintable characters. `(PR #2807) <https://github.com/apple/foundationdb/pull/2807>`_.
* The data distributor used too much CPU in large clusters (broken in 6.2.16). `(PR #2806) <https://github.com/apple/foundationdb/pull/2806>`_.
Status
------
* Added ``cluster.workload.operations.memory_errors`` to measure the number of requests rejected by the proxies because the memory limit has been exceeded. `(PR #2812) <https://github.com/apple/foundationdb/pull/2812>`_.
* Added ``cluster.workload.operations.location_requests`` to measure the number of outgoing key server location responses from the proxies. `(PR #2812) <https://github.com/apple/foundationdb/pull/2812>`_.
* Added ``cluster.recovery_state.active_generations`` to track the number of generations for which the cluster still requires transaction logs. `(PR #2814) <https://github.com/apple/foundationdb/pull/2814>`_.
* Added ``network.tls_policy_failures`` to the ``processes`` section to record the number of TLS policy failures each process has observed. `(PR #2811) <https://github.com/apple/foundationdb/pull/2811>`_.
Features
--------
* Added ``--debug-tls`` as a command line argument to ``fdbcli`` to help diagnose TLS issues. `(PR #2810) <https://github.com/apple/foundationdb/pull/2810>`_.
6.2.18
======
Fixes
-----
* When configuring a cluster to usable_regions=2, data distribution would not react to machine failures while copying data to the remote region. `(PR #2774) <https://github.com/apple/foundationdb/pull/2774>`_.
* When a cluster is configured with usable_regions=2, data distribution could push a cluster into saturation by relocating too many shards simulatenously. `(PR #2776) <https://github.com/apple/foundationdb/pull/2776>`_.
* Do not allow the cluster controller to mark any process as failed within 30 seconds of startup. `(PR #2780) <https://github.com/apple/foundationdb/pull/2780>`_.
* Backup could not establish TLS connections (broken in 6.2.16). `(PR #2775) <https://github.com/apple/foundationdb/pull/2775>`_.
* Certificates were not refreshed automatically (broken in 6.2.16). `(PR #2781) <https://github.com/apple/foundationdb/pull/2781>`_.
Performance
-----------
* Improved the efficiency of establishing large numbers of network connections. `(PR #2777) <https://github.com/apple/foundationdb/pull/2777>`_.
Features
--------
* Add support for setting knobs to modify the behavior of ``fdbcli``. `(PR #2773) <https://github.com/apple/foundationdb/pull/2773>`_.
Other Changes
-------------
* Setting invalid knobs in backup and DR binaries is now a warning instead of an error and will not result in the application being terminated. `(PR #2773) <https://github.com/apple/foundationdb/pull/2773>`_.
6.2.17
======
Fixes
-----
* Restored the ability to set TLS configuration using environment variables (broken in 6.2.16). `(PR #2755) <https://github.com/apple/foundationdb/pull/2755>`_.
6.2.16
======
Performance
-----------
* Reduced tail commit latencies by improving commit pipelining on the proxies. `(PR #2589) <https://github.com/apple/foundationdb/pull/2589>`_.
* Data distribution does a better job balancing data when disks are more than 70% full. `(PR #2722) <https://github.com/apple/foundationdb/pull/2722>`_.
* Reverse range reads could read too much data from disk, resulting in poor performance relative to forward range reads. `(PR #2650) <https://github.com/apple/foundationdb/pull/2650>`_.
* Switched from LibreSSL to OpenSSL to improve the speed of establishing connections. `(PR #2646) <https://github.com/apple/foundationdb/pull/2646>`_.
* The cluster controller does a better job avoiding multiple recoveries when first recruited. `(PR #2698) <https://github.com/apple/foundationdb/pull/2698>`_.
Fixes
-----
* Storage servers could fail to advance their version correctly in response to empty commits. `(PR #2617) <https://github.com/apple/foundationdb/pull/2617>`_.
* Status could not label more than 5 processes as proxies. `(PR #2653) <https://github.com/apple/foundationdb/pull/2653>`_.
* The ``TR_FLAG_DISABLE_MACHINE_TEAM_REMOVER``, ``TR_FLAG_REMOVE_MT_WITH_MOST_TEAMS``, ``TR_FLAG_DISABLE_SERVER_TEAM_REMOVER``, and ``BUGGIFY_ALL_COORDINATION`` knobs could not be set at runtime. `(PR #2661) <https://github.com/apple/foundationdb/pull/2661>`_.
@ -151,17 +106,11 @@ Fixes
6.2.15
======
Fixes
-----
* TLS throttling could block legitimate connections. `(PR #2575) <https://github.com/apple/foundationdb/pull/2575>`_.
6.2.14
======
Fixes
-----
* Data distribution was prioritizing shard merges too highly. `(PR #2562) <https://github.com/apple/foundationdb/pull/2562>`_.
* Status would incorrectly mark clusters as having no fault tolerance. `(PR #2562) <https://github.com/apple/foundationdb/pull/2562>`_.
* A proxy could run out of memory if disconnected from the cluster for too long. `(PR #2562) <https://github.com/apple/foundationdb/pull/2562>`_.
@ -169,26 +118,16 @@ Fixes
6.2.13
======
Performance
-----------
* Optimized the commit path the proxies to significantly reduce commit latencies in large clusters. `(PR #2536) <https://github.com/apple/foundationdb/pull/2536>`_.
* Data distribution could create temporarily untrackable shards which could not be split if they became hot. `(PR #2546) <https://github.com/apple/foundationdb/pull/2546>`_.
6.2.12
======
Performance
-----------
* Throttle TLS connect attempts from misconfigured clients. `(PR #2529) <https://github.com/apple/foundationdb/pull/2529>`_.
* Reduced master recovery times in large clusters. `(PR #2430) <https://github.com/apple/foundationdb/pull/2430>`_.
* Improved performance while a remote region is catching up. `(PR #2527) <https://github.com/apple/foundationdb/pull/2527>`_.
* The data distribution algorithm does a better job preventing hot shards while recovering from machine failures. `(PR #2526) <https://github.com/apple/foundationdb/pull/2526>`_.
Fixes
-----
* Improve the reliability of a ``kill`` command from ``fdbcli``. `(PR #2512) <https://github.com/apple/foundationdb/pull/2512>`_.
* The ``--traceclock`` parameter to fdbserver incorrectly had no effect. `(PR #2420) <https://github.com/apple/foundationdb/pull/2420>`_.
* Clients could throw an internal error during ``commit`` if client buggification was enabled. `(PR #2427) <https://github.com/apple/foundationdb/pull/2427>`_.
@ -198,9 +137,6 @@ Fixes
6.2.11
======
Fixes
-----
* Clients could hang indefinitely on reads if all storage servers holding a keyrange were removed from a cluster since the last time the client read a key in the range. `(PR #2377) <https://github.com/apple/foundationdb/pull/2377>`_.
* In rare scenarios, status could falsely report no replicas remain of some data. `(PR #2380) <https://github.com/apple/foundationdb/pull/2380>`_.
* Latency band tracking could fail to configure correctly after a recovery or upon process startup. `(PR #2371) <https://github.com/apple/foundationdb/pull/2371>`_.
@ -208,17 +144,11 @@ Fixes
6.2.10
======
Fixes
-----
* ``backup_agent`` crashed on startup. `(PR #2356) <https://github.com/apple/foundationdb/pull/2356>`_.
6.2.9
=====
Fixes
-----
* Small clusters using specific sets of process classes could cause the data distributor to be continuously killed and re-recruited. `(PR #2344) <https://github.com/apple/foundationdb/pull/2344>`_.
* The data distributor and ratekeeper could be recruited on non-optimal processes. `(PR #2344) <https://github.com/apple/foundationdb/pull/2344>`_.
* A ``kill`` command from ``fdbcli`` could take a long time before being executed by a busy process. `(PR #2339) <https://github.com/apple/foundationdb/pull/2339>`_.
@ -228,9 +158,6 @@ Fixes
6.2.8
=====
Fixes
-----
* Significantly improved the rate at which the transaction logs in a remote region can pull data from the primary region. `(PR #2307) <https://github.com/apple/foundationdb/pull/2307>`_ `(PR #2323) <https://github.com/apple/foundationdb/pull/2323>`_.
* The ``system_kv_size_bytes`` status field could report a size much larger than the actual size of the system keyspace. `(PR #2305) <https://github.com/apple/foundationdb/pull/2305>`_.

View File

@ -4,13 +4,7 @@
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
6.3.9
=====
Features
@ -68,6 +62,9 @@ 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>`_
* Commit latencies could become large because of inaccurate compute estimates. [6.3.9] `(PR #3845) <https://github.com/apple/foundationdb/pull/3845>`_
* Added a timeout on TLS handshakes to prevent them from hanging indefinitely. [6.3.9] `(PR #3850) <https://github.com/apple/foundationdb/pull/3850>`_
Status
------
@ -76,6 +73,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
--------
@ -86,6 +84,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>`_
@ -112,12 +112,17 @@ Other Changes
* Updated boost to 1.72. `(PR #2684) <https://github.com/apple/foundationdb/pull/2684>`_
* Calling ``fdb_run_network`` multiple times in a single run of a client program now returns an error instead of causing undefined behavior. [6.3.1] `(PR #3229) <https://github.com/apple/foundationdb/pull/3229>`_
* Blob backup URL parameter ``request_timeout`` changed to ``request_timeout_min``, with prior name still supported. `(PR #3533) <https://github.com/apple/foundationdb/pull/3533>`_
* Support query command in backup CLI that allows users to query restorable files by key ranges. [6.3.6] `(PR #3703) <https://github.com/apple/foundationdb/pull/3703>`_
* Report missing old tlogs information when in recovery before storage servers are fully recovered. [6.3.6] `(PR #3706) <https://github.com/apple/foundationdb/pull/3706>`_
* Updated OpenSSL to version 1.1.1h. [6.3.7] `(PR #3809) <https://github.com/apple/foundationdb/pull/3809>`_
* Lowered the amount of time a watch will remain registered on a storage server from 900 seconds to 30 seconds. [6.3.8] `(PR #3833) <https://github.com/apple/foundationdb/pull/3833>`_
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+
---------------------------
@ -126,6 +131,10 @@ 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>`_
* Only return the error code ``batch_transaction_throttled`` for API versions greater than or equal to 630. [6.3.6] `(PR #3799) <https://github.com/apple/foundationdb/pull/3799>`_
* The fault tolerance calculation in status did not take into account region configurations. [6.3.8] `(PR #3836) <https://github.com/apple/foundationdb/pull/3836>`_
* Get read version tail latencies were high because some proxies were serving more read versions than other proxies. [6.3.9] `(PR #3845) <https://github.com/apple/foundationdb/pull/3845>`_
Earlier release notes
---------------------

View File

@ -18,6 +18,10 @@
* limitations under the License.
*/
#include "fdbclient/JsonBuilder.h"
#include "flow/Arena.h"
#include "flow/Error.h"
#include "flow/Trace.h"
#define BOOST_DATE_TIME_NO_LIB
#include <boost/interprocess/managed_shared_memory.hpp>
@ -81,7 +85,22 @@ enum enumProgramExe {
};
enum enumBackupType {
BACKUP_UNDEFINED=0, BACKUP_START, BACKUP_MODIFY, BACKUP_STATUS, BACKUP_ABORT, BACKUP_WAIT, BACKUP_DISCONTINUE, BACKUP_PAUSE, BACKUP_RESUME, BACKUP_EXPIRE, BACKUP_DELETE, BACKUP_DESCRIBE, BACKUP_LIST, BACKUP_DUMP, BACKUP_CLEANUP
BACKUP_UNDEFINED = 0,
BACKUP_START,
BACKUP_MODIFY,
BACKUP_STATUS,
BACKUP_ABORT,
BACKUP_WAIT,
BACKUP_DISCONTINUE,
BACKUP_PAUSE,
BACKUP_RESUME,
BACKUP_EXPIRE,
BACKUP_DELETE,
BACKUP_DESCRIBE,
BACKUP_LIST,
BACKUP_QUERY,
BACKUP_DUMP,
BACKUP_CLEANUP
};
enum enumDBType {
@ -104,7 +123,7 @@ enum {
OPT_USE_PARTITIONED_LOG,
// Backup and Restore constants
OPT_TAGNAME, OPT_BACKUPKEYS, OPT_WAITFORDONE,
OPT_TAGNAME, OPT_BACKUPKEYS, OPT_WAITFORDONE, OPT_BACKUPKEYS_FILTER,
// Backup Modify
OPT_MOD_ACTIVE_INTERVAL, OPT_MOD_VERIFY_UID,
@ -585,6 +604,40 @@ CSimpleOpt::SOption g_rgBackupListOptions[] = {
SO_END_OF_OPTIONS
};
CSimpleOpt::SOption g_rgBackupQueryOptions[] = {
#ifdef _WIN32
{ OPT_PARENTPID, "--parentpid", SO_REQ_SEP },
#endif
{ OPT_RESTORE_TIMESTAMP, "--query_restore_timestamp", SO_REQ_SEP },
{ OPT_DESTCONTAINER, "-d", SO_REQ_SEP },
{ OPT_DESTCONTAINER, "--destcontainer", SO_REQ_SEP },
{ OPT_RESTORE_VERSION, "-qrv", SO_REQ_SEP },
{ OPT_RESTORE_VERSION, "--query_restore_version", SO_REQ_SEP },
{ OPT_BACKUPKEYS_FILTER, "-k", SO_REQ_SEP },
{ OPT_BACKUPKEYS_FILTER, "--keys", SO_REQ_SEP },
{ OPT_TRACE, "--log", SO_NONE },
{ OPT_TRACE_DIR, "--logdir", SO_REQ_SEP },
{ OPT_TRACE_FORMAT, "--trace_format", SO_REQ_SEP },
{ OPT_TRACE_LOG_GROUP, "--loggroup", SO_REQ_SEP },
{ OPT_QUIET, "-q", SO_NONE },
{ OPT_QUIET, "--quiet", SO_NONE },
{ OPT_VERSION, "-v", SO_NONE },
{ OPT_VERSION, "--version", SO_NONE },
{ OPT_CRASHONERROR, "--crash", SO_NONE },
{ OPT_MEMLIMIT, "-m", SO_REQ_SEP },
{ OPT_MEMLIMIT, "--memory", SO_REQ_SEP },
{ OPT_HELP, "-?", SO_NONE },
{ OPT_HELP, "-h", SO_NONE },
{ OPT_HELP, "--help", SO_NONE },
{ OPT_DEVHELP, "--dev-help", SO_NONE },
{ OPT_BLOB_CREDENTIALS, "--blob_credentials", SO_REQ_SEP },
{ OPT_KNOB, "--knob_", SO_REQ_SEP },
#ifndef TLS_DISABLED
TLS_OPTION_FLAGS
#endif
SO_END_OF_OPTIONS
};
// g_rgRestoreOptions is used by fdbrestore and fastrestore_tool
CSimpleOpt::SOption g_rgRestoreOptions[] = {
#ifdef _WIN32
@ -918,13 +971,16 @@ void printBackupContainerInfo() {
static void printBackupUsage(bool devhelp) {
printf("FoundationDB " FDB_VT_PACKAGE_NAME " (v" FDB_VT_VERSION ")\n");
printf("Usage: %s (start | status | abort | wait | discontinue | pause | resume | expire | delete | describe | list | cleanup) [OPTIONS]\n\n", exeBackup.toString().c_str());
printf("Usage: %s (start | status | abort | wait | discontinue | pause | resume | expire | delete | describe | "
"list | query | cleanup) [OPTIONS]\n\n",
exeBackup.toString().c_str());
printf(" -C CONNFILE The path of a file containing the connection string for the\n"
" FoundationDB cluster. The default is first the value of the\n"
" FDB_CLUSTER_FILE environment variable, then `./fdb.cluster',\n"
" then `%s'.\n", platform::getDefaultClusterFilePath().c_str());
printf(" -d, --destcontainer URL\n"
" The Backup container URL for start, modify, describe, expire, and delete operations.\n");
" The Backup container URL for start, modify, describe, query, expire, and delete "
"operations.\n");
printBackupContainerInfo();
printf(" -b, --base_url BASEURL\n"
" Base backup URL for list operations. This looks like a Backup URL but without a backup name.\n");
@ -938,6 +994,12 @@ static void printBackupUsage(bool devhelp) {
printf(" --delete_before_days NUM_DAYS\n"
" Another way to specify version cutoff for expire operations. Deletes data files containing no data at or after a\n"
" version approximately NUM_DAYS days worth of versions prior to the latest log version in the backup.\n");
printf(" -qrv --query_restore_version VERSION\n"
" For query operations, set target version for restoring a backup. Set -1 for maximum\n"
" restorable version (default) and -2 for minimum restorable version.\n");
printf(" --query_restore_timestamp DATETIME\n"
" For query operations, instead of a numeric version, use this to specify a timestamp in %s\n", BackupAgentBase::timeFormat().c_str());
printf(" and it will be converted to a version from that time using metadata in the cluster file.\n");
printf(" --restorable_after_timestamp DATETIME\n"
" For expire operations, set minimum acceptable restorability to the version equivalent of DATETIME and later.\n");
printf(" --restorable_after_version VERSION\n"
@ -956,8 +1018,8 @@ static void printBackupUsage(bool devhelp) {
" Specifies a UID to verify against the BackupUID of the running backup. If provided, the UID is verified in the same transaction\n"
" which sets the new backup parameters (if the UID matches).\n");
printf(" -e ERRORLIMIT The maximum number of errors printed by status (default is 10).\n");
printf(" -k KEYS List of key ranges to backup.\n"
" If not specified, the entire database will be backed up.\n");
printf(" -k KEYS List of key ranges to backup or to filter the backup in query operations.\n"
" If not specified, the entire database will be backed up or no filter will be applied.\n");
printf(" --partitioned_log_experimental Starts with new type of backup system using partitioned logs.\n");
printf(" -n, --dryrun For backup start or restore start, performs a trial run with no actual changes made.\n");
printf(" --log Enables trace file logging for the CLI session.\n"
@ -1273,6 +1335,7 @@ enumBackupType getBackupType(std::string backupType)
values["delete"] = BACKUP_DELETE;
values["describe"] = BACKUP_DESCRIBE;
values["list"] = BACKUP_LIST;
values["query"] = BACKUP_QUERY;
values["dump"] = BACKUP_DUMP;
values["modify"] = BACKUP_MODIFY;
}
@ -2400,6 +2463,135 @@ ACTOR Future<Void> describeBackup(const char *name, std::string destinationConta
return Void();
}
static void reportBackupQueryError(UID operationId, JsonBuilderObject& result, std::string errorMessage) {
result["error"] = errorMessage;
printf("%s\n", result.getJson().c_str());
TraceEvent("BackupQueryFailure").detail("OperationId", operationId).detail("Reason", errorMessage);
}
// If restoreVersion is invalidVersion or latestVersion, use the maximum or minimum restorable version respectively for
// selected key ranges. If restoreTimestamp is specified, any specified restoreVersion will be overriden to the version
// resolved to that timestamp.
ACTOR Future<Void> queryBackup(const char* name, std::string destinationContainer,
Standalone<VectorRef<KeyRangeRef>> keyRangesFilter, Version restoreVersion,
std::string originalClusterFile, std::string restoreTimestamp, bool verbose) {
state UID operationId = deterministicRandom()->randomUniqueID();
state JsonBuilderObject result;
state std::string errorMessage;
result["key_ranges_filter"] = printable(keyRangesFilter);
result["destination_container"] = destinationContainer;
TraceEvent("BackupQueryStart")
.detail("OperationId", operationId)
.detail("DestinationContainer", destinationContainer)
.detail("KeyRangesFilter", printable(keyRangesFilter))
.detail("SpecifiedRestoreVersion", restoreVersion)
.detail("RestoreTimestamp", restoreTimestamp)
.detail("BackupClusterFile", originalClusterFile);
// Resolve restoreTimestamp if given
if (!restoreTimestamp.empty()) {
if (originalClusterFile.empty()) {
reportBackupQueryError(
operationId, result,
format("an original cluster file must be given in order to resolve restore target timestamp '%s'",
restoreTimestamp.c_str()));
return Void();
}
if (!fileExists(originalClusterFile)) {
reportBackupQueryError(operationId, result,
format("The specified original source database cluster file '%s' does not exist\n",
originalClusterFile.c_str()));
return Void();
}
Database origDb = Database::createDatabase(originalClusterFile, Database::API_VERSION_LATEST);
Version v = wait(timeKeeperVersionFromDatetime(restoreTimestamp, origDb));
result["restore_timestamp"] = restoreTimestamp;
result["restore_timestamp_resolved_version"] = v;
restoreVersion = v;
}
try {
state Reference<IBackupContainer> bc = openBackupContainer(name, destinationContainer);
if (restoreVersion == invalidVersion) {
BackupDescription desc = wait(bc->describeBackup());
if (desc.maxRestorableVersion.present()) {
restoreVersion = desc.maxRestorableVersion.get();
// Use continuous log end version for the maximum restorable version for the key ranges.
} else if (keyRangesFilter.size() && desc.contiguousLogEnd.present()) {
restoreVersion = desc.contiguousLogEnd.get();
} else {
reportBackupQueryError(
operationId, result,
errorMessage = format("the backup for the specified key ranges is not restorable to any version"));
}
}
if (restoreVersion < 0 && restoreVersion != latestVersion) {
reportBackupQueryError(operationId, result,
errorMessage =
format("the specified restorable version %ld is not valid", restoreVersion));
return Void();
}
Optional<RestorableFileSet> fileSet = wait(bc->getRestoreSet(restoreVersion, keyRangesFilter));
if (fileSet.present()) {
int64_t totalRangeFilesSize = 0, totalLogFilesSize = 0;
result["restore_version"] = fileSet.get().targetVersion;
JsonBuilderArray rangeFilesJson;
JsonBuilderArray logFilesJson;
for (const auto& rangeFile : fileSet.get().ranges) {
JsonBuilderObject object;
object["file_name"] = rangeFile.fileName;
object["file_size"] = rangeFile.fileSize;
object["version"] = rangeFile.version;
object["key_range"] = fileSet.get().keyRanges.count(rangeFile.fileName) == 0
? "none"
: fileSet.get().keyRanges.at(rangeFile.fileName).toString();
rangeFilesJson.push_back(object);
totalRangeFilesSize += rangeFile.fileSize;
}
for (const auto& log : fileSet.get().logs) {
JsonBuilderObject object;
object["file_name"] = log.fileName;
object["file_size"] = log.fileSize;
object["begin_version"] = log.beginVersion;
object["end_version"] = log.endVersion;
logFilesJson.push_back(object);
totalLogFilesSize += log.fileSize;
}
result["total_range_files_size"] = totalRangeFilesSize;
result["total_log_files_size"] = totalLogFilesSize;
if (verbose) {
result["ranges"] = rangeFilesJson;
result["logs"] = logFilesJson;
}
TraceEvent("BackupQueryReceivedRestorableFilesSet")
.detail("DestinationContainer", destinationContainer)
.detail("KeyRangesFilter", printable(keyRangesFilter))
.detail("ActualRestoreVersion", fileSet.get().targetVersion)
.detail("NumRangeFiles", fileSet.get().ranges.size())
.detail("NumLogFiles", fileSet.get().logs.size())
.detail("RangeFilesBytes", totalRangeFilesSize)
.detail("LogFilesBytes", totalLogFilesSize);
} else {
reportBackupQueryError(operationId, result, "no restorable files set found for specified key ranges");
return Void();
}
} catch (Error& e) {
reportBackupQueryError(operationId, result, e.what());
return Void();
}
printf("%s\n", result.getJson().c_str());
return Void();
}
ACTOR Future<Void> listBackup(std::string baseUrl) {
try {
std::vector<std::string> containers = wait(IBackupContainer::listContainers(baseUrl));
@ -2769,6 +2961,9 @@ int main(int argc, char* argv[]) {
case BACKUP_LIST:
args = new CSimpleOpt(argc - 1, &argv[1], g_rgBackupListOptions, SO_O_EXACT);
break;
case BACKUP_QUERY:
args = new CSimpleOpt(argc - 1, &argv[1], g_rgBackupQueryOptions, SO_O_EXACT);
break;
case BACKUP_MODIFY:
args = new CSimpleOpt(argc - 1, &argv[1], g_rgBackupModifyOptions, SO_O_EXACT);
break;
@ -2908,6 +3103,7 @@ int main(int argc, char* argv[]) {
std::string addPrefix;
std::string removePrefix;
Standalone<VectorRef<KeyRangeRef>> backupKeys;
Standalone<VectorRef<KeyRangeRef>> backupKeysFilter;
int maxErrors = 20;
Version restoreVersion = invalidVersion;
std::string restoreTimestamp;
@ -3128,6 +3324,15 @@ int main(int argc, char* argv[]) {
return FDB_EXIT_ERROR;
}
break;
case OPT_BACKUPKEYS_FILTER:
try {
addKeyRange(args->OptionArg(), backupKeysFilter);
}
catch (Error &) {
printHelpTeaser(argv[0]);
return FDB_EXIT_ERROR;
}
break;
case OPT_DESTCONTAINER:
destinationContainer = args->OptionArg();
// If the url starts with '/' then prepend "file://" for backwards compatibility
@ -3661,6 +3866,12 @@ int main(int argc, char* argv[]) {
f = stopAfter( listBackup(baseUrl) );
break;
case BACKUP_QUERY:
initTraceFile();
f = stopAfter(queryBackup(argv[0], destinationContainer, backupKeysFilter, restoreVersion,
restoreClusterFileOrig, restoreTimestamp, !quietDisplay));
break;
case BACKUP_DUMP:
initTraceFile();
f = stopAfter( dumpBackupData(argv[0], destinationContainer, dumpBegin, dumpEnd) );

View File

@ -20,6 +20,7 @@
#include "boost/lexical_cast.hpp"
#include "fdbclient/NativeAPI.actor.h"
#include "fdbclient/FDBTypes.h"
#include "fdbclient/Status.h"
#include "fdbclient/StatusClient.h"
#include "fdbclient/DatabaseContext.h"
@ -1029,10 +1030,10 @@ void printStatus(StatusObjectReader statusObj, StatusClient::StatusLevel level,
if (statusObjConfig.has("regions")) {
outputString += "\n Regions: ";
regions = statusObjConfig["regions"].get_array();
bool isPrimary = false;
std::vector<std::string> regionSatelliteDCs;
std::string regionDC;
for (StatusObjectReader region : regions) {
bool isPrimary = false;
std::vector<std::string> regionSatelliteDCs;
std::string regionDC;
for (StatusObjectReader dc : region["datacenters"].get_array()) {
if (!dc.has("satellite")) {
regionDC = dc["id"].get_str();
@ -1207,14 +1208,54 @@ void printStatus(StatusObjectReader statusObj, StatusClient::StatusLevel level,
int minLoss = std::min(availLoss, dataLoss);
const char *faultDomain = machinesAreZones ? "machine" : "zone";
if (minLoss == 1)
outputString += format("1 %s", faultDomain);
else
outputString += format("%d %ss", minLoss, faultDomain);
outputString += format("%d %ss", minLoss, faultDomain);
if (dataLoss > availLoss){
outputString += format(" (%d without data loss)", dataLoss);
}
if (dataLoss == -1) {
ASSERT_WE_THINK(availLoss == -1);
outputString += format(
"\n\n Warning: the database may have data loss and availability loss. Please restart "
"following tlog interfaces, otherwise storage servers may never be able to catch "
"up.\n");
StatusObjectReader logs;
if (statusObjCluster.has("logs")) {
for (StatusObjectReader logEpoch : statusObjCluster.last().get_array()) {
bool possiblyLosingData;
if (logEpoch.get("possibly_losing_data", possiblyLosingData) &&
!possiblyLosingData) {
continue;
}
// Current epoch doesn't have an end version.
int64_t epoch, beginVersion, endVersion = invalidVersion;
bool current;
logEpoch.get("epoch", epoch);
logEpoch.get("begin_version", beginVersion);
logEpoch.get("end_version", endVersion);
logEpoch.get("current", current);
std::string missing_log_interfaces;
if (logEpoch.has("log_interfaces")) {
for (StatusObjectReader logInterface : logEpoch.last().get_array()) {
bool healthy;
std::string address, id;
if (logInterface.get("healthy", healthy) && !healthy) {
logInterface.get("id", id);
logInterface.get("address", address);
missing_log_interfaces += format("%s,%s ", id.c_str(), address.c_str());
}
}
}
outputString += format(
" %s log epoch: %ld begin: %ld end: %s, missing "
"log interfaces(id,address): %s\n",
current ? "Current" : "Old", epoch, beginVersion,
endVersion == invalidVersion ? "(unknown)" : format("%ld", endVersion).c_str(),
missing_log_interfaces.c_str());
}
}
}
}
}

View File

@ -944,5 +944,7 @@ Value makePadding(int size);
ACTOR Future<Void> transformRestoredDatabase(Database cx, Standalone<VectorRef<KeyRangeRef>> backupRanges,
Key addPrefix, Key removePrefix);
void simulateBlobFailure();
#include "flow/unactorcompiler.h"
#endif

View File

@ -22,10 +22,12 @@
#include "fdbclient/BackupAgent.actor.h"
#include "fdbclient/FDBTypes.h"
#include "fdbclient/JsonBuilder.h"
#include "flow/Arena.h"
#include "flow/Trace.h"
#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"
@ -243,7 +245,7 @@ std::string BackupDescription::toJSON() const {
* file written will be after the start version of the snapshot's execution.
*
* Log files are at file paths like
* /plogs/...log,startVersion,endVersion,UID,tagID-of-N,blocksize
* /plogs/.../log,startVersion,endVersion,UID,tagID-of-N,blocksize
* /logs/.../log,startVersion,endVersion,UID,blockSize
* where ... is a multi level path which sorts lexically into version order and results in approximately 1
* unique folder per day containing about 5,000 files. Logs after FDB 6.3 are stored in "plogs"
@ -1341,19 +1343,44 @@ public:
ACTOR static Future<KeyRange> getSnapshotFileKeyRange_impl(Reference<BackupContainerFileSystem> bc,
RangeFile file) {
state Reference<IAsyncFile> inFile = wait(bc->readFile(file.fileName));
state int readFileRetries = 0;
state bool beginKeySet = false;
state Key beginKey;
state Key endKey;
state int64_t j = 0;
for (; j < file.fileSize; j += file.blockSize) {
int64_t len = std::min<int64_t>(file.blockSize, file.fileSize - j);
Standalone<VectorRef<KeyValueRef>> blockData = wait(fileBackup::decodeRangeFileBlock(inFile, j, len));
if (!beginKeySet) {
beginKey = blockData.front().key;
beginKeySet = true;
loop {
try {
state Reference<IAsyncFile> inFile = wait(bc->readFile(file.fileName));
beginKeySet = false;
state int64_t j = 0;
for (; j < file.fileSize; j += file.blockSize) {
int64_t len = std::min<int64_t>(file.blockSize, file.fileSize - j);
Standalone<VectorRef<KeyValueRef>> blockData =
wait(fileBackup::decodeRangeFileBlock(inFile, j, len));
if (!beginKeySet) {
beginKey = blockData.front().key;
beginKeySet = true;
}
endKey = blockData.back().key;
}
break;
} catch (Error& e) {
if (e.code() == error_code_restore_bad_read ||
e.code() == error_code_restore_unsupported_file_version ||
e.code() == error_code_restore_corrupted_data_padding) { // no retriable error
TraceEvent(SevError, "BackupContainerGetSnapshotFileKeyRange").error(e);
throw;
} else if (e.code() == error_code_http_request_failed || e.code() == error_code_connection_failed ||
e.code() == error_code_timed_out || e.code() == error_code_lookup_failed) {
// blob http request failure, retry
TraceEvent(SevWarnAlways, "BackupContainerGetSnapshotFileKeyRangeConnectionFailure")
.detail("Retries", ++readFileRetries)
.error(e);
wait(delayJittered(0.1));
} else {
TraceEvent(SevError, "BackupContainerGetSnapshotFileKeyRangeUnexpectedError").error(e);
throw;
}
}
endKey = blockData.back().key;
}
return KeyRange(KeyRangeRef(beginKey, endKey));
}
@ -1363,24 +1390,57 @@ public:
return getSnapshotFileKeyRange_impl(Reference<BackupContainerFileSystem>::addRef(this), file);
}
ACTOR static Future<Optional<RestorableFileSet>> getRestoreSet_impl(Reference<BackupContainerFileSystem> bc, Version targetVersion) {
// Find the most recent keyrange snapshot to end at or before targetVersion
state Optional<KeyspaceSnapshotFile> snapshot;
std::vector<KeyspaceSnapshotFile> snapshots = wait(bc->listKeyspaceSnapshots());
for(auto const &s : snapshots) {
if(s.endVersion <= targetVersion)
snapshot = s;
}
ACTOR static Future<Optional<RestorableFileSet>> getRestoreSet_impl(Reference<BackupContainerFileSystem> bc,
Version targetVersion,
VectorRef<KeyRangeRef> keyRangesFilter) {
// Find the most recent keyrange snapshot through which we can restore filtered key ranges into targetVersion.
state std::vector<KeyspaceSnapshotFile> snapshots = wait(bc->listKeyspaceSnapshots());
state int i = snapshots.size() - 1;
for (; i >= 0; i--) {
// The smallest version of filtered range files >= snapshot beginVersion > targetVersion
if (targetVersion >= 0 && snapshots[i].beginVersion > targetVersion) {
continue;
}
if(snapshot.present()) {
state RestorableFileSet restorable;
restorable.snapshot = snapshot.get();
restorable.targetVersion = targetVersion;
state Version minKeyRangeVersion = MAX_VERSION;
state Version maxKeyRangeVersion = -1;
std::pair<std::vector<RangeFile>, std::map<std::string, KeyRange>> results =
wait(bc->readKeyspaceSnapshot(snapshot.get()));
restorable.ranges = std::move(results.first);
restorable.keyRanges = std::move(results.second);
wait(bc->readKeyspaceSnapshot(snapshots[i]));
// Old backup does not have metadata about key ranges and can not be filtered with key ranges.
if (keyRangesFilter.size() && results.second.empty() && !results.first.empty()) {
throw backup_not_filterable_with_key_ranges();
}
// Filter by keyRangesFilter.
if (keyRangesFilter.empty()) {
restorable.ranges = std::move(results.first);
restorable.keyRanges = std::move(results.second);
minKeyRangeVersion = snapshots[i].beginVersion;
maxKeyRangeVersion = snapshots[i].endVersion;
} else {
for (const auto& rangeFile : results.first) {
const auto& keyRange = results.second.at(rangeFile.fileName);
if (keyRange.intersects(keyRangesFilter)) {
restorable.ranges.push_back(rangeFile);
restorable.keyRanges[rangeFile.fileName] = keyRange;
minKeyRangeVersion = std::min(minKeyRangeVersion, rangeFile.version);
maxKeyRangeVersion = std::max(maxKeyRangeVersion, rangeFile.version);
}
}
// No range file matches 'keyRangesFilter'.
if (restorable.ranges.empty()) {
throw backup_not_overlapped_with_keys_filter();
}
}
// 'latestVersion' represents using the minimum restorable version in a snapshot.
restorable.targetVersion = targetVersion == latestVersion ? maxKeyRangeVersion : targetVersion;
// Any version < maxKeyRangeVersion is not restorable.
if (restorable.targetVersion < maxKeyRangeVersion) continue;
restorable.snapshot = snapshots[i];
// TODO: Reenable the sanity check after TooManyFiles error is resolved
if (false && g_network->isSimulated()) {
// Sanity check key ranges
@ -1394,18 +1454,21 @@ public:
}
}
// No logs needed if there is a complete key space snapshot at the target version.
if (snapshot.get().beginVersion == snapshot.get().endVersion &&
snapshot.get().endVersion == targetVersion) {
// No logs needed if there is a complete filtered key space snapshot at the target version.
if (minKeyRangeVersion == maxKeyRangeVersion && maxKeyRangeVersion == restorable.targetVersion) {
restorable.continuousBeginVersion = restorable.continuousEndVersion = invalidVersion;
TraceEvent("BackupContainerGetRestorableFilesWithoutLogs")
.detail("KeyRangeVersion", restorable.targetVersion)
.detail("NumberOfRangeFiles", restorable.ranges.size())
.detail("KeyRangesFilter", printable(keyRangesFilter));
return Optional<RestorableFileSet>(restorable);
}
// FIXME: check if there are tagged logs. for each tag, there is no version gap.
state std::vector<LogFile> logs;
state std::vector<LogFile> plogs;
wait(store(logs, bc->listLogFiles(snapshot.get().beginVersion, targetVersion, false)) &&
store(plogs, bc->listLogFiles(snapshot.get().beginVersion, targetVersion, true)));
wait(store(logs, bc->listLogFiles(minKeyRangeVersion, restorable.targetVersion, false)) &&
store(plogs, bc->listLogFiles(minKeyRangeVersion, restorable.targetVersion, true)));
if (plogs.size() > 0) {
logs.swap(plogs);
@ -1417,13 +1480,12 @@ public:
// Remove duplicated log files that can happen for old epochs.
std::vector<LogFile> filtered = filterDuplicates(logs);
restorable.logs.swap(filtered);
// sort by version order again for continuous analysis
std::sort(restorable.logs.begin(), restorable.logs.end());
if (isPartitionedLogsContinuous(restorable.logs, snapshot.get().beginVersion, targetVersion)) {
restorable.continuousBeginVersion = snapshot.get().beginVersion;
restorable.continuousEndVersion = targetVersion + 1; // not inclusive
if (isPartitionedLogsContinuous(restorable.logs, minKeyRangeVersion, restorable.targetVersion)) {
restorable.continuousBeginVersion = minKeyRangeVersion;
restorable.continuousEndVersion = restorable.targetVersion + 1; // not inclusive
return Optional<RestorableFileSet>(restorable);
}
return Optional<RestorableFileSet>();
@ -1431,24 +1493,23 @@ public:
// List logs in version order so log continuity can be analyzed
std::sort(logs.begin(), logs.end());
// 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) {
if (!logs.empty() && logs.front().beginVersion <= minKeyRangeVersion) {
Version end = logs.begin()->endVersion;
computeRestoreEndVersion(logs, &restorable.logs, &end, targetVersion);
if (end >= targetVersion) {
computeRestoreEndVersion(logs, &restorable.logs, &end, restorable.targetVersion);
if (end >= restorable.targetVersion) {
restorable.continuousBeginVersion = logs.begin()->beginVersion;
restorable.continuousEndVersion = end;
return Optional<RestorableFileSet>(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,
VectorRef<KeyRangeRef> keyRangesFilter) final {
return getRestoreSet_impl(Reference<BackupContainerFileSystem>::addRef(this), targetVersion, keyRangesFilter);
}
private:
@ -1609,9 +1670,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;

View File

@ -280,9 +280,11 @@ public:
virtual Future<BackupFileList> dumpFileList(Version begin = 0, Version end = std::numeric_limits<Version>::max()) = 0;
// 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;
// Get exactly the files necessary to restore the key space filtered by the specified key ranges to targetVersion.
// If targetVersion is 'latestVersion', use the minimum restorable version in a snapshot. Returns non-present if
// restoring to the given version is not possible.
virtual Future<Optional<RestorableFileSet>> getRestoreSet(Version targetVersion,
VectorRef<KeyRangeRef> keyRangesFilter = {}) = 0;
// Get an IBackupContainer based on a container spec string
static Reference<IBackupContainer> openContainer(std::string url);

View File

@ -94,23 +94,15 @@ struct ClientVersionRef {
ClientVersionRef(Arena &arena, ClientVersionRef const& cv) : clientVersion(arena, cv.clientVersion), sourceVersion(arena, cv.sourceVersion), protocolVersion(arena, cv.protocolVersion) {}
ClientVersionRef(StringRef clientVersion, StringRef sourceVersion, StringRef protocolVersion) : clientVersion(clientVersion), sourceVersion(sourceVersion), protocolVersion(protocolVersion) {}
ClientVersionRef(std::string versionString) {
size_t index = versionString.find(",");
if(index == versionString.npos) {
ClientVersionRef(StringRef versionString) {
std::vector<StringRef> parts = versionString.splitAny(LiteralStringRef(","));
if (parts.size() != 3) {
initUnknown();
return;
}
clientVersion = StringRef((uint8_t*)&versionString[0], index);
size_t nextIndex = versionString.find(",", index+1);
if(index == versionString.npos) {
initUnknown();
return;
}
sourceVersion = StringRef((uint8_t*)&versionString[index+1], nextIndex-(index+1));
protocolVersion = StringRef((uint8_t*)&versionString[nextIndex+1], versionString.length()-(nextIndex+1));
clientVersion = parts[0];
sourceVersion = parts[1];
protocolVersion = parts[2];
}
void initUnknown() {

View File

@ -133,15 +133,19 @@ struct DatabaseConfiguration {
}
//Killing an entire datacenter counts as killing one zone in modes that support it
int32_t maxZoneFailuresTolerated() const {
int32_t maxZoneFailuresTolerated(int fullyReplicatedRegions, bool forAvailability) const {
int worstSatellite = regions.size() ? std::numeric_limits<int>::max() : 0;
int regionsWithNonNegativePriority = 0;
for(auto& r : regions) {
if(r.priority >= 0) {
regionsWithNonNegativePriority++;
}
worstSatellite = std::min(worstSatellite, r.satelliteTLogReplicationFactor - r.satelliteTLogWriteAntiQuorum);
if(r.satelliteTLogUsableDcsFallback > 0) {
worstSatellite = std::min(worstSatellite, r.satelliteTLogReplicationFactorFallback - r.satelliteTLogWriteAntiQuorumFallback);
}
}
if(usableRegions > 1 && worstSatellite > 0) {
if(usableRegions > 1 && fullyReplicatedRegions > 1 && worstSatellite > 0 && (!forAvailability || regionsWithNonNegativePriority > 1)) {
return 1 + std::min(std::max(tLogReplicationFactor - 1 - tLogWriteAntiQuorum, worstSatellite - 1), storageTeamSize - 1);
} else if(worstSatellite > 0) {
return std::min(tLogReplicationFactor + worstSatellite - 2 - tLogWriteAntiQuorum, storageTeamSize - 1);

View File

@ -139,8 +139,8 @@ public:
bool sampleReadTags();
Reference<ProxyInfo> getMasterProxies(bool useProvisionalProxies);
Future<Reference<ProxyInfo>> getMasterProxiesFuture(bool useProvisionalProxies);
Reference<ProxyInfo> getMasterProxies(bool useProvisionalProxies, bool useGrvProxies = false);
Future<Reference<ProxyInfo>> getMasterProxiesFuture(bool useProvisionalProxies, bool useGrvProxies = false);
Future<Void> onMasterProxiesChanged();
Future<HealthMetrics> getHealthMetrics(bool detailed);
@ -193,6 +193,7 @@ public:
AsyncTrigger masterProxiesChangeTrigger;
Future<Void> monitorMasterProxiesInfoChange;
Reference<ProxyInfo> masterProxies;
Reference<ProxyInfo> grvProxies;
bool provisional;
UID masterProxiesLastChange;
LocalityData clientLocality;

View File

@ -230,6 +230,7 @@ std::string describe( std::set<T> const& items, int max_items = -1 ) {
std::string printable( const StringRef& val );
std::string printable( const std::string& val );
std::string printable( const KeyRangeRef& range );
std::string printable(const VectorRef<KeyRangeRef>& val);
std::string printable( const VectorRef<StringRef>& val );
std::string printable( const VectorRef<KeyValueRef>& val );
std::string printable( const KeyValueRef& val );
@ -261,6 +262,14 @@ struct KeyRangeRef {
bool contains( const KeyRef& key ) const { return begin <= key && key < end; }
bool contains( const KeyRangeRef& keys ) const { return begin <= keys.begin && keys.end <= end; }
bool intersects( const KeyRangeRef& keys ) const { return begin < keys.end && keys.begin < end; }
bool intersects(const VectorRef<KeyRangeRef>& keysVec) const {
for (const auto& keys : keysVec) {
if (intersects(keys)) {
return true;
}
}
return false;
}
bool empty() const { return begin == end; }
bool singleKeyRange() const { return equalsKeyAfter(begin, end); }

View File

@ -557,7 +557,9 @@ namespace fileBackup {
if(rLen != len)
throw restore_bad_read();
Standalone<VectorRef<KeyValueRef>> results({}, buf.arena());
simulateBlobFailure();
Standalone<VectorRef<KeyValueRef>> results({}, buf.arena());
state StringRefReader reader(buf, restore_corrupted_data());
try {
@ -597,17 +599,17 @@ namespace fileBackup {
if(b != 0xFF)
throw restore_corrupted_data_padding();
return results;
return results;
} catch(Error &e) {
TraceEvent(SevWarn, "FileRestoreCorruptRangeFileBlock")
.error(e)
.detail("Filename", file->getFilename())
.detail("BlockOffset", offset)
.detail("BlockLen", len)
.detail("ErrorRelativeOffset", reader.rptr - buf.begin())
.detail("ErrorAbsoluteOffset", reader.rptr - buf.begin() + offset);
throw;
TraceEvent(SevWarn, "FileRestoreDecodeRangeFileBlockFailed")
.error(e)
.detail("Filename", file->getFilename())
.detail("BlockOffset", offset)
.detail("BlockLen", len)
.detail("ErrorRelativeOffset", reader.rptr - buf.begin())
.detail("ErrorAbsoluteOffset", reader.rptr - buf.begin() + offset);
throw;
}
}
@ -4963,3 +4965,18 @@ ACTOR Future<Void> transformRestoredDatabase(Database cx, Standalone<VectorRef<K
return Void();
}
void simulateBlobFailure() {
if (BUGGIFY && deterministicRandom()->random01() < 0.01) { // Simulate blob failures
double i = deterministicRandom()->random01();
if (i < 0.5) {
throw http_request_failed();
} else if (i < 0.7) {
throw connection_failed();
} else if (i < 0.8) {
throw timed_out();
} else if (i < 0.9) {
throw lookup_failed();
}
}
}

View File

@ -92,7 +92,7 @@ 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( 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;
@ -103,7 +103,7 @@ void ClientKnobs::initialize(bool randomize) {
init( WATCH_POLLING_TIME, 1.0 ); if( randomize && BUGGIFY ) WATCH_POLLING_TIME = 5.0;
init( NO_RECENT_UPDATES_DURATION, 20.0 ); if( randomize && BUGGIFY ) NO_RECENT_UPDATES_DURATION = 0.1;
init( FAST_WATCH_TIMEOUT, 20.0 ); if( randomize && BUGGIFY ) FAST_WATCH_TIMEOUT = 1.0;
init( WATCH_TIMEOUT, 900.0 ); if( randomize && BUGGIFY ) WATCH_TIMEOUT = 20.0;
init( WATCH_TIMEOUT, 30.0 ); if( randomize && BUGGIFY ) WATCH_TIMEOUT = 20.0;
// Core
init( CORE_VERSIONSPERSECOND, 1e6 );

View File

@ -28,6 +28,7 @@
#include "fdbclient/DatabaseContext.h"
#include "fdbrpc/simulator.h"
#include "fdbclient/StatusClient.h"
#include "flow/Trace.h"
#include "flow/UnitTest.h"
#include "fdbrpc/ReplicationPolicy.h"
#include "fdbrpc/Replication.h"

View File

@ -191,7 +191,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);
}
};

View File

@ -1378,7 +1378,7 @@ void ClientInfo::loadProtocolVersion() {
}
char *next;
std::string protocolVersionStr = ClientVersionRef(version).protocolVersion.toString();
std::string protocolVersionStr = ClientVersionRef(StringRef(version)).protocolVersion.toString();
protocolVersion = ProtocolVersion(strtoull(protocolVersionStr.c_str(), &next, 16));
ASSERT(protocolVersion.version() != 0 && protocolVersion.version() != ULLONG_MAX);

View File

@ -148,6 +148,12 @@ std::string printable( const KeyRangeRef& range ) {
return printable(range.begin) + " - " + printable(range.end);
}
std::string printable(const VectorRef<KeyRangeRef>& val) {
std::string s;
for (int i = 0; i < val.size(); i++) s = s + printable(val[i]) + " ";
return s;
}
int unhex( char c ) {
if (c >= '0' && c <= '9')
return c-'0';
@ -899,8 +905,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().proxies.size() ) {
masterProxies = Reference<ProxyInfo>( new ProxyInfo( clientInfo->get().proxies, false ) );
grvProxies = Reference<ProxyInfo>( new ProxyInfo( clientInfo->get().proxies, true ) );
}
server_interf.clear();
locationCache.insert( allKeys, Reference<LocationInfo>() );
break;
@ -909,8 +917,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().proxies.size() ) {
masterProxies = Reference<ProxyInfo>( new ProxyInfo( clientInfo->get().proxies, false ) );
grvProxies = Reference<ProxyInfo>( new ProxyInfo( clientInfo->get().proxies, true ) );
}
server_interf.clear();
locationCache.insert( allKeys, Reference<LocationInfo>() );
break;
@ -952,6 +962,7 @@ 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);
@ -1226,14 +1237,9 @@ void setNetworkOption(FDBNetworkOptions::Option option, Optional<StringRef> valu
ASSERT(value.present());
Standalone<VectorRef<ClientVersionRef>> supportedVersions;
std::string versionString = value.get().toString();
size_t index = 0;
size_t nextIndex = 0;
while(nextIndex != versionString.npos) {
nextIndex = versionString.find(';', index);
supportedVersions.push_back_deep(supportedVersions.arena(), ClientVersionRef(versionString.substr(index, nextIndex-index)));
index = nextIndex + 1;
std::vector<StringRef> supportedVersionsStrings = value.get().splitAny(LiteralStringRef(";"));
for (StringRef versionString: supportedVersionsStrings) {
supportedVersions.push_back_deep(supportedVersions.arena(), ClientVersionRef(versionString));
}
ASSERT(supportedVersions.size() > 0);
@ -1292,25 +1298,30 @@ void stopNetwork() {
closeTraceFile();
}
Reference<ProxyInfo> DatabaseContext::getMasterProxies(bool useProvisionalProxies) {
Reference<ProxyInfo> DatabaseContext::getMasterProxies(bool useProvisionalProxies, bool useGrvProxies) {
if (masterProxiesLastChange != clientInfo->get().id) {
masterProxiesLastChange = clientInfo->get().id;
masterProxies.clear();
grvProxies.clear();
if( clientInfo->get().proxies.size() ) {
masterProxies = Reference<ProxyInfo>( new ProxyInfo( clientInfo->get().proxies ));
masterProxies = Reference<ProxyInfo>( new ProxyInfo( clientInfo->get().proxies, false ) );
grvProxies = Reference<ProxyInfo>( new ProxyInfo( clientInfo->get().proxies, true ) );
provisional = clientInfo->get().proxies[0].provisional;
}
}
if(provisional && !useProvisionalProxies) {
return Reference<ProxyInfo>();
}
if(useGrvProxies) {
return grvProxies;
}
return masterProxies;
}
//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) {
ACTOR Future<Reference<ProxyInfo>> getMasterProxiesFuture(DatabaseContext *cx, bool useProvisionalProxies, bool useGrvProxies) {
loop{
Reference<ProxyInfo> proxies = cx->getMasterProxies(useProvisionalProxies);
Reference<ProxyInfo> proxies = cx->getMasterProxies(useProvisionalProxies, useGrvProxies);
if (proxies)
return proxies;
wait( cx->onMasterProxiesChanged() );
@ -1318,8 +1329,8 @@ ACTOR Future<Reference<ProxyInfo>> getMasterProxiesFuture(DatabaseContext *cx, b
}
//Returns a future which will not be set until the ProxyInfo of this DatabaseContext is not NULL
Future<Reference<ProxyInfo>> DatabaseContext::getMasterProxiesFuture(bool useProvisionalProxies) {
return ::getMasterProxiesFuture(this, useProvisionalProxies);
Future<Reference<ProxyInfo>> DatabaseContext::getMasterProxiesFuture(bool useProvisionalProxies, bool useGrvProxies) {
return ::getMasterProxiesFuture(this, useProvisionalProxies, useGrvProxies);
}
void GetRangeLimits::decrement( VectorRef<KeyValueRef> const& data ) {
@ -3351,7 +3362,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;
@ -3387,7 +3398,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;
@ -3453,15 +3464,15 @@ void Transaction::setOption( FDBTransactionOptions::Option option, Optional<Stri
}
ACTOR Future<GetReadVersionReply> getConsistentReadVersion( DatabaseContext *cx, uint32_t transactionCount, TransactionPriority priority, uint32_t flags, TransactionTagMap<uint32_t> tags, Optional<UID> debugID ) {
try {
++cx->transactionReadVersionBatches;
if( debugID.present() )
g_traceBatch.addEvent("TransactionDebug", debugID.get().first(), "NativeAPI.getConsistentReadVersion.Before");
loop {
++cx->transactionReadVersionBatches;
if( debugID.present() )
g_traceBatch.addEvent("TransactionDebug", debugID.get().first(), "NativeAPI.getConsistentReadVersion.Before");
loop {
try {
state GetReadVersionRequest req( 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 ( GetReadVersionReply v = wait( basicLoadBalance( cx->getMasterProxies(flags & GetReadVersionRequest::FLAG_USE_PROVISIONAL_PROXIES, true), &MasterProxyInterface::getConsistentReadVersion, req, cx->taskID ) ) ) {
if(tags.size() != 0) {
auto &priorityThrottledTags = cx->throttledTags[priority];
for(auto& tag : tags) {
@ -3487,11 +3498,15 @@ ACTOR Future<GetReadVersionReply> getConsistentReadVersion( DatabaseContext *cx,
return v;
}
}
} catch (Error& e) {
if (e.code() != error_code_broken_promise && e.code() != error_code_batch_transaction_throttled)
TraceEvent(SevError, "GetConsistentReadVersionError").error(e);
if(e.code() == error_code_batch_transaction_throttled && !cx->apiVersionAtLeast(630)) {
wait(delayJittered(5.0));
} else {
throw;
}
}
} catch (Error& e) {
if (e.code() != error_code_broken_promise && e.code() != error_code_batch_transaction_throttled)
TraceEvent(SevError, "GetConsistentReadVersionError").error(e);
throw;
}
}
@ -3755,9 +3770,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());
@ -3779,7 +3794,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) {
vector<pair<KeyRange, Reference<LocationInfo>>> locations = wait(getKeyRangeLocations(
cx, keys, std::numeric_limits<int>::max(), false, &StorageServerInterface::waitMetrics, TransactionInfo(TaskPriority::DataDistribution)));
@ -3862,7 +3877,7 @@ ACTOR Future< StorageMetrics > extractMetrics( Future<std::pair<Optional<Storage
return x.first.get();
}
ACTOR Future<Standalone<VectorRef<KeyRangeRef>>> getReadHotRanges(Database cx, KeyRange keys) {
ACTOR Future<Standalone<VectorRef<ReadHotRangeWithMetrics>>> getReadHotRanges(Database cx, KeyRange keys) {
loop {
int64_t shardLimit = 100; // Shard limit here does not really matter since this function is currently only used
// to find the read-hot sub ranges within a read-hot shard.
@ -3889,13 +3904,22 @@ ACTOR Future<Standalone<VectorRef<KeyRangeRef>>> getReadHotRanges(Database cx, K
}
wait(waitForAll(fReplies));
Standalone<VectorRef<KeyRangeRef>> 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);
@ -3994,7 +4018,7 @@ ACTOR Future<Standalone<VectorRef<DDMetricsRef>>> waitDataDistributionMetricsLis
}
}
Future<Standalone<VectorRef<KeyRangeRef>>> Transaction::getReadHotRanges(KeyRange const& keys) {
Future<Standalone<VectorRef<ReadHotRangeWithMetrics>>> Transaction::getReadHotRanges(KeyRange const& keys) {
return ::getReadHotRanges(cx, keys);
}

View File

@ -261,7 +261,7 @@ public:
// Pass a negative value for `shardLimit` to indicate no limit on the shard number.
Future< StorageMetrics > getStorageMetrics( KeyRange const& keys, int shardLimit );
Future< Standalone<VectorRef<KeyRef>> > splitStorageMetrics( KeyRange const& keys, StorageMetrics const& limit, StorageMetrics const& estimated );
Future<Standalone<VectorRef<KeyRangeRef>>> getReadHotRanges(KeyRange const& keys);
Future<Standalone<VectorRef<ReadHotRangeWithMetrics>>> getReadHotRanges(KeyRange const& keys);
// If checkWriteConflictRanges is true, existing write conflict ranges will be searched for this key
void set( const KeyRef& key, const ValueRef& value, bool addConflictRange = true );

View File

@ -54,6 +54,7 @@ struct RestoreSysInfo;
struct RestoreApplierInterface;
struct RestoreFinishRequest;
struct RestoreSamplesRequest;
struct RestoreUpdateRateRequest;
// RestoreSysInfo includes information each (type of) restore roles should know.
// At this moment, it only include appliers. We keep the name for future extension.
@ -112,7 +113,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();
@ -174,6 +175,7 @@ struct RestoreApplierInterface : RestoreRoleInterface {
RequestStream<RestoreVersionBatchRequest> initVersionBatch;
RequestStream<RestoreSimpleRequest> collectRestoreRoleInterfaces;
RequestStream<RestoreFinishRequest> finishRestore;
RequestStream<RestoreUpdateRateRequest> updateRate;
bool operator==(RestoreWorkerInterface const& r) const { return id() == r.id(); }
bool operator!=(RestoreWorkerInterface const& r) const { return id() != r.id(); }
@ -193,15 +195,16 @@ struct RestoreApplierInterface : RestoreRoleInterface {
initVersionBatch.getEndpoint(TaskPriority::LoadBalancedEndpoint);
collectRestoreRoleInterfaces.getEndpoint(TaskPriority::LoadBalancedEndpoint);
finishRestore.getEndpoint(TaskPriority::LoadBalancedEndpoint);
updateRate.getEndpoint(TaskPriority::LoadBalancedEndpoint);
}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, *(RestoreRoleInterface*)this, heartbeat, sendMutationVector, applyToDB, initVersionBatch,
collectRestoreRoleInterfaces, finishRestore);
collectRestoreRoleInterfaces, finishRestore, updateRate);
}
std::string toString() { return nodeID.toString(); }
std::string toString() const { return nodeID.toString(); }
};
struct RestoreControllerInterface : RestoreRoleInterface {
@ -226,7 +229,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 +252,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 +284,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 +348,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 +376,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 +408,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 +437,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 +462,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 +483,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 +502,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 +529,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 +562,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 +590,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,13 +612,57 @@ struct RestoreFinishRequest : TimedRequest {
serializer(ar, terminate, reply);
}
std::string toString() {
std::string toString() const {
std::stringstream ss;
ss << "RestoreFinishRequest terminate:" << terminate;
return ss.str();
}
};
struct RestoreUpdateRateReply : TimedRequest {
constexpr static FileIdentifier file_identifier = 13018414;
UID id;
double remainMB; // remaining data in MB to write to DB;
RestoreUpdateRateReply() = default;
explicit RestoreUpdateRateReply(UID id, double remainMB) : id(id), remainMB(remainMB) {}
std::string toString() const {
std::stringstream ss;
ss << "RestoreUpdateRateReply NodeID:" << id.toString() << " remainMB:" << remainMB;
return ss.str();
}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, id, remainMB);
}
};
struct RestoreUpdateRateRequest : TimedRequest {
constexpr static FileIdentifier file_identifier = 13018415;
int batchIndex;
double writeMB;
ReplyPromise<RestoreUpdateRateReply> reply;
RestoreUpdateRateRequest() = default;
explicit RestoreUpdateRateRequest(int batchIndex, double writeMB) : batchIndex(batchIndex), writeMB(writeMB) {}
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, batchIndex, writeMB, reply);
}
std::string toString() const {
std::stringstream ss;
ss << "RestoreUpdateRateRequest batchIndex:" << batchIndex << " writeMB:" << writeMB;
return ss.str();
}
};
struct RestoreRequest {
constexpr static FileIdentifier file_identifier = 49589770;

View File

@ -269,15 +269,20 @@ const KeyRef JSONSchemas::statusSchema = LiteralStringRef(R"statusSchema(
"run_loop_busy":0.2
}
},
"old_logs":[
"logs":[
{
"logs":[
"log_interfaces":[
{
"id":"7f8d623d0cb9966e",
"healthy":true,
"address":"1.2.3.4:1234"
}
],
"epoch":1,
"current":false,
"begin_version":23,
"end_version":112315141,
"possibly_losing_data":true,
"log_replication_factor":3,
"log_write_anti_quorum":0,
"log_fault_tolerance":2,

View File

@ -423,9 +423,30 @@ struct SplitMetricsRequest {
}
};
// Should always be used inside a `Standalone`.
struct ReadHotRangeWithMetrics {
KeyRangeRef keys;
double density;
double readBandwidth;
ReadHotRangeWithMetrics() = default;
ReadHotRangeWithMetrics(KeyRangeRef const& keys, double density, double readBandwidth)
: keys(keys), density(density), readBandwidth(readBandwidth) {}
ReadHotRangeWithMetrics(Arena& arena, const ReadHotRangeWithMetrics& rhs)
: keys(arena, rhs.keys), density(rhs.density), readBandwidth(rhs.readBandwidth) {}
int expectedSize() { return keys.expectedSize() + sizeof(density) + sizeof(readBandwidth); }
template <class Ar>
void serialize(Ar& ar) {
serializer(ar, keys, density, readBandwidth);
}
};
struct ReadHotSubRangeReply {
constexpr static FileIdentifier file_identifier = 10424537;
Standalone<VectorRef<KeyRangeRef>> readHotRanges;
Standalone<VectorRef<ReadHotRangeWithMetrics>> readHotRanges;
template <class Ar>
void serialize(Ar& ar) {

View File

@ -880,6 +880,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");

View File

@ -345,6 +345,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;

View File

@ -195,16 +195,7 @@ struct PingReceiver : NetworkMessageReceiver {
class TransportData {
public:
TransportData(uint64_t transportId)
: endpointNotFoundReceiver(endpoints),
pingReceiver(endpoints),
warnAlwaysForLargePacket(true),
lastIncompatibleMessage(0),
transportId(transportId),
numIncompatibleConnections(0)
{
degraded = Reference<AsyncVar<bool>>( new AsyncVar<bool>(false) );
}
TransportData(uint64_t transportId);
~TransportData();
@ -228,6 +219,7 @@ public:
std::unordered_map<NetworkAddress, Reference<struct Peer>> peers;
std::unordered_map<NetworkAddress, std::pair<double, double>> closedPeers;
HealthMonitor healthMonitor;
std::set<NetworkAddress> orderedAddresses;
Reference<AsyncVar<bool>> degraded;
bool warnAlwaysForLargePacket;
@ -252,8 +244,56 @@ public:
uint64_t transportId;
Future<Void> multiVersionCleanup;
Future<Void> pingLogger;
};
ACTOR Future<Void> pingLatencyLogger(TransportData* self) {
state NetworkAddress lastAddress = NetworkAddress();
loop {
if(self->orderedAddresses.size()) {
auto it = self->orderedAddresses.upper_bound(lastAddress);
if(it == self->orderedAddresses.end()) {
it = self->orderedAddresses.begin();
}
lastAddress = *it;
auto peer = self->getPeer(lastAddress);
if(!peer) {
TraceEvent(SevWarnAlways, "MissingNetworkAddress").suppressFor(10.0).detail("PeerAddr", lastAddress);
}
if(peer && peer->pingLatencies.getPopulationSize() >= 10) {
TraceEvent("PingLatency")
.detail("PeerAddr", lastAddress)
.detail("MinLatency", peer->pingLatencies.min())
.detail("MaxLatency", peer->pingLatencies.max())
.detail("MeanLatency", peer->pingLatencies.mean())
.detail("MedianLatency", peer->pingLatencies.median())
.detail("P90Latency", peer->pingLatencies.percentile(0.90))
.detail("Count", peer->pingLatencies.getPopulationSize())
.detail("BytesReceived", peer->bytesReceived - peer->lastLoggedBytesReceived);
peer->pingLatencies.clear();
peer->lastLoggedBytesReceived = peer->bytesReceived;
wait(delay(FLOW_KNOBS->PING_LOGGING_INTERVAL));
} else if(it == self->orderedAddresses.begin()) {
wait(delay(FLOW_KNOBS->PING_LOGGING_INTERVAL));
}
} else {
wait(delay(FLOW_KNOBS->PING_LOGGING_INTERVAL));
}
}
}
TransportData::TransportData(uint64_t transportId)
: endpointNotFoundReceiver(endpoints),
pingReceiver(endpoints),
warnAlwaysForLargePacket(true),
lastIncompatibleMessage(0),
transportId(transportId),
numIncompatibleConnections(0)
{
degraded = Reference<AsyncVar<bool>>( new AsyncVar<bool>(false) );
pingLogger = pingLatencyLogger(this);
}
#define CONNECT_PACKET_V0 0x0FDB00A444020001LL
#define CONNECT_PACKET_V0_SIZE 14
@ -379,10 +419,14 @@ ACTOR Future<Void> connectionMonitor( Reference<Peer> peer ) {
FlowTransport::transport().sendUnreliable( SerializeSource<ReplyPromise<Void>>(reply), remotePingEndpoint, true );
state int64_t startingBytes = peer->bytesReceived;
state int timeouts = 0;
state double startTime = now();
loop {
choose {
when (wait( delay( FLOW_KNOBS->CONNECTION_MONITOR_TIMEOUT ) )) {
if(startingBytes == peer->bytesReceived) {
if(peer->destination.isPublic()) {
peer->pingLatencies.addSample(now() - startTime);
}
TraceEvent("ConnectionTimeout").suppressFor(1.0).detail("WithAddr", peer->destination);
throw connection_failed();
}
@ -396,6 +440,9 @@ ACTOR Future<Void> connectionMonitor( Reference<Peer> peer ) {
timeouts++;
}
when (wait( reply.getFuture() )) {
if(peer->destination.isPublic()) {
peer->pingLatencies.addSample(now() - startTime);
}
break;
}
when (wait( peer->resetPing.onTrigger())) {
@ -560,6 +607,7 @@ ACTOR Future<Void> connectionKeeper( Reference<Peer> self,
}
} else {
self->outgoingConnectionIdle = false;
self->lastConnectTime = now();
}
firstConnFailedTime.reset();
@ -666,6 +714,7 @@ ACTOR Future<Void> connectionKeeper( Reference<Peer> self,
TraceEvent("PeerDestroy").error(e).suppressFor(1.0).detail("PeerAddr", self->destination);
self->connect.cancel();
self->transport->peers.erase(self->destination);
self->transport->orderedAddresses.erase(self->destination);
return Void();
}
}
@ -675,7 +724,8 @@ ACTOR Future<Void> connectionKeeper( Reference<Peer> self,
Peer::Peer(TransportData* transport, NetworkAddress const& destination)
: transport(transport), destination(destination), outgoingConnectionIdle(true), lastConnectTime(0.0),
reconnectionDelay(FLOW_KNOBS->INITIAL_RECONNECTION_TIME), compatible(true), outstandingReplies(0),
incompatibleProtocolVersionNewer(false), peerReferences(-1), bytesReceived(0), lastDataPacketSentTime(now()) {
incompatibleProtocolVersionNewer(false), peerReferences(-1), bytesReceived(0), lastDataPacketSentTime(now()),
pingLatencies(destination.isPublic() ? FLOW_KNOBS->PING_SAMPLE_AMOUNT : 1), lastLoggedBytesReceived(0) {
IFailureMonitor::failureMonitor().setStatus(destination, FailureStatus(false));
}
@ -733,7 +783,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)
@ -1154,6 +1204,9 @@ Reference<Peer> TransportData::getOrOpenPeer( NetworkAddress const& address, boo
peer->connect = connectionKeeper(peer);
}
peers[address] = peer;
if(address.isPublic()) {
orderedAddresses.insert(address);
}
}
return peer;

View File

@ -28,6 +28,7 @@
#include "flow/network.h"
#include "flow/FileIdentifier.h"
#include "flow/Net2Packet.h"
#include "fdbrpc/ContinuousSample.h"
#pragma pack(push, 4)
class Endpoint {
@ -142,6 +143,8 @@ struct Peer : public ReferenceCounted<Peer> {
int64_t bytesReceived;
double lastDataPacketSentTime;
int outstandingReplies;
ContinuousSample<double> pingLatencies;
int64_t lastLoggedBytesReceived;
explicit Peer(TransportData* transport, NetworkAddress const& destination);

View File

@ -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();

View File

@ -62,10 +62,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;
@ -81,7 +81,10 @@ struct AlternativeInfo {
template <class T>
class ModelInterface : public ReferenceCounted<ModelInterface<T>> {
public:
ModelInterface( const vector<T>& v ) {
//If balanceOnRequests is true, the client will load balance based on the number of GRVs released by each proxy
//If balanceOnRequests is false, the client will load balance based on the CPU usage of each proxy
//Only requests which take from the GRV budget on the proxy should set balanceOnRequests to true
ModelInterface( const vector<T>& v, bool balanceOnRequests ) : balanceOnRequests(balanceOnRequests) {
for(int i = 0; i < v.size(); i++) {
alternatives.push_back(AlternativeInfo(v[i], 1.0/v.size(), (i+1.0)/v.size()));
}
@ -100,26 +103,32 @@ 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 totalBusy = 0;
for(auto& it : alternatives) {
totalRequests += it.recentRequests;
int busyMetric = balanceOnRequests ? it.processBusyTime/FLOW_KNOBS->BASIC_LOAD_BALANCE_COMPUTE_PRECISION :
it.processBusyTime%FLOW_KNOBS->BASIC_LOAD_BALANCE_COMPUTE_PRECISION;
totalBusy += busyMetric;
if(now() - it.lastUpdate > FLOW_KNOBS->BASIC_LOAD_BALANCE_UPDATE_RATE/2.0) {
return;
}
}
if(totalRequests < 1000) {
if((balanceOnRequests && totalBusy < FLOW_KNOBS->BASIC_LOAD_BALANCE_MIN_REQUESTS*alternatives.size()) ||
(!balanceOnRequests && totalBusy < FLOW_KNOBS->BASIC_LOAD_BALANCE_COMPUTE_PRECISION*FLOW_KNOBS->BASIC_LOAD_BALANCE_MIN_CPU*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;
int busyMetric = balanceOnRequests ? it.processBusyTime/FLOW_KNOBS->BASIC_LOAD_BALANCE_COMPUTE_PRECISION :
it.processBusyTime%FLOW_KNOBS->BASIC_LOAD_BALANCE_COMPUTE_PRECISION;
it.probability += (1.0/alternatives.size()-(busyMetric/totalBusy))*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;
@ -153,6 +162,7 @@ public:
private:
vector<AlternativeInfo<T>> alternatives;
Future<Void> updater;
bool balanceOnRequests;
};
template <class T>

View File

@ -67,13 +67,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), cpuTicks(0),
rebooting(false), fault_injection_p1(0), fault_injection_p2(0),
fault_injection_r(0), machine(0), cleared(false) {}
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), cpuTicks(0), rebooting(false), fault_injection_p1(0), fault_injection_p2(0),
fault_injection_r(0), machine(0), cleared(false) {
uid = deterministicRandom()->randomUniqueID();
}
Future<KillType> onShutdown() { return shutdownSignal.getFuture(); }

View File

@ -425,7 +425,7 @@ struct BackupData {
ACTOR static Future<Version> _getMinKnownCommittedVersion(BackupData* self) {
loop {
GetReadVersionRequest request(1, TransactionPriority::DEFAULT,
GetReadVersionRequest request(0, TransactionPriority::DEFAULT,
GetReadVersionRequest::FLAG_USE_MIN_KNOWN_COMMITTED_VERSION);
choose {
when(wait(self->cx->onMasterProxiesChanged())) {}

View File

@ -152,6 +152,7 @@ set(FDBSERVER_SRCS
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

View File

@ -2919,10 +2919,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();

View File

@ -877,8 +877,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()) {
@ -2068,7 +2068,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())
@ -2772,7 +2772,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);
@ -4476,7 +4476,7 @@ 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<ProxyInfo> proxies(new ProxyInfo(db->get().client.proxies, false));
choose {
when (wait(db->onChange())) {}
@ -4817,6 +4817,21 @@ ACTOR Future<Void> ddSnapCreateCore(DistributorSnapRequest snapReq, Reference<As
return Void();
}
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;
rep.storageMetricsList = result.get();
req.reply.send(rep);
}
return Void();
}
ACTOR Future<Void> ddSnapCreate(DistributorSnapRequest snapReq, Reference<AsyncVar<struct ServerDBInfo>> db ) {
state Future<Void> dbInfoChange = db->onChange();
if (!setDDEnabled(false, snapReq.snapUID)) {
@ -4940,16 +4955,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));

View File

@ -299,11 +299,14 @@ ACTOR Future<Void> readHotDetector(DataDistributionTracker* self) {
state Transaction tr(self->cx);
loop {
try {
Standalone<VectorRef<KeyRangeRef>> readHotRanges = wait(tr.getReadHotRanges(keys));
Standalone<VectorRef<ReadHotRangeWithMetrics>> readHotRanges = wait(tr.getReadHotRanges(keys));
for (auto& keyRange : readHotRanges) {
TraceEvent("ReadHotRangeLog")
.detail("KeyRangeBegin", keyRange.begin)
.detail("KeyRangeEnd", keyRange.end);
.detail("ReadDensity", keyRange.density)
.detail("ReadBandwidth", keyRange.readBandwidth)
.detail("ReadDensityThreshold", SERVER_KNOBS->SHARD_MAX_READ_DENSITY_RATIO)
.detail("KeyRangeBegin", keyRange.keys.begin)
.detail("KeyRangeEnd", keyRange.keys.end);
}
break;
} catch (Error& e) {

View File

@ -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() = 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.

View File

@ -239,7 +239,6 @@ struct RocksDBKeyValueStore : IKeyValueStore {
if (cursor->Valid() && toStringRef(cursor->key()) == a.keys.end) {
cursor->Prev();
}
while (cursor->Valid() && toStringRef(cursor->key()) >= a.keys.begin) {
KeyValueRef kv(toStringRef(cursor->key()), toStringRef(cursor->value()));
accumulatedBytes += sizeof(KeyValueRef) + kv.expectedSize();
@ -252,6 +251,7 @@ struct RocksDBKeyValueStore : IKeyValueStore {
}
s = cursor->status();
}
if (!s.ok()) {
TraceEvent(SevError, "RocksDBError").detail("Error", s.ToString()).detail("Method", "ReadRange");
}

View File

@ -94,8 +94,13 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
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_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 );
@ -144,17 +149,17 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
init( SHARD_BYTES_PER_SQRT_BYTES, 45 ); if( buggifySmallShards ) SHARD_BYTES_PER_SQRT_BYTES = 0;//Approximately 10000 bytes per shard
init( MAX_SHARD_BYTES, 500000000 );
init( KEY_SERVER_SHARD_BYTES, 500000000 );
init( SHARD_MAX_READ_DENSITY_RATIO, 2.0);
init( SHARD_MAX_READ_DENSITY_RATIO, 8.0); if (randomize && BUGGIFY) SHARD_MAX_READ_DENSITY_RATIO = 2.0;
/*
The bytesRead/byteSize radio. Will be declared as read hot when larger than this. 2.0 was chosen to avoid reporting table scan as read hot.
The bytesRead/byteSize radio. Will be declared as read hot when larger than this. 8.0 was chosen to avoid reporting table scan as read hot.
*/
init ( SHARD_READ_HOT_BANDWITH_MIN_PER_KSECONDS, 166667 * 1000);
init ( SHARD_READ_HOT_BANDWITH_MIN_PER_KSECONDS, 1666667 * 1000);
/*
The read bandwidth of a given shard needs to be larger than this value in order to be evaluated if it's read hot. The roughly 167KB per second is calculated as following:
- Heuristic data suggests that each storage process can do max 50K read operations per second
The read bandwidth of a given shard needs to be larger than this value in order to be evaluated if it's read hot. The roughly 1.67MB per second is calculated as following:
- Heuristic data suggests that each storage process can do max 500K read operations per second
- Each read has a minimum cost of EMPTY_READ_PENALTY, which is 20 bytes
- Thus that gives a minimum 1MB per second
- But to be conservative, set that number to be 1/6 of 1MB, which is roughly 166,667 bytes per second
- Thus that gives a minimum 10MB per second
- But to be conservative, set that number to be 1/6 of 10MB, which is roughly 1,666,667 bytes per second
Shard with a read bandwidth smaller than this value will never be too busy to handle the reads.
*/
init( SHARD_MAX_BYTES_READ_PER_KSEC_JITTER, 0.1 );
@ -346,7 +351,7 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
init( COMMIT_TRANSACTION_BATCH_COUNT_MAX, 32768 ); if( randomize && BUGGIFY ) COMMIT_TRANSACTION_BATCH_COUNT_MAX = 1000; // Do NOT increase this number beyond 32768, as CommitIds only budget 2 bytes for storing transaction id within each batch
init( COMMIT_BATCHES_MEM_BYTES_HARD_LIMIT, 8LL << 30 ); if (randomize && BUGGIFY) COMMIT_BATCHES_MEM_BYTES_HARD_LIMIT = deterministicRandom()->randomInt64(100LL << 20, 8LL << 30);
init( COMMIT_BATCHES_MEM_FRACTION_OF_TOTAL, 0.5 );
init( COMMIT_BATCHES_MEM_TO_TOTAL_MEM_SCALE_FACTOR, 10.0 );
init( COMMIT_BATCHES_MEM_TO_TOTAL_MEM_SCALE_FACTOR, 5.0 );
// these settings disable batch bytes scaling. Try COMMIT_TRANSACTION_BATCH_BYTES_MAX=1e6, COMMIT_TRANSACTION_BATCH_BYTES_SCALE_BASE=50000, COMMIT_TRANSACTION_BATCH_BYTES_SCALE_POWER=0.5?
init( COMMIT_TRANSACTION_BATCH_BYTES_MIN, 100000 );
@ -366,8 +371,8 @@ void ServerKnobs::initialize(bool randomize, ClientKnobs* clientKnobs, bool isSi
init( REQUIRED_MIN_RECOVERY_DURATION, 0.080 ); if( shortRecoveryDuration ) REQUIRED_MIN_RECOVERY_DURATION = 0.01;
init( ALWAYS_CAUSAL_READ_RISKY, false );
init( MAX_COMMIT_UPDATES, 2000 ); if( randomize && BUGGIFY ) MAX_COMMIT_UPDATES = 1;
init( MIN_PROXY_COMPUTE, 0.001 );
init( MAX_PROXY_COMPUTE, 2.0 );
init( MAX_COMPUTE_PER_OPERATION, 0.1 );
init( PROXY_COMPUTE_BUCKETS, 20000 );
init( PROXY_COMPUTE_GROWTH_RATE, 0.01 );
init( TXN_STATE_SEND_AMOUNT, 4 );
@ -531,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 );
@ -614,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 ? 10 * 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; }
@ -634,17 +639,29 @@ 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; }
init( FASTRESTORE_TXN_CLEAR_MAX, 100 ); if( randomize && BUGGIFY ) { FASTRESTORE_TXN_CLEAR_MAX = deterministicRandom()->random01() * 100 + 1; }
init( FASTRESTORE_TXN_RETRY_MAX, 10 ); if( randomize && BUGGIFY ) { FASTRESTORE_TXN_RETRY_MAX = deterministicRandom()->random01() * 100 + 1; }
init( FASTRESTORE_TXN_EXTRA_DELAY, 0.1 ); if( randomize && BUGGIFY ) { FASTRESTORE_TXN_EXTRA_DELAY = deterministicRandom()->random01() * 1 + 0.001;}
init( FASTRESTORE_TXN_EXTRA_DELAY, 0.0 ); if( randomize && BUGGIFY ) { FASTRESTORE_TXN_EXTRA_DELAY = deterministicRandom()->random01() * 1 + 0.001;}
init( FASTRESTORE_NOT_WRITE_DB, false ); // Perf test only: set it to true will cause simulation failure
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( FASTRESTORE_WRITE_BW_MB, 70 ); if( randomize && BUGGIFY ) { FASTRESTORE_WRITE_BW_MB = deterministicRandom()->random01() < 0.5 ? 2 : 100;}
init( FASTRESTORE_RATE_UPDATE_SECONDS, 1.0 ); if( randomize && BUGGIFY ) { FASTRESTORE_RATE_UPDATE_SECONDS = deterministicRandom()->random01() < 0.5 ? 0.1 : 2;}
init( REDWOOD_DEFAULT_PAGE_SIZE, 4096 );
init( REDWOOD_KVSTORE_CONCURRENT_READS, 64 );

View File

@ -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;
@ -296,8 +301,8 @@ public:
double REQUIRED_MIN_RECOVERY_DURATION;
bool ALWAYS_CAUSAL_READ_RISKY;
int MAX_COMMIT_UPDATES;
double MIN_PROXY_COMPUTE;
double MAX_PROXY_COMPUTE;
double MAX_COMPUTE_PER_OPERATION;
int PROXY_COMPUTE_BUCKETS;
double PROXY_COMPUTE_GROWTH_RATE;
int TXN_STATE_SEND_AMOUNT;
@ -540,6 +545,7 @@ public:
int64_t TIME_KEEPER_MAX_ENTRIES;
// Fast Restore
// TODO: After 6.3, review FR knobs, remove unneeded ones and change default value
int64_t FASTRESTORE_FAILURE_TIMEOUT;
int64_t FASTRESTORE_HEARTBEAT_INTERVAL;
double FASTRESTORE_SAMPLING_PERCENT;
@ -552,7 +558,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
@ -577,6 +583,18 @@ 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
double FASTRESTORE_WRITE_BW_MB; // target aggregated write bandwidth from all appliers
double FASTRESTORE_RATE_UPDATE_SECONDS; // how long to update appliers target write rate
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.

View File

@ -36,12 +36,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

View File

@ -139,7 +139,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();

View File

@ -100,8 +100,11 @@ struct ProxyStats {
Deque<int> requestBuckets;
double lastBucketBegin;
double bucketInterval;
int64_t maxComputeNS;
int64_t minComputeNS;
void updateRequestBuckets() {
void updateRequestBuckets() {
while(now() - lastBucketBegin > bucketInterval) {
lastBucketBegin += bucketInterval;
recentRequests -= requestBuckets.front();
@ -110,19 +113,33 @@ struct ProxyStats {
}
}
void addRequest() {
void addRequest(int transactionCount) {
updateRequestBuckets();
++recentRequests;
++requestBuckets.back();
recentRequests += transactionCount;
requestBuckets.back() += transactionCount;
}
int getRecentRequests() {
int getRecentRequests() {
updateRequestBuckets();
return recentRequests*FLOW_KNOBS->BASIC_LOAD_BALANCE_UPDATE_RATE/(FLOW_KNOBS->BASIC_LOAD_BALANCE_UPDATE_RATE-(lastBucketBegin+bucketInterval-now()));
return recentRequests/(FLOW_KNOBS->BASIC_LOAD_BALANCE_UPDATE_RATE-(lastBucketBegin+bucketInterval-now()));
}
int64_t getAndResetMaxCompute() {
int64_t r = maxComputeNS;
maxComputeNS = 0;
return r;
}
int64_t getAndResetMinCompute() {
int64_t r = minComputeNS;
minComputeNS = 1e12;
return r;
}
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),
: cc("ProxyStats", id.toString()), recentRequests(0), lastBucketBegin(now()),
maxComputeNS(0), minComputeNS(1e12),
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),
@ -141,12 +158,14 @@ struct ProxyStats {
lastCommitVersionAssigned(0),
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("CommitLatencyBands", id, SERVER_KNOBS->STORAGE_LOGGING_DELAY),
grvLatencyBands("GRVLatencyBands", 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; });
specialCounter(cc, "MaxCompute", [this](){ return this->getAndResetMaxCompute(); });
specialCounter(cc, "MinCompute", [this](){ return this->getAndResetMinCompute(); });
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);
@ -310,7 +329,6 @@ ACTOR Future<Void> queueTransactionStartRequests(
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());
@ -320,6 +338,7 @@ ACTOR Future<Void> queueTransactionStartRequests(
req.reply.send(rep);
TraceEvent(SevWarnAlways, "ProxyGRVThresholdExceeded").suppressFor(60);
} else {
stats->addRequest(req.transactionCount);
// TODO: check whether this is reasonable to do in the fast path
for(auto tag : req.tags) {
(*transactionTagCounter)[tag.first] += tag.second;
@ -535,7 +554,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 );
@ -553,6 +573,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
@ -621,7 +648,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
@ -868,7 +894,7 @@ ACTOR Future<Void> commitBatch(
int conflictRangeCount = 0;
state int64_t 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);
//for(auto& m : trs[t].transaction.mutations)
@ -1238,13 +1264,15 @@ ACTOR Future<Void> commitBatch(
}
computeDuration += g_network->timer() - computeStart;
if(computeDuration > SERVER_KNOBS->MIN_PROXY_COMPUTE && batchOperations > 0) {
double computePerOperation = computeDuration/batchOperations;
if(batchOperations > 0) {
double computePerOperation = std::min( SERVER_KNOBS->MAX_COMPUTE_PER_OPERATION, computeDuration/batchOperations );
if(computePerOperation <= self->commitComputePerOperation[latencyBucket]) {
self->commitComputePerOperation[latencyBucket] = computePerOperation;
} else {
self->commitComputePerOperation[latencyBucket] = SERVER_KNOBS->PROXY_COMPUTE_GROWTH_RATE*computePerOperation + ((1.0-SERVER_KNOBS->PROXY_COMPUTE_GROWTH_RATE)*self->commitComputePerOperation[latencyBucket]);
}
self->stats.maxComputeNS = std::max<int64_t>(self->stats.maxComputeNS, 1e9*self->commitComputePerOperation[latencyBucket]);
self->stats.minComputeNS = std::min<int64_t>(self->stats.minComputeNS, 1e9*self->commitComputePerOperation[latencyBucket]);
}
/////// Phase 4: Logging (network bound; pipelined up to MAX_READ_TRANSACTION_LIFE_VERSIONS (limited by loop above))
@ -1431,7 +1459,8 @@ ACTOR Future<GetReadVersionReply> getLiveCommittedVersion(ProxyCommitData* commi
rep = v;
}
}
rep.recentRequests = commitData->stats.getRecentRequests();
rep.processBusyTime = FLOW_KNOBS->BASIC_LOAD_BALANCE_COMPUTE_PRECISION*std::min((std::numeric_limits<int>::max()/FLOW_KNOBS->BASIC_LOAD_BALANCE_COMPUTE_PRECISION)-1,commitData->stats.getRecentRequests());
rep.processBusyTime += FLOW_KNOBS->BASIC_LOAD_BALANCE_COMPUTE_PRECISION*(g_network->isSimulated() ? deterministicRandom()->random01() : g_network->networkInfo.metrics.lastRunLoopBusyness);
if (debugID.present()) {
g_traceBatch.addEvent("TransactionDebug", debugID.get().first(), "MasterProxyServer.getLiveCommittedVersion.After");
@ -1702,7 +1731,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;

View File

@ -1298,6 +1298,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;

View File

@ -2134,6 +2134,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") );
@ -2301,6 +2306,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 ) {

View File

@ -40,6 +40,7 @@ ACTOR static Future<Void> handleSendMutationVectorRequest(RestoreSendVersionedMu
Reference<RestoreApplierData> self);
ACTOR static Future<Void> handleApplyToDBRequest(RestoreVersionBatchRequest req, Reference<RestoreApplierData> self,
Database cx);
void handleUpdateRateRequest(RestoreUpdateRateRequest req, Reference<RestoreApplierData> self);
ACTOR Future<Void> restoreApplierCore(RestoreApplierInterface applierInterf, int nodeIndex, Database cx) {
state Reference<RestoreApplierData> self =
@ -71,12 +72,17 @@ ACTOR Future<Void> restoreApplierCore(RestoreApplierInterface applierInterf, int
req, self, cx)); // TODO: Check how FDB uses TaskPriority for ACTORS. We may need to add
// priority here to avoid requests at later VB block requests at earlier VBs
}
when(RestoreUpdateRateRequest req = waitNext(applierInterf.updateRate.getFuture())) {
requestTypeStr = "updateRate";
handleUpdateRateRequest(req, self);
}
when(RestoreVersionBatchRequest req = waitNext(applierInterf.initVersionBatch.getFuture())) {
requestTypeStr = "initVersionBatch";
actors.add(handleInitVersionBatchRequest(req, self));
}
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,8 +94,10 @@ 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())
bool isError = e.code() != error_code_operation_cancelled;
TraceEvent(isError ? SevError : SevWarnAlways, "FastRestoreApplierError", self->id())
.detail("RequestType", requestTypeStr)
.error(e, true);
actors.clear(false);
@ -108,12 +116,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 +140,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 +152,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 +189,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 +209,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);
@ -199,6 +223,7 @@ ACTOR static Future<Void> applyClearRangeMutations(Standalone<VectorRef<KeyRange
loop {
try {
// TODO: Consider clearrange traffic in write traffic control
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
for (auto& range : ranges) {
@ -296,7 +321,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 +329,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 +423,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 +433,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 +448,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++) {
@ -440,31 +469,55 @@ ACTOR static Future<Void> precomputeMutationsResult(Reference<ApplierBatchData>
return Void();
}
bool okToReleaseTxns(double targetMB, double applyingDataBytes) {
return applyingDataBytes < targetMB * 1024 * 1024;
}
ACTOR static Future<Void> shouldReleaseTransaction(double* targetMB, double* applyingDataBytes,
AsyncTrigger* releaseTxns) {
loop {
if (okToReleaseTxns(*targetMB, *applyingDataBytes)) {
break;
} else {
wait(releaseTxns->onTrigger());
wait(delay(0.0)); // Avoid all waiting txns are triggered at the same time and all decide to proceed before
// applyingDataBytes has a chance to update
}
}
return Void();
}
// Apply mutations in batchData->stagingKeys [begin, end).
ACTOR static Future<Void> applyStagingKeysBatch(std::map<Key, StagingKey>::iterator begin,
std::map<Key, StagingKey>::iterator end, Database cx,
FlowLock* applyStagingKeysBatchLock, UID applierID,
ApplierBatchData::Counters* cc) {
std::map<Key, StagingKey>::iterator end, Database cx, UID applierID,
ApplierBatchData::Counters* cc, double* appliedBytes,
double* applyingDataBytes, double* targetMB,
AsyncTrigger* releaseTxnTrigger) {
if (SERVER_KNOBS->FASTRESTORE_NOT_WRITE_DB) {
TraceEvent("FastRestoreApplierPhaseApplyStagingKeysBatchSkipped", applierID).detail("Begin", begin->first);
ASSERT(!g_network->isSimulated());
return Void();
}
wait(applyStagingKeysBatchLock->take(TaskPriority::RestoreApplierWriteDB)); // Q: Do we really need the lock?
state FlowLock::Releaser releaser(*applyStagingKeysBatchLock);
wait(shouldReleaseTransaction(targetMB, applyingDataBytes, releaseTxnTrigger));
state Reference<ReadYourWritesTransaction> tr(new ReadYourWritesTransaction(cx));
state int sets = 0;
state int clears = 0;
state Key endKey = begin->second.key;
state Key endKey = begin->first;
state double txnSize = 0;
state double txnSizeUsed = 0; // txn size accounted in applyingDataBytes
TraceEvent(SevFRDebugInfo, "FastRestoreApplierPhaseApplyStagingKeysBatch", applierID).detail("Begin", begin->first);
loop {
try {
txnSize = 0;
txnSizeUsed = 0;
tr->setOption(FDBTransactionOptions::ACCESS_SYSTEM_KEYS);
tr->setOption(FDBTransactionOptions::LOCK_AWARE);
std::map<Key, StagingKey>::iterator iter = begin;
while (iter != end) {
if (iter->second.type == MutationRef::SetValue) {
tr->set(iter->second.key, iter->second.val);
txnSize += iter->second.totalSize();
cc->appliedMutations += 1;
TraceEvent(SevFRMutationInfo, "FastRestoreApplierPhaseApplyStagingKeysBatch", applierID)
.detail("SetKey", iter->second.key);
@ -478,6 +531,7 @@ ACTOR static Future<Void> applyStagingKeysBatch(std::map<Key, StagingKey>::itera
.detail("SubVersion", iter->second.version.sub);
}
tr->clear(singleKeyRange(iter->second.key));
txnSize += iter->second.totalSize();
cc->appliedMutations += 1;
TraceEvent(SevFRMutationInfo, "FastRestoreApplierPhaseApplyStagingKeysBatch", applierID)
.detail("ClearKey", iter->second.key);
@ -485,7 +539,7 @@ ACTOR static Future<Void> applyStagingKeysBatch(std::map<Key, StagingKey>::itera
} else {
ASSERT(false);
}
endKey = iter != end ? iter->second.key : endKey;
endKey = iter != end ? iter->first : endKey;
iter++;
if (sets > 10000000 || clears > 10000000) {
TraceEvent(SevError, "FastRestoreApplierPhaseApplyStagingKeysBatchInfiniteLoop", applierID)
@ -499,12 +553,22 @@ ACTOR static Future<Void> applyStagingKeysBatch(std::map<Key, StagingKey>::itera
.detail("End", endKey)
.detail("Sets", sets)
.detail("Clears", clears);
tr->addWriteConflictRange(KeyRangeRef(begin->first, keyAfter(endKey))); // Reduce resolver load
txnSizeUsed = txnSize;
*applyingDataBytes += txnSizeUsed; // Must account for applying bytes before wait for write traffic control
wait(tr->commit());
cc->appliedTxns += 1;
cc->appliedBytes += txnSize;
*appliedBytes += txnSize;
*applyingDataBytes -= txnSizeUsed;
if (okToReleaseTxns(*targetMB, *applyingDataBytes)) {
releaseTxnTrigger->trigger();
}
break;
} catch (Error& e) {
cc->appliedTxnRetries += 1;
wait(tr->onError(e));
*applyingDataBytes -= txnSizeUsed;
}
}
return Void();
@ -521,13 +585,14 @@ ACTOR static Future<Void> applyStagingKeys(Reference<ApplierBatchData> batchData
TraceEvent("FastRestoreApplerPhaseApplyStagingKeysStart", applierID)
.detail("BatchIndex", batchIndex)
.detail("StagingKeys", batchData->stagingKeys.size());
batchData->totalBytesToWrite = 0;
while (cur != batchData->stagingKeys.end()) {
txnSize += cur->second.expectedMutationSize();
txnSize += cur->second.totalSize(); // should be consistent with receivedBytes accounting method
if (txnSize > SERVER_KNOBS->FASTRESTORE_TXN_BATCH_MAX_BYTES) {
fBatches.push_back(applyStagingKeysBatch(begin, cur, cx, &batchData->applyStagingKeysBatchLock, applierID,
&batchData->counters));
batchData->counters.appliedBytes += txnSize;
batchData->appliedBytes += txnSize;
fBatches.push_back(applyStagingKeysBatch(begin, cur, cx, applierID, &batchData->counters,
&batchData->appliedBytes, &batchData->applyingDataBytes,
&batchData->targetWriteRateMB, &batchData->releaseTxnTrigger));
batchData->totalBytesToWrite += txnSize;
begin = cur;
txnSize = 0;
txnBatches++;
@ -535,10 +600,10 @@ ACTOR static Future<Void> applyStagingKeys(Reference<ApplierBatchData> batchData
cur++;
}
if (begin != batchData->stagingKeys.end()) {
fBatches.push_back(applyStagingKeysBatch(begin, cur, cx, &batchData->applyStagingKeysBatchLock, applierID,
&batchData->counters));
batchData->counters.appliedBytes += txnSize;
batchData->appliedBytes += txnSize;
fBatches.push_back(applyStagingKeysBatch(begin, cur, cx, applierID, &batchData->counters,
&batchData->appliedBytes, &batchData->applyingDataBytes,
&batchData->targetWriteRateMB, &batchData->releaseTxnTrigger));
batchData->totalBytesToWrite += txnSize;
txnBatches++;
}
@ -547,18 +612,19 @@ ACTOR static Future<Void> applyStagingKeys(Reference<ApplierBatchData> batchData
TraceEvent("FastRestoreApplerPhaseApplyStagingKeysDone", applierID)
.detail("BatchIndex", batchIndex)
.detail("StagingKeys", batchData->stagingKeys.size())
.detail("TransactionBatches", txnBatches);
.detail("TransactionBatches", txnBatches)
.detail("TotalBytesToWrite", batchData->totalBytesToWrite);
return Void();
}
// Write mutations to the destination DB
ACTOR Future<Void> writeMutationsToDB(UID applierID, int64_t batchIndex, Reference<ApplierBatchData> batchData,
Database cx) {
TraceEvent("FastRestoreApplerPhaseApplyTxnStart", applierID).detail("BatchIndex", batchIndex);
TraceEvent("FastRestoreApplierPhaseApplyTxnStart", applierID).detail("BatchIndex", batchIndex);
wait(precomputeMutationsResult(batchData, applierID, batchIndex, cx));
wait(applyStagingKeys(batchData, applierID, batchIndex, cx));
TraceEvent("FastRestoreApplerPhaseApplyTxnDone", applierID)
TraceEvent("FastRestoreApplierPhaseApplyTxnDone", applierID)
.detail("BatchIndex", batchIndex)
.detail("AppliedBytes", batchData->appliedBytes)
.detail("ReceivedBytes", batchData->receivedBytes);
@ -566,6 +632,55 @@ ACTOR Future<Void> writeMutationsToDB(UID applierID, int64_t batchIndex, Referen
return Void();
}
void handleUpdateRateRequest(RestoreUpdateRateRequest req, Reference<RestoreApplierData> self) {
TraceEvent ev("FastRestoreApplierUpdateRateRequest", self->id());
ev.suppressFor(10)
.detail("BatchIndex", req.batchIndex)
.detail("FinishedBatch", self->finishedBatch.get())
.detail("WriteMB", req.writeMB);
double remainingDataMB = 0;
if (self->finishedBatch.get() == req.batchIndex - 1) { // current applying batch
Reference<ApplierBatchData> batchData = self->batch[req.batchIndex];
ASSERT(batchData.isValid());
batchData->targetWriteRateMB = req.writeMB;
remainingDataMB = batchData->totalBytesToWrite > 0
? std::max(0.0, batchData->totalBytesToWrite - batchData->appliedBytes) / 1024 / 1024
: batchData->receivedBytes / 1024 / 1024;
ev.detail("TotalBytesToWrite", batchData->totalBytesToWrite)
.detail("AppliedBytes", batchData->appliedBytes)
.detail("ReceivedBytes", batchData->receivedBytes)
.detail("TargetWriteRateMB", batchData->targetWriteRateMB)
.detail("RemainingDataMB", remainingDataMB);
}
req.reply.send(RestoreUpdateRateReply(self->id(), remainingDataMB));
return;
}
ACTOR static Future<Void> traceRate(const char* context, Reference<ApplierBatchData> batchData, int batchIndex,
UID nodeID, NotifiedVersion* finishedVB, bool once = false) {
loop {
if ((finishedVB->get() != batchIndex - 1) || !batchData.isValid()) {
break;
}
TraceEvent(context, nodeID)
.suppressFor(10)
.detail("BatchIndex", batchIndex)
.detail("FinishedBatchIndex", finishedVB->get())
.detail("TotalDataToWriteMB", batchData->totalBytesToWrite / 1024 / 1024)
.detail("AppliedBytesMB", batchData->appliedBytes / 1024 / 1024)
.detail("TargetBytesMB", batchData->targetWriteRateMB)
.detail("InflightBytesMB", batchData->applyingDataBytes)
.detail("ReceivedBytes", batchData->receivedBytes);
if (once) {
break;
}
wait(delay(5.0));
}
return Void();
}
ACTOR static Future<Void> handleApplyToDBRequest(RestoreVersionBatchRequest req, Reference<RestoreApplierData> self,
Database cx) {
TraceEvent("FastRestoreApplierPhaseHandleApplyToDBStart", self->id())
@ -578,7 +693,9 @@ ACTOR static Future<Void> handleApplyToDBRequest(RestoreVersionBatchRequest req,
state bool isDuplicated = true;
if (self->finishedBatch.get() == req.batchIndex - 1) {
Reference<ApplierBatchData> batchData = self->batch[req.batchIndex];
// duplicate request from earlier version batch will be ignored
state Reference<ApplierBatchData> batchData = self->batch[req.batchIndex];
ASSERT(batchData.isValid());
TraceEvent("FastRestoreApplierPhaseHandleApplyToDBRunning", self->id())
.detail("BatchIndex", req.batchIndex)
.detail("FinishedBatch", self->finishedBatch.get())
@ -592,6 +709,8 @@ ACTOR static Future<Void> handleApplyToDBRequest(RestoreVersionBatchRequest req,
batchData->dbApplier = Never();
batchData->dbApplier = writeMutationsToDB(self->id(), req.batchIndex, batchData, cx);
batchData->vbState = ApplierVersionBatchState::WRITE_TO_DB;
batchData->rateTracer = traceRate("FastRestoreApplierTransactionRateControl", batchData, req.batchIndex,
self->id(), &self->finishedBatch);
}
ASSERT(batchData->dbApplier.present());
@ -600,11 +719,14 @@ ACTOR static Future<Void> handleApplyToDBRequest(RestoreVersionBatchRequest req,
wait(batchData->dbApplier.get());
// Multiple actor invokation can wait on req.batchIndex-1;
// Multiple actors can wait on req.batchIndex-1;
// Avoid setting finishedBatch when finishedBatch > req.batchIndex
if (self->finishedBatch.get() == req.batchIndex - 1) {
batchData->rateTracer =
traceRate("FastRestoreApplierTransactionRateControlDone", batchData, req.batchIndex, self->id(),
&self->finishedBatch, true /*print once*/); // Track the last rate info
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) {

View File

@ -54,7 +54,7 @@ struct StagingKey {
LogMessageVersion version; // largest version of set or clear for the key
std::map<LogMessageVersion, Standalone<MutationRef>> pendingMutations; // mutations not set or clear type
explicit StagingKey() : version(0), type(MutationRef::MAX_ATOMIC_OP) {}
explicit StagingKey(Key key) : key(key), version(0), type(MutationRef::MAX_ATOMIC_OP) {}
// Add mutation m at newVersion to stagingKey
// Assume: SetVersionstampedKey and SetVersionstampedValue have been converted to set
@ -118,7 +118,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())
@ -151,7 +151,7 @@ struct StagingKey {
}
for (; lb != pendingMutations.end(); lb++) {
MutationRef mutation = lb->second;
if (type == MutationRef::CompareAndClear) { // Special atomicOp
if (mutation.type == MutationRef::CompareAndClear) { // Special atomicOp
Arena arena;
Optional<StringRef> inputVal;
if (hasBaseValue()) {
@ -170,14 +170,14 @@ struct StagingKey {
val = applyAtomicOp(inputVal, mutation.param2, (MutationRef::Type)mutation.type);
type = MutationRef::SetValue; // Precomputed result should be set to DB.
} else if (mutation.type == MutationRef::SetValue || mutation.type == MutationRef::ClearRange) {
type = MutationRef::SetValue; // Precomputed result should be set to DB.
type = MutationRef::SetValue;
TraceEvent(SevError, "FastRestoreApplierPrecomputeResultUnexpectedSet", applierID)
.detail("BatchIndex", batchIndex)
.detail("Context", context)
.detail("MutationType", getTypeString(mutation.type))
.detail("Version", lb->first.toString());
} else {
TraceEvent(SevWarnAlways, "FastRestoreApplierPrecomputeResultSkipUnexpectedBackupMutation", applierID)
TraceEvent(SevError, "FastRestoreApplierPrecomputeResultSkipUnexpectedBackupMutation", applierID)
.detail("BatchIndex", batchIndex)
.detail("Context", context)
.detail("MutationType", getTypeString(mutation.type))
@ -202,7 +202,7 @@ struct StagingKey {
return pendingMutations.empty() || version >= pendingMutations.rbegin()->first;
}
int expectedMutationSize() { return key.size() + val.size(); }
int totalSize() { return MutationRef::OVERHEAD_BYTES + key.size() + val.size(); }
};
// The range mutation received on applier.
@ -247,7 +247,6 @@ struct ApplierBatchData : public ReferenceCounted<ApplierBatchData> {
VersionedMutationsMap kvOps; // Mutations at each version
std::map<Key, StagingKey> stagingKeys;
std::set<StagingKeyRange> stagingKeyRanges;
FlowLock applyStagingKeysBatchLock;
Future<Void> pollMetrics;
@ -256,8 +255,13 @@ struct ApplierBatchData : public ReferenceCounted<ApplierBatchData> {
long receiveMutationReqs;
// Stats
long receivedBytes;
long appliedBytes;
double receivedBytes; // received mutation size
double appliedBytes; // after coalesce, how many bytes to write to DB
double targetWriteRateMB; // target amount of data outstanding for DB;
double totalBytesToWrite; // total amount of data in bytes to write
double applyingDataBytes; // amount of data in flight of committing
AsyncTrigger releaseTxnTrigger; // trigger to release more txns
Future<Void> rateTracer; // trace transaction rate control info
// Status counters
struct Counters {
@ -283,18 +287,22 @@ struct ApplierBatchData : public ReferenceCounted<ApplierBatchData> {
void delref() { return ReferenceCounted<ApplierBatchData>::delref(); }
explicit ApplierBatchData(UID nodeID, int batchIndex)
: counters(this, nodeID, batchIndex), applyStagingKeysBatchLock(SERVER_KNOBS->FASTRESTORE_APPLYING_PARALLELISM),
vbState(ApplierVersionBatchState::NOT_INIT), receiveMutationReqs(0), receivedBytes(0), appliedBytes(0) {
: counters(this, nodeID, batchIndex),
targetWriteRateMB(SERVER_KNOBS->FASTRESTORE_WRITE_BW_MB / SERVER_KNOBS->FASTRESTORE_NUM_APPLIERS),
totalBytesToWrite(-1), applyingDataBytes(0), vbState(ApplierVersionBatchState::NOT_INIT),
receiveMutationReqs(0), receivedBytes(0), appliedBytes(0) {
pollMetrics = traceCounters(format("FastRestoreApplierMetrics%d", batchIndex), nodeID,
SERVER_KNOBS->FASTRESTORE_ROLE_LOGGING_DELAY, &counters.cc,
nodeID.toString() + "/RestoreApplierMetrics/" + std::to_string(batchIndex));
TraceEvent("FastRestoreApplierMetricsCreated").detail("Node", nodeID);
}
~ApplierBatchData() = default;
~ApplierBatchData() {
rateTracer = Void(); // cancel actor
}
void addMutation(MutationRef m, LogMessageVersion ver) {
if (!isRangeMutation(m)) {
auto item = stagingKeys.emplace(m.param1, StagingKey());
auto item = stagingKeys.emplace(m.param1, StagingKey(m.param1));
item.first->second.add(m, ver);
} else {
stagingKeyRanges.insert(StagingKeyRange(m, ver));

View File

@ -312,6 +312,8 @@ ACTOR Future<Standalone<VectorRef<KeyValueRef>>> decodeLogFileBlock(Reference<IA
int rLen = wait(file->read(mutateString(buf), len, offset));
if (rLen != len) throw restore_bad_read();
simulateBlobFailure();
Standalone<VectorRef<KeyValueRef>> results({}, buf.arena());
state StringRefReader reader(buf, restore_corrupted_data());

View File

@ -307,6 +307,12 @@ Future<Void> getBatchReplies(RequestStream<Request> Interface::*channel, std::ma
if (ongoingReplies[j].isReady()) {
std::get<2>(replyDurations[ongoingRepliesIndex[j]]) = now();
--oustandingReplies;
} else if (ongoingReplies[j].isError()) {
// When this happens,
// the above assertion ASSERT(ongoingReplies.size() == oustandingReplies) will fail
TraceEvent(SevError, "FastRestoreGetBatchRepliesReplyError")
.detail("OngoingReplyIndex", j)
.detail("FutureError", ongoingReplies[j].getError().what());
}
}
}

View File

@ -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;
@ -104,24 +105,23 @@ ACTOR Future<Void> sampleBackups(Reference<RestoreControllerData> self, RestoreC
}
ACTOR Future<Void> startRestoreController(Reference<RestoreWorkerData> controllerWorker, Database cx) {
state ActorCollection actors(false);
ASSERT(controllerWorker.isValid());
ASSERT(controllerWorker->controllerInterf.present());
state Reference<RestoreControllerData> self =
Reference<RestoreControllerData>(new RestoreControllerData(controllerWorker->controllerInterf.get().id()));
state Future<Void> error = actorCollection(self->addActor.getFuture());
try {
// recruitRestoreRoles must come after controllerWorker has finished collectWorkerInterface
wait(recruitRestoreRoles(controllerWorker, self));
actors.add(updateHeartbeatTime(self));
actors.add(checkRolesLiveness(self));
actors.add(updateProcessMetrics(self));
actors.add(traceProcessMetrics(self, "RestoreController"));
actors.add(sampleBackups(self, controllerWorker->controllerInterf.get()));
// self->addActor.send(updateHeartbeatTime(self));
self->addActor.send(checkRolesLiveness(self));
self->addActor.send(updateProcessMetrics(self));
self->addActor.send(traceProcessMetrics(self, "RestoreController"));
self->addActor.send(sampleBackups(self, controllerWorker->controllerInterf.get()));
wait(startProcessRestoreRequests(self, cx));
wait(startProcessRestoreRequests(self, cx) || error);
} catch (Error& e) {
if (e.code() != error_code_operation_cancelled) {
TraceEvent(SevError, "FastRestoreControllerStart").detail("Reason", "Unexpected unhandled error").error(e);
@ -164,7 +164,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 +303,6 @@ 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);
self->initBackupContainer(request.url);
@ -356,7 +358,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 +380,11 @@ ACTOR static Future<Version> processRestoreRequest(Reference<RestoreControllerDa
wait(delay(SERVER_KNOBS->FASTRESTORE_VB_LAUNCH_DELAY));
}
wait(waitForAll(fBatches));
try {
wait(waitForAll(fBatches));
} catch (Error& e) {
TraceEvent(SevError, "FastRestoreControllerDispatchVersionBatchesUnexpectedError").error(e);
}
TraceEvent("FastRestoreController").detail("RestoreToVersion", request.targetVersion);
return request.targetVersion;
@ -436,6 +442,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)
@ -616,7 +623,7 @@ void splitKeyRangeForAppliers(Reference<ControllerBatchData> batchData,
ASSERT(batchData->samplesSize >= 0);
// Sanity check: samples should not be used after freed
ASSERT((batchData->samplesSize > 0 && !batchData->samples.empty()) ||
batchData->samplesSize == 0 && batchData->samples.empty());
(batchData->samplesSize == 0 && batchData->samples.empty()));
int numAppliers = appliersInterf.size();
double slotSize = std::max(batchData->samplesSize / numAppliers, 1.0);
double cumulativeSize = slotSize;
@ -739,7 +746,9 @@ ACTOR static Future<Version> collectBackupFiles(Reference<IBackupContainer> bc,
std::cout << "Restore to version: " << request.targetVersion << "\nBackupDesc: \n" << desc.toString() << "\n\n";
}
Optional<RestorableFileSet> restorable = wait(bc->getRestoreSet(request.targetVersion));
state VectorRef<KeyRangeRef> restoreRanges;
restoreRanges.add(request.range);
Optional<RestorableFileSet> restorable = wait(bc->getRestoreSet(request.targetVersion, restoreRanges));
if (!restorable.present()) {
TraceEvent(SevWarn, "FastRestoreControllerPhaseCollectBackupFiles")
@ -899,6 +908,49 @@ ACTOR static Future<Void> initializeVersionBatch(std::map<UID, RestoreApplierInt
return Void();
}
// Calculate the amount of data each applier should keep outstanding to DB;
// This is the amount of data that are in in-progress transactions.
ACTOR static Future<Void> updateApplierWriteBW(Reference<ControllerBatchData> batchData,
std::map<UID, RestoreApplierInterface> appliersInterf, int batchIndex) {
state std::unordered_map<UID, double> applierRemainMB;
state double totalRemainMB = SERVER_KNOBS->FASTRESTORE_WRITE_BW_MB;
state double standardAvgBW = SERVER_KNOBS->FASTRESTORE_WRITE_BW_MB / SERVER_KNOBS->FASTRESTORE_NUM_APPLIERS;
state int loopCount = 0;
state std::vector<RestoreUpdateRateReply> replies;
state std::vector<std::pair<UID, RestoreUpdateRateRequest>> requests;
for (auto& applier : appliersInterf) {
applierRemainMB[applier.first] = SERVER_KNOBS->FASTRESTORE_WRITE_BW_MB / SERVER_KNOBS->FASTRESTORE_NUM_APPLIERS;
}
loop {
requests.clear();
for (auto& applier : appliersInterf) {
double writeRate = totalRemainMB > 1 ? (applierRemainMB[applier.first] / totalRemainMB) *
SERVER_KNOBS->FASTRESTORE_WRITE_BW_MB
: standardAvgBW;
requests.emplace_back(applier.first, RestoreUpdateRateRequest(batchIndex, writeRate));
}
replies.clear();
wait(getBatchReplies(
&RestoreApplierInterface::updateRate, appliersInterf, requests, &replies,
TaskPriority::DefaultEndpoint)); // DefaultEndpoint has higher priority than fast restore endpoints
ASSERT(replies.size() == requests.size());
totalRemainMB = 0;
for (int i = 0; i < replies.size(); i++) {
UID& applierID = requests[i].first;
applierRemainMB[applierID] = replies[i].remainMB;
totalRemainMB += replies[i].remainMB;
}
ASSERT(totalRemainMB >= 0);
double delayTime = SERVER_KNOBS->FASTRESTORE_RATE_UPDATE_SECONDS;
if (loopCount == 0) { // First loop: Need to update writeRate quicker
delayTime = 0.2;
}
loopCount++;
wait(delay(delayTime));
}
}
// Ask each applier to apply its received mutations to DB
// NOTE: Controller cannot start applying mutations at batchIndex until all appliers have applied for (batchIndex - 1)
// because appliers at different batchIndex may have overlapped key ranges.
@ -912,6 +964,8 @@ ACTOR static Future<Void> notifyApplierToApplyMutations(Reference<ControllerBatc
wait(finishedBatch->whenAtLeast(batchIndex - 1));
state Future<Void> updateRate;
if (finishedBatch->get() == batchIndex - 1) {
// Prepare the applyToDB requests
std::vector<std::pair<UID, RestoreVersionBatchRequest>> requests;
@ -931,6 +985,7 @@ ACTOR static Future<Void> notifyApplierToApplyMutations(Reference<ControllerBatc
batchData->applyToDB = Never();
batchData->applyToDB = getBatchReplies(&RestoreApplierInterface::applyToDB, appliersInterf, requests,
&replies, TaskPriority::RestoreApplierWriteDB);
updateRate = updateApplierWriteBW(batchData, appliersInterf, batchIndex);
} else {
TraceEvent(SevError, "FastRestoreControllerPhaseApplyToDB")
.detail("BatchIndex", batchIndex)
@ -1042,6 +1097,7 @@ ACTOR static Future<Void> signalRestoreCompleted(Reference<RestoreControllerData
}
// Update the most recent time when controller receives hearbeat from each loader and applier
// TODO: Replace the heartbeat mechanism with FDB failure monitoring mechanism
ACTOR static Future<Void> updateHeartbeatTime(Reference<RestoreControllerData> self) {
wait(self->recruitedRoles.getFuture());
@ -1077,10 +1133,18 @@ ACTOR static Future<Void> updateHeartbeatTime(Reference<RestoreControllerData> s
}
fTimeout = delay(SERVER_KNOBS->FASTRESTORE_HEARTBEAT_DELAY);
wait(waitForAll(fReplies) || fTimeout);
// Here we have to handle error, otherwise controller worker will fail and exit.
try {
wait(waitForAll(fReplies) || fTimeout);
} catch (Error& e) {
// This should be an ignorable error.
TraceEvent(g_network->isSimulated() ? SevWarnAlways : SevError, "FastRestoreUpdateHeartbeatError").error(e);
}
// Update the most recent heart beat time for each role
for (int i = 0; i < fReplies.size(); ++i) {
if (fReplies[i].isReady()) {
if (!fReplies[i].isError() && fReplies[i].isReady()) {
double currentTime = now();
auto item = self->rolesHeartBeatTime.emplace(nodes[i], currentTime);
item.first->second = currentTime;

View File

@ -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(); }
@ -173,7 +177,8 @@ struct RestoreControllerData : RestoreRoleData, public ReferenceCounted<RestoreC
versionBatches.clear();
batch.clear();
batchStatus.clear();
finishedBatch = NotifiedVersion();
finishedBatch = NotifiedVersion(0);
versionBatchId = NotifiedVersion(0);
ASSERT(runningVersionBatches.get() == 0);
}
@ -217,6 +222,7 @@ struct RestoreControllerData : RestoreRoleData, public ReferenceCounted<RestoreC
}
TraceEvent("FastRestoreVersionBatchesSummary")
.detail("VersionBatches", versionBatches.size())
.detail("LogFiles", logFiles)
.detail("RangeFiles", rangeFiles)
.detail("LogBytes", logSize)
@ -310,6 +316,7 @@ struct RestoreControllerData : RestoreRoleData, public ReferenceCounted<RestoreC
}
}
} else {
// TODO: Check why this may happen?!
TraceEvent(SevError, "FastRestoreBuildVersionBatch")
.detail("RangeIndex", rangeIdx)
.detail("RangeFiles", rangeFiles.size())

View File

@ -49,28 +49,186 @@ 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);
ACTOR static Future<Void> parseLogFileToMutationsOnLoader(NotifiedVersion* pProcessedFileOffset,
SerializedMutationListMap* mutationMap,
Reference<IBackupContainer> bc, RestoreAsset asset);
ACTOR static Future<Void> _parseRangeFileToMutationsOnLoader(
std::map<LoadingParam, VersionedMutationsMap>::iterator kvOpsIter,
std::map<LoadingParam, SampledMutationsVec>::iterator samplesIter, LoaderCounters* cc,
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]";
@ -86,12 +244,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";
@ -113,10 +279,11 @@ 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",
self->id())
bool isError = e.code() != error_code_operation_cancelled; // == error_code_broken_promise
TraceEvent(isError ? SevError : SevWarnAlways, "FastRestoreLoaderError", self->id())
.detail("RequestType", requestTypeStr)
.error(e, true);
actors.clear(false);
@ -189,8 +356,11 @@ 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)
simulateBlobFailure();
TraceEvent("FastRestoreLoaderDecodingLogFile")
.detail("BatchIndex", asset.batchIndex)
.detail("Filename", asset.filename)
.detail("Offset", asset.offset)
.detail("Length", asset.len);
@ -284,6 +454,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);
@ -293,6 +464,39 @@ ACTOR static Future<Void> _parsePartitionedLogFileOnLoader(
return Void();
}
// wrapper of _parsePartitionedLogFileOnLoader to retry on blob error
ACTOR static Future<Void> parsePartitionedLogFileOnLoader(
KeyRangeMap<Version>* pRangeVersions, NotifiedVersion* processedFileOffset,
std::map<LoadingParam, VersionedMutationsMap>::iterator kvOpsIter,
std::map<LoadingParam, SampledMutationsVec>::iterator samplesIter, LoaderCounters* cc,
Reference<IBackupContainer> bc, RestoreAsset asset) {
state int readFileRetries = 0;
loop {
try {
wait(_parsePartitionedLogFileOnLoader(pRangeVersions, processedFileOffset, kvOpsIter, samplesIter, cc, bc,
asset));
break;
} catch (Error& e) {
if (e.code() == error_code_restore_bad_read || e.code() == error_code_restore_unsupported_file_version ||
e.code() == error_code_restore_corrupted_data_padding) { // no retriable error
TraceEvent(SevError, "FastRestoreFileRestoreCorruptedPartitionedLogFileBlock").error(e);
throw;
} else if (e.code() == error_code_http_request_failed || e.code() == error_code_connection_failed ||
e.code() == error_code_timed_out || e.code() == error_code_lookup_failed) {
// blob http request failure, retry
TraceEvent(SevWarnAlways, "FastRestoreDecodedPartitionedLogFileConnectionFailure")
.detail("Retries", ++readFileRetries)
.error(e);
wait(delayJittered(0.1));
} else {
TraceEvent(SevError, "FastRestoreParsePartitionedLogFileOnLoaderUnexpectedError").error(e);
throw;
}
}
}
return Void();
}
ACTOR Future<Void> _processLoadingParam(KeyRangeMap<Version>* pRangeVersions, LoadingParam param,
Reference<LoaderBatchData> batchData, UID loaderID,
Reference<IBackupContainer> bc) {
@ -305,8 +509,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());
@ -328,12 +533,12 @@ ACTOR Future<Void> _processLoadingParam(KeyRangeMap<Version>* pRangeVersions, Lo
} else {
// TODO: Sanity check the log file's range is overlapped with the restored version range
if (param.isPartitionedLog()) {
fileParserFutures.push_back(_parsePartitionedLogFileOnLoader(pRangeVersions, &processedFileOffset,
kvOpsPerLPIter, samplesIter,
&batchData->counters, bc, subAsset));
fileParserFutures.push_back(parsePartitionedLogFileOnLoader(pRangeVersions, &processedFileOffset,
kvOpsPerLPIter, samplesIter,
&batchData->counters, bc, subAsset));
} else {
fileParserFutures.push_back(
_parseLogFileToMutationsOnLoader(&processedFileOffset, &mutationMap, bc, subAsset));
parseLogFileToMutationsOnLoader(&processedFileOffset, &mutationMap, bc, subAsset));
}
}
}
@ -344,7 +549,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();
}
@ -355,6 +562,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;
@ -379,6 +587,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())
@ -414,15 +623,17 @@ ACTOR Future<Void> handleLoadFileRequest(RestoreLoadFileRequest req, Reference<R
state int samplesMessages = fSendSamples.size();
wait(waitForAll(fSendSamples));
} catch (Error& e) { // In case ci.samples throws broken_promise due to unstable network
if (e.code() == error_code_broken_promise) {
if (e.code() == error_code_broken_promise || e.code() == error_code_operation_cancelled) {
TraceEvent(SevWarnAlways, "FastRestoreLoaderPhaseLoadFileSendSamples")
.detail("SamplesMessages", samplesMessages);
.detail("SamplesMessages", samplesMessages)
.error(e, true);
} else {
TraceEvent(SevError, "FastRestoreLoaderPhaseLoadFileSendSamplesUnexpectedError").error(e, true);
}
}
// 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)
@ -435,16 +646,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) {
@ -484,17 +708,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 {
@ -503,6 +730,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();
}
}
@ -533,9 +761,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;
@ -544,6 +774,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)
@ -642,7 +886,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());
@ -666,7 +910,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());
@ -675,11 +919,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();
}
@ -780,6 +1035,7 @@ void _parseSerializedMutation(KeyRangeMap<Version>* pRangeVersions,
SerializedMutationListMap& mutationMap = *pmutationMap;
TraceEvent(SevFRMutationInfo, "FastRestoreLoaderParseSerializedLogMutation")
.detail("BatchIndex", asset.batchIndex)
.detail("RestoreAsset", asset.toString());
Arena tempArena;
@ -882,6 +1138,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)
@ -890,19 +1147,39 @@ ACTOR static Future<Void> _parseRangeFileToMutationsOnLoader(
// Sanity check the range file is within the restored version range
ASSERT_WE_THINK(asset.isInVersionRange(version));
// The set of key value version is rangeFile.version. the key-value set in the same range file has the same version
Reference<IAsyncFile> inFile = wait(bc->readFile(asset.filename));
state Standalone<VectorRef<KeyValueRef>> blockData;
try {
Standalone<VectorRef<KeyValueRef>> kvs =
wait(fileBackup::decodeRangeFileBlock(inFile, asset.offset, asset.len));
TraceEvent("FastRestoreLoader")
.detail("DecodedRangeFile", asset.filename)
.detail("DataSize", kvs.contents().size());
blockData = kvs;
} catch (Error& e) {
TraceEvent(SevError, "FileRestoreCorruptRangeFileBlock").error(e);
throw;
// should retry here
state int readFileRetries = 0;
loop {
try {
// The set of key value version is rangeFile.version. the key-value set in the same range file has the same
// version
Reference<IAsyncFile> inFile = wait(bc->readFile(asset.filename));
Standalone<VectorRef<KeyValueRef>> kvs =
wait(fileBackup::decodeRangeFileBlock(inFile, asset.offset, asset.len));
TraceEvent("FastRestoreLoaderDecodedRangeFile")
.detail("BatchIndex", asset.batchIndex)
.detail("Filename", asset.filename)
.detail("DataSize", kvs.contents().size());
blockData = kvs;
break;
} catch (Error& e) {
if (e.code() == error_code_restore_bad_read || e.code() == error_code_restore_unsupported_file_version ||
e.code() == error_code_restore_corrupted_data_padding) { // no retriable error
TraceEvent(SevError, "FastRestoreFileRestoreCorruptedRangeFileBlock").error(e);
throw;
} else if (e.code() == error_code_http_request_failed || e.code() == error_code_connection_failed ||
e.code() == error_code_timed_out || e.code() == error_code_lookup_failed) {
// blob http request failure, retry
TraceEvent(SevWarnAlways, "FastRestoreDecodedRangeFileConnectionFailure")
.detail("Retries", ++readFileRetries)
.error(e);
wait(delayJittered(0.1));
} else {
TraceEvent(SevError, "FastRestoreParseRangeFileOnLoaderUnexpectedError").error(e);
throw;
}
}
}
// First and last key are the range for this file
@ -952,6 +1229,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());
@ -981,6 +1259,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());
@ -998,6 +1277,36 @@ ACTOR static Future<Void> _parseLogFileToMutationsOnLoader(NotifiedVersion* pPro
return Void();
}
// retry on _parseLogFileToMutationsOnLoader
ACTOR static Future<Void> parseLogFileToMutationsOnLoader(NotifiedVersion* pProcessedFileOffset,
SerializedMutationListMap* pMutationMap,
Reference<IBackupContainer> bc, RestoreAsset asset) {
state int readFileRetries = 0;
loop {
try {
wait(_parseLogFileToMutationsOnLoader(pProcessedFileOffset, pMutationMap, bc, asset));
break;
} catch (Error& e) {
if (e.code() == error_code_restore_bad_read || e.code() == error_code_restore_unsupported_file_version ||
e.code() == error_code_restore_corrupted_data_padding) { // non retriable error
TraceEvent(SevError, "FastRestoreFileRestoreCorruptedLogFileBlock").error(e);
throw;
} else if (e.code() == error_code_http_request_failed || e.code() == error_code_connection_failed ||
e.code() == error_code_timed_out || e.code() == error_code_lookup_failed) {
// blob http request failure, retry
TraceEvent(SevWarnAlways, "FastRestoreDecodedLogFileConnectionFailure")
.detail("Retries", ++readFileRetries)
.error(e);
wait(delayJittered(0.1));
} else {
TraceEvent(SevError, "FastRestoreParseLogFileToMutationsOnLoaderUnexpectedError").error(e);
throw;
}
}
}
return Void();
}
// Return applier IDs that are used to apply key-values
std::vector<UID> getApplierIDs(std::map<Key, UID>& rangeToApplier) {
std::vector<UID> applierIDs;
@ -1018,6 +1327,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);

View File

@ -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;

View File

@ -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));

View File

@ -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);

View File

@ -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()));

View File

@ -1623,25 +1623,28 @@ static int getExtraTLogEligibleZones(const vector<WorkerDetails>& workers, const
if(configuration.regions.size() == 0) {
return allZones.size() - std::max(configuration.tLogReplicationFactor, configuration.storageTeamSize);
}
int extraTlogEligibleZones = configuration.usableRegions == 1 ? 0 : std::numeric_limits<int>::max();
int extraTlogEligibleZones = 0;
int regionsWithNonNegativePriority = 0;
for(auto& region : configuration.regions) {
int eligible = dcId_zone[region.dcId].size() - std::max(configuration.remoteTLogReplicationFactor, std::max(configuration.tLogReplicationFactor, configuration.storageTeamSize) );
//FIXME: does not take into account fallback satellite policies
if(region.satelliteTLogReplicationFactor > 0 && configuration.usableRegions > 1) {
int totalSatelliteEligible = 0;
for(auto& sat : region.satellites) {
totalSatelliteEligible += dcId_zone[sat.dcId].size();
if( region.priority >= 0 ) {
int eligible = dcId_zone[region.dcId].size() - std::max(configuration.remoteTLogReplicationFactor, std::max(configuration.tLogReplicationFactor, configuration.storageTeamSize) );
//FIXME: does not take into account fallback satellite policies
if(region.satelliteTLogReplicationFactor > 0 && configuration.usableRegions > 1) {
int totalSatelliteEligible = 0;
for(auto& sat : region.satellites) {
totalSatelliteEligible += dcId_zone[sat.dcId].size();
}
eligible = std::min<int>( eligible, totalSatelliteEligible - region.satelliteTLogReplicationFactor );
}
eligible = std::min<int>( eligible, totalSatelliteEligible - region.satelliteTLogReplicationFactor );
}
if( configuration.usableRegions == 1 ) {
if( region.priority >= 0 ) {
extraTlogEligibleZones = std::max( extraTlogEligibleZones, eligible );
if(eligible >= 0) {
regionsWithNonNegativePriority++;
}
} else {
extraTlogEligibleZones = std::min( extraTlogEligibleZones, eligible );
extraTlogEligibleZones = std::max( extraTlogEligibleZones, eligible );
}
}
if(regionsWithNonNegativePriority > 1) {
extraTlogEligibleZones++;
}
return extraTlogEligibleZones;
}
@ -1908,82 +1911,117 @@ ACTOR static Future<JsonBuilderObject> clusterSummaryStatisticsFetcher(WorkerEve
return statusObj;
}
static JsonBuilderArray oldTlogFetcher(int* oldLogFaultTolerance, Reference<AsyncVar<ServerDBInfo>> db, std::unordered_map<NetworkAddress, WorkerInterface> const& address_workers) {
JsonBuilderArray oldTlogsArray;
static JsonBuilderObject tlogFetcher(int* logFaultTolerance, const std::vector<TLogSet>& tLogs,
std::unordered_map<NetworkAddress, WorkerInterface> const& address_workers) {
JsonBuilderObject statusObj;
JsonBuilderArray logsObj;
Optional<int32_t> sat_log_replication_factor, sat_log_write_anti_quorum, sat_log_fault_tolerance,
log_replication_factor, log_write_anti_quorum, log_fault_tolerance, remote_log_replication_factor,
remote_log_fault_tolerance;
if(db->get().recoveryState >= RecoveryState::ACCEPTING_COMMITS) {
for(auto it : db->get().logSystemConfig.oldTLogs) {
JsonBuilderObject statusObj;
JsonBuilderArray logsObj;
Optional<int32_t> sat_log_replication_factor, sat_log_write_anti_quorum, sat_log_fault_tolerance, log_replication_factor, log_write_anti_quorum, log_fault_tolerance, remote_log_replication_factor, remote_log_fault_tolerance;
int minFaultTolerance = 1000;
int localSetsWithNonNegativeFaultTolerance = 0;
int maxFaultTolerance = 0;
for(int i = 0; i < it.tLogs.size(); i++) {
int failedLogs = 0;
for(auto& log : it.tLogs[i].tLogs) {
JsonBuilderObject logObj;
bool failed = !log.present() || !address_workers.count(log.interf().address());
logObj["id"] = log.id().shortString();
logObj["healthy"] = !failed;
if(log.present()) {
logObj["address"] = log.interf().address().toString();
}
logsObj.push_back(logObj);
if(failed) {
failedLogs++;
}
}
maxFaultTolerance = std::max(maxFaultTolerance, it.tLogs[i].tLogReplicationFactor - 1 - it.tLogs[i].tLogWriteAntiQuorum - failedLogs);
if(it.tLogs[i].isLocal && it.tLogs[i].locality == tagLocalitySatellite) {
sat_log_replication_factor = it.tLogs[i].tLogReplicationFactor;
sat_log_write_anti_quorum = it.tLogs[i].tLogWriteAntiQuorum;
sat_log_fault_tolerance = it.tLogs[i].tLogReplicationFactor - 1 - it.tLogs[i].tLogWriteAntiQuorum - failedLogs;
}
else if(it.tLogs[i].isLocal) {
log_replication_factor = it.tLogs[i].tLogReplicationFactor;
log_write_anti_quorum = it.tLogs[i].tLogWriteAntiQuorum;
log_fault_tolerance = it.tLogs[i].tLogReplicationFactor - 1 - it.tLogs[i].tLogWriteAntiQuorum - failedLogs;
}
else {
remote_log_replication_factor = it.tLogs[i].tLogReplicationFactor;
remote_log_fault_tolerance = it.tLogs[i].tLogReplicationFactor - 1 - failedLogs;
}
for (int i = 0; i < tLogs.size(); i++) {
int failedLogs = 0;
for (auto& log : tLogs[i].tLogs) {
JsonBuilderObject logObj;
bool failed = !log.present() || !address_workers.count(log.interf().address());
logObj["id"] = log.id().shortString();
logObj["healthy"] = !failed;
if (log.present()) {
logObj["address"] = log.interf().address().toString();
}
*oldLogFaultTolerance = std::min(*oldLogFaultTolerance, maxFaultTolerance);
statusObj["logs"] = logsObj;
logsObj.push_back(logObj);
if (failed) {
failedLogs++;
}
}
if (sat_log_replication_factor.present())
statusObj["satellite_log_replication_factor"] = sat_log_replication_factor.get();
if (sat_log_write_anti_quorum.present())
statusObj["satellite_log_write_anti_quorum"] = sat_log_write_anti_quorum.get();
if (sat_log_fault_tolerance.present())
statusObj["satellite_log_fault_tolerance"] = sat_log_fault_tolerance.get();
if (tLogs[i].isLocal) {
int currentFaultTolerance = tLogs[i].tLogReplicationFactor - 1 - tLogs[i].tLogWriteAntiQuorum - failedLogs;
if(currentFaultTolerance >= 0) {
localSetsWithNonNegativeFaultTolerance++;
}
minFaultTolerance = std::min(minFaultTolerance, currentFaultTolerance);
}
if (log_replication_factor.present())
statusObj["log_replication_factor"] = log_replication_factor.get();
if (log_write_anti_quorum.present())
statusObj["log_write_anti_quorum"] = log_write_anti_quorum.get();
if (log_fault_tolerance.present())
statusObj["log_fault_tolerance"] = log_fault_tolerance.get();
if (remote_log_replication_factor.present())
statusObj["remote_log_replication_factor"] = remote_log_replication_factor.get();
if (remote_log_fault_tolerance.present())
statusObj["remote_log_fault_tolerance"] = remote_log_fault_tolerance.get();
oldTlogsArray.push_back(statusObj);
if (tLogs[i].isLocal && tLogs[i].locality == tagLocalitySatellite) {
sat_log_replication_factor = tLogs[i].tLogReplicationFactor;
sat_log_write_anti_quorum = tLogs[i].tLogWriteAntiQuorum;
sat_log_fault_tolerance = tLogs[i].tLogReplicationFactor - 1 - tLogs[i].tLogWriteAntiQuorum - failedLogs;
} else if (tLogs[i].isLocal) {
log_replication_factor = tLogs[i].tLogReplicationFactor;
log_write_anti_quorum = tLogs[i].tLogWriteAntiQuorum;
log_fault_tolerance = tLogs[i].tLogReplicationFactor - 1 - tLogs[i].tLogWriteAntiQuorum - failedLogs;
} else {
remote_log_replication_factor = tLogs[i].tLogReplicationFactor;
remote_log_fault_tolerance = tLogs[i].tLogReplicationFactor - 1 - failedLogs;
}
}
if(minFaultTolerance == 1000) {
//just in case we do not have any tlog sets
minFaultTolerance = 0;
}
if(localSetsWithNonNegativeFaultTolerance > 1) {
minFaultTolerance++;
}
*logFaultTolerance = std::min(*logFaultTolerance, minFaultTolerance);
statusObj["log_interfaces"] = logsObj;
// We may lose logs in this log generation, storage servers may never be able to catch up this log
// generation.
statusObj["possibly_losing_data"] = minFaultTolerance < 0;
return oldTlogsArray;
if (sat_log_replication_factor.present())
statusObj["satellite_log_replication_factor"] = sat_log_replication_factor.get();
if (sat_log_write_anti_quorum.present())
statusObj["satellite_log_write_anti_quorum"] = sat_log_write_anti_quorum.get();
if (sat_log_fault_tolerance.present()) statusObj["satellite_log_fault_tolerance"] = sat_log_fault_tolerance.get();
if (log_replication_factor.present()) statusObj["log_replication_factor"] = log_replication_factor.get();
if (log_write_anti_quorum.present()) statusObj["log_write_anti_quorum"] = log_write_anti_quorum.get();
if (log_fault_tolerance.present()) statusObj["log_fault_tolerance"] = log_fault_tolerance.get();
if (remote_log_replication_factor.present())
statusObj["remote_log_replication_factor"] = remote_log_replication_factor.get();
if (remote_log_fault_tolerance.present())
statusObj["remote_log_fault_tolerance"] = remote_log_fault_tolerance.get();
return statusObj;
}
static JsonBuilderObject faultToleranceStatusFetcher(DatabaseConfiguration configuration, ServerCoordinators coordinators, std::vector<WorkerDetails>& workers, int extraTlogEligibleZones, int minReplicasRemaining, bool underMaintenance) {
static JsonBuilderArray tlogFetcher(int* logFaultTolerance, Reference<AsyncVar<ServerDBInfo>> db,
std::unordered_map<NetworkAddress, WorkerInterface> const& address_workers) {
JsonBuilderArray tlogsArray;
JsonBuilderObject tlogsStatus;
tlogsStatus = tlogFetcher(logFaultTolerance, db->get().logSystemConfig.tLogs, address_workers);
tlogsStatus["epoch"] = db->get().logSystemConfig.epoch;
tlogsStatus["current"] = true;
if (db->get().logSystemConfig.recoveredAt.present()) {
tlogsStatus["begin_version"] = db->get().logSystemConfig.recoveredAt.get();
}
tlogsArray.push_back(tlogsStatus);
for (auto it : db->get().logSystemConfig.oldTLogs) {
JsonBuilderObject oldTlogsStatus = tlogFetcher(logFaultTolerance, it.tLogs, address_workers);
oldTlogsStatus["epoch"] = it.epoch;
oldTlogsStatus["current"] = false;
oldTlogsStatus["begin_version"] = it.epochBegin;
oldTlogsStatus["end_version"] = it.epochEnd;
tlogsArray.push_back(oldTlogsStatus);
}
return tlogsArray;
}
static JsonBuilderObject faultToleranceStatusFetcher(DatabaseConfiguration configuration,
ServerCoordinators coordinators,
std::vector<WorkerDetails>& workers, int extraTlogEligibleZones,
int minReplicasRemaining, int oldLogFaultTolerance,
int fullyReplicatedRegions,
bool underMaintenance) {
JsonBuilderObject statusObj;
// without losing data
int32_t maxZoneFailures = configuration.maxZoneFailuresTolerated();
int32_t maxZoneFailures = configuration.maxZoneFailuresTolerated(fullyReplicatedRegions, false);
if(underMaintenance) {
maxZoneFailures--;
}
@ -2011,17 +2049,24 @@ static JsonBuilderObject faultToleranceStatusFetcher(DatabaseConfiguration confi
}
maxCoordinatorZoneFailures += 1;
}
// max zone failures that we can tolerate to not lose data
int zoneFailuresWithoutLosingData = std::min(maxZoneFailures, maxCoordinatorZoneFailures);
if (minReplicasRemaining >= 0){
zoneFailuresWithoutLosingData = std::min(zoneFailuresWithoutLosingData, minReplicasRemaining - 1);
}
statusObj["max_zone_failures_without_losing_data"] = std::max(zoneFailuresWithoutLosingData, 0);
// oldLogFaultTolerance means max failures we can tolerate to lose logs data. -1 means we lose data or availability.
zoneFailuresWithoutLosingData = std::max(std::min(zoneFailuresWithoutLosingData, oldLogFaultTolerance), -1);
statusObj["max_zone_failures_without_losing_data"] = zoneFailuresWithoutLosingData;
// without losing availablity
statusObj["max_zone_failures_without_losing_availability"] = std::max(std::min(extraTlogEligibleZones, zoneFailuresWithoutLosingData), 0);
int32_t maxAvaiabilityZoneFailures = configuration.maxZoneFailuresTolerated(fullyReplicatedRegions, true);
if(underMaintenance) {
maxAvaiabilityZoneFailures--;
}
statusObj["max_zone_failures_without_losing_availability"] =
std::max(std::min(maxAvaiabilityZoneFailures,std::min(extraTlogEligibleZones, zoneFailuresWithoutLosingData)), -1);
return statusObj;
}
@ -2198,7 +2243,7 @@ ACTOR Future<JsonBuilderObject> lockedStatusFetcher(Reference<AsyncVar<ServerDBI
return statusObj;
}
ACTOR Future<Optional<Value>> getActivePrimaryDC(Database cx, JsonBuilderArray* messages) {
ACTOR Future<Optional<Value>> getActivePrimaryDC(Database cx, int* fullyReplicatedRegions, JsonBuilderArray* messages) {
state ReadYourWritesTransaction tr(cx);
state Future<Void> readTimeout = delay(5); // so that we won't loop forever
@ -2209,12 +2254,17 @@ ACTOR Future<Optional<Value>> getActivePrimaryDC(Database cx, JsonBuilderArray*
}
tr.setOption(FDBTransactionOptions::READ_SYSTEM_KEYS);
tr.setOption(FDBTransactionOptions::PRIORITY_SYSTEM_IMMEDIATE);
Optional<Value> res = wait(timeoutError(tr.get(primaryDatacenterKey), 5));
if (!res.present()) {
state Future<Standalone<RangeResultRef>> fReplicaKeys = tr.getRange(datacenterReplicasKeys, CLIENT_KNOBS->TOO_MANY);
state Future<Optional<Value>> fPrimaryDatacenterKey = tr.get(primaryDatacenterKey);
wait(timeoutError(success(fPrimaryDatacenterKey) && success(fReplicaKeys), 5));
*fullyReplicatedRegions = fReplicaKeys.get().size();
if (!fPrimaryDatacenterKey.get().present()) {
messages->push_back(
JsonString::makeMessage("primary_dc_missing", "Unable to determine primary datacenter."));
}
return res;
return fPrimaryDatacenterKey.get();
} catch (Error& e) {
if (e.code() == error_code_timed_out) {
messages->push_back(
@ -2405,7 +2455,8 @@ ACTOR Future<StatusReply> clusterGetStatus(
state Future<ErrorOr<vector<std::pair<MasterProxyInterface, EventMap>>>> proxyFuture = errorOr(getProxiesAndMetrics(db, address_workers));
state int minReplicasRemaining = -1;
state Future<Optional<Value>> primaryDCFO = getActivePrimaryDC(cx, &messages);
state int fullyReplicatedRegions = -1;
state Future<Optional<Value>> primaryDCFO = getActivePrimaryDC(cx, &fullyReplicatedRegions, &messages);
std::vector<Future<JsonBuilderObject>> futures2;
futures2.push_back(dataStatusFetcher(ddWorker, configuration.get(), &minReplicasRemaining));
futures2.push_back(workloadStatusFetcher(db, workers, mWorker, rkWorker, &qos, &data_overlay, &status_incomplete_reasons, storageServerFuture));
@ -2413,21 +2464,23 @@ ACTOR Future<StatusReply> clusterGetStatus(
futures2.push_back(lockedStatusFetcher(db, &messages, &status_incomplete_reasons));
futures2.push_back(clusterSummaryStatisticsFetcher(pMetrics, storageServerFuture, tLogFuture, &status_incomplete_reasons));
state std::vector<JsonBuilderObject> workerStatuses = wait(getAll(futures2));
wait(success(primaryDCFO));
int oldLogFaultTolerance = 100;
if(db->get().recoveryState >= RecoveryState::ACCEPTING_COMMITS && db->get().logSystemConfig.oldTLogs.size() > 0) {
statusObj["old_logs"] = oldTlogFetcher(&oldLogFaultTolerance, db, address_workers);
int logFaultTolerance = 100;
if (db->get().recoveryState >= RecoveryState::ACCEPTING_COMMITS) {
statusObj["logs"] = tlogFetcher(&logFaultTolerance, db, address_workers);
}
if(configuration.present()) {
int extraTlogEligibleZones = getExtraTLogEligibleZones(workers, configuration.get());
statusObj["fault_tolerance"] = faultToleranceStatusFetcher(configuration.get(), coordinators, workers, extraTlogEligibleZones, minReplicasRemaining, loadResult.present() && loadResult.get().healthyZone.present());
statusObj["fault_tolerance"] = faultToleranceStatusFetcher(
configuration.get(), coordinators, workers, extraTlogEligibleZones, minReplicasRemaining,
logFaultTolerance, fullyReplicatedRegions, loadResult.present() && loadResult.get().healthyZone.present());
}
state JsonBuilderObject configObj =
configurationFetcher(configuration, coordinators, &status_incomplete_reasons);
wait(success(primaryDCFO));
if (primaryDCFO.get().present()) {
statusObj["active_primary_dc"] = primaryDCFO.get().get();
}

View File

@ -416,9 +416,10 @@ struct StorageServerMetrics {
// Given a read hot shard, this function will divide the shard into chunks and find those chunks whose
// readBytes/sizeBytes exceeds the `readDensityRatio`. Please make sure to run unit tests
// `StorageMetricsSampleTests.txt` after change made.
std::vector<KeyRangeRef> getReadHotRanges(KeyRangeRef shard, double readDensityRatio, int64_t baseChunkSize,
int64_t minShardReadBandwidthPerKSeconds) {
std::vector<KeyRangeRef> toReturn;
std::vector<ReadHotRangeWithMetrics> getReadHotRanges(KeyRangeRef shard, double readDensityRatio,
int64_t baseChunkSize,
int64_t minShardReadBandwidthPerKSeconds) {
std::vector<ReadHotRangeWithMetrics> toReturn;
double shardSize = (double)byteSample.getEstimate(shard);
int64_t shardReadBandwidth = bytesReadSample.getEstimate(shard);
if (shardReadBandwidth * SERVER_KNOBS->STORAGE_METRICS_AVERAGE_INTERVAL_PER_KSECONDS <=
@ -428,7 +429,9 @@ struct StorageServerMetrics {
if (shardSize <= baseChunkSize) {
// Shard is small, use it as is
if (bytesReadSample.getEstimate(shard) > (readDensityRatio * shardSize)) {
toReturn.push_back(shard);
toReturn.emplace_back(shard, bytesReadSample.getEstimate(shard) / shardSize,
bytesReadSample.getEstimate(shard) /
SERVER_KNOBS->STORAGE_METRICS_AVERAGE_INTERVAL);
}
return toReturn;
}
@ -450,14 +453,15 @@ struct StorageServerMetrics {
if (bytesReadSample.getEstimate(KeyRangeRef(beginKey, *endKey)) >
(readDensityRatio * std::max(baseChunkSize, byteSample.getEstimate(KeyRangeRef(beginKey, *endKey))))) {
auto range = KeyRangeRef(beginKey, *endKey);
if (!toReturn.empty() && toReturn.back().end == range.begin) {
if (!toReturn.empty() && toReturn.back().keys.end == range.begin) {
// in case two consecutive chunks both are over the ratio, merge them.
auto updatedTail = KeyRangeRef(toReturn.back().begin, *endKey);
range = KeyRangeRef(toReturn.back().keys.begin, *endKey);
toReturn.pop_back();
toReturn.push_back(updatedTail);
} else {
toReturn.push_back(range);
}
toReturn.emplace_back(
range,
(double)bytesReadSample.getEstimate(range) / std::max(baseChunkSize, byteSample.getEstimate(range)),
bytesReadSample.getEstimate(range) / SERVER_KNOBS->STORAGE_METRICS_AVERAGE_INTERVAL);
}
beginKey = *endKey;
endKey = byteSample.sample.index(byteSample.sample.sumTo(byteSample.sample.lower_bound(beginKey)) +
@ -468,10 +472,10 @@ struct StorageServerMetrics {
void getReadHotRanges(ReadHotSubRangeRequest req) {
ReadHotSubRangeReply reply;
std::vector<KeyRangeRef> v = getReadHotRanges(req.keys, SERVER_KNOBS->SHARD_MAX_READ_DENSITY_RATIO,
SERVER_KNOBS->READ_HOT_SUB_RANGE_CHUNK_SIZE,
SERVER_KNOBS->SHARD_READ_HOT_BANDWITH_MIN_PER_KSECONDS);
reply.readHotRanges = VectorRef<KeyRangeRef>(v.data(), v.size());
auto _ranges = getReadHotRanges(req.keys, SERVER_KNOBS->SHARD_MAX_READ_DENSITY_RATIO,
SERVER_KNOBS->READ_HOT_SUB_RANGE_CHUNK_SIZE,
SERVER_KNOBS->SHARD_READ_HOT_BANDWITH_MIN_PER_KSECONDS);
reply.readHotRanges = VectorRef(_ranges.data(), _ranges.size());
req.reply.send(reply);
}
@ -515,11 +519,11 @@ TEST_CASE("/fdbserver/StorageMetricSample/readHotDetect/simple") {
ssm.byteSample.sample.insert(LiteralStringRef("But"), 100 * sampleUnit);
ssm.byteSample.sample.insert(LiteralStringRef("Cat"), 300 * sampleUnit);
vector<KeyRangeRef> t =
std::vector<ReadHotRangeWithMetrics> t =
ssm.getReadHotRanges(KeyRangeRef(LiteralStringRef("A"), LiteralStringRef("C")), 2.0, 200 * sampleUnit, 0);
ASSERT(t.size() == 1 && (*t.begin()).begin == LiteralStringRef("Bah") &&
(*t.begin()).end == LiteralStringRef("Bob"));
ASSERT(t.size() == 1 && (*t.begin()).keys.begin == LiteralStringRef("Bah") &&
(*t.begin()).keys.end == LiteralStringRef("Bob"));
return Void();
}
@ -546,12 +550,12 @@ TEST_CASE("/fdbserver/StorageMetricSample/readHotDetect/moreThanOneRange") {
ssm.byteSample.sample.insert(LiteralStringRef("Cat"), 300 * sampleUnit);
ssm.byteSample.sample.insert(LiteralStringRef("Dah"), 300 * sampleUnit);
vector<KeyRangeRef> t =
std::vector<ReadHotRangeWithMetrics> t =
ssm.getReadHotRanges(KeyRangeRef(LiteralStringRef("A"), LiteralStringRef("D")), 2.0, 200 * sampleUnit, 0);
ASSERT(t.size() == 2 && (*t.begin()).begin == LiteralStringRef("Bah") &&
(*t.begin()).end == LiteralStringRef("Bob"));
ASSERT(t.at(1).begin == LiteralStringRef("Cat") && t.at(1).end == LiteralStringRef("Dah"));
ASSERT(t.size() == 2 && (*t.begin()).keys.begin == LiteralStringRef("Bah") &&
(*t.begin()).keys.end == LiteralStringRef("Bob"));
ASSERT(t.at(1).keys.begin == LiteralStringRef("Cat") && t.at(1).keys.end == LiteralStringRef("Dah"));
return Void();
}
@ -579,12 +583,12 @@ TEST_CASE("/fdbserver/StorageMetricSample/readHotDetect/consecutiveRanges") {
ssm.byteSample.sample.insert(LiteralStringRef("Cat"), 300 * sampleUnit);
ssm.byteSample.sample.insert(LiteralStringRef("Dah"), 300 * sampleUnit);
vector<KeyRangeRef> t =
std::vector<ReadHotRangeWithMetrics> t =
ssm.getReadHotRanges(KeyRangeRef(LiteralStringRef("A"), LiteralStringRef("D")), 2.0, 200 * sampleUnit, 0);
ASSERT(t.size() == 2 && (*t.begin()).begin == LiteralStringRef("Bah") &&
(*t.begin()).end == LiteralStringRef("But"));
ASSERT(t.at(1).begin == LiteralStringRef("Cat") && t.at(1).end == LiteralStringRef("Dah"));
ASSERT(t.size() == 2 && (*t.begin()).keys.begin == LiteralStringRef("Bah") &&
(*t.begin()).keys.end == LiteralStringRef("But"));
ASSERT(t.at(1).keys.begin == LiteralStringRef("Cat") && t.at(1).keys.end == LiteralStringRef("Dah"));
return Void();
}

View File

@ -2559,6 +2559,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 );
ProtocolVersion protocolVersion = BinaryReader::fromStringRef<ProtocolVersion>( fProtocolVersions.get()[idx].value, Unversioned() );
TLogSpillType logSpillType = BinaryReader::fromStringRef<TLogSpillType>( fTLogSpillTypes.get()[idx].value, AssumeVersion(protocolVersion) );
@ -2767,6 +2772,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 );
stopAllTLogs(self, recruited.id());

View File

@ -497,6 +497,34 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
}
}
ACTOR static Future<Void> pushResetChecker( Reference<ConnectionResetInfo> self, NetworkAddress addr ) {
self->slowReplies = 0;
self->fastReplies = 0;
wait(delay(SERVER_KNOBS->PUSH_STATS_INTERVAL));
TraceEvent("SlowPushStats").detail("PeerAddress", addr).detail("SlowReplies", self->slowReplies).detail("FastReplies", self->fastReplies);
if(self->slowReplies >= SERVER_KNOBS->PUSH_STATS_SLOW_AMOUNT && self->slowReplies/double(self->slowReplies+self->fastReplies) >= SERVER_KNOBS->PUSH_STATS_SLOW_RATIO) {
FlowTransport::transport().resetConnection(addr);
self->lastReset = now();
}
return Void();
}
ACTOR static Future<TLogCommitReply> recordPushMetrics( Reference<ConnectionResetInfo> self, NetworkAddress addr, Future<TLogCommitReply> in ) {
state double startTime = now();
TLogCommitReply t = wait(in);
if(now()-self->lastReset > SERVER_KNOBS->PUSH_RESET_INTERVAL) {
if(now()-startTime > SERVER_KNOBS->PUSH_MAX_LATENCY) {
if(self->resetCheck.isReady()) {
self->resetCheck = pushResetChecker(self, addr);
}
self->slowReplies++;
} else {
self->fastReplies++;
}
}
return t;
}
Future<Version> push(Version prevVersion, Version version, Version knownCommittedVersion,
Version minKnownCommittedVersion, LogPushData& data, Optional<UID> debugID) final {
// FIXME: Randomize request order as in LegacyLogSystem?
@ -505,10 +533,15 @@ struct TagPartitionedLogSystem : ILogSystem, ReferenceCounted<TagPartitionedLogS
int location = 0;
for(auto& it : tLogs) {
if(it->isLocal && it->logServers.size()) {
if(it->connectionResetTrackers.size() == 0) {
for(int i = 0; i < it->logServers.size(); i++) {
it->connectionResetTrackers.push_back(Reference<ConnectionResetInfo>( new ConnectionResetInfo() ));
}
}
vector<Future<Void>> tLogCommitResults;
for(int loc=0; loc< it->logServers.size(); loc++) {
Standalone<StringRef> msg = data.getMessages(location);
allReplies.push_back( it->logServers[loc]->get().interf().commit.getReply( TLogCommitRequest( msg.arena(), prevVersion, version, knownCommittedVersion, minKnownCommittedVersion, msg, debugID ), TaskPriority::ProxyTLogCommitReply ) );
allReplies.push_back( recordPushMetrics( it->connectionResetTrackers[loc], it->logServers[loc]->get().interf().address(), it->logServers[loc]->get().interf().commit.getReply( TLogCommitRequest( msg.arena(), prevVersion, version, knownCommittedVersion, minKnownCommittedVersion, msg, debugID ), TaskPriority::ProxyTLogCommitReply ) ) );
Future<Void> commitSuccess = success(allReplies.back());
addActor.get().send(commitSuccess);
tLogCommitResults.push_back(commitSuccess);

View File

@ -92,7 +92,18 @@ std::string toString(LogicalPageID id) {
if (id == invalidLogicalPageID) {
return "LogicalPageID{invalid}";
}
return format("LogicalPageID{%" PRId64 "}", id);
return format("LogicalPageID{%u}", id);
}
std::string toString(Version v) {
if (v == invalidVersion) {
return "invalidVersion";
}
return format("@%" PRId64, v);
}
std::string toString(bool b) {
return b ? "true" : "false";
}
template <typename T>
@ -136,6 +147,11 @@ std::string toString(const Optional<T>& o) {
return "<not present>";
}
template <typename F, typename S>
std::string toString(const std::pair<F, S>& o) {
return format("{%s, %s}", toString(o.first).c_str(), toString(o.second).c_str());
}
// A FIFO queue of T stored as a linked list of pages.
// Main operations are pop(), pushBack(), pushFront(), and flush().
//
@ -765,6 +781,8 @@ struct RedwoodMetrics {
unsigned int lazyClearRequeueExt;
unsigned int lazyClearFree;
unsigned int lazyClearFreeExt;
unsigned int forceUpdate;
unsigned int detachChild;
double buildStoredPct;
double buildFillPct;
unsigned int buildItemCount;
@ -797,6 +815,12 @@ struct RedwoodMetrics {
unsigned int btreeLeafPreload;
unsigned int btreeLeafPreloadExt;
// Return number of pages read or written, from cache or disk
unsigned int pageOps() const {
// All page reads are either a cache hit, probe hit, or a disk read
return pagerDiskWrite + pagerDiskRead + pagerCacheHit + pagerProbeHit;
}
double startTime;
Level& level(unsigned int level) {
@ -807,9 +831,9 @@ struct RedwoodMetrics {
return levels[level - 1];
}
// This will populate a trace event and/or a string with Redwood metrics. The string is a
// reasonably well formatted page of information
void getFields(TraceEvent* e, std::string* s = nullptr) {
// This will populate a trace event and/or a string with Redwood metrics.
// The string is a reasonably well formatted page of information
void getFields(TraceEvent* e, std::string* s = nullptr, bool skipZeroes = false) {
std::pair<const char*, unsigned int> metrics[] = { { "BTreePreload", btreeLeafPreload },
{ "BTreePreloadExt", btreeLeafPreloadExt },
{ "", 0 },
@ -837,21 +861,26 @@ struct RedwoodMetrics {
{ "PagerRemapCopy", pagerRemapCopy },
{ "PagerRemapSkip", pagerRemapSkip } };
double elapsed = now() - startTime;
for (auto& m : metrics) {
if (*m.first == '\0') {
if (s != nullptr) {
*s += "\n";
}
} else {
if (s != nullptr) {
*s += format("%-15s %-8u %8u/s ", m.first, m.second, int(m.second / elapsed));
}
if (e != nullptr) {
if (e != nullptr) {
for (auto& m : metrics) {
char c = m.first[0];
if(c != 0 && (!skipZeroes || m.second != 0) ) {
e->detail(m.first, m.second);
}
}
}
if(s != nullptr) {
for (auto& m : metrics) {
if (*m.first == '\0') {
*s += "\n";
} else if(!skipZeroes || m.second != 0) {
*s += format("%-15s %-8u %8u/s ", m.first, m.second, int(m.second / elapsed));
}
}
}
for (int i = 0; i < btreeLevels; ++i) {
auto& level = levels[i];
std::pair<const char*, unsigned int> metrics[] = {
@ -869,37 +898,44 @@ struct RedwoodMetrics {
{ "LazyClear", level.lazyClearFree },
{ "LazyClearExt", level.lazyClearFreeExt },
{ "", 0 },
{ "ForceUpdate", level.forceUpdate },
{ "DetachChild", level.detachChild },
{ "", 0 },
{ "-BldAvgCount", level.pageBuild ? level.buildItemCount / level.pageBuild : 0 },
{ "-BldAvgFillPct", level.pageBuild ? level.buildFillPct / level.pageBuild * 100 : 0 },
{ "-BldAvgStoredPct", level.pageBuild ? level.buildStoredPct / level.pageBuild * 100 : 0 },
{ "", 0 },
{ "-ModAvgCount", level.pageModify ? level.modifyItemCount / level.pageModify : 0 },
{ "-ModAvgFillPct", level.pageModify ? level.modifyFillPct / level.pageModify * 100 : 0 },
{ "-ModAvgStoredPct", level.pageModify ? level.modifyStoredPct / level.pageModify * 100 : 0 }
{ "-ModAvgStoredPct", level.pageModify ? level.modifyStoredPct / level.pageModify * 100 : 0 },
{ "", 0 },
};
if(e != nullptr) {
for (auto& m : metrics) {
char c = m.first[0];
if(c != 0 && (!skipZeroes || m.second != 0) ) {
e->detail(format("L%d%s", i + 1, m.first + (c == '-' ? 1 : 0)), m.second);
}
}
}
if (s != nullptr) {
*s += format("\nLevel %d\n\t", i + 1);
}
for (auto& m : metrics) {
const char* name = m.first;
bool rate = elapsed != 0;
if (*name == '-') {
++name;
rate = false;
}
if (*name == '\0') {
if (s != nullptr) {
for (auto& m : metrics) {
const char* name = m.first;
bool rate = elapsed != 0;
if (*name == '-') {
++name;
rate = false;
}
if (*name == '\0') {
*s += "\n\t";
}
} else {
if (s != nullptr) {
} else if(!skipZeroes || m.second != 0) {
*s += format("%-15s %8u %8u/s ", name, m.second, rate ? int(m.second / elapsed) : 0);
}
if (e != nullptr) {
e->detail(format("L%d%s", i + 1, name), m.second);
}
}
}
}
@ -1124,22 +1160,32 @@ public:
};
struct RemappedPage {
RemappedPage() : version(invalidVersion) {}
RemappedPage(Version v, LogicalPageID o, LogicalPageID n) : version(v), originalPageID(o), newPageID(n) {}
enum Type { NONE = 'N', REMAP = 'R', FREE = 'F', DETACH = 'D' };
RemappedPage(Version v = invalidVersion, LogicalPageID o = invalidLogicalPageID, LogicalPageID n = invalidLogicalPageID) : version(v), originalPageID(o), newPageID(n) {}
Version version;
LogicalPageID originalPageID;
LogicalPageID newPageID;
bool isFree() const {
return newPageID == invalidLogicalPageID;
static Type getTypeOf(LogicalPageID newPageID) {
if(newPageID == invalidLogicalPageID) {
return FREE;
}
if(newPageID == 0) {
return DETACH;
}
return REMAP;
}
Type getType() const {
return getTypeOf(newPageID);
}
bool operator<(const RemappedPage& rhs) { return version < rhs.version; }
std::string toString() const {
return format("RemappedPage(%s -> %s @%" PRId64 "}", ::toString(originalPageID).c_str(),
::toString(newPageID).c_str(), version);
return format("RemappedPage(%c: %s -> %s %s}", getType(), ::toString(originalPageID).c_str(),
::toString(newPageID).c_str(), ::toString(version).c_str());
}
};
@ -1484,6 +1530,35 @@ public:
}
}
LogicalPageID detachRemappedPage(LogicalPageID pageID, Version v) override {
auto i = remappedPages.find(pageID);
if(i == remappedPages.end()) {
// Page is not remapped
return invalidLogicalPageID;
}
// Get the page that id was most recently remapped to
auto iLast = i->second.rbegin();
LogicalPageID newID = iLast->second;
ASSERT(RemappedPage::getTypeOf(newID) == RemappedPage::REMAP);
// If the last change remap was also at v then change the remap to a delete, as it's essentially
// the same as the original page being deleted at that version and newID being used from then on.
if(iLast->first == v) {
debug_printf("DWALPager(%s) op=detachDelete originalID=%s newID=%s @%" PRId64 " oldestVersion=%" PRId64 "\n", filename.c_str(),
toString(pageID).c_str(), toString(newID).c_str(), v, pLastCommittedHeader->oldestVersion);
iLast->second = invalidLogicalPageID;
remapQueue.pushBack(RemappedPage{ v, pageID, invalidLogicalPageID });
} else {
debug_printf("DWALPager(%s) op=detach originalID=%s newID=%s @%" PRId64 " oldestVersion=%" PRId64 "\n", filename.c_str(),
toString(pageID).c_str(), toString(newID).c_str(), v, pLastCommittedHeader->oldestVersion);
// Mark id as converted to its last remapped location as of v
i->second[v] = 0;
remapQueue.pushBack(RemappedPage{ v, pageID, 0 });
}
return newID;
}
void freePage(LogicalPageID pageID, Version v) override {
// If pageID has been remapped, then it can't be freed until all existing remaps for that page have been undone,
// so queue it for later deletion
@ -1588,13 +1663,13 @@ public:
auto j = i->second.upper_bound(v);
if (j != i->second.begin()) {
--j;
debug_printf("DWALPager(%s) read %s @%" PRId64 " -> %s\n", filename.c_str(), toString(pageID).c_str(),
debug_printf("DWALPager(%s) op=readAtVersionRemapped %s @%" PRId64 " -> %s\n", filename.c_str(), toString(pageID).c_str(),
v, toString(j->second).c_str());
pageID = j->second;
ASSERT(pageID != invalidLogicalPageID);
}
} else {
debug_printf("DWALPager(%s) read %s @%" PRId64 " (not remapped)\n", filename.c_str(),
debug_printf("DWALPager(%s) op=readAtVersionNotRemapped %s @%" PRId64 " (not remapped)\n", filename.c_str(),
toString(pageID).c_str(), v);
}
@ -1623,29 +1698,126 @@ public:
return std::min(pLastCommittedHeader->oldestVersion, snapshots.front().version);
}
ACTOR static Future<Void> remapCopyAndFree(DWALPager* self, RemappedPage p, VersionToPageMapT *m, VersionToPageMapT::iterator i) {
debug_printf("DWALPager(%s) remapCleanup copyAndFree %s\n", self->filename.c_str(), p.toString().c_str());
ACTOR static Future<Void> removeRemapEntry(DWALPager* self, RemappedPage p, Version oldestRetainedVersion) {
// Get iterator to the versioned page map entry for the original page
state PageToVersionedMapT::iterator iPageMapPair = self->remappedPages.find(p.originalPageID);
// The iterator must be valid and not empty and its first page map entry must match p's version
ASSERT(iPageMapPair != self->remappedPages.end());
ASSERT(!iPageMapPair->second.empty());
state VersionToPageMapT::iterator iVersionPagePair = iPageMapPair->second.find(p.version);
ASSERT(iVersionPagePair != iPageMapPair->second.end());
// Read the data from the page that the original was mapped to
Reference<IPage> data = wait(self->readPage(p.newPageID, false));
RemappedPage::Type firstType = p.getType();
state RemappedPage::Type secondType;
bool secondAfterOldestRetainedVersion = false;
state bool deleteAtSameVersion = false;
if(p.newPageID == iVersionPagePair->second) {
auto nextEntry = iVersionPagePair;
++nextEntry;
if(nextEntry == iPageMapPair->second.end()) {
secondType = RemappedPage::NONE;
} else {
secondType = RemappedPage::getTypeOf(nextEntry->second);
secondAfterOldestRetainedVersion = nextEntry->first >= oldestRetainedVersion;
}
} else {
ASSERT(iVersionPagePair->second == invalidLogicalPageID);
secondType = RemappedPage::FREE;
deleteAtSameVersion = true;
}
ASSERT(firstType == RemappedPage::REMAP || secondType == RemappedPage::NONE);
// Write the data to the original page so it can be read using its original pageID
self->updatePage(p.originalPageID, data);
++g_redwoodMetrics.pagerRemapCopy;
// Scenarios and actions to take:
//
// The first letter (firstType) is the type of the entry just popped from the remap queue.
// The second letter (secondType) is the type of the next item in the queue for the same
// original page ID, if present. If not present, secondType will be NONE.
//
// Since the next item can be arbitrarily ahead in the queue, secondType is determined by
// looking at the remappedPages structure.
//
// R == Remap F == Free D == Detach | == oldestRetaineedVersion
//
// R R | free new ID
// R F | free new ID if R and D are at different versions
// R D | do nothing
// R | R copy new to original ID, free new ID
// R | F copy new to original ID, free new ID
// R | D copy new to original ID
// R | copy new to original ID, free new ID
// F | free original ID
// D | free original ID
//
// Note that
//
// Special case: Page is detached while it is being read in remapCopyAndFree()
// Initial state: R |
// Start remapCopyAndFree(), intending to copy new, ID to originalID and free newID
// New state: R | D
// Read of newID completes.
// Copy new contents over original, do NOT free new ID
// Later popped state: D |
// free original ID
//
state bool freeNewID = (firstType == RemappedPage::REMAP && secondType != RemappedPage::DETACH && !deleteAtSameVersion);
state bool copyNewToOriginal = (firstType == RemappedPage::REMAP && (secondAfterOldestRetainedVersion || secondType == RemappedPage::NONE));
state bool freeOriginalID = (firstType == RemappedPage::FREE || firstType == RemappedPage::DETACH);
// Now that the page data has been copied to the original page, the versioned page map entry is no longer
// needed and the new page ID can be freed as of the next commit.
m->erase(i);
self->freeUnmappedPage(p.newPageID, 0);
++g_redwoodMetrics.pagerRemapFree;
debug_printf("DWALPager(%s) remapCleanup %s secondType=%c mapEntry=%s oldestRetainedVersion=%" PRId64 " \n",
self->filename.c_str(), p.toString().c_str(), secondType, ::toString(*iVersionPagePair).c_str(), oldestRetainedVersion);
if(copyNewToOriginal) {
debug_printf("DWALPager(%s) remapCleanup copy %s\n", self->filename.c_str(), p.toString().c_str());
// Read the data from the page that the original was mapped to
Reference<IPage> data = wait(self->readPage(p.newPageID, false, true));
// Write the data to the original page so it can be read using its original pageID
self->updatePage(p.originalPageID, data);
++g_redwoodMetrics.pagerRemapCopy;
} else if (firstType == RemappedPage::REMAP) {
++g_redwoodMetrics.pagerRemapSkip;
}
// Now that the page contents have been copied to the original page, if the corresponding map entry
// represented the remap and there wasn't a delete later in the queue at p for the same version then
// erase the entry.
if(!deleteAtSameVersion) {
debug_printf("DWALPager(%s) remapCleanup deleting map entry %s\n", self->filename.c_str(), p.toString().c_str());
// Erase the entry and set iVersionPagePair to the next entry or end
iVersionPagePair = iPageMapPair->second.erase(iVersionPagePair);
// If the map is now empty, delete it
if(iPageMapPair->second.empty()) {
debug_printf("DWALPager(%s) remapCleanup deleting empty map %s\n", self->filename.c_str(), p.toString().c_str());
self->remappedPages.erase(iPageMapPair);
} else if(freeNewID && secondType == RemappedPage::NONE && iVersionPagePair != iPageMapPair->second.end() && RemappedPage::getTypeOf(iVersionPagePair->second) == RemappedPage::DETACH) {
// If we intend to free the new ID and there was no map entry, one could have been added during the wait above.
// If so, and if it was a detach operation, then we can't free the new page ID as its lifetime will be managed
// by the client starting at some later version.
freeNewID = false;
}
}
if(freeNewID) {
debug_printf("DWALPager(%s) remapCleanup freeNew %s\n", self->filename.c_str(), p.toString().c_str());
self->freeUnmappedPage(p.newPageID, 0);
++g_redwoodMetrics.pagerRemapFree;
}
if(freeOriginalID) {
debug_printf("DWALPager(%s) remapCleanup freeOriginal %s\n", self->filename.c_str(), p.toString().c_str());
self->freeUnmappedPage(p.originalPageID, 0);
++g_redwoodMetrics.pagerRemapFree;
}
return Void();
}
ACTOR static Future<Void> remapCleanup(DWALPager* self) {
state ActorCollection copies(true);
state ActorCollection tasks(true);
state Promise<Void> signal;
copies.add(signal.getFuture());
tasks.add(signal.getFuture());
self->remapCleanupStop = false;
@ -1654,8 +1826,7 @@ public:
state Version oldestRetainedVersion = self->effectiveOldestVersion();
// Cutoff is the version we can pop to
state RemappedPage cutoff;
cutoff.version = oldestRetainedVersion - self->remapCleanupWindow;
state RemappedPage cutoff(oldestRetainedVersion - self->remapCleanupWindow);
// Minimum version we must pop to before obeying stop command.
state Version minStopVersion = cutoff.version - (self->remapCleanupWindow * SERVER_KNOBS->REDWOOD_REMAP_CLEANUP_LAG);
@ -1663,46 +1834,15 @@ public:
loop {
state Optional<RemappedPage> p = wait(self->remapQueue.pop(cutoff));
debug_printf("DWALPager(%s) remapCleanup popped %s\n", self->filename.c_str(), ::toString(p).c_str());
// Stop if we have reached the cutoff version, which is the start of the cleanup coalescing window
if (!p.present()) {
break;
}
// Get iterator to the versioned page map entry for the original page
auto iPageMapPair = self->remappedPages.find(p.get().originalPageID);
// The iterator must be valid and not empty and its first page map entry must match p's version
ASSERT(iPageMapPair != self->remappedPages.end());
ASSERT(!iPageMapPair->second.empty());
auto iVersionPagePair = iPageMapPair->second.begin();
ASSERT(iVersionPagePair->first == p.get().version);
// If this is a free page entry then free the original page ID
if(p.get().isFree()) {
debug_printf("DWALPager(%s) remapCleanup free %s\n", self->filename.c_str(),
p.get().toString().c_str());
self->freeUnmappedPage(p.get().originalPageID, 0);
++g_redwoodMetrics.pagerRemapFree;
// There can't be any more entries in the page map after this one so verify that
// the map size is 1 and erase the map for p's original page ID.
ASSERT(iPageMapPair->second.size() == 1);
self->remappedPages.erase(iPageMapPair);
}
else {
// If there is no next page map entry or there is but it is after the oldest retained version
// then p must be copied to unmap it.
auto iNextVersionPagePair = iVersionPagePair;
++iNextVersionPagePair;
if(iNextVersionPagePair == iPageMapPair->second.end() || iNextVersionPagePair->first > oldestRetainedVersion) {
// Copy the remapped page to the original so it can be freed.
copies.add(remapCopyAndFree(self, p.get(), &iPageMapPair->second, iVersionPagePair));
}
else {
debug_printf("DWALPager(%s) remapCleanup skipAndFree %s\n", self->filename.c_str(), p.get().toString().c_str());
self->freeUnmappedPage(p.get().newPageID, 0);
++g_redwoodMetrics.pagerRemapFree;
++g_redwoodMetrics.pagerRemapSkip;
iPageMapPair->second.erase(iVersionPagePair);
}
Future<Void> task = removeRemapEntry(self, p.get(), oldestRetainedVersion);
if(!task.isReady()) {
tasks.add(task);
}
// If the stop flag is set and we've reached the minimum stop version according the the allowed lag then stop.
@ -1713,7 +1853,7 @@ public:
debug_printf("DWALPager(%s) remapCleanup stopped (stop=%d)\n", self->filename.c_str(), self->remapCleanupStop);
signal.send(Void());
wait(copies.getResult());
wait(tasks.getResult());
return Void();
}
@ -1889,8 +2029,7 @@ public:
Future<int64_t> getUserPageCount() override {
return map(getUserPageCount_cleanup(this), [=](Void) {
int64_t userPages = pHeader->pageCount - 2 - freeList.numPages - freeList.numEntries -
delayedFreeList.numPages - delayedFreeList.numEntries - remapQueue.numPages
- remapQueue.numEntries;
delayedFreeList.numPages - delayedFreeList.numEntries - remapQueue.numPages;
debug_printf("DWALPager(%s) userPages=%" PRId64 " totalPageCount=%" PRId64 " freeQueuePages=%" PRId64
" freeQueueCount=%" PRId64 " delayedFreeQueuePages=%" PRId64 " delayedFreeQueueCount=%" PRId64
@ -2871,6 +3010,38 @@ public:
typedef FIFOQueue<LazyClearQueueEntry> LazyClearQueueT;
struct ParentInfo {
ParentInfo() {
count = 0;
bits = 0;
}
void clear() {
count = 0;
bits = 0;
}
static uint32_t mask(LogicalPageID id) {
return 1 << (id & 31);
}
void pageUpdated(LogicalPageID child) {
auto m = mask(child);
if((bits & m) == 0) {
bits |= m;
++count;
}
}
bool maybeUpdated(LogicalPageID child) {
return (mask(child) & bits) != 0;
}
uint32_t bits;
int count;
};
typedef std::unordered_map<LogicalPageID, ParentInfo> ParentInfoMapT;
#pragma pack(push, 1)
struct MetaKey {
static constexpr int FORMAT_VERSION = 8;
@ -2924,8 +3095,8 @@ public:
// durable once the following call to commit() returns
void set(KeyValueRef keyValue) {
++g_redwoodMetrics.opSet;
++g_redwoodMetrics.opSetKeyBytes += keyValue.key.size();
++g_redwoodMetrics.opSetValueBytes += keyValue.value.size();
g_redwoodMetrics.opSetKeyBytes += keyValue.key.size();
g_redwoodMetrics.opSetValueBytes += keyValue.value.size();
m_pBuffer->insert(keyValue.key).mutation().setBoundaryValue(m_pBuffer->copyToArena(keyValue.value));
}
@ -3025,7 +3196,7 @@ public:
// If this page is height 2, then the children are leaves so free them directly
if (btPage.height == 2) {
debug_printf("LazyClear: freeing child %s\n", toString(btChildPageID).c_str());
self->freeBtreePage(btChildPageID, v);
self->freeBTreePage(btChildPageID, v);
freedPages += btChildPageID.size();
metrics.lazyClearFree += 1;
metrics.lazyClearFreeExt += (btChildPageID.size() - 1);
@ -3044,7 +3215,7 @@ public:
// Free the page, now that its children have either been freed or queued
debug_printf("LazyClear: freeing queue entry %s\n", toString(entry.pageID).c_str());
self->freeBtreePage(entry.pageID, v);
self->freeBTreePage(entry.pageID, v);
freedPages += entry.pageID.size();
metrics.lazyClearFree += 1;
metrics.lazyClearFreeExt += entry.pageID.size() - 1;
@ -3149,7 +3320,7 @@ public:
return commit_impl(this);
}
ACTOR static Future<Void> destroyAndCheckSanity_impl(VersionedBTree* self) {
ACTOR static Future<Void> clearAllAndCheckSanity_impl(VersionedBTree* self) {
ASSERT(g_network->isSimulated());
debug_printf("Clearing tree.\n");
@ -3194,7 +3365,7 @@ public:
return Void();
}
Future<Void> destroyAndCheckSanity() { return destroyAndCheckSanity_impl(this); }
Future<Void> clearAllAndCheckSanity() { return clearAllAndCheckSanity_impl(this); }
private:
// Represents a change to a single key - set, clear, or atomic op
@ -3415,6 +3586,8 @@ private:
Future<Void> m_init;
std::string m_name;
int m_blockSize;
std::unordered_map<LogicalPageID, ParentInfo> parents;
ParentInfoMapT childUpdateTracker;
// MetaKey changes size so allocate space for it to expand into
union {
@ -3606,7 +3779,7 @@ private:
// must be rewritten anyway to count for the change in child count or child links.
// Free the old IDs, but only once (before the first output record is added).
if (records.empty()) {
self->freeBtreePage(previousID, v);
self->freeBTreePage(previousID, v);
}
for (p = 0; p < pages.size(); ++p) {
LogicalPageID id = wait(self->m_pager->newPageID());
@ -3774,7 +3947,7 @@ private:
}
}
void freeBtreePage(BTreePageIDRef btPageID, Version v) {
void freeBTreePage(BTreePageIDRef btPageID, Version v) {
// Free individual pages at v
for (LogicalPageID id : btPageID) {
m_pager->freePage(id, v);
@ -3783,7 +3956,7 @@ private:
// Write new version of pageID at version v using page as its data.
// Attempts to reuse original id(s) in btPageID, returns BTreePageID.
ACTOR static Future<BTreePageIDRef> updateBtreePage(VersionedBTree* self, BTreePageIDRef oldID, Arena* arena,
ACTOR static Future<BTreePageIDRef> updateBTreePage(VersionedBTree* self, BTreePageIDRef oldID, Arena* arena,
Reference<IPage> page, Version writeVersion) {
state BTreePageIDRef newID;
newID.resize(*arena, oldID.size());
@ -3881,19 +4054,23 @@ private:
// If the last record in the range has a null link then this will be null.
const RedwoodRecordRef* expectedUpperBound;
bool inPlaceUpdate;
// CommitSubtree will call one of the following three functions based on its exit path
// Subtree was cleared.
void cleared() {
inPlaceUpdate = false;
childrenChanged = true;
expectedUpperBound = nullptr;
}
// Page was updated in-place through edits and written to maybeNewID
void updatedInPlace(BTreePageIDRef maybeNewID, BTreePage* btPage, int capacity) {
inPlaceUpdate = true;
auto& metrics = g_redwoodMetrics.level(btPage->height);
metrics.pageModify += 1;
metrics.pageModify += (maybeNewID.size() - 1);
metrics.pageModifyExt += (maybeNewID.size() - 1);
metrics.modifyFillPct += (double)btPage->size() / capacity;
metrics.modifyStoredPct += (double)btPage->kvBytes / capacity;
metrics.modifyItemCount += btPage->tree().numItems;
@ -3915,6 +4092,7 @@ private:
// writePages() was used to build 1 or more replacement pages.
void rebuilt(Standalone<VectorRef<RedwoodRecordRef>> newRecords) {
inPlaceUpdate = false;
newLinks = newRecords;
childrenChanged = true;
@ -3955,14 +4133,15 @@ private:
struct InternalPageModifier {
InternalPageModifier() {}
InternalPageModifier(BTreePage* p, BTreePage::BinaryTree::Mirror* m, bool updating)
: btPage(p), m(m), updating(updating), changesMade(false) {}
InternalPageModifier(BTreePage* p, BTreePage::BinaryTree::Mirror* m, bool updating, ParentInfo *parentInfo)
: btPage(p), m(m), updating(updating), changesMade(false), parentInfo(parentInfo) {}
bool updating;
BTreePage* btPage;
BTreePage::BinaryTree::Mirror* m;
Standalone<VectorRef<RedwoodRecordRef>> rebuild;
bool changesMade;
ParentInfo *parentInfo;
bool empty() const {
if (updating) {
@ -4058,6 +4237,13 @@ private:
// endpoint.
changesMade = true;
} else {
if(u.inPlaceUpdate) {
for(auto id : u.decodeLowerBound->getChildPage()) {
parentInfo->pageUpdated(id);
}
}
keep(u.cBegin, u.cEnd);
}
@ -4229,7 +4415,7 @@ private:
debug_printf("%s Inserted %s [mutation, boundary start]\n", context.c_str(),
rec.toString().c_str());
} else {
debug_printf("%s Inserted failed for %s [mutation, boundary start]\n", context.c_str(),
debug_printf("%s Insert failed for %s [mutation, boundary start]\n", context.c_str(),
rec.toString().c_str());
switchToLinearMerge();
}
@ -4342,12 +4528,12 @@ private:
// If the tree is now empty, delete the page
if (deltaTree.numItems == 0) {
update->cleared();
self->freeBtreePage(rootID, writeVersion);
self->freeBTreePage(rootID, writeVersion);
debug_printf("%s Page updates cleared all entries, returning %s\n", context.c_str(),
toString(*update).c_str());
} else {
// Otherwise update it.
BTreePageIDRef newID = wait(self->updateBtreePage(self, rootID, &update->newLinks.arena(),
BTreePageIDRef newID = wait(self->updateBTreePage(self, rootID, &update->newLinks.arena(),
page.castTo<IPage>(), writeVersion));
update->updatedInPlace(newID, btPage, newID.size() * self->m_blockSize);
@ -4360,7 +4546,7 @@ private:
// If everything in the page was deleted then this page should be deleted as of the new version
if (merged.empty()) {
update->cleared();
self->freeBtreePage(rootID, writeVersion);
self->freeBTreePage(rootID, writeVersion);
debug_printf("%s All leaf page contents were cleared, returning %s\n", context.c_str(),
toString(*update).c_str());
@ -4514,7 +4700,7 @@ private:
if (btPage->height == 2) {
debug_printf("%s: freeing child page in cleared subtree range: %s\n",
context.c_str(), ::toString(rec.getChildPage()).c_str());
self->freeBtreePage(rec.getChildPage(), writeVersion);
self->freeBTreePage(rec.getChildPage(), writeVersion);
} else {
debug_printf("%s: queuing subtree deletion cleared subtree range: %s\n",
context.c_str(), ::toString(rec.getChildPage()).c_str());
@ -4550,7 +4736,10 @@ private:
wait(waitForAll(recursions));
debug_printf("%s Recursions done, processing slice updates.\n", context.c_str());
state InternalPageModifier m(btPage, cursor.mirror, tryToUpdate);
// Note: parentInfo could be invalid after a wait and must be re-initialized.
// All uses below occur before waits so no reinitialization is done.
state ParentInfo *parentInfo = &self->childUpdateTracker[rootID.front()];
state InternalPageModifier m(btPage, cursor.mirror, tryToUpdate, parentInfo);
// Apply the possible changes for each subtree range recursed to, except the last one.
// For each range, the expected next record, if any, is checked against the first boundary
@ -4568,25 +4757,103 @@ private:
context.c_str(), m.changesMade, update->toString().c_str());
m.applyUpdate(*slices.back(), m.changesMade ? update->subtreeUpperBound : update->decodeUpperBound);
state bool detachChildren = (parentInfo->count > 2);
state bool forceUpdate = false;
if(!m.changesMade && detachChildren) {
debug_printf("%s Internal page forced rewrite because at least %d children have been updated in-place.\n", context.c_str(), parentInfo->count);
forceUpdate = true;
if(!m.updating) {
page = self->cloneForUpdate(page);
cursor = getCursor(page);
btPage = (BTreePage*)page->begin();
m.btPage = btPage;
m.m = cursor.mirror;
m.updating = true;
}
++g_redwoodMetrics.level(btPage->height).forceUpdate;
}
// If page contents have changed
if (m.changesMade) {
if ((m.empty())) {
if (m.changesMade || forceUpdate) {
if (m.empty()) {
update->cleared();
debug_printf("%s All internal page children were deleted so deleting this page too, returning %s\n",
context.c_str(), toString(*update).c_str());
self->freeBtreePage(rootID, writeVersion);
self->freeBTreePage(rootID, writeVersion);
self->childUpdateTracker.erase(rootID.front());
} else {
if (m.updating) {
// Page was updated in place
BTreePageIDRef newID = wait(self->updateBtreePage(self, rootID, &update->newLinks.arena(),
// Page was updated in place (or being forced to be updated in place to update child page ids)
debug_printf("%s Internal page modified in-place tryUpdate=%d forceUpdate=%d detachChildren=%d\n", context.c_str(), tryToUpdate, forceUpdate, detachChildren);
if(detachChildren) {
int detached = 0;
cursor.moveFirst();
auto &stats = g_redwoodMetrics.level(btPage->height);
while(cursor.valid()) {
if(cursor.get().value.present()) {
for(auto &p : cursor.get().getChildPage()) {
if(parentInfo->maybeUpdated(p)) {
LogicalPageID newID = self->m_pager->detachRemappedPage(p, writeVersion);
if(newID != invalidLogicalPageID) {
debug_printf("%s Detach updated %u -> %u\n", context.c_str(), p, newID);
p = newID;
++stats.detachChild;
++detached;
}
}
}
}
cursor.moveNext();
}
parentInfo->clear();
if(forceUpdate && detached == 0) {
debug_printf("%s No children detached during forced update, returning %s\n", context.c_str(), toString(*update).c_str());
return Void();
}
}
BTreePageIDRef newID = wait(self->updateBTreePage(self, rootID, &update->newLinks.arena(),
page.castTo<IPage>(), writeVersion));
debug_printf(
"%s commitSubtree(): Internal page updated in-place at version %s, new contents: %s\n", context.c_str(), toString(writeVersion).c_str(),
btPage->toString(false, newID, snapshot->getVersion(), update->decodeLowerBound, update->decodeUpperBound)
.c_str());
update->updatedInPlace(newID, btPage, newID.size() * self->m_blockSize);
debug_printf("%s Internal page updated in-place, returning %s\n", context.c_str(),
toString(*update).c_str());
} else {
// Page was rebuilt, possibly split.
debug_printf("%s Internal page modified, creating replacements.\n", context.c_str());
debug_printf("%s Internal page could not be modified, rebuilding replacement(s).\n", context.c_str());
if(detachChildren) {
auto &stats = g_redwoodMetrics.level(btPage->height);
for(auto &rec : m.rebuild) {
if(rec.value.present()) {
BTreePageIDRef oldPages = rec.getChildPage();
BTreePageIDRef newPages;
for(int i = 0; i < oldPages.size(); ++i) {
LogicalPageID p = oldPages[i];
if(parentInfo->maybeUpdated(p)) {
LogicalPageID newID = self->m_pager->detachRemappedPage(p, writeVersion);
if(newID != invalidLogicalPageID) {
// Rebuild record values reference original page memory so make a copy
if(newPages.empty()) {
newPages = BTreePageIDRef(m.rebuild.arena(), oldPages);
rec.setChildPage(newPages);
}
debug_printf("%s Detach updated %u -> %u\n", context.c_str(), p, newID);
newPages[i] = newID;
++stats.detachChild;
}
}
}
}
}
parentInfo->clear();
}
Standalone<VectorRef<RedwoodRecordRef>> newChildEntries =
wait(writePages(self, update->subtreeLowerBound, update->subtreeUpperBound, m.rebuild,
@ -4988,7 +5255,7 @@ public:
bool isValid() const { return valid; }
std::string toString() const {
std::string r;
std::string r = format("{ptr=%p %s ", this, ::toString(pager->getVersion()).c_str());
for (int i = 0; i < path.size(); ++i) {
r += format("[%d/%d: %s] ", i + 1, path.size(),
path[i].cursor.valid() ? path[i].cursor.get().toString(path[i].btPage->isLeaf()).c_str()
@ -4997,6 +5264,7 @@ public:
if (!valid) {
r += " (invalid) ";
}
r += "}";
return r;
}
@ -5017,6 +5285,8 @@ public:
const RedwoodRecordRef& upperBound) {
Reference<const IPage>& page = pages[id.front()];
if (page.isValid()) {
// The pager won't see this access so count it as a cache hit
++g_redwoodMetrics.pagerCacheHit;
path.push_back(arena, { (BTreePage*)page->begin(), getCursor(page) });
return Void();
}
@ -6090,11 +6360,15 @@ ACTOR Future<Void> verify(VersionedBTree* btree, FutureStream<Version> vStream,
committedVersions.pop_front();
}
// Choose a random committed version, or sometimes the latest (which could be ahead of the latest version
// from vStream)
v = (committedVersions.empty() || deterministicRandom()->random01() < 0.25)
? btree->getLastCommittedVersion()
: committedVersions[deterministicRandom()->randomInt(0, committedVersions.size())];
// Continue if the versions list is empty, which won't wait until it reaches the oldest readable
// btree version which will already be in vStream.
if(committedVersions.empty()) {
continue;
}
// Choose a random committed version.
v = committedVersions[deterministicRandom()->randomInt(0, committedVersions.size())];
debug_printf("Using committed version %" PRId64 "\n", v);
// Get a cursor at v so that v doesn't get expired between the possibly serial steps below.
state Reference<IStoreCursor> cur = btree->readAtVersion(v);
@ -6958,24 +7232,24 @@ TEST_CASE("!/redwood/correctness/btree") {
state int pageSize =
shortTest ? 200 : (deterministicRandom()->coinflip() ? 4096 : deterministicRandom()->randomInt(200, 400));
state int64_t targetPageOps = shortTest ? 50000 : 1000000;
state bool pagerMemoryOnly = shortTest && (deterministicRandom()->random01() < .01);
state int maxKeySize = deterministicRandom()->randomInt(1, pageSize * 2);
state int maxValueSize = randomSize(pageSize * 25);
state int maxCommitSize = shortTest ? 1000 : randomSize(std::min<int>((maxKeySize + maxValueSize) * 20000, 10e6));
state int mutationBytesTarget =
shortTest ? 100000 : randomSize(std::min<int>(maxCommitSize * 100, pageSize * 100000));
state double clearProbability = deterministicRandom()->random01() * .1;
state double clearSingleKeyProbability = deterministicRandom()->random01();
state double clearPostSetProbability = deterministicRandom()->random01() * .1;
state double coldStartProbability = pagerMemoryOnly ? 0 : (deterministicRandom()->random01() * 0.3);
state double advanceOldVersionProbability = deterministicRandom()->random01();
state double maxDuration = 60;
state int64_t cacheSizeBytes =
pagerMemoryOnly ? 2e9 : (BUGGIFY ? deterministicRandom()->randomInt(1, 10 * pageSize) : 0);
state Version versionIncrement = deterministicRandom()->randomInt64(1, 1e8);
state Version remapCleanupWindow = deterministicRandom()->randomInt64(0, versionIncrement * 50);
state int maxVerificationMapEntries = 300e3;
printf("\n");
printf("targetPageOps: %" PRId64 "\n", targetPageOps);
printf("pagerMemoryOnly: %d\n", pagerMemoryOnly);
printf("serialTest: %d\n", serialTest);
printf("shortTest: %d\n", shortTest);
@ -6983,7 +7257,6 @@ TEST_CASE("!/redwood/correctness/btree") {
printf("maxKeySize: %d\n", maxKeySize);
printf("maxValueSize: %d\n", maxValueSize);
printf("maxCommitSize: %d\n", maxCommitSize);
printf("mutationBytesTarget: %d\n", mutationBytesTarget);
printf("clearProbability: %f\n", clearProbability);
printf("clearSingleKeyProbability: %f\n", clearSingleKeyProbability);
printf("clearPostSetProbability: %f\n", clearPostSetProbability);
@ -6992,14 +7265,13 @@ TEST_CASE("!/redwood/correctness/btree") {
printf("cacheSizeBytes: %s\n", cacheSizeBytes == 0 ? "default" : format("%" PRId64, cacheSizeBytes).c_str());
printf("versionIncrement: %" PRId64 "\n", versionIncrement);
printf("remapCleanupWindow: %" PRId64 "\n", remapCleanupWindow);
printf("maxVerificationMapEntries: %d\n", maxVerificationMapEntries);
printf("\n");
printf("Deleting existing test data...\n");
deleteFile(pagerFile);
printf("Initializing...\n");
state double startTime = now();
pager = new DWALPager(pageSize, pagerFile, cacheSizeBytes, remapCleanupWindow, pagerMemoryOnly);
state VersionedBTree* btree = new VersionedBTree(pager, pagerFile);
wait(btree->init());
@ -7026,14 +7298,12 @@ TEST_CASE("!/redwood/correctness/btree") {
state PromiseStream<Version> committedVersions;
state Future<Void> verifyTask = verify(btree, committedVersions.getFuture(), &written, &errorCount, serialTest);
state Future<Void> randomTask = serialTest ? Void() : (randomReader(btree) || btree->getError());
committedVersions.send(lastVer);
state Future<Void> commit = Void();
state int64_t totalPageOps = 0;
while (mutationBytes.get() < mutationBytesTarget && (now() - startTime) < maxDuration) {
if (now() - startTime > 600) {
mutationBytesTarget = mutationBytes.get();
}
while (totalPageOps < targetPageOps && written.size() < maxVerificationMapEntries) {
// Sometimes increment the version
if (deterministicRandom()->random01() < 0.10) {
++version;
@ -7128,15 +7398,13 @@ TEST_CASE("!/redwood/correctness/btree") {
keys.insert(kv.key);
}
// Commit at end or after this commit's mutation bytes are reached
if (mutationBytes.get() >= mutationBytesTarget || mutationBytesThisCommit >= mutationBytesTargetThisCommit) {
// Commit after any limits for this commit or the total test are reached
if (totalPageOps >= targetPageOps || written.size() >= maxVerificationMapEntries || mutationBytesThisCommit >= mutationBytesTargetThisCommit) {
// Wait for previous commit to finish
wait(commit);
printf("Committed. Next commit %d bytes, %" PRId64
"/%d (%.2f%%) Stats: Insert %.2f MB/s ClearedKeys %.2f MB/s Total %.2f\n",
mutationBytesThisCommit, mutationBytes.get(), mutationBytesTarget,
(double)mutationBytes.get() / mutationBytesTarget * 100,
(keyBytesInserted.rate() + valueBytesInserted.rate()) / 1e6, keyBytesCleared.rate() / 1e6,
printf("Committed. Next commit %d bytes, %" PRId64 " bytes.", mutationBytesThisCommit, mutationBytes.get());
printf(" Stats: Insert %.2f MB/s ClearedKeys %.2f MB/s Total %.2f\n",
(keyBytesInserted.rate() + valueBytesInserted.rate()) / 1e6, keyBytesCleared.rate() / 1e6,
mutationBytes.rate() / 1e6);
Version v = version; // Avoid capture of version as a member of *this
@ -7149,8 +7417,14 @@ TEST_CASE("!/redwood/correctness/btree") {
btree->getOldestVersion() + 1));
}
commit = map(btree->commit(), [=](Void) {
commit = map(btree->commit(), [=,&ops=totalPageOps](Void) {
// Update pager ops before clearing metrics
ops += g_redwoodMetrics.pageOps();
printf("PageOps %" PRId64 "/%" PRId64 " (%.2f%%) VerificationMapEntries %d/%d (%.2f%%)\n",
ops, targetPageOps, ops * 100.0 / targetPageOps,
written.size(), maxVerificationMapEntries, written.size() * 100.0 / maxVerificationMapEntries);
printf("Committed:\n%s\n", g_redwoodMetrics.toString(true).c_str());
// Notify the background verifier that version is committed and therefore readable
committedVersions.send(v);
return Void();
@ -7200,6 +7474,7 @@ TEST_CASE("!/redwood/correctness/btree") {
committedVersions = PromiseStream<Version>();
verifyTask = verify(btree, committedVersions.getFuture(), &written, &errorCount, serialTest);
randomTask = randomReader(btree) || btree->getError();
committedVersions.send(v);
}
version += versionIncrement;
@ -7207,7 +7482,7 @@ TEST_CASE("!/redwood/correctness/btree") {
}
// Check for errors
if (errorCount != 0) throw internal_error();
ASSERT(errorCount == 0);
}
debug_printf("Waiting for outstanding commit\n");
@ -7218,11 +7493,18 @@ TEST_CASE("!/redwood/correctness/btree") {
wait(verifyTask);
// Check for errors
if (errorCount != 0) throw internal_error();
ASSERT(errorCount == 0);
wait(btree->destroyAndCheckSanity());
// Reopen pager and btree with a remap cleanup window of 0 to reclaim all old pages
state Future<Void> closedFuture = btree->onClosed();
btree->close();
wait(closedFuture);
btree = new VersionedBTree(new DWALPager(pageSize, pagerFile, cacheSizeBytes, 0), pagerFile);
wait(btree->init());
Future<Void> closedFuture = btree->onClosed();
wait(btree->clearAllAndCheckSanity());
closedFuture = btree->onClosed();
btree->close();
debug_printf("Closing.\n");
wait(closedFuture);
@ -7328,7 +7610,7 @@ TEST_CASE("!/redwood/performance/set") {
state int minValueSize = 100;
state int maxValueSize = 500;
state int minConsecutiveRun = 1;
state int maxConsecutiveRun = 10;
state int maxConsecutiveRun = 100000;
state char firstKeyChar = 'a';
state char lastKeyChar = 'm';
state Version remapCleanupWindow = SERVER_KNOBS->REDWOOD_REMAP_CLEANUP_WINDOW;

View File

@ -46,7 +46,8 @@ ACTOR Future<Void> waitFailureClient(RequestStream<ReplyPromise<Void>> waitFailu
if (!x.present()) {
if (trace) {
TraceEvent("WaitFailureClient")
.detail("FailedEndpoint", waitFailure.getEndpoint().getPrimaryAddress().toString());
.detail("FailedEndpoint", waitFailure.getEndpoint().getPrimaryAddress().toString())
.detail("Token", waitFailure.getEndpoint().token);
}
return Void();
}

View File

@ -1889,11 +1889,13 @@ int main(int argc, char* argv[]) {
g_network->run();
}
} else if (role == MultiTester) {
setupRunLoopProfiler();
f = stopAfter(runTests(opts.connectionFile, TEST_TYPE_FROM_FILE,
opts.testOnServers ? TEST_ON_SERVERS : TEST_ON_TESTERS, opts.minTesterCount,
opts.testFile, StringRef(), opts.localities));
g_network->run();
} else if (role == Test) {
setupRunLoopProfiler();
auto m = startSystemMonitor(opts.dataFolder, opts.zoneId, opts.zoneId);
f = stopAfter(runTests(opts.connectionFile, TEST_TYPE_FROM_FILE, TEST_HERE, 1, opts.testFile, StringRef(),
opts.localities));

345
fdbserver/fdbserver.vcxproj Normal file
View File

@ -0,0 +1,345 @@
<?xml version="1.0" encoding="utf-8"?>
<Project DefaultTargets="Build" ToolsVersion="14.1" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
<Import Project="$(SolutionDir)versions.target" />
<PropertyGroup Condition="'$(Release)' != 'true' ">
<PreReleaseDecoration>-PRERELEASE</PreReleaseDecoration>
</PropertyGroup>
<PropertyGroup Condition="'$(Release)' == 'true' ">
<PreReleaseDecoration>
</PreReleaseDecoration>
<PreprocessorDefinitions>FDB_CLEAN_BUILD;%(PreprocessorDefinitions)</PreprocessorDefinitions>
</PropertyGroup>
<ItemGroup Label="ProjectConfigurations">
<ProjectConfiguration Include="Debug|X64">
<Configuration>Debug</Configuration>
<Platform>X64</Platform>
</ProjectConfiguration>
<ProjectConfiguration Include="Release|X64">
<Configuration>Release</Configuration>
<Platform>X64</Platform>
</ProjectConfiguration>
</ItemGroup>
<ItemGroup>
<ClCompile Include="ApplyMetadataMutation.cpp" />
<ActorCompiler Include="ClusterController.actor.cpp" />
<ActorCompiler Include="DataDistribution.actor.cpp" />
<ActorCompiler Include="DataDistributionQueue.actor.cpp" />
<ActorCompiler Include="DataDistributionTracker.actor.cpp" />
<ActorCompiler Include="masterserver.actor.cpp" />
<ActorCompiler Include="MoveKeys.actor.cpp" />
<ActorCompiler Include="fdbserver.actor.cpp" />
<ActorCompiler Include="pubsub.actor.cpp" />
<ActorCompiler Include="storageserver.actor.cpp" />
<ActorCompiler Include="TLogServer.actor.cpp" />
<ActorCompiler Include="worker.actor.cpp" />
</ItemGroup>
<ItemGroup>
<ActorCompiler Include="VersionedBTree.actor.cpp" />
<ActorCompiler Include="Coordination.actor.cpp" />
<ActorCompiler Include="CoordinatedState.actor.cpp" />
<ActorCompiler Include="CoroFlow.actor.cpp" />
<ActorCompiler Include="MasterProxyServer.actor.cpp" />
<ActorCompiler Include="KeyValueStoreSQLite.actor.cpp" />
<ActorCompiler Include="LeaderElection.actor.cpp" />
<ActorCompiler Include="Ratekeeper.actor.cpp" />
<ActorCompiler Include="DiskQueue.actor.cpp" />
<ActorCompiler Include="KeyValueStoreMemory.actor.cpp" />
<ActorCompiler Include="SimulatedCluster.actor.cpp" />
<ActorCompiler Include="KeyValueStoreCompressTestData.actor.cpp" />
<ClCompile Include="Knobs.cpp" />
<ActorCompiler Include="FDBExecHelper.actor.cpp" />
<ActorCompiler Include="QuietDatabase.actor.cpp" />
<ActorCompiler Include="networktest.actor.cpp" />
<ActorCompiler Include="workloads\Unreadable.actor.cpp" />
<ActorCompiler Include="workloads\SaveAndKill.actor.cpp" />
<ActorCompiler Include="Resolver.actor.cpp" />
<ActorCompiler Include="Restore.actor.cpp" />
<ActorCompiler Include="LogSystemDiskQueueAdapter.actor.cpp" />
<ActorCompiler Include="LogSystemPeekCursor.actor.cpp" />
<ActorCompiler Include="LogRouter.actor.cpp" />
<ClCompile Include="LatencyBandConfig.cpp" />
<ActorCompiler Include="OldTLogServer_4_6.actor.cpp" />
<ActorCompiler Include="OldTLogServer_6_0.actor.cpp" />
<ClCompile Include="SkipList.cpp" />
<ActorCompiler Include="WaitFailure.actor.cpp" />
<ActorCompiler Include="tester.actor.cpp" />
</ItemGroup>
<ItemGroup>
<ActorCompiler Include="workloads\Cycle.actor.cpp" />
<ActorCompiler Include="workloads\SlowTaskWorkload.actor.cpp" />
<ActorCompiler Include="workloads\PubSubMultiples.actor.cpp" />
<ActorCompiler Include="workloads\RandomClogging.actor.cpp" />
<ActorCompiler Include="workloads\Inventory.actor.cpp" />
<ActorCompiler Include="workloads\BulkLoad.actor.cpp" />
<ActorCompiler Include="workloads\MachineAttrition.actor.cpp" />
<ActorCompiler Include="workloads\LocalRatekeeper.actor.cpp" />
<ActorCompiler Include="workloads\KillRegion.actor.cpp" />
<ActorCompiler Include="workloads\ReadWrite.actor.cpp" />
<ClCompile Include="sqlite\btree.c">
<ExcludedFromBuild>true</ExcludedFromBuild>
</ClCompile>
<ClCompile Include="sqlite\sqlite3.amalgamation.c" />
<ActorCompiler Include="Status.actor.cpp" />
<ActorCompiler Include="TagPartitionedLogSystem.actor.cpp" />
<ActorCompiler Include="workloads\DDBalance.actor.cpp" />
<ActorCompiler Include="workloads\FileSystem.actor.cpp" />
<ActorCompiler Include="workloads\ChangeConfig.actor.cpp" />
<ClCompile Include="VFSAsync.cpp" />
<ActorCompiler Include="workloads\ConflictRange.actor.cpp" />
<ActorCompiler Include="workloads\ApiWorkload.actor.cpp" />
<ActorCompiler Include="workloads\ApiCorrectness.actor.cpp" />
<ClCompile Include="workloads\AsyncFile.cpp" />
<ActorCompiler Include="workloads\AsyncFileCorrectness.actor.cpp" />
<ActorCompiler Include="workloads\AsyncFileRead.actor.cpp" />
<ActorCompiler Include="workloads\AsyncFileWrite.actor.cpp" />
<ActorCompiler Include="workloads\BackgroundSelectors.actor.cpp" />
<ActorCompiler Include="workloads\ConsistencyCheck.actor.cpp" />
<ActorCompiler Include="workloads\CpuProfiler.actor.cpp" />
<ActorCompiler Include="workloads\DDMetrics.actor.cpp" />
<ActorCompiler Include="workloads\DDMetricsExclude.actor.cpp" />
<ActorCompiler Include="workloads\ConfigureDatabase.actor.cpp" />
<ActorCompiler Include="workloads\CommitBugCheck.actor.cpp" />
<ActorCompiler Include="workloads\FastTriggeredWatches.actor.cpp" />
<ActorCompiler Include="workloads\DiskDurabilityTest.actor.cpp" />
<ActorCompiler Include="workloads\DummyWorkload.actor.cpp" />
<ActorCompiler Include="workloads\BackupCorrectness.actor.cpp" />
<ActorCompiler Include="workloads\AtomicOps.actor.cpp" />
<ActorCompiler Include="workloads\AtomicOpsApiCorrectness.actor.cpp" />
<ActorCompiler Include="workloads\ClientTransactionProfileCorrectness.actor.cpp" />
<ActorCompiler Include="workloads\TriggerRecovery.actor.cpp" />
<ActorCompiler Include="workloads\SuspendProcesses.actor.cpp" />
<ActorCompiler Include="workloads\BackupToDBAbort.actor.cpp" />
<ActorCompiler Include="workloads\BackupToDBCorrectness.actor.cpp" />
<ActorCompiler Include="workloads\BackupToDBUpgrade.actor.cpp" />
<ActorCompiler Include="workloads\AtomicSwitchover.actor.cpp" />
<ActorCompiler Include="workloads\AtomicRestore.actor.cpp" />
<ClCompile Include="workloads\Fuzz.cpp" />
<ActorCompiler Include="workloads\Sideband.actor.cpp" />
<ActorCompiler Include="workloads\Storefront.actor.cpp" />
<ActorCompiler Include="workloads\UnitPerf.actor.cpp" />
<ActorCompiler Include="workloads\RandomSelector.actor.cpp" />
<ActorCompiler Include="workloads\SelectorCorrectness.actor.cpp" />
<ActorCompiler Include="workloads\KVStoreTest.actor.cpp" />
<ActorCompiler Include="workloads\StreamingRead.actor.cpp" />
<ActorCompiler Include="workloads\Throttling.actor.cpp" />
<ActorCompiler Include="workloads\Throughput.actor.cpp" />
<ActorCompiler Include="workloads\WriteBandwidth.actor.cpp" />
<ActorCompiler Include="workloads\QueuePush.actor.cpp" />
<ActorCompiler Include="workloads\Rollback.actor.cpp" />
<ActorCompiler Include="workloads\LogMetrics.actor.cpp" />
<ActorCompiler Include="workloads\Performance.actor.cpp" />
<ActorCompiler Include="workloads\Ping.actor.cpp" />
<ActorCompiler Include="workloads\RandomMoveKeys.actor.cpp" />
<ActorCompiler Include="workloads\TargetedKill.actor.cpp" />
<ActorCompiler Include="workloads\TimeKeeperCorrectness.actor.cpp" />
<ActorCompiler Include="workloads\DifferentClustersSameRV.actor.cpp" />
<ActorCompiler Include="workloads\WriteDuringRead.actor.cpp" />
<ActorCompiler Include="workloads\Watches.actor.cpp" />
<ActorCompiler Include="workloads\ThreadSafety.actor.cpp" />
<ActorCompiler Include="workloads\RemoveServersSafely.actor.cpp" />
<ActorCompiler Include="workloads\Increment.actor.cpp" />
<ActorCompiler Include="workloads\FuzzApiCorrectness.actor.cpp" />
<ActorCompiler Include="workloads\LockDatabase.actor.cpp" />
<ActorCompiler Include="workloads\LockDatabaseFrequently.actor.cpp" />
<ActorCompiler Include="workloads\LowLatency.actor.cpp" />
<ClCompile Include="workloads\MemoryKeyValueStore.cpp" />
<ActorCompiler Include="workloads\RyowCorrectness.actor.cpp" />
<ActorCompiler Include="workloads\IndexScan.actor.cpp" />
<ActorCompiler Include="workloads\WatchAndWait.actor.cpp" />
<ActorCompiler Include="workloads\MetricLogging.actor.cpp" />
<ActorCompiler Include="workloads\RYWPerformance.actor.cpp" />
<ActorCompiler Include="workloads\RYWDisable.actor.cpp" />
<ActorCompiler Include="workloads\UnitTests.actor.cpp" />
<ActorCompiler Include="workloads\WorkerErrors.actor.cpp" />
<ActorCompiler Include="workloads\MemoryLifetime.actor.cpp" />
<ActorCompiler Include="workloads\TaskBucketCorrectness.actor.cpp" />
<ActorCompiler Include="workloads\StatusWorkload.actor.cpp" />
<ActorCompiler Include="workloads\VersionStamp.actor.cpp" />
<ActorCompiler Include="workloads\Serializability.actor.cpp" />
<ActorCompiler Include="workloads\DiskDurability.actor.cpp" />
<ActorCompiler Include="workloads\SnapTest.actor.cpp" />
<ActorCompiler Include="workloads\Mako.actor.cpp" />
<ActorCompiler Include="workloads\ExternalWorkload.actor.cpp" />
</ItemGroup>
<ItemGroup>
<ClInclude Include="ApplyMetadataMutation.h" />
<ClInclude Include="ClusterRecruitmentInterface.h" />
<ClInclude Include="ConflictSet.h" />
<ClInclude Include="CoordinatedState.h" />
<ClInclude Include="CoordinationInterface.h" />
<ClInclude Include="CoroFlow.h" />
<ActorCompiler Include="DataDistribution.actor.h">
<EnableCompile>false</EnableCompile>
</ActorCompiler>
<ClInclude Include="DataDistributorInterface.h" />
<ClInclude Include="DBCoreState.h" />
<ActorCompiler Include="FDBExecHelper.actor.h">
<EnableCompile>false</EnableCompile>
</ActorCompiler>
<ClInclude Include="IDiskQueue.h" />
<ClInclude Include="IKeyValueStore.h" />
<ClInclude Include="IPager.h" />
<ClInclude Include="IVersionedStore.h" />
<ClInclude Include="LatencyBandConfig.h" />
<ClInclude Include="LeaderElection.h" />
<ClInclude Include="LogProtocolMessage.h" />
<ClInclude Include="LogSystem.h" />
<ClInclude Include="LogSystemConfig.h" />
<ClInclude Include="LogSystemDiskQueueAdapter.h" />
<ClInclude Include="MasterInterface.h" />
<ActorCompiler Include="MoveKeys.actor.h">
<EnableCompile Condition="'$(Configuration)|$(Platform)'=='Debug|X64'">false</EnableCompile>
<EnableCompile Condition="'$(Configuration)|$(Platform)'=='Release|X64'">false</EnableCompile>
</ActorCompiler>
<ClInclude Include="NetworkTest.h" />
<ActorCompiler Include="Orderer.actor.h">
<EnableCompile Condition="'$(Configuration)|$(Platform)'=='Release|X64'">false</EnableCompile>
</ActorCompiler>
<ClInclude Include="pubsub.h" />
<ClInclude Include="Knobs.h" />
<ActorCompiler Include="StorageMetrics.actor.h">
<EnableCompile>false</EnableCompile>
</ActorCompiler>
<ClInclude Include="QuietDatabase.h" />
<ClInclude Include="RatekeeperInterface.h" />
<ClInclude Include="RecoveryState.h" />
<ClInclude Include="ResolverInterface.h" />
<ClInclude Include="RestoreInterface.h" />
<ClInclude Include="ServerDBInfo.h" />
<ClInclude Include="SimulatedCluster.h" />
<ClInclude Include="sqlite\btree.h" />
<ClInclude Include="sqlite\hash.h" />
<ClInclude Include="sqlite\sqlite3.h" />
<ClInclude Include="sqlite\sqlite3ext.h" />
<ClInclude Include="sqlite\sqliteInt.h" />
<ClInclude Include="sqlite\sqliteLimit.h" />
<ClInclude Include="Status.h" />
<ClInclude Include="StorageMetrics.h" />
<ClInclude Include="template_fdb.h" />
<ClInclude Include="TLogInterface.h" />
<ClInclude Include="WaitFailure.h" />
<ActorCompiler Include="TesterInterface.actor.h">
<EnableCompile Condition="'$(Configuration)|$(Platform)'=='Debug|X64'">false</EnableCompile>
<EnableCompile Condition="'$(Configuration)|$(Platform)'=='Release|X64'">false</EnableCompile>
</ActorCompiler>
<ActorCompiler Include="WorkerInterface.actor.h" />
<ActorCompiler Include="workloads\BulkSetup.actor.h">
<EnableCompile>false</EnableCompile>
</ActorCompiler>
<ActorCompiler Include="workloads\AsyncFile.actor.h" />
<ClInclude Include="workloads\ApiWorkload.h" />
<ClInclude Include="workloads\MemoryKeyValueStore.h" />
<ActorCompiler Include="workloads\workloads.actor.h">
<EnableCompile Condition="'$(Configuration)|$(Platform)'=='Debug|X64'">false</EnableCompile>
<EnableCompile Condition="'$(Configuration)|$(Platform)'=='Release|X64'">false</EnableCompile>
</ActorCompiler>
</ItemGroup>
<PropertyGroup Label="Globals">
<ProjectGUID>{8E959DA5-5925-45CE-BFC4-C84EB632A29A}</ProjectGUID>
<TargetFrameworkVersion>v4.5.2</TargetFrameworkVersion>
<Keyword>Win32Proj</Keyword>
<RootNamespace>flow</RootNamespace>
</PropertyGroup>
<PropertyGroup>
<OutDir>$(SolutionDir)bin\$(Configuration)\</OutDir>
<IntDir>$(SystemDrive)\temp\msvcfdb\$(Platform)$(Configuration)\$(MSBuildProjectName)\</IntDir>
<BuildLogFile>$(IntDir)\$(MSBuildProjectName).log</BuildLogFile>
</PropertyGroup>
<Import Project="$(VCTargetsPath)\Microsoft.Cpp.Default.props" />
<PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|X64'" Label="Configuration">
<ConfigurationType>Application</ConfigurationType>
<CharacterSet>MultiByte</CharacterSet>
<PlatformToolset>v141</PlatformToolset>
</PropertyGroup>
<PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|X64'" Label="Configuration">
<ConfigurationType>Application</ConfigurationType>
<CharacterSet>MultiByte</CharacterSet>
<PlatformToolset>v141</PlatformToolset>
</PropertyGroup>
<Import Project="$(VCTargetsPath)\Microsoft.Cpp.props" />
<ImportGroup Label="ExtensionSettings">
</ImportGroup>
<ImportGroup Label="PropertySheets">
<Import Project="$(LocalAppData)\Microsoft\VisualStudio\10.0\Microsoft.Cpp.$(Platform).user.props" Condition="exists('$(LocalAppData)\Microsoft\VisualStudio\10.0\Microsoft.Cpp.$(Platform).user.props')" />
</ImportGroup>
<PropertyGroup Label="UserMacros" />
<PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Debug|X64'">
<LinkIncremental>true</LinkIncremental>
<IncludePath>$(IncludePath);../;C:\Program Files\boost_1_67_0</IncludePath>
</PropertyGroup>
<PropertyGroup Condition="'$(Configuration)|$(Platform)'=='Release|X64'">
<LinkIncremental>false</LinkIncremental>
<IncludePath>$(IncludePath);../;C:\Program Files\boost_1_67_0</IncludePath>
<CustomBuildBeforeTargets>PreBuildEvent</CustomBuildBeforeTargets>
</PropertyGroup>
<ItemDefinitionGroup>
<Lib>
<AdditionalDependencies>$(TargetDir)fdbclient.lib</AdditionalDependencies>
</Lib>
<ClCompile>
<PreprocessorDefinitions>FDB_VT_VERSION="$(Version)$(PreReleaseDecoration)";FDB_VT_PACKAGE_NAME="$(PackageName)";%(PreprocessorDefinitions)</PreprocessorDefinitions>
<LanguageStandard>stdcpp17</LanguageStandard>
</ClCompile>
</ItemDefinitionGroup>
<ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Debug|X64'">
<ClCompile>
<PrecompiledHeader>
</PrecompiledHeader>
<WarningLevel>Level3</WarningLevel>
<MinimalRebuild>false</MinimalRebuild>
<DebugInformationFormat>ProgramDatabase</DebugInformationFormat>
<Optimization>Disabled</Optimization>
<BasicRuntimeChecks>EnableFastChecks</BasicRuntimeChecks>
<RuntimeLibrary>MultiThreadedDebug</RuntimeLibrary>
<PreprocessorDefinitions>TLS_DISABLED;WIN32;_WIN32_WINNT=0x0502;WINVER=0x0502;BOOST_ALL_NO_LIB;NTDDI_VERSION=0x05020000;_DEBUG;_HAS_ITERATOR_DEBUGGING=0;_CONSOLE;_CRT_SECURE_NO_WARNINGS;%(PreprocessorDefinitions)</PreprocessorDefinitions>
<AdditionalIncludeDirectories>%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
<MultiProcessorCompilation>true</MultiProcessorCompilation>
<AdditionalOptions>/bigobj @../flow/no_intellisense.opt %(AdditionalOptions)</AdditionalOptions>
<PreprocessToFile>false</PreprocessToFile>
<LanguageStandard>stdcpp17</LanguageStandard>
</ClCompile>
<Link>
<SubSystem>Console</SubSystem>
<GenerateDebugInformation>true</GenerateDebugInformation>
<AdditionalDependencies>$(SolutionDir)bin\$(Configuration)\fdbclient.lib;Advapi32.lib</AdditionalDependencies>
</Link>
</ItemDefinitionGroup>
<ItemDefinitionGroup Condition="'$(Configuration)|$(Platform)'=='Release|X64'">
<ClCompile>
<WarningLevel>Level3</WarningLevel>
<PrecompiledHeader>
</PrecompiledHeader>
<DebugInformationFormat>ProgramDatabase</DebugInformationFormat>
<Optimization>Full</Optimization>
<RuntimeLibrary>MultiThreaded</RuntimeLibrary>
<IntrinsicFunctions>true</IntrinsicFunctions>
<PreprocessorDefinitions>TLS_DISABLED;WIN32;_WIN32_WINNT=0x0502;WINVER=0x0502;BOOST_ALL_NO_LIB;NTDDI_VERSION=0x05020000;NDEBUG;_CONSOLE;_CRT_SECURE_NO_WARNINGS;%(PreprocessorDefinitions)</PreprocessorDefinitions>
<AdditionalIncludeDirectories>%(AdditionalIncludeDirectories)</AdditionalIncludeDirectories>
<EnableEnhancedInstructionSet>NotSet</EnableEnhancedInstructionSet>
<EnablePREfast>false</EnablePREfast>
<AdditionalOptions>/bigobj @../flow/no_intellisense.opt %(AdditionalOptions)</AdditionalOptions>
<MultiProcessorCompilation>true</MultiProcessorCompilation>
<FavorSizeOrSpeed>Speed</FavorSizeOrSpeed>
<BufferSecurityCheck>false</BufferSecurityCheck>
<MinimalRebuild>false</MinimalRebuild>
<LanguageStandard>stdcpp17</LanguageStandard>
</ClCompile>
<Link>
<SubSystem>Console</SubSystem>
<GenerateDebugInformation>true</GenerateDebugInformation>
<EnableCOMDATFolding>false</EnableCOMDATFolding>
<OptimizeReferences>false</OptimizeReferences>
<AdditionalDependencies>$(SolutionDir)bin\$(Configuration)\fdbclient.lib;Advapi32.lib</AdditionalDependencies>
<AdditionalOptions>/LTCG %(AdditionalOptions)</AdditionalOptions>
</Link>
</ItemDefinitionGroup>
<ImportGroup Label="ExtensionTargets">
<Import Project="..\flow\actorcompiler\ActorCompiler.targets" />
</ImportGroup>
<Import Project="$(VCTargetsPath)\Microsoft.Cpp.targets" />
<Target Name="MyPreCompileSteps" AfterTargets="CLCompile">
<Exec Command="..\bin\$(Configuration)\coveragetool.exe &quot;$(OutDir)coverage.$(TargetName).xml&quot; @(ActorCompiler -> '%(RelativeDir)%(Filename)%(Extension)', ' ')" />
<Exec Command="..\bin\$(Configuration)\coveragetool.exe &quot;$(OutDir)coverage.$(TargetName).include.xml&quot; @(CLInclude -> '%(RelativeDir)%(Filename)%(Extension)', ' ')" />
<Exec Command="..\bin\$(Configuration)\coveragetool.exe &quot;$(OutDir)coverage.$(TargetName).compile.xml&quot; @(CLCompile -> '%(RelativeDir)%(Filename)%(Extension)', ' ')" />
</Target>
</Project>

View File

@ -0,0 +1,400 @@
<?xml version="1.0" encoding="utf-8"?>
<Project ToolsVersion="4.0" xmlns="http://schemas.microsoft.com/developer/msbuild/2003">
<ItemGroup>
<ActorCompiler Include="ClusterController.actor.cpp" />
<ActorCompiler Include="DataDistribution.actor.cpp" />
<ActorCompiler Include="DataDistributionQueue.actor.cpp" />
<ActorCompiler Include="DataDistributionTracker.actor.cpp" />
<ActorCompiler Include="masterserver.actor.cpp" />
<ActorCompiler Include="MoveKeys.actor.cpp" />
<ActorCompiler Include="fdbserver.actor.cpp" />
<ActorCompiler Include="pubsub.actor.cpp" />
<ActorCompiler Include="storageserver.actor.cpp" />
<ActorCompiler Include="TLogServer.actor.cpp" />
<ActorCompiler Include="worker.actor.cpp" />
<ActorCompiler Include="WaitFailure.actor.cpp" />
<ActorCompiler Include="MasterProxyServer.actor.cpp" />
<ActorCompiler Include="tester.actor.cpp" />
<ActorCompiler Include="workloads\Cycle.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\RandomClogging.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\BulkLoad.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\MachineAttrition.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\LocalRatekeeper.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\KillRegion.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\ReadWrite.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\Sideband.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\Storefront.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\Inventory.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\PubSubMultiples.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\UnitPerf.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\FileSystem.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\DDBalance.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="StorageMetrics.actor.h" />
<ActorCompiler Include="workloads\RandomSelector.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\SelectorCorrectness.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\KVStoreTest.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\ChangeConfig.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="KeyValueStoreSQLite.actor.cpp" />
<ActorCompiler Include="LeaderElection.actor.cpp" />
<ActorCompiler Include="workloads\StreamingRead.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\BulkSetup.actor.h">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="Ratekeeper.actor.cpp" />
<ActorCompiler Include="Status.actor.cpp" />
<ActorCompiler Include="workloads\ConflictRange.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\Throttling.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\Throughput.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\AsyncFileCorrectness.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\AsyncFileRead.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\AsyncFileWrite.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\AsyncFile.actor.h">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="DiskQueue.actor.cpp" />
<ActorCompiler Include="workloads\BackgroundSelectors.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\WriteBandwidth.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\QueuePush.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="KeyValueStoreMemory.actor.cpp" />
<ActorCompiler Include="SimulatedCluster.actor.cpp" />
<ActorCompiler Include="KeyValueStoreCompressTestData.actor.cpp" />
<ActorCompiler Include="Coordination.actor.cpp" />
<ActorCompiler Include="CoordinatedState.actor.cpp" />
<ActorCompiler Include="workloads\Rollback.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="QuietDatabase.actor.cpp" />
<ActorCompiler Include="workloads\ConsistencyCheck.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\ConfigureDatabase.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\CpuProfiler.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\DDMetrics.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\LogMetrics.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\Performance.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\Ping.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\RandomMoveKeys.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\TargetedKill.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\WriteDuringRead.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\CommitBugCheck.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\Watches.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\ThreadSafety.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\RemoveServersSafely.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="networktest.actor.cpp" />
<ActorCompiler Include="workloads\SaveAndKill.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\Increment.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\ApiWorkload.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\ApiCorrectness.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\RyowCorrectness.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\IndexScan.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\FastTriggeredWatches.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\WatchAndWait.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\MetricLogging.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\RYWPerformance.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\RYWDisable.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="Resolver.actor.cpp" />
<ActorCompiler Include="LogSystemDiskQueueAdapter.actor.cpp" />
<ActorCompiler Include="Orderer.actor.h" />
<ActorCompiler Include="workloads\DiskDurabilityTest.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="TagPartitionedLogSystem.actor.cpp" />
<ActorCompiler Include="LogSystemPeekCursor.actor.cpp" />
<ActorCompiler Include="workloads\UnitTests.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\FuzzApiCorrectness.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\MemoryLifetime.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\DummyWorkload.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\BackupCorrectness.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\TaskBucketCorrectness.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\AtomicOps.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\AtomicOpsApiCorrectness.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\ClientTransactionProfileCorrectness.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\TriggerRecovery.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\SuspendProcesses.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\StatusWorkload.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\Unreadable.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\VersionStamp.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="CoroFlow.actor.cpp" />
<ActorCompiler Include="workloads\Serializability.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\LockDatabase.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\LockDatabaseFrequently.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\TimeKeeperCorrectness.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\DifferentClustersSameRV.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\BackupToDBAbort.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\BackupToDBCorrectness.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\BackupToDBUpgrade.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\AtomicSwitchover.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\AtomicRestore.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="OldTLogServer.actor.cpp" />
<ActorCompiler Include="LogRouter.actor.cpp" />
<ActorCompiler Include="workloads\SlowTaskWorkload.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\DiskDurability.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\LowLatency.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\SlowTaskWorkload.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\WorkerErrors.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\DiskDurability.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="workloads\Mako.actor.cpp">
<Filter>workloads</Filter>
</ActorCompiler>
<ActorCompiler Include="OldTLogServer.actor.cpp" />
</ItemGroup>
<ItemGroup>
<ClCompile Include="SkipList.cpp" />
<ClCompile Include="workloads\Fuzz.cpp">
<Filter>workloads</Filter>
</ClCompile>
<ClCompile Include="sqlite\btree.c">
<Filter>sqlite</Filter>
</ClCompile>
<ClCompile Include="sqlite\sqlite3.amalgamation.c">
<Filter>sqlite</Filter>
</ClCompile>
<ClCompile Include="VFSAsync.cpp" />
<ClCompile Include="workloads\AsyncFile.cpp">
<Filter>workloads</Filter>
</ClCompile>
<ClCompile Include="Knobs.cpp" />
<ClCompile Include="workloads\MemoryKeyValueStore.cpp">
<Filter>workloads</Filter>
</ClCompile>
<ClCompile Include="LatencyBandConfig.cpp" />
</ItemGroup>
<ItemGroup>
<ClInclude Include="ConflictSet.h" />
<ClInclude Include="DataDistribution.actor.h" />
<ClInclude Include="DataDistributorInterface.h" />
<ClInclude Include="MoveKeys.actor.h" />
<ClInclude Include="pubsub.h" />
<ClInclude Include="Knobs.h" />
<ClInclude Include="WorkerInterface.h" />
<ClInclude Include="WaitFailure.h" />
<ClInclude Include="TesterInterface.actor.h" />
<ClInclude Include="workloads\workloads.actor.h">
<Filter>workloads</Filter>
</ClInclude>
<ClInclude Include="IKeyValueStore.h" />
<ClInclude Include="ClusterRecruitmentInterface.h" />
<ClInclude Include="MasterInterface.h" />
<ClInclude Include="TLogInterface.h" />
<ClInclude Include="sqlite\sqlite3.h">
<Filter>sqlite</Filter>
</ClInclude>
<ClInclude Include="sqlite\sqlite3ext.h">
<Filter>sqlite</Filter>
</ClInclude>
<ClInclude Include="sqlite\btree.h">
<Filter>sqlite</Filter>
</ClInclude>
<ClInclude Include="sqlite\sqliteInt.h">
<Filter>sqlite</Filter>
</ClInclude>
<ClInclude Include="sqlite\sqliteLimit.h">
<Filter>sqlite</Filter>
</ClInclude>
<ClInclude Include="sqlite\hash.h">
<Filter>sqlite</Filter>
</ClInclude>
<ClInclude Include="LeaderElection.h" />
<ClInclude Include="StorageMetrics.h" />
<ClInclude Include="RatekeeperInterface.h" />
<ClInclude Include="Status.h" />
<ClInclude Include="IDiskQueue.h" />
<ClInclude Include="CoroFlow.h" />
<ClInclude Include="SimulatedCluster.h" />
<ClInclude Include="CoordinatedState.h" />
<ClInclude Include="ServerDBInfo.h" />
<ClInclude Include="QuietDatabase.h" />
<ClInclude Include="CoordinationInterface.h" />
<ClInclude Include="NetworkTest.h" />
<ClInclude Include="workloads\MemoryKeyValueStore.h">
<Filter>workloads</Filter>
</ClInclude>
<ClInclude Include="workloads\ApiWorkload.h">
<Filter>workloads</Filter>
</ClInclude>
<ClInclude Include="ResolverInterface.h" />
<ClInclude Include="DBCoreState.h" />
<ClInclude Include="LogSystem.h" />
<ClInclude Include="LogSystemDiskQueueAdapter.h" />
<ClInclude Include="LogSystemConfig.h" />
<ClInclude Include="ApplyMetadataMutation.h" />
<ClInclude Include="RecoveryState.h" />
<ClInclude Include="LogProtocolMessage.h" />
<ClInclude Include="IPager.h" />
<ClInclude Include="IVersionedStore.h" />
<ClInclude Include="template_fdb.h" />
<ClInclude Include="LatencyBandConfig.h" />
</ItemGroup>
<ItemGroup>
<Filter Include="workloads">
<UniqueIdentifier>{6a79fc02-2f89-451d-9dd5-999d753b3159}</UniqueIdentifier>
</Filter>
<Filter Include="sqlite">
<UniqueIdentifier>{de5e282f-8d97-4054-b795-0a75b772326f}</UniqueIdentifier>
</Filter>
</ItemGroup>
</Project>

View File

@ -1011,6 +1011,11 @@ ACTOR Future<Void> getValueQ( StorageServer* data, GetValueRequest req ) {
return Void();
};
// Pessimistic estimate the number of overhead bytes used by each
// watch. Watch key references are stored in an AsyncMap<Key,bool>, and actors
// must be kept alive until the watch is finished.
static constexpr size_t WATCH_OVERHEAD_BYTES = 1000;
ACTOR Future<Void> watchValue_impl( StorageServer* data, WatchValueRequest req ) {
try {
++data->counters.watchQueries;
@ -1058,7 +1063,7 @@ ACTOR Future<Void> watchValue_impl( StorageServer* data, WatchValueRequest req )
}
++data->numWatches;
data->watchBytes += ( req.key.expectedSize() + req.value.expectedSize() + 1000 );
data->watchBytes += (req.key.expectedSize() + req.value.expectedSize() + WATCH_OVERHEAD_BYTES);
try {
if(latest < minVersion) {
// If the version we read is less than minVersion, then we may fail to be notified of any changes that occur up to or including minVersion
@ -1071,10 +1076,10 @@ ACTOR Future<Void> watchValue_impl( StorageServer* data, WatchValueRequest req )
}
wait(watchFuture);
--data->numWatches;
data->watchBytes -= ( req.key.expectedSize() + req.value.expectedSize() + 1000 );
data->watchBytes -= (req.key.expectedSize() + req.value.expectedSize() + WATCH_OVERHEAD_BYTES);
} catch( Error &e ) {
--data->numWatches;
data->watchBytes -= ( req.key.expectedSize() + req.value.expectedSize() + 1000 );
data->watchBytes -= (req.key.expectedSize() + req.value.expectedSize() + WATCH_OVERHEAD_BYTES);
throw;
}
} catch( Error &e ) {
@ -3945,7 +3950,7 @@ ACTOR Future<Void> replaceInterface( StorageServer* self, StorageServerInterface
loop {
state Future<Void> infoChanged = self->db->onChange();
state Reference<ProxyInfo> proxies( new ProxyInfo(self->db->get().client.proxies) );
state Reference<ProxyInfo> proxies( new ProxyInfo(self->db->get().client.proxies, false) );
choose {
when( GetStorageServerRejoinInfoReply _rep = wait( proxies->size() ? basicLoadBalance( proxies, &MasterProxyInterface::getStorageServerRejoinInfo, GetStorageServerRejoinInfoRequest(ssi.id(), ssi.locality.dcId()) ) : Never() ) ) {
state GetStorageServerRejoinInfoReply rep = _rep;

View File

@ -1378,6 +1378,11 @@ ACTOR Future<Void> workerServer(
DUMPTOKEN( recruited.lock );
DUMPTOKEN( recruited.getQueuingMetrics );
DUMPTOKEN( recruited.confirmRunning );
DUMPTOKEN( recruited.waitFailure );
DUMPTOKEN( recruited.recoveryFinished );
DUMPTOKEN( recruited.disablePopRequest );
DUMPTOKEN( recruited.enablePopRequest );
DUMPTOKEN( recruited.snapRequest );
errorForwarders.add( zombie(recruited, forwardError( errors, Role::LOG_ROUTER, recruited.id(),
logRouter( recruited, req, dbInfo ) ) ) );

View File

@ -91,6 +91,9 @@ public:
//The API being used by this client
TransactionType transactionType;
// Maximum time to reset DB to the original state
double resetDBTimeout;
ApiCorrectnessWorkload(WorkloadContext const& wcx) : ApiWorkload(wcx), numRandomOperations("Num Random Operations") {
numGets = getOption(options, LiteralStringRef("numGets"), 1000);
numGetRanges = getOption(options, LiteralStringRef("numGetRanges"), 100);
@ -106,6 +109,8 @@ public:
int maxTransactionBytes = getOption(options, LiteralStringRef("maxTransactionBytes"), 500000);
maxKeysPerTransaction = std::max(1, maxTransactionBytes / (maxValueLength + maxLongKeyLength));
resetDBTimeout = getOption(options, LiteralStringRef("resetDBTimeout"), 1800.0);
if(maxTransactionBytes > 500000) {
TraceEvent("RemapEventSeverity").detail("TargetEvent", "LargePacketSent").detail("OriginalSeverity", SevWarnAlways).detail("NewSeverity", SevInfo);
TraceEvent("RemapEventSeverity").detail("TargetEvent", "LargePacketReceived").detail("OriginalSeverity", SevWarnAlways).detail("NewSeverity", SevInfo);
@ -146,9 +151,9 @@ public:
wait(timeout(self->runScriptedTest(self, data), 600, Void()));
if(!self->hasFailed()) {
//Return database to original state (for a maximum of 1800 seconds)
// Return database to original state (for a maximum of resetDBTimeout seconds)
try {
wait(timeoutError(::success(self->runSet(data, self)), 1800));
wait(timeoutError(::success(self->runSet(data, self)), self->resetDBTimeout));
}
catch(Error &e) {
if(e.code() == error_code_timed_out) {

View File

@ -93,7 +93,9 @@ Future<Void> ApiWorkload::start(Database const& cx) {
void ApiWorkload::testFailure(std::string reason)
{
printf("test failure on client %d: %s\n", clientPrefixInt, reason.c_str());
TraceEvent(SevError, "TestFailure").detail("Reason", description() + reason).detail("Workload", "ApiCorrectness");
TraceEvent(SevError, "TestFailure")
.detail("Reason", description() + " " + reason)
.detail("Workload", "ApiCorrectness");
success = false;
}

View File

@ -55,8 +55,7 @@ struct AtomicOpsWorkload : TestWorkload {
ubsum = 0;
int64_t randNum = sharedRandomNumber / 10;
if(opType == -1)
opType = randNum % 8;
if (opType == -1) opType = randNum % 10;
switch(opType) {
case 0:
@ -91,6 +90,18 @@ struct AtomicOpsWorkload : TestWorkload {
TEST(true); //Testing atomic ByteMax
opType = MutationRef::ByteMax;
break;
case 8:
TEST(true); // Testing atomic MinV2
opType = MutationRef::MinV2;
break;
case 9:
TEST(true); // Testing atomic AndV2
opType = MutationRef::AndV2;
break;
// case 10:
// TEST(true); // Testing atomic CompareAndClear Not supported yet
// opType = MutationRef::CompareAndClear
// break;
default:
ASSERT(false);
}

View File

@ -468,6 +468,7 @@ struct BackupAndParallelRestoreCorrectnessWorkload : TestWorkload {
.detail("LastBackupContainer", lastBackupContainer->getURL())
.detail("RestoreAfter", self->restoreAfter)
.detail("BackupTag", printable(self->backupTag));
// start restoring
auto container = IBackupContainer::openContainer(lastBackupContainer->getURL());
BackupDescription desc = wait(container->describeBackup());

View File

@ -0,0 +1,73 @@
/*
* LockDatabaseFrequently.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/NativeAPI.actor.h"
#include "fdbserver/TesterInterface.actor.h"
#include "fdbserver/workloads/workloads.actor.h"
#include "fdbclient/ManagementAPI.actor.h"
#include "flow/actorcompiler.h" // This must be the last #include.
struct LockDatabaseFrequentlyWorkload : TestWorkload {
double delayBetweenLocks;
double testDuration;
PerfIntCounter lockCount{ "LockCount" };
LockDatabaseFrequentlyWorkload(WorkloadContext const& wcx) : TestWorkload(wcx) {
delayBetweenLocks = getOption(options, LiteralStringRef("delayBetweenLocks"), 0.1);
testDuration = getOption(options, LiteralStringRef("testDuration"), 60);
}
std::string description() override { return "LockDatabaseFrequently"; }
Future<Void> setup(Database const& cx) override { return Void(); }
Future<Void> start(Database const& cx) override { return clientId == 0 ? worker(this, cx) : Void(); }
Future<bool> check(Database const& cx) override { return true; }
void getMetrics(vector<PerfMetric>& m) override {
if (clientId == 0) {
m.push_back(lockCount.getMetric());
}
}
ACTOR static Future<Void> worker(LockDatabaseFrequentlyWorkload* self, Database cx) {
state Future<Void> end = delay(self->testDuration);
state double lastLock = g_network->now();
state double lastUnlock = g_network->now() + self->delayBetweenLocks / 2;
loop {
wait(lockAndUnlock(self, cx, &lastLock, &lastUnlock));
++self->lockCount;
if (end.isReady()) {
return Void();
}
}
}
ACTOR static Future<Void> lockAndUnlock(LockDatabaseFrequentlyWorkload* self, Database cx, double* lastLock,
double* lastUnlock) {
state UID uid = deterministicRandom()->randomUniqueID();
wait(lockDatabase(cx, uid) && poisson(lastLock, self->delayBetweenLocks));
wait(unlockDatabase(cx, uid) && poisson(lastUnlock, self->delayBetweenLocks));
return Void();
}
};
WorkloadFactory<LockDatabaseFrequentlyWorkload> LockDatabaseFrequentlyWorkloadFactory("LockDatabaseFrequently");

View File

@ -99,15 +99,15 @@ struct ReadHotDetectionWorkload : TestWorkload {
// TraceEvent("RHDCheckPhaseLog")
// .detail("KeyRangeSize", sm.bytes)
// .detail("KeyRangeReadBandwith", sm.bytesReadPerKSecond);
Standalone<VectorRef<KeyRangeRef>> keyRanges = wait(tr.getReadHotRanges(self->wholeRange));
Standalone<VectorRef<ReadHotRangeWithMetrics>> keyRanges = wait(tr.getReadHotRanges(self->wholeRange));
// TraceEvent("RHDCheckPhaseLog")
// .detail("KeyRangesSize", keyRanges.size())
// .detail("ReadKey", self->readKey.printable().c_str())
// .detail("KeyRangesBackBeginKey", keyRanges.back().begin)
// .detail("KeyRangesBackEndKey", keyRanges.back().end);
// Loose check.
for (auto kr : keyRanges) {
if (kr.contains(self->readKey)) {
for (const auto& kr : keyRanges) {
if (kr.keys.contains(self->readKey)) {
self->passed = true;
}
}

View File

@ -27,6 +27,10 @@
#include "fdbserver/ServerDBInfo.h"
#include "flow/actorcompiler.h" // This must be the last #include.
// Choose a random proxy and a random tLog, represented as unclogTlog.
// The workload first clogs network link between the chosen proxy and all tLogs but the unclogTlog;
// While the network is still clogged, the workload kills the proxy and clogs the unclogged tlog's interface.
// Note: The clogged network link's latency will become "clogDuration".
struct RollbackWorkload : TestWorkload {
bool enableFailures, multiple, enabled;
double meanDelay, clogDuration, testDuration;
@ -80,19 +84,21 @@ struct RollbackWorkload : TestWorkload {
.detail("Proxy", proxy.address())
.detail("UncloggedTLog", uncloggedTLog);
for(int t=0; t<tlogs.size(); t++)
if (t != utIndex)
for (int t = 0; t < tlogs.size(); t++) {
if (t != utIndex) {
g_simulator.clogPair(
proxy.address().ip,
tlogs[t].address().ip,
self->clogDuration );
//g_simulator.clogInterface( g_simulator.getProcess( system.tlogs[t].commit.getEndpoint() ), self->clogDuration, ClogAll );
}
}
// While the clogged machines are still clogged...
wait( delay( self->clogDuration/3 ) );
system = self->dbInfo->get();
// Kill the proxy and the unclogged tlog
// Kill the proxy and clog the unclogged tlog
if (self->enableFailures) {
g_simulator.killProcess( g_simulator.getProcessByAddress( proxy.address() ), ISimulator::KillInstantly );
g_simulator.clogInterface( uncloggedTLog.ip, self->clogDuration, ClogAll );

View File

@ -119,6 +119,10 @@ struct SpecialKeySpaceCorrectnessWorkload : TestWorkload {
Future<Void> f;
{
ReadYourWritesTransaction ryw{ cx->clone() };
if(!ryw.getDatabase()->apiVersionAtLeast(630)) {
//This test is not valid for API versions smaller than 630
return;
}
f = success(ryw.get(LiteralStringRef("\xff\xff/status/json")));
TEST(!f.isReady());
}

View File

@ -347,7 +347,7 @@ struct VersionStampWorkload : TestWorkload {
}
catch (Error &e) {
err = e;
if (err.code() == error_code_database_locked) {
if (err.code() == error_code_database_locked && g_simulator.extraDB != nullptr) {
//TraceEvent("VST_CommitDatabaseLocked");
cx_is_primary = !cx_is_primary;
tr = ReadYourWritesTransaction(cx_is_primary ? cx : extraDB);

View File

@ -562,6 +562,15 @@ public:
memcpy(dst, data, length);
return dst + length;
}
std::vector<StringRef> splitAny(StringRef sep) const {
StringRef r = *this;
std::vector<StringRef> tokens;
while (r.size()) {
tokens.push_back(r.eatAny(sep, nullptr));
}
return tokens;
}
private:
// Unimplemented; blocks conversion through std::string
@ -670,7 +679,7 @@ inline bool operator==(const StringRef& lhs, const StringRef& rhs) {
if (lhs.size() == 0 && rhs.size() == 0) {
return true;
}
return lhs.size() == rhs.size() && !memcmp(lhs.begin(), rhs.begin(), lhs.size());
return lhs.size() == rhs.size() && memcmp(lhs.begin(), rhs.begin(), lhs.size()) == 0;
}
inline bool operator<(const StringRef& lhs, const StringRef& rhs) {
if (std::min(lhs.size(), rhs.size()) > 0) {

View File

@ -82,11 +82,12 @@ Error systemErrorCodeToError();
inline Error actor_cancelled() { return Error( error_code_operation_cancelled ); }
enum { error_code_actor_cancelled = error_code_operation_cancelled };
extern Error internal_error_impl( const char* file, int line );
extern Error internal_error_impl(const char* file, int line);
extern Error internal_error_impl(const char* msg, const char* file, int line);
extern Error internal_error_impl(const char * a_nm, long long a, const char * op_nm, const char * b_nm, long long b, const char * file, int line);
#define inernal_error_msg(msg) internal_error_impl(msg, __FILE__, __LINE__)
#define internal_error() internal_error_impl(__FILE__, __LINE__)
#define internal_error_msg(msg) internal_error_impl(msg, __FILE__, __LINE__)
extern bool isAssertDisabled( int line );
//#define ASSERT( condition ) ((void)0)

View File

@ -76,11 +76,14 @@ void FlowKnobs::initialize(bool randomize, bool isSimulated) {
init( MAX_RECONNECTION_TIME, 0.5 );
init( RECONNECTION_TIME_GROWTH_RATE, 1.2 );
init( RECONNECTION_RESET_TIME, 5.0 );
init( ALWAYS_ACCEPT_DELAY, 15.0 );
init( ACCEPT_BATCH_SIZE, 10 );
init( TOO_MANY_CONNECTIONS_CLOSED_RESET_DELAY, 5.0 );
init( TOO_MANY_CONNECTIONS_CLOSED_TIMEOUT, 20.0 );
init( PEER_UNAVAILABLE_FOR_LONG_TIME_TIMEOUT, 3600.0 );
init( INCOMPATIBLE_PEER_DELAY_BEFORE_LOGGING, 5.0 );
init( PING_LOGGING_INTERVAL, 1.0 );
init( PING_SAMPLE_AMOUNT, 100 );
init( TLS_CERT_REFRESH_DELAY_SECONDS, 12*60*60 );
init( TLS_SERVER_CONNECTION_THROTTLE_TIMEOUT, 9.0 );
@ -210,10 +213,13 @@ void FlowKnobs::initialize(bool randomize, bool isSimulated) {
init( FUTURE_VERSION_BACKOFF_GROWTH, 2.0 );
init( LOAD_BALANCE_MAX_BAD_OPTIONS, 1 ); //should be the same as MAX_MACHINES_FALLING_BEHIND
init( LOAD_BALANCE_PENALTY_IS_BAD, true );
init( BASIC_LOAD_BALANCE_UPDATE_RATE, 2.0 );
init( BASIC_LOAD_BALANCE_MAX_CHANGE, 0.05 );
init( BASIC_LOAD_BALANCE_UPDATE_RATE, 10.0 ); //should be longer than the rate we log network metrics
init( BASIC_LOAD_BALANCE_MAX_CHANGE, 0.10 );
init( BASIC_LOAD_BALANCE_MAX_PROB, 2.0 );
init( BASIC_LOAD_BALANCE_BUCKETS, 40 );
init( BASIC_LOAD_BALANCE_MIN_REQUESTS, 20 ); //do not adjust LB probabilities if the proxies are less than releasing less than 20 transactions per second
init( BASIC_LOAD_BALANCE_MIN_CPU, 0.05 ); //do not adjust LB probabilities if the proxies are less than 5% utilized
init( BASIC_LOAD_BALANCE_BUCKETS, 40 ); //proxies bin recent GRV requests into 40 time bins
init( BASIC_LOAD_BALANCE_COMPUTE_PRECISION, 10000 ); //determines how much of the LB usage is holding the CPU usage of the proxy
// Health Monitor
init( FAILURE_DETECTION_DELAY, 4.0 ); if( randomize && BUGGIFY ) FAILURE_DETECTION_DELAY = 1.0;

Some files were not shown because too many files have changed in this diff Show More