Better errors logging, avoid zero-bytes bug in keyMayExist

This commit is contained in:
Andrea Cavalli 2022-03-18 15:33:54 +01:00
parent 59c37c0fc9
commit 28b4fdee50
5 changed files with 147 additions and 107 deletions

View File

@ -3,7 +3,6 @@ package it.cavallium.dbengine.database.collections;
import io.netty5.buffer.api.Buffer; import io.netty5.buffer.api.Buffer;
import io.netty5.buffer.api.Resource; import io.netty5.buffer.api.Resource;
import io.netty5.buffer.api.Send; import io.netty5.buffer.api.Send;
import io.netty5.buffer.api.internal.ResourceSupport;
import it.cavallium.dbengine.client.CompositeSnapshot; import it.cavallium.dbengine.client.CompositeSnapshot;
import it.cavallium.dbengine.database.Delta; import it.cavallium.dbengine.database.Delta;
import it.cavallium.dbengine.database.LLDictionary; import it.cavallium.dbengine.database.LLDictionary;
@ -22,7 +21,6 @@ import it.unimi.dsi.fastutil.objects.Object2ObjectLinkedOpenHashMap;
import it.unimi.dsi.fastutil.objects.Object2ObjectSortedMap; import it.unimi.dsi.fastutil.objects.Object2ObjectSortedMap;
import it.unimi.dsi.fastutil.objects.Object2ObjectSortedMaps; import it.unimi.dsi.fastutil.objects.Object2ObjectSortedMaps;
import java.util.Collections; import java.util.Collections;
import java.util.List;
import java.util.Map; import java.util.Map;
import java.util.Map.Entry; import java.util.Map.Entry;
import java.util.Objects; import java.util.Objects;
@ -35,7 +33,6 @@ import org.jetbrains.annotations.Nullable;
import reactor.core.publisher.Flux; import reactor.core.publisher.Flux;
import reactor.core.publisher.Mono; import reactor.core.publisher.Mono;
import reactor.core.publisher.SynchronousSink; import reactor.core.publisher.SynchronousSink;
import reactor.util.function.Tuple2;
/** /**
* Optimized implementation of "DatabaseMapDictionary with SubStageGetterSingle" * Optimized implementation of "DatabaseMapDictionary with SubStageGetterSingle"
@ -73,18 +70,26 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
return new DatabaseMapDictionary<>(dictionary, prefixKey, keySuffixSerializer, valueSerializer, onClose); return new DatabaseMapDictionary<>(dictionary, prefixKey, keySuffixSerializer, valueSerializer, onClose);
} }
private void deserializeValue(Send<Buffer> valueToReceive, SynchronousSink<U> sink) { private void deserializeValue(T keySuffix, Send<Buffer> valueToReceive, SynchronousSink<U> sink) {
try (var value = valueToReceive.receive()) { try (var value = valueToReceive.receive()) {
try {
sink.next(valueSerializer.deserialize(value)); sink.next(valueSerializer.deserialize(value));
} catch (IndexOutOfBoundsException ex) { } catch (IndexOutOfBoundsException ex) {
var exMessage = ex.getMessage(); var exMessage = ex.getMessage();
if (exMessage != null && exMessage.contains("read 0 to 0, write 0 to ")) { if (exMessage != null && exMessage.contains("read 0 to 0, write 0 to ")) {
var totalZeroBytesErrors = this.totalZeroBytesErrors.incrementAndGet(); var totalZeroBytesErrors = this.totalZeroBytesErrors.incrementAndGet();
if (totalZeroBytesErrors < 512 || totalZeroBytesErrors % 10000 == 0) { if (totalZeroBytesErrors < 512 || totalZeroBytesErrors % 10000 == 0) {
LOG.error("Unexpected zero-bytes value in column " try (var keySuffixBytes = serializeKeySuffixToKey(keySuffix)) {
+ dictionary.getDatabaseName() + ":" + dictionary.getColumnName() LOG.error("Unexpected zero-bytes value at " + dictionary.getDatabaseName()
+ " total=" + totalZeroBytesErrors + ":" + dictionary.getColumnName()
); + ":" + LLUtils.toStringSafe(this.keyPrefix)
+ ":" + keySuffix + "(" + LLUtils.toStringSafe(keySuffixBytes) + ") total=" + totalZeroBytesErrors);
} catch (SerializationException e) {
LOG.error("Unexpected zero-bytes value at " + dictionary.getDatabaseName()
+ ":" + dictionary.getColumnName()
+ ":" + LLUtils.toStringSafe(this.keyPrefix)
+ ":" + keySuffix + "(?) total=" + totalZeroBytesErrors);
}
} }
sink.complete(); sink.complete();
} else { } else {
@ -94,6 +99,7 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
sink.error(ex); sink.error(ex);
} }
} }
}
private Buffer serializeValue(U value) throws SerializationException { private Buffer serializeValue(U value) throws SerializationException {
var valSizeHint = valueSerializer.getSerializedSizeHint(); var valSizeHint = valueSerializer.getSerializedSizeHint();
@ -227,7 +233,7 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix).send()), Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix).send()),
existsAlmostCertainly existsAlmostCertainly
) )
.handle(this::deserializeValue); .handle((valueToReceive, sink) -> deserializeValue(keySuffix, valueToReceive, sink));
} }
@Override @Override
@ -252,7 +258,7 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix).send()); var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix).send());
return dictionary return dictionary
.update(keyMono, getSerializedUpdater(updater), updateReturnMode) .update(keyMono, getSerializedUpdater(updater), updateReturnMode)
.handle(this::deserializeValue); .handle((valueToReceive, sink) -> deserializeValue(keySuffix, valueToReceive, sink));
} }
@Override @Override
@ -314,7 +320,9 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
public Mono<U> putValueAndGetPrevious(T keySuffix, U value) { public Mono<U> putValueAndGetPrevious(T keySuffix, U value) {
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix).send()); var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix).send());
var valueMono = Mono.fromCallable(() -> serializeValue(value).send()); var valueMono = Mono.fromCallable(() -> serializeValue(value).send());
return dictionary.put(keyMono, valueMono, LLDictionaryResultType.PREVIOUS_VALUE).handle(this::deserializeValue); return dictionary
.put(keyMono, valueMono, LLDictionaryResultType.PREVIOUS_VALUE)
.handle((valueToReceive, sink) -> deserializeValue(keySuffix, valueToReceive, sink));
} }
@Override @Override
@ -323,7 +331,10 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
var valueMono = Mono.fromCallable(() -> serializeValue(value).send()); var valueMono = Mono.fromCallable(() -> serializeValue(value).send());
return dictionary return dictionary
.put(keyMono, valueMono, LLDictionaryResultType.PREVIOUS_VALUE) .put(keyMono, valueMono, LLDictionaryResultType.PREVIOUS_VALUE)
.handle(this::deserializeValue) .handle((Send<Buffer> valueToReceive, SynchronousSink<U> sink) -> deserializeValue(keySuffix,
valueToReceive,
sink
))
.map(oldValue -> !Objects.equals(oldValue, value)) .map(oldValue -> !Objects.equals(oldValue, value))
.defaultIfEmpty(value != null); .defaultIfEmpty(value != null);
} }
@ -340,7 +351,9 @@ public class DatabaseMapDictionary<T, U> extends DatabaseMapDictionaryDeep<T, U,
@Override @Override
public Mono<U> removeAndGetPrevious(T keySuffix) { public Mono<U> removeAndGetPrevious(T keySuffix) {
var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix).send()); var keyMono = Mono.fromCallable(() -> serializeKeySuffixToKey(keySuffix).send());
return dictionary.remove(keyMono, LLDictionaryResultType.PREVIOUS_VALUE).handle(this::deserializeValue); return dictionary
.remove(keyMono, LLDictionaryResultType.PREVIOUS_VALUE)
.handle((valueToReceive, sink) -> deserializeValue(keySuffix, valueToReceive, sink));
} }
@Override @Override

View File

@ -502,13 +502,14 @@ public class DatabaseMapDictionaryDeep<T, U, US extends DatabaseStage<U>> extend
.dictionary .dictionary
.getRange(deepMap.resolveSnapshot(snapshot), deepMap.rangeMono) .getRange(deepMap.resolveSnapshot(snapshot), deepMap.rangeMono)
.handle((entrySend, sink) -> { .handle((entrySend, sink) -> {
K1 key1 = null;
K2 key2 = null;
try (var entry = entrySend.receive()) { try (var entry = entrySend.receive()) {
var keyBuf = entry.getKeyUnsafe(); var keyBuf = entry.getKeyUnsafe();
var valueBuf = entry.getValueUnsafe(); var valueBuf = entry.getValueUnsafe();
try {
assert keyBuf != null; assert keyBuf != null;
keyBuf.skipReadable(deepMap.keyPrefixLength); keyBuf.skipReadable(deepMap.keyPrefixLength);
K1 key1;
K2 key2;
try (var key1Buf = keyBuf.split(deepMap.keySuffixLength)) { try (var key1Buf = keyBuf.split(deepMap.keySuffixLength)) {
key1 = keySuffix1Serializer.deserialize(key1Buf); key1 = keySuffix1Serializer.deserialize(key1Buf);
} }
@ -521,13 +522,17 @@ public class DatabaseMapDictionaryDeep<T, U, US extends DatabaseStage<U>> extend
if (exMessage != null && exMessage.contains("read 0 to 0, write 0 to ")) { if (exMessage != null && exMessage.contains("read 0 to 0, write 0 to ")) {
var totalZeroBytesErrors = deepMap.totalZeroBytesErrors.incrementAndGet(); var totalZeroBytesErrors = deepMap.totalZeroBytesErrors.incrementAndGet();
if (totalZeroBytesErrors < 512 || totalZeroBytesErrors % 10000 == 0) { if (totalZeroBytesErrors < 512 || totalZeroBytesErrors % 10000 == 0) {
LOG.error("Unexpected zero-bytes value in column " + deepMap.dictionary.getDatabaseName() + ":" LOG.error("Unexpected zero-bytes value at " + deepMap.dictionary.getDatabaseName()
+ deepMap.dictionary.getColumnName() + " total=" + totalZeroBytesErrors); + ":" + deepMap.dictionary.getColumnName()
+ ":[" + key1
+ ":" + key2
+ "](" + LLUtils.toStringSafe(keyBuf) + ") total=" + totalZeroBytesErrors);
} }
sink.complete(); sink.complete();
} else { } else {
sink.error(ex); sink.error(ex);
} }
}
} catch (SerializationException ex) { } catch (SerializationException ex) {
sink.error(ex); sink.error(ex);
} }

View File

@ -92,8 +92,8 @@ public class DatabaseSingle<U> extends ResourceSupport<DatabaseStage<U>, Databas
} catch (IndexOutOfBoundsException ex) { } catch (IndexOutOfBoundsException ex) {
var exMessage = ex.getMessage(); var exMessage = ex.getMessage();
if (exMessage != null && exMessage.contains("read 0 to 0, write 0 to ")) { if (exMessage != null && exMessage.contains("read 0 to 0, write 0 to ")) {
LOG.error("Unexpected zero-bytes value in column " LOG.error("Unexpected zero-bytes value at "
+ dictionary.getDatabaseName() + ":" + dictionary.getColumnName()); + dictionary.getDatabaseName() + ":" + dictionary.getColumnName() + ":" + LLUtils.toStringSafe(key));
sink.complete(); sink.complete();
} else { } else {
sink.error(ex); sink.error(ex);

View File

@ -4,6 +4,7 @@ import static io.netty5.buffer.api.StandardAllocationTypes.OFF_HEAP;
import static it.cavallium.dbengine.database.LLUtils.INITIAL_DIRECT_READ_BYTE_BUF_SIZE_BYTES; import static it.cavallium.dbengine.database.LLUtils.INITIAL_DIRECT_READ_BYTE_BUF_SIZE_BYTES;
import static java.util.Objects.requireNonNull; import static java.util.Objects.requireNonNull;
import static org.rocksdb.KeyMayExist.KeyMayExistEnum.kExistsWithValue; import static org.rocksdb.KeyMayExist.KeyMayExistEnum.kExistsWithValue;
import static org.rocksdb.KeyMayExist.KeyMayExistEnum.kExistsWithoutValue;
import io.micrometer.core.instrument.Gauge; import io.micrometer.core.instrument.Gauge;
import io.micrometer.core.instrument.MeterRegistry; import io.micrometer.core.instrument.MeterRegistry;
@ -138,7 +139,21 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
resultBuffer.close(); resultBuffer.close();
return null; return null;
} }
// todo: kExistsWithValue is not reliable (read below),
// in some cases it should be treated as kExistsWithoutValue
case kExistsWithValue:
case kExistsWithoutValue: { case kExistsWithoutValue: {
boolean isKExistsWithoutValue = false;
if (keyMayExistState == kExistsWithoutValue) {
isKExistsWithoutValue = true;
} else {
// todo: "size == 0 || resultWritable.limit() == 0" is checked because keyMayExist is broken,
// and sometimes it returns an empty array, as if it exists
if (size == 0 || resultWritable.limit() == 0) {
isKExistsWithoutValue = true;
}
}
if (isKExistsWithoutValue) {
assert keyMayExistValueLength == 0; assert keyMayExistValueLength == 0;
resultWritable.clear(); resultWritable.clear();
// real data size // real data size
@ -148,7 +163,8 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
return null; return null;
} }
} }
case kExistsWithValue: { }
default: {
// real data size // real data size
this.lastDataSizeMetric.set(size); this.lastDataSizeMetric.set(size);
assert size >= 0; assert size >= 0;
@ -170,9 +186,6 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
return resultBuffer.writerOffset(resultWritable.limit()); return resultBuffer.writerOffset(resultWritable.limit());
} }
} }
default: {
throw new IllegalStateException();
}
} }
} catch (Throwable t) { } catch (Throwable t) {
resultBuffer.close(); resultBuffer.close();
@ -189,7 +202,9 @@ public sealed abstract class AbstractRocksDBColumn<T extends RocksDB> implements
requireNonNull(keyArray); requireNonNull(keyArray);
Holder<byte[]> data = new Holder<>(); Holder<byte[]> data = new Holder<>();
if (db.keyMayExist(cfh, readOptions, keyArray, data)) { if (db.keyMayExist(cfh, readOptions, keyArray, data)) {
if (data.getValue() != null) { // todo: "data.getValue().length > 0" is checked because keyMayExist is broken, and sometimes it
// returns an empty array, as if it exists
if (data.getValue() != null && data.getValue().length > 0) {
return LLUtils.fromByteArray(alloc, data.getValue()); return LLUtils.fromByteArray(alloc, data.getValue());
} else { } else {
byte[] result = db.get(cfh, readOptions, keyArray); byte[] result = db.get(cfh, readOptions, keyArray);

View File

@ -62,7 +62,11 @@ public final class PessimisticRocksDBColumn extends AbstractRocksDBColumn<Transa
Send<Buffer> sentPrevData; Send<Buffer> sentPrevData;
Send<Buffer> sentCurData; Send<Buffer> sentCurData;
boolean changed; boolean changed;
if (logger.isTraceEnabled()) {
logger.trace(MARKER_ROCKSDB, "Reading {} (before update lock)", LLUtils.toStringSafe(key));
}
var prevDataArray = tx.getForUpdate(readOptions, cfh, keyArray, true); var prevDataArray = tx.getForUpdate(readOptions, cfh, keyArray, true);
try {
if (logger.isTraceEnabled()) { if (logger.isTraceEnabled()) {
logger.trace(MARKER_ROCKSDB, logger.trace(MARKER_ROCKSDB,
"Reading {}: {} (before update)", "Reading {}: {} (before update)",
@ -124,6 +128,9 @@ public final class PessimisticRocksDBColumn extends AbstractRocksDBColumn<Transa
sentCurData = newData == null ? null : newData.send(); sentCurData = newData == null ? null : newData.send();
} }
} }
} finally {
tx.undoGetForUpdate(cfh, keyArray);
}
return switch (returnMode) { return switch (returnMode) {
case NOTHING -> { case NOTHING -> {
if (sentPrevData != null) { if (sentPrevData != null) {