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/06/24 12:06:38 UTC

[ignite-3] 01/02: Simplify row size estimation. Fallback to single vartable format.

This is an automated email from the ASF dual-hosted git repository.

amashenkov pushed a commit to branch ignite-14743-compaction
in repository https://gitbox.apache.org/repos/asf/ignite-3.git

commit 40b85de0f3ffeeeae788a38661d1752cc1bddf0a
Author: Andrew Mashenkov <an...@gmail.com>
AuthorDate: Thu Jun 24 01:16:43 2021 +0300

    Simplify row size estimation.
    Fallback to single vartable format.
---
 .../internal/schema/marshaller/MarshallerUtil.java |  3 +-
 .../marshaller/reflection/JavaSerializer.java      |  2 +-
 .../ignite/internal/schema/row/RowAssembler.java   | 80 +++++-----------------
 .../ignite/internal/table/TupleBuilderImpl.java    |  2 +-
 .../ignite/internal/table/TupleMarshallerImpl.java | 22 +++---
 5 files changed, 28 insertions(+), 81 deletions(-)

diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/MarshallerUtil.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/MarshallerUtil.java
index 265c3f9..ecf77be 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/MarshallerUtil.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/MarshallerUtil.java
@@ -21,7 +21,6 @@ import java.util.BitSet;
 import java.util.UUID;
 import org.apache.ignite.internal.schema.InvalidTypeException;
 import org.apache.ignite.internal.schema.NativeType;
