pack Tuples with a single byte array allocation of the right size

This commit is contained in:
Alec Grieser 2019-02-25 21:59:16 -08:00
parent a74dfa5487
commit 663d750e1d
No known key found for this signature in database
GPG Key ID: CAF63551C60D3462
8 changed files with 547 additions and 237 deletions

View File

@ -20,7 +20,6 @@
package com.apple.foundationdb.tuple;
import java.math.BigInteger;
import java.nio.ByteBuffer;
import java.nio.ByteOrder;
import java.util.Arrays;
@ -154,7 +153,10 @@ public class ByteArrayUtil {
* @return a newly created array where {@code pattern} replaced with {@code replacement}
*/
public static byte[] replace(byte[] src, byte[] pattern, byte[] replacement) {
return join(replacement, split(src, pattern));
if(src == null) {
return null;
}
return replace(src, 0, src.length, pattern, replacement);
}
/**
@ -171,7 +173,69 @@ public class ByteArrayUtil {
*/
public static byte[] replace(byte[] src, int offset, int length,
byte[] pattern, byte[] replacement) {
return join(replacement, split(src, offset, length, pattern));
if(pattern == null || pattern.length == 0) {
return Arrays.copyOfRange(src, offset, offset + length);
}
ByteBuffer dest;
if(replacement == null || replacement.length != pattern.length) {
// Array might change size. This is the "tricky" case.
byte patternFirst = pattern[0];
int patternOccurrences = 0;
int currentPosition = offset;
while(currentPosition < offset + length) {
if(src[currentPosition] == patternFirst && regionEquals(src, currentPosition, pattern)) {
patternOccurrences++;
currentPosition += pattern.length;
}
else {
currentPosition++;
}
}
if(patternOccurrences == 0) {
// Pattern doesn't occur. Just return a copy of the needed region.
return Arrays.copyOfRange(src, offset, offset + length);
}
int replacementLength = (replacement == null) ? 0 : replacement.length;
int newLength = length + patternOccurrences * (replacementLength - pattern.length);
if(newLength == 0) {
return new byte[0];
}
else {
dest = ByteBuffer.allocate(newLength);
}
}
else {
// No matter what, the array will stay the same size as replacement.length = pattern.length
dest = ByteBuffer.allocate(length);
}
replace(src, offset, length, pattern, replacement, dest);
return dest.array();
}
static void replace(byte[] src, int offset, int length, byte[] pattern, byte[] replacement, ByteBuffer dest) {
if(pattern == null || pattern.length == 0) {
dest.put(src, offset, length);
return;
}
byte patternFirst = pattern[0];
int lastPosition = offset;
int currentPosition = offset;
while(currentPosition < offset + length) {
if(src[currentPosition] == patternFirst && regionEquals(src, currentPosition, pattern)) {
dest.put(src, lastPosition, currentPosition - lastPosition);
if(replacement != null) {
dest.put(replacement);
}
currentPosition += pattern.length;
lastPosition = currentPosition;
}
else {
currentPosition++;
}
}
dest.put(src, lastPosition, currentPosition - lastPosition);
}
/**
@ -203,7 +267,7 @@ public class ByteArrayUtil {
* @return a list of byte arrays from {@code src} now not containing {@code delimiter}
*/
public static List<byte[]> split(byte[] src, int offset, int length, byte[] delimiter) {
List<byte[]> parts = new LinkedList<byte[]>();
List<byte[]> parts = new LinkedList<>();
int idx = offset;
int lastSplitEnd = offset;
while(idx <= (offset+length) - delimiter.length) {
@ -225,13 +289,6 @@ public class ByteArrayUtil {
return parts;
}
static int bisectLeft(BigInteger[] arr, BigInteger i) {
int n = Arrays.binarySearch(arr, i);
if(n >= 0)
return n;
return (n + 1) * -1;
}
/**
* Compare byte arrays for equality and ordering purposes. Elements in the array
* are interpreted and compared as unsigned bytes. Neither parameter
@ -276,61 +333,6 @@ public class ByteArrayUtil {
return true;
}
/**
* Scan through an array of bytes to find the first occurrence of a specific value.
*
* @param src array to scan. Must not be {@code null}.
* @param what the value for which to search.
* @param start the index at which to start the search. If this is at or after
* the end of {@code src}, the result will always be {@code -1}.
* @param end the index one past the last entry at which to search
*
* @return return the location of the first instance of {@code value}, or
* {@code -1} if not found.
*/
static int findNext(byte[] src, byte what, int start, int end) {
for(int i = start; i < end; i++) {
if(src[i] == what)
return i;
}
return -1;
}
/**
* Gets the index of the first element after the next occurrence of the byte sequence [nm]
* @param v the bytes to scan through
* @param n first character to find
* @param m second character to find
* @param start the index at which to start the scan
*
* @return the index after the next occurrence of [nm]
*/
static int findTerminator(byte[] v, byte n, byte m, int start) {
return findTerminator(v, n, m, start, v.length);
}
/**
* Gets the index of the first element after the next occurrence of the byte sequence [nm]
* @param v the bytes to scan through
* @param n first character to find
* @param m second character to find
* @param start the index at which to start the scan
* @param end the index at which to stop the search (exclusive)
*
* @return the index after the next occurrence of [nm]
*/
static int findTerminator(byte[] v, byte n, byte m, int start, int end) {
int pos = start;
while(true) {
pos = findNext(v, n, pos, end);
if(pos < 0)
return end;
if(pos + 1 == end || v[pos+1] != m)
return pos;
pos += 2;
}
}
/**
* Computes the first key that would sort outside the range prefixed by {@code key}.
* {@code key} must be non-null, and contain at least some character this is not
@ -417,5 +419,14 @@ public class ByteArrayUtil {
return s.toString();
}
static int nullCount(byte[] val) {
int nulls = 0;
for(int i = 0; i < val.length; i++) {
if(val[i] == 0x00)
nulls += 1;
}
return nulls;
}
private ByteArrayUtil() {}
}

View File

@ -71,5 +71,48 @@ final class StringUtil {
return Character.compare(c1, c2);
}
static int packedSize(String s) {
final int strLength = s.length();
int size = 0;
int pos = 0;
while(pos < strLength) {
char c = s.charAt(pos);
if(c == '\0') {
// Null is encoded as \x00\xff
size += 2;
}
else if(c <= 0x7f) {
// ASCII code point. Only 1 byte.
size += 1;
}
else if(c <= 0x07ff) {
// 2 byte code point
size += 2;
}
else if(Character.isHighSurrogate(c)) {
if(pos + 1 < s.length() && Character.isLowSurrogate(s.charAt(pos + 1))) {
// High surrogate followed by low surrogate means the code point
// is between U+10000 and U+10FFFF, so it requires 4 bytes.
size += 4;
pos += 1;
}
else {
throw new IllegalArgumentException("malformed UTF-16 has high surrogate not followed by low surrogate");
}
}
else if(Character.isLowSurrogate(c)) {
throw new IllegalArgumentException("malformed UTF-16 has low surrogate without prior high surrogate");
}
else {
// 3 byte code point
size += 3;
}
pos += 1;
}
return size;
}
private StringUtil() {}
}

View File

@ -73,6 +73,7 @@ public class Tuple implements Comparable<Tuple>, Iterable<Object> {
private List<Object> elements;
private int memoizedHash = 0;
private byte[] packed = null;
private int memoizedPackedSize = -1;
private Tuple(List<? extends Object> elements, Object newItem) {
this(elements);
@ -83,12 +84,6 @@ public class Tuple implements Comparable<Tuple>, Iterable<Object> {
this.elements = new ArrayList<>(elements);
}
private enum VersionstampExpectations {
UNKNOWN,
HAS_INCOMPLETE,
HAS_NO_INCOMPLETE
}
/**
* Creates a copy of this {@code Tuple} with an appended last element. The parameter
* is untyped but only {@link String}, {@code byte[]}, {@link Number}s, {@link UUID}s,
@ -313,13 +308,15 @@ public class Tuple implements Comparable<Tuple>, Iterable<Object> {
byte[] packInternal(byte[] prefix, boolean copy) {
boolean hasPrefix = prefix != null && prefix.length > 1;
if(packed == null) {
byte[] result = TupleUtil.pack(elements, prefix);
byte[] result = TupleUtil.pack(elements, prefix, getPackedSize());
if(hasPrefix) {
packed = Arrays.copyOfRange(result, prefix.length, result.length);
memoizedPackedSize = packed.length;
return result;
}
else {
packed = result;
memoizedPackedSize = packed.length;
}
}
if(hasPrefix) {
@ -366,21 +363,23 @@ public class Tuple implements Comparable<Tuple>, Iterable<Object> {
* @throws IllegalArgumentException if there is not exactly one incomplete {@link Versionstamp} included in this {@code Tuple}
*/
public byte[] packWithVersionstamp(byte[] prefix) {
return TupleUtil.packWithVersionstamp(elements, prefix);
return TupleUtil.packWithVersionstamp(elements, prefix, getPackedSize());
}
byte[] packWithVersionstampInternal(byte[] prefix, boolean copy) {
boolean hasPrefix = prefix != null && prefix.length > 0;
if(packed == null) {
byte[] result = TupleUtil.packWithVersionstamp(elements, prefix);
byte[] result = TupleUtil.packWithVersionstamp(elements, prefix, getPackedSize());
if(hasPrefix) {
byte[] withoutPrefix = Arrays.copyOfRange(result, prefix.length, result.length);
TupleUtil.adjustVersionPosition(packed, -1 * prefix.length);
packed = withoutPrefix;
memoizedPackedSize = packed.length;
return result;
}
else {
packed = result;
memoizedPackedSize = packed.length;
}
}
if(hasPrefix) {
@ -398,13 +397,13 @@ public class Tuple implements Comparable<Tuple>, Iterable<Object> {
}
}
byte[] packMaybeVersionstamp(byte[] prefix) {
byte[] packMaybeVersionstamp() {
if(packed == null) {
if(hasIncompleteVersionstamp()) {
return packWithVersionstampInternal(prefix, false);
return packWithVersionstampInternal(null, false);
}
else {
return packInternal(prefix, false);
return packInternal(null, false);
}
}
else {
@ -489,6 +488,7 @@ public class Tuple implements Comparable<Tuple>, Iterable<Object> {
Tuple t = new Tuple();
t.elements = TupleUtil.unpack(bytes, offset, length);
t.packed = Arrays.copyOfRange(bytes, offset, offset + length);
t.memoizedPackedSize = length;
return t;
}
@ -727,11 +727,14 @@ public class Tuple implements Comparable<Tuple>, Iterable<Object> {
Object o = this.elements.get(index);
if(o == null) {
return null;
} else if(o instanceof Tuple) {
}
else if(o instanceof Tuple) {
return (Tuple)o;
} else if(o instanceof List<?>) {
return Tuple.fromItems((List<? extends Object>)o);
} else {
}
else if(o instanceof List<?>) {
return Tuple.fromItems((List<?>)o);
}
else {
throw new ClassCastException("Cannot convert item of type " + o.getClass() + " to tuple");
}
}
@ -824,16 +827,23 @@ public class Tuple implements Comparable<Tuple>, Iterable<Object> {
}
/**
* Get the number of bytes in the packed representation of this {@code Tuple}. Note that at the
* moment, this number is calculated by packing the {@code Tuple} and looking at its size. This method
* will memoize the result, however, so asking the same {@code Tuple} for its size multiple times
* is a fast operation.
* Get the number of bytes in the packed representation of this {@code Tuple}. This is done by summing
* the serialized sizes of all of the elements of this {@code Tuple} and does not pack everything
* into a single {@code Tuple}. The return value of this function is stored within this {@code Tuple}
* after this function has been called so that subsequent calls on the same object are fast. This method
* does not validate that there is no more than one incomplete {@link Versionstamp} in this {@code Tuple}.
*
* @return the number of bytes in the packed representation of this {@code Tuple}
*/
public int getPackedSize() {
byte[] p = packMaybeVersionstamp(null);
return p.length;
if(memoizedPackedSize < 0) {
memoizedPackedSize = getPackedSize(false);
}
return memoizedPackedSize;
}
int getPackedSize(boolean nested) {
return TupleUtil.getPackedSize(elements, nested);
}
/**
@ -871,7 +881,7 @@ public class Tuple implements Comparable<Tuple>, Iterable<Object> {
@Override
public int hashCode() {
if(memoizedHash == 0) {
memoizedHash = Arrays.hashCode(packMaybeVersionstamp(null));
memoizedHash = Arrays.hashCode(packMaybeVersionstamp());
}
return memoizedHash;
}

View File

@ -36,11 +36,10 @@ import com.apple.foundationdb.FDB;
class TupleUtil {
private static final byte nil = 0x00;
private static final BigInteger[] BIG_INT_SIZE_LIMITS;
private static final Charset UTF8;
private static final Charset UTF8 = Charset.forName("UTF-8");
private static final BigInteger LONG_MIN_VALUE = BigInteger.valueOf(Long.MIN_VALUE);
private static final BigInteger LONG_MAX_VALUE = BigInteger.valueOf(Long.MAX_VALUE);
private static final IterableComparator iterableComparator;
private static final IterableComparator iterableComparator = new IterableComparator();
private static final byte BYTES_CODE = 0x01;
private static final byte STRING_CODE = 0x02;
@ -57,26 +56,11 @@ class TupleUtil {
private static final byte[] NULL_ARR = new byte[] {nil};
private static final byte[] NULL_ESCAPED_ARR = new byte[] {nil, (byte)0xFF};
private static final byte[] BYTES_ARR = new byte[]{BYTES_CODE};
private static final byte[] STRING_ARR = new byte[]{STRING_CODE};
private static final byte[] NESTED_ARR = new byte[]{NESTED_CODE};
private static final byte[] INT_ZERO_ARR = new byte[]{INT_ZERO_CODE};
private static final byte[] FALSE_ARR = new byte[]{FALSE_CODE};
private static final byte[] TRUE_ARR = new byte[]{TRUE_CODE};
private static final byte[] VERSIONSTAMP_ARR = new byte[]{VERSIONSTAMP_CODE};
static {
BIG_INT_SIZE_LIMITS = new BigInteger[9];
for(int i = 0; i < BIG_INT_SIZE_LIMITS.length; i++) {
BIG_INT_SIZE_LIMITS[i] = (BigInteger.ONE).shiftLeft(i * 8).subtract(BigInteger.ONE);
}
UTF8 = Charset.forName("UTF-8");
iterableComparator = new IterableComparator();
}
static class DecodeState {
final List<Object> values;
int end;
int nullCount; // Basically a hack to allow findTerminator to return the terminator and null count
DecodeState() {
values = new ArrayList<>();
@ -87,15 +71,36 @@ class TupleUtil {
values.add(value);
this.end = end;
}
int findNullTerminator(byte[] bytes, int from, int to) {
nullCount = 0;
int x = from;
while(x < to) {
if(bytes[x] == 0x00) {
if(x + 1 >= to || bytes[x + 1] != (byte)0xFF) {
return x;
}
else {
nullCount++;
x += 2;
}
}
else {
x += 1;
}
}
throw new IllegalArgumentException("no terminator found for bytes starting at " + from);
}
}
static class EncodeState {
final List<byte[]> encodedValues;
final ByteBuffer encodedBytes;
int totalLength;
int versionPos;
EncodeState(int capacity) {
this.encodedValues = new ArrayList<>(capacity);
EncodeState(ByteBuffer dest) {
encodedBytes = dest;
encodedBytes.order(ByteOrder.BIG_ENDIAN);
totalLength = 0;
versionPos = -1;
}
@ -104,25 +109,52 @@ class TupleUtil {
if(versionPos >= 0 && this.versionPos >= 0) {
throw new IllegalArgumentException("Multiple incomplete Versionstamps included in Tuple");
}
encodedValues.add(encoded);
encodedBytes.put(encoded);
totalLength += encoded.length;
this.versionPos = versionPos;
return this;
}
EncodeState add(byte[] encoded) {
encodedValues.add(encoded);
encodedBytes.put(encoded);
totalLength += encoded.length;
return this;
}
}
static int byteLength(byte[] bytes) {
for(int i = 0; i < bytes.length; i++) {
if(bytes[i] == 0x00) continue;
return bytes.length - i;
EncodeState add(byte[] encoded, int offset, int length) {
encodedBytes.put(encoded, offset, length);
totalLength += length;
return this;
}
EncodeState addNullEscaped(byte[] encoded) {
int nullCount = ByteArrayUtil.nullCount(encoded);
if(nullCount == 0) {
encodedBytes.put(encoded);
}
else {
ByteArrayUtil.replace(encoded, 0, encoded.length, NULL_ARR, NULL_ESCAPED_ARR, encodedBytes);
}
return this;
}
EncodeState add(byte b) {
encodedBytes.put(b);
totalLength++;
return this;
}
EncodeState add(int i) {
encodedBytes.putInt(i);
totalLength += Integer.BYTES;
return this;
}
EncodeState add(long l) {
encodedBytes.putLong(l);
totalLength += Long.BYTES;
return this;
}
return 0;
}
// These four functions are for adjusting the encoding of floating point numbers so
@ -153,11 +185,16 @@ class TupleUtil {
return Double.longBitsToDouble(origBits);
}
// Get the number of bytes in the representation of a long.
static int byteCount(long i) {
// Get the minimal number of bytes in the representation of a long.
static int minimalByteCount(long i) {
return (Long.SIZE + 7 - Long.numberOfLeadingZeros(i >= 0 ? i : -i)) / 8;
}
static int minimalByteCount(BigInteger i) {
int bitLength = (i.compareTo(BigInteger.ZERO) >= 0) ? i.bitLength() : i.negate().bitLength();
return (bitLength + 7) / 8;
}
private static void adjustVersionPosition300(byte[] packed, int delta) {
int offsetOffset = packed.length - Short.BYTES;
ByteBuffer buffer = ByteBuffer.wrap(packed, offsetOffset, Short.BYTES).order(ByteOrder.LITTLE_ENDIAN);
@ -224,7 +261,7 @@ class TupleUtil {
state.add(NULL_ESCAPED_ARR);
}
else {
state.add(NULL_ARR);
state.add(nil);
}
}
else if(t instanceof byte[])
@ -258,133 +295,104 @@ class TupleUtil {
}
static void encode(EncodeState state, byte[] bytes) {
byte[] escaped = ByteArrayUtil.replace(bytes, NULL_ARR, NULL_ESCAPED_ARR);
state.add(BYTES_ARR).add(escaped).add(NULL_ARR);
state.add(BYTES_CODE).addNullEscaped(bytes).add(nil);
}
static void encode(EncodeState state, String s) {
byte[] escaped = ByteArrayUtil.replace(s.getBytes(UTF8), NULL_ARR, NULL_ESCAPED_ARR);
state.add(STRING_ARR).add(escaped).add(NULL_ARR);
byte[] bytes = s.getBytes(UTF8);
state.add(STRING_CODE).addNullEscaped(bytes).add(nil);
}
static void encode(EncodeState state, BigInteger i) {
//System.out.println("Encoding integral " + i);
if(i.equals(BigInteger.ZERO)) {
state.add(INT_ZERO_ARR);
state.add(INT_ZERO_CODE);
return;
}
byte[] bytes = i.toByteArray();
int n = minimalByteCount(i);
if(n > 0xff) {
throw new IllegalArgumentException("BigInteger magnitude is too large (more than 255 bytes)");
}
if(i.compareTo(BigInteger.ZERO) > 0) {
if(i.compareTo(BIG_INT_SIZE_LIMITS[BIG_INT_SIZE_LIMITS.length-1]) > 0) {
int length = byteLength(bytes);
if(length > 0xff) {
throw new IllegalArgumentException("BigInteger magnitude is too large (more than 255 bytes)");
}
byte[] intBytes = new byte[length + 2];
intBytes[0] = POS_INT_END;
intBytes[1] = (byte)(length);
System.arraycopy(bytes, bytes.length - length, intBytes, 2, length);
state.add(intBytes);
byte[] bytes = i.toByteArray();
if(n > Long.BYTES) {
state.add(POS_INT_END);
state.add((byte)n);
state.add(bytes, bytes.length - n, n);
}
else {
int n = ByteArrayUtil.bisectLeft(BIG_INT_SIZE_LIMITS, i);
assert n <= BIG_INT_SIZE_LIMITS.length;
//System.out.println(" -- integral has 'n' of " + n + " and output bytes of " + bytes.length);
byte[] intBytes = new byte[n + 1];
intBytes[0] = (byte) (INT_ZERO_CODE + n);
System.arraycopy(bytes, bytes.length - n, intBytes, 1, n);
state.add(intBytes);
state.add((byte)(INT_ZERO_CODE + n));
state.add(bytes, bytes.length - n, n);
}
}
else {
if(i.negate().compareTo(BIG_INT_SIZE_LIMITS[BIG_INT_SIZE_LIMITS.length - 1]) > 0) {
int length = byteLength(i.negate().toByteArray());
if (length > 0xff) {
throw new IllegalArgumentException("BigInteger magnitude is too large (more than 255 bytes)");
byte[] bytes = i.subtract(BigInteger.ONE).toByteArray();
if(n > Long.BYTES) {
state.add(NEG_INT_START);
state.add((byte)(n ^ 0xff));
if(bytes.length >= n) {
state.add(bytes, bytes.length - n, n);
}
BigInteger offset = BigInteger.ONE.shiftLeft(length * 8).subtract(BigInteger.ONE);
byte[] adjusted = i.add(offset).toByteArray();
byte[] intBytes = new byte[length + 2];
intBytes[0] = NEG_INT_START;
intBytes[1] = (byte) (length ^ 0xff);
if (adjusted.length >= length) {
System.arraycopy(adjusted, adjusted.length - length, intBytes, 2, length);
} else {
Arrays.fill(intBytes, 2, intBytes.length - adjusted.length, (byte) 0x00);
System.arraycopy(adjusted, 0, intBytes, intBytes.length - adjusted.length, adjusted.length);
else {
for(int x = 0; x < n - bytes.length; x++) {
state.add((byte)0x00);
}
state.add(bytes, 0, bytes.length);
}
state.add(intBytes);
}
else {
int n = ByteArrayUtil.bisectLeft(BIG_INT_SIZE_LIMITS, i.negate());
assert n >= 0 && n < BIG_INT_SIZE_LIMITS.length; // can we do this? it seems to be required for the following statement
long maxv = BIG_INT_SIZE_LIMITS[n].add(i).longValue();
byte[] adjustedBytes = ByteBuffer.allocate(8).order(ByteOrder.BIG_ENDIAN).putLong(maxv).array();
byte[] intBytes = new byte[n + 1];
intBytes[0] = (byte) (INT_ZERO_CODE - n);
System.arraycopy(adjustedBytes, adjustedBytes.length - n, intBytes, 1, n);
state.add(intBytes);
state.add((byte)(INT_ZERO_CODE - n));
if(bytes.length >= n) {
state.add(bytes, bytes.length - n, n);
}
else {
for(int x = 0; x < n - bytes.length; x++) {
state.add((byte)0x00);
}
state.add(bytes, 0, bytes.length);
}
}
}
}
static void encode(EncodeState state, long i) {
if(i == 0L) {
state.add(INT_ZERO_ARR);
state.add(INT_ZERO_CODE);
return;
}
int n = byteCount(i);
byte[] intBytes = new byte[n + 1];
int n = minimalByteCount(i);
// First byte encodes number of bytes (as difference from INT_ZERO_CODE)
intBytes[0] = (byte)(INT_ZERO_CODE + (i >= 0 ? n : -n));
state.add((byte)(INT_ZERO_CODE + (i >= 0 ? n : -n)));
// For positive integers, copy the bytes in big-endian order excluding leading 0x00 bytes.
// For negative integers, copy the bytes of the one's complement representation excluding
// the leading 0xff bytes. As Java stores negative values in two's complement, we subtract 1
// from negative values.
long val = Long.reverseBytes((i >= 0) ? i : (i - 1)) >> (Long.SIZE - 8 * n);
for(int x = 1; x < intBytes.length; x++) {
intBytes[x] = (byte)(val & 0xff);
for(int x = 0; x < n; x++) {
state.add((byte)(val & 0xff));
val >>= 8;
}
state.add(intBytes);
}
static void encode(EncodeState state, Float f) {
byte[] floatBytes = ByteBuffer.allocate(1 + Float.BYTES).order(ByteOrder.BIG_ENDIAN)
.put(FLOAT_CODE)
.putInt(encodeFloatBits(f))
.array();
state.add(floatBytes);
state.add(FLOAT_CODE).add(encodeFloatBits(f));
}
static void encode(EncodeState state, Double d) {
byte[] doubleBytes = ByteBuffer.allocate(1 + Double.BYTES).order(ByteOrder.BIG_ENDIAN)
.put(DOUBLE_CODE)
.putLong(encodeDoubleBits(d))
.array();
state.add(doubleBytes);
state.add(DOUBLE_CODE).add(encodeDoubleBits(d));
}
static void encode(EncodeState state, Boolean b) {
if(b) {
state.add(TRUE_ARR);
}
else {
state.add(FALSE_ARR);
}
state.add(b ? TRUE_CODE : FALSE_CODE);
}
static void encode(EncodeState state, UUID uuid) {
byte[] uuidBytes = ByteBuffer.allocate(17).put(UUID_CODE).order(ByteOrder.BIG_ENDIAN)
.putLong(uuid.getMostSignificantBits()).putLong(uuid.getLeastSignificantBits())
.array();
state.add(uuidBytes);
state.add(UUID_CODE).add(uuid.getMostSignificantBits()).add(uuid.getLeastSignificantBits());
}
static void encode(EncodeState state, Versionstamp v) {
state.add(VERSIONSTAMP_ARR);
state.add(VERSIONSTAMP_CODE);
if(v.isComplete()) {
state.add(v.getBytes());
}
@ -394,11 +402,11 @@ class TupleUtil {
}
static void encode(EncodeState state, List<?> value) {
state.add(NESTED_ARR);
state.add(NESTED_CODE);
for(Object t : value) {
encode(state, t, true);
}
state.add(NULL_ARR);
state.add(nil);
}
static void decode(DecodeState state, byte[] rep, int pos, int last) {
@ -411,17 +419,32 @@ class TupleUtil {
state.add(null, start);
}
else if(code == BYTES_CODE) {
int end = ByteArrayUtil.findTerminator(rep, (byte)0x0, (byte)0xff, start, last);
int end = state.findNullTerminator(rep, start, last);
//System.out.println("End of byte string: " + end);
byte[] range = ByteArrayUtil.replace(rep, start, end - start, NULL_ESCAPED_ARR, new byte[] { nil });
byte[] range;
if(state.nullCount == 0) {
range = Arrays.copyOfRange(rep, start, end);
}
else {
ByteBuffer dest = ByteBuffer.allocate(end - start - state.nullCount);
ByteArrayUtil.replace(rep, start, end - start, NULL_ESCAPED_ARR, NULL_ARR, dest);
range = dest.array();
}
//System.out.println(" -> byte string contents: '" + ArrayUtils.printable(range) + "'");
state.add(range, end + 1);
}
else if(code == STRING_CODE) {
int end = ByteArrayUtil.findTerminator(rep, (byte)0x0, (byte)0xff, start, last);
int end = state.findNullTerminator(rep, start, last);
//System.out.println("End of UTF8 string: " + end);
byte[] stringBytes = ByteArrayUtil.replace(rep, start, end - start, NULL_ESCAPED_ARR, new byte[] { nil });
String str = new String(stringBytes, UTF8);
String str;
if(state.nullCount == 0) {
str = new String(rep, start, end - start, UTF8);
}
else {
ByteBuffer dest = ByteBuffer.allocate(end - start - state.nullCount);
ByteArrayUtil.replace(rep, start, end - start, NULL_ESCAPED_ARR, NULL_ARR, dest);
str = new String(dest.array(), UTF8);
}
//System.out.println(" -> UTF8 string contents: '" + str + "'");
state.add(str, end + 1);
}
@ -442,19 +465,23 @@ class TupleUtil {
state.add(true, start);
}
else if(code == UUID_CODE) {
ByteBuffer bb = ByteBuffer.wrap(rep, start, 16).order(ByteOrder.BIG_ENDIAN);
ByteBuffer bb = ByteBuffer.wrap(rep, start, 2 * Long.BYTES).order(ByteOrder.BIG_ENDIAN);
long msb = bb.getLong();
long lsb = bb.getLong();
state.add(new UUID(msb, lsb), start + 16);
}
else if(code == POS_INT_END) {
int n = rep[start] & 0xff;
BigInteger res = new BigInteger(ByteArrayUtil.join(new byte[]{0x00}, Arrays.copyOfRange(rep, start+1, start+n+1)));
byte[] intBytes = new byte[n + 1];
System.arraycopy(rep, start + 1, intBytes, 1, n);
BigInteger res = new BigInteger(intBytes);
state.add(res, start + n + 1);
}
else if(code == NEG_INT_START) {
int n = (rep[start] ^ 0xff) & 0xff;
BigInteger origValue = new BigInteger(ByteArrayUtil.join(new byte[]{0x00}, Arrays.copyOfRange(rep, start+1, start+n+1)));
byte[] intBytes = new byte[n + 1];
System.arraycopy(rep, start + 1, intBytes, 1, n);
BigInteger origValue = new BigInteger(intBytes);
BigInteger offset = BigInteger.ONE.shiftLeft(n*8).subtract(BigInteger.ONE);
state.add(origValue.subtract(offset), start + n + 1);
}
@ -464,7 +491,7 @@ class TupleUtil {
int n = positive ? code - INT_ZERO_CODE : INT_ZERO_CODE - code;
int end = start + n;
if(rep.length < end) {
if(rep.length < last) {
throw new RuntimeException("Invalid tuple (possible truncation)");
}
@ -509,9 +536,9 @@ class TupleUtil {
else if(code == NESTED_CODE) {
DecodeState subResult = new DecodeState();
int endPos = start;
while(endPos < rep.length) {
while(endPos < last) {
if(rep[endPos] == nil) {
if(endPos + 1 < rep.length && rep[endPos+1] == (byte)0xff) {
if(endPos + 1 < last && rep[endPos+1] == (byte)0xff) {
subResult.add(null, endPos + 2);
endPos += 2;
} else {
@ -631,19 +658,27 @@ class TupleUtil {
//System.out.println("Joining whole tuple...");
}
static byte[] pack(List<Object> items, byte[] prefix) {
EncodeState state = new EncodeState(2 * items.size() + (prefix == null ? 0 : 1));
static byte[] pack(List<Object> items, byte[] prefix, int expectedSize) {
ByteBuffer dest = ByteBuffer.allocate(expectedSize + (prefix != null ? prefix.length : 0));
EncodeState state = new EncodeState(dest);
if(prefix != null) {
state.add(prefix);
}
encodeAll(state, items, prefix);
if(state.versionPos >= 0) {
throw new IllegalArgumentException("Incomplete Versionstamp included in vanilla tuple packInternal");
}
else {
return ByteArrayUtil.join(null, state.encodedValues);
return dest.array();
}
}
static byte[] packWithVersionstamp(List<Object> items, byte[] prefix) {
EncodeState state = new EncodeState(2 * items.size() + (prefix == null ? 1 : 2));
static byte[] packWithVersionstamp(List<Object> items, byte[] prefix, int expectedSize) {
ByteBuffer dest = ByteBuffer.allocate(expectedSize + (prefix != null ? prefix.length : 0));
EncodeState state = new EncodeState(dest);
if(prefix != null) {
state.add(prefix);
}
encodeAll(state, items, prefix);
if(state.versionPos < 0) {
throw new IllegalArgumentException("No incomplete Versionstamp included in tuple packInternal with versionstamp");
@ -652,15 +687,73 @@ class TupleUtil {
if(state.versionPos > 0xffff) {
throw new IllegalArgumentException("Tuple has incomplete version at position " + state.versionPos + " which is greater than the maximum " + 0xffff);
}
dest.order(ByteOrder.LITTLE_ENDIAN);
if (FDB.instance().getAPIVersion() < 520) {
state.add(ByteBuffer.allocate(Short.BYTES).order(ByteOrder.LITTLE_ENDIAN).putShort((short)state.versionPos).array());
dest.putShort((short)state.versionPos);
} else {
state.add(ByteBuffer.allocate(Integer.BYTES).order(ByteOrder.LITTLE_ENDIAN).putInt(state.versionPos).array());
dest.putInt(state.versionPos);
}
return ByteArrayUtil.join(null, state.encodedValues);
return dest.array();
}
}
static int getPackedSize(List<?> items, boolean nested) {
int packedSize = 0;
for(Object item : items) {
if(item == null)
packedSize += nested ? 2 : 1;
else if(item instanceof byte[]) {
byte[] bytes = (byte[])item;
packedSize += 2 + bytes.length + ByteArrayUtil.nullCount((byte[])item);
}
else if(item instanceof String) {
try {
int strPackedSize = StringUtil.packedSize((String)item);
packedSize += 2 + strPackedSize;
}
catch (IllegalArgumentException e) {
// The unicode was malformed. Grab the array and count the bytes
byte[] strBytes = ((String)item).getBytes(UTF8);
packedSize += 2 + strBytes.length + ByteArrayUtil.nullCount(strBytes);
}
}
else if(item instanceof Float)
packedSize += 1 + Float.BYTES;
else if(item instanceof Double)
packedSize += 1 + Double.BYTES;
else if(item instanceof Boolean)
packedSize += 1;
else if(item instanceof UUID)
packedSize += 1 + 2 * Long.BYTES;
else if(item instanceof BigInteger) {
BigInteger bigInt = (BigInteger)item;
int byteCount = minimalByteCount(bigInt);
// If byteCount <= 8, then the encoding uses 1 byte for both the size
// and type code. If byteCount > 8, then there is 1 byte for the type code
// and 1 byte for the length. In both cases, the value is followed by
// the byte count.
packedSize += byteCount + ((byteCount <= 8) ? 1 : 2);
}
else if(item instanceof Number)
packedSize += 1 + minimalByteCount(((Number)item).longValue());
else if(item instanceof Versionstamp) {
packedSize += 1 + Versionstamp.LENGTH;
Versionstamp versionstamp = (Versionstamp)item;
if(!versionstamp.isComplete()) {
int suffixSize = FDB.instance().getAPIVersion() < 520 ? Short.BYTES : Integer.BYTES;
packedSize += suffixSize;
}
}
else if(item instanceof List<?>)
packedSize += 2 + getPackedSize((List<?>)item, true);
else if(item instanceof Tuple)
packedSize += 2 + ((Tuple)item).getPackedSize(true);
else
throw new IllegalArgumentException("unknown type " + item.getClass() + " for tuple packing");
}
return packedSize;
}
static boolean hasIncompleteVersionstamp(Stream<?> items) {
return items.anyMatch(item -> {
if(item == null) {
@ -683,10 +776,10 @@ class TupleUtil {
public static void main(String[] args) {
try {
byte[] bytes = pack(Collections.singletonList(4), null);
byte[] bytes = pack(Collections.singletonList(4), null, 2);
DecodeState result = new DecodeState();
decode(result, bytes, 0, bytes.length);
int val = (int)result.values.get(0);
int val = ((Number)result.values.get(0)).intValue();
assert 4 == val;
}
catch(Exception e) {
@ -695,7 +788,7 @@ class TupleUtil {
}
try {
byte[] bytes = pack(Collections.singletonList("\u021Aest \u0218tring"), null);
byte[] bytes = pack(Collections.singletonList("\u021Aest \u0218tring"), null, 15);
DecodeState result = new DecodeState();
decode(result, bytes, 0, bytes.length);
String string = (String)result.values.get(0);

View File

@ -412,7 +412,11 @@ public class AsyncStackTester {
return inst.popParams(listSize).thenAcceptAsync(rawElements -> {
List<Tuple> tuples = new ArrayList<>(listSize);
for(Object o : rawElements) {
tuples.add(Tuple.fromBytes((byte[])o));
// Unpacking a tuple keeps around the serialized representation and uses
// it for comparison if it's available. To test semantic comparison, recreate
// the tuple from the item list.
Tuple t = Tuple.fromBytes((byte[])o);
tuples.add(Tuple.fromList(t.getItems()));
}
Collections.sort(tuples);
for(Tuple t : tuples) {

View File

@ -368,9 +368,13 @@ public class StackTester {
else if (op == StackOperation.TUPLE_SORT) {
int listSize = StackUtils.getInt(inst.popParam().join());
List<Object> rawElements = inst.popParams(listSize).join();
List<Tuple> tuples = new ArrayList<Tuple>(listSize);
List<Tuple> tuples = new ArrayList<>(listSize);
for(Object o : rawElements) {
tuples.add(Tuple.fromBytes((byte[])o));
// Unpacking a tuple keeps around the serialized representation and uses
// it for comparison if it's available. To test semantic comparison, recreate
// the tuple from the item list.
Tuple t = Tuple.fromBytes((byte[])o);
tuples.add(Tuple.fromList(t.getItems()));
}
Collections.sort(tuples);
for(Tuple t : tuples) {

View File

@ -16,7 +16,8 @@ public class TuplePerformanceTest {
private enum GeneratedTypes {
ALL,
LONG,
FLOATING_POINT
FLOATING_POINT,
STRING_LIKE
}
private final Random r;
@ -77,7 +78,7 @@ public class TuplePerformanceTest {
values.add(nested);
}
}
return Tuple.fromItems(values);
return Tuple.fromList(values);
}
public Tuple createLongsTuple(int length) {
@ -91,7 +92,7 @@ public class TuplePerformanceTest {
}
values.add(val);
}
return Tuple.fromItems(values);
return Tuple.fromList(values);
}
public Tuple createFloatingPointTuple(int length) {
@ -112,7 +113,41 @@ public class TuplePerformanceTest {
values.add(Double.longBitsToDouble(r.nextLong()));
}
}
return Tuple.fromItems(values);
return Tuple.fromList(values);
}
public Tuple createStringLikeTuple(int length) {
List<Object> values = new ArrayList<>(length);
for(int i = 0; i < length; i++) {
double choice = r.nextDouble();
if(choice < 0.4) {
byte[] arr = new byte[r.nextInt(20)];
r.nextBytes(arr);
values.add(arr);
}
else if(choice < 0.8) {
// Random ASCII codepoints
int[] codepoints = new int[r.nextInt(20)];
for(int x = 0; x < codepoints.length; x++) {
codepoints[x] = r.nextInt(0x7F);
}
values.add(new String(codepoints, 0, codepoints.length));
}
else if(choice < 0.9) {
// All zeroes
byte[] zeroes = new byte[r.nextInt(20)];
values.add(zeroes);
}
else {
// Random Unicode codepoints
int[] codepoints = new int[r.nextInt(20)];
for(int x = 0; x < codepoints.length; x++) {
codepoints[x] = r.nextInt(0x10FFFF);
}
values.add(new String(codepoints, 0, codepoints.length));
}
}
return Tuple.fromList(values);
}
public Tuple createTuple(int length) {
@ -123,6 +158,8 @@ public class TuplePerformanceTest {
return createLongsTuple(length);
case FLOATING_POINT:
return createFloatingPointTuple(length);
case STRING_LIKE:
return createStringLikeTuple(length);
default:
throw new IllegalStateException("unknown generated types " + generatedTypes);
}
@ -143,6 +180,7 @@ public class TuplePerformanceTest {
long unpackNanos = 0L;
long equalsNanos = 0L;
long equalsArrayNanos = 0L;
long sizeNanos = 0L;
long hashNanos = 0L;
long secondHashNanos = 0L;
long subspacePackNanos = 0L;
@ -150,6 +188,9 @@ public class TuplePerformanceTest {
long totalLength = 0L;
long totalBytes = 0L;
for(int i = 0; i < iterations; i++) {
if(i % 100_000 == 0) {
System.out.println(" iteration " + i);
}
int length = r.nextInt(20);
Tuple t = createTuple(length);
@ -157,8 +198,8 @@ public class TuplePerformanceTest {
byte[] serialized = t.pack();
long endNanos = System.nanoTime();
packNanos += endNanos - startNanos;
totalLength += length;
totalBytes += serialized.length;
totalLength += t.size();
totalBytes += t.getPackedSize();
startNanos = System.nanoTime();
Tuple t2 = Tuple.fromBytes(serialized);
@ -182,6 +223,15 @@ public class TuplePerformanceTest {
endNanos = System.nanoTime();
equalsArrayNanos += endNanos - startNanos;
tCopy = Tuple.fromList(t.getItems());
startNanos = System.nanoTime();
int size = tCopy.getPackedSize();
endNanos = System.nanoTime();
if (size != t.pack().length) {
throw new RuntimeException("packed size did not match actual packed length: " + t + " -- " + " " + tCopy.getPackedSize() + " instead of " + t.getPackedSize());
}
sizeNanos += endNanos - startNanos;
startNanos = System.nanoTime();
byte[] subspacePacked = subspace.pack(t);
endNanos = System.nanoTime();
@ -229,6 +279,8 @@ public class TuplePerformanceTest {
System.out.printf(" Equals time per tuple: %f \u03BCs%n", equalsNanos * 1e-3 / iterations);
System.out.printf(" Equals time (using packed): %f s%n", equalsArrayNanos * 1e-9);
System.out.printf(" Equals time (using packed) per tuple: %f \u03BCs%n", equalsArrayNanos * 1e-3 / iterations);
System.out.printf(" Size time: %f s%n", sizeNanos * 1e-9);
System.out.printf(" Size time per tuple: %f \u03BCs%n", sizeNanos * 1e-3 / iterations);
System.out.printf(" Subspace pack time: %f s%n", subspacePackNanos * 1e-9);
System.out.printf(" Subspace pack time per tuple: %f \u03BCs%n", subspacePackNanos * 1e-3 / iterations);
System.out.printf(" Subspace unpack time: %f s%n", subspaceUnpackNanos * 1e-9);

View File

@ -20,10 +20,6 @@
package com.apple.foundationdb.test;
import com.apple.foundationdb.TransactionContext;
import com.apple.foundationdb.tuple.ByteArrayUtil;
import com.apple.foundationdb.tuple.Tuple;
import java.math.BigInteger;
import java.util.ArrayList;
import java.util.Arrays;
@ -31,6 +27,11 @@ import java.util.List;
import java.util.Objects;
import java.util.UUID;
import com.apple.foundationdb.TransactionContext;
import com.apple.foundationdb.tuple.ByteArrayUtil;
import com.apple.foundationdb.tuple.Tuple;
import com.apple.foundationdb.tuple.Versionstamp;
public class TupleTest {
private static final byte FF = (byte)0xff;
@ -40,6 +41,7 @@ public class TupleTest {
// FDB fdb = FDB.selectAPIVersion(610);
serializedForms();
comparisons();
replaceTests();
/*
try(Database db = fdb.open()) {
runTests(reps, db);
@ -70,6 +72,7 @@ public class TupleTest {
private static void serializedForms() {
List<TupleSerialization> serializations = new ArrayList<>();
TupleSerialization.addAll(serializations,
Tuple.from(), new byte[0],
Tuple.from(0L), new byte[]{0x14},
Tuple.from(BigInteger.ZERO), new byte[]{0x14},
Tuple.from(1L), new byte[]{0x15, 0x01},
@ -116,6 +119,9 @@ public class TupleTest {
Tuple.from(Double.longBitsToDouble(Long.MAX_VALUE)), new byte[]{0x21, FF, FF, FF, FF, FF, FF, FF, FF},
Tuple.from(Float.intBitsToFloat(~0)), new byte[]{0x20, 0x00, 0x00, 0x00, 0x00},
Tuple.from(Double.longBitsToDouble(~0L)), new byte[]{0x21, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00, 0x00},
Tuple.from((Object)new byte[0]), new byte[]{0x01, 0x00},
Tuple.from((Object)new byte[]{0x01, 0x02, 0x03}), new byte[]{0x01, 0x01, 0x02, 0x03, 0x00},
Tuple.from((Object)new byte[]{0x00, 0x00, 0x00, 0x04}), new byte[]{0x01, 0x00, FF, 0x00, FF, 0x00, FF, 0x04, 0x00},
Tuple.from(""), new byte[]{0x02, 0x00},
Tuple.from("hello"), new byte[]{0x02, 'h', 'e', 'l', 'l', 'o', 0x00},
Tuple.from("\u4e2d\u6587"), new byte[]{0x02, (byte)0xe4, (byte)0xb8, (byte)0xad, (byte)0xe6, (byte)0x96, (byte)0x87, 0x00},
@ -123,17 +129,42 @@ public class TupleTest {
Tuple.from(new String(new int[]{0x1f525}, 0, 1)), new byte[]{0x02, (byte)0xf0, (byte)0x9f, (byte)0x94, (byte)0xa5, 0x00},
Tuple.from("\ud83d\udd25"), new byte[]{0x02, (byte)0xf0, (byte)0x9f, (byte)0x94, (byte)0xa5, 0x00},
Tuple.from("\ud83e\udd6f"), new byte[]{0x02, (byte)0xf0, (byte)0x9f, (byte)0xa5, (byte)0xaf, 0x00},
Tuple.from("\ud83d"), new byte[]{0x02, 0x3f, 0x00},
Tuple.from("\udd25\ud83e\udd6f"), new byte[]{0x02, 0x3f, (byte)0xf0, (byte)0x9f, (byte)0xa5, (byte)0xaf, 0x00}, // malformed string - low surrogate without high surrogate
Tuple.from("a\udd25\ud83e\udd6f"), new byte[]{0x02, 'a', 0x3f, (byte)0xf0, (byte)0x9f, (byte)0xa5, (byte)0xaf, 0x00} // malformed string - low surrogate without high surrogate
Tuple.from("a\udd25\ud83e\udd6f"), new byte[]{0x02, 'a', 0x3f, (byte)0xf0, (byte)0x9f, (byte)0xa5, (byte)0xaf, 0x00}, // malformed string - low surrogate without high surrogate
Tuple.from(Tuple.from((Object)null)), new byte[]{0x05, 0x00, FF, 0x00},
Tuple.from(Tuple.from(null, "hello")), new byte[]{0x05, 0x00, FF, 0x02, 'h', 'e', 'l', 'l', 'o', 0x00, 0x00},
Tuple.from(Arrays.asList(null, "hello")), new byte[]{0x05, 0x00, FF, 0x02, 'h', 'e', 'l', 'l', 'o', 0x00, 0x00},
Tuple.from(Tuple.from(null, "hell\0")), new byte[]{0x05, 0x00, FF, 0x02, 'h', 'e', 'l', 'l', 0x00, FF, 0x00, 0x00},
Tuple.from(Arrays.asList(null, "hell\0")), new byte[]{0x05, 0x00, FF, 0x02, 'h', 'e', 'l', 'l', 0x00, FF, 0x00, 0x00},
Tuple.from(Tuple.from((Object)null), "hello"), new byte[]{0x05, 0x00, FF, 0x00, 0x02, 'h', 'e', 'l', 'l', 'o', 0x00},
Tuple.from(Tuple.from((Object)null), "hello", new byte[]{0x01, 0x00}, new byte[0]), new byte[]{0x05, 0x00, FF, 0x00, 0x02, 'h', 'e', 'l', 'l', 'o', 0x00, 0x01, 0x01, 0x00, FF, 0x00, 0x01, 0x00},
Tuple.from(new UUID(0xba5eba11, 0x5ca1ab1e)), new byte[]{0x30, FF, FF, FF, FF, (byte)0xba, 0x5e, (byte)0xba, 0x11, 0x00, 0x00, 0x00, 0x00, 0x5c, (byte)0xa1, (byte)0xab, 0x1e},
Tuple.from(false), new byte[]{0x26},
Tuple.from(true), new byte[]{0x27},
Tuple.from((short)0x3019), new byte[]{0x16, 0x30, 0x19},
Tuple.from((byte)0x03), new byte[]{0x15, 0x03},
Tuple.from(Versionstamp.complete(new byte[]{(byte)0xaa, (byte)0xbb, (byte)0xcc, (byte)0xdd, (byte)0xee, FF, 0x00, 0x01, 0x02, 0x03})), new byte[]{0x33, (byte)0xaa, (byte)0xbb, (byte)0xcc, (byte)0xdd, (byte)0xee, FF, 0x00, 0x01, 0x02, 0x03, 0x00, 0x00},
Tuple.from(Versionstamp.complete(new byte[]{0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a}, 657)), new byte[]{0x33, 0x01, 0x02, 0x03, 0x04, 0x05, 0x06, 0x07, 0x08, 0x09, 0x0a, 0x02, (byte)0x91}
);
Tuple bigTuple = new Tuple();
List<byte[]> serializedForms = new ArrayList<>();
for(TupleSerialization serialization : serializations) {
bigTuple = bigTuple.addAll(serialization.tuple);
serializedForms.add(serialization.serialization);
}
serializations.add(new TupleSerialization(bigTuple, ByteArrayUtil.join(null, serializedForms)));
for(TupleSerialization serialization : serializations) {
System.out.println("Packing " + serialization.tuple + " (expecting: " + ByteArrayUtil.printable(serialization.serialization) + ")");
if(serialization.tuple.getPackedSize() != serialization.serialization.length) {
throw new RuntimeException("Tuple " + serialization.tuple + " packed size " + serialization.tuple.getPackedSize() + " does not match expected packed size " + serialization.serialization.length);
}
if(!Arrays.equals(serialization.tuple.pack(), serialization.serialization)) {
throw new RuntimeException("Tuple " + serialization.tuple + " has serialization " + ByteArrayUtil.printable(serialization.tuple.pack()) +
" which does not match expected serialization " + ByteArrayUtil.printable(serialization.serialization));
}
if(!Objects.equals(serialization.tuple, Tuple.fromBytes(serialization.serialization))) {
if(!Objects.equals(serialization.tuple, Tuple.fromItems(Tuple.fromBytes(serialization.serialization).getItems()))) {
throw new RuntimeException("Tuple " + serialization.tuple + " does not match deserialization " + Tuple.fromBytes(serialization.serialization) +
" which comes from serialization " + ByteArrayUtil.printable(serialization.serialization));
}
@ -176,6 +207,16 @@ public class TupleTest {
Tuple.from((Object)new byte[]{0x00, FF}),
Tuple.from((Object)new byte[]{0x7f}),
Tuple.from((Object)new byte[]{(byte)0x80}),
Tuple.from(null, new byte[0]),
Tuple.from(null, new byte[]{0x00}),
Tuple.from(null, new byte[]{0x00, FF}),
Tuple.from(null, new byte[]{0x7f}),
Tuple.from(null, new byte[]{(byte)0x80}),
Tuple.from(Tuple.from(null, new byte[0])),
Tuple.from(Tuple.from(null, new byte[]{0x00})),
Tuple.from(Tuple.from(null, new byte[]{0x00, FF})),
Tuple.from(Tuple.from(null, new byte[]{0x7f})),
Tuple.from(Tuple.from(null, new byte[]{(byte)0x80})),
Tuple.from("a"),
Tuple.from("\u03bc\u03ac\u03b8\u03b7\u03bc\u03b1"),
Tuple.from("\u03bc\u03b1\u0301\u03b8\u03b7\u03bc\u03b1"),
@ -195,7 +236,18 @@ public class TupleTest {
Tuple.from(new UUID(-1, 0)),
Tuple.from(new UUID(-1, -1)),
Tuple.from(new UUID(1, -1)),
Tuple.from(new UUID(1, 1))
Tuple.from(new UUID(1, 1)),
Tuple.from(false),
Tuple.from(true),
Tuple.from(Arrays.asList(0, 1, 2)),
Tuple.from(Arrays.asList(0, 1), "hello"),
Tuple.from(Arrays.asList(0, 1), "help"),
Tuple.from(Versionstamp.complete(new byte[]{0x0a, (byte)0xbb, (byte)0xcc, (byte)0xdd, (byte)0xee, FF, 0x00, 0x01, 0x02, 0x03})),
Tuple.from(Versionstamp.complete(new byte[]{(byte)0xaa, (byte)0xbb, (byte)0xcc, (byte)0xdd, (byte)0xee, FF, 0x00, 0x01, 0x02, 0x03})),
Tuple.from(Versionstamp.complete(new byte[]{(byte)0xaa, (byte)0xbb, (byte)0xcc, (byte)0xdd, (byte)0xee, FF, 0x00, 0x01, 0x02, 0x03}, 1)),
Tuple.from(Versionstamp.complete(new byte[]{(byte)0xaa, (byte)0xbb, (byte)0xcc, (byte)0xdd, (byte)0xee, FF, 0x00, 0x01, 0x02, 0x03}, 0xa101)),
Tuple.from(Versionstamp.complete(new byte[]{(byte)0xaa, (byte)0xbb, (byte)0xcc, (byte)0xdd, (byte)0xee, FF, 0x00, 0x01, 0x02, 0x03}, 65535))
);
for(Tuple t1 : tuples) {
@ -209,6 +261,47 @@ public class TupleTest {
if(Integer.signum(semanticComparison) != Integer.signum(byteComparison)) {
throw new RuntimeException("Tuple t1 and t2 comparison mismatched: semantic = " + semanticComparison + " while byte order = " + byteComparison);
}
int implicitByteComparison = t1.compareTo(t2);
if(Integer.signum(semanticComparison) != Integer.signum(implicitByteComparison)) {
throw new RuntimeException("Tuple t1 and t2 comparison mismatched: semantic = " + semanticComparison + " while implicit byte order = " + implicitByteComparison);
}
}
}
}
// These should be in ArrayUtilTest, but those can't be run at the moment, so here they go.
private static void replaceTests() {
List<byte[]> arrays = Arrays.asList(
new byte[]{0x01, 0x02, 0x01, 0x02}, new byte[]{0x01, 0x02}, new byte[]{0x03, 0x04}, new byte[]{0x03, 0x04, 0x03, 0x04},
new byte[]{0x01, 0x02, 0x01, 0x02}, new byte[]{0x01, 0x02}, new byte[]{0x03}, new byte[]{0x03, 0x03},
new byte[]{0x01, 0x02, 0x01, 0x02}, new byte[]{0x01, 0x02}, new byte[]{0x03, 0x04, 0x05}, new byte[]{0x03, 0x04, 0x05, 0x03, 0x04, 0x05},
new byte[]{0x00, 0x01, 0x02, 0x00, 0x01, 0x02, 0x00}, new byte[]{0x01, 0x02}, new byte[]{0x03, 0x04, 0x05}, new byte[]{0x00, 0x03, 0x04, 0x05, 0x00, 0x03, 0x04, 0x05, 0x00},
new byte[]{0x01, 0x01, 0x01, 0x01}, new byte[]{0x01, 0x02}, new byte[]{0x03, 0x04}, new byte[]{0x01, 0x01, 0x01, 0x01},
new byte[]{0x01, 0x01, 0x01, 0x01}, new byte[]{0x01, 0x02}, new byte[]{0x03}, new byte[]{0x01, 0x01, 0x01, 0x01},
new byte[]{0x01, 0x01, 0x01, 0x01}, new byte[]{0x01, 0x02}, new byte[]{0x03, 0x04, 0x05}, new byte[]{0x01, 0x01, 0x01, 0x01},
new byte[]{0x01, 0x01, 0x01, 0x01, 0x01}, new byte[]{0x01, 0x01}, new byte[]{0x03, 0x04, 0x05}, new byte[]{0x03, 0x04, 0x05, 0x03, 0x04, 0x05, 0x01},
new byte[]{0x01, 0x01, 0x01, 0x01, 0x01}, new byte[]{0x01, 0x01}, new byte[]{0x03, 0x04}, new byte[]{0x03, 0x04, 0x03, 0x04, 0x01},
new byte[]{0x01, 0x01, 0x01, 0x01, 0x01}, new byte[]{0x01, 0x01}, new byte[]{0x03}, new byte[]{0x03, 0x03, 0x01},
new byte[]{0x01, 0x02, 0x01, 0x02}, new byte[]{0x01, 0x02}, null, new byte[0],
new byte[]{0x01, 0x02, 0x01, 0x02}, new byte[]{0x01, 0x02}, new byte[0], new byte[0],
new byte[]{0x01, 0x02, 0x01, 0x02}, null, new byte[]{0x04}, new byte[]{0x01, 0x02, 0x01, 0x02},
new byte[]{0x01, 0x02, 0x01, 0x02}, new byte[0], new byte[]{0x04}, new byte[]{0x01, 0x02, 0x01, 0x02},
null, new byte[]{0x01, 0x02}, new byte[]{0x04}, null
);
for(int i = 0; i < arrays.size(); i += 4) {
byte[] src = arrays.get(i);
byte[] pattern = arrays.get(i + 1);
byte[] replacement = arrays.get(i + 2);
byte[] expectedResults = arrays.get(i + 3);
byte[] results = ByteArrayUtil.replace(src, pattern, replacement);
if(!Arrays.equals(results, expectedResults)) {
throw new RuntimeException("results " + ByteArrayUtil.printable(results) + " did not match expected results " +
ByteArrayUtil.printable(expectedResults) + " when replacing " + ByteArrayUtil.printable(pattern) +
" with " + ByteArrayUtil.printable(replacement) + " in " + ByteArrayUtil.printable(src));
}
if(src != null && src == results) {
throw new RuntimeException("src and results array are pointer-equal when replacing " + ByteArrayUtil.printable(pattern) +
" with " + ByteArrayUtil.printable(replacement) + " in " + ByteArrayUtil.printable(src));
}
}
}