You are viewing a plain text version of this content. The canonical link for it is here.
Posted to notifications@ignite.apache.org by GitBox <gi...@apache.org> on 2022/08/31 09:20:00 UTC

[GitHub] [ignite-3] SammyVimes commented on a diff in pull request #1021: IGNITE-17535 Implementing a hash index B+Tree

SammyVimes commented on code in PR #1021:
URL: https://github.com/apache/ignite-3/pull/1021#discussion_r959312323


##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/io/AbstractDataPageIo.java:
##########
@@ -1235,99 +1235,32 @@ public int addRowFragment(
     ) throws IgniteInternalCheckedException {
         assertPageType(pageAddr);
 
-        return addRowFragment(pageMem, pageId, pageAddr, written, rowSize, row.link(), row, null, pageSize);
-    }
-
-    /**
-     * Adds this payload as a fragment to this data page.
-     *
-     * @param pageId Page ID to use to construct a link.
-     * @param pageAddr Page address.
-     * @param payload Payload bytes.
-     * @param lastLink Link to the previous written fragment (link to the tail).
-     * @param pageSize Page size.
-     * @throws IgniteInternalCheckedException If failed.
-     */
-    public void addRowFragment(
-            long pageId,
-            long pageAddr,
-            byte[] payload,
-            long lastLink,
-            int pageSize
-    ) throws IgniteInternalCheckedException {
-        assertPageType(pageAddr);
+        assert row != null;
 
-        addRowFragment(null, pageId, pageAddr, 0, 0, lastLink, null, payload, pageSize);
-    }
-
-    /**
-     * Adds maximum possible fragment of the given row to this data page and sets respective link to the row.
-     *
-     * @param pageMem Page memory.
-     * @param pageId Page ID to use to construct a link.
-     * @param pageAddr Page address.
-     * @param written Number of bytes of row size that was already written.
-     * @param rowSize Row size.
-     * @param lastLink Link to the previous written fragment (link to the tail).
-     * @param row Row.
-     * @param payload Payload bytes.
-     * @param pageSize Page size.
-     * @return Written payload size.
-     * @throws IgniteInternalCheckedException If failed.
-     */
-    private int addRowFragment(
-            PageMemory pageMem,
-            long pageId,
-            long pageAddr,
-            int written,
-            int rowSize,
-            long lastLink,
-            T row,
-            byte[] payload,
-            int pageSize
-    ) throws IgniteInternalCheckedException {
-        assert payload == null ^ row == null;
+        long lastLink = row.link();
 
         int directCnt = getDirectCount(pageAddr);
         int indirectCnt = getIndirectCount(pageAddr);
 
-        int payloadSize = payload != null ? payload.length :
-                Math.min(rowSize - written, getFreeSpace(pageAddr));
+        int payloadSize = Math.min(rowSize - written, getFreeSpace(pageAddr));
 
-        if (row != null) {
-            int remain = rowSize - written - payloadSize;
-            int hdrSize = row.headerSize();
-
-            // We need page header (i.e. MVCC info) is located entirely on the very first page in chain.
-            // So we force moving it to the next page if it could not fit entirely on this page.
-            if (remain > 0 && remain < hdrSize) {
-                payloadSize -= hdrSize - remain;
-            }
-        }
+        assert payloadSize >= row.headerSize() || written >= row.headerSize();
 
         int fullEntrySize = getPageEntrySize(payloadSize, SHOW_PAYLOAD_LEN | SHOW_LINK | SHOW_ITEM);
         int dataOff = getDataOffsetForWrite(pageAddr, fullEntrySize, directCnt, indirectCnt, pageSize);
 
-        if (payload == null) {
-            ByteBuffer buf = pageMem.pageBuffer(pageAddr);
-
-            buf.position(dataOff);
+        ByteBuffer buf = pageMem.pageBuffer(pageAddr);
 
-            short p = (short) (payloadSize | FRAGMENTED_FLAG);
+        buf.position(dataOff);
 
-            buf.putShort(p);
-            buf.putLong(lastLink);
+        short p = (short) (payloadSize | FRAGMENTED_FLAG);

Review Comment:
   Variable name (I see it was like this, but git shows it as a new line of code)



##########
modules/page-memory/src/main/java/org/apache/ignite/internal/pagememory/util/PartitionlessLinks.java:
##########
@@ -27,23 +27,22 @@
 import static org.apache.ignite.internal.pagememory.util.PageUtils.putShort;
 
 import java.nio.ByteBuffer;
-import org.apache.ignite.internal.pagememory.util.PageIdUtils;
 
 /**
- * Handling of <em>partitionless links</em>, that is, page memory links from which partition ID is removed. They are used to spare storage
- * space in cases when we know the partition ID from the context.
+ * Handling of <em>partitionless links</em>, that is, page memory links from which partition ID is removed.
+ *
+ * <p>They are used to sparing storage space in cases when we know the partition ID from the context.

Review Comment:
   ```suggestion
    * <p>They are used to save storage space in cases if we know the partition ID from the context.
   ```



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/freelist/IndexColumns.java:
##########
@@ -0,0 +1,120 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.ignite.internal.storage.pagememory.index.freelist;
+
+import static org.apache.ignite.internal.pagememory.util.PageIdUtils.NULL_LINK;
+
+import java.nio.ByteBuffer;
+import org.apache.ignite.internal.pagememory.Storable;
+import org.apache.ignite.internal.pagememory.io.AbstractDataPageIo;
+import org.apache.ignite.internal.pagememory.io.IoVersions;
+import org.apache.ignite.internal.storage.pagememory.index.freelist.io.IndexColumnsDataIo;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Index columns to store in free list.
+ */
+public class IndexColumns implements Storable {
+    /** Size offset. */
+    public static final int SIZE_OFFSET = 0;
+
+    /** Value offset. Value goes right after the size. */
+    public static final int VALUE_OFFSET = SIZE_OFFSET + Integer.BYTES;
+
+    /** Partition ID. */
+    private final int partitionId;
+
+    /** Link value. */
+    private long link = NULL_LINK;
+
+    /** Byte buffer with binary tuple data. */
+    private final @Nullable ByteBuffer valueBuffer;
+
+    /**
+     * Constructor.
+     *
+     * @param partitionId Partition ID.
+     * @param valueBuffer Value buffer.
+     */
+
+    public IndexColumns(int partitionId, @Nullable ByteBuffer valueBuffer) {
+        this.partitionId = partitionId;
+        this.valueBuffer = valueBuffer;
+    }
+
+    /**
+     * Constructor.
+     *
+     * @param partitionId Partition ID.
+     * @param link Link.
+     * @param valueBuffer Value buffer.
+     */
+    public IndexColumns(int partitionId, long link, @Nullable ByteBuffer valueBuffer) {
+        this.partitionId = partitionId;
+        this.link = link;
+        this.valueBuffer = valueBuffer;
+    }
+
+    /**
+     * Returns the size of binary tuple.
+     */
+    public int valueSize() {
+        assert valueBuffer != null;
+
+        return valueBuffer.limit();
+    }
+
+    /**
+     * Returns a byte buffer that contains binary tuple data.
+     */
+    public ByteBuffer valueBuffer() {
+        return valueBuffer;

Review Comment:
   Should it be read-only for safety?



##########
modules/storage-page-memory/src/main/java/org/apache/ignite/internal/storage/pagememory/index/hash/RemoveHashIndexRowInvokeClosure.java:
##########
@@ -0,0 +1,99 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements. See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You 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 org.apache.ignite.internal.storage.pagememory.index.hash;
+
+
+import static org.apache.ignite.internal.pagememory.util.PageIdUtils.NULL_LINK;
+
+import org.apache.ignite.internal.pagememory.metric.IoStatisticsHolder;
+import org.apache.ignite.internal.pagememory.tree.BplusTree;
+import org.apache.ignite.internal.pagememory.tree.IgniteTree.InvokeClosure;
+import org.apache.ignite.internal.pagememory.tree.IgniteTree.OperationType;
+import org.apache.ignite.internal.storage.pagememory.index.freelist.IndexColumns;
+import org.apache.ignite.internal.storage.pagememory.index.freelist.IndexColumnsFreeList;
+import org.apache.ignite.lang.IgniteInternalCheckedException;
+import org.jetbrains.annotations.Nullable;
+
+/**
+ * Insert closure that removes corresponding {@link IndexColumns} from a {@link IndexColumnsFreeList} after removing it from the
+ * {@link HashIndexTree}.
+ */
+public class RemoveHashIndexRowInvokeClosure implements InvokeClosure<HashIndexRow> {
+    /** Hash index row instance for removal. */
+    private final HashIndexRow hashIndexRow;
+
+    /** Free list to insert data into in case of necessity. */
+    private final IndexColumnsFreeList freeList;
+
+    /** Statistics holder to track IO operations. */
+    private final IoStatisticsHolder statHolder;
+
+    /** Operation type, either {@link OperationType#REMOVE} or {@link OperationType#NOOP} if row is missing. */
+    private OperationType operationType = OperationType.REMOVE;
+
+    /**
+     * Constructor.
+     *
+     * @param hashIndexRow Hash index row instance for removal.
+     * @param freeList Free list to insert data into in case of necessity.
+     * @param statHolder Statistics holder to track IO operations.
+     */
+    public RemoveHashIndexRowInvokeClosure(HashIndexRow hashIndexRow, IndexColumnsFreeList freeList, IoStatisticsHolder statHolder) {
+        assert hashIndexRow.indexColumns().link() == 0L;
+
+        this.hashIndexRow = hashIndexRow;
+        this.freeList = freeList;
+        this.statHolder = statHolder;
+    }
+
+    @Override
+    public void call(@Nullable HashIndexRow oldRow) {
+        if (oldRow == null) {
+            operationType = OperationType.NOOP;
+        } else {
+            hashIndexRow.indexColumns().link(oldRow.indexColumns().link());
+        }
+    }
+
+    @Override
+    public @Nullable HashIndexRow newRow() {
+        return null;
+    }
+
+    @Override
+    public OperationType operationType() {
+        return operationType;
+    }
+
+    /**
+     * Method to call after {@link BplusTree#invoke(Object, Object, InvokeClosure)} has completed.
+     *
+     * @throws IgniteInternalCheckedException If failed to remove data from the free list.
+     */
+    public void afterCompletion() throws IgniteInternalCheckedException {

Review Comment:
   This method is unused, is it on purpose?



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: notifications-unsubscribe@ignite.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org