-import org.apache.ignite.internal.schema.row.RowAssembler;
 import org.apache.ignite.internal.util.ObjectFactory;
 
 /**
@@ -41,7 +40,7 @@ public final class MarshallerUtil {
                 return ((byte[])val).length;
 
             case STRING:
-                return RowAssembler.utf8EncodedLength((CharSequence)val);
+                return ((CharSequence)val).length() << 1; //RowAssembler.utf8EncodedLength((CharSequence)val);
 
             default:
                 throw new InvalidTypeException("Unsupported variable-length type: " + type);
diff --git a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/reflection/JavaSerializer.java b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/reflection/JavaSerializer.java
index 57b5bdd..2e4f4b3 100644
--- a/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/reflection/JavaSerializer.java
+++ b/modules/schema/src/main/java/org/apache/ignite/internal/schema/marshaller/reflection/JavaSerializer.java
@@ -98,7 +98,7 @@ public class JavaSerializer extends AbstractSerializer {
      */
     private ObjectStatistic collectObjectStats(Columns cols, Marshaller marsh, Object obj) {
         if (obj == null || !cols.hasVarlengthColumns())
-            return new ObjectStatistic(0, 0);
+            return new ObjectStatistic(0, cols.fixsizeMaxLen());
 
         int cnt = 0;
         int size = cols.fixsizeMaxLen();
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 b639480..d07f621 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
@@ -57,12 +57,6 @@ public class RowAssembler {
     /** The number of non-null varlen columns in values chunk. */
     private final int valVarlenCols;
 
-    /** Value null-map size in bytes. */
-    private int valNullMapSize;
-
-    /** Value write mode. */
-    private final VarTableFormat valWriteMode;
-
     /** Current columns chunk. */
     private Columns curCols;
 
@@ -93,9 +87,6 @@ public class RowAssembler {
     /** Current offset for the next column to be appended. */
     protected int curOff;
 
-    /** Vartable format helper. */
-    private VarTableFormat format;
-
     /**
      * Calculates encoded string length.
      *
@@ -140,10 +131,8 @@ public class RowAssembler {
     ) {
         this(schema,
             0,
-            schema.keyColumns().nullMapSize() > 0,
             nonNullVarlenKeyCols,
             0,
-            schema.valueColumns().nullMapSize() > 0,
             nonNullVarlenValCols);
     }
 
@@ -154,46 +143,15 @@ public class RowAssembler {
      *
      * @param schema Row schema.
      * @param keyDataSize Key payload size. Estimated upper-bound or zero if unknown.
-     * @param nonNullVarlenKeyCols Number of non-null varlen columns in key chunk.
-     * @param valDataSize Value data size. Estimated upper-bound or zero if unknown.
-     * @param nonNullVarlenValCols Number of non-null varlen columns in value chunk.
-     */
-    public RowAssembler(
-        SchemaDescriptor schema,
-        int keyDataSize,
-        int nonNullVarlenKeyCols,
-        int valDataSize,
-        int nonNullVarlenValCols
-    ) {
-        this(schema,
-            keyDataSize,
-            schema.keyColumns().nullMapSize() > 0,
-            nonNullVarlenKeyCols,
-            valDataSize,
-            schema.valueColumns().nullMapSize() > 0,
-            nonNullVarlenValCols);
-    }
-
-    /**
-     * Creates RowAssembler for chunks with estimated sizes.
-     * <p>
-     * RowAssembler will apply optimizations based on chunks sizes estimations.
-     *
-     * @param schema Row schema.
-     * @param keyDataSize Key payload size. Estimated upper-bound or zero if unknown.
-     * @param keyHasNulls Null flag. {@code True} if key has nulls values, {@code false} otherwise.
      * @param keyVarlenCols Number of non-null varlen columns in key chunk.
      * @param valDataSize Value data size. Estimated upper-bound or zero if unknown.
-     * @param valHasNulls Null flag. {@code True} if value has nulls values, {@code false} otherwise.
      * @param valVarlenCols Number of non-null varlen columns in value chunk.
      */
     public RowAssembler(
         SchemaDescriptor schema,
         int keyDataSize,
-        boolean keyHasNulls,
         int keyVarlenCols,
         int valDataSize,
-        boolean valHasNulls,
         int valVarlenCols
     ) {
         this.schema = schema;
@@ -204,20 +162,21 @@ public class RowAssembler {
         hash = 0;
         strEncoder = null;
 
-        final int keyNullMapSize = keyHasNulls ? schema.keyColumns().nullMapSize() : 0;
-        valNullMapSize = valHasNulls ? schema.valueColumns().nullMapSize() : 0;
-
-        final VarTableFormat keyWriteMode = VarTableFormat.format(keyDataSize);
-        valWriteMode = VarTableFormat.format(valDataSize);
+        final int keyNullMapSize = schema.keyColumns().nullMapSize();
+        final int valNullMapSize = schema.valueColumns().nullMapSize();
 
         int size = BinaryRow.HEADER_SIZE +
-            keyWriteMode.chunkSize(keyDataSize, keyNullMapSize, keyVarlenCols) +
-            valWriteMode.chunkSize(valDataSize, valNullMapSize, valVarlenCols);
+            +keyDataSize + keyNullMapSize + varTableLength(keyVarlenCols, Integer.BYTES) +
+            valDataSize + valNullMapSize + varTableLength(valVarlenCols, Integer.BYTES);
 
         buf = new ExpandableByteBuf(size);
         buf.putShort(0, (short)schema.version());
 
-        initChunk(KEY_CHUNK_OFFSET, keyNullMapSize, keyVarlenCols, keyWriteMode);
+        initChunk(KEY_CHUNK_OFFSET, keyNullMapSize, keyVarlenCols);
+    }
+
+    private int varTableLength(int entries, int entrySize) {
+        return entries <= 1 ? 0 : Short.BYTES + entries * entrySize;
     }
 
     /**
@@ -569,9 +528,9 @@ public class RowAssembler {
                 assert varTblOff < dataOff :
                     "Illegal writing of varlen when 'omit vartable' flag is set for a chunk.";
 
-                assert varTblOff + format.vartableLength(curVartblEntry - 1) == dataOff : "Vartable overlow: size=" + curVartblEntry;
+                assert varTblOff + varTableLength(curVartblEntry - 1, Integer.BYTES) == dataOff : "Vartable overlow: size=" + curVartblEntry;
 
-                format.writeVartableSize(buf, varTblOff, curVartblEntry - 1);
+                buf.putShort(varTblOff, (short)(curVartblEntry - 1));
             }
 
             if (schema.valueColumns() == curCols)
@@ -582,12 +541,7 @@ public class RowAssembler {
             curCol = 0;
 
             // Create value chunk writer.
-            initChunk(
-                BinaryRow.HEADER_SIZE + size/* Key chunk size */,
-                valNullMapSize,
-                valVarlenCols,
-                valWriteMode
-            );
+            initChunk(BinaryRow.HEADER_SIZE + size/* Key chunk size */, curCols.nullMapSize(), valVarlenCols);
         }
     }
 
@@ -608,7 +562,7 @@ public class RowAssembler {
         if (entryIdx == 0)
             return; // Omit offset for very first varlen.
 
-        format.writeVarlenOffset(buf, varTblOff, entryIdx - 1, off);
+        buf.putInt(varTblOff + Short.BYTES + (entryIdx - 1) * Integer.BYTES, off);
     }
 
     /**
@@ -617,20 +571,18 @@ public class RowAssembler {
      * @param baseOff Chunk base offset.
      * @param nullMapLen Null-map length in bytes.
      * @param vartblEntries Vartable entries.
-     * @param format Vartable format helper.
      */
-    private void initChunk(int baseOff, int nullMapLen, int vartblEntries, VarTableFormat format) {
+    private void initChunk(int baseOff, int nullMapLen, int vartblEntries) {
         this.baseOff = baseOff;
-        this.format = format;
 
-        int vartblLen = format.vartableLength(vartblEntries - 1);
+        final int vartblLen = varTableLength(vartblEntries, Integer.BYTES);
 
         varTblOff = nullmapOff() + nullMapLen;
         dataOff = varTblOff + vartblLen;
         curOff = dataOff;
         curVartblEntry = 0;
 
-        int flags = format.formatFlags();
+        int flags = 0;
 
         if (nullMapLen == 0)
             flags |= VarTableFormat.OMIT_NULL_MAP_FLAG;
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/TupleBuilderImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/TupleBuilderImpl.java
index 66e93aa..1d4aae4 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/TupleBuilderImpl.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/TupleBuilderImpl.java
@@ -48,7 +48,7 @@ public class TupleBuilderImpl implements TupleBuilder, Tuple {
         Objects.requireNonNull(schemaDesc);
 
         this.schemaDesc = schemaDesc;
-        map = new HashMap<>();
+        map = new HashMap<>(schemaDesc.length());
     }
 
     /** {@inheritDoc} */
diff --git a/modules/table/src/main/java/org/apache/ignite/internal/table/TupleMarshallerImpl.java b/modules/table/src/main/java/org/apache/ignite/internal/table/TupleMarshallerImpl.java
index 0406bd6..dc0b59a 100644
--- a/modules/table/src/main/java/org/apache/ignite/internal/table/TupleMarshallerImpl.java
+++ b/modules/table/src/main/java/org/apache/ignite/internal/table/TupleMarshallerImpl.java
@@ -99,20 +99,21 @@ public class TupleMarshallerImpl implements TupleMarshaller {
 
         TupleStatistics chunk = new TupleStatistics();
 
-        for (int i = 0; i < cols.length(); i++) {
+        chunk.payloadLen = cols.fixsizeMaxLen();
+
+        if (!cols.hasVarlengthColumns())
+            return chunk;
+
+        for (int i = cols.firstVarlengthColumn(); i < cols.length(); i++) {
             Column col = cols.column(i);
 
             Object val = (tup.contains(col.name())) ? tup.value(col.name()) : col.defaultValue();
 
             if (val == null)
-                chunk.hasNulls = true;
-            else if (col.type().spec().fixedLength())
-                chunk.payloadLen += col.type().sizeInBytes();
-            else {
-                chunk.nonNullVarlen++;
+                continue;
 
-                chunk.payloadLen += getValueSize(val, col.type());
-            }
+            chunk.nonNullVarlen++;
+            chunk.payloadLen += getValueSize(val, col.type());
         }
 
         return chunk;
@@ -150,10 +151,8 @@ public class TupleMarshallerImpl implements TupleMarshaller {
         return new RowAssembler(
             schema,
             keyStat.payloadLen,
-            keyStat.hasNulls,
             keyStat.nonNullVarlen,
             valStat.payloadLen,
-            valStat.hasNulls,
             valStat.nonNullVarlen);
     }
 
@@ -231,9 +230,6 @@ public class TupleMarshallerImpl implements TupleMarshaller {
      * Tuple statistics record.
      */
     private static class TupleStatistics {
-        /** Tuple has nulls. */
-        boolean hasNulls;
-
         /** Payload length in bytes. */
         int payloadLen;