deserializeCompressedPrimitives(
compressStarted = true;
addIncReaderIndexExpr(groupExpressions, buffer, acc);
}
- fieldValue = new Invoke(buffer, "readVarLong", PRIMITIVE_LONG_TYPE);
+ fieldValue = LongSerializer.readLong(buffer, fury.longEncoding());
}
} else {
throw new IllegalStateException("impossible");
diff --git a/java/fury-core/src/main/java/io/fury/config/Config.java b/java/fury-core/src/main/java/io/fury/config/Config.java
index 1a38c79590..adb399dc7a 100644
--- a/java/fury-core/src/main/java/io/fury/config/Config.java
+++ b/java/fury-core/src/main/java/io/fury/config/Config.java
@@ -46,6 +46,7 @@ public class Config implements Serializable {
private final boolean compressString;
private final boolean compressInt;
private final boolean compressLong;
+ private final LongEncoding longEncoding;
private final boolean requireClassRegistration;
private final boolean registerGuavaTypes;
private final boolean shareMetaContext;
@@ -61,7 +62,8 @@ public Config(FuryBuilder builder) {
timeRefIgnored = !trackingRef || builder.timeRefIgnored;
compressString = builder.compressString;
compressInt = builder.compressInt;
- compressLong = builder.compressLong;
+ longEncoding = builder.longEncoding;
+ compressLong = longEncoding != LongEncoding.LE_RAW_BYTES;
requireClassRegistration = builder.requireClassRegistration;
registerGuavaTypes = builder.registerGuavaTypes;
codeGenEnabled = builder.codeGenEnabled;
@@ -137,6 +139,11 @@ public boolean compressLong() {
return compressLong;
}
+ /** Returns long encoding. */
+ public LongEncoding longEncoding() {
+ return longEncoding;
+ }
+
public boolean requireClassRegistration() {
return requireClassRegistration;
}
diff --git a/java/fury-core/src/main/java/io/fury/config/FuryBuilder.java b/java/fury-core/src/main/java/io/fury/config/FuryBuilder.java
index e4578fcbe6..b729aaa3d4 100644
--- a/java/fury-core/src/main/java/io/fury/config/FuryBuilder.java
+++ b/java/fury-core/src/main/java/io/fury/config/FuryBuilder.java
@@ -27,6 +27,7 @@
import io.fury.serializer.TimeSerializers;
import io.fury.util.LoggerFactory;
import io.fury.util.Platform;
+import java.util.Objects;
import java.util.concurrent.TimeUnit;
import org.slf4j.Logger;
@@ -60,7 +61,7 @@ public final class FuryBuilder {
boolean timeRefIgnored = true;
ClassLoader classLoader;
boolean compressInt = true;
- boolean compressLong = false;
+ public LongEncoding longEncoding = LongEncoding.SLI;
boolean compressString = true;
CompatibleMode compatibleMode = CompatibleMode.SCHEMA_CONSISTENT;
boolean checkJdkClassSerializable = true;
@@ -115,7 +116,7 @@ public FuryBuilder ignoreTimeRef(boolean ignoreTimeRef) {
/** Use variable length encoding for int/long. */
public FuryBuilder withNumberCompressed(boolean numberCompressed) {
this.compressInt = numberCompressed;
- this.compressLong = numberCompressed;
+ withLongCompressed(numberCompressed);
return this;
}
@@ -125,9 +126,17 @@ public FuryBuilder withIntCompressed(boolean intCompressed) {
return this;
}
- /** Use variable length encoding for long. */
+ /**
+ * Use variable length encoding for long. Enabled by default, use {@link LongEncoding#SLI} (Small
+ * long as int) for long encoding.
+ */
public FuryBuilder withLongCompressed(boolean longCompressed) {
- this.compressLong = longCompressed;
+ return withLongCompressed(longCompressed ? LongEncoding.SLI : LongEncoding.LE_RAW_BYTES);
+ }
+
+ /** Use variable length encoding for long. */
+ public FuryBuilder withLongCompressed(LongEncoding longEncoding) {
+ this.longEncoding = Objects.requireNonNull(longEncoding);
return this;
}
diff --git a/java/fury-core/src/main/java/io/fury/config/LongEncoding.java b/java/fury-core/src/main/java/io/fury/config/LongEncoding.java
new file mode 100644
index 0000000000..7c8c82af2a
--- /dev/null
+++ b/java/fury-core/src/main/java/io/fury/config/LongEncoding.java
@@ -0,0 +1,45 @@
+/*
+ * Copyright 2023 The Fury 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 io.fury.config;
+
+/**
+ * Encoding option for long. Default encoding is fury SLI(Small long as int) encoding: {@link #SLI}.
+ *
+ * @author chaokunyang
+ */
+public enum LongEncoding {
+ /**
+ * Fury SLI(Small long as int) Encoding:
+ * If long is in [0xc0000000, 0x3fffffff], encode as 4 bytes int: `| little-endian: ((int)
+ * value) << 1 |`
+ * Otherwise write as 9 bytes: `| 0b1 | little-endian 8bytes long |`.
+ *
+ * Faster than {@link #PVL}, but compression is not good as {@link #PVL} such as for ints
+ * in short range.
+ */
+ SLI,
+ /**
+ * Fury Progressive Variable-length Long Encoding:
+ *
positive long format: first bit in every byte indicate whether has next byte, then next
+ * byte should be read util first bit is unset.
+ * Negative number will be converted to positive number by ` (v << 1) ^ (v >> 63)` to reduce
+ * cost of small negative numbers.
+ */
+ PVL,
+ /** Write long as little endian 8bytes, no compression. */
+ LE_RAW_BYTES,
+}
diff --git a/java/fury-core/src/main/java/io/fury/memory/MemoryBuffer.java b/java/fury-core/src/main/java/io/fury/memory/MemoryBuffer.java
index 526a0bbc4d..b4ecb373a6 100644
--- a/java/fury-core/src/main/java/io/fury/memory/MemoryBuffer.java
+++ b/java/fury-core/src/main/java/io/fury/memory/MemoryBuffer.java
@@ -21,6 +21,7 @@
import static com.google.common.base.Preconditions.checkArgument;
import com.google.common.base.Preconditions;
+import io.fury.annotation.CodegenInvoke;
import io.fury.util.Platform;
import java.nio.BufferOverflowException;
import java.nio.BufferUnderflowException;
@@ -1633,6 +1634,7 @@ public int writeVarLong(long value) {
return unsafeWritePositiveVarLong(value);
}
+ @CodegenInvoke
public int unsafeWriteVarLong(long value) {
value = (value << 1) ^ (value >> 63);
return unsafeWritePositiveVarLong(value);
@@ -1818,6 +1820,92 @@ private long readPositiveVarLongSlow() {
return result;
}
+ /**
+ * Write long using fury SLI(Small long as int) encoding. If long is in [0xc0000000, 0x3fffffff],
+ * encode as 4 bytes int: | little-endian: ((int) value) << 1 |; Otherwise write as 9 bytes: | 0b1
+ * | little-endian 8bytes long |
+ */
+ public int writeSliLong(long value) {
+ ensure(writerIndex + 9);
+ return unsafeWriteSliLong(value);
+ }
+
+ private static final int HALF_MAX_INT_VALUE = Integer.MAX_VALUE / 2;
+ private static final int HALF_MIN_INT_VALUE = Integer.MIN_VALUE / 2;
+ private static final byte BIG_LONG_FLAG = 0b1; // bit 0 set, means big long.
+
+ /** Write long using fury SLI(Small Long as Int) encoding. */
+ public int unsafeWriteSliLong(long value) {
+ final int writerIndex = this.writerIndex;
+ final long pos = address + writerIndex;
+ if (value >= HALF_MIN_INT_VALUE && value <= HALF_MAX_INT_VALUE) {
+ // write:
+ // 00xxx -> 0xxx
+ // 11xxx -> 1xxx
+ // read:
+ // 0xxx -> 00xxx
+ // 1xxx -> 11xxx
+ int v = ((int) value) << 1; // bit 0 unset, means int.
+ if (LITTLE_ENDIAN) {
+ UNSAFE.putInt(heapMemory, pos, v);
+ } else {
+ UNSAFE.putInt(heapMemory, pos, Integer.reverseBytes(v));
+ }
+ this.writerIndex += 4;
+ return 4;
+ } else {
+ UNSAFE.putByte(heapMemory, pos, BIG_LONG_FLAG);
+ if (LITTLE_ENDIAN) {
+ UNSAFE.putLong(heapMemory, pos + 1, value);
+ } else {
+ UNSAFE.putLong(heapMemory, pos + 1, Long.reverseBytes(value));
+ }
+ this.writerIndex += 9;
+ return 9;
+ }
+ }
+
+ /** Read fury SLI(Small Long as Int) encoded long. */
+ public long readSliLong() {
+ int readIdx = readerIndex;
+ final long pos = address + readIdx;
+ int size = this.size;
+ if (BoundsChecking.BOUNDS_CHECKING_ENABLED && readIdx > size - 4) {
+ throwIndexOutOfBoundsException(readIdx, size, 4);
+ }
+ if (LITTLE_ENDIAN) {
+ int i = UNSAFE.getInt(heapMemory, pos);
+ if ((i & 0b1) != 0b1) {
+ readerIndex = readIdx + 4;
+ return i >> 1;
+ } else {
+ if (BoundsChecking.BOUNDS_CHECKING_ENABLED && readIdx > size - 9) {
+ throwIndexOutOfBoundsException(readIdx, size, 9);
+ }
+ readerIndex = readIdx + 9;
+ return UNSAFE.getLong(heapMemory, pos + 1);
+ }
+ } else {
+ int i = Integer.reverseBytes(UNSAFE.getInt(heapMemory, pos));
+ if ((i & 0b1) != 0b1) {
+ readerIndex = readIdx + 4;
+ return i >> 1;
+ } else {
+ if (BoundsChecking.BOUNDS_CHECKING_ENABLED && readIdx > size - 9) {
+ throwIndexOutOfBoundsException(readIdx, size, 9);
+ }
+ readerIndex = readIdx + 9;
+ return Long.reverseBytes(UNSAFE.getLong(heapMemory, pos + 1));
+ }
+ }
+ }
+
+ private void throwIndexOutOfBoundsException(int readIdx, int size, int need) {
+ throw new IndexOutOfBoundsException(
+ String.format(
+ "readerIndex(%d) + length(%d) exceeds size(%d): %s", readIdx, need, size, this));
+ }
+
public void writeBytes(byte[] bytes) {
writeBytes(bytes, 0, bytes.length);
}
diff --git a/java/fury-core/src/main/java/io/fury/serializer/CompatibleSerializer.java b/java/fury-core/src/main/java/io/fury/serializer/CompatibleSerializer.java
index 1f338d89d5..c6dc0c8469 100644
--- a/java/fury-core/src/main/java/io/fury/serializer/CompatibleSerializer.java
+++ b/java/fury-core/src/main/java/io/fury/serializer/CompatibleSerializer.java
@@ -186,11 +186,7 @@ private void writeFieldValue(
buffer.writeFloat((Float) fieldValue);
return;
case ClassResolver.PRIMITIVE_LONG_CLASS_ID:
- if (fury.compressLong()) {
- buffer.writeVarLong((Long) fieldValue);
- } else {
- buffer.writeLong((Long) fieldValue);
- }
+ fury.writeLong(buffer, (Long) fieldValue);
return;
case ClassResolver.PRIMITIVE_DOUBLE_CLASS_ID:
buffer.writeDouble((Double) fieldValue);
@@ -589,11 +585,7 @@ private Object readFieldValue(FieldResolver.FieldInfo fieldInfo, MemoryBuffer bu
case ClassResolver.PRIMITIVE_FLOAT_CLASS_ID:
return buffer.readFloat();
case ClassResolver.PRIMITIVE_LONG_CLASS_ID:
- if (fury.compressLong()) {
- return buffer.readVarLong();
- } else {
- return buffer.readLong();
- }
+ return fury.readLong(buffer);
case ClassResolver.PRIMITIVE_DOUBLE_CLASS_ID:
return buffer.readDouble();
case ClassResolver.STRING_CLASS_ID:
diff --git a/java/fury-core/src/main/java/io/fury/serializer/MetaSharedSerializer.java b/java/fury-core/src/main/java/io/fury/serializer/MetaSharedSerializer.java
index 683740fa6d..e1c7c5dc3c 100644
--- a/java/fury-core/src/main/java/io/fury/serializer/MetaSharedSerializer.java
+++ b/java/fury-core/src/main/java/io/fury/serializer/MetaSharedSerializer.java
@@ -91,7 +91,7 @@ public MetaSharedSerializer(Fury fury, Class type, ClassDef classDef) {
Collection descriptors = consolidateFields(fury.getClassResolver(), type, classDef);
DescriptorGrouper descriptorGrouper =
DescriptorGrouper.createDescriptorGrouper(
- descriptors, true, fury.compressInt(), fury.compressLong());
+ descriptors, true, fury.compressInt(), fury.getConfig().compressLong());
// d.getField() may be null if not exists in this class when meta share enabled.
isRecord = RecordUtils.isRecord(type);
if (isRecord) {
@@ -273,11 +273,7 @@ static boolean skipPrimitiveFieldValueFailed(Fury fury, short classId, MemoryBuf
buffer.increaseReaderIndex(4);
return false;
case ClassResolver.PRIMITIVE_LONG_CLASS_ID:
- if (fury.compressLong()) {
- buffer.readVarLong();
- } else {
- buffer.increaseReaderIndex(8);
- }
+ fury.readLong(buffer);
return false;
case ClassResolver.PRIMITIVE_DOUBLE_CLASS_ID:
buffer.increaseReaderIndex(8);
diff --git a/java/fury-core/src/main/java/io/fury/serializer/ObjectSerializer.java b/java/fury-core/src/main/java/io/fury/serializer/ObjectSerializer.java
index a3b444c0b9..6eb78af87b 100644
--- a/java/fury-core/src/main/java/io/fury/serializer/ObjectSerializer.java
+++ b/java/fury-core/src/main/java/io/fury/serializer/ObjectSerializer.java
@@ -504,11 +504,7 @@ static boolean writePrimitiveFieldValueFailed(
case ClassResolver.PRIMITIVE_LONG_CLASS_ID:
{
long fieldValue = (long) fieldAccessor.get(targetObject);
- if (fury.compressLong()) {
- buffer.writeVarLong(fieldValue);
- } else {
- buffer.writeLong(fieldValue);
- }
+ fury.writeLong(buffer, fieldValue);
return false;
}
case ClassResolver.PRIMITIVE_DOUBLE_CLASS_ID:
@@ -550,11 +546,7 @@ static boolean writePrimitiveFieldValueFailed(
case ClassResolver.PRIMITIVE_LONG_CLASS_ID:
{
long fieldValue = Platform.getLong(targetObject, fieldOffset);
- if (fury.compressLong()) {
- buffer.writeVarLong(fieldValue);
- } else {
- buffer.writeLong(fieldValue);
- }
+ fury.writeLong(buffer, fieldValue);
return false;
}
case ClassResolver.PRIMITIVE_DOUBLE_CLASS_ID:
@@ -650,11 +642,7 @@ static boolean writeBasicObjectFieldValueFailed(
buffer.writeByte(Fury.NULL_FLAG);
} else {
buffer.writeByte(Fury.NOT_NULL_VALUE_FLAG);
- if (fury.compressLong()) {
- buffer.writeVarLong((Long) (fieldValue));
- } else {
- buffer.writeLong((Long) (fieldValue));
- }
+ fury.writeLong(buffer, (Long) fieldValue);
}
return false;
}
@@ -713,11 +701,7 @@ static boolean readPrimitiveFieldValueFailed(
fieldAccessor.set(targetObject, buffer.readFloat());
return false;
case ClassResolver.PRIMITIVE_LONG_CLASS_ID:
- if (fury.compressLong()) {
- fieldAccessor.set(targetObject, buffer.readVarLong());
- } else {
- fieldAccessor.set(targetObject, buffer.readLong());
- }
+ fieldAccessor.set(targetObject, fury.readLong(buffer));
return false;
case ClassResolver.PRIMITIVE_DOUBLE_CLASS_ID:
fieldAccessor.set(targetObject, buffer.readDouble());
@@ -758,11 +742,7 @@ private static boolean readPrimitiveFieldValueFailed(
Platform.putFloat(targetObject, fieldOffset, buffer.readFloat());
return false;
case ClassResolver.PRIMITIVE_LONG_CLASS_ID:
- if (fury.compressLong()) {
- Platform.putLong(targetObject, fieldOffset, buffer.readVarLong());
- } else {
- Platform.putLong(targetObject, fieldOffset, buffer.readLong());
- }
+ Platform.putLong(targetObject, fieldOffset, fury.readLong(buffer));
return false;
case ClassResolver.PRIMITIVE_DOUBLE_CLASS_ID:
Platform.putDouble(targetObject, fieldOffset, buffer.readDouble());
@@ -854,11 +834,7 @@ static boolean readBasicObjectFieldValueFailed(
if (buffer.readByte() == Fury.NULL_FLAG) {
fieldAccessor.putObject(targetObject, null);
} else {
- if (fury.compressLong()) {
- fieldAccessor.putObject(targetObject, buffer.readVarLong());
- } else {
- fieldAccessor.putObject(targetObject, buffer.readLong());
- }
+ fieldAccessor.putObject(targetObject, fury.readLong(buffer));
}
return false;
}
diff --git a/java/fury-core/src/main/java/io/fury/serializer/PrimitiveSerializers.java b/java/fury-core/src/main/java/io/fury/serializer/PrimitiveSerializers.java
index 2e9b7764bd..20904676ab 100644
--- a/java/fury-core/src/main/java/io/fury/serializer/PrimitiveSerializers.java
+++ b/java/fury-core/src/main/java/io/fury/serializer/PrimitiveSerializers.java
@@ -16,8 +16,13 @@
package io.fury.serializer;
+import static io.fury.type.TypeUtils.PRIMITIVE_LONG_TYPE;
+
import com.google.common.base.Preconditions;
import io.fury.Fury;
+import io.fury.codegen.Expression;
+import io.fury.codegen.Expression.Invoke;
+import io.fury.config.LongEncoding;
import io.fury.memory.MemoryBuffer;
import io.fury.type.Type;
@@ -198,7 +203,7 @@ public Integer xread(MemoryBuffer buffer) {
public static final class LongSerializer
extends Serializers.CrossLanguageCompatibleSerializer {
- private final boolean compressNumber;
+ private final LongEncoding longEncoding;
public LongSerializer(Fury fury, Class> cls) {
super(
@@ -206,24 +211,71 @@ public LongSerializer(Fury fury, Class> cls) {
(Class) cls,
Type.INT64.getId(),
!(cls.isPrimitive() || fury.isBasicTypesRefIgnored()));
- compressNumber = fury.compressLong();
+ longEncoding = fury.longEncoding();
}
@Override
public void write(MemoryBuffer buffer, Long value) {
- if (compressNumber) {
- buffer.writeVarLong(value);
- } else {
- buffer.writeLong(value);
- }
+ writeLong(buffer, value, longEncoding);
}
@Override
public Long read(MemoryBuffer buffer) {
- if (compressNumber) {
- return buffer.readVarLong();
+ return readLong(buffer, longEncoding);
+ }
+
+ public static String writeLongFunc(LongEncoding longEncoding, boolean ensureBounds) {
+ switch (longEncoding) {
+ case LE_RAW_BYTES:
+ return ensureBounds ? "writeLong" : "unsafeWriteVarLong";
+ case SLI:
+ return ensureBounds ? "writeSliLong" : "unsafeWriteSliLong";
+ case PVL:
+ return ensureBounds ? "writeVarLong" : "unsafeWriteVarLong";
+ default:
+ throw new UnsupportedOperationException("Unsupported long encoding " + longEncoding);
+ }
+ }
+
+ public static Expression writeLong(
+ Expression buffer, Expression v, LongEncoding longEncoding, boolean ensureBounds) {
+ return new Invoke(buffer, writeLongFunc(longEncoding, ensureBounds), v);
+ }
+
+ public static void writeLong(MemoryBuffer buffer, long value, LongEncoding longEncoding) {
+ if (longEncoding == LongEncoding.SLI) {
+ buffer.writeSliLong(value);
+ } else if (longEncoding == LongEncoding.LE_RAW_BYTES) {
+ buffer.writeLong(value);
} else {
+ buffer.writeVarLong(value);
+ }
+ }
+
+ public static long readLong(MemoryBuffer buffer, LongEncoding longEncoding) {
+ if (longEncoding == LongEncoding.SLI) {
+ return buffer.readSliLong();
+ } else if (longEncoding == LongEncoding.LE_RAW_BYTES) {
return buffer.readLong();
+ } else {
+ return buffer.readVarLong();
+ }
+ }
+
+ public static Expression readLong(Expression buffer, LongEncoding longEncoding) {
+ return new Invoke(buffer, readLongFunc(longEncoding), PRIMITIVE_LONG_TYPE);
+ }
+
+ public static String readLongFunc(LongEncoding longEncoding) {
+ switch (longEncoding) {
+ case LE_RAW_BYTES:
+ return "readLong";
+ case SLI:
+ return "readSliLong";
+ case PVL:
+ return "readVarLong";
+ default:
+ throw new UnsupportedOperationException("Unsupported long encoding " + longEncoding);
}
}
diff --git a/java/fury-core/src/main/java/io/fury/serializer/Serializers.java b/java/fury-core/src/main/java/io/fury/serializer/Serializers.java
index 2a90e8915f..82a3c07da3 100644
--- a/java/fury-core/src/main/java/io/fury/serializer/Serializers.java
+++ b/java/fury-core/src/main/java/io/fury/serializer/Serializers.java
@@ -126,11 +126,7 @@ public static Object readPrimitiveValue(Fury fury, MemoryBuffer buffer, short cl
case ClassResolver.PRIMITIVE_FLOAT_CLASS_ID:
return buffer.readFloat();
case ClassResolver.PRIMITIVE_LONG_CLASS_ID:
- if (fury.compressLong()) {
- return buffer.readVarLong();
- } else {
- return buffer.readLong();
- }
+ return fury.readLong(buffer);
case ClassResolver.PRIMITIVE_DOUBLE_CLASS_ID:
return buffer.readDouble();
default:
diff --git a/java/fury-core/src/test/java/io/fury/FuryTest.java b/java/fury-core/src/test/java/io/fury/FuryTest.java
index be4ccf260a..bcbd90bc72 100644
--- a/java/fury-core/src/test/java/io/fury/FuryTest.java
+++ b/java/fury-core/src/test/java/io/fury/FuryTest.java
@@ -469,9 +469,6 @@ public void testSerializeJavaObject() {
Fury fury = Fury.builder().requireClassRegistration(false).withLanguage(Language.JAVA).build();
BeanA beanA = BeanA.createBeanA(2);
assertEquals(fury.deserializeJavaObject(fury.serializeJavaObject(beanA), BeanA.class), beanA);
- assertThrows(
- Exception.class,
- () -> fury.deserializeJavaObject(fury.serializeJavaObjectAndClass(beanA), BeanA.class));
assertEquals(
fury.deserializeJavaObjectAndClass(fury.serializeJavaObjectAndClass(beanA)), beanA);
assertEquals(
diff --git a/java/fury-core/src/test/java/io/fury/memory/MemoryBufferTest.java b/java/fury-core/src/test/java/io/fury/memory/MemoryBufferTest.java
index 3bc8a449aa..87fabf511d 100644
--- a/java/fury-core/src/test/java/io/fury/memory/MemoryBufferTest.java
+++ b/java/fury-core/src/test/java/io/fury/memory/MemoryBufferTest.java
@@ -535,4 +535,44 @@ public void testGetShortB() {
assertEquals(MemoryBuffer.getShortB(data, 0), (short) 0xaced);
assertEquals(MemoryBuffer.fromByteArray(data).getShortB(0), (short) 0xaced);
}
+
+ @Test
+ public void testWriteSliLong() {
+ MemoryBuffer buf = MemoryUtils.buffer(8);
+ checkSliLong(buf, -1, 4);
+ for (int i = 0; i < 10; i++) {
+ for (int j = 0; j < i; j++) {
+ checkSliLong(buf(i), -1, 4);
+ checkSliLong(buf(i), 1, 4);
+ checkSliLong(buf(i), 1L << 6, 4);
+ checkSliLong(buf(i), 1L << 7, 4);
+ checkSliLong(buf(i), -(2 << 5), 4);
+ checkSliLong(buf(i), -(2 << 6), 4);
+ checkSliLong(buf(i), 1L << 28, 4);
+ checkSliLong(buf(i), Integer.MAX_VALUE / 2, 4);
+ checkSliLong(buf(i), Integer.MIN_VALUE / 2, 4);
+ checkSliLong(buf(i), -1L << 30, 4);
+ checkSliLong(buf(i), 1L << 30, 9);
+ checkSliLong(buf(i), Integer.MAX_VALUE, 9);
+ checkSliLong(buf(i), Integer.MIN_VALUE, 9);
+ checkSliLong(buf(i), -1L << 31, 9);
+ checkSliLong(buf(i), 1L << 31, 9);
+ checkSliLong(buf(i), -1L << 32, 9);
+ checkSliLong(buf(i), 1L << 32, 9);
+ checkSliLong(buf(i), Long.MAX_VALUE, 9);
+ checkSliLong(buf(i), Long.MIN_VALUE, 9);
+ }
+ }
+ }
+
+ private void checkSliLong(MemoryBuffer buf, long value, int bytesWritten) {
+ int readerIndex = buf.readerIndex();
+ assertEquals(buf.writerIndex(), readerIndex);
+ int actualBytesWritten = buf.writeSliLong(value);
+ assertEquals(actualBytesWritten, bytesWritten);
+ long varLong = buf.readSliLong();
+ assertEquals(buf.writerIndex(), buf.readerIndex());
+ assertEquals(value, varLong);
+ assertEquals(buf.slice(readerIndex, buf.readerIndex() - readerIndex).readSliLong(), value);
+ }
}
diff --git a/java/fury-core/src/test/java/io/fury/serializer/CodegenSerializerTest.java b/java/fury-core/src/test/java/io/fury/serializer/CodegenSerializerTest.java
index f2788ae094..4b9c43b57e 100644
--- a/java/fury-core/src/test/java/io/fury/serializer/CodegenSerializerTest.java
+++ b/java/fury-core/src/test/java/io/fury/serializer/CodegenSerializerTest.java
@@ -200,6 +200,7 @@ public void testCompressInt() {
Fury.builder()
.withLanguage(Language.JAVA)
.withIntCompressed(true)
+ .withLongCompressed(false)
.withCompatibleMode(CompatibleMode.SCHEMA_CONSISTENT)
.requireClassRegistration(false)
.build();