ibessonov commented on code in PR #787:
URL: https://github.com/apache/ignite-3/pull/787#discussion_r853854608


##########
modules/storage-rocksdb/src/main/java/org/apache/ignite/internal/storage/rocksdb/RocksDbMvPartitionStorage.java:
##########
@@ -0,0 +1,494 @@
+/*
+ * 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.rocksdb;
+
+import static java.lang.ThreadLocal.withInitial;
+import static java.nio.ByteBuffer.allocateDirect;
+import static java.nio.ByteOrder.BIG_ENDIAN;
+import static org.apache.ignite.internal.storage.IgniteRowId.MAX_ROW_ID_SIZE;
+import static org.apache.ignite.internal.util.ArrayUtils.BYTE_EMPTY_ARRAY;
+
+import java.nio.ByteBuffer;
+import java.util.Arrays;
+import java.util.NoSuchElementException;
+import java.util.UUID;
+import java.util.function.Predicate;
+import org.apache.ignite.internal.schema.BinaryRow;
+import org.apache.ignite.internal.schema.ByteBufferRow;
+import org.apache.ignite.internal.storage.IgniteRowId;
+import org.apache.ignite.internal.storage.MvPartitionStorage;
+import org.apache.ignite.internal.storage.StorageException;
+import org.apache.ignite.internal.storage.TxIdMismatchException;
+import org.apache.ignite.internal.tx.Timestamp;
+import org.apache.ignite.internal.util.Cursor;
+import org.apache.ignite.internal.util.GridUnsafe;
+import org.apache.ignite.internal.util.IgniteUtils;
+import org.apache.ignite.lang.IgniteInternalException;
+import org.jetbrains.annotations.Nullable;
+import org.rocksdb.ColumnFamilyHandle;
+import org.rocksdb.ReadOptions;
+import org.rocksdb.RocksDB;
+import org.rocksdb.RocksDBException;
+import org.rocksdb.RocksIterator;
+import org.rocksdb.Slice;
+import org.rocksdb.WriteOptions;
+
+/**
+ * Mult-versioned partition storage implementation based on RocksDB. Stored 
data has the following format:
+ * <pre><code>
+ * | partId (2 bytes, BE) | rowId ({@link #igniteRowIdSize} bytes) 
|</code></pre>
+ * or
+ * <pre><code>
+ * | partId (2 bytes, BE) | rowId ({@link #igniteRowIdSize} bytes) | timestamp 
(16 bytes, DESC) |</code></pre>
+ * depending on transaction status. Pending transactions data doesn't have a 
timestamp assigned.
+ *
+ * <p/>BE means Big Endian, meaning that lexicographical bytes order matches a 
natural order of partitions.
+ *
+ * <p/>DESC means that timestamps are sorted from newest to oldest (N2O). 
Please refer to {@link #putTimestamp(ByteBuffer, Timestamp)} to
+ * see how it's achieved. Missing timestamp could be interpreted as a moment 
infinitely far away in the future.
+ */
+public class RocksDbMvPartitionStorage implements MvPartitionStorage {
+    /** Position of row id inside of the key. */
+    private static final int ROW_ID_OFFSET = Short.BYTES;
+
+    /** Timestamp size in bytes. */
+    private static final int TIMESTAMP_SIZE = 2 * Long.BYTES;
+
+    /** Maximum possible size of the key. */
+    private static final int MAX_KEY_SIZE = /* partId */ ROW_ID_OFFSET + /* 
rowId */ MAX_ROW_ID_SIZE + /* timestamp */ TIMESTAMP_SIZE;
+
+    /** Threadlocal direct buffer instance to read keys from RocksDB. */
+    private static final ThreadLocal<ByteBuffer> MV_KEY_BUFFER = 
withInitial(() -> allocateDirect(MAX_KEY_SIZE).order(BIG_ENDIAN));
+
+    /** Threadlocal on-heap byte buffer instance to use for key manipulations. 
*/
+    private final ThreadLocal<ByteBuffer> heapKeyBuffer;
+
+    /** Size of the {@link IgniteRowId} identifiers for this particular 
partition storage. */
+    private final int igniteRowIdSize;
+
+    /**
+     * Partition ID (should be treated as an unsigned short).
+     *
+     * <p/>Partition IDs are always stored in the big endian order, since they 
need to be compared lexicographically.
+     */
+    private final int partId;
+
+    /** RocksDb instance. */
+    private final RocksDB db;
+
+    /** Partitions column family. */
+    private final ColumnFamilyHandle cf;
+
+    /** Write options. */
+    private final WriteOptions writeOpts = new 
WriteOptions().setDisableWAL(true);
+
+    /** Upper bound for scans and reads. */
+    private final Slice upperBound;
+
+    /**
+     * Constructor.
+     *
+     * @param igniteRowIdSize Size of row id.
+     * @param partId Partition id.
+     * @param db RocksDB instance.
+     * @param cf Column family handle to store partition data.
+     */
+    public RocksDbMvPartitionStorage(int igniteRowIdSize, int partId, RocksDB 
db, ColumnFamilyHandle cf) {
+        this.igniteRowIdSize = igniteRowIdSize;
+        this.partId = partId;
+        this.db = db;
+        this.cf = cf;
+
+        heapKeyBuffer = withInitial(() ->
+                ByteBuffer.allocate(Short.BYTES + igniteRowIdSize + 
ROW_ID_OFFSET * Long.BYTES)
+                        .order(BIG_ENDIAN)
+                        .putShort((short) partId)
+        );
+
+        upperBound = new Slice(partitionEndPrefix());
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void addWrite(IgniteRowId rowId, @Nullable BinaryRow row, UUID 
txId) throws TxIdMismatchException, StorageException {
+        // Prepare a buffer with partition id and row id.
+        ByteBuffer keyBuf = prepareHeapKeyBuf(rowId);
+
+        try {
+            // Check concurrent transaction data.
+            if (RocksDB.NOT_FOUND != db.get(cf, keyBuf.array(), 0, 
keyBuf.position(), BYTE_EMPTY_ARRAY, 0, 0)) {
+                throw new TxIdMismatchException();
+            }
+
+            if (row == null) {
+                // Write empty array as a tombstone.
+                db.put(cf, writeOpts, keyBuf.array(), 0, keyBuf.position(), 
BYTE_EMPTY_ARRAY, 0, 0);
+            } else {
+                byte[] valueBytes = row.bytes();
+
+                // Write binary row data as a value.
+                db.put(cf, writeOpts, keyBuf.array(), 0, keyBuf.position(), 
valueBytes, 0, valueBytes.length);
+            }
+        } catch (RocksDBException e) {
+            throw new StorageException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void abortWrite(IgniteRowId rowId) throws StorageException {
+        // Prepare a buffer with partition id and row id.
+        ByteBuffer keyBuf = prepareHeapKeyBuf(rowId);
+
+        try {
+            // Perform unconditional remove for the key without associated 
timestamp.
+            db.delete(cf, writeOpts, keyBuf.array(), 0, keyBuf.position());
+        } catch (RocksDBException e) {
+            throw new StorageException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public void commitWrite(IgniteRowId rowId, Timestamp timestamp) throws 
StorageException {
+        // Prepare a buffer with partition id and row id.
+        ByteBuffer keyBuf = prepareHeapKeyBuf(rowId);
+
+        try {
+            // Read a value associated with pending write.
+            byte[] valueBytes = db.get(cf, keyBuf.array(), 0, 
keyBuf.position());
+
+            // Delete pending write.
+            db.delete(cf, writeOpts, keyBuf.array(), 0, keyBuf.position());
+
+            // Add timestamp to the key put the value back into the storage.
+            putTimestamp(keyBuf, timestamp);
+
+            db.put(cf, writeOpts, keyBuf.array(), 0, keyBuf.position(), 
valueBytes, 0, valueBytes.length);
+        } catch (RocksDBException e) {
+            throw new StorageException(e);
+        }
+    }
+
+    /** {@inheritDoc} */
+    @Override
+    public @Nullable BinaryRow read(IgniteRowId rowId, @Nullable Timestamp 
timestamp) {
+        // Prepare a buffer with partition id and row id.
+        ByteBuffer keyBuf = prepareHeapKeyBuf(rowId);
+
+        try (
+                // Set next partition as an upper bound.
+                var readOpts = new 
ReadOptions().setIterateUpperBound(upperBound);
+                RocksIterator it = db.newIterator(cf, readOpts)
+        ) {
+            if (timestamp == null) {
+                // Seek to the first appearance of row id if timestamp isn't 
set.
+                // Since timestamps are sorted from newest to oldest, first 
occurance will always be the latest version.
+                // Unfortunately, copy here is unavoidable with current API.
+                it.seek(Arrays.copyOf(keyBuf.array(), keyBuf.position()));
+            } else {
+                // Put timestamp restriction according to N2O timestamps order.
+                putTimestamp(keyBuf, timestamp);
+
+                // This seek will either find a key with timestamp that's less 
or equal than required value, or a different key whatsoever.
+                // It is guaranteed by descending order of timestamps.
+                it.seek(keyBuf.array());
+            }
+
+            // Return null if nothing was found.
+            if (invalid(it)) {
+                return null;
+            }
+
+            // There's no guarantee that required key even exists. If it 
doesn't, then "seek" will point to a different key, obviously.
+            // To avoid returning its value, we have to check that actual key 
matches what we need.
+            // Here we prepare direct buffer to read key without timestamp. 
Shared direct buffer is used to avoid extra memory allocations.
+            ByteBuffer directBuffer = 
MV_KEY_BUFFER.get().position(0).limit(ROW_ID_OFFSET + igniteRowIdSize);
+
+            it.key(directBuffer);
+
+            // Comparison starts from the position of the row id.
+            directBuffer.position(ROW_ID_OFFSET);
+
+            // Return null if seek found a wrong key.
+            if (rowId.compareTo(directBuffer, false) != 0) {

Review Comment:
   I don't know, IDEA shows you names of parameters in such cases, and you 
barely ever read code using different software



-- 
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: [email protected]

For queries about this service, please contact Infrastructure at:
[email protected]

Reply via email to