You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@ignite.apache.org by am...@apache.org on 2021/10/29 09:04:52 UTC
[ignite-3] branch main updated: IGNITE-15744 Fix vartable size
overflow issue (#409)
This is an automated email from the ASF dual-hosted git repository.
amashenkov pushed a commit to branch main
in repository https://gitbox.apache.org/repos/asf/ignite-3.git
The following commit(s) were added to refs/heads/main by this push:
new c086555 IGNITE-15744 Fix vartable size overflow issue (#409)
c086555 is described below
commit c086555b25bac0bb38d86e268328b1aed4a3a92f
Author: Andrew V. Mashenkov <AM...@users.noreply.github.com>
AuthorDate: Fri Oct 29 12:04:46 2021 +0300
IGNITE-15744 Fix vartable size overflow issue (#409)
---
.../ignite/internal/schema/row/RowAssembler.java | 4 +-
.../ignite/internal/schema/row/VarTableFormat.java | 53 ++++++++++--------
.../org/apache/ignite/internal/schema/RowTest.java | 65 +++++++++++++---------
3 files changed, 73 insertions(+), 49 deletions(-)
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/RowAssembler.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/RowAssembler.java
index e4f7032..08f7a8b 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/RowAssembler.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/RowAssembler.java
@@ -871,11 +871,11 @@ public class RowAssembler {
assert varTblOff < dataOff : "Illegal writing of varlen when 'omit vartable' flag is set for a chunk.";
assert varTblOff + varTableChunkLength(curVartblEntry, Integer.BYTES) == dataOff : "Vartable overlow: size=" + curVartblEntry;
- final VarTableFormat format = VarTableFormat.format(curOff - dataOff);
+ final VarTableFormat format = VarTableFormat.format(curOff - dataOff, valVartblLen);
curOff -= format.compactVarTable(buf, varTblOff, curVartblEntry - 1);
- flags |= format.formatFlags() << (isKeyChunk() ? KEY_FLAGS_OFFSET : VAL_FLAGS_OFFSET);
+ flags |= format.formatId() << (isKeyChunk() ? KEY_FLAGS_OFFSET : VAL_FLAGS_OFFSET);
}
// Write sizes.
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/VarTableFormat.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/VarTableFormat.java
index 1dabf87..cbf8361 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/VarTableFormat.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/row/VarTableFormat.java
@@ -27,7 +27,7 @@ import org.apache.ignite.internal.util.Constants;
* using proper format {{@link #compactVarTable(ExpandableByteBuf, int, int)}}.
* The vartable format is coded into chunk flags.
*
- * @see #format(int)
+ * @see #format(int, int)
* @see #fromFlags(int)
*/
abstract class VarTableFormat {
@@ -54,11 +54,12 @@ abstract class VarTableFormat {
* to write vartable in a compact way.
*
* @param payloadLen Payload size in bytes.
+ * @param valVartblLen
* @return Vartable format helper.
*/
- static VarTableFormat format(int payloadLen) {
+ static VarTableFormat format(int payloadLen, int valVartblLen) {
if (payloadLen > 0) {
- if (payloadLen < 256)
+ if (payloadLen < 256 && valVartblLen < 256)
return TINY;
if (payloadLen < 64 * Constants.KiB)
@@ -75,10 +76,12 @@ abstract class VarTableFormat {
* @return Vartable format helper.
*/
public static VarTableFormat fromFlags(int chunkFlags) {
- switch (chunkFlags & FORMAT_CODE_MASK) {
- case 1:
+ int formatId = chunkFlags & FORMAT_CODE_MASK;
+
+ switch (formatId) {
+ case TinyFormat.FORMAT_ID:
return TINY;
- case 2:
+ case MediumFormat.FORMAT_ID:
return MEDIUM;
default:
return LARGE;
@@ -91,25 +94,25 @@ abstract class VarTableFormat {
/** Size of cartable size field. */
private final int vartblSizeFieldSize;
- /** Format flags. */
- private final byte flags;
+ /** Format id. */
+ private final byte formatId;
/**
* @param vartblSizeFieldSize Size of vartalble size field (in bytes).
* @param vartblEntrySize Size of vartable entry (in bytes).
- * @param flags Format specific flags.
+ * @param formatId Format specific flags.
*/
- VarTableFormat(int vartblSizeFieldSize, int vartblEntrySize, byte flags) {
+ VarTableFormat(int vartblSizeFieldSize, int vartblEntrySize, byte formatId) {
this.vartblEntrySize = vartblEntrySize;
this.vartblSizeFieldSize = vartblSizeFieldSize;
- this.flags = flags;
+ this.formatId = formatId;
}
/**
- * @return Format specific flags for a chunk.
+ * @return Format id.
*/
- public byte formatFlags() {
- return flags;
+ public byte formatId() {
+ return formatId;
}
/**
@@ -165,11 +168,13 @@ abstract class VarTableFormat {
* Chunk format for small rows (with payload size less 256 bytes).
*/
private static class TinyFormat extends VarTableFormat {
+ private static final byte FORMAT_ID = 1;
+
/**
* Creates chunk format.
*/
TinyFormat() {
- super(Byte.BYTES, Byte.BYTES, (byte)1);
+ super(Byte.BYTES, Byte.BYTES, FORMAT_ID);
}
/** {@inheritDoc} */
@@ -184,12 +189,12 @@ abstract class VarTableFormat {
/** {@inheritDoc} */
@Override public int compactVarTable(ExpandableByteBuf buf, int vartblOff, int entres) {
- assert entres > 0;
+ assert entres > 0 && entres < 0xFFFF;
buf.put(vartblOff, (byte)entres);
- int dstOff = vartblOff + 1;
- int srcOff = vartblOff + 2;
+ int dstOff = vartblOff + Byte.BYTES;
+ int srcOff = vartblOff + Short.BYTES;
for (int i = 0; i < entres; i++, srcOff += Integer.BYTES, dstOff++)
buf.put(dstOff, buf.get(srcOff));
@@ -204,11 +209,13 @@ abstract class VarTableFormat {
* Chunk format for rows of medium size (with payload size up to 64Kb).
*/
private static class MediumFormat extends VarTableFormat {
+ private static final byte FORMAT_ID = 2;
+
/**
* Creates chunk format.
*/
MediumFormat() {
- super(Short.BYTES, Short.BYTES, (byte)2);
+ super(Short.BYTES, Short.BYTES, FORMAT_ID);
}
/** {@inheritDoc} */
@@ -225,8 +232,8 @@ abstract class VarTableFormat {
@Override public int compactVarTable(ExpandableByteBuf buf, int vartblOff, int entries) {
buf.putShort(vartblOff, (short)entries);
- int dstOff = vartblOff + 2;
- int srcOff = vartblOff + 2;
+ int dstOff = vartblOff + Short.BYTES;
+ int srcOff = vartblOff + Short.BYTES;
for (int i = 0; i < entries; i++, srcOff += Integer.BYTES, dstOff += Short.BYTES)
buf.putShort(dstOff, buf.getShort(srcOff));
@@ -241,11 +248,13 @@ abstract class VarTableFormat {
* Chunk format for large rows (with payload size 64+Kb).
*/
private static class LargeFormat extends VarTableFormat {
+ private static byte FORMAT_ID = 0;
+
/**
* Creates chunk format.
*/
LargeFormat() {
- super(Short.BYTES, Integer.BYTES, (byte)0);
+ super(Short.BYTES, Integer.BYTES, FORMAT_ID);
}
/** {@inheritDoc} */
diff --git a/modules/schema/src/test/java/org/apache/ignite/internal/schema/RowTest.java b/modules/schema/src/test/java/org/apache/ignite/internal/schema/RowTest.java
index eea1ec0..006abe2 100644
--- a/modules/schema/src/test/java/org/apache/ignite/internal/schema/RowTest.java
+++ b/modules/schema/src/test/java/org/apache/ignite/internal/schema/RowTest.java
@@ -74,7 +74,7 @@ public class RowTest {
}
/**
- * Check row serialization for schema with nullable fix-sized columns only.
+ * Check row serialization for a schema with nullable fix-sized columns only.
*/
@Test
public void nullableFixSizedColumns() {
@@ -114,7 +114,7 @@ public class RowTest {
}
/**
- * Check row serialization for schema with non-nullable fix-sized columns only.
+ * Check row serialization for a schema with non-nullable fix-sized columns only.
*/
@Test
public void fixSizedColumns() {
@@ -154,7 +154,7 @@ public class RowTest {
}
/**
- * Check row serialization for schema with various columns.
+ * Check row serialization for a schema with various columns.
*/
@Test
public void mixedColumns() {
@@ -186,7 +186,7 @@ public class RowTest {
}
/**
- * Check row serialization for schema with various columns.
+ * Check row serialization for a schema with various columns.
*/
@Test
public void temporalColumns() {
@@ -212,7 +212,7 @@ public class RowTest {
}
/**
- * Check row serialization for schema with non-nullable varlen columns only.
+ * Check row serialization for a schema with non-nullable varlen columns only.
*/
@Test
public void varlenColumns() {
@@ -234,7 +234,7 @@ public class RowTest {
}
/**
- * Check row serialization for schema with nullable varlen columns only.
+ * Check row serialization for a schema with nullable varlen columns only.
*/
@Test
public void nullableVarlenColumns() {
@@ -254,16 +254,16 @@ public class RowTest {
}
/**
- * Check row serialization for schema with large varlen columns (64Kb+).
+ * Check row serialization for a schema with large varlen columns (64Kb+).
*/
@Test
public void largeVarlenColumns() {
- Column[] keyCols = new Column[] {
+ Column[] keyCols = new Column[]{
new Column("keyBytesCol", BYTES, false),
new Column("keyStringCol", STRING, false),
};
- Column[] valCols = new Column[] {
+ Column[] valCols = new Column[]{
new Column("valBytesCol", BYTES, true),
new Column("valStringCol", STRING, true),
};
@@ -290,6 +290,29 @@ public class RowTest {
}
/**
+ * Check row serialization for a schema with many empty varlen columns (255+).
+ */
+ @Test
+ public void bigVartable() {
+ Column[] keyCols = new Column[]{
+ new Column("id", INT64, false),
+ };
+
+ int columnCount = 260;
+
+ Column[] valCols = IntStream.range(1, columnCount)
+ .mapToObj(i -> new Column("val" + i, STRING, true))
+ .toArray(Column[]::new);
+
+ SchemaDescriptor sch = new SchemaDescriptor(1, keyCols, valCols);
+
+ Object[] checkArr = IntStream.range(0, columnCount)
+ .mapToObj(i -> i == 0 ? 42L : (i > columnCount - 5) ? "str" : "").toArray();
+
+ checkValues(sch, checkArr);
+ }
+
+ /**
* Check row serialization for 256+ fixsized columns.
*/
@Test
@@ -469,43 +492,35 @@ public class RowTest {
if (schema.isKeyColumn(i)) {
nonNullVarLenKeyCols++;
nonNullVarLenKeySize += val.length;
- }
- else {
+ } else {
nonNullVarLenValCols++;
nonNullVarLenValSize += val.length;
}
- }
- else if (type == NativeTypeSpec.STRING) {
+ } else if (type == NativeTypeSpec.STRING) {
if (schema.isKeyColumn(i)) {
nonNullVarLenKeyCols++;
nonNullVarLenKeySize += RowAssembler.utf8EncodedLength((CharSequence)vals[i]);
- }
- else {
+ } else {
nonNullVarLenValCols++;
nonNullVarLenValSize += RowAssembler.utf8EncodedLength((CharSequence)vals[i]);
}
- }
- else if (type == NativeTypeSpec.NUMBER) {
+ } else if (type == NativeTypeSpec.NUMBER) {
if (schema.isKeyColumn(i)) {
nonNullVarLenKeyCols++;
nonNullVarLenKeySize += RowAssembler.sizeInBytes((BigInteger)vals[i]);
- }
- else {
+ } else {
nonNullVarLenValCols++;
nonNullVarLenValSize += RowAssembler.sizeInBytes((BigInteger)vals[i]);
}
- }
- else if (type == NativeTypeSpec.DECIMAL) {
+ } else if (type == NativeTypeSpec.DECIMAL) {
if (schema.isKeyColumn(i)) {
nonNullVarLenKeyCols++;
nonNullVarLenKeySize += RowAssembler.sizeInBytes((BigDecimal)vals[i]);
- }
- else {
+ } else {
nonNullVarLenValCols++;
nonNullVarLenValSize += RowAssembler.sizeInBytes((BigDecimal)vals[i]);
}
- }
- else
+ } else
throw new IllegalStateException("Unsupported variable-length type: " + type);
}
